diff --git a/.gitignore b/.gitignore index 34939e3a97aaa..20095dd97343e 100644 --- a/.gitignore +++ b/.gitignore @@ -5,6 +5,7 @@ *.ipr *.iml *.iws +*.pyc .idea/ .idea_modules/ sbt/*.jar @@ -49,9 +50,12 @@ dependency-reduced-pom.xml checkpoint derby.log dist/ -spark-*-bin.tar.gz +dev/create-release/*txt +dev/create-release/*final +spark-*-bin-*.tgz unit-tests.log /lib/ +ec2/lib/ rat-results.txt scalastyle.txt scalastyle-output.xml diff --git a/.rat-excludes b/.rat-excludes index d8bee1f8e49c9..769defbac11b7 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -64,3 +64,4 @@ dist/* logs .*scalastyle-output.xml .*dependency-reduced-pom.xml +known_translations diff --git a/LICENSE b/LICENSE index 3c667bf45059a..0a42d389e4c3c 100644 --- a/LICENSE +++ b/LICENSE @@ -646,7 +646,8 @@ THE SOFTWARE. ======================================================================== For Scala Interpreter classes (all .scala files in repl/src/main/scala -except for Main.Scala, SparkHelper.scala and ExecutorClassLoader.scala): +except for Main.Scala, SparkHelper.scala and ExecutorClassLoader.scala), +and for SerializableMapWrapper in JavaUtils.scala: ======================================================================== Copyright (c) 2002-2013 EPFL diff --git a/README.md b/README.md index 8d57d50da96c9..16628bd406775 100644 --- a/README.md +++ b/README.md @@ -26,7 +26,7 @@ To build Spark and its example programs, run: (You do not need to do this if you downloaded a pre-built package.) More detailed documentation is available from the project site, at -["Building Spark with Maven"](http://spark.apache.org/docs/latest/building-with-maven.html). +["Building Spark with Maven"](http://spark.apache.org/docs/latest/building-spark.html). ## Interactive Scala Shell diff --git a/assembly/pom.xml b/assembly/pom.xml index 4e2b773e7d2f3..78fb908f9a9ef 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -169,16 +169,6 @@ - - yarn-alpha - - - org.apache.spark - spark-yarn-alpha_${scala.binary.version} - ${project.version} - - - yarn diff --git a/bin/beeline.cmd b/bin/beeline.cmd new file mode 100644 index 0000000000000..8293f311029dd --- /dev/null +++ b/bin/beeline.cmd @@ -0,0 +1,21 @@ +@echo off + +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + +set SPARK_HOME=%~dp0.. +cmd /V /E /C %SPARK_HOME%\bin\spark-class.cmd org.apache.hive.beeline.BeeLine %* diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 298641f2684de..a31ea73d3ce19 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -25,7 +25,11 @@ FWDIR="$(cd "`dirname "$0"`"/..; pwd)" . "$FWDIR"/bin/load-spark-env.sh -CLASSPATH="$SPARK_CLASSPATH:$SPARK_SUBMIT_CLASSPATH" +if [ -n "$SPARK_CLASSPATH" ]; then + CLASSPATH="$SPARK_CLASSPATH:$SPARK_SUBMIT_CLASSPATH" +else + CLASSPATH="$SPARK_SUBMIT_CLASSPATH" +fi # Build up classpath if [ -n "$SPARK_CONF_DIR" ]; then @@ -68,14 +72,14 @@ else assembly_folder="$ASSEMBLY_DIR" fi -num_jars="$(ls "$assembly_folder" | grep "spark-assembly.*hadoop.*\.jar" | wc -l)" +num_jars="$(ls "$assembly_folder" | grep "spark-assembly.*hadoop.*\.jar$" | wc -l)" if [ "$num_jars" -eq "0" ]; then echo "Failed to find Spark assembly in $assembly_folder" echo "You need to build Spark before running this program." exit 1 fi if [ "$num_jars" -gt "1" ]; then - jars_list=$(ls "$assembly_folder" | grep "spark-assembly.*hadoop.*.jar") + jars_list=$(ls "$assembly_folder" | grep "spark-assembly.*hadoop.*.jar$") echo "Found multiple Spark assembly jars in $assembly_folder:" echo "$jars_list" echo "Please remove all but one jar." @@ -108,7 +112,7 @@ else datanucleus_dir="$FWDIR"/lib_managed/jars fi -datanucleus_jars="$(find "$datanucleus_dir" 2>/dev/null | grep "datanucleus-.*\\.jar")" +datanucleus_jars="$(find "$datanucleus_dir" 2>/dev/null | grep "datanucleus-.*\\.jar$")" datanucleus_jars="$(echo "$datanucleus_jars" | tr "\n" : | sed s/:$//g)" if [ -n "$datanucleus_jars" ]; then diff --git a/bin/spark-shell b/bin/spark-shell index 4a0670fc6c8aa..cca5aa0676123 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -45,6 +45,13 @@ source "$FWDIR"/bin/utils.sh SUBMIT_USAGE_FUNCTION=usage gatherSparkSubmitOpts "$@" +# SPARK-4161: scala does not assume use of the java classpath, +# so we need to add the "-Dscala.usejavacp=true" flag mnually. We +# do this specifically for the Spark shell because the scala REPL +# has its own class loader, and any additional classpath specified +# through spark.driver.extraClassPath is not automatically propagated. +SPARK_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Dscala.usejavacp=true" + function main() { if $cygwin; then # Workaround for issue involving JLine and Cygwin diff --git a/bin/spark-shell2.cmd b/bin/spark-shell2.cmd index 2ee60b4e2a2b3..1d1a40da315eb 100644 --- a/bin/spark-shell2.cmd +++ b/bin/spark-shell2.cmd @@ -19,4 +19,23 @@ rem set SPARK_HOME=%~dp0.. -cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd --class org.apache.spark.repl.Main %* spark-shell +echo "%*" | findstr " --help -h" >nul +if %ERRORLEVEL% equ 0 ( + call :usage + exit /b 0 +) + +call %SPARK_HOME%\bin\windows-utils.cmd %* +if %ERRORLEVEL% equ 1 ( + call :usage + exit /b 1 +) + +cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd --class org.apache.spark.repl.Main %SUBMISSION_OPTS% spark-shell %APPLICATION_OPTS% + +exit /b 0 + +:usage +echo "Usage: .\bin\spark-shell.cmd [options]" >&2 +%SPARK_HOME%\bin\spark-submit --help 2>&1 | findstr /V "Usage" 1>&2 +exit /b 0 diff --git a/bin/spark-sql b/bin/spark-sql index 63d00437d508d..3b6cc420fea81 100755 --- a/bin/spark-sql +++ b/bin/spark-sql @@ -23,6 +23,8 @@ # Enter posix mode for bash set -o posix +# NOTE: This exact class name is matched downstream by SparkSubmit. +# Any changes need to be reflected there. CLASS="org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" # Figure out where Spark is installed diff --git a/bin/windows-utils.cmd b/bin/windows-utils.cmd new file mode 100644 index 0000000000000..1082a952dac99 --- /dev/null +++ b/bin/windows-utils.cmd @@ -0,0 +1,59 @@ +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + +rem Gather all spark-submit options into SUBMISSION_OPTS + +set SUBMISSION_OPTS= +set APPLICATION_OPTS= + +rem NOTE: If you add or remove spark-sumbmit options, +rem modify NOT ONLY this script but also SparkSubmitArgument.scala + +:OptsLoop +if "x%1"=="x" ( + goto :OptsLoopEnd +) + +SET opts="\<--master\> \<--deploy-mode\> \<--class\> \<--name\> \<--jars\> \<--py-files\> \<--files\>" +SET opts="%opts:~1,-1% \<--conf\> \<--properties-file\> \<--driver-memory\> \<--driver-java-options\>" +SET opts="%opts:~1,-1% \<--driver-library-path\> \<--driver-class-path\> \<--executor-memory\>" +SET opts="%opts:~1,-1% \<--driver-cores\> \<--total-executor-cores\> \<--executor-cores\> \<--queue\>" +SET opts="%opts:~1,-1% \<--num-executors\> \<--archives\>" + +echo %1 | findstr %opts% >nul +if %ERRORLEVEL% equ 0 ( + if "x%2"=="x" ( + echo "%1" requires an argument. >&2 + exit /b 1 + ) + set SUBMISSION_OPTS=%SUBMISSION_OPTS% %1 %2 + shift + shift + goto :OptsLoop +) +echo %1 | findstr "\<--verbose\> \<-v\> \<--supervise\>" >nul +if %ERRORLEVEL% equ 0 ( + set SUBMISSION_OPTS=%SUBMISSION_OPTS% %1 + shift + goto :OptsLoop +) +set APPLICATION_OPTS=%APPLICATION_OPTS% %1 +shift +goto :OptsLoop + +:OptsLoopEnd +exit /b 0 diff --git a/conf/metrics.properties.template b/conf/metrics.properties.template index 30bcab0c93302..96b6844f0aabb 100644 --- a/conf/metrics.properties.template +++ b/conf/metrics.properties.template @@ -77,8 +77,8 @@ # sample false Whether to show entire set of samples for histograms ('false' or 'true') # # * Default path is /metrics/json for all instances except the master. The master has two paths: -# /metrics/aplications/json # App information -# /metrics/master/json # Master information +# /metrics/applications/json # App information +# /metrics/master/json # Master information # org.apache.spark.metrics.sink.GraphiteSink # Name: Default: Description: diff --git a/core/pom.xml b/core/pom.xml index 1feb00b3a7fb8..c5c41b2b5de42 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -352,9 +352,9 @@ - + - + diff --git a/core/src/main/java/org/apache/spark/SparkJobInfo.java b/core/src/main/java/org/apache/spark/SparkJobInfo.java index 4e3c983b1170a..e31c4401632a6 100644 --- a/core/src/main/java/org/apache/spark/SparkJobInfo.java +++ b/core/src/main/java/org/apache/spark/SparkJobInfo.java @@ -17,13 +17,15 @@ package org.apache.spark; +import java.io.Serializable; + /** * Exposes information about Spark Jobs. * * This interface is not designed to be implemented outside of Spark. We may add additional methods * which may break binary compatibility with outside implementations. */ -public interface SparkJobInfo { +public interface SparkJobInfo extends Serializable { int jobId(); int[] stageIds(); JobExecutionStatus status(); diff --git a/core/src/main/java/org/apache/spark/SparkStageInfo.java b/core/src/main/java/org/apache/spark/SparkStageInfo.java index fd74321093658..b7d462abd72d6 100644 --- a/core/src/main/java/org/apache/spark/SparkStageInfo.java +++ b/core/src/main/java/org/apache/spark/SparkStageInfo.java @@ -17,13 +17,15 @@ package org.apache.spark; +import java.io.Serializable; + /** * Exposes information about Spark Stages. * * This interface is not designed to be implemented outside of Spark. We may add additional methods * which may break binary compatibility with outside implementations. */ -public interface SparkStageInfo { +public interface SparkStageInfo extends Serializable { int stageId(); int currentAttemptId(); long submissionTime(); diff --git a/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js b/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js index d33c5c769d683..14ba37d7c9bd9 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js +++ b/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js @@ -28,17 +28,12 @@ $(function() { $(this).find('.expand-additional-metrics-arrow').toggleClass('arrow-closed'); }); - $("input:checkbox:not(:checked)").each(function() { - var column = "table ." + $(this).attr("name"); - $(column).hide(); - }); - // Stripe table rows after rows have been hidden to ensure correct striping. - stripeTables(); + stripeSummaryTable(); $("input:checkbox").click(function() { var column = "table ." + $(this).attr("name"); $(column).toggle(); - stripeTables(); + stripeSummaryTable(); }); $("#select-all-metrics").click(function() { diff --git a/core/src/main/resources/org/apache/spark/ui/static/table.js b/core/src/main/resources/org/apache/spark/ui/static/table.js index 6bb03015abb51..656147e40d13e 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/table.js +++ b/core/src/main/resources/org/apache/spark/ui/static/table.js @@ -15,16 +15,18 @@ * limitations under the License. */ -/* Adds background colors to stripe table rows. This is necessary (instead of using css or the - * table striping provided by bootstrap) to appropriately stripe tables with hidden rows. */ -function stripeTables() { - $("table.table-striped-custom").each(function() { - $(this).find("tr:not(:hidden)").each(function (index) { - if (index % 2 == 1) { - $(this).css("background-color", "#f9f9f9"); - } else { - $(this).css("background-color", "#ffffff"); - } - }); +/* Adds background colors to stripe table rows in the summary table (on the stage page). This is + * necessary (instead of using css or the table striping provided by bootstrap) because the summary + * table has hidden rows. + * + * An ID selector (rather than a class selector) is used to ensure this runs quickly even on pages + * with thousands of task rows (ID selectors are much faster than class selectors). */ +function stripeSummaryTable() { + $("#task-summary-table").find("tr:not(:hidden)").each(function (index) { + if (index % 2 == 1) { + $(this).css("background-color", "#f9f9f9"); + } else { + $(this).css("background-color", "#ffffff"); + } }); } diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index db57712c83503..5751964b792ce 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -168,3 +168,19 @@ span.additional-metric-title { border-left: 5px solid black; display: inline-block; } + +.version { + line-height: 30px; + vertical-align: bottom; + font-size: 12px; + padding: 0; + margin: 0; + font-weight: bold; + color: #777; +} + +/* Hide all additional metrics by default. This is done here rather than using JavaScript to + * avoid slow page loads for stage pages with large numbers (e.g., thousands) of tasks. */ +.scheduler_delay, .deserialization_time, .serialization_time, .getting_result_time { + display: none; +} diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 2301caafb07ff..5f31bfba3f8d6 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -18,6 +18,8 @@ package org.apache.spark import java.io.{ObjectInputStream, Serializable} +import java.util.concurrent.atomic.AtomicLong +import java.lang.ThreadLocal import scala.collection.generic.Growable import scala.collection.mutable.Map @@ -228,6 +230,7 @@ GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Serializa */ class Accumulator[T](@transient initialValue: T, param: AccumulatorParam[T], name: Option[String]) extends Accumulable[T,T](initialValue, param, name) { + def this(initialValue: T, param: AccumulatorParam[T]) = this(initialValue, param, None) } @@ -244,15 +247,47 @@ trait AccumulatorParam[T] extends AccumulableParam[T, T] { } } +object AccumulatorParam { + + // The following implicit objects were in SparkContext before 1.2 and users had to + // `import SparkContext._` to enable them. Now we move them here to make the compiler find + // them automatically. However, as there are duplicate codes in SparkContext for backward + // compatibility, please update them accordingly if you modify the following implicit objects. + + implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] { + def addInPlace(t1: Double, t2: Double): Double = t1 + t2 + def zero(initialValue: Double) = 0.0 + } + + implicit object IntAccumulatorParam extends AccumulatorParam[Int] { + def addInPlace(t1: Int, t2: Int): Int = t1 + t2 + def zero(initialValue: Int) = 0 + } + + implicit object LongAccumulatorParam extends AccumulatorParam[Long] { + def addInPlace(t1: Long, t2: Long) = t1 + t2 + def zero(initialValue: Long) = 0L + } + + implicit object FloatAccumulatorParam extends AccumulatorParam[Float] { + def addInPlace(t1: Float, t2: Float) = t1 + t2 + def zero(initialValue: Float) = 0f + } + + // TODO: Add AccumulatorParams for other types, e.g. lists and strings +} + // TODO: The multi-thread support in accumulators is kind of lame; check // if there's a more intuitive way of doing it right -private object Accumulators { +private[spark] object Accumulators { // TODO: Use soft references? => need to make readObject work properly then val originals = Map[Long, Accumulable[_, _]]() - val localAccums = Map[Thread, Map[Long, Accumulable[_, _]]]() + val localAccums = new ThreadLocal[Map[Long, Accumulable[_, _]]]() { + override protected def initialValue() = Map[Long, Accumulable[_, _]]() + } var lastId: Long = 0 - def newId: Long = synchronized { + def newId(): Long = synchronized { lastId += 1 lastId } @@ -261,22 +296,21 @@ private object Accumulators { if (original) { originals(a.id) = a } else { - val accums = localAccums.getOrElseUpdate(Thread.currentThread, Map()) - accums(a.id) = a + localAccums.get()(a.id) = a } } // Clear the local (non-original) accumulators for the current thread def clear() { synchronized { - localAccums.remove(Thread.currentThread) + localAccums.get.clear } } // Get the values of the local accumulators for the current thread (by ID) def values: Map[Long, Any] = synchronized { val ret = Map[Long, Any]() - for ((id, accum) <- localAccums.getOrElse(Thread.currentThread, Map())) { + for ((id, accum) <- localAccums.get) { ret(id) = accum.localValue } return ret diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index 79c9c451d273d..09eb9605fb799 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -34,7 +34,9 @@ case class Aggregator[K, V, C] ( mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) { - private val externalSorting = SparkEnv.get.conf.getBoolean("spark.shuffle.spill", true) + // When spilling is enabled sorting will happen externally, but not necessarily with an + // ExternalSorter. + private val isSpillEnabled = SparkEnv.get.conf.getBoolean("spark.shuffle.spill", true) @deprecated("use combineValuesByKey with TaskContext argument", "0.9.0") def combineValuesByKey(iter: Iterator[_ <: Product2[K, V]]): Iterator[(K, C)] = @@ -42,7 +44,7 @@ case class Aggregator[K, V, C] ( def combineValuesByKey(iter: Iterator[_ <: Product2[K, V]], context: TaskContext): Iterator[(K, C)] = { - if (!externalSorting) { + if (!isSpillEnabled) { val combiners = new AppendOnlyMap[K,C] var kv: Product2[K, V] = null val update = (hadValue: Boolean, oldValue: C) => { @@ -71,9 +73,9 @@ case class Aggregator[K, V, C] ( combineCombinersByKey(iter, null) def combineCombinersByKey(iter: Iterator[_ <: Product2[K, C]], context: TaskContext) - : Iterator[(K, C)] = + : Iterator[(K, C)] = { - if (!externalSorting) { + if (!isSpillEnabled) { val combiners = new AppendOnlyMap[K,C] var kc: Product2[K, C] = null val update = (hadValue: Boolean, oldValue: C) => { diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index ab2594cfc02eb..9a7cd4523e5ab 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -60,6 +60,9 @@ abstract class NarrowDependency[T](_rdd: RDD[T]) extends Dependency[T] { * @param serializer [[org.apache.spark.serializer.Serializer Serializer]] to use. If set to None, * the default serializer, as specified by `spark.serializer` config option, will * be used. + * @param keyOrdering key ordering for RDD's shuffles + * @param aggregator map/reduce-side aggregator for RDD's shuffle + * @param mapSideCombine whether to perform partial aggregation (also known as map-side combine) */ @DeveloperApi class ShuffleDependency[K, V, C]( diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala new file mode 100644 index 0000000000000..a46a81eabd965 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * A client that communicates with the cluster manager to request or kill executors. + */ +private[spark] trait ExecutorAllocationClient { + + /** + * Request an additional number of executors from the cluster manager. + * Return whether the request is acknowledged by the cluster manager. + */ + def requestExecutors(numAdditionalExecutors: Int): Boolean + + /** + * Request that the cluster manager kill the specified executors. + * Return whether the request is acknowledged by the cluster manager. + */ + def killExecutors(executorIds: Seq[String]): Boolean + + /** + * Request that the cluster manager kill the specified executor. + * Return whether the request is acknowledged by the cluster manager. + */ + def killExecutor(executorId: String): Boolean = killExecutors(Seq(executorId)) +} diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 88adb892998af..e9e90e3f2f65a 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -60,11 +60,13 @@ import org.apache.spark.scheduler._ * spark.dynamicAllocation.executorIdleTimeout (K) - * If an executor has been idle for this duration, remove it */ -private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging { +private[spark] class ExecutorAllocationManager( + client: ExecutorAllocationClient, + listenerBus: LiveListenerBus, + conf: SparkConf) + extends Logging { import ExecutorAllocationManager._ - private val conf = sc.conf - // Lower and upper bounds on the number of executors. These are required. private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1) private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1) @@ -168,7 +170,7 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging * Register for scheduler callbacks to decide when to add and remove executors. */ def start(): Unit = { - sc.addSparkListener(listener) + listenerBus.addListener(listener) startPolling() } @@ -253,7 +255,7 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging val actualNumExecutorsToAdd = math.min(numExecutorsToAdd, maxNumExecutorsToAdd) val newTotalExecutors = numExistingExecutors + actualNumExecutorsToAdd - val addRequestAcknowledged = testing || sc.requestExecutors(actualNumExecutorsToAdd) + val addRequestAcknowledged = testing || client.requestExecutors(actualNumExecutorsToAdd) if (addRequestAcknowledged) { logInfo(s"Requesting $actualNumExecutorsToAdd new executor(s) because " + s"tasks are backlogged (new desired total will be $newTotalExecutors)") @@ -295,7 +297,7 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging } // Send a request to the backend to kill this executor - val removeRequestAcknowledged = testing || sc.killExecutor(executorId) + val removeRequestAcknowledged = testing || client.killExecutor(executorId) if (removeRequestAcknowledged) { logInfo(s"Removing executor $executorId because it has been idle for " + s"$executorIdleTimeout seconds (new desired total will be ${numExistingExecutors - 1})") diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 7d96962c4acd7..a074ab8ece1b7 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -72,7 +72,7 @@ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster /** * Class that keeps track of the location of the map output of * a stage. This is abstract because different versions of MapOutputTracker - * (driver and worker) use different HashMap to store its metadata. + * (driver and executor) use different HashMap to store its metadata. */ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging { private val timeout = AkkaUtils.askTimeout(conf) @@ -81,11 +81,11 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging var trackerActor: ActorRef = _ /** - * This HashMap has different behavior for the master and the workers. + * This HashMap has different behavior for the driver and the executors. * - * On the master, it serves as the source of map outputs recorded from ShuffleMapTasks. - * On the workers, it simply serves as a cache, in which a miss triggers a fetch from the - * master's corresponding HashMap. + * On the driver, it serves as the source of map outputs recorded from ShuffleMapTasks. + * On the executors, it simply serves as a cache, in which a miss triggers a fetch from the + * driver's corresponding HashMap. * * Note: because mapStatuses is accessed concurrently, subclasses should make sure it's a * thread-safe map. @@ -99,7 +99,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging protected var epoch: Long = 0 protected val epochLock = new AnyRef - /** Remembers which map output locations are currently being fetched on a worker. */ + /** Remembers which map output locations are currently being fetched on an executor. */ private val fetching = new HashSet[Int] /** @@ -136,14 +136,12 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging logInfo("Don't have map outputs for shuffle " + shuffleId + ", fetching them") var fetchedStatuses: Array[MapStatus] = null fetching.synchronized { - if (fetching.contains(shuffleId)) { - // Someone else is fetching it; wait for them to be done - while (fetching.contains(shuffleId)) { - try { - fetching.wait() - } catch { - case e: InterruptedException => - } + // Someone else is fetching it; wait for them to be done + while (fetching.contains(shuffleId)) { + try { + fetching.wait() + } catch { + case e: InterruptedException => } } @@ -198,8 +196,8 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging /** * Called from executors to update the epoch number, potentially clearing old outputs - * because of a fetch failure. Each worker task calls this with the latest epoch - * number on the master at the time it was created. + * because of a fetch failure. Each executor task calls this with the latest epoch + * number on the driver at the time it was created. */ def updateEpoch(newEpoch: Long) { epochLock.synchronized { @@ -231,7 +229,7 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) private var cacheEpoch = epoch /** - * Timestamp based HashMap for storing mapStatuses and cached serialized statuses in the master, + * Timestamp based HashMap for storing mapStatuses and cached serialized statuses in the driver, * so that statuses are dropped only by explicit de-registering or by TTL-based cleaning (if set). * Other than these two scenarios, nothing should be dropped from this HashMap. */ @@ -341,7 +339,7 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) } /** - * MapOutputTracker for the workers, which fetches map output information from the driver's + * MapOutputTracker for the executors, which fetches map output information from the driver's * MapOutputTrackerMaster. */ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTracker(conf) { diff --git a/core/src/main/scala/org/apache/spark/Partition.scala b/core/src/main/scala/org/apache/spark/Partition.scala index 27892dbd2a0bc..dd3f28e4197e3 100644 --- a/core/src/main/scala/org/apache/spark/Partition.scala +++ b/core/src/main/scala/org/apache/spark/Partition.scala @@ -18,11 +18,11 @@ package org.apache.spark /** - * A partition of an RDD. + * An identifier for a partition in an RDD. */ trait Partition extends Serializable { /** - * Get the split's index within its parent RDD + * Get the partition's index within its parent RDD */ def index: Int diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index dbff9d12b5ad7..ec82d09cd079b 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -93,19 +93,19 @@ import org.apache.spark.network.sasl.SecretKeyHolder * Note that SASL is pluggable as to what mechanism it uses. We currently use * DIGEST-MD5 but this could be changed to use Kerberos or other in the future. * Spark currently supports "auth" for the quality of protection, which means - * the connection is not supporting integrity or privacy protection (encryption) + * the connection does not support integrity or privacy protection (encryption) * after authentication. SASL also supports "auth-int" and "auth-conf" which - * SPARK could be support in the future to allow the user to specify the quality + * SPARK could support in the future to allow the user to specify the quality * of protection they want. If we support those, the messages will also have to * be wrapped and unwrapped via the SaslServer/SaslClient.wrap/unwrap API's. * * Since the NioBlockTransferService does asynchronous messages passing, the SASL * authentication is a bit more complex. A ConnectionManager can be both a client - * and a Server, so for a particular connection is has to determine what to do. + * and a Server, so for a particular connection it has to determine what to do. * A ConnectionId was added to be able to track connections and is used to * match up incoming messages with connections waiting for authentication. - * The ConnectionManager tracks all the sendingConnections using the ConnectionId - * and waits for the response from the server and does the handshake before sending + * The ConnectionManager tracks all the sendingConnections using the ConnectionId, + * waits for the response from the server, and does the handshake before sending * the real message. * * The NettyBlockTransferService ensures that SASL authentication is performed @@ -114,14 +114,14 @@ import org.apache.spark.network.sasl.SecretKeyHolder * * - HTTP for the Spark UI -> the UI was changed to use servlets so that javax servlet filters * can be used. Yarn requires a specific AmIpFilter be installed for security to work - * properly. For non-Yarn deployments, users can write a filter to go through a - * companies normal login service. If an authentication filter is in place then the + * properly. For non-Yarn deployments, users can write a filter to go through their + * organization's normal login service. If an authentication filter is in place then the * SparkUI can be configured to check the logged in user against the list of users who * have view acls to see if that user is authorized. * The filters can also be used for many different purposes. For instance filters * could be used for logging, encryption, or compression. * - * The exact mechanisms used to generate/distributed the shared secret is deployment specific. + * The exact mechanisms used to generate/distribute the shared secret are deployment-specific. * * For Yarn deployments, the secret is automatically generated using the Akka remote * Crypt.generateSecureCookie() API. The secret is placed in the Hadoop UGI which gets passed @@ -138,7 +138,7 @@ import org.apache.spark.network.sasl.SecretKeyHolder * All the nodes (Master and Workers) and the applications need to have the same shared secret. * This again is not ideal as one user could potentially affect another users application. * This should be enhanced in the future to provide better protection. - * If the UI needs to be secured the user needs to install a javax servlet filter to do the + * If the UI needs to be secure, the user needs to install a javax servlet filter to do the * authentication. Spark will then use that user to compare against the view acls to do * authorization. If not filter is in place the user is generally null and no authorization * can take place. @@ -151,8 +151,8 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging with private val authOn = sparkConf.getBoolean("spark.authenticate", false) // keep spark.ui.acls.enable for backwards compatibility with 1.0 - private var aclsOn = sparkConf.getOption("spark.acls.enable").getOrElse( - sparkConf.get("spark.ui.acls.enable", "false")).toBoolean + private var aclsOn = + sparkConf.getBoolean("spark.acls.enable", sparkConf.getBoolean("spark.ui.acls.enable", false)) // admin acls should be set before view or modify acls private var adminAcls: Set[String] = diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 4c6c86c7bad78..c14764f773982 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -61,7 +61,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { throw new NullPointerException("null key") } if (value == null) { - throw new NullPointerException("null value") + throw new NullPointerException("null value for " + key) } settings(key) = value this diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index ae8bbfb56f493..57bc3d4e4ae36 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -64,7 +64,7 @@ import org.apache.spark.util._ * @param config a Spark Config object describing the application configuration. Any settings in * this config overrides the default configs as well as system properties. */ -class SparkContext(config: SparkConf) extends Logging { +class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationClient { // The call site where this SparkContext was constructed. private val creationSite: CallSite = Utils.getCallSite() @@ -83,6 +83,8 @@ class SparkContext(config: SparkConf) extends Logging { // contains a map from hostname to a list of input format splits on the host. private[spark] var preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map() + val startTime = System.currentTimeMillis() + /** * Create a SparkContext that loads settings from system properties (for instance, when * launching with ./bin/spark-submit). @@ -170,6 +172,9 @@ class SparkContext(config: SparkConf) extends Logging { private[spark] def this(master: String, appName: String, sparkHome: String, jars: Seq[String]) = this(master, appName, sparkHome, jars, Map(), Map()) + // log out Spark Version in Spark driver log + logInfo(s"Running Spark version $SPARK_VERSION") + private[spark] val conf = config.clone() conf.validateSettings() @@ -269,8 +274,6 @@ class SparkContext(config: SparkConf) extends Logging { /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(conf) - val startTime = System.currentTimeMillis() - // Add each JAR given through the constructor if (jars != null) { jars.foreach(addJar) @@ -344,6 +347,8 @@ class SparkContext(config: SparkConf) extends Logging { // The metrics system for Driver need to be set spark.app.id to app ID. // So it should start after we get app ID from the task scheduler and set spark.app.id. metricsSystem.start() + // Attach the driver metrics servlet handler to the web ui after the metrics system is started. + metricsSystem.getServletHandlers.foreach(handler => ui.foreach(_.attachHandler(handler))) // Optionally log Spark events private[spark] val eventLogger: Option[EventLoggingListener] = { @@ -357,9 +362,13 @@ class SparkContext(config: SparkConf) extends Logging { } // Optionally scale number of executors dynamically based on workload. Exposed for testing. + private val dynamicAllocationEnabled = conf.getBoolean("spark.dynamicAllocation.enabled", false) + private val dynamicAllocationTesting = conf.getBoolean("spark.dynamicAllocation.testing", false) private[spark] val executorAllocationManager: Option[ExecutorAllocationManager] = - if (conf.getBoolean("spark.dynamicAllocation.enabled", false)) { - Some(new ExecutorAllocationManager(this)) + if (dynamicAllocationEnabled) { + assert(master.contains("yarn") || dynamicAllocationTesting, + "Dynamic allocation of executors is currently only supported in YARN mode") + Some(new ExecutorAllocationManager(this, listenerBus, conf)) } else { None } @@ -988,7 +997,9 @@ class SparkContext(config: SparkConf) extends Logging { * This is currently only supported in Yarn mode. Return whether the request is received. */ @DeveloperApi - def requestExecutors(numAdditionalExecutors: Int): Boolean = { + override def requestExecutors(numAdditionalExecutors: Int): Boolean = { + assert(master.contains("yarn") || dynamicAllocationTesting, + "Requesting executors is currently only supported in YARN mode") schedulerBackend match { case b: CoarseGrainedSchedulerBackend => b.requestExecutors(numAdditionalExecutors) @@ -1004,7 +1015,9 @@ class SparkContext(config: SparkConf) extends Logging { * This is currently only supported in Yarn mode. Return whether the request is received. */ @DeveloperApi - def killExecutors(executorIds: Seq[String]): Boolean = { + override def killExecutors(executorIds: Seq[String]): Boolean = { + assert(master.contains("yarn") || dynamicAllocationTesting, + "Killing executors is currently only supported in YARN mode") schedulerBackend match { case b: CoarseGrainedSchedulerBackend => b.killExecutors(executorIds) @@ -1020,7 +1033,7 @@ class SparkContext(config: SparkConf) extends Logging { * This is currently only supported in Yarn mode. Return whether the request is received. */ @DeveloperApi - def killExecutor(executorId: String): Boolean = killExecutors(Seq(executorId)) + override def killExecutor(executorId: String): Boolean = super.killExecutor(executorId) /** The version of Spark on which this application is running. */ def version = SPARK_VERSION @@ -1624,47 +1637,74 @@ object SparkContext extends Logging { private[spark] val DRIVER_IDENTIFIER = "" - implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] { + // The following deprecated objects have already been copied to `object AccumulatorParam` to + // make the compiler find them automatically. They are duplicate codes only for backward + // compatibility, please update `object AccumulatorParam` accordingly if you plan to modify the + // following ones. + + @deprecated("Replaced by implicit objects in AccumulatorParam. This is kept here only for " + + "backward compatibility.", "1.3.0") + object DoubleAccumulatorParam extends AccumulatorParam[Double] { def addInPlace(t1: Double, t2: Double): Double = t1 + t2 def zero(initialValue: Double) = 0.0 } - implicit object IntAccumulatorParam extends AccumulatorParam[Int] { + @deprecated("Replaced by implicit objects in AccumulatorParam. This is kept here only for " + + "backward compatibility.", "1.3.0") + object IntAccumulatorParam extends AccumulatorParam[Int] { def addInPlace(t1: Int, t2: Int): Int = t1 + t2 def zero(initialValue: Int) = 0 } - implicit object LongAccumulatorParam extends AccumulatorParam[Long] { + @deprecated("Replaced by implicit objects in AccumulatorParam. This is kept here only for " + + "backward compatibility.", "1.3.0") + object LongAccumulatorParam extends AccumulatorParam[Long] { def addInPlace(t1: Long, t2: Long) = t1 + t2 def zero(initialValue: Long) = 0L } - implicit object FloatAccumulatorParam extends AccumulatorParam[Float] { + @deprecated("Replaced by implicit objects in AccumulatorParam. This is kept here only for " + + "backward compatibility.", "1.3.0") + object FloatAccumulatorParam extends AccumulatorParam[Float] { def addInPlace(t1: Float, t2: Float) = t1 + t2 def zero(initialValue: Float) = 0f } - // TODO: Add AccumulatorParams for other types, e.g. lists and strings + // The following deprecated functions have already been moved to `object RDD` to + // make the compiler find them automatically. They are still kept here for backward compatibility + // and just call the corresponding functions in `object RDD`. - implicit def rddToPairRDDFunctions[K, V](rdd: RDD[(K, V)]) + @deprecated("Replaced by implicit functions in the RDD companion object. This is " + + "kept here only for backward compatibility.", "1.3.0") + def rddToPairRDDFunctions[K, V](rdd: RDD[(K, V)]) (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) = { - new PairRDDFunctions(rdd) + RDD.rddToPairRDDFunctions(rdd) } - implicit def rddToAsyncRDDActions[T: ClassTag](rdd: RDD[T]) = new AsyncRDDActions(rdd) + @deprecated("Replaced by implicit functions in the RDD companion object. This is " + + "kept here only for backward compatibility.", "1.3.0") + def rddToAsyncRDDActions[T: ClassTag](rdd: RDD[T]) = RDD.rddToAsyncRDDActions(rdd) - implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable: ClassTag]( + @deprecated("Replaced by implicit functions in the RDD companion object. This is " + + "kept here only for backward compatibility.", "1.3.0") + def rddToSequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable: ClassTag]( rdd: RDD[(K, V)]) = - new SequenceFileRDDFunctions(rdd) + RDD.rddToSequenceFileRDDFunctions(rdd) - implicit def rddToOrderedRDDFunctions[K : Ordering : ClassTag, V: ClassTag]( + @deprecated("Replaced by implicit functions in the RDD companion object. This is " + + "kept here only for backward compatibility.", "1.3.0") + def rddToOrderedRDDFunctions[K : Ordering : ClassTag, V: ClassTag]( rdd: RDD[(K, V)]) = - new OrderedRDDFunctions[K, V, (K, V)](rdd) + RDD.rddToOrderedRDDFunctions(rdd) - implicit def doubleRDDToDoubleRDDFunctions(rdd: RDD[Double]) = new DoubleRDDFunctions(rdd) + @deprecated("Replaced by implicit functions in the RDD companion object. This is " + + "kept here only for backward compatibility.", "1.3.0") + def doubleRDDToDoubleRDDFunctions(rdd: RDD[Double]) = RDD.doubleRDDToDoubleRDDFunctions(rdd) - implicit def numericRDDToDoubleRDDFunctions[T](rdd: RDD[T])(implicit num: Numeric[T]) = - new DoubleRDDFunctions(rdd.map(x => num.toDouble(x))) + @deprecated("Replaced by implicit functions in the RDD companion object. This is " + + "kept here only for backward compatibility.", "1.3.0") + def numericRDDToDoubleRDDFunctions[T](rdd: RDD[T])(implicit num: Numeric[T]) = + RDD.numericRDDToDoubleRDDFunctions(rdd) // Implicit conversions to common Writable types, for saveAsSequenceFile @@ -1690,40 +1730,49 @@ object SparkContext extends Logging { arr.map(x => anyToWritable(x)).toArray) } - // Helper objects for converting common types to Writable - private def simpleWritableConverter[T, W <: Writable: ClassTag](convert: W => T) - : WritableConverter[T] = { - val wClass = classTag[W].runtimeClass.asInstanceOf[Class[W]] - new WritableConverter[T](_ => wClass, x => convert(x.asInstanceOf[W])) - } + // The following deprecated functions have already been moved to `object WritableConverter` to + // make the compiler find them automatically. They are still kept here for backward compatibility + // and just call the corresponding functions in `object WritableConverter`. - implicit def intWritableConverter(): WritableConverter[Int] = - simpleWritableConverter[Int, IntWritable](_.get) + @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + + "backward compatibility.", "1.3.0") + def intWritableConverter(): WritableConverter[Int] = + WritableConverter.intWritableConverter() - implicit def longWritableConverter(): WritableConverter[Long] = - simpleWritableConverter[Long, LongWritable](_.get) + @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + + "backward compatibility.", "1.3.0") + def longWritableConverter(): WritableConverter[Long] = + WritableConverter.longWritableConverter() - implicit def doubleWritableConverter(): WritableConverter[Double] = - simpleWritableConverter[Double, DoubleWritable](_.get) + @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + + "backward compatibility.", "1.3.0") + def doubleWritableConverter(): WritableConverter[Double] = + WritableConverter.doubleWritableConverter() - implicit def floatWritableConverter(): WritableConverter[Float] = - simpleWritableConverter[Float, FloatWritable](_.get) + @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + + "backward compatibility.", "1.3.0") + def floatWritableConverter(): WritableConverter[Float] = + WritableConverter.floatWritableConverter() - implicit def booleanWritableConverter(): WritableConverter[Boolean] = - simpleWritableConverter[Boolean, BooleanWritable](_.get) + @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + + "backward compatibility.", "1.3.0") + def booleanWritableConverter(): WritableConverter[Boolean] = + WritableConverter.booleanWritableConverter() - implicit def bytesWritableConverter(): WritableConverter[Array[Byte]] = { - simpleWritableConverter[Array[Byte], BytesWritable](bw => - // getBytes method returns array which is longer then data to be returned - Arrays.copyOfRange(bw.getBytes, 0, bw.getLength) - ) - } + @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + + "backward compatibility.", "1.3.0") + def bytesWritableConverter(): WritableConverter[Array[Byte]] = + WritableConverter.bytesWritableConverter() - implicit def stringWritableConverter(): WritableConverter[String] = - simpleWritableConverter[String, Text](_.toString) + @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + + "backward compatibility.", "1.3.0") + def stringWritableConverter(): WritableConverter[String] = + WritableConverter.stringWritableConverter() - implicit def writableWritableConverter[T <: Writable]() = - new WritableConverter[T](_.runtimeClass.asInstanceOf[Class[T]], _.asInstanceOf[T]) + @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + + "backward compatibility.", "1.3.0") + def writableWritableConverter[T <: Writable](): WritableConverter[T] = + WritableConverter.writableWritableConverter() /** * Find the JAR from which a given class was loaded, to make it easy for users to pass @@ -1950,3 +1999,46 @@ private[spark] class WritableConverter[T]( val writableClass: ClassTag[T] => Class[_ <: Writable], val convert: Writable => T) extends Serializable + +object WritableConverter { + + // Helper objects for converting common types to Writable + private[spark] def simpleWritableConverter[T, W <: Writable: ClassTag](convert: W => T) + : WritableConverter[T] = { + val wClass = classTag[W].runtimeClass.asInstanceOf[Class[W]] + new WritableConverter[T](_ => wClass, x => convert(x.asInstanceOf[W])) + } + + // The following implicit functions were in SparkContext before 1.2 and users had to + // `import SparkContext._` to enable them. Now we move them here to make the compiler find + // them automatically. However, we still keep the old functions in SparkContext for backward + // compatibility and forward to the following functions directly. + + implicit def intWritableConverter(): WritableConverter[Int] = + simpleWritableConverter[Int, IntWritable](_.get) + + implicit def longWritableConverter(): WritableConverter[Long] = + simpleWritableConverter[Long, LongWritable](_.get) + + implicit def doubleWritableConverter(): WritableConverter[Double] = + simpleWritableConverter[Double, DoubleWritable](_.get) + + implicit def floatWritableConverter(): WritableConverter[Float] = + simpleWritableConverter[Float, FloatWritable](_.get) + + implicit def booleanWritableConverter(): WritableConverter[Boolean] = + simpleWritableConverter[Boolean, BooleanWritable](_.get) + + implicit def bytesWritableConverter(): WritableConverter[Array[Byte]] = { + simpleWritableConverter[Array[Byte], BytesWritable] { bw => + // getBytes method returns array which is longer then data to be returned + Arrays.copyOfRange(bw.getBytes, 0, bw.getLength) + } + } + + implicit def stringWritableConverter(): WritableConverter[String] = + simpleWritableConverter[String, Text](_.toString) + + implicit def writableWritableConverter[T <: Writable](): WritableConverter[T] = + new WritableConverter[T](_.runtimeClass.asInstanceOf[Class[T]], _.asInstanceOf[T]) +} diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index e464b32e61dd6..43436a1697000 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -156,7 +156,15 @@ object SparkEnv extends Logging { assert(conf.contains("spark.driver.port"), "spark.driver.port is not set on the driver!") val hostname = conf.get("spark.driver.host") val port = conf.get("spark.driver.port").toInt - create(conf, SparkContext.DRIVER_IDENTIFIER, hostname, port, true, isLocal, listenerBus) + create( + conf, + SparkContext.DRIVER_IDENTIFIER, + hostname, + port, + isDriver = true, + isLocal = isLocal, + listenerBus = listenerBus + ) } /** @@ -169,10 +177,18 @@ object SparkEnv extends Logging { hostname: String, port: Int, numCores: Int, - isLocal: Boolean, - actorSystem: ActorSystem = null): SparkEnv = { - create(conf, executorId, hostname, port, false, isLocal, defaultActorSystem = actorSystem, - numUsableCores = numCores) + isLocal: Boolean): SparkEnv = { + val env = create( + conf, + executorId, + hostname, + port, + isDriver = false, + isLocal = isLocal, + numUsableCores = numCores + ) + SparkEnv.set(env) + env } /** @@ -186,7 +202,6 @@ object SparkEnv extends Logging { isDriver: Boolean, isLocal: Boolean, listenerBus: LiveListenerBus = null, - defaultActorSystem: ActorSystem = null, numUsableCores: Int = 0): SparkEnv = { // Listener bus is only used on the driver @@ -196,20 +211,17 @@ object SparkEnv extends Logging { val securityManager = new SecurityManager(conf) - // If an existing actor system is already provided, use it. - // This is the case when an executor is launched in coarse-grained mode. - val (actorSystem, boundPort) = - Option(defaultActorSystem) match { - case Some(as) => (as, port) - case None => - val actorSystemName = if (isDriver) driverActorSystemName else executorActorSystemName - AkkaUtils.createActorSystem(actorSystemName, hostname, port, conf, securityManager) - } + // Create the ActorSystem for Akka and get the port it binds to. + val (actorSystem, boundPort) = { + val actorSystemName = if (isDriver) driverActorSystemName else executorActorSystemName + AkkaUtils.createActorSystem(actorSystemName, hostname, port, conf, securityManager) + } // Figure out which port Akka actually bound to in case the original port is 0 or occupied. - // This is so that we tell the executors the correct port to connect to. if (isDriver) { conf.set("spark.driver.port", boundPort.toString) + } else { + conf.set("spark.executor.port", boundPort.toString) } // Create an instance of the class with the given name, possibly initializing it with our conf @@ -383,7 +395,7 @@ object SparkEnv extends Logging { val sparkProperties = (conf.getAll ++ schedulerMode).sorted // System properties that are not java classpaths - val systemProperties = System.getProperties.iterator.toSeq + val systemProperties = Utils.getSystemProperties.toSeq val otherProperties = systemProperties.filter { case (k, _) => k != "java.class.path" && !k.startsWith("spark.") }.sorted diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index e37f3acaf6e30..7af3538262fd6 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -32,13 +32,13 @@ import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} import org.apache.spark.{HashPartitioner, Partitioner} import org.apache.spark.Partitioner._ -import org.apache.spark.SparkContext.rddToPairRDDFunctions import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.api.java.JavaUtils.mapAsSerializableJavaMap import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, PairFunction} import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.{OrderedRDDFunctions, RDD} +import org.apache.spark.rdd.RDD.rddToPairRDDFunctions import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 5a8e5bb1f721a..bd451634e53d2 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -28,7 +28,6 @@ import com.google.common.base.Optional import org.apache.hadoop.io.compress.CompressionCodec import org.apache.spark._ -import org.apache.spark.SparkContext._ import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaPairRDD._ import org.apache.spark.api.java.JavaSparkContext.fakeClassTag @@ -212,8 +211,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ - def groupBy[K](f: JFunction[T, K]): JavaPairRDD[K, JIterable[T]] = { - implicit val ctagK: ClassTag[K] = fakeClassTag + def groupBy[U](f: JFunction[T, U]): JavaPairRDD[U, JIterable[T]] = { + // The type parameter is U instead of K in order to work around a compiler bug; see SPARK-4459 + implicit val ctagK: ClassTag[U] = fakeClassTag implicit val ctagV: ClassTag[JList[T]] = fakeClassTag JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f)(fakeClassTag))) } @@ -222,10 +222,11 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ - def groupBy[K](f: JFunction[T, K], numPartitions: Int): JavaPairRDD[K, JIterable[T]] = { - implicit val ctagK: ClassTag[K] = fakeClassTag + def groupBy[U](f: JFunction[T, U], numPartitions: Int): JavaPairRDD[U, JIterable[T]] = { + // The type parameter is U instead of K in order to work around a compiler bug; see SPARK-4459 + implicit val ctagK: ClassTag[U] = fakeClassTag implicit val ctagV: ClassTag[JList[T]] = fakeClassTag - JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f, numPartitions)(fakeClassTag[K]))) + JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f, numPartitions)(fakeClassTag[U]))) } /** @@ -459,8 +460,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Creates tuples of the elements in this RDD by applying `f`. */ - def keyBy[K](f: JFunction[T, K]): JavaPairRDD[K, T] = { - implicit val ctag: ClassTag[K] = fakeClassTag + def keyBy[U](f: JFunction[T, U]): JavaPairRDD[U, T] = { + // The type parameter is U instead of K in order to work around a compiler bug; see SPARK-4459 + implicit val ctag: ClassTag[U] = fakeClassTag JavaPairRDD.fromRDD(rdd.keyBy(f)) } diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 6a6d9bf6857d3..97f5c9f257e09 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -33,7 +33,7 @@ import org.apache.hadoop.mapred.{InputFormat, JobConf} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.spark._ -import org.apache.spark.SparkContext._ +import org.apache.spark.AccumulatorParam._ import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.broadcast.Broadcast diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala index b52d0a5028e84..86e94931300f8 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala @@ -19,7 +19,8 @@ package org.apache.spark.api.java import com.google.common.base.Optional -import scala.collection.convert.Wrappers.MapWrapper +import java.{util => ju} +import scala.collection.mutable private[spark] object JavaUtils { def optionToOptional[T](option: Option[T]): Optional[T] = @@ -32,7 +33,64 @@ private[spark] object JavaUtils { def mapAsSerializableJavaMap[A, B](underlying: collection.Map[A, B]) = new SerializableMapWrapper(underlying) + // Implementation is copied from scala.collection.convert.Wrappers.MapWrapper, + // but implements java.io.Serializable. It can't just be subclassed to make it + // Serializable since the MapWrapper class has no no-arg constructor. This class + // doesn't need a no-arg constructor though. class SerializableMapWrapper[A, B](underlying: collection.Map[A, B]) - extends MapWrapper(underlying) with java.io.Serializable + extends ju.AbstractMap[A, B] with java.io.Serializable { self => + override def size = underlying.size + + override def get(key: AnyRef): B = try { + underlying get key.asInstanceOf[A] match { + case None => null.asInstanceOf[B] + case Some(v) => v + } + } catch { + case ex: ClassCastException => null.asInstanceOf[B] + } + + override def entrySet: ju.Set[ju.Map.Entry[A, B]] = new ju.AbstractSet[ju.Map.Entry[A, B]] { + def size = self.size + + def iterator = new ju.Iterator[ju.Map.Entry[A, B]] { + val ui = underlying.iterator + var prev : Option[A] = None + + def hasNext = ui.hasNext + + def next() = { + val (k, v) = ui.next + prev = Some(k) + new ju.Map.Entry[A, B] { + import scala.util.hashing.byteswap32 + def getKey = k + def getValue = v + def setValue(v1 : B) = self.put(k, v1) + override def hashCode = byteswap32(k.hashCode) + (byteswap32(v.hashCode) << 16) + override def equals(other: Any) = other match { + case e: ju.Map.Entry[_, _] => k == e.getKey && v == e.getValue + case _ => false + } + } + } + + def remove() { + prev match { + case Some(k) => + underlying match { + case mm: mutable.Map[a, _] => + mm remove k + prev = None + case _ => + throw new UnsupportedOperationException("remove") + } + case _ => + throw new IllegalStateException("next must be called at least once before remove") + } + } + } + } + } } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index b80c771d58a8f..bad40e6529f74 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -19,7 +19,7 @@ package org.apache.spark.api.python import java.io._ import java.net._ -import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collections} +import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, UUID, Collections} import org.apache.spark.input.PortableDataStream @@ -34,7 +34,6 @@ import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.{InputFormat, OutputFormat, JobConf} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, OutputFormat => NewOutputFormat} import org.apache.spark._ -import org.apache.spark.SparkContext._ import org.apache.spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD @@ -47,7 +46,7 @@ private[spark] class PythonRDD( pythonIncludes: JList[String], preservePartitoning: Boolean, pythonExec: String, - broadcastVars: JList[Broadcast[Array[Array[Byte]]]], + broadcastVars: JList[Broadcast[PythonBroadcast]], accumulator: Accumulator[JList[Array[Byte]]]) extends RDD[Array[Byte]](parent) { @@ -230,8 +229,7 @@ private[spark] class PythonRDD( if (!oldBids.contains(broadcast.id)) { // send new broadcast dataOut.writeLong(broadcast.id) - dataOut.writeLong(broadcast.value.map(_.length.toLong).sum) - broadcast.value.foreach(dataOut.write) + PythonRDD.writeUTF(broadcast.value.path, dataOut) oldBids.add(broadcast.id) } } @@ -368,24 +366,8 @@ private[spark] object PythonRDD extends Logging { } } - def readBroadcastFromFile( - sc: JavaSparkContext, - filename: String): Broadcast[Array[Array[Byte]]] = { - val size = new File(filename).length() - val file = new DataInputStream(new FileInputStream(filename)) - val blockSize = 1 << 20 - val n = ((size + blockSize - 1) / blockSize).toInt - val obj = new Array[Array[Byte]](n) - try { - for (i <- 0 until n) { - val length = if (i < (n - 1)) blockSize else (size % blockSize).toInt - obj(i) = new Array[Byte](length) - file.readFully(obj(i)) - } - } finally { - file.close() - } - sc.broadcast(obj) + def readBroadcastFromFile(sc: JavaSparkContext, path: String): Broadcast[PythonBroadcast] = { + sc.broadcast(new PythonBroadcast(path)) } def writeIteratorToStream[T](iter: Iterator[T], dataOut: DataOutputStream) { @@ -824,3 +806,49 @@ private class PythonAccumulatorParam(@transient serverHost: String, serverPort: } } } + +/** + * An Wrapper for Python Broadcast, which is written into disk by Python. It also will + * write the data into disk after deserialization, then Python can read it from disks. + */ +private[spark] class PythonBroadcast(@transient var path: String) extends Serializable { + + /** + * Read data from disks, then copy it to `out` + */ + private def writeObject(out: ObjectOutputStream): Unit = Utils.tryOrIOException { + val in = new FileInputStream(new File(path)) + try { + Utils.copyStream(in, out) + } finally { + in.close() + } + } + + /** + * Write data into disk, using randomly generated name. + */ + private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { + val dir = new File(Utils.getLocalDir(SparkEnv.get.conf)) + val file = File.createTempFile("broadcast", "", dir) + path = file.getAbsolutePath + val out = new FileOutputStream(file) + try { + Utils.copyStream(in, out) + } finally { + out.close() + } + } + + /** + * Delete the file once the object is GCed. + */ + override def finalize() { + if (!path.isEmpty) { + val file = new File(path) + if (file.exists()) { + file.delete() + } + } + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index b9dd8557ee904..243d8edb72ed3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -88,10 +88,14 @@ private[deploy] object DeployMessages { case class KillDriver(driverId: String) extends DeployMessage + case class ApplicationFinished(id: String) + // Worker internal case object WorkDirCleanup // Sent to Worker actor periodically for cleaning up app folders + case object ReregisterWithMaster // used when a worker attempts to reconnect to a master + // AppClient to Master case class RegisterApplication(appDescription: ApplicationDescription) @@ -173,4 +177,5 @@ private[deploy] object DeployMessages { // Liveness checks in various places case object SendHeartbeat + } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 60ee115e393ce..57f9faf5ddd1d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -24,6 +24,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.fs.FileSystem.Statistics import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext} import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.UserGroupInformation @@ -183,6 +184,17 @@ class SparkHadoopUtil extends Logging { Class.forName("org.apache.hadoop.fs.FileSystem$Statistics$StatisticsData") statisticsDataClass.getDeclaredMethod(methodName) } + + /** + * Using reflection to get the Configuration from JobContext/TaskAttemptContext. If we directly + * call `JobContext/TaskAttemptContext.getConfiguration`, it will generate different byte codes + * for Hadoop 1.+ and Hadoop 2.+ because JobContext/TaskAttemptContext is class in Hadoop 1.+ + * while it's interface in Hadoop 2.+. + */ + def getConfigurationFromJobContext(context: JobContext): Configuration = { + val method = context.getClass.getMethod("getConfiguration") + method.invoke(context).asInstanceOf[Configuration] + } } object SparkHadoopUtil { diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 8a62519bd2315..955cbd6dab96d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -142,6 +142,8 @@ object SparkSubmit { printErrorAndExit("Cluster deploy mode is currently not supported for python applications.") case (_, CLUSTER) if isShell(args.primaryResource) => printErrorAndExit("Cluster deploy mode is not applicable to Spark shells.") + case (_, CLUSTER) if isSqlShell(args.mainClass) => + printErrorAndExit("Cluster deploy mode is not applicable to Spark SQL shell.") case _ => } @@ -279,6 +281,11 @@ object SparkSubmit { sysProps.getOrElseUpdate(k, v) } + // Ignore invalid spark.driver.host in cluster modes. + if (deployMode == CLUSTER) { + sysProps -= ("spark.driver.host") + } + // Resolve paths in certain spark properties val pathConfigs = Seq( "spark.jars", @@ -345,6 +352,11 @@ object SparkSubmit { System.exit(CLASS_NOT_FOUND_EXIT_STATUS) } + // SPARK-4170 + if (classOf[scala.App].isAssignableFrom(mainClass)) { + printWarning("Subclasses of scala.App may not work correctly. Use a main() method instead.") + } + val mainMethod = mainClass.getMethod("main", new Array[String](0).getClass) if (!Modifier.isStatic(mainMethod.getModifiers)) { throw new IllegalStateException("The main method in the given main class must be static") @@ -388,6 +400,13 @@ object SparkSubmit { primaryResource == SPARK_SHELL || primaryResource == PYSPARK_SHELL } + /** + * Return whether the given main class represents a sql shell. + */ + private[spark] def isSqlShell(mainClass: String): Boolean = { + mainClass == "org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" + } + /** * Return whether the given primary resource requires running python. */ diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index f0e9ee67f6a67..f174bc1af59b4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -120,6 +120,8 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St name = Option(name).orElse(sparkProperties.get("spark.app.name")).orNull jars = Option(jars).orElse(sparkProperties.get("spark.jars")).orNull deployMode = Option(deployMode).orElse(env.get("DEPLOY_MODE")).orNull + numExecutors = Option(numExecutors) + .getOrElse(sparkProperties.get("spark.executor.instances").orNull) // Try to set main class from JAR if no --class argument is given if (mainClass == null && !isPython && primaryResource != null) { @@ -212,7 +214,10 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St """.stripMargin } - /** Fill in values by parsing user options. */ + /** + * Fill in values by parsing user options. + * NOTE: Any changes here must be reflected in YarnClientSchedulerBackend. + */ private def parseOpts(opts: Seq[String]): Unit = { val EQ_SEPARATED_OPT="""(--[^=]+)=(.+)""".r diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala index d2687faad62b1..2eab9981845e8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala @@ -151,7 +151,8 @@ private[spark] object SparkSubmitDriverBootstrapper { val isWindows = Utils.isWindows val isSubprocess = sys.env.contains("IS_SUBPROCESS") if (!isWindows) { - val stdinThread = new RedirectThread(System.in, process.getOutputStream, "redirect stdin") + val stdinThread = new RedirectThread(System.in, process.getOutputStream, "redirect stdin", + propagateEof = true) stdinThread.start() // Spark submit (JVM) may run as a subprocess, and so this JVM should terminate on // broken pipe, signaling that the parent process has exited. This is the case if the diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala index 98a93d1fcb2a3..4efebcaa350fe 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala @@ -134,6 +134,7 @@ private[spark] class AppClient( val fullId = appId + "/" + id logInfo("Executor added: %s on %s (%s) with %d cores".format(fullId, workerId, hostPort, cores)) + master ! ExecutorStateChanged(appId, id, ExecutorState.RUNNING, None, None) listener.executorAdded(fullId, workerId, hostPort, cores, memory) case ExecutorUpdated(id, state, message, exitStatus) => diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 2d1609b973607..792d15b99ea0d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -17,34 +17,41 @@ package org.apache.spark.deploy.history -import java.io.FileNotFoundException +import java.io.{BufferedInputStream, FileNotFoundException, InputStream} import scala.collection.mutable import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.fs.permission.AccessControlException import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.io.CompressionCodec import org.apache.spark.scheduler._ import org.apache.spark.ui.SparkUI import org.apache.spark.util.Utils +/** + * A class that provides application history from event logs stored in the file system. + * This provider checks for new finished applications in the background periodically and + * renders the history application UI by parsing the associated event logs. + */ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider with Logging { + import FsHistoryProvider._ + private val NOT_STARTED = "" // Interval between each check for event log updates private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", conf.getInt("spark.history.updateInterval", 10)) * 1000 - private val logDir = conf.get("spark.history.fs.logDirectory", null) - private val resolvedLogDir = Option(logDir) - .map { d => Utils.resolveURI(d) } - .getOrElse { throw new IllegalArgumentException("Logging directory must be specified.") } + private val logDir = conf.getOption("spark.history.fs.logDirectory") + .map { d => Utils.resolveURI(d).toString } + .getOrElse(DEFAULT_LOG_DIR) - private val fs = Utils.getHadoopFileSystem(resolvedLogDir, - SparkHadoopUtil.get.newConfiguration(conf)) + private val fs = Utils.getHadoopFileSystem(logDir, SparkHadoopUtil.get.newConfiguration(conf)) // A timestamp of when the disk was last accessed to check for log updates private var lastLogCheckTimeMs = -1L @@ -59,6 +66,12 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis @volatile private var applications: mutable.LinkedHashMap[String, FsApplicationHistoryInfo] = new mutable.LinkedHashMap() + // Constants used to parse Spark 1.0.0 log directories. + private[history] val LOG_PREFIX = "EVENT_LOG_" + private[history] val SPARK_VERSION_PREFIX = "SPARK_VERSION_" + private[history] val COMPRESSION_CODEC_PREFIX = "COMPRESSION_CODEC_" + private[history] val APPLICATION_COMPLETE = "APPLICATION_COMPLETE" + /** * A background thread that periodically checks for event log updates on disk. * @@ -85,21 +98,28 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis initialize() - private def initialize() { + private def initialize(): Unit = { // Validate the log directory. - val path = new Path(resolvedLogDir) + val path = new Path(logDir) if (!fs.exists(path)) { - throw new IllegalArgumentException( - "Logging directory specified does not exist: %s".format(resolvedLogDir)) + var msg = s"Log directory specified does not exist: $logDir." + if (logDir == DEFAULT_LOG_DIR) { + msg += " Did you configure the correct one through spark.fs.history.logDirectory?" + } + throw new IllegalArgumentException(msg) } if (!fs.getFileStatus(path).isDir) { throw new IllegalArgumentException( - "Logging directory specified is not a directory: %s".format(resolvedLogDir)) + "Logging directory specified is not a directory: %s".format(logDir)) } checkForLogs() - logCheckingThread.setDaemon(true) - logCheckingThread.start() + + // Disable the background thread during tests. + if (!conf.contains("spark.testing")) { + logCheckingThread.setDaemon(true) + logCheckingThread.start() + } } override def getListing() = applications.values @@ -107,8 +127,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis override def getAppUI(appId: String): Option[SparkUI] = { try { applications.get(appId).map { info => - val (replayBus, appListener) = createReplayBus(fs.getFileStatus( - new Path(logDir, info.logDir))) + val replayBus = new ReplayListenerBus() val ui = { val conf = this.conf.clone() val appSecManager = new SecurityManager(conf) @@ -117,15 +136,17 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis // Do not call ui.bind() to avoid creating a new server for each application } - replayBus.replay() + val appListener = new ApplicationEventListener() + replayBus.addListener(appListener) + val appInfo = replay(fs.getFileStatus(new Path(logDir, info.logPath)), replayBus) - ui.setAppName(s"${appListener.appName.getOrElse(NOT_STARTED)} ($appId)") + ui.setAppName(s"${appInfo.name} ($appId)") val uiAclsEnabled = conf.getBoolean("spark.history.ui.acls.enable", false) ui.getSecurityManager.setAcls(uiAclsEnabled) // make sure to set admin acls before view acls so they are properly picked up ui.getSecurityManager.setAdminAcls(appListener.adminAcls.getOrElse("")) - ui.getSecurityManager.setViewAcls(appListener.sparkUser.getOrElse(NOT_STARTED), + ui.getSecurityManager.setViewAcls(appInfo.sparkUser, appListener.viewAcls.getOrElse("")) ui } @@ -134,49 +155,52 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis } } - override def getConfig(): Map[String, String] = - Map("Event Log Location" -> resolvedLogDir.toString) + override def getConfig(): Map[String, String] = Map("Event log directory" -> logDir.toString) /** * Builds the application list based on the current contents of the log directory. * Tries to reuse as much of the data already in memory as possible, by not reading * applications that haven't been updated since last time the logs were checked. */ - private def checkForLogs() = { + private[history] def checkForLogs(): Unit = { lastLogCheckTimeMs = getMonotonicTimeMs() logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs)) - try { - val logStatus = fs.listStatus(new Path(resolvedLogDir)) - val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]() - // Load all new logs from the log directory. Only directories that have a modification time - // later than the last known log directory will be loaded. + try { var newLastModifiedTime = lastModifiedTime - val logInfos = logDirs - .filter { dir => - if (fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))) { - val modTime = getModificationTime(dir) - newLastModifiedTime = math.max(newLastModifiedTime, modTime) - modTime > lastModifiedTime - } else { - false + val statusList = Option(fs.listStatus(new Path(logDir))).map(_.toSeq) + .getOrElse(Seq[FileStatus]()) + val logInfos = statusList + .filter { entry => + try { + val isFinishedApplication = + if (isLegacyLogDirectory(entry)) { + fs.exists(new Path(entry.getPath(), APPLICATION_COMPLETE)) + } else { + !entry.getPath().getName().endsWith(EventLoggingListener.IN_PROGRESS) + } + + if (isFinishedApplication) { + val modTime = getModificationTime(entry) + newLastModifiedTime = math.max(newLastModifiedTime, modTime) + modTime >= lastModifiedTime + } else { + false + } + } catch { + case e: AccessControlException => + // Do not use "logInfo" since these messages can get pretty noisy if printed on + // every poll. + logDebug(s"No permission to read $entry, ignoring.") + false } } - .flatMap { dir => + .flatMap { entry => try { - val (replayBus, appListener) = createReplayBus(dir) - replayBus.replay() - Some(new FsApplicationHistoryInfo( - dir.getPath().getName(), - appListener.appId.getOrElse(dir.getPath().getName()), - appListener.appName.getOrElse(NOT_STARTED), - appListener.startTime.getOrElse(-1L), - appListener.endTime.getOrElse(-1L), - getModificationTime(dir), - appListener.sparkUser.getOrElse(NOT_STARTED))) + Some(replay(entry, new ReplayListenerBus())) } catch { case e: Exception => - logInfo(s"Failed to load application log data from $dir.", e) + logError(s"Failed to load application log data from $entry.", e) None } } @@ -210,42 +234,109 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis applications = newApps } } catch { - case t: Throwable => logError("Exception in checking for event log updates", t) + case e: Exception => logError("Exception in checking for event log updates", e) } } - private def createReplayBus(logDir: FileStatus): (ReplayListenerBus, ApplicationEventListener) = { - val path = logDir.getPath() - val elogInfo = EventLoggingListener.parseLoggingInfo(path, fs) - val replayBus = new ReplayListenerBus(elogInfo.logPaths, fs, elogInfo.compressionCodec) - val appListener = new ApplicationEventListener - replayBus.addListener(appListener) - (replayBus, appListener) + /** + * Replays the events in the specified log file and returns information about the associated + * application. + */ + private def replay(eventLog: FileStatus, bus: ReplayListenerBus): FsApplicationHistoryInfo = { + val logPath = eventLog.getPath() + val (logInput, sparkVersion) = + if (isLegacyLogDirectory(eventLog)) { + openLegacyEventLog(logPath) + } else { + EventLoggingListener.openEventLog(logPath, fs) + } + try { + val appListener = new ApplicationEventListener + bus.addListener(appListener) + bus.replay(logInput, sparkVersion) + new FsApplicationHistoryInfo( + logPath.getName(), + appListener.appId.getOrElse(logPath.getName()), + appListener.appName.getOrElse(NOT_STARTED), + appListener.startTime.getOrElse(-1L), + appListener.endTime.getOrElse(-1L), + getModificationTime(eventLog), + appListener.sparkUser.getOrElse(NOT_STARTED)) + } finally { + logInput.close() + } } - /** Return when this directory was last modified. */ - private def getModificationTime(dir: FileStatus): Long = { - try { - val logFiles = fs.listStatus(dir.getPath) - if (logFiles != null && !logFiles.isEmpty) { - logFiles.map(_.getModificationTime).max - } else { - dir.getModificationTime + /** + * Loads a legacy log directory. This assumes that the log directory contains a single event + * log file (along with other metadata files), which is the case for directories generated by + * the code in previous releases. + * + * @return 2-tuple of (input stream of the events, version of Spark which wrote the log) + */ + private[history] def openLegacyEventLog(dir: Path): (InputStream, String) = { + val children = fs.listStatus(dir) + var eventLogPath: Path = null + var codecName: Option[String] = None + var sparkVersion: String = null + + children.foreach { child => + child.getPath().getName() match { + case name if name.startsWith(LOG_PREFIX) => + eventLogPath = child.getPath() + + case codec if codec.startsWith(COMPRESSION_CODEC_PREFIX) => + codecName = Some(codec.substring(COMPRESSION_CODEC_PREFIX.length())) + + case version if version.startsWith(SPARK_VERSION_PREFIX) => + sparkVersion = version.substring(SPARK_VERSION_PREFIX.length()) + + case _ => } - } catch { - case t: Throwable => - logError("Exception in accessing modification time of %s".format(dir.getPath), t) - -1L + } + + if (eventLogPath == null || sparkVersion == null) { + throw new IllegalArgumentException(s"$dir is not a Spark application log directory.") + } + + val codec = try { + codecName.map { c => CompressionCodec.createCodec(conf, c) } + } catch { + case e: Exception => + throw new IllegalArgumentException(s"Unknown compression codec $codecName.") + } + + val in = new BufferedInputStream(fs.open(eventLogPath)) + (codec.map(_.compressedInputStream(in)).getOrElse(in), sparkVersion) + } + + /** + * Return whether the specified event log path contains a old directory-based event log. + * Previously, the event log of an application comprises of multiple files in a directory. + * As of Spark 1.3, these files are consolidated into a single one that replaces the directory. + * See SPARK-2261 for more detail. + */ + private def isLegacyLogDirectory(entry: FileStatus): Boolean = entry.isDir() + + private def getModificationTime(fsEntry: FileStatus): Long = { + if (fsEntry.isDir) { + fs.listStatus(fsEntry.getPath).map(_.getModificationTime()).max + } else { + fsEntry.getModificationTime() } } /** Returns the system's mononotically increasing time. */ - private def getMonotonicTimeMs() = System.nanoTime() / (1000 * 1000) + private def getMonotonicTimeMs(): Long = System.nanoTime() / (1000 * 1000) + +} +private object FsHistoryProvider { + val DEFAULT_LOG_DIR = "file:/tmp/spark-events" } private class FsApplicationHistoryInfo( - val logDir: String, + val logPath: String, id: String, name: String, startTime: Long, diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index 0e249e51a77d8..0d5dcfb1ddffe 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -26,6 +26,7 @@ import org.apache.spark.ui.{WebUIPage, UIUtils} private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { private val pageSize = 20 + private val plusOrMinus = 2 def render(request: HttpServletRequest): Seq[Node] = { val requestedPage = Option(request.getParameter("page")).getOrElse("1").toInt @@ -39,6 +40,9 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { val last = Math.min(actualFirst + pageSize, allApps.size) - 1 val pageCount = allApps.size / pageSize + (if (allApps.size % pageSize > 0) 1 else 0) + val secondPageFromLeft = 2 + val secondPageFromRight = pageCount - 1 + val appTable = UIUtils.listingTable(appHeader, appRow, apps) val providerConfig = parent.getProviderConfig() val content = @@ -48,17 +52,48 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { {providerConfig.map { case (k, v) =>
  • {k}: {v}
  • }} { + // This displays the indices of pages that are within `plusOrMinus` pages of + // the current page. Regardless of where the current page is, this also links + // to the first and last page. If the current page +/- `plusOrMinus` is greater + // than the 2nd page from the first page or less than the 2nd page from the last + // page, `...` will be displayed. if (allApps.size > 0) { + val leftSideIndices = + rangeIndices(actualPage - plusOrMinus until actualPage, 1 < _) + val rightSideIndices = + rangeIndices(actualPage + 1 to actualPage + plusOrMinus, _ < pageCount) +

    Showing {actualFirst + 1}-{last + 1} of {allApps.size} - - {if (actualPage > 1) <} - {if (actualPage < pageCount) >} - + + { + if (actualPage > 1) { + < + 1 + } + } + {if (actualPage - plusOrMinus > secondPageFromLeft) " ... "} + {leftSideIndices} + {actualPage} + {rightSideIndices} + {if (actualPage + plusOrMinus < secondPageFromRight) " ... "} + { + if (actualPage < pageCount) { + {pageCount} + > + } + } +

    ++ appTable } else { -

    No Completed Applications Found

    +

    No completed applications found!

    ++ +

    Did you specify the correct logging directory? + Please verify your setting of + spark.history.fs.logDirectory and whether you have the permissions to + access it.
    It is also possible that your application did not run to + completion or did not stop the SparkContext. +

    } } @@ -75,6 +110,10 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { "Spark User", "Last Updated") + private def rangeIndices(range: Seq[Int], condition: Int => Boolean): Seq[Node] = { + range.filter(condition).map(nextPage => {nextPage} ) + } + private def appRow(info: ApplicationHistoryInfo): Seq[Node] = { val uiAddress = HistoryServer.UI_PATH_PREFIX + s"/${info.id}" val startTime = UIUtils.formatDate(info.startTime) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index ce00c0ffd21e0..fa9bfe5426b6c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -158,11 +158,12 @@ class HistoryServer( /** * The recommended way of starting and stopping a HistoryServer is through the scripts - * start-history-server.sh and stop-history-server.sh. The path to a base log directory - * is must be specified, while the requested UI port is optional. For example: + * start-history-server.sh and stop-history-server.sh. The path to a base log directory, + * as well as any other relevant history server configuration, should be specified via + * the $SPARK_HISTORY_OPTS environment variable. For example: * - * ./sbin/spark-history-server.sh /tmp/spark-events - * ./sbin/spark-history-server.sh hdfs://1.2.3.4:9000/spark-events + * export SPARK_HISTORY_OPTS="-Dspark.history.fs.logDirectory=/tmp/spark-events" + * ./sbin/start-history-server.sh * * This launches the HistoryServer as a Spark daemon. */ diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala index 5bce32a04d16d..b1270ade9f750 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala @@ -17,14 +17,13 @@ package org.apache.spark.deploy.history -import org.apache.spark.SparkConf +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.util.Utils /** * Command-line parser for the master. */ -private[spark] class HistoryServerArguments(conf: SparkConf, args: Array[String]) { - private var logDir: String = null +private[spark] class HistoryServerArguments(conf: SparkConf, args: Array[String]) extends Logging { private var propertiesFile: String = null parse(args.toList) @@ -32,7 +31,8 @@ private[spark] class HistoryServerArguments(conf: SparkConf, args: Array[String] private def parse(args: List[String]): Unit = { args match { case ("--dir" | "-d") :: value :: tail => - logDir = value + logWarning("Setting log directory through the command line is deprecated as of " + + "Spark 1.1.0. Please set this through spark.history.fs.logDirectory instead.") conf.set("spark.history.fs.logDirectory", value) System.setProperty("spark.history.fs.logDirectory", value) parse(tail) @@ -78,9 +78,10 @@ private[spark] class HistoryServerArguments(conf: SparkConf, args: Array[String] | (default 50) |FsHistoryProvider options: | - | spark.history.fs.logDirectory Directory where app logs are stored (required) - | spark.history.fs.updateInterval How often to reload log data from storage (in seconds, - | default 10) + | spark.history.fs.logDirectory Directory where app logs are stored + | (default: file:/tmp/spark-events) + | spark.history.fs.updateInterval How often to reload log data from storage + | (in seconds, default: 10) |""".stripMargin) System.exit(exitCode) } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala index 6ff2aa5244847..36a2e2c6a6349 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala @@ -18,12 +18,13 @@ package org.apache.spark.deploy.master import java.io._ -import java.nio.ByteBuffer + +import scala.reflect.ClassTag + +import akka.serialization.Serialization import org.apache.spark.Logging -import org.apache.spark.serializer.Serializer -import scala.reflect.ClassTag /** * Stores data in a single on-disk directory with one file per application and worker. @@ -34,10 +35,9 @@ import scala.reflect.ClassTag */ private[spark] class FileSystemPersistenceEngine( val dir: String, - val serialization: Serializer) + val serialization: Serialization) extends PersistenceEngine with Logging { - val serializer = serialization.newInstance() new File(dir).mkdir() override def persist(name: String, obj: Object): Unit = { @@ -56,17 +56,17 @@ private[spark] class FileSystemPersistenceEngine( private def serializeIntoFile(file: File, value: AnyRef) { val created = file.createNewFile() if (!created) { throw new IllegalStateException("Could not create file: " + file) } - - val out = serializer.serializeStream(new FileOutputStream(file)) + val serializer = serialization.findSerializerFor(value) + val serialized = serializer.toBinary(value) + val out = new FileOutputStream(file) try { - out.writeObject(value) + out.write(serialized) } finally { out.close() } - } - def deserializeFromFile[T](file: File): T = { + private def deserializeFromFile[T](file: File)(implicit m: ClassTag[T]): T = { val fileData = new Array[Byte](file.length().asInstanceOf[Int]) val dis = new DataInputStream(new FileInputStream(file)) try { @@ -74,7 +74,9 @@ private[spark] class FileSystemPersistenceEngine( } finally { dis.close() } - - serializer.deserializeStream(dis).readObject() + val clazz = m.runtimeClass.asInstanceOf[Class[T]] + val serializer = serialization.serializerFor(clazz) + serializer.fromBinary(fileData).asInstanceOf[T] } + } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 021454e25804c..e8a5cfc746fed 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -17,6 +17,7 @@ package org.apache.spark.deploy.master +import java.io.FileNotFoundException import java.net.URLEncoder import java.text.SimpleDateFormat import java.util.Date @@ -30,7 +31,9 @@ import scala.util.Random import akka.actor._ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} +import akka.serialization.Serialization import akka.serialization.SerializationExtension +import org.apache.hadoop.fs.Path import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, @@ -55,6 +58,7 @@ private[spark] class Master( import context.dispatcher // to use Akka's scheduler.schedule() val conf = new SparkConf + val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs val WORKER_TIMEOUT = conf.getLong("spark.worker.timeout", 60) * 1000 @@ -119,6 +123,7 @@ private[spark] class Master( override def preStart() { logInfo("Starting Spark master at " + masterUrl) + logInfo(s"Running Spark version ${org.apache.spark.SPARK_VERSION}") // Listen for remote client disconnection events, since they don't go through Akka's watch() context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) webUi.bind() @@ -128,19 +133,26 @@ private[spark] class Master( masterMetricsSystem.registerSource(masterSource) masterMetricsSystem.start() applicationMetricsSystem.start() + // Attach the master and app metrics servlet handler to the web ui after the metrics systems are + // started. + masterMetricsSystem.getServletHandlers.foreach(webUi.attachHandler) + applicationMetricsSystem.getServletHandlers.foreach(webUi.attachHandler) val (persistenceEngine_, leaderElectionAgent_) = RECOVERY_MODE match { case "ZOOKEEPER" => logInfo("Persisting recovery state to ZooKeeper") - val zkFactory = new ZooKeeperRecoveryModeFactory(conf) + val zkFactory = + new ZooKeeperRecoveryModeFactory(conf, SerializationExtension(context.system)) (zkFactory.createPersistenceEngine(), zkFactory.createLeaderElectionAgent(this)) case "FILESYSTEM" => - val fsFactory = new FileSystemRecoveryModeFactory(conf) + val fsFactory = + new FileSystemRecoveryModeFactory(conf, SerializationExtension(context.system)) (fsFactory.createPersistenceEngine(), fsFactory.createLeaderElectionAgent(this)) case "CUSTOM" => val clazz = Class.forName(conf.get("spark.deploy.recoveryMode.factory")) - val factory = clazz.getConstructor(conf.getClass) - .newInstance(conf).asInstanceOf[StandaloneRecoveryModeFactory] + val factory = clazz.getConstructor(conf.getClass, Serialization.getClass) + .newInstance(conf, SerializationExtension(context.system)) + .asInstanceOf[StandaloneRecoveryModeFactory] (factory.createPersistenceEngine(), factory.createLeaderElectionAgent(this)) case _ => (new BlackHolePersistenceEngine(), new MonarchyLeaderAgent(this)) @@ -506,7 +518,7 @@ private[spark] class Master( val shuffledAliveWorkers = Random.shuffle(workers.toSeq.filter(_.state == WorkerState.ALIVE)) val numWorkersAlive = shuffledAliveWorkers.size var curPos = 0 - + for (driver <- waitingDrivers.toList) { // iterate over a copy of waitingDrivers // We assign workers to each waiting driver in a round-robin fashion. For each driver, we // start from the last worker that was assigned a driver, and continue onwards until we have @@ -693,6 +705,11 @@ private[spark] class Master( } persistenceEngine.removeApplication(app) schedule() + + // Tell all workers that the application has finished, so they can clean up any app state. + workers.foreach { w => + w.actor ! ApplicationFinished(app.id) + } } } @@ -703,41 +720,50 @@ private[spark] class Master( def rebuildSparkUI(app: ApplicationInfo): Boolean = { val appName = app.desc.name val notFoundBasePath = HistoryServer.UI_PATH_PREFIX + "/not-found" - val eventLogDir = app.desc.eventLogDir.getOrElse { - // Event logging is not enabled for this application - app.desc.appUiUrl = notFoundBasePath - return false + val eventLogFile = app.desc.eventLogDir + .map { dir => EventLoggingListener.getLogPath(dir, app.id) } + .getOrElse { + // Event logging is not enabled for this application + app.desc.appUiUrl = notFoundBasePath + return false } + val fs = Utils.getHadoopFileSystem(eventLogFile, hadoopConf) - val appEventLogDir = EventLoggingListener.getLogDirPath(eventLogDir, app.id) - val fileSystem = Utils.getHadoopFileSystem(appEventLogDir, - SparkHadoopUtil.get.newConfiguration(conf)) - val eventLogInfo = EventLoggingListener.parseLoggingInfo(appEventLogDir, fileSystem) - val eventLogPaths = eventLogInfo.logPaths - val compressionCodec = eventLogInfo.compressionCodec - - if (eventLogPaths.isEmpty) { - // Event logging is enabled for this application, but no event logs are found + if (fs.exists(new Path(eventLogFile + EventLoggingListener.IN_PROGRESS))) { + // Event logging is enabled for this application, but the application is still in progress val title = s"Application history not found (${app.id})" - var msg = s"No event logs found for application $appName in $appEventLogDir." + var msg = s"Application $appName is still in progress." logWarning(msg) - msg += " Did you specify the correct logging directory?" msg = URLEncoder.encode(msg, "UTF-8") app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&title=$title" return false } try { - val replayBus = new ReplayListenerBus(eventLogPaths, fileSystem, compressionCodec) + val (logInput, sparkVersion) = EventLoggingListener.openEventLog(new Path(eventLogFile), fs) + val replayBus = new ReplayListenerBus() val ui = SparkUI.createHistoryUI(new SparkConf, replayBus, new SecurityManager(conf), appName + " (completed)", HistoryServer.UI_PATH_PREFIX + s"/${app.id}") - replayBus.replay() + try { + replayBus.replay(logInput, sparkVersion) + } finally { + logInput.close() + } appIdToUI(app.id) = ui webUi.attachSparkUI(ui) // Application UI is successfully rebuilt, so link the Master UI to it - app.desc.appUiUrl = ui.getBasePath + app.desc.appUiUrl = ui.basePath true } catch { + case fnf: FileNotFoundException => + // Event logging is enabled for this application, but no event logs are found + val title = s"Application history not found (${app.id})" + var msg = s"No event logs found for application $appName in $eventLogFile." + logWarning(msg) + msg += " Did you specify the correct logging directory?" + msg = URLEncoder.encode(msg, "UTF-8") + app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&title=$title" + false case e: Exception => // Relay exception message to application UI page val title = s"Application history load error (${app.id})" diff --git a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala index d9d36c1ed5f9f..1096eb0368357 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala @@ -17,9 +17,10 @@ package org.apache.spark.deploy.master +import akka.serialization.Serialization + import org.apache.spark.{Logging, SparkConf} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.serializer.JavaSerializer /** * ::DeveloperApi:: @@ -29,7 +30,7 @@ import org.apache.spark.serializer.JavaSerializer * */ @DeveloperApi -abstract class StandaloneRecoveryModeFactory(conf: SparkConf) { +abstract class StandaloneRecoveryModeFactory(conf: SparkConf, serializer: Serialization) { /** * PersistenceEngine defines how the persistent data(Information about worker, driver etc..) @@ -48,21 +49,21 @@ abstract class StandaloneRecoveryModeFactory(conf: SparkConf) { * LeaderAgent in this case is a no-op. Since leader is forever leader as the actual * recovery is made by restoring from filesystem. */ -private[spark] class FileSystemRecoveryModeFactory(conf: SparkConf) - extends StandaloneRecoveryModeFactory(conf) with Logging { +private[spark] class FileSystemRecoveryModeFactory(conf: SparkConf, serializer: Serialization) + extends StandaloneRecoveryModeFactory(conf, serializer) with Logging { val RECOVERY_DIR = conf.get("spark.deploy.recoveryDirectory", "") def createPersistenceEngine() = { logInfo("Persisting recovery state to directory: " + RECOVERY_DIR) - new FileSystemPersistenceEngine(RECOVERY_DIR, new JavaSerializer(conf)) + new FileSystemPersistenceEngine(RECOVERY_DIR, serializer) } def createLeaderElectionAgent(master: LeaderElectable) = new MonarchyLeaderAgent(master) } -private[spark] class ZooKeeperRecoveryModeFactory(conf: SparkConf) - extends StandaloneRecoveryModeFactory(conf) { - def createPersistenceEngine() = new ZooKeeperPersistenceEngine(new JavaSerializer(conf), conf) +private[spark] class ZooKeeperRecoveryModeFactory(conf: SparkConf, serializer: Serialization) + extends StandaloneRecoveryModeFactory(conf, serializer) { + def createPersistenceEngine() = new ZooKeeperPersistenceEngine(conf, serializer) def createLeaderElectionAgent(master: LeaderElectable) = new ZooKeeperLeaderElectionAgent(master, conf) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala index 96c2139eb02f0..e11ac031fb9c6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -17,27 +17,24 @@ package org.apache.spark.deploy.master +import akka.serialization.Serialization + import scala.collection.JavaConversions._ +import scala.reflect.ClassTag import org.apache.curator.framework.CuratorFramework import org.apache.zookeeper.CreateMode import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.serializer.Serializer -import java.nio.ByteBuffer -import scala.reflect.ClassTag - -private[spark] class ZooKeeperPersistenceEngine(val serialization: Serializer, conf: SparkConf) +private[spark] class ZooKeeperPersistenceEngine(conf: SparkConf, val serialization: Serialization) extends PersistenceEngine with Logging { val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/master_status" val zk: CuratorFramework = SparkCuratorUtil.newClient(conf) - val serializer = serialization.newInstance() - SparkCuratorUtil.mkdir(zk, WORKING_DIR) @@ -59,14 +56,17 @@ private[spark] class ZooKeeperPersistenceEngine(val serialization: Serializer, c } private def serializeIntoFile(path: String, value: AnyRef) { - val serialized = serializer.serialize(value) - zk.create().withMode(CreateMode.PERSISTENT).forPath(path, serialized.array()) + val serializer = serialization.findSerializerFor(value) + val serialized = serializer.toBinary(value) + zk.create().withMode(CreateMode.PERSISTENT).forPath(path, serialized) } - def deserializeFromFile[T](filename: String): Option[T] = { + def deserializeFromFile[T](filename: String)(implicit m: ClassTag[T]): Option[T] = { val fileData = zk.getData().forPath(WORKING_DIR + "/" + filename) + val clazz = m.runtimeClass.asInstanceOf[Class[T]] + val serializer = serialization.serializerFor(clazz) try { - Some(serializer.deserialize(ByteBuffer.wrap(fileData))) + Some(serializer.fromBinary(fileData).asInstanceOf[T]) } catch { case e: Exception => { logWarning("Exception while reading persisted file, deleting", e) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index d86ec1e03e45c..73400c5affb5d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -41,8 +41,6 @@ class MasterWebUI(val master: Master, requestedPort: Int) attachPage(new HistoryNotFoundPage(this)) attachPage(new MasterPage(this)) attachHandler(createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static")) - master.masterMetricsSystem.getServletHandlers.foreach(attachHandler) - master.applicationMetricsSystem.getServletHandlers.foreach(attachHandler) } /** Attach a reconstructed UI to this Master UI. Only valid after bind(). */ diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 8ba6a01bbcb97..acbdf0d8bd7bc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -47,6 +47,7 @@ private[spark] class ExecutorRunner( val executorDir: File, val workerUrl: String, val conf: SparkConf, + val appLocalDirs: Seq[String], var state: ExecutorState.Value) extends Logging { @@ -77,7 +78,7 @@ private[spark] class ExecutorRunner( /** * Kill executor process, wait for exit and notify worker to update resource status. * - * @param message the exception message which caused the executor's death + * @param message the exception message which caused the executor's death */ private def killProcess(message: Option[String]) { var exitCode: Option[Int] = None @@ -129,6 +130,7 @@ private[spark] class ExecutorRunner( logInfo("Launch command: " + command.mkString("\"", "\" \"", "\"")) builder.directory(executorDir) + builder.environment.put("SPARK_LOCAL_DIRS", appLocalDirs.mkString(",")) // In case we are running this from within the Spark Shell, avoid creating a "scala" // parent process for the executor command builder.environment.put("SPARK_LAUNCH_WITH_SCALA", "0") @@ -144,8 +146,6 @@ private[spark] class ExecutorRunner( Files.write(header, stderr, UTF_8) stderrAppender = FileAppender(process.getErrorStream, stderr, conf) - state = ExecutorState.RUNNING - worker ! ExecutorStateChanged(appId, execId, state, None, None) // Wait for it to exit; executor may exit with code 0 (when driver instructs it to shutdown) // or with nonzero exit code val exitCode = process.waitFor() diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index ca262de832e25..f0f3da5eec4df 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -21,10 +21,9 @@ import java.io.File import java.io.IOException import java.text.SimpleDateFormat import java.util.{UUID, Date} -import java.util.concurrent.TimeUnit import scala.collection.JavaConversions._ -import scala.collection.mutable.HashMap +import scala.collection.mutable.{HashMap, HashSet} import scala.concurrent.duration._ import scala.language.postfixOps import scala.util.Random @@ -110,6 +109,8 @@ private[spark] class Worker( val finishedExecutors = new HashMap[String, ExecutorRunner] val drivers = new HashMap[String, DriverRunner] val finishedDrivers = new HashMap[String, DriverRunner] + val appDirectories = new HashMap[String, Seq[String]] + val finishedApps = new HashSet[String] // The shuffle service is not actually started unless configured. val shuffleService = new StandaloneWorkerShuffleService(conf, securityMgr) @@ -154,6 +155,7 @@ private[spark] class Worker( assert(!registered) logInfo("Starting Spark worker %s:%d with %d cores, %s RAM".format( host, port, cores, Utils.megabytesToString(memory))) + logInfo(s"Running Spark version ${org.apache.spark.SPARK_VERSION}") logInfo("Spark home: " + sparkHome) createWorkDir() context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) @@ -164,6 +166,8 @@ private[spark] class Worker( metricsSystem.registerSource(workerSource) metricsSystem.start() + // Attach the worker metrics servlet handler to the web ui after the metrics system is started. + metricsSystem.getServletHandlers.foreach(webUi.attachHandler) } def changeMaster(url: String, uiUrl: String) { @@ -177,6 +181,9 @@ private[spark] class Worker( throw new SparkException("Invalid spark URL: " + x) } connected = true + // Cancel any outstanding re-registration attempts because we found a new master + registrationRetryTimer.foreach(_.cancel()) + registrationRetryTimer = None } private def tryRegisterAllMasters() { @@ -187,7 +194,12 @@ private[spark] class Worker( } } - private def retryConnectToMaster() { + /** + * Re-register with the master because a network failure or a master failure has occurred. + * If the re-registration attempt threshold is exceeded, the worker exits with error. + * Note that for thread-safety this should only be called from the actor. + */ + private def reregisterWithMaster(): Unit = { Utils.tryOrExit { connectionAttemptCount += 1 if (registered) { @@ -195,12 +207,40 @@ private[spark] class Worker( registrationRetryTimer = None } else if (connectionAttemptCount <= TOTAL_REGISTRATION_RETRIES) { logInfo(s"Retrying connection to master (attempt # $connectionAttemptCount)") - tryRegisterAllMasters() + /** + * Re-register with the active master this worker has been communicating with. If there + * is none, then it means this worker is still bootstrapping and hasn't established a + * connection with a master yet, in which case we should re-register with all masters. + * + * It is important to re-register only with the active master during failures. Otherwise, + * if the worker unconditionally attempts to re-register with all masters, the following + * race condition may arise and cause a "duplicate worker" error detailed in SPARK-4592: + * + * (1) Master A fails and Worker attempts to reconnect to all masters + * (2) Master B takes over and notifies Worker + * (3) Worker responds by registering with Master B + * (4) Meanwhile, Worker's previous reconnection attempt reaches Master B, + * causing the same Worker to register with Master B twice + * + * Instead, if we only register with the known active master, we can assume that the + * old master must have died because another master has taken over. Note that this is + * still not safe if the old master recovers within this interval, but this is a much + * less likely scenario. + */ + if (master != null) { + master ! RegisterWorker( + workerId, host, port, cores, memory, webUi.boundPort, publicAddress) + } else { + // We are retrying the initial registration + tryRegisterAllMasters() + } + // We have exceeded the initial registration retry threshold + // All retries from now on should use a higher interval if (connectionAttemptCount == INITIAL_REGISTRATION_RETRIES) { registrationRetryTimer.foreach(_.cancel()) registrationRetryTimer = Some { context.system.scheduler.schedule(PROLONGED_REGISTRATION_RETRY_INTERVAL, - PROLONGED_REGISTRATION_RETRY_INTERVAL)(retryConnectToMaster) + PROLONGED_REGISTRATION_RETRY_INTERVAL, self, ReregisterWithMaster) } } } else { @@ -220,7 +260,7 @@ private[spark] class Worker( connectionAttemptCount = 0 registrationRetryTimer = Some { context.system.scheduler.schedule(INITIAL_REGISTRATION_RETRY_INTERVAL, - INITIAL_REGISTRATION_RETRY_INTERVAL)(retryConnectToMaster) + INITIAL_REGISTRATION_RETRY_INTERVAL, self, ReregisterWithMaster) } case Some(_) => logInfo("Not spawning another attempt to register with the master, since there is an" + @@ -257,7 +297,7 @@ private[spark] class Worker( val isAppStillRunning = executors.values.map(_.appId).contains(appIdFromDir) dir.isDirectory && !isAppStillRunning && !Utils.doesDirectoryContainAnyNewFiles(dir, APP_DATA_RETENTION_SECS) - }.foreach { dir => + }.foreach { dir => logInfo(s"Removing directory: ${dir.getPath}") Utils.deleteRecursively(dir) } @@ -302,8 +342,19 @@ private[spark] class Worker( throw new IOException("Failed to create directory " + executorDir) } + // Create local dirs for the executor. These are passed to the executor via the + // SPARK_LOCAL_DIRS environment variable, and deleted by the Worker when the + // application finishes. + val appLocalDirs = appDirectories.get(appId).getOrElse { + Utils.getOrCreateLocalRootDirs(conf).map { dir => + Utils.createDirectory(dir).getAbsolutePath() + }.toSeq + } + appDirectories(appId) = appLocalDirs + val manager = new ExecutorRunner(appId, execId, appDesc, cores_, memory_, - self, workerId, host, sparkHome, executorDir, akkaUrl, conf, ExecutorState.LOADING) + self, workerId, host, sparkHome, executorDir, akkaUrl, conf, appLocalDirs, + ExecutorState.LOADING) executors(appId + "/" + execId) = manager manager.start() coresUsed += cores_ @@ -340,6 +391,7 @@ private[spark] class Worker( message.map(" message " + _).getOrElse("") + exitStatus.map(" exitStatus " + _).getOrElse("")) } + maybeCleanupApplication(appId) } case KillExecutor(masterUrl, appId, execId) => @@ -400,12 +452,18 @@ private[spark] class Worker( logInfo(s"$x Disassociated !") masterDisconnected() - case RequestWorkerState => { + case RequestWorkerState => sender ! WorkerStateResponse(host, port, workerId, executors.values.toList, finishedExecutors.values.toList, drivers.values.toList, finishedDrivers.values.toList, activeMasterUrl, cores, memory, coresUsed, memoryUsed, activeMasterWebUiUrl) - } + + case ReregisterWithMaster => + reregisterWithMaster() + + case ApplicationFinished(id) => + finishedApps += id + maybeCleanupApplication(id) } private def masterDisconnected() { @@ -414,6 +472,19 @@ private[spark] class Worker( registerWithMaster() } + private def maybeCleanupApplication(id: String): Unit = { + val shouldCleanup = finishedApps.contains(id) && !executors.values.exists(_.appId == id) + if (shouldCleanup) { + finishedApps -= id + appDirectories.remove(id).foreach { dirList => + logInfo(s"Cleaning up local directories for application $id") + dirList.foreach { dir => + Utils.deleteRecursively(new File(dir)) + } + } + } + } + def generateWorkerId(): String = { "worker-%s-%s-%d".format(createDateFormat.format(new Date), host, port) } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index b07942a9ca729..7ac81a2d87efd 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -50,7 +50,6 @@ class WorkerWebUI( attachHandler(createStaticHandler(WorkerWebUI.STATIC_RESOURCE_BASE, "/static")) attachHandler(createServletHandler("/log", (request: HttpServletRequest) => logPage.renderLog(request), worker.securityMgr)) - worker.metricsSystem.getServletHandlers.foreach(attachHandler) } } diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 5f46f3b1f085e..c794a7bc3599e 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import scala.concurrent.Await -import akka.actor.{Actor, ActorSelection, ActorSystem, Props} +import akka.actor.{Actor, ActorSelection, Props} import akka.pattern.Patterns import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent} @@ -38,8 +38,7 @@ private[spark] class CoarseGrainedExecutorBackend( executorId: String, hostPort: String, cores: Int, - sparkProperties: Seq[(String, String)], - actorSystem: ActorSystem) + env: SparkEnv) extends Actor with ActorLogReceive with ExecutorBackend with Logging { Utils.checkHostPort(hostPort, "Expected hostport") @@ -58,8 +57,7 @@ private[spark] class CoarseGrainedExecutorBackend( case RegisteredExecutor => logInfo("Successfully registered with driver") val (hostname, _) = Utils.parseHostPort(hostPort) - executor = new Executor(executorId, hostname, sparkProperties, cores, isLocal = false, - actorSystem) + executor = new Executor(executorId, hostname, env, isLocal = false) case RegisterExecutorFailed(message) => logError("Slave registration failed: " + message) @@ -70,7 +68,7 @@ private[spark] class CoarseGrainedExecutorBackend( logError("Received LaunchTask command but executor was null") System.exit(1) } else { - val ser = SparkEnv.get.closureSerializer.newInstance() + val ser = env.closureSerializer.newInstance() val taskDesc = ser.deserialize[TaskDescription](data.value) logInfo("Got assigned task " + taskDesc.taskId) executor.launchTask(this, taskDesc.taskId, taskDesc.name, taskDesc.serializedTask) @@ -128,21 +126,25 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { Seq[(String, String)](("spark.app.id", appId)) fetcher.shutdown() - // Create a new ActorSystem using driver's Spark properties to run the backend. + // Create SparkEnv using properties we fetched from the driver. val driverConf = new SparkConf().setAll(props) - val (actorSystem, boundPort) = AkkaUtils.createActorSystem( - SparkEnv.executorActorSystemName, - hostname, port, driverConf, new SecurityManager(driverConf)) - // set it + val env = SparkEnv.createExecutorEnv( + driverConf, executorId, hostname, port, cores, isLocal = false) + + // SparkEnv sets spark.driver.port so it shouldn't be 0 anymore. + val boundPort = env.conf.getInt("spark.executor.port", 0) + assert(boundPort != 0) + + // Start the CoarseGrainedExecutorBackend actor. val sparkHostPort = hostname + ":" + boundPort - actorSystem.actorOf( + env.actorSystem.actorOf( Props(classOf[CoarseGrainedExecutorBackend], - driverUrl, executorId, sparkHostPort, cores, props, actorSystem), + driverUrl, executorId, sparkHostPort, cores, env), name = "Executor") workerUrl.foreach { url => - actorSystem.actorOf(Props(classOf[WorkerWatcher], url), name = "WorkerWatcher") + env.actorSystem.actorOf(Props(classOf[WorkerWatcher], url), name = "WorkerWatcher") } - actorSystem.awaitTermination() + env.actorSystem.awaitTermination() } } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 5fa584591d935..0f99cd9f3b08a 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -26,7 +26,7 @@ import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.util.control.NonFatal -import akka.actor.{Props, ActorSystem} +import akka.actor.Props import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil @@ -42,10 +42,8 @@ import org.apache.spark.util.{SparkUncaughtExceptionHandler, AkkaUtils, Utils} private[spark] class Executor( executorId: String, slaveHostname: String, - properties: Seq[(String, String)], - numCores: Int, - isLocal: Boolean = false, - actorSystem: ActorSystem = null) + env: SparkEnv, + isLocal: Boolean = false) extends Logging { // Application dependencies (added through SparkContext) that we've fetched so far on this node. @@ -55,6 +53,8 @@ private[spark] class Executor( private val EMPTY_BYTE_BUFFER = ByteBuffer.wrap(new Array[Byte](0)) + private val conf = env.conf + @volatile private var isStopped = false // No ip or host:port - just hostname @@ -65,10 +65,6 @@ private[spark] class Executor( // Make sure the local hostname we report matches the cluster scheduler's name for this host Utils.setCustomHostname(slaveHostname) - // Set spark.* properties from executor arg - val conf = new SparkConf(true) - conf.setAll(properties) - if (!isLocal) { // Setup an uncaught exception handler for non-local mode. // Make any thread terminations due to uncaught exceptions kill the entire @@ -77,21 +73,11 @@ private[spark] class Executor( } val executorSource = new ExecutorSource(this, executorId) - - // Initialize Spark environment (using system properties read above) conf.set("spark.executor.id", executorId) - private val env = { - if (!isLocal) { - val port = conf.getInt("spark.executor.port", 0) - val _env = SparkEnv.createExecutorEnv( - conf, executorId, slaveHostname, port, numCores, isLocal, actorSystem) - SparkEnv.set(_env) - _env.metricsSystem.registerSource(executorSource) - _env.blockManager.initialize(conf.getAppId) - _env - } else { - SparkEnv.get - } + + if (!isLocal) { + env.metricsSystem.registerSource(executorSource) + env.blockManager.initialize(conf.getAppId) } // Create an actor for receiving RPCs from the driver @@ -145,6 +131,8 @@ private[spark] class Executor( } } + private def gcTime = ManagementFactory.getGarbageCollectorMXBeans.map(_.getCollectionTime).sum + class TaskRunner( execBackend: ExecutorBackend, val taskId: Long, taskName: String, serializedTask: ByteBuffer) extends Runnable { @@ -152,6 +140,7 @@ private[spark] class Executor( @volatile private var killed = false @volatile var task: Task[Any] = _ @volatile var attemptedTask: Option[Task[Any]] = None + @volatile var startGCTime: Long = _ def kill(interruptThread: Boolean) { logInfo(s"Executor is trying to kill $taskName (TID $taskId)") @@ -164,15 +153,13 @@ private[spark] class Executor( override def run() { val deserializeStartTime = System.currentTimeMillis() Thread.currentThread.setContextClassLoader(replClassLoader) - val ser = SparkEnv.get.closureSerializer.newInstance() + val ser = env.closureSerializer.newInstance() logInfo(s"Running $taskName (TID $taskId)") execBackend.statusUpdate(taskId, TaskState.RUNNING, EMPTY_BYTE_BUFFER) var taskStart: Long = 0 - def gcTime = ManagementFactory.getGarbageCollectorMXBeans.map(_.getCollectionTime).sum - val startGCTime = gcTime + startGCTime = gcTime try { - Accumulators.clear() val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(serializedTask) updateDependencies(taskFiles, taskJars) task = ser.deserialize[Task[Any]](taskBytes, Thread.currentThread.getContextClassLoader) @@ -201,7 +188,7 @@ private[spark] class Executor( throw new TaskKilledException } - val resultSer = SparkEnv.get.serializer.newInstance() + val resultSer = env.serializer.newInstance() val beforeSerialization = System.currentTimeMillis() val valueBytes = resultSer.serialize(value) val afterSerialization = System.currentTimeMillis() @@ -278,6 +265,8 @@ private[spark] class Executor( env.shuffleMemoryManager.releaseMemoryForThisThread() // Release memory used by this thread for unrolling blocks env.blockManager.memoryStore.releaseUnrollMemoryForThisThread() + // Release memory used by this thread for accumulators + Accumulators.clear() runningTasks.remove(taskId) } } @@ -334,7 +323,7 @@ private[spark] class Executor( * SparkContext. Also adds any new JARs we fetched to the class loader. */ private def updateDependencies(newFiles: HashMap[String, Long], newJars: HashMap[String, Long]) { - val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + lazy val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) synchronized { // Fetch missing dependencies for ((name, timestamp) <- newFiles if currentFiles.getOrElse(name, -1L) < timestamp) { @@ -375,10 +364,13 @@ private[spark] class Executor( while (!isStopped) { val tasksMetrics = new ArrayBuffer[(Long, TaskMetrics)]() + val curGCTime = gcTime + for (taskRunner <- runningTasks.values()) { if (!taskRunner.attemptedTask.isEmpty) { Option(taskRunner.task).flatMap(_.metrics).foreach { metrics => metrics.updateShuffleReadMetrics + metrics.jvmGCTime = curGCTime - taskRunner.startGCTime if (isLocal) { // JobProgressListener will hold an reference of it during // onExecutorMetricsUpdate(), then JobProgressListener can not see diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index f15e6bc33fb41..a098d07bd8659 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -25,7 +25,7 @@ import org.apache.mesos.protobuf.ByteString import org.apache.mesos.{Executor => MesosExecutor, ExecutorDriver, MesosExecutorDriver, MesosNativeLibrary} import org.apache.mesos.Protos.{TaskStatus => MesosTaskStatus, _} -import org.apache.spark.{Logging, TaskState} +import org.apache.spark.{Logging, TaskState, SparkConf, SparkEnv} import org.apache.spark.TaskState.TaskState import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.{SignalLogger, Utils} @@ -64,11 +64,15 @@ private[spark] class MesosExecutorBackend this.driver = driver val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray) ++ Seq[(String, String)](("spark.app.id", frameworkInfo.getId.getValue)) + val conf = new SparkConf(loadDefaults = true).setAll(properties) + val port = conf.getInt("spark.executor.port", 0) + val env = SparkEnv.createExecutorEnv( + conf, executorId, slaveInfo.getHostname, port, cpusPerTask, isLocal = false) + executor = new Executor( executorId, slaveInfo.getHostname, - properties, - cpusPerTask) + env) } override def launchTask(d: ExecutorDriver, taskInfo: TaskInfo) { diff --git a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala index 89b29af2000c8..c219d21fbefa9 100644 --- a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala +++ b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala @@ -21,6 +21,7 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.io.{BytesWritable, LongWritable} import org.apache.hadoop.mapreduce.lib.input.FileInputFormat import org.apache.hadoop.mapreduce.{InputSplit, JobContext, RecordReader, TaskAttemptContext} +import org.apache.spark.deploy.SparkHadoopUtil /** * Custom Input Format for reading and splitting flat binary files that contain records, @@ -33,7 +34,7 @@ private[spark] object FixedLengthBinaryInputFormat { /** Retrieves the record length property from a Hadoop configuration */ def getRecordLength(context: JobContext): Int = { - context.getConfiguration.get(RECORD_LENGTH_PROPERTY).toInt + SparkHadoopUtil.get.getConfigurationFromJobContext(context).get(RECORD_LENGTH_PROPERTY).toInt } } diff --git a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala index 36a1e5d475f46..67a96925da019 100644 --- a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala +++ b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala @@ -24,6 +24,7 @@ import org.apache.hadoop.io.compress.CompressionCodecFactory import org.apache.hadoop.io.{BytesWritable, LongWritable} import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext} import org.apache.hadoop.mapreduce.lib.input.FileSplit +import org.apache.spark.deploy.SparkHadoopUtil /** * FixedLengthBinaryRecordReader is returned by FixedLengthBinaryInputFormat. @@ -82,7 +83,7 @@ private[spark] class FixedLengthBinaryRecordReader // the actual file we will be reading from val file = fileSplit.getPath // job configuration - val job = context.getConfiguration + val job = SparkHadoopUtil.get.getConfigurationFromJobContext(context) // check compression val codec = new CompressionCodecFactory(job).getCodec(file) if (codec != null) { diff --git a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala index 457472547fcbb..593a62b3e3b32 100644 --- a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala +++ b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala @@ -28,6 +28,7 @@ import org.apache.hadoop.mapreduce.{InputSplit, JobContext, RecordReader, TaskAt import org.apache.hadoop.mapreduce.lib.input.{CombineFileInputFormat, CombineFileRecordReader, CombineFileSplit} import org.apache.spark.annotation.Experimental +import org.apache.spark.deploy.SparkHadoopUtil /** * A general format for reading whole files in as streams, byte arrays, @@ -145,7 +146,8 @@ class PortableDataStream( private val confBytes = { val baos = new ByteArrayOutputStream() - context.getConfiguration.write(new DataOutputStream(baos)) + SparkHadoopUtil.get.getConfigurationFromJobContext(context). + write(new DataOutputStream(baos)) baos.toByteArray } diff --git a/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala b/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala index d3601cca832b2..aaef7c74eea33 100644 --- a/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala +++ b/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala @@ -19,7 +19,6 @@ package org.apache.spark.input import scala.collection.JavaConversions._ -import org.apache.hadoop.conf.{Configuration, Configurable} import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.InputSplit import org.apache.hadoop.mapreduce.JobContext @@ -38,18 +37,13 @@ private[spark] class WholeTextFileInputFormat override protected def isSplitable(context: JobContext, file: Path): Boolean = false - private var conf: Configuration = _ - def setConf(c: Configuration) { - conf = c - } - def getConf: Configuration = conf - override def createRecordReader( split: InputSplit, context: TaskAttemptContext): RecordReader[String, String] = { - val reader = new WholeCombineFileRecordReader(split, context) - reader.setConf(conf) + val reader = + new ConfigurableCombineFileRecordReader(split, context, classOf[WholeTextFileRecordReader]) + reader.setConf(getConf) reader } diff --git a/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala b/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala index 6d59b24eb0596..31bde8a78f3c6 100644 --- a/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala +++ b/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala @@ -17,7 +17,7 @@ package org.apache.spark.input -import org.apache.hadoop.conf.{Configuration, Configurable} +import org.apache.hadoop.conf.{Configuration, Configurable => HConfigurable} import com.google.common.io.{ByteStreams, Closeables} import org.apache.hadoop.io.Text @@ -26,6 +26,19 @@ import org.apache.hadoop.mapreduce.InputSplit import org.apache.hadoop.mapreduce.lib.input.{CombineFileSplit, CombineFileRecordReader} import org.apache.hadoop.mapreduce.RecordReader import org.apache.hadoop.mapreduce.TaskAttemptContext +import org.apache.spark.deploy.SparkHadoopUtil + + +/** + * A trait to implement [[org.apache.hadoop.conf.Configurable Configurable]] interface. + */ +private[spark] trait Configurable extends HConfigurable { + private var conf: Configuration = _ + def setConf(c: Configuration) { + conf = c + } + def getConf: Configuration = conf +} /** * A [[org.apache.hadoop.mapreduce.RecordReader RecordReader]] for reading a single whole text file @@ -38,14 +51,9 @@ private[spark] class WholeTextFileRecordReader( index: Integer) extends RecordReader[String, String] with Configurable { - private var conf: Configuration = _ - def setConf(c: Configuration) { - conf = c - } - def getConf: Configuration = conf - private[this] val path = split.getPath(index) - private[this] val fs = path.getFileSystem(context.getConfiguration) + private[this] val fs = path.getFileSystem( + SparkHadoopUtil.get.getConfigurationFromJobContext(context)) // True means the current file has been processed, then skip it. private[this] var processed = false @@ -87,29 +95,24 @@ private[spark] class WholeTextFileRecordReader( /** - * A [[org.apache.hadoop.mapreduce.RecordReader RecordReader]] for reading a single whole text file - * out in a key-value pair, where the key is the file path and the value is the entire content of - * the file. + * A [[org.apache.hadoop.mapreduce.lib.input.CombineFileRecordReader CombineFileRecordReader]] + * that can pass Hadoop Configuration to [[org.apache.hadoop.conf.Configurable Configurable]] + * RecordReaders. */ -private[spark] class WholeCombineFileRecordReader( +private[spark] class ConfigurableCombineFileRecordReader[K, V]( split: InputSplit, - context: TaskAttemptContext) - extends CombineFileRecordReader[String, String]( + context: TaskAttemptContext, + recordReaderClass: Class[_ <: RecordReader[K, V] with HConfigurable]) + extends CombineFileRecordReader[K, V]( split.asInstanceOf[CombineFileSplit], context, - classOf[WholeTextFileRecordReader] + recordReaderClass ) with Configurable { - private var conf: Configuration = _ - def setConf(c: Configuration) { - conf = c - } - def getConf: Configuration = conf - override def initNextRecordReader(): Boolean = { val r = super.initNextRecordReader() if (r) { - this.curReader.asInstanceOf[WholeTextFileRecordReader].setConf(conf) + this.curReader.asInstanceOf[HConfigurable].setConf(getConf) } r } diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 1ac7f4e448eb1..f856890d279f4 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -21,11 +21,12 @@ import java.io.{InputStream, OutputStream} import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} import net.jpountz.lz4.{LZ4BlockInputStream, LZ4BlockOutputStream} -import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} +import org.xerial.snappy.{Snappy, SnappyInputStream, SnappyOutputStream} import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.Utils +import org.apache.spark.Logging /** * :: DeveloperApi :: @@ -44,25 +45,33 @@ trait CompressionCodec { def compressedInputStream(s: InputStream): InputStream } - private[spark] object CompressionCodec { + private val configKey = "spark.io.compression.codec" private val shortCompressionCodecNames = Map( "lz4" -> classOf[LZ4CompressionCodec].getName, "lzf" -> classOf[LZFCompressionCodec].getName, "snappy" -> classOf[SnappyCompressionCodec].getName) def createCodec(conf: SparkConf): CompressionCodec = { - createCodec(conf, conf.get("spark.io.compression.codec", DEFAULT_COMPRESSION_CODEC)) + createCodec(conf, conf.get(configKey, DEFAULT_COMPRESSION_CODEC)) } def createCodec(conf: SparkConf, codecName: String): CompressionCodec = { val codecClass = shortCompressionCodecNames.getOrElse(codecName.toLowerCase, codecName) - val ctor = Class.forName(codecClass, true, Utils.getContextOrSparkClassLoader) - .getConstructor(classOf[SparkConf]) - ctor.newInstance(conf).asInstanceOf[CompressionCodec] + val codec = try { + val ctor = Class.forName(codecClass, true, Utils.getContextOrSparkClassLoader) + .getConstructor(classOf[SparkConf]) + Some(ctor.newInstance(conf).asInstanceOf[CompressionCodec]) + } catch { + case e: ClassNotFoundException => None + case e: IllegalArgumentException => None + } + codec.getOrElse(throw new IllegalArgumentException(s"Codec [$codecName] is not available. " + + s"Consider setting $configKey=$FALLBACK_COMPRESSION_CODEC")) } + val FALLBACK_COMPRESSION_CODEC = "lzf" val DEFAULT_COMPRESSION_CODEC = "snappy" val ALL_COMPRESSION_CODECS = shortCompressionCodecNames.values.toSeq } @@ -120,6 +129,12 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { @DeveloperApi class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec { + try { + Snappy.getNativeLibraryVersion + } catch { + case e: Error => throw new IllegalArgumentException + } + override def compressedOutputStream(s: OutputStream): OutputStream = { val blockSize = conf.getInt("spark.io.compression.snappy.block.size", 32768) new SnappyOutputStream(s, blockSize) diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 5dd67b0cbf683..45633e3de01dd 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -76,22 +76,36 @@ private[spark] class MetricsSystem private ( private val sources = new mutable.ArrayBuffer[Source] private val registry = new MetricRegistry() + private var running: Boolean = false + // Treat MetricsServlet as a special sink as it should be exposed to add handlers to web ui private var metricsServlet: Option[MetricsServlet] = None - /** Get any UI handlers used by this metrics system. */ - def getServletHandlers = metricsServlet.map(_.getHandlers).getOrElse(Array()) + /** + * Get any UI handlers used by this metrics system; can only be called after start(). + */ + def getServletHandlers = { + require(running, "Can only call getServletHandlers on a running MetricsSystem") + metricsServlet.map(_.getHandlers).getOrElse(Array()) + } metricsConfig.initialize() def start() { + require(!running, "Attempting to start a MetricsSystem that is already running") + running = true registerSources() registerSinks() sinks.foreach(_.start) } def stop() { - sinks.foreach(_.stop) + if (running) { + sinks.foreach(_.stop) + } else { + logWarning("Stopping a MetricsSystem that is not running") + } + running = false } def report() { @@ -107,7 +121,7 @@ private[spark] class MetricsSystem private ( * @return An unique metric name for each combination of * application, executor/driver and metric source. */ - def buildRegistryName(source: Source): String = { + private[spark] def buildRegistryName(source: Source): String = { val appId = conf.getOption("spark.app.id") val executorId = conf.getOption("spark.executor.id") val defaultName = MetricRegistry.name(source.sourceName) @@ -144,7 +158,7 @@ private[spark] class MetricsSystem private ( }) } - def registerSources() { + private def registerSources() { val instConfig = metricsConfig.getInstance(instance) val sourceConfigs = metricsConfig.subProperties(instConfig, MetricsSystem.SOURCE_REGEX) @@ -160,7 +174,7 @@ private[spark] class MetricsSystem private ( } } - def registerSinks() { + private def registerSinks() { val instConfig = metricsConfig.getInstance(instance) val sinkConfigs = metricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index 0027cbb0ff1fb..3f0950dae1f24 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -60,7 +60,7 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage } transportContext = new TransportContext(transportConf, rpcHandler) clientFactory = transportContext.createClientFactory(bootstrap.toList) - server = transportContext.createServer() + server = transportContext.createServer(conf.getInt("spark.blockManager.port", 0)) appId = conf.getAppId logInfo("Server created on " + server.getPort) } diff --git a/core/src/main/scala/org/apache/spark/network/netty/SparkTransportConf.scala b/core/src/main/scala/org/apache/spark/network/netty/SparkTransportConf.scala index ce4225cae6d88..cef203006d685 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/SparkTransportConf.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/SparkTransportConf.scala @@ -20,7 +20,24 @@ package org.apache.spark.network.netty import org.apache.spark.SparkConf import org.apache.spark.network.util.{TransportConf, ConfigProvider} +/** + * Provides a utility for transforming from a SparkConf inside a Spark JVM (e.g., Executor, + * Driver, or a standalone shuffle service) into a TransportConf with details on our environment + * like the number of cores that are allocated to this JVM. + */ object SparkTransportConf { + /** + * Specifies an upper bound on the number of Netty threads that Spark requires by default. + * In practice, only 2-4 cores should be required to transfer roughly 10 Gb/s, and each core + * that we use will have an initial overhead of roughly 32 MB of off-heap memory, which comes + * at a premium. + * + * Thus, this value should still retain maximum throughput and reduce wasted off-heap memory + * allocation. It can be overridden by setting the number of serverThreads and clientThreads + * manually in Spark's configuration. + */ + private val MAX_DEFAULT_NETTY_THREADS = 8 + /** * Utility for creating a [[TransportConf]] from a [[SparkConf]]. * @param numUsableCores if nonzero, this will restrict the server and client threads to only @@ -29,15 +46,28 @@ object SparkTransportConf { */ def fromSparkConf(_conf: SparkConf, numUsableCores: Int = 0): TransportConf = { val conf = _conf.clone - if (numUsableCores > 0) { - // Only set if serverThreads/clientThreads not already set. - conf.set("spark.shuffle.io.serverThreads", - conf.get("spark.shuffle.io.serverThreads", numUsableCores.toString)) - conf.set("spark.shuffle.io.clientThreads", - conf.get("spark.shuffle.io.clientThreads", numUsableCores.toString)) - } + + // Specify thread configuration based on our JVM's allocation of cores (rather than necessarily + // assuming we have all the machine's cores). + // NB: Only set if serverThreads/clientThreads not already set. + val numThreads = defaultNumThreads(numUsableCores) + conf.set("spark.shuffle.io.serverThreads", + conf.get("spark.shuffle.io.serverThreads", numThreads.toString)) + conf.set("spark.shuffle.io.clientThreads", + conf.get("spark.shuffle.io.clientThreads", numThreads.toString)) + new TransportConf(new ConfigProvider { override def get(name: String): String = conf.get(name) }) } + + /** + * Returns the default number of threads for both the Netty client and server thread pools. + * If numUsableCores is 0, we will use Runtime get an approximate number of available cores. + */ + private def defaultNumThreads(numUsableCores: Int): Int = { + val availableCores = + if (numUsableCores > 0) numUsableCores else Runtime.getRuntime.availableProcessors() + math.min(availableCores, MAX_DEFAULT_NETTY_THREADS) + } } diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index df4b085d2251e..243b71c980864 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -83,9 +83,21 @@ private[nio] class ConnectionManager( private val ackTimeout = conf.getInt("spark.core.connection.ack.wait.timeout", 60) + // Get the thread counts from the Spark Configuration. + // + // Even though the ThreadPoolExecutor constructor takes both a minimum and maximum value, + // we only query for the minimum value because we are using LinkedBlockingDeque. + // + // The JavaDoc for ThreadPoolExecutor points out that when using a LinkedBlockingDeque (which is + // an unbounded queue) no more than corePoolSize threads will ever be created, so only the "min" + // parameter is necessary. + private val handlerThreadCount = conf.getInt("spark.core.connection.handler.threads.min", 20) + private val ioThreadCount = conf.getInt("spark.core.connection.io.threads.min", 4) + private val connectThreadCount = conf.getInt("spark.core.connection.connect.threads.min", 1) + private val handleMessageExecutor = new ThreadPoolExecutor( - conf.getInt("spark.core.connection.handler.threads.min", 20), - conf.getInt("spark.core.connection.handler.threads.max", 60), + handlerThreadCount, + handlerThreadCount, conf.getInt("spark.core.connection.handler.threads.keepalive", 60), TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable](), Utils.namedThreadFactory("handle-message-executor")) { @@ -96,12 +108,11 @@ private[nio] class ConnectionManager( logError("Error in handleMessageExecutor is not handled properly", t) } } - } private val handleReadWriteExecutor = new ThreadPoolExecutor( - conf.getInt("spark.core.connection.io.threads.min", 4), - conf.getInt("spark.core.connection.io.threads.max", 32), + ioThreadCount, + ioThreadCount, conf.getInt("spark.core.connection.io.threads.keepalive", 60), TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable](), Utils.namedThreadFactory("handle-read-write-executor")) { @@ -112,14 +123,13 @@ private[nio] class ConnectionManager( logError("Error in handleReadWriteExecutor is not handled properly", t) } } - } // Use a different, yet smaller, thread pool - infrequently used with very short lived tasks : // which should be executed asap private val handleConnectExecutor = new ThreadPoolExecutor( - conf.getInt("spark.core.connection.connect.threads.min", 1), - conf.getInt("spark.core.connection.connect.threads.max", 8), + connectThreadCount, + connectThreadCount, conf.getInt("spark.core.connection.connect.threads.keepalive", 60), TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable](), Utils.namedThreadFactory("handle-connect-executor")) { @@ -130,7 +140,6 @@ private[nio] class ConnectionManager( logError("Error in handleConnectExecutor is not handled properly", t) } } - } private val serverChannel = ServerSocketChannel.open() diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala index 436dbed1730bc..5ad73c3d27f47 100644 --- a/core/src/main/scala/org/apache/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -27,8 +27,8 @@ package org.apache * contains operations available only on RDDs of Doubles; and * [[org.apache.spark.rdd.SequenceFileRDDFunctions]] contains operations available on RDDs that can * be saved as SequenceFiles. These operations are automatically available on any RDD of the right - * type (e.g. RDD[(Int, Int)] through implicit conversions when you - * `import org.apache.spark.SparkContext._`. + * type (e.g. RDD[(Int, Int)] through implicit conversions except `saveAsSequenceFile`. You need to + * `import org.apache.spark.SparkContext._` to make `saveAsSequenceFile` work. * * Java programmers should reference the [[org.apache.spark.api.java]] package * for Spark programming APIs in Java. diff --git a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala index 9f9f10b7ebc3a..646df283ac069 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -27,7 +27,6 @@ import org.apache.spark.{ComplexFutureAction, FutureAction, Logging} /** * A set of asynchronous RDD actions available through an implicit conversion. - * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. */ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Logging { diff --git a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala index 6e66ddbdef788..1f755db485812 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala @@ -24,12 +24,12 @@ import org.apache.spark.input.StreamFileInputFormat import org.apache.spark.{ Partition, SparkContext } private[spark] class BinaryFileRDD[T]( - sc: SparkContext, - inputFormatClass: Class[_ <: StreamFileInputFormat[T]], - keyClass: Class[String], - valueClass: Class[T], - @transient conf: Configuration, - minPartitions: Int) + sc: SparkContext, + inputFormatClass: Class[_ <: StreamFileInputFormat[T]], + keyClass: Class[String], + valueClass: Class[T], + @transient conf: Configuration, + minPartitions: Int) extends NewHadoopRDD[String, T](sc, inputFormatClass, keyClass, valueClass, conf) { override def getPartitions: Array[Partition] = { diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index ffc0a8a6d67eb..70edf191d928a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -60,7 +60,7 @@ private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep] * A RDD that cogroups its parents. For each key k in parent RDDs, the resulting RDD contains a * tuple with the list of values for that key. * - * Note: This is an internal API. We recommend users use RDD.coGroup(...) instead of + * Note: This is an internal API. We recommend users use RDD.cogroup(...) instead of * instantiating this directly. * @param rdds parent RDDs. @@ -70,8 +70,8 @@ private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep] class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: Partitioner) extends RDD[(K, Array[Iterable[_]])](rdds.head.context, Nil) { - // For example, `(k, a) cogroup (k, b)` produces k -> Seq(ArrayBuffer as, ArrayBuffer bs). - // Each ArrayBuffer is represented as a CoGroup, and the resulting Seq as a CoGroupCombiner. + // For example, `(k, a) cogroup (k, b)` produces k -> Array(ArrayBuffer as, ArrayBuffer bs). + // Each ArrayBuffer is represented as a CoGroup, and the resulting Array as a CoGroupCombiner. // CoGroupValue is the intermediate state of each value before being merged in compute. private type CoGroup = CompactBuffer[Any] private type CoGroupValue = (Any, Int) // Int is dependency number diff --git a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala index 9fab1d78abb04..b073eba8a1574 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -35,11 +35,10 @@ import org.apache.spark.util.Utils * @param preferredLocation the preferred location for this partition */ private[spark] case class CoalescedRDDPartition( - index: Int, - @transient rdd: RDD[_], - parentsIndices: Array[Int], - @transient preferredLocation: String = "" - ) extends Partition { + index: Int, + @transient rdd: RDD[_], + parentsIndices: Array[Int], + @transient preferredLocation: Option[String] = None) extends Partition { var parents: Seq[Partition] = parentsIndices.map(rdd.partitions(_)) @throws(classOf[IOException]) @@ -55,9 +54,10 @@ private[spark] case class CoalescedRDDPartition( * @return locality of this coalesced partition between 0 and 1 */ def localFraction: Double = { - val loc = parents.count(p => - rdd.context.getPreferredLocs(rdd, p.index).map(tl => tl.host).contains(preferredLocation)) - + val loc = parents.count { p => + val parentPreferredLocations = rdd.context.getPreferredLocs(rdd, p.index).map(_.host) + preferredLocation.exists(parentPreferredLocations.contains) + } if (parents.size == 0) 0.0 else (loc.toDouble / parents.size.toDouble) } } @@ -73,9 +73,9 @@ private[spark] case class CoalescedRDDPartition( * @param balanceSlack used to trade-off balance and locality. 1.0 is all locality, 0 is all balance */ private[spark] class CoalescedRDD[T: ClassTag]( - @transient var prev: RDD[T], - maxPartitions: Int, - balanceSlack: Double = 0.10) + @transient var prev: RDD[T], + maxPartitions: Int, + balanceSlack: Double = 0.10) extends RDD[T](prev.context, Nil) { // Nil since we implement getDependencies override def getPartitions: Array[Partition] = { @@ -113,7 +113,7 @@ private[spark] class CoalescedRDD[T: ClassTag]( * @return the machine most preferred by split */ override def getPreferredLocations(partition: Partition): Seq[String] = { - List(partition.asInstanceOf[CoalescedRDDPartition].preferredLocation) + partition.asInstanceOf[CoalescedRDDPartition].preferredLocation.toSeq } } @@ -147,7 +147,7 @@ private[spark] class CoalescedRDD[T: ClassTag]( * */ -private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) { +private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) { def compare(o1: PartitionGroup, o2: PartitionGroup): Boolean = o1.size < o2.size def compare(o1: Option[PartitionGroup], o2: Option[PartitionGroup]): Boolean = @@ -341,8 +341,14 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanc } } -private[spark] case class PartitionGroup(prefLoc: String = "") { +private case class PartitionGroup(prefLoc: Option[String] = None) { var arr = mutable.ArrayBuffer[Partition]() - def size = arr.size } + +private object PartitionGroup { + def apply(prefLoc: String): PartitionGroup = { + require(prefLoc != "", "Preferred location must not be empty") + PartitionGroup(Some(prefLoc)) + } +} diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index e0494ee39657c..e66f83bb34e30 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -27,7 +27,6 @@ import org.apache.spark.util.StatCounter /** * Extra functions available on RDDs of Doubles through an implicit conversion. - * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. */ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { /** Add up the elements in this RDD. */ diff --git a/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala deleted file mode 100644 index d8f87d4e3690e..0000000000000 --- a/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.rdd - -import scala.reflect.ClassTag - -import org.apache.spark.{Partition, TaskContext} - -private[spark] -class FlatMappedRDD[U: ClassTag, T: ClassTag]( - prev: RDD[T], - f: T => TraversableOnce[U]) - extends RDD[U](prev) { - - override def getPartitions: Array[Partition] = firstParent[T].partitions - - override def compute(split: Partition, context: TaskContext) = - firstParent[T].iterator(split, context).flatMap(f) -} diff --git a/core/src/main/scala/org/apache/spark/rdd/FlatMappedValuesRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FlatMappedValuesRDD.scala deleted file mode 100644 index 7c9023f62d3b6..0000000000000 --- a/core/src/main/scala/org/apache/spark/rdd/FlatMappedValuesRDD.scala +++ /dev/null @@ -1,35 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.rdd - -import org.apache.spark.{Partition, TaskContext} - -private[spark] -class FlatMappedValuesRDD[K, V, U](prev: RDD[_ <: Product2[K, V]], f: V => TraversableOnce[U]) - extends RDD[(K, U)](prev) { - - override def getPartitions = firstParent[Product2[K, V]].partitions - - override val partitioner = firstParent[Product2[K, V]].partitioner - - override def compute(split: Partition, context: TaskContext) = { - firstParent[Product2[K, V]].iterator(split, context).flatMap { case Product2(k, v) => - f(v).map(x => (k, x)) - } - } -} diff --git a/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala deleted file mode 100644 index f6463fa715a71..0000000000000 --- a/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.rdd - -import scala.reflect.ClassTag - -import org.apache.spark.{Partition, TaskContext} - -private[spark] class GlommedRDD[T: ClassTag](prev: RDD[T]) - extends RDD[Array[T]](prev) { - - override def getPartitions: Array[Partition] = firstParent[T].partitions - - override def compute(split: Partition, context: TaskContext) = - Array(firstParent[T].iterator(split, context).toArray).iterator -} diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index a157e36e2286e..0001c2329c83a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -131,7 +131,7 @@ class HadoopRDD[K, V]( // used to build JobTracker ID private val createTime = new Date() - private val shouldCloneJobConf = sc.conf.get("spark.hadoop.cloneConf", "false").toBoolean + private val shouldCloneJobConf = sc.conf.getBoolean("spark.hadoop.cloneConf", false) // Returns a JobConf that will be used on slaves to obtain input splits for Hadoop reads. protected def getJobConf(): JobConf = { diff --git a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala index 0e38f224ac81d..642a12c1edf6c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala @@ -21,8 +21,11 @@ import java.sql.{Connection, ResultSet} import scala.reflect.ClassTag -import org.apache.spark.{Logging, Partition, SparkContext, TaskContext} +import org.apache.spark.api.java.JavaSparkContext.fakeClassTag +import org.apache.spark.api.java.function.{Function => JFunction} +import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.util.NextIterator +import org.apache.spark.{Logging, Partition, SparkContext, TaskContext} private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) extends Partition { override def index = idx @@ -125,5 +128,82 @@ object JdbcRDD { def resultSetToObjectArray(rs: ResultSet): Array[Object] = { Array.tabulate[Object](rs.getMetaData.getColumnCount)(i => rs.getObject(i + 1)) } -} + trait ConnectionFactory extends Serializable { + @throws[Exception] + def getConnection: Connection + } + + /** + * Create an RDD that executes an SQL query on a JDBC connection and reads results. + * For usage example, see test case JavaAPISuite.testJavaJdbcRDD. + * + * @param connectionFactory a factory that returns an open Connection. + * The RDD takes care of closing the connection. + * @param sql the text of the query. + * The query must contain two ? placeholders for parameters used to partition the results. + * E.g. "select title, author from books where ? <= id and id <= ?" + * @param lowerBound the minimum value of the first placeholder + * @param upperBound the maximum value of the second placeholder + * The lower and upper bounds are inclusive. + * @param numPartitions the number of partitions. + * Given a lowerBound of 1, an upperBound of 20, and a numPartitions of 2, + * the query would be executed twice, once with (1, 10) and once with (11, 20) + * @param mapRow a function from a ResultSet to a single row of the desired result type(s). + * This should only call getInt, getString, etc; the RDD takes care of calling next. + * The default maps a ResultSet to an array of Object. + */ + def create[T]( + sc: JavaSparkContext, + connectionFactory: ConnectionFactory, + sql: String, + lowerBound: Long, + upperBound: Long, + numPartitions: Int, + mapRow: JFunction[ResultSet, T]): JavaRDD[T] = { + + val jdbcRDD = new JdbcRDD[T]( + sc.sc, + () => connectionFactory.getConnection, + sql, + lowerBound, + upperBound, + numPartitions, + (resultSet: ResultSet) => mapRow.call(resultSet))(fakeClassTag) + + new JavaRDD[T](jdbcRDD)(fakeClassTag) + } + + /** + * Create an RDD that executes an SQL query on a JDBC connection and reads results. Each row is + * converted into a `Object` array. For usage example, see test case JavaAPISuite.testJavaJdbcRDD. + * + * @param connectionFactory a factory that returns an open Connection. + * The RDD takes care of closing the connection. + * @param sql the text of the query. + * The query must contain two ? placeholders for parameters used to partition the results. + * E.g. "select title, author from books where ? <= id and id <= ?" + * @param lowerBound the minimum value of the first placeholder + * @param upperBound the maximum value of the second placeholder + * The lower and upper bounds are inclusive. + * @param numPartitions the number of partitions. + * Given a lowerBound of 1, an upperBound of 20, and a numPartitions of 2, + * the query would be executed twice, once with (1, 10) and once with (11, 20) + */ + def create( + sc: JavaSparkContext, + connectionFactory: ConnectionFactory, + sql: String, + lowerBound: Long, + upperBound: Long, + numPartitions: Int): JavaRDD[Array[Object]] = { + + val mapRow = new JFunction[ResultSet, Array[Object]] { + override def call(resultSet: ResultSet): Array[Object] = { + resultSetToObjectArray(resultSet) + } + } + + create(sc, connectionFactory, sql, lowerBound, upperBound, numPartitions, mapRow) + } +} diff --git a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala index d0dbfef35d03c..144f679a59460 100644 --- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala @@ -24,10 +24,9 @@ import org.apache.spark.annotation.DeveloperApi /** * Extra functions available on RDDs of (key, value) pairs where the key is sortable through - * an implicit conversion. Import `org.apache.spark.SparkContext._` at the top of your program to - * use these functions. They will work with any key type `K` that has an implicit `Ordering[K]` in - * scope. Ordering objects already exist for all of the standard primitive types. Users can also - * define their own orderings for custom types, or to override the default ordering. The implicit + * an implicit conversion. They will work with any key type `K` that has an implicit `Ordering[K]` + * in scope. Ordering objects already exist for all of the standard primitive types. Users can also + * define their own orderings for custom types, or to override the default ordering. The implicit * ordering that is in the closest scope will be used. * * {{{ diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 8c2c959e73bb6..4469c89e6bb1c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -37,7 +37,6 @@ RecordWriter => NewRecordWriter} import org.apache.spark._ import org.apache.spark.Partitioner.defaultPartitioner -import org.apache.spark.SparkContext._ import org.apache.spark.annotation.Experimental import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.executor.{DataWriteMethod, OutputMetrics} @@ -50,7 +49,6 @@ import org.apache.spark.util.random.StratifiedSamplingUtils /** * Extra functions available on RDDs of (key, value) pairs through an implicit conversion. - * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. */ class PairRDDFunctions[K, V](self: RDD[(K, V)]) (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) @@ -86,7 +84,10 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) throw new SparkException("Default partitioner cannot partition array keys.") } } - val aggregator = new Aggregator[K, V, C](createCombiner, mergeValue, mergeCombiners) + val aggregator = new Aggregator[K, V, C]( + self.context.clean(createCombiner), + self.context.clean(mergeValue), + self.context.clean(mergeCombiners)) if (self.partitioner == Some(partitioner)) { self.mapPartitions(iter => { val context = TaskContext.get() @@ -122,11 +123,11 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) def aggregateByKey[U: ClassTag](zeroValue: U, partitioner: Partitioner)(seqOp: (U, V) => U, combOp: (U, U) => U): RDD[(K, U)] = { // Serialize the zero value to a byte array so that we can get a new clone of it on each key - val zeroBuffer = SparkEnv.get.closureSerializer.newInstance().serialize(zeroValue) + val zeroBuffer = SparkEnv.get.serializer.newInstance().serialize(zeroValue) val zeroArray = new Array[Byte](zeroBuffer.limit) zeroBuffer.get(zeroArray) - lazy val cachedSerializer = SparkEnv.get.closureSerializer.newInstance() + lazy val cachedSerializer = SparkEnv.get.serializer.newInstance() val createZero = () => cachedSerializer.deserialize[U](ByteBuffer.wrap(zeroArray)) combineByKey[U]((v: V) => seqOp(createZero(), v), seqOp, combOp, partitioner) @@ -167,12 +168,12 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) */ def foldByKey(zeroValue: V, partitioner: Partitioner)(func: (V, V) => V): RDD[(K, V)] = { // Serialize the zero value to a byte array so that we can get a new clone of it on each key - val zeroBuffer = SparkEnv.get.closureSerializer.newInstance().serialize(zeroValue) + val zeroBuffer = SparkEnv.get.serializer.newInstance().serialize(zeroValue) val zeroArray = new Array[Byte](zeroBuffer.limit) zeroBuffer.get(zeroArray) // When deserializing, use a lazy val to create just one instance of the serializer per task - lazy val cachedSerializer = SparkEnv.get.closureSerializer.newInstance() + lazy val cachedSerializer = SparkEnv.get.serializer.newInstance() val createZero = () => cachedSerializer.deserialize[V](ByteBuffer.wrap(zeroArray)) combineByKey[V]((v: V) => func(createZero(), v), func, func, partitioner) @@ -482,7 +483,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) */ def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = { this.cogroup(other, partitioner).flatMapValues( pair => - for (v <- pair._1; w <- pair._2) yield (v, w) + for (v <- pair._1.iterator; w <- pair._2.iterator) yield (v, w) ) } @@ -495,9 +496,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) def leftOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, Option[W]))] = { this.cogroup(other, partitioner).flatMapValues { pair => if (pair._2.isEmpty) { - pair._1.map(v => (v, None)) + pair._1.iterator.map(v => (v, None)) } else { - for (v <- pair._1; w <- pair._2) yield (v, Some(w)) + for (v <- pair._1.iterator; w <- pair._2.iterator) yield (v, Some(w)) } } } @@ -512,9 +513,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) : RDD[(K, (Option[V], W))] = { this.cogroup(other, partitioner).flatMapValues { pair => if (pair._1.isEmpty) { - pair._2.map(w => (None, w)) + pair._2.iterator.map(w => (None, w)) } else { - for (v <- pair._1; w <- pair._2) yield (Some(v), w) + for (v <- pair._1.iterator; w <- pair._2.iterator) yield (Some(v), w) } } } @@ -530,9 +531,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) def fullOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner) : RDD[(K, (Option[V], Option[W]))] = { this.cogroup(other, partitioner).flatMapValues { - case (vs, Seq()) => vs.map(v => (Some(v), None)) - case (Seq(), ws) => ws.map(w => (None, Some(w))) - case (vs, ws) => for (v <- vs; w <- ws) yield (Some(v), Some(w)) + case (vs, Seq()) => vs.iterator.map(v => (Some(v), None)) + case (Seq(), ws) => ws.iterator.map(w => (None, Some(w))) + case (vs, ws) => for (v <- vs.iterator; w <- ws.iterator) yield (Some(v), Some(w)) } } @@ -662,7 +663,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) */ def mapValues[U](f: V => U): RDD[(K, U)] = { val cleanF = self.context.clean(f) - new MappedValuesRDD(self, cleanF) + new MapPartitionsRDD[(K, U), (K, V)](self, + (context, pid, iter) => iter.map { case (k, v) => (k, cleanF(v)) }, + preservesPartitioning = true) } /** @@ -671,7 +674,11 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) */ def flatMapValues[U](f: V => TraversableOnce[U]): RDD[(K, U)] = { val cleanF = self.context.clean(f) - new FlatMappedValuesRDD(self, cleanF) + new MapPartitionsRDD[(K, U), (K, V)](self, + (context, pid, iter) => iter.flatMap { case (k, v) => + cleanF(v).map(x => (k, x)) + }, + preservesPartitioning = true) } /** diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala index 56ac7a69be0d3..ed79032893d33 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -63,7 +63,7 @@ private[spark] class PipedRDD[T: ClassTag]( /** * A FilenameFilter that accepts anything that isn't equal to the name passed in. - * @param name of file or directory to leave out + * @param filterName of file or directory to leave out */ class NotEqualsFileNameFilter(filterName: String) extends FilenameFilter { def accept(dir: File, name: String): Boolean = { diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index e4025bcf48db6..f47c2d1fcdcc7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -17,10 +17,11 @@ package org.apache.spark.rdd -import java.util.{Properties, Random} +import java.util.Random import scala.collection.{mutable, Map} import scala.collection.mutable.ArrayBuffer +import scala.language.implicitConversions import scala.reflect.{classTag, ClassTag} import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus @@ -28,20 +29,19 @@ import org.apache.hadoop.io.BytesWritable import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.io.NullWritable import org.apache.hadoop.io.Text +import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred.TextOutputFormat import org.apache.spark._ import org.apache.spark.Partitioner._ -import org.apache.spark.SparkContext._ import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD -import org.apache.spark.broadcast.Broadcast import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.CountEvaluator import org.apache.spark.partial.GroupedCountEvaluator import org.apache.spark.partial.PartialResult import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.{BoundedPriorityQueue, Utils, CallSite} +import org.apache.spark.util.{BoundedPriorityQueue, Utils} import org.apache.spark.util.collection.OpenHashMap import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler, BernoulliCellSampler, SamplingUtils} @@ -56,8 +56,9 @@ import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler, Bernoulli * Doubles; and * [[org.apache.spark.rdd.SequenceFileRDDFunctions]] contains operations available on RDDs that * can be saved as SequenceFiles. - * These operations are automatically available on any RDD of the right type (e.g. RDD[(Int, Int)] - * through implicit conversions when you `import org.apache.spark.SparkContext._`. + * All operations are automatically available on any RDD of the right type (e.g. RDD[(Int, Int)] + * through implicit conversions except `saveAsSequenceFile`. You need to + * `import org.apache.spark.SparkContext._` to make `saveAsSequenceFile` work. * * Internally, each RDD is characterized by five main properties: * @@ -268,19 +269,30 @@ abstract class RDD[T: ClassTag]( /** * Return a new RDD by applying a function to all elements of this RDD. */ - def map[U: ClassTag](f: T => U): RDD[U] = new MappedRDD(this, sc.clean(f)) + def map[U: ClassTag](f: T => U): RDD[U] = { + val cleanF = sc.clean(f) + new MapPartitionsRDD[U, T](this, (context, pid, iter) => iter.map(cleanF)) + } /** * Return a new RDD by first applying a function to all elements of this * RDD, and then flattening the results. */ - def flatMap[U: ClassTag](f: T => TraversableOnce[U]): RDD[U] = - new FlatMappedRDD(this, sc.clean(f)) + def flatMap[U: ClassTag](f: T => TraversableOnce[U]): RDD[U] = { + val cleanF = sc.clean(f) + new MapPartitionsRDD[U, T](this, (context, pid, iter) => iter.flatMap(cleanF)) + } /** * Return a new RDD containing only the elements that satisfy a predicate. */ - def filter(f: T => Boolean): RDD[T] = new FilteredRDD(this, sc.clean(f)) + def filter(f: T => Boolean): RDD[T] = { + val cleanF = sc.clean(f) + new MapPartitionsRDD[T, T]( + this, + (context, pid, iter) => iter.filter(cleanF), + preservesPartitioning = true) + } /** * Return a new RDD containing the distinct elements in this RDD. @@ -501,7 +513,9 @@ abstract class RDD[T: ClassTag]( /** * Return an RDD created by coalescing all elements within each partition into an array. */ - def glom(): RDD[Array[T]] = new GlommedRDD(this) + def glom(): RDD[Array[T]] = { + new MapPartitionsRDD[Array[T], T](this, (context, pid, iter) => Iterator(iter.toArray)) + } /** * Return the Cartesian product of this RDD and another one, that is, the RDD of all pairs of @@ -1160,7 +1174,26 @@ abstract class RDD[T: ClassTag]( * Save this RDD as a text file, using string representations of elements. */ def saveAsTextFile(path: String) { - this.map(x => (NullWritable.get(), new Text(x.toString))) + // https://issues.apache.org/jira/browse/SPARK-2075 + // + // NullWritable is a `Comparable` in Hadoop 1.+, so the compiler cannot find an implicit + // Ordering for it and will use the default `null`. However, it's a `Comparable[NullWritable]` + // in Hadoop 2.+, so the compiler will call the implicit `Ordering.ordered` method to create an + // Ordering for `NullWritable`. That's why the compiler will generate different anonymous + // classes for `saveAsTextFile` in Hadoop 1.+ and Hadoop 2.+. + // + // Therefore, here we provide an explicit Ordering `null` to make sure the compiler generate + // same bytecodes for `saveAsTextFile`. + val nullWritableClassTag = implicitly[ClassTag[NullWritable]] + val textClassTag = implicitly[ClassTag[Text]] + val r = this.mapPartitions { iter => + val text = new Text() + iter.map { x => + text.set(x.toString) + (NullWritable.get(), text) + } + } + RDD.rddToPairRDDFunctions(r)(nullWritableClassTag, textClassTag, null) .saveAsHadoopFile[TextOutputFormat[NullWritable, Text]](path) } @@ -1168,7 +1201,17 @@ abstract class RDD[T: ClassTag]( * Save this RDD as a compressed text file, using string representations of elements. */ def saveAsTextFile(path: String, codec: Class[_ <: CompressionCodec]) { - this.map(x => (NullWritable.get(), new Text(x.toString))) + // https://issues.apache.org/jira/browse/SPARK-2075 + val nullWritableClassTag = implicitly[ClassTag[NullWritable]] + val textClassTag = implicitly[ClassTag[Text]] + val r = this.mapPartitions { iter => + val text = new Text() + iter.map { x => + text.set(x.toString) + (NullWritable.get(), text) + } + } + RDD.rddToPairRDDFunctions(r)(nullWritableClassTag, textClassTag, null) .saveAsHadoopFile[TextOutputFormat[NullWritable, Text]](path, codec) } @@ -1383,3 +1426,46 @@ abstract class RDD[T: ClassTag]( new JavaRDD(this)(elementClassTag) } } + + +/** + * Defines implicit functions that provide extra functionalities on RDDs of specific types. + * + * For example, [[RDD.rddToPairRDDFunctions]] converts an RDD into a [[PairRDDFunctions]] for + * key-value-pair RDDs, and enabling extra functionalities such as [[PairRDDFunctions.reduceByKey]]. + */ +object RDD { + + // The following implicit functions were in SparkContext before 1.2 and users had to + // `import SparkContext._` to enable them. Now we move them here to make the compiler find + // them automatically. However, we still keep the old functions in SparkContext for backward + // compatibility and forward to the following functions directly. + + implicit def rddToPairRDDFunctions[K, V](rdd: RDD[(K, V)]) + (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null): PairRDDFunctions[K, V] = { + new PairRDDFunctions(rdd) + } + + implicit def rddToAsyncRDDActions[T: ClassTag](rdd: RDD[T]): AsyncRDDActions[T] = { + new AsyncRDDActions(rdd) + } + + implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable: ClassTag]( + rdd: RDD[(K, V)]): SequenceFileRDDFunctions[K, V] = { + new SequenceFileRDDFunctions(rdd) + } + + implicit def rddToOrderedRDDFunctions[K : Ordering : ClassTag, V: ClassTag](rdd: RDD[(K, V)]) + : OrderedRDDFunctions[K, V, (K, V)] = { + new OrderedRDDFunctions[K, V, (K, V)](rdd) + } + + implicit def doubleRDDToDoubleRDDFunctions(rdd: RDD[Double]): DoubleRDDFunctions = { + new DoubleRDDFunctions(rdd) + } + + implicit def numericRDDToDoubleRDDFunctions[T](rdd: RDD[T])(implicit num: Numeric[T]) + : DoubleRDDFunctions = { + new DoubleRDDFunctions(rdd.map(x => num.toDouble(x))) + } +} diff --git a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala index 9a1efc83cbe6a..2b48916951430 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala @@ -24,7 +24,6 @@ import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapred.SequenceFileOutputFormat import org.apache.spark.Logging -import org.apache.spark.SparkContext._ /** * Extra functions available on RDDs of (key, value) pairs to create a Hadoop SequenceFile, diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala index 996f2cd3f34a3..95b2dd954e9f4 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala @@ -77,7 +77,7 @@ private[spark] abstract class ZippedPartitionsBaseRDD[V: ClassTag]( private[spark] class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag]( sc: SparkContext, - f: (Iterator[A], Iterator[B]) => Iterator[V], + var f: (Iterator[A], Iterator[B]) => Iterator[V], var rdd1: RDD[A], var rdd2: RDD[B], preservesPartitioning: Boolean = false) @@ -92,13 +92,14 @@ private[spark] class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag] super.clearDependencies() rdd1 = null rdd2 = null + f = null } } private[spark] class ZippedPartitionsRDD3 [A: ClassTag, B: ClassTag, C: ClassTag, V: ClassTag]( sc: SparkContext, - f: (Iterator[A], Iterator[B], Iterator[C]) => Iterator[V], + var f: (Iterator[A], Iterator[B], Iterator[C]) => Iterator[V], var rdd1: RDD[A], var rdd2: RDD[B], var rdd3: RDD[C], @@ -117,13 +118,14 @@ private[spark] class ZippedPartitionsRDD3 rdd1 = null rdd2 = null rdd3 = null + f = null } } private[spark] class ZippedPartitionsRDD4 [A: ClassTag, B: ClassTag, C: ClassTag, D:ClassTag, V: ClassTag]( sc: SparkContext, - f: (Iterator[A], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V], + var f: (Iterator[A], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V], var rdd1: RDD[A], var rdd2: RDD[B], var rdd3: RDD[C], @@ -145,5 +147,6 @@ private[spark] class ZippedPartitionsRDD4 rdd2 = null rdd3 = null rdd4 = null + f = null } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 22449517d100f..cb8ccfbdbdcbb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -449,7 +449,6 @@ class DAGScheduler( } // data structures based on StageId stageIdToStage -= stageId - logDebug("After removal of stage %d, remaining stages = %d" .format(stageId, stageIdToStage.size)) } @@ -751,14 +750,15 @@ class DAGScheduler( localExecutionEnabled && allowLocal && finalStage.parents.isEmpty && partitions.length == 1 if (shouldRunLocally) { // Compute very short actions like first() or take() with no parent stages locally. - listenerBus.post(SparkListenerJobStart(job.jobId, Array[Int](), properties)) + listenerBus.post(SparkListenerJobStart(job.jobId, Seq.empty, properties)) runLocally(job) } else { jobIdToActiveJob(jobId) = job activeJobs += job finalStage.resultOfJob = Some(job) - listenerBus.post(SparkListenerJobStart(job.jobId, jobIdToStageIds(jobId).toArray, - properties)) + val stageIds = jobIdToStageIds(jobId).toArray + val stageInfos = stageIds.flatMap(id => stageIdToStage.get(id).map(_.latestInfo)) + listenerBus.post(SparkListenerJobStart(job.jobId, stageInfos, properties)) submitStage(finalStage) } } @@ -901,6 +901,34 @@ class DAGScheduler( } } + /** Merge updates from a task to our local accumulator values */ + private def updateAccumulators(event: CompletionEvent): Unit = { + val task = event.task + val stage = stageIdToStage(task.stageId) + if (event.accumUpdates != null) { + try { + Accumulators.add(event.accumUpdates) + event.accumUpdates.foreach { case (id, partialValue) => + val acc = Accumulators.originals(id).asInstanceOf[Accumulable[Any, Any]] + // To avoid UI cruft, ignore cases where value wasn't updated + if (acc.name.isDefined && partialValue != acc.zero) { + val name = acc.name.get + val stringPartialValue = Accumulators.stringifyPartialValue(partialValue) + val stringValue = Accumulators.stringifyValue(acc.value) + stage.latestInfo.accumulables(id) = AccumulableInfo(id, name, stringValue) + event.taskInfo.accumulables += + AccumulableInfo(id, name, Some(stringPartialValue), stringValue) + } + } + } catch { + // If we see an exception during accumulator update, just log the + // error and move on. + case e: Exception => + logError(s"Failed to update accumulators for $task", e) + } + } + } + /** * Responds to a task finishing. This is called inside the event loop so it assumes that it can * modify the scheduler's internal state. Use taskEnded() to post a task end event from outside. @@ -941,27 +969,6 @@ class DAGScheduler( } event.reason match { case Success => - if (event.accumUpdates != null) { - try { - Accumulators.add(event.accumUpdates) - event.accumUpdates.foreach { case (id, partialValue) => - val acc = Accumulators.originals(id).asInstanceOf[Accumulable[Any, Any]] - // To avoid UI cruft, ignore cases where value wasn't updated - if (acc.name.isDefined && partialValue != acc.zero) { - val name = acc.name.get - val stringPartialValue = Accumulators.stringifyPartialValue(partialValue) - val stringValue = Accumulators.stringifyValue(acc.value) - stage.latestInfo.accumulables(id) = AccumulableInfo(id, name, stringValue) - event.taskInfo.accumulables += - AccumulableInfo(id, name, Some(stringPartialValue), stringValue) - } - } - } catch { - // If we see an exception during accumulator update, just log the error and move on. - case e: Exception => - logError(s"Failed to update accumulators for $task", e) - } - } listenerBus.post(SparkListenerTaskEnd(stageId, stage.latestInfo.attemptId, taskType, event.reason, event.taskInfo, event.taskMetrics)) stage.pendingTasks -= task @@ -970,6 +977,7 @@ class DAGScheduler( stage.resultOfJob match { case Some(job) => if (!job.finished(rt.outputId)) { + updateAccumulators(event) job.finished(rt.outputId) = true job.numFinished += 1 // If the whole job has finished, remove it @@ -994,6 +1002,7 @@ class DAGScheduler( } case smt: ShuffleMapTask => + updateAccumulators(event) val status = event.result.asInstanceOf[MapStatus] val execId = status.location.executorId logDebug("ShuffleMapTask finished on " + execId) @@ -1082,7 +1091,6 @@ class DAGScheduler( } failedStages += failedStage failedStages += mapStage - // Mark the map whose fetch failed as broken in the map stage if (mapId != -1) { mapStage.removeOutputLoc(mapId, bmAddress) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 597dbc884913c..27bf4f1599076 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -17,20 +17,23 @@ package org.apache.spark.scheduler +import java.io._ +import java.net.URI + import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import com.google.common.base.Charsets import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path} import org.apache.hadoop.fs.permission.FsPermission import org.json4s.JsonAST.JValue import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{Logging, SparkConf, SparkContext} +import org.apache.spark.{Logging, SparkConf, SPARK_VERSION} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec -import org.apache.spark.SPARK_VERSION -import org.apache.spark.util.{FileLogger, JsonProtocol, Utils} +import org.apache.spark.util.{JsonProtocol, Utils} /** * A SparkListener that logs events to persistent storage. @@ -58,36 +61,78 @@ private[spark] class EventLoggingListener( private val shouldOverwrite = sparkConf.getBoolean("spark.eventLog.overwrite", false) private val testing = sparkConf.getBoolean("spark.eventLog.testing", false) private val outputBufferSize = sparkConf.getInt("spark.eventLog.buffer.kb", 100) * 1024 - val logDir = EventLoggingListener.getLogDirPath(logBaseDir, appId) - val logDirName: String = logDir.split("/").last - protected val logger = new FileLogger(logDir, sparkConf, hadoopConf, outputBufferSize, - shouldCompress, shouldOverwrite, Some(LOG_FILE_PERMISSIONS)) + private val fileSystem = Utils.getHadoopFileSystem(new URI(logBaseDir), hadoopConf) + + // Only defined if the file system scheme is not local + private var hadoopDataStream: Option[FSDataOutputStream] = None + + // The Hadoop APIs have changed over time, so we use reflection to figure out + // the correct method to use to flush a hadoop data stream. See SPARK-1518 + // for details. + private val hadoopFlushMethod = { + val cls = classOf[FSDataOutputStream] + scala.util.Try(cls.getMethod("hflush")).getOrElse(cls.getMethod("sync")) + } + + private var writer: Option[PrintWriter] = None // For testing. Keep track of all JSON serialized events that have been logged. private[scheduler] val loggedEvents = new ArrayBuffer[JValue] + // Visible for tests only. + private[scheduler] val logPath = getLogPath(logBaseDir, appId) + /** - * Begin logging events. - * If compression is used, log a file that indicates which compression library is used. + * Creates the log file in the configured log directory. */ def start() { - logger.start() - logInfo("Logging events to %s".format(logDir)) - if (shouldCompress) { - val codec = - sparkConf.get("spark.io.compression.codec", CompressionCodec.DEFAULT_COMPRESSION_CODEC) - logger.newFile(COMPRESSION_CODEC_PREFIX + codec) + if (!fileSystem.isDirectory(new Path(logBaseDir))) { + throw new IllegalArgumentException(s"Log directory $logBaseDir does not exist.") + } + + val workingPath = logPath + IN_PROGRESS + val uri = new URI(workingPath) + val path = new Path(workingPath) + val defaultFs = FileSystem.getDefaultUri(hadoopConf).getScheme + val isDefaultLocal = defaultFs == null || defaultFs == "file" + + if (shouldOverwrite && fileSystem.exists(path)) { + logWarning(s"Event log $path already exists. Overwriting...") + fileSystem.delete(path, true) } - logger.newFile(SPARK_VERSION_PREFIX + SPARK_VERSION) - logger.newFile(LOG_PREFIX + logger.fileIndex) + + /* The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). + * Therefore, for local files, use FileOutputStream instead. */ + val dstream = + if ((isDefaultLocal && uri.getScheme == null) || uri.getScheme == "file") { + new FileOutputStream(uri.getPath) + } else { + hadoopDataStream = Some(fileSystem.create(path)) + hadoopDataStream.get + } + + val compressionCodec = + if (shouldCompress) { + Some(CompressionCodec.createCodec(sparkConf)) + } else { + None + } + + fileSystem.setPermission(path, LOG_FILE_PERMISSIONS) + val logStream = initEventLog(new BufferedOutputStream(dstream, outputBufferSize), + compressionCodec) + writer = Some(new PrintWriter(logStream)) + + logInfo("Logging events to %s".format(logPath)) } /** Log the event as JSON. */ private def logEvent(event: SparkListenerEvent, flushLogger: Boolean = false) { val eventJson = JsonProtocol.sparkEventToJson(event) - logger.logLine(compact(render(eventJson))) + writer.foreach(_.println(compact(render(eventJson)))) if (flushLogger) { - logger.flush() + writer.foreach(_.flush()) + hadoopDataStream.foreach(hadoopFlushMethod.invoke(_)) } if (testing) { loggedEvents += eventJson @@ -123,130 +168,164 @@ private[spark] class EventLoggingListener( logEvent(event, flushLogger = true) override def onApplicationEnd(event: SparkListenerApplicationEnd) = logEvent(event, flushLogger = true) + // No-op because logging every update would be overkill override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate) { } /** - * Stop logging events. - * In addition, create an empty special file to indicate application completion. + * Stop logging events. The event log file will be renamed so that it loses the + * ".inprogress" suffix. */ def stop() = { - logger.newFile(APPLICATION_COMPLETE) - logger.stop() + writer.foreach(_.close()) + + val target = new Path(logPath) + if (fileSystem.exists(target)) { + if (shouldOverwrite) { + logWarning(s"Event log $target already exists. Overwriting...") + fileSystem.delete(target, true) + } else { + throw new IOException("Target log file already exists (%s)".format(logPath)) + } + } + fileSystem.rename(new Path(logPath + IN_PROGRESS), target) } + } private[spark] object EventLoggingListener extends Logging { + // Suffix applied to the names of files still being written by applications. + val IN_PROGRESS = ".inprogress" val DEFAULT_LOG_DIR = "/tmp/spark-events" - val LOG_PREFIX = "EVENT_LOG_" - val SPARK_VERSION_PREFIX = "SPARK_VERSION_" - val COMPRESSION_CODEC_PREFIX = "COMPRESSION_CODEC_" - val APPLICATION_COMPLETE = "APPLICATION_COMPLETE" - val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort) - // A cache for compression codecs to avoid creating the same codec many times - private val codecMap = new mutable.HashMap[String, CompressionCodec] + private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort) - def isEventLogFile(fileName: String): Boolean = { - fileName.startsWith(LOG_PREFIX) - } + // Marker for the end of header data in a log file. After this marker, log data, potentially + // compressed, will be found. + private val HEADER_END_MARKER = "=== LOG_HEADER_END ===" - def isSparkVersionFile(fileName: String): Boolean = { - fileName.startsWith(SPARK_VERSION_PREFIX) - } + // To avoid corrupted files causing the heap to fill up. Value is arbitrary. + private val MAX_HEADER_LINE_LENGTH = 4096 - def isCompressionCodecFile(fileName: String): Boolean = { - fileName.startsWith(COMPRESSION_CODEC_PREFIX) - } + // A cache for compression codecs to avoid creating the same codec many times + private val codecMap = new mutable.HashMap[String, CompressionCodec] - def isApplicationCompleteFile(fileName: String): Boolean = { - fileName == APPLICATION_COMPLETE - } + /** + * Write metadata about the event log to the given stream. + * + * The header is a serialized version of a map, except it does not use Java serialization to + * avoid incompatibilities between different JDKs. It writes one map entry per line, in + * "key=value" format. + * + * The very last entry in the header is the `HEADER_END_MARKER` marker, so that the parsing code + * can know when to stop. + * + * The format needs to be kept in sync with the openEventLog() method below. Also, it cannot + * change in new Spark versions without some other way of detecting the change (like some + * metadata encoded in the file name). + * + * @param logStream Raw output stream to the even log file. + * @param compressionCodec Optional compression codec to use. + * @return A stream where to write event log data. This may be a wrapper around the original + * stream (for example, when compression is enabled). + */ + def initEventLog( + logStream: OutputStream, + compressionCodec: Option[CompressionCodec]): OutputStream = { + val meta = mutable.HashMap(("version" -> SPARK_VERSION)) + compressionCodec.foreach { codec => + meta += ("compressionCodec" -> codec.getClass().getName()) + } - def parseSparkVersion(fileName: String): String = { - if (isSparkVersionFile(fileName)) { - fileName.replaceAll(SPARK_VERSION_PREFIX, "") - } else "" - } + def write(entry: String) = { + val bytes = entry.getBytes(Charsets.UTF_8) + if (bytes.length > MAX_HEADER_LINE_LENGTH) { + throw new IOException(s"Header entry too long: ${entry}") + } + logStream.write(bytes, 0, bytes.length) + } - def parseCompressionCodec(fileName: String): String = { - if (isCompressionCodecFile(fileName)) { - fileName.replaceAll(COMPRESSION_CODEC_PREFIX, "") - } else "" + meta.foreach { case (k, v) => write(s"$k=$v\n") } + write(s"$HEADER_END_MARKER\n") + compressionCodec.map(_.compressedOutputStream(logStream)).getOrElse(logStream) } /** - * Return a file-system-safe path to the log directory for the given application. + * Return a file-system-safe path to the log file for the given application. * - * @param logBaseDir A base directory for the path to the log directory for given application. + * @param logBaseDir Directory where the log file will be written. * @param appId A unique app ID. * @return A path which consists of file-system-safe characters. */ - def getLogDirPath(logBaseDir: String, appId: String): String = { + def getLogPath(logBaseDir: String, appId: String): String = { val name = appId.replaceAll("[ :/]", "-").replaceAll("[${}'\"]", "_").toLowerCase Utils.resolveURI(logBaseDir) + "/" + name.stripSuffix("/") } /** - * Parse the event logging information associated with the logs in the given directory. + * Opens an event log file and returns an input stream to the event data. * - * Specifically, this looks for event log files, the Spark version file, the compression - * codec file (if event logs are compressed), and the application completion file (if the - * application has run to completion). + * @return 2-tuple (event input stream, Spark version of event data) */ - def parseLoggingInfo(logDir: Path, fileSystem: FileSystem): EventLoggingInfo = { + def openEventLog(log: Path, fs: FileSystem): (InputStream, String) = { + // It's not clear whether FileSystem.open() throws FileNotFoundException or just plain + // IOException when a file does not exist, so try our best to throw a proper exception. + if (!fs.exists(log)) { + throw new FileNotFoundException(s"File $log does not exist.") + } + + val in = new BufferedInputStream(fs.open(log)) + // Read a single line from the input stream without buffering. + // We cannot use BufferedReader because we must avoid reading + // beyond the end of the header, after which the content of the + // file may be compressed. + def readLine(): String = { + val bytes = new ByteArrayOutputStream() + var next = in.read() + var count = 0 + while (next != '\n') { + if (next == -1) { + throw new IOException("Unexpected end of file.") + } + bytes.write(next) + count = count + 1 + if (count > MAX_HEADER_LINE_LENGTH) { + throw new IOException("Maximum header line length exceeded.") + } + next = in.read() + } + new String(bytes.toByteArray(), Charsets.UTF_8) + } + + // Parse the header metadata in the form of k=v pairs + // This assumes that every line before the header end marker follows this format try { - val fileStatuses = fileSystem.listStatus(logDir) - val filePaths = - if (fileStatuses != null) { - fileStatuses.filter(!_.isDir).map(_.getPath).toSeq - } else { - Seq[Path]() + val meta = new mutable.HashMap[String, String]() + var foundEndMarker = false + while (!foundEndMarker) { + readLine() match { + case HEADER_END_MARKER => + foundEndMarker = true + case entry => + val prop = entry.split("=", 2) + if (prop.length != 2) { + throw new IllegalArgumentException("Invalid metadata in log file.") + } + meta += (prop(0) -> prop(1)) } - if (filePaths.isEmpty) { - logWarning("No files found in logging directory %s".format(logDir)) } - EventLoggingInfo( - logPaths = filePaths.filter { path => isEventLogFile(path.getName) }, - sparkVersion = filePaths - .find { path => isSparkVersionFile(path.getName) } - .map { path => parseSparkVersion(path.getName) } - .getOrElse(""), - compressionCodec = filePaths - .find { path => isCompressionCodecFile(path.getName) } - .map { path => - val codec = EventLoggingListener.parseCompressionCodec(path.getName) - val conf = new SparkConf - conf.set("spark.io.compression.codec", codec) - codecMap.getOrElseUpdate(codec, CompressionCodec.createCodec(conf)) - }, - applicationComplete = filePaths.exists { path => isApplicationCompleteFile(path.getName) } - ) + + val sparkVersion = meta.get("version").getOrElse( + throw new IllegalArgumentException("Missing Spark version in log metadata.")) + val codec = meta.get("compressionCodec").map { codecName => + codecMap.getOrElseUpdate(codecName, CompressionCodec.createCodec(new SparkConf, codecName)) + } + (codec.map(_.compressedInputStream(in)).getOrElse(in), sparkVersion) } catch { case e: Exception => - logError("Exception in parsing logging info from directory %s".format(logDir), e) - EventLoggingInfo.empty + in.close() + throw e } } - /** - * Parse the event logging information associated with the logs in the given directory. - */ - def parseLoggingInfo(logDir: String, fileSystem: FileSystem): EventLoggingInfo = { - parseLoggingInfo(new Path(logDir), fileSystem) - } -} - - -/** - * Information needed to process the event logs associated with an application. - */ -private[spark] case class EventLoggingInfo( - logPaths: Seq[Path], - sparkVersion: String, - compressionCodec: Option[CompressionCodec], - applicationComplete: Boolean = false) - -private[spark] object EventLoggingInfo { - def empty = EventLoggingInfo(Seq[Path](), "", None, applicationComplete = false) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index 01d5943d777f3..1efce124c0a6b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -122,7 +122,7 @@ private[spark] class CompressedMapStatus( /** * A [[MapStatus]] implementation that only stores the average size of non-empty blocks, - * plus a bitmap for tracking which blocks are non-empty. During serialization, this bitmap + * plus a bitmap for tracking which blocks are empty. During serialization, this bitmap * is compressed. * * @param loc location where the task is being executed diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index f89724d4ea196..584f4e7789d1a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -17,74 +17,45 @@ package org.apache.spark.scheduler -import java.io.{BufferedInputStream, InputStream} +import java.io.{InputStream, IOException} import scala.io.Source -import org.apache.hadoop.fs.{Path, FileSystem} import org.json4s.jackson.JsonMethods._ import org.apache.spark.Logging -import org.apache.spark.io.CompressionCodec import org.apache.spark.util.JsonProtocol /** - * A SparkListenerBus that replays logged events from persisted storage. - * - * This assumes the given paths are valid log files, where each line can be deserialized into - * exactly one SparkListenerEvent. + * A SparkListenerBus that can be used to replay events from serialized event data. */ -private[spark] class ReplayListenerBus( - logPaths: Seq[Path], - fileSystem: FileSystem, - compressionCodec: Option[CompressionCodec]) - extends SparkListenerBus with Logging { - - private var replayed = false - - if (logPaths.length == 0) { - logWarning("Log path provided contains no log files.") - } +private[spark] class ReplayListenerBus extends SparkListenerBus with Logging { /** - * Replay each event in the order maintained in the given logs. - * This should only be called exactly once. + * Replay each event in the order maintained in the given stream. The stream is expected to + * contain one JSON-encoded SparkListenerEvent per line. + * + * This method can be called multiple times, but the listener behavior is undefined after any + * error is thrown by this method. + * + * @param logData Stream containing event log data. + * @param version Spark version that generated the events. */ - def replay() { - assert(!replayed, "ReplayListenerBus cannot replay events more than once") - logPaths.foreach { path => - // Keep track of input streams at all levels to close them later - // This is necessary because an exception can occur in between stream initializations - var fileStream: Option[InputStream] = None - var bufferedStream: Option[InputStream] = None - var compressStream: Option[InputStream] = None - var currentLine = "" - try { - fileStream = Some(fileSystem.open(path)) - bufferedStream = Some(new BufferedInputStream(fileStream.get)) - compressStream = Some(wrapForCompression(bufferedStream.get)) - - // Parse each line as an event and post the event to all attached listeners - val lines = Source.fromInputStream(compressStream.get).getLines() - lines.foreach { line => - currentLine = line - postToAll(JsonProtocol.sparkEventFromJson(parse(line))) - } - } catch { - case e: Exception => - logError("Exception in parsing Spark event log %s".format(path), e) - logError("Malformed line: %s\n".format(currentLine)) - } finally { - fileStream.foreach(_.close()) - bufferedStream.foreach(_.close()) - compressStream.foreach(_.close()) + def replay(logData: InputStream, version: String) { + var currentLine: String = null + try { + val lines = Source.fromInputStream(logData).getLines() + lines.foreach { line => + currentLine = line + postToAll(JsonProtocol.sparkEventFromJson(parse(line))) } + } catch { + case ioe: IOException => + throw ioe + case e: Exception => + logError("Exception in parsing Spark event log.", e) + logError("Malformed line: %s\n".format(currentLine)) } - replayed = true } - /** If a compression codec is specified, wrap the given stream in a compression stream. */ - private def wrapForCompression(stream: InputStream): InputStream = { - compressionCodec.map(_.compressedInputStream(stream)).getOrElse(stream) - } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 86afe3bd5265f..b62b0c1312693 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -56,8 +56,15 @@ case class SparkListenerTaskEnd( extends SparkListenerEvent @DeveloperApi -case class SparkListenerJobStart(jobId: Int, stageIds: Seq[Int], properties: Properties = null) - extends SparkListenerEvent +case class SparkListenerJobStart( + jobId: Int, + stageInfos: Seq[StageInfo], + properties: Properties = null) + extends SparkListenerEvent { + // Note: this is here for backwards-compatibility with older versions of this event which + // only stored stageIds and not StageInfos: + val stageIds: Seq[Int] = stageInfos.map(_.stageId) +} @DeveloperApi case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 2552d03d18d06..d7dde4fe38436 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -45,7 +45,7 @@ import org.apache.spark.util.Utils private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) extends Serializable { final def run(attemptId: Long): T = { - context = new TaskContextImpl(stageId, partitionId, attemptId, false) + context = new TaskContextImpl(stageId, partitionId, attemptId, runningLocally = false) TaskContextHelper.setTaskContext(context) context.taskMetrics.hostname = Utils.localHostName() taskThread = Thread.currentThread() diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index 819b51e12ad8c..4896ec845bbc9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -19,6 +19,7 @@ package org.apache.spark.scheduler import java.nio.ByteBuffer +import scala.language.existentials import scala.util.control.NonFatal import org.apache.spark._ diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index cabdc655f89bf..28e6147509f78 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -249,7 +249,7 @@ private[spark] class TaskSetManager( * This method also cleans up any tasks in the list that have already * been launched, since we want that to happen lazily. */ - private def findTaskFromList(execId: String, list: ArrayBuffer[Int]): Option[Int] = { + private def dequeueTaskFromList(execId: String, list: ArrayBuffer[Int]): Option[Int] = { var indexOffset = list.size while (indexOffset > 0) { indexOffset -= 1 @@ -290,7 +290,7 @@ private[spark] class TaskSetManager( * an attempt running on this host, in case the host is slow. In addition, the task should meet * the given locality constraint. */ - private def findSpeculativeTask(execId: String, host: String, locality: TaskLocality.Value) + private def dequeueSpeculativeTask(execId: String, host: String, locality: TaskLocality.Value) : Option[(Int, TaskLocality.Value)] = { speculatableTasks.retain(index => !successful(index)) // Remove finished tasks from set @@ -366,22 +366,22 @@ private[spark] class TaskSetManager( * * @return An option containing (task index within the task set, locality, is speculative?) */ - private def findTask(execId: String, host: String, maxLocality: TaskLocality.Value) + private def dequeueTask(execId: String, host: String, maxLocality: TaskLocality.Value) : Option[(Int, TaskLocality.Value, Boolean)] = { - for (index <- findTaskFromList(execId, getPendingTasksForExecutor(execId))) { + for (index <- dequeueTaskFromList(execId, getPendingTasksForExecutor(execId))) { return Some((index, TaskLocality.PROCESS_LOCAL, false)) } if (TaskLocality.isAllowed(maxLocality, TaskLocality.NODE_LOCAL)) { - for (index <- findTaskFromList(execId, getPendingTasksForHost(host))) { + for (index <- dequeueTaskFromList(execId, getPendingTasksForHost(host))) { return Some((index, TaskLocality.NODE_LOCAL, false)) } } if (TaskLocality.isAllowed(maxLocality, TaskLocality.NO_PREF)) { // Look for noPref tasks after NODE_LOCAL for minimize cross-rack traffic - for (index <- findTaskFromList(execId, pendingTasksWithNoPrefs)) { + for (index <- dequeueTaskFromList(execId, pendingTasksWithNoPrefs)) { return Some((index, TaskLocality.PROCESS_LOCAL, false)) } } @@ -389,20 +389,20 @@ private[spark] class TaskSetManager( if (TaskLocality.isAllowed(maxLocality, TaskLocality.RACK_LOCAL)) { for { rack <- sched.getRackForHost(host) - index <- findTaskFromList(execId, getPendingTasksForRack(rack)) + index <- dequeueTaskFromList(execId, getPendingTasksForRack(rack)) } { return Some((index, TaskLocality.RACK_LOCAL, false)) } } if (TaskLocality.isAllowed(maxLocality, TaskLocality.ANY)) { - for (index <- findTaskFromList(execId, allPendingTasks)) { + for (index <- dequeueTaskFromList(execId, allPendingTasks)) { return Some((index, TaskLocality.ANY, false)) } } // find a speculative task if all others tasks have been scheduled - findSpeculativeTask(execId, host, maxLocality).map { + dequeueSpeculativeTask(execId, host, maxLocality).map { case (taskIndex, allowedLocality) => (taskIndex, allowedLocality, true)} } @@ -436,7 +436,7 @@ private[spark] class TaskSetManager( } } - findTask(execId, host, allowedLocality) match { + dequeueTask(execId, host, allowedLocality) match { case Some((index, taskLocality, speculative)) => { // Found a task; do some bookkeeping and return a task description val task = tasks(index) @@ -704,7 +704,7 @@ private[spark] class TaskSetManager( // Re-enqueue pending tasks for this host based on the status of the cluster. Note // that it's okay if we add a task to the same queue twice (if it had multiple preferred - // locations), because findTaskFromList will skip already-running tasks. + // locations), because dequeueTaskFromList will skip already-running tasks. for (index <- getPendingTasksForExecutor(execId)) { addPendingTask(index, readding=true) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 047fae104b485..fe9914b50bc54 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -27,7 +27,7 @@ import akka.actor._ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} -import org.apache.spark.{SparkEnv, Logging, SparkException, TaskState} +import org.apache.spark.{ExecutorAllocationClient, Logging, SparkEnv, SparkException, TaskState} import org.apache.spark.scheduler.{SchedulerBackend, SlaveLost, TaskDescription, TaskSchedulerImpl, WorkerOffer} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.util.{ActorLogReceive, SerializableBuffer, AkkaUtils, Utils} @@ -42,7 +42,7 @@ import org.apache.spark.util.{ActorLogReceive, SerializableBuffer, AkkaUtils, Ut */ private[spark] class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSystem: ActorSystem) - extends SchedulerBackend with Logging + extends ExecutorAllocationClient with SchedulerBackend with Logging { // Use an atomic variable to track total number of cores in the cluster for simplicity and speed var totalCoreCount = new AtomicInteger(0) @@ -127,7 +127,13 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste makeOffers() case KillTask(taskId, executorId, interruptThread) => - executorDataMap(executorId).executorActor ! KillTask(taskId, executorId, interruptThread) + executorDataMap.get(executorId) match { + case Some(executorInfo) => + executorInfo.executorActor ! KillTask(taskId, executorId, interruptThread) + case None => + // Ignoring the task kill since the executor is not registered. + logWarning(s"Attempted to kill task $taskId for unknown executor $executorId.") + } case StopDriver => sender ! true @@ -301,7 +307,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste * Request an additional number of executors from the cluster manager. * Return whether the request is acknowledged. */ - final def requestExecutors(numAdditionalExecutors: Int): Boolean = synchronized { + final override def requestExecutors(numAdditionalExecutors: Int): Boolean = synchronized { logInfo(s"Requesting $numAdditionalExecutors additional executor(s) from the cluster manager") logDebug(s"Number of pending executors is now $numPendingExecutors") numPendingExecutors += numAdditionalExecutors @@ -328,7 +334,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste * Request that the cluster manager kill the specified executors. * Return whether the kill request is acknowledged. */ - final def killExecutors(executorIds: Seq[String]): Boolean = { + final override def killExecutors(executorIds: Seq[String]): Boolean = synchronized { logInfo(s"Requesting to kill executor(s) ${executorIds.mkString(", ")}") val filteredExecutorIds = new ArrayBuffer[String] executorIds.foreach { id => diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index d13795186c48e..10e6886c16a4f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -208,10 +208,12 @@ private[spark] class MesosSchedulerBackend( */ override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) { inClassLoader() { - val (acceptedOffers, declinedOffers) = offers.partition { o => + // Fail-fast on offers we know will be rejected + val (usableOffers, unUsableOffers) = offers.partition { o => val mem = getResource(o.getResourcesList, "mem") val cpus = getResource(o.getResourcesList, "cpus") val slaveId = o.getSlaveId.getValue + // TODO(pwendell): Should below be 1 + scheduler.CPUS_PER_TASK? (mem >= MemoryUtils.calculateTotalMemory(sc) && // need at least 1 for executor, 1 for task cpus >= 2 * scheduler.CPUS_PER_TASK) || @@ -219,11 +221,12 @@ private[spark] class MesosSchedulerBackend( cpus >= scheduler.CPUS_PER_TASK) } - val offerableWorkers = acceptedOffers.map { o => + val workerOffers = usableOffers.map { o => val cpus = if (slaveIdsWithExecutors.contains(o.getSlaveId.getValue)) { getResource(o.getResourcesList, "cpus").toInt } else { // If the executor doesn't exist yet, subtract CPU for executor + // TODO(pwendell): Should below just subtract "1"? getResource(o.getResourcesList, "cpus").toInt - scheduler.CPUS_PER_TASK } @@ -233,17 +236,20 @@ private[spark] class MesosSchedulerBackend( cpus) } - val slaveIdToOffer = acceptedOffers.map(o => o.getSlaveId.getValue -> o).toMap + val slaveIdToOffer = usableOffers.map(o => o.getSlaveId.getValue -> o).toMap val mesosTasks = new HashMap[String, JArrayList[MesosTaskInfo]] + val slavesIdsOfAcceptedOffers = HashSet[String]() + // Call into the TaskSchedulerImpl - scheduler.resourceOffers(offerableWorkers) - .filter(!_.isEmpty) + val acceptedOffers = scheduler.resourceOffers(workerOffers).filter(!_.isEmpty) + acceptedOffers .foreach { offer => offer.foreach { taskDesc => val slaveId = taskDesc.executorId slaveIdsWithExecutors += slaveId + slavesIdsOfAcceptedOffers += slaveId taskIdToSlaveId(taskDesc.taskId) = slaveId mesosTasks.getOrElseUpdate(slaveId, new JArrayList[MesosTaskInfo]) .add(createMesosTask(taskDesc, slaveId)) @@ -257,7 +263,14 @@ private[spark] class MesosSchedulerBackend( d.launchTasks(Collections.singleton(slaveIdToOffer(slaveId).getId), tasks, filters) } - declinedOffers.foreach(o => d.declineOffer(o.getId)) + // Decline offers that weren't used + // NOTE: This logic assumes that we only get a single offer for each host in a given batch + for (o <- usableOffers if !slavesIdsOfAcceptedOffers.contains(o.getSlaveId.getValue)) { + d.declineOffer(o.getId) + } + + // Decline offers we ruled out immediately + unUsableOffers.foreach(o => d.declineOffer(o.getId)) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index a2f1f14264a99..b3bd3110ac809 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -41,17 +41,18 @@ private case class StopExecutor() * and the TaskSchedulerImpl. */ private[spark] class LocalActor( - scheduler: TaskSchedulerImpl, - executorBackend: LocalBackend, - private val totalCores: Int) extends Actor with ActorLogReceive with Logging { + scheduler: TaskSchedulerImpl, + executorBackend: LocalBackend, + private val totalCores: Int) + extends Actor with ActorLogReceive with Logging { private var freeCores = totalCores private val localExecutorId = SparkContext.DRIVER_IDENTIFIER private val localExecutorHostname = "localhost" - val executor = new Executor( - localExecutorId, localExecutorHostname, scheduler.conf.getAll, totalCores, isLocal = true) + private val executor = new Executor( + localExecutorId, localExecutorHostname, SparkEnv.get, isLocal = true) override def receiveWithLogging = { case ReviveOffers => diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala index 801ae54086053..a44a8e1249256 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala @@ -20,8 +20,8 @@ package org.apache.spark.shuffle import org.apache.spark.{TaskContext, ShuffleDependency} /** - * Pluggable interface for shuffle systems. A ShuffleManager is created in SparkEnv on both the - * driver and executors, based on the spark.shuffle.manager setting. The driver registers shuffles + * Pluggable interface for shuffle systems. A ShuffleManager is created in SparkEnv on the driver + * and on each executor, based on the spark.shuffle.manager setting. The driver registers shuffles * with it, and executors (or tasks running locally in the driver) can ask to read and write data. * * NOTE: this will be instantiated by SparkEnv so its constructor can take a SparkConf and diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala index ee91a368b76ea..3bcc7178a3d8b 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala @@ -66,8 +66,9 @@ private[spark] class ShuffleMemoryManager(maxMemory: Long) extends Logging { val curMem = threadMemory(threadId) val freeMemory = maxMemory - threadMemory.values.sum - // How much we can grant this thread; don't let it grow to more than 1 / numActiveThreads - val maxToGrant = math.min(numBytes, (maxMemory / numActiveThreads) - curMem) + // How much we can grant this thread; don't let it grow to more than 1 / numActiveThreads; + // don't let it be negative + val maxToGrant = math.min(numBytes, math.max(0, (maxMemory / numActiveThreads) - curMem)) if (curMem < maxMemory / (2 * numActiveThreads)) { // We want to let each thread get at least 1 / (2 * numActiveThreads) before blocking; diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala index 5baf45db45c17..de72148ccc7ac 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala @@ -45,9 +45,9 @@ private[spark] class HashShuffleReader[K, C]( } else { new InterruptibleIterator(context, dep.aggregator.get.combineValuesByKey(iter, context)) } - } else if (dep.aggregator.isEmpty && dep.mapSideCombine) { - throw new IllegalStateException("Aggregator is empty for map-side combine") } else { + require(!dep.mapSideCombine, "Map-side combine without Aggregator specified!") + // Convert the Product2s to pairs since this is what downstream RDDs currently expect iter.asInstanceOf[Iterator[Product2[K, C]]].map(pair => (pair._1, pair._2)) } diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala index 183a30373b28c..755f17d6aa15a 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala @@ -56,9 +56,8 @@ private[spark] class HashShuffleWriter[K, V]( } else { records } - } else if (dep.aggregator.isEmpty && dep.mapSideCombine) { - throw new IllegalStateException("Aggregator is empty for map-side combine") } else { + require(!dep.mapSideCombine, "Map-side combine without Aggregator specified!") records } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index d75f9d7311fad..27496c5a289cb 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -50,9 +50,7 @@ private[spark] class SortShuffleWriter[K, V, C]( /** Write a bunch of records to this task's output */ override def write(records: Iterator[_ <: Product2[K, V]]): Unit = { if (dep.mapSideCombine) { - if (!dep.aggregator.isDefined) { - throw new IllegalStateException("Aggregator is empty for map-side combine") - } + require(dep.aggregator.isDefined, "Map-side combine without Aggregator specified!") sorter = new ExternalSorter[K, V, C]( dep.aggregator, Some(dep.partitioner), dep.keyOrdering, dep.serializer) sorter.insertAll(records) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 308c59eda594d..d7b184f8a10e9 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -1014,8 +1014,10 @@ private[spark] class BlockManager( // If we get here, the block write failed. logWarning(s"Block $blockId was marked as failure. Nothing to drop") return None + } else if (blockInfo.get(blockId).isEmpty) { + logWarning(s"Block $blockId was already dropped.") + return None } - var blockIsUpdated = false val level = info.level diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 685b2e11440fb..9cbda41223a8b 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -73,9 +73,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus case UpdateBlockInfo( blockManagerId, blockId, storageLevel, deserializedSize, size, tachyonSize) => - // TODO: Ideally we want to handle all the message replies in receive instead of in the - // individual private methods. - updateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size, tachyonSize) + sender ! updateBlockInfo( + blockManagerId, blockId, storageLevel, deserializedSize, size, tachyonSize) case GetLocations(blockId) => sender ! getLocations(blockId) @@ -355,23 +354,21 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus storageLevel: StorageLevel, memSize: Long, diskSize: Long, - tachyonSize: Long) { + tachyonSize: Long): Boolean = { if (!blockManagerInfo.contains(blockManagerId)) { if (blockManagerId.isDriver && !isLocal) { // We intentionally do not register the master (except in local mode), // so we should not indicate failure. - sender ! true + return true } else { - sender ! false + return false } - return } if (blockId == null) { blockManagerInfo(blockManagerId).updateLastSeenMs() - sender ! true - return + return true } blockManagerInfo(blockManagerId).updateBlockInfo( @@ -395,7 +392,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus if (locations.size == 0) { blockLocations.remove(blockId) } - sender ! true + true } private def getLocations(blockId: BlockId): Seq[BlockManagerId] = { diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 58fba54710510..bb2ae9f3f4586 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -17,7 +17,7 @@ package org.apache.spark.storage -import java.io.File +import java.io.{IOException, File} import java.text.SimpleDateFormat import java.util.{Date, Random, UUID} @@ -71,7 +71,9 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon old } else { val newDir = new File(localDirs(dirId), "%02x".format(subDirId)) - newDir.mkdir() + if (!newDir.exists() && !newDir.mkdir()) { + throw new IOException(s"Failed to create local dir in $newDir.") + } subDirs(dirId)(subDirId) = newDir newDir } diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index 6b1f57a069431..2499c11a65b0e 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -17,6 +17,7 @@ package org.apache.spark.storage +import java.io.{InputStream, IOException} import java.util.concurrent.LinkedBlockingQueue import scala.collection.mutable.{ArrayBuffer, HashSet, Queue} @@ -265,7 +266,7 @@ final class ShuffleBlockFetcherIterator( // Get Local Blocks fetchLocalBlocks() - logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") + logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime)) } override def hasNext: Boolean = numBlocksProcessed < numBlocksToFetch @@ -289,17 +290,22 @@ final class ShuffleBlockFetcherIterator( } val iteratorTry: Try[Iterator[Any]] = result match { - case FailureFetchResult(_, e) => Failure(e) - case SuccessFetchResult(blockId, _, buf) => { - val is = blockManager.wrapForCompression(blockId, buf.createInputStream()) - val iter = serializer.newInstance().deserializeStream(is).asIterator - Success(CompletionIterator[Any, Iterator[Any]](iter, { - // Once the iterator is exhausted, release the buffer and set currentResult to null - // so we don't release it again in cleanup. - currentResult = null - buf.release() - })) - } + case FailureFetchResult(_, e) => + Failure(e) + case SuccessFetchResult(blockId, _, buf) => + // There is a chance that createInputStream can fail (e.g. fetching a local file that does + // not exist, SPARK-4085). In that case, we should propagate the right exception so + // the scheduler gets a FetchFailedException. + Try(buf.createInputStream()).map { is0 => + val is = blockManager.wrapForCompression(blockId, is0) + val iter = serializer.newInstance().deserializeStream(is).asIterator + CompletionIterator[Any, Iterator[Any]](iter, { + // Once the iterator is exhausted, release the buffer and set currentResult to null + // so we don't release it again in cleanup. + currentResult = null + buf.release() + }) + } } (result.blockId, iteratorTry) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index 56edc4fe2e4ad..e5e1cf5a69a19 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -18,6 +18,7 @@ package org.apache.spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} +import java.util.concurrent.ConcurrentHashMap import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.Utils @@ -220,8 +221,7 @@ object StorageLevel { getCachedStorageLevel(obj) } - private[spark] val storageLevelCache = - new java.util.concurrent.ConcurrentHashMap[StorageLevel, StorageLevel]() + private[spark] val storageLevelCache = new ConcurrentHashMap[StorageLevel, StorageLevel]() private[spark] def getCachedStorageLevel(level: StorageLevel): StorageLevel = { storageLevelCache.putIfAbsent(level, level) diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 049938f827291..0c24ad2760e08 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -23,7 +23,7 @@ import org.apache.spark.storage.StorageStatusListener import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.env.{EnvironmentListener, EnvironmentTab} import org.apache.spark.ui.exec.{ExecutorsListener, ExecutorsTab} -import org.apache.spark.ui.jobs.{JobProgressListener, JobProgressTab} +import org.apache.spark.ui.jobs.{JobsTab, JobProgressListener, StagesTab} import org.apache.spark.ui.storage.{StorageListener, StorageTab} /** @@ -43,19 +43,20 @@ private[spark] class SparkUI private ( extends WebUI(securityManager, SparkUI.getUIPort(conf), conf, basePath, "SparkUI") with Logging { + val killEnabled = sc.map(_.conf.getBoolean("spark.ui.killEnabled", true)).getOrElse(false) + /** Initialize all components of the server. */ def initialize() { - val jobProgressTab = new JobProgressTab(this) - attachTab(jobProgressTab) + attachTab(new JobsTab(this)) + val stagesTab = new StagesTab(this) + attachTab(stagesTab) attachTab(new StorageTab(this)) attachTab(new EnvironmentTab(this)) attachTab(new ExecutorsTab(this)) attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) - attachHandler(createRedirectHandler("/", "/stages", basePath = basePath)) + attachHandler(createRedirectHandler("/", "/jobs", basePath = basePath)) attachHandler( - createRedirectHandler("/stages/stage/kill", "/stages", jobProgressTab.handleKillRequest)) - // If the UI is live, then serve - sc.foreach { _.env.metricsSystem.getServletHandlers.foreach(attachHandler) } + createRedirectHandler("/stages/stage/kill", "/stages", stagesTab.handleKillRequest)) } initialize() diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 7bc1e24d58711..7486cb6b1bbc0 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -26,7 +26,8 @@ import org.apache.spark.Logging /** Utility functions for generating XML pages with spark content. */ private[spark] object UIUtils extends Logging { - val TABLE_CLASS = "table table-bordered table-striped-custom table-condensed sortable" + val TABLE_CLASS_NOT_STRIPED = "table table-bordered table-condensed sortable" + val TABLE_CLASS_STRIPED = TABLE_CLASS_NOT_STRIPED + " table-striped" // SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use. private val dateFormat = new ThreadLocal[SimpleDateFormat]() { @@ -169,7 +170,8 @@ private[spark] object UIUtils extends Logging { title: String, content: => Seq[Node], activeTab: SparkUITab, - refreshInterval: Option[Int] = None): Seq[Node] = { + refreshInterval: Option[Int] = None, + helpText: Option[String] = None): Seq[Node] = { val appName = activeTab.appName val shortAppName = if (appName.length < 36) appName else appName.take(32) + "..." @@ -178,6 +180,11 @@ private[spark] object UIUtils extends Logging { {tab.name} } + val helpButton: Seq[Node] = helpText.map { helpText => + + (?) + + }.getOrElse(Seq.empty) @@ -187,9 +194,12 @@ private[spark] object UIUtils extends Logging {