diff --git a/.gitignore b/.gitignore index c67cffa1c4375..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/ +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/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/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/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/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index cdf85bfbf326f..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 @@ -169,8 +169,18 @@ span.additional-metric-title { 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, .gc_time, .deserialization_time, .serialization_time, .getting_result_time { +.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 000bbd6b532ad..5f31bfba3f8d6 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -19,6 +19,7 @@ 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 @@ -278,10 +279,12 @@ object AccumulatorParam { // 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 { @@ -293,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..49dae5231a92c 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. diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 9b0d5be7a7ab2..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() @@ -172,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() @@ -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 @@ -1630,28 +1643,28 @@ object SparkContext extends Logging { // following ones. @deprecated("Replaced by implicit objects in AccumulatorParam. This is kept here only for " + - "backward compatibility.", "1.2.0") + "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 } @deprecated("Replaced by implicit objects in AccumulatorParam. This is kept here only for " + - "backward compatibility.", "1.2.0") + "backward compatibility.", "1.3.0") object IntAccumulatorParam extends AccumulatorParam[Int] { def addInPlace(t1: Int, t2: Int): Int = t1 + t2 def zero(initialValue: Int) = 0 } @deprecated("Replaced by implicit objects in AccumulatorParam. This is kept here only for " + - "backward compatibility.", "1.2.0") + "backward compatibility.", "1.3.0") object LongAccumulatorParam extends AccumulatorParam[Long] { def addInPlace(t1: Long, t2: Long) = t1 + t2 def zero(initialValue: Long) = 0L } @deprecated("Replaced by implicit objects in AccumulatorParam. This is kept here only for " + - "backward compatibility.", "1.2.0") + "backward compatibility.", "1.3.0") object FloatAccumulatorParam extends AccumulatorParam[Float] { def addInPlace(t1: Float, t2: Float) = t1 + t2 def zero(initialValue: Float) = 0f @@ -1662,34 +1675,34 @@ object SparkContext extends Logging { // and just call the corresponding functions in `object RDD`. @deprecated("Replaced by implicit functions in the RDD companion object. This is " + - "kept here only for backward compatibility.", "1.2.0") + "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) = { RDD.rddToPairRDDFunctions(rdd) } @deprecated("Replaced by implicit functions in the RDD companion object. This is " + - "kept here only for backward compatibility.", "1.2.0") + "kept here only for backward compatibility.", "1.3.0") def rddToAsyncRDDActions[T: ClassTag](rdd: RDD[T]) = RDD.rddToAsyncRDDActions(rdd) @deprecated("Replaced by implicit functions in the RDD companion object. This is " + - "kept here only for backward compatibility.", "1.2.0") + "kept here only for backward compatibility.", "1.3.0") def rddToSequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable: ClassTag]( rdd: RDD[(K, V)]) = RDD.rddToSequenceFileRDDFunctions(rdd) @deprecated("Replaced by implicit functions in the RDD companion object. This is " + - "kept here only for backward compatibility.", "1.2.0") + "kept here only for backward compatibility.", "1.3.0") def rddToOrderedRDDFunctions[K : Ordering : ClassTag, V: ClassTag]( rdd: RDD[(K, V)]) = RDD.rddToOrderedRDDFunctions(rdd) @deprecated("Replaced by implicit functions in the RDD companion object. This is " + - "kept here only for backward compatibility.", "1.2.0") + "kept here only for backward compatibility.", "1.3.0") def doubleRDDToDoubleRDDFunctions(rdd: RDD[Double]) = RDD.doubleRDDToDoubleRDDFunctions(rdd) @deprecated("Replaced by implicit functions in the RDD companion object. This is " + - "kept here only for backward compatibility.", "1.2.0") + "kept here only for backward compatibility.", "1.3.0") def numericRDDToDoubleRDDFunctions[T](rdd: RDD[T])(implicit num: Numeric[T]) = RDD.numericRDDToDoubleRDDFunctions(rdd) @@ -1722,43 +1735,43 @@ object SparkContext extends Logging { // and just call the corresponding functions in `object WritableConverter`. @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + - "backward compatibility.", "1.2.0") + "backward compatibility.", "1.3.0") def intWritableConverter(): WritableConverter[Int] = WritableConverter.intWritableConverter() @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + - "backward compatibility.", "1.2.0") + "backward compatibility.", "1.3.0") def longWritableConverter(): WritableConverter[Long] = WritableConverter.longWritableConverter() @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + - "backward compatibility.", "1.2.0") + "backward compatibility.", "1.3.0") def doubleWritableConverter(): WritableConverter[Double] = WritableConverter.doubleWritableConverter() @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + - "backward compatibility.", "1.2.0") + "backward compatibility.", "1.3.0") def floatWritableConverter(): WritableConverter[Float] = WritableConverter.floatWritableConverter() @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + - "backward compatibility.", "1.2.0") + "backward compatibility.", "1.3.0") def booleanWritableConverter(): WritableConverter[Boolean] = WritableConverter.booleanWritableConverter() @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + - "backward compatibility.", "1.2.0") + "backward compatibility.", "1.3.0") def bytesWritableConverter(): WritableConverter[Array[Byte]] = WritableConverter.bytesWritableConverter() @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + - "backward compatibility.", "1.2.0") + "backward compatibility.", "1.3.0") def stringWritableConverter(): WritableConverter[String] = WritableConverter.stringWritableConverter() @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + - "backward compatibility.", "1.2.0") - def writableWritableConverter[T <: Writable]() = + "backward compatibility.", "1.3.0") + def writableWritableConverter[T <: Writable](): WritableConverter[T] = WritableConverter.writableWritableConverter() /** @@ -2017,15 +2030,15 @@ object WritableConverter { simpleWritableConverter[Boolean, BooleanWritable](_.get) implicit def bytesWritableConverter(): WritableConverter[Array[Byte]] = { - simpleWritableConverter[Array[Byte], BytesWritable](bw => + 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]() = + 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..6656df44d3599 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 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/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 e0bc00e1eb249..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 @@ -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 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 c46f84de8444a..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,6 +88,8 @@ 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 @@ -175,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 0c7d247519447..955cbd6dab96d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -281,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", 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 82a54dbfb5330..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,14 +17,16 @@ 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 @@ -64,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. * @@ -90,7 +98,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis initialize() - private def initialize() { + private def initialize(): Unit = { // Validate the log directory. val path = new Path(logDir) if (!fs.exists(path)) { @@ -106,8 +114,12 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis } 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 @@ -115,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) @@ -125,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 } @@ -149,41 +162,45 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis * 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(logDir)) - 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 } } @@ -217,37 +234,100 @@ 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) } @@ -256,7 +336,7 @@ private object FsHistoryProvider { } 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 5fdc350cd8512..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,13 +52,38 @@ 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 { @@ -81,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/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 7b32c505def9b..81fa0770bbaf9 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 @@ -32,6 +33,7 @@ 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, @@ -56,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 @@ -129,6 +132,10 @@ 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" => @@ -510,7 +517,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 @@ -697,6 +704,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) + } } } @@ -707,41 +719,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/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 eb11163538b20..86a87ec22235e 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 @@ -23,7 +23,7 @@ import java.text.SimpleDateFormat import java.util.{UUID, Date} 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 @@ -109,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) @@ -163,6 +165,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) { @@ -292,7 +296,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) } @@ -337,8 +341,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_ @@ -375,6 +390,7 @@ private[spark] class Worker( message.map(" message " + _).getOrElse("") + exitStatus.map(" exitStatus " + _).getOrElse("")) } + maybeCleanupApplication(appId) } case KillExecutor(masterUrl, appId, execId) => @@ -444,6 +460,9 @@ private[spark] class Worker( case ReregisterWithMaster => reregisterWithMaster() + case ApplicationFinished(id) => + finishedApps += id + maybeCleanupApplication(id) } private def masterDisconnected() { @@ -452,6 +471,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 835157fc520aa..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) } } @@ -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/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/FilteredRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala deleted file mode 100644 index 9e41b3d1e2d4f..0000000000000 --- a/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.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 scala.reflect.ClassTag - -import org.apache.spark.{Partition, TaskContext} - -private[spark] class FilteredRDD[T: ClassTag]( - prev: RDD[T], - f: T => Boolean) - extends RDD[T](prev) { - - override def getPartitions: Array[Partition] = firstParent[T].partitions - - override val partitioner = prev.partitioner // Since filter cannot change a partition's keys - - override def compute(split: Partition, context: TaskContext) = - firstParent[T].iterator(split, context).filter(f) -} 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/MappedValuesRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala deleted file mode 100644 index a60952eee5901..0000000000000 --- a/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala +++ /dev/null @@ -1,33 +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 MappedValuesRDD[K, V, U](prev: RDD[_ <: Product2[K, V]], f: V => U) - extends RDD[(K, U)](prev) { - - override def getPartitions = firstParent[Product2[K, U]].partitions - - override val partitioner = firstParent[Product2[K, U]].partitioner - - override def compute(split: Partition, context: TaskContext): Iterator[(K, U)] = { - firstParent[Product2[K, V]].iterator(split, context).map { pair => (pair._1, f(pair._2)) } - } -} 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/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 3add4a76192ca..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,7 +17,7 @@ package org.apache.spark.rdd -import java.util.{Properties, Random} +import java.util.Random import scala.collection.{mutable, Map} import scala.collection.mutable.ArrayBuffer @@ -34,16 +34,14 @@ 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} @@ -58,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: * @@ -270,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. @@ -503,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 @@ -1162,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) } @@ -1170,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) } @@ -1386,6 +1427,13 @@ abstract class RDD[T: ClassTag]( } } + +/** + * 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 @@ -1394,22 +1442,30 @@ object RDD { // 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) = { + (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]) = new AsyncRDDActions(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)]) = + rdd: RDD[(K, V)]): SequenceFileRDDFunctions[K, V] = { new SequenceFileRDDFunctions(rdd) + } - implicit def rddToOrderedRDDFunctions[K : Ordering : ClassTag, V: ClassTag]( - rdd: RDD[(K, V)]) = + 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]) = new DoubleRDDFunctions(rdd) + implicit def doubleRDDToDoubleRDDFunctions(rdd: RDD[Double]): DoubleRDDFunctions = { + new DoubleRDDFunctions(rdd) + } - implicit def numericRDDToDoubleRDDFunctions[T](rdd: RDD[T])(implicit num: Numeric[T]) = + 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/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/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/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 88b196ac64368..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) @@ -307,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 @@ -334,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/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 83170f7c5a4ab..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} @@ -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 176907dffa46a..0c24ad2760e08 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -57,8 +57,6 @@ private[spark] class SparkUI private ( attachHandler(createRedirectHandler("/", "/jobs", basePath = basePath)) attachHandler( createRedirectHandler("/stages/stage/kill", "/stages", stagesTab.handleKillRequest)) - // If the UI is live, then serve - sc.foreach { _.env.metricsSystem.getServletHandlers.foreach(attachHandler) } } 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 315327c3c6b7c..7486cb6b1bbc0 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -181,7 +181,9 @@ private[spark] object UIUtils extends Logging { } val helpButton: Seq[Node] = helpText.map { helpText => - (?) + + (?) + }.getOrElse(Seq.empty) @@ -192,9 +194,12 @@ private[spark] object UIUtils extends Logging { - } @@ -243,11 +243,6 @@ private[spark] object UIUtils extends Logging { {content} - } diff --git a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala index 18d2b5075aa08..b4677447c8872 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala @@ -20,7 +20,6 @@ package org.apache.spark.ui import scala.util.Random import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.SparkContext._ import org.apache.spark.scheduler.SchedulingMode /** diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index bfa54f8492068..09a936c2234c0 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -132,13 +132,6 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { Task Deserialization Time -
  • - - - GC Time - -
  • @@ -168,7 +161,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { ("Executor ID / Host", ""), ("Launch Time", ""), ("Duration", ""), ("Scheduler Delay", TaskDetailsClassNames.SCHEDULER_DELAY), ("Task Deserialization Time", TaskDetailsClassNames.TASK_DESERIALIZATION_TIME), - ("GC Time", TaskDetailsClassNames.GC_TIME), + ("GC Time", ""), ("Result Serialization Time", TaskDetailsClassNames.RESULT_SERIALIZATION_TIME), ("Getting Result Time", TaskDetailsClassNames.GETTING_RESULT_TIME)) ++ {if (hasAccumulators) Seq(("Accumulators", "")) else Nil} ++ @@ -308,7 +301,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { {deserializationQuantiles} - {gcQuantiles}, + {gcQuantiles}, {serializationQuantiles} , @@ -429,7 +422,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { class={TaskDetailsClassNames.TASK_DESERIALIZATION_TIME}> {UIUtils.formatDuration(taskDeserializationTime.toLong)} - + {if (gcTime > 0) UIUtils.formatDuration(gcTime) else ""} fileSystem.setPermission(path, p) } - val bstream = new BufferedOutputStream(dstream, outputBufferSize) - val cstream = if (compress) compressionCodec.compressedOutputStream(bstream) else bstream - new PrintWriter(cstream) - } - - /** - * Log the message to the given writer. - * @param msg The message to be logged - * @param withTime Whether to prepend message with a timestamp - */ - def log(msg: String, withTime: Boolean = false) { - val writeInfo = if (!withTime) { - msg - } else { - val date = new Date(System.currentTimeMillis) - dateFormat.get.format(date) + ": " + msg - } - writer.foreach(_.print(writeInfo)) - } - - /** - * Log the message to the given writer as a new line. - * @param msg The message to be logged - * @param withTime Whether to prepend message with a timestamp - */ - def logLine(msg: String, withTime: Boolean = false) = log(msg + "\n", withTime) - - /** - * Flush the writer to disk manually. - * - * When using a Hadoop filesystem, we need to invoke the hflush or sync - * method. In HDFS, hflush guarantees that the data gets to all the - * DataNodes. - */ - def flush() { - writer.foreach(_.flush()) - hadoopDataStream.foreach(hadoopFlushMethod.invoke(_)) - } - - /** - * Close the writer. Any subsequent calls to log or flush will have no effect. - */ - def close() { - writer.foreach(_.close()) - writer = None - } - - /** - * Start a writer for a new file, closing the existing one if it exists. - * @param fileName Name of the new file, defaulting to the file index if not provided. - * @param perms Permissions to put on the new file. - */ - def newFile(fileName: String = "", perms: Option[FsPermission] = None) { - fileIndex += 1 - writer.foreach(_.close()) - val name = fileName match { - case "" => fileIndex.toString - case _ => fileName - } - writer = Some(createWriter(name, perms)) - } - - /** - * Close all open writers, streams, and file systems. Any subsequent uses of this FileLogger - * instance will throw exceptions. - */ - def stop() { - hadoopDataStream.foreach(_.close()) - writer.foreach(_.close()) - } -} diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 336b0798cade9..0d771baaa6abc 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -246,8 +246,11 @@ private[spark] object Utils extends Logging { retval } - /** Create a temporary directory inside the given parent directory */ - def createTempDir(root: String = System.getProperty("java.io.tmpdir")): File = { + /** + * Create a directory inside the given parent directory. The directory is guaranteed to be + * newly created, and is not marked for automatic deletion. + */ + def createDirectory(root: String): File = { var attempts = 0 val maxAttempts = 10 var dir: File = null @@ -265,6 +268,15 @@ private[spark] object Utils extends Logging { } catch { case e: SecurityException => dir = null; } } + dir + } + + /** + * Create a temporary directory inside the given parent directory. The directory will be + * automatically deleted when the VM shuts down. + */ + def createTempDir(root: String = System.getProperty("java.io.tmpdir")): File = { + val dir = createDirectory(root) registerShutdownDeleteDir(dir) dir } @@ -385,16 +397,12 @@ private[spark] object Utils extends Logging { } finally { lock.release() } - if (targetFile.exists && !Files.equal(cachedFile, targetFile)) { - if (conf.getBoolean("spark.files.overwrite", false)) { - targetFile.delete() - logInfo((s"File $targetFile exists and does not match contents of $url, " + - s"replacing it with $url")) - } else { - throw new SparkException(s"File $targetFile exists and does not match contents of $url") - } - } - Files.copy(cachedFile, targetFile) + copyFile( + url, + cachedFile, + targetFile, + conf.getBoolean("spark.files.overwrite", false) + ) } else { doFetchFile(url, targetDir, fileName, conf, securityMgr, hadoopConf) } @@ -411,6 +419,104 @@ private[spark] object Utils extends Logging { FileUtil.chmod(targetFile.getAbsolutePath, "a+x") } + /** + * Download `in` to `tempFile`, then move it to `destFile`. + * + * If `destFile` already exists: + * - no-op if its contents equal those of `sourceFile`, + * - throw an exception if `fileOverwrite` is false, + * - attempt to overwrite it otherwise. + * + * @param url URL that `sourceFile` originated from, for logging purposes. + * @param in InputStream to download. + * @param tempFile File path to download `in` to. + * @param destFile File path to move `tempFile` to. + * @param fileOverwrite Whether to delete/overwrite an existing `destFile` that does not match + * `sourceFile` + */ + private def downloadFile( + url: String, + in: InputStream, + tempFile: File, + destFile: File, + fileOverwrite: Boolean): Unit = { + + try { + val out = new FileOutputStream(tempFile) + Utils.copyStream(in, out, closeStreams = true) + copyFile(url, tempFile, destFile, fileOverwrite, removeSourceFile = true) + } finally { + // Catch-all for the couple of cases where for some reason we didn't move `tempFile` to + // `destFile`. + if (tempFile.exists()) { + tempFile.delete() + } + } + } + + /** + * Copy `sourceFile` to `destFile`. + * + * If `destFile` already exists: + * - no-op if its contents equal those of `sourceFile`, + * - throw an exception if `fileOverwrite` is false, + * - attempt to overwrite it otherwise. + * + * @param url URL that `sourceFile` originated from, for logging purposes. + * @param sourceFile File path to copy/move from. + * @param destFile File path to copy/move to. + * @param fileOverwrite Whether to delete/overwrite an existing `destFile` that does not match + * `sourceFile` + * @param removeSourceFile Whether to remove `sourceFile` after / as part of moving/copying it to + * `destFile`. + */ + private def copyFile( + url: String, + sourceFile: File, + destFile: File, + fileOverwrite: Boolean, + removeSourceFile: Boolean = false): Unit = { + + if (destFile.exists) { + if (!Files.equal(sourceFile, destFile)) { + if (fileOverwrite) { + logInfo( + s"File $destFile exists and does not match contents of $url, replacing it with $url" + ) + if (!destFile.delete()) { + throw new SparkException( + "Failed to delete %s while attempting to overwrite it with %s".format( + destFile.getAbsolutePath, + sourceFile.getAbsolutePath + ) + ) + } + } else { + throw new SparkException( + s"File $destFile exists and does not match contents of $url") + } + } else { + // Do nothing if the file contents are the same, i.e. this file has been copied + // previously. + logInfo( + "%s has been previously copied to %s".format( + sourceFile.getAbsolutePath, + destFile.getAbsolutePath + ) + ) + return + } + } + + // The file does not exist in the target directory. Copy or move it there. + if (removeSourceFile) { + Files.move(sourceFile, destFile) + } else { + logInfo(s"Copying ${sourceFile.getAbsolutePath} to ${destFile.getAbsolutePath}") + Files.copy(sourceFile, destFile) + } + } + /** * Download a file to target directory. Supports fetching the file in a variety of ways, * including HTTP, HDFS and files on a standard filesystem, based on the URL parameter. @@ -425,8 +531,7 @@ private[spark] object Utils extends Logging { conf: SparkConf, securityMgr: SecurityManager, hadoopConf: Configuration) { - val tempDir = getLocalDir(conf) - val tempFile = File.createTempFile("fetchFileTemp", null, new File(tempDir)) + val tempFile = File.createTempFile("fetchFileTemp", null, new File(targetDir.getAbsolutePath)) val targetFile = new File(targetDir, filename) val uri = new URI(url) val fileOverwrite = conf.getBoolean("spark.files.overwrite", defaultValue = false) @@ -450,67 +555,17 @@ private[spark] object Utils extends Logging { uc.setReadTimeout(timeout) uc.connect() val in = uc.getInputStream() - val out = new FileOutputStream(tempFile) - Utils.copyStream(in, out, closeStreams = true) - if (targetFile.exists && !Files.equal(tempFile, targetFile)) { - if (fileOverwrite) { - targetFile.delete() - logInfo(("File %s exists and does not match contents of %s, " + - "replacing it with %s").format(targetFile, url, url)) - } else { - tempFile.delete() - throw new SparkException( - "File " + targetFile + " exists and does not match contents of" + " " + url) - } - } - Files.move(tempFile, targetFile) + downloadFile(url, in, tempFile, targetFile, fileOverwrite) case "file" => // In the case of a local file, copy the local file to the target directory. // Note the difference between uri vs url. val sourceFile = if (uri.isAbsolute) new File(uri) else new File(url) - var shouldCopy = true - if (targetFile.exists) { - if (!Files.equal(sourceFile, targetFile)) { - if (fileOverwrite) { - targetFile.delete() - logInfo(("File %s exists and does not match contents of %s, " + - "replacing it with %s").format(targetFile, url, url)) - } else { - throw new SparkException( - "File " + targetFile + " exists and does not match contents of" + " " + url) - } - } else { - // Do nothing if the file contents are the same, i.e. this file has been copied - // previously. - logInfo(sourceFile.getAbsolutePath + " has been previously copied to " - + targetFile.getAbsolutePath) - shouldCopy = false - } - } - - if (shouldCopy) { - // The file does not exist in the target directory. Copy it there. - logInfo("Copying " + sourceFile.getAbsolutePath + " to " + targetFile.getAbsolutePath) - Files.copy(sourceFile, targetFile) - } + copyFile(url, sourceFile, targetFile, fileOverwrite) case _ => // Use the Hadoop filesystem library, which supports file://, hdfs://, s3://, and others val fs = getHadoopFileSystem(uri, hadoopConf) val in = fs.open(new Path(uri)) - val out = new FileOutputStream(tempFile) - Utils.copyStream(in, out, closeStreams = true) - if (targetFile.exists && !Files.equal(tempFile, targetFile)) { - if (fileOverwrite) { - targetFile.delete() - logInfo(("File %s exists and does not match contents of %s, " + - "replacing it with %s").format(targetFile, url, url)) - } else { - tempFile.delete() - throw new SparkException( - "File " + targetFile + " exists and does not match contents of" + " " + url) - } - } - Files.move(tempFile, targetFile) + downloadFile(url, in, tempFile, targetFile, fileOverwrite) } } @@ -1025,13 +1080,6 @@ private[spark] object Utils extends Logging { * @param skipClass Function that is used to exclude non-user-code classes. */ def getCallSite(skipClass: String => Boolean = coreExclusionFunction): CallSite = { - val trace = Thread.currentThread.getStackTrace().filterNot { ste: StackTraceElement => - // When running under some profilers, the current stack trace might contain some bogus - // frames. This is intended to ensure that we don't crash in these situations by - // ignoring any frames that we can't examine. - ste == null || ste.getMethodName == null || ste.getMethodName.contains("getStackTrace") - } - // Keep crawling up the stack trace until we find the first function not inside of the spark // package. We track the last (shallowest) contiguous Spark method. This might be an RDD // transformation, a SparkContext function (such as parallelize), or anything else that leads @@ -1041,27 +1089,34 @@ private[spark] object Utils extends Logging { var firstUserLine = 0 var insideSpark = true var callStack = new ArrayBuffer[String]() :+ "" - - for (el <- trace) { - if (insideSpark) { - if (skipClass(el.getClassName)) { - lastSparkMethod = if (el.getMethodName == "") { - // Spark method is a constructor; get its class name - el.getClassName.substring(el.getClassName.lastIndexOf('.') + 1) + + Thread.currentThread.getStackTrace().foreach { ste: StackTraceElement => + // When running under some profilers, the current stack trace might contain some bogus + // frames. This is intended to ensure that we don't crash in these situations by + // ignoring any frames that we can't examine. + if (ste != null && ste.getMethodName != null + && !ste.getMethodName.contains("getStackTrace")) { + if (insideSpark) { + if (skipClass(ste.getClassName)) { + lastSparkMethod = if (ste.getMethodName == "") { + // Spark method is a constructor; get its class name + ste.getClassName.substring(ste.getClassName.lastIndexOf('.') + 1) + } else { + ste.getMethodName + } + callStack(0) = ste.toString // Put last Spark method on top of the stack trace. } else { - el.getMethodName + firstUserLine = ste.getLineNumber + firstUserFile = ste.getFileName + callStack += ste.toString + insideSpark = false } - callStack(0) = el.toString // Put last Spark method on top of the stack trace. } else { - firstUserLine = el.getLineNumber - firstUserFile = el.getFileName - callStack += el.toString - insideSpark = false + callStack += ste.toString } - } else { - callStack += el.toString } } + val callStackDepth = System.getProperty("spark.callstack.depth", "20").toInt CallSite( shortForm = s"$lastSparkMethod at $firstUserFile:$firstUserLine", @@ -1792,19 +1847,29 @@ private[spark] object Utils extends Logging { /** * A utility class to redirect the child process's stdout or stderr. */ -private[spark] class RedirectThread(in: InputStream, out: OutputStream, name: String) +private[spark] class RedirectThread( + in: InputStream, + out: OutputStream, + name: String, + propagateEof: Boolean = false) extends Thread(name) { setDaemon(true) override def run() { scala.util.control.Exception.ignoring(classOf[IOException]) { // FIXME: We copy the stream on the level of bytes to avoid encoding problems. - val buf = new Array[Byte](1024) - var len = in.read(buf) - while (len != -1) { - out.write(buf, 0, len) - out.flush() - len = in.read(buf) + try { + val buf = new Array[Byte](1024) + var len = in.read(buf) + while (len != -1) { + out.write(buf, 0, len) + out.flush() + len = in.read(buf) + } + } finally { + if (propagateEof) { + out.close() + } } } } diff --git a/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala index 4fa357edd6f07..2ae308dacf1ae 100644 --- a/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala @@ -25,7 +25,6 @@ import scala.reflect.ClassTag import org.apache.commons.math3.distribution.PoissonDistribution import org.apache.spark.Logging -import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD /** diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 59c86eecac5e8..e5bdad6bda2fa 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -323,6 +323,47 @@ public Boolean call(Integer x) { Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds } + @Test + public void groupByOnPairRDD() { + // Regression test for SPARK-4459 + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); + Function, Boolean> areOdd = + new Function, Boolean>() { + @Override + public Boolean call(Tuple2 x) { + return (x._1() % 2 == 0) && (x._2() % 2 == 0); + } + }; + JavaPairRDD pairRDD = rdd.zip(rdd); + JavaPairRDD>> oddsAndEvens = pairRDD.groupBy(areOdd); + Assert.assertEquals(2, oddsAndEvens.count()); + Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds + + oddsAndEvens = pairRDD.groupBy(areOdd, 1); + Assert.assertEquals(2, oddsAndEvens.count()); + Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds + } + + @SuppressWarnings("unchecked") + @Test + public void keyByOnPairRDD() { + // Regression test for SPARK-4459 + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); + Function, String> sumToString = + new Function, String>() { + @Override + public String call(Tuple2 x) { + return String.valueOf(x._1() + x._2()); + } + }; + JavaPairRDD pairRDD = rdd.zip(rdd); + JavaPairRDD> keyed = pairRDD.keyBy(sumToString); + Assert.assertEquals(7, keyed.count()); + Assert.assertEquals(1, (long) keyed.lookup("2").get(0)._1()); + } + @SuppressWarnings("unchecked") @Test public void cogroup() { @@ -1316,6 +1357,19 @@ public Tuple2 call(Integer x) { pairRDD.collectAsMap(); // Used to crash with ClassCastException } + @SuppressWarnings("unchecked") + @Test + public void collectAsMapAndSerialize() throws Exception { + JavaPairRDD rdd = + sc.parallelizePairs(Arrays.asList(new Tuple2("foo", 1))); + Map map = rdd.collectAsMap(); + ByteArrayOutputStream bytes = new ByteArrayOutputStream(); + new ObjectOutputStream(bytes).writeObject(map); + Map deserializedMap = (Map) + new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray())).readObject(); + Assert.assertEquals(1, deserializedMap.get("foo").intValue()); + } + @Test @SuppressWarnings("unchecked") public void sampleByKey() { diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 52d1d5277658e..f087fc550dde3 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -22,7 +22,6 @@ import scala.collection.mutable import org.scalatest.FunSuite import org.scalatest.Matchers -import org.apache.spark.SparkContext._ class AccumulatorSuite extends FunSuite with Matchers with LocalSparkContext { diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index a41914a1a9d0c..3b10b3a042317 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -23,7 +23,6 @@ import scala.reflect.ClassTag import org.scalatest.FunSuite -import org.apache.spark.SparkContext._ import org.apache.spark.rdd._ import org.apache.spark.storage.{BlockId, StorageLevel, TestBlockId} import org.apache.spark.util.Utils diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index 2e3fc5ef0e336..ae2ae7ed0d3aa 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -28,7 +28,6 @@ import org.scalatest.concurrent.{PatienceConfiguration, Eventually} import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ -import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD import org.apache.spark.storage._ import org.apache.spark.shuffle.hash.HashShuffleManager diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 429199f2075c6..998f3008ec0ea 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -23,7 +23,6 @@ import org.scalatest.concurrent.Timeouts._ import org.scalatest.Matchers import org.scalatest.time.{Millis, Span} -import org.apache.spark.SparkContext._ import org.apache.spark.storage.{RDDBlockId, StorageLevel} class NotSerializableClass diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index ce804f94f3267..c817f6dcede75 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -35,6 +35,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { .setMaster("local") .setAppName("test-executor-allocation-manager") .set("spark.dynamicAllocation.enabled", "true") + .set("spark.dynamicAllocation.testing", "true") intercept[SparkException] { new SparkContext(conf) } SparkEnv.get.stop() // cleanup the created environment SparkContext.clearActiveContext() diff --git a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala index 55799f55146cb..bac6fdbcdc976 100644 --- a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala @@ -17,11 +17,8 @@ package org.apache.spark -import java.util.concurrent.atomic.AtomicInteger - import org.scalatest.BeforeAndAfterAll -import org.apache.spark.SparkContext._ import org.apache.spark.network.TransportContext import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.server.TransportServer diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index 2229e6acc425d..1212d0b43207d 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark import org.scalatest.FunSuite -import org.apache.spark.SparkContext._ import org.apache.spark.util.NonSerializable import java.io.NotSerializableException diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index 379c2a6ea4b55..49426545c767e 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -23,7 +23,6 @@ import java.util.jar.{JarEntry, JarOutputStream} import com.google.common.io.ByteStreams import org.scalatest.FunSuite -import org.apache.spark.SparkContext._ import org.apache.spark.util.Utils class FileServerSuite extends FunSuite with LocalSparkContext { diff --git a/core/src/test/scala/org/apache/spark/FutureActionSuite.scala b/core/src/test/scala/org/apache/spark/FutureActionSuite.scala index db9c25fc457a4..f5cdb01ec9504 100644 --- a/core/src/test/scala/org/apache/spark/FutureActionSuite.scala +++ b/core/src/test/scala/org/apache/spark/FutureActionSuite.scala @@ -22,7 +22,6 @@ import scala.concurrent.duration.Duration import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} -import org.apache.spark.SparkContext._ class FutureActionSuite extends FunSuite with BeforeAndAfter with Matchers with LocalSparkContext { diff --git a/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala b/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala index 8e4a9e2c9f56c..d895230ecf330 100644 --- a/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala +++ b/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark import org.scalatest.FunSuite import org.apache.spark.rdd.RDD -import org.apache.spark.SparkContext._ class ImplicitOrderingSuite extends FunSuite with LocalSparkContext { // Tests that PairRDDFunctions grabs an implicit Ordering in various cases where it should. diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index a57430e829ced..41ed2bce55ce1 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -27,7 +27,6 @@ import scala.concurrent.future import org.scalatest.{BeforeAndAfter, FunSuite} import org.scalatest.Matchers -import org.apache.spark.SparkContext._ import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskStart} /** diff --git a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala index 646ede30ae6ff..b7532314ada01 100644 --- a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala +++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala @@ -22,7 +22,6 @@ import scala.math.abs import org.scalatest.{FunSuite, PrivateMethodTester} -import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD import org.apache.spark.util.StatCounter diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 5d20b4dc1561a..58a96245a9b53 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -20,10 +20,10 @@ package org.apache.spark import org.scalatest.FunSuite import org.scalatest.Matchers -import org.apache.spark.SparkContext._ import org.apache.spark.ShuffleSuite.NonJavaSerializableClass import org.apache.spark.rdd.{CoGroupedRDD, OrderedRDDFunctions, RDD, ShuffledRDD, SubtractedRDD} import org.apache.spark.serializer.KryoSerializer +import org.apache.spark.storage.{ShuffleDataBlockId, ShuffleBlockId} import org.apache.spark.util.MutablePair abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext { @@ -264,6 +264,28 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex } } } + + test("[SPARK-4085] rerun map stage if reduce stage cannot find its local shuffle file") { + val myConf = conf.clone().set("spark.test.noStageRetry", "false") + sc = new SparkContext("local", "test", myConf) + val rdd = sc.parallelize(1 to 10, 2).map((_, 1)).reduceByKey(_ + _) + rdd.count() + + // Delete one of the local shuffle blocks. + val hashFile = sc.env.blockManager.diskBlockManager.getFile(new ShuffleBlockId(0, 0, 0)) + val sortFile = sc.env.blockManager.diskBlockManager.getFile(new ShuffleDataBlockId(0, 0, 0)) + assert(hashFile.exists() || sortFile.exists()) + + if (hashFile.exists()) { + hashFile.delete() + } + if (sortFile.exists()) { + sortFile.delete() + } + + // This count should retry the execution of the previous stage and rerun shuffle. + rdd.count() + } } object ShuffleSuite { diff --git a/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala index 8577e4ac7e33e..41d6ea29d5b06 100644 --- a/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala @@ -25,7 +25,6 @@ import org.scalatest.{Matchers, FunSuite} import org.scalatest.concurrent.Eventually._ import org.apache.spark.JobExecutionStatus._ -import org.apache.spark.SparkContext._ class StatusTrackerSuite extends FunSuite with Matchers with LocalSparkContext { diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 3f1cd0752e766..aa65f7e8915e6 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -119,7 +119,7 @@ class JsonProtocolSuite extends FunSuite { def createExecutorRunner(): ExecutorRunner = { new ExecutorRunner("appId", 123, createAppDesc(), 4, 1234, null, "workerId", "host", new File("sparkHome"), new File("workDir"), "akka://worker", - new SparkConf, ExecutorState.RUNNING) + new SparkConf, Seq("localDir"), ExecutorState.RUNNING) } def createDriverRunner(): DriverRunner = { diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala new file mode 100644 index 0000000000000..d719e9301f4fd --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -0,0 +1,185 @@ +/* + * 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.deploy.history + +import java.io.{File, FileOutputStream, OutputStreamWriter} + +import scala.io.Source + +import com.google.common.io.Files +import org.apache.hadoop.fs.Path +import org.json4s.jackson.JsonMethods._ +import org.scalatest.{BeforeAndAfter, FunSuite} +import org.scalatest.Matchers + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.io._ +import org.apache.spark.scheduler._ +import org.apache.spark.util.{JsonProtocol, Utils} + +class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers with Logging { + + private var testDir: File = null + + private var provider: FsHistoryProvider = null + + before { + testDir = Utils.createTempDir() + provider = new FsHistoryProvider(new SparkConf() + .set("spark.history.fs.logDirectory", testDir.getAbsolutePath()) + .set("spark.history.fs.updateInterval", "0")) + } + + after { + Utils.deleteRecursively(testDir) + } + + test("Parse new and old application logs") { + val conf = new SparkConf() + .set("spark.history.fs.logDirectory", testDir.getAbsolutePath()) + .set("spark.history.fs.updateInterval", "0") + val provider = new FsHistoryProvider(conf) + + // Write a new-style application log. + val logFile1 = new File(testDir, "new1") + writeFile(logFile1, true, None, + SparkListenerApplicationStart("app1-1", None, 1L, "test"), + SparkListenerApplicationEnd(2L) + ) + + // Write an unfinished app, new-style. + writeFile(new File(testDir, "new2" + EventLoggingListener.IN_PROGRESS), true, None, + SparkListenerApplicationStart("app2-2", None, 1L, "test") + ) + + // Write an old-style application log. + val oldLog = new File(testDir, "old1") + oldLog.mkdir() + createEmptyFile(new File(oldLog, provider.SPARK_VERSION_PREFIX + "1.0")) + writeFile(new File(oldLog, provider.LOG_PREFIX + "1"), false, None, + SparkListenerApplicationStart("app3", None, 2L, "test"), + SparkListenerApplicationEnd(3L) + ) + createEmptyFile(new File(oldLog, provider.APPLICATION_COMPLETE)) + + // Write an unfinished app, old-style. + val oldLog2 = new File(testDir, "old2") + oldLog2.mkdir() + createEmptyFile(new File(oldLog2, provider.SPARK_VERSION_PREFIX + "1.0")) + writeFile(new File(oldLog2, provider.LOG_PREFIX + "1"), false, None, + SparkListenerApplicationStart("app4", None, 2L, "test") + ) + + // Force a reload of data from the log directory, and check that both logs are loaded. + // Take the opportunity to check that the offset checks work as expected. + provider.checkForLogs() + + val list = provider.getListing().toSeq + list should not be (null) + list.size should be (2) + + list(0) should be (ApplicationHistoryInfo(oldLog.getName(), "app3", 2L, 3L, + oldLog.lastModified(), "test")) + list(1) should be (ApplicationHistoryInfo(logFile1.getName(), "app1-1", 1L, 2L, + logFile1.lastModified(), "test")) + + // Make sure the UI can be rendered. + list.foreach { case info => + val appUi = provider.getAppUI(info.id) + appUi should not be null + } + } + + test("Parse legacy logs with compression codec set") { + val testCodecs = List((classOf[LZFCompressionCodec].getName(), true), + (classOf[SnappyCompressionCodec].getName(), true), + ("invalid.codec", false)) + + testCodecs.foreach { case (codecName, valid) => + val codec = if (valid) CompressionCodec.createCodec(new SparkConf(), codecName) else null + val logDir = new File(testDir, codecName) + logDir.mkdir() + createEmptyFile(new File(logDir, provider.SPARK_VERSION_PREFIX + "1.0")) + writeFile(new File(logDir, provider.LOG_PREFIX + "1"), false, Option(codec), + SparkListenerApplicationStart("app2", None, 2L, "test"), + SparkListenerApplicationEnd(3L) + ) + createEmptyFile(new File(logDir, provider.COMPRESSION_CODEC_PREFIX + codecName)) + + val logPath = new Path(logDir.getAbsolutePath()) + try { + val (logInput, sparkVersion) = provider.openLegacyEventLog(logPath) + try { + Source.fromInputStream(logInput).getLines().toSeq.size should be (2) + } finally { + logInput.close() + } + } catch { + case e: IllegalArgumentException => + valid should be (false) + } + } + } + + test("SPARK-3697: ignore directories that cannot be read.") { + val logFile1 = new File(testDir, "new1") + writeFile(logFile1, true, None, + SparkListenerApplicationStart("app1-1", None, 1L, "test"), + SparkListenerApplicationEnd(2L) + ) + val logFile2 = new File(testDir, "new2") + writeFile(logFile2, true, None, + SparkListenerApplicationStart("app1-2", None, 1L, "test"), + SparkListenerApplicationEnd(2L) + ) + logFile2.setReadable(false, false) + + val conf = new SparkConf() + .set("spark.history.fs.logDirectory", testDir.getAbsolutePath()) + .set("spark.history.fs.updateInterval", "0") + val provider = new FsHistoryProvider(conf) + provider.checkForLogs() + + val list = provider.getListing().toSeq + list should not be (null) + list.size should be (1) + } + + private def writeFile(file: File, isNewFormat: Boolean, codec: Option[CompressionCodec], + events: SparkListenerEvent*) = { + val out = + if (isNewFormat) { + EventLoggingListener.initEventLog(new FileOutputStream(file), codec) + } else { + val fileStream = new FileOutputStream(file) + codec.map(_.compressedOutputStream(fileStream)).getOrElse(fileStream) + } + val writer = new OutputStreamWriter(out, "UTF-8") + try { + events.foreach(e => writer.write(compact(render(JsonProtocol.sparkEventToJson(e))) + "\n")) + } finally { + writer.close() + } + } + + private def createEmptyFile(file: File) = { + new FileOutputStream(file).close() + } + +} diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index 196217062991e..6f233d7cf97aa 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -33,7 +33,8 @@ class ExecutorRunnerTest extends FunSuite { val appDesc = new ApplicationDescription("app name", Some(8), 500, Command("foo", Seq(appId), Map(), Seq(), Seq(), Seq()), "appUiUrl") val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", - new File(sparkHome), new File("ooga"), "blah", new SparkConf, ExecutorState.RUNNING) + new File(sparkHome), new File("ooga"), "blah", new SparkConf, Seq("localDir"), + ExecutorState.RUNNING) val builder = CommandUtils.buildProcessBuilder(appDesc.command, 512, sparkHome, er.substituteVariables) assert(builder.command().last === appId) } diff --git a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala index 25be7f25c21bb..8c6035fb367fe 100644 --- a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala +++ b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala @@ -85,4 +85,10 @@ class CompressionCodecSuite extends FunSuite { assert(codec.getClass === classOf[SnappyCompressionCodec]) testCodec(codec) } + + test("bad compression codec") { + intercept[IllegalArgumentException] { + CompressionCodec.createCodec(conf, "foobar") + } + } } diff --git a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala index 3b833f2e41867..f2b0ea1063a72 100644 --- a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala @@ -27,7 +27,6 @@ import org.scalatest.{BeforeAndAfterAll, FunSuite} import org.scalatest.concurrent.Timeouts import org.scalatest.time.SpanSugar._ -import org.apache.spark.SparkContext._ import org.apache.spark.{SparkContext, SparkException, LocalSparkContext} class AsyncRDDActionsSuite extends FunSuite with BeforeAndAfterAll with Timeouts { diff --git a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala index f89bdb6e07dea..de306533752c1 100644 --- a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark.rdd import org.scalatest.FunSuite import org.apache.spark._ -import org.apache.spark.SparkContext._ class DoubleRDDSuite extends FunSuite with SharedSparkContext { // Verify tests on the histogram functionality. We test with both evenly diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 3620e251cc139..108f70af43f37 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -29,7 +29,6 @@ import org.apache.hadoop.mapreduce.{JobContext => NewJobContext, OutputCommitter OutputFormat => NewOutputFormat, RecordWriter => NewRecordWriter, TaskAttemptContext => NewTaskAttempContext} import org.apache.spark.{Partitioner, SharedSparkContext} -import org.apache.spark.SparkContext._ import org.apache.spark.util.Utils import org.scalatest.FunSuite diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index e079ca3b1e896..6836e9ab0fd6b 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -24,11 +24,9 @@ import scala.reflect.ClassTag import org.scalatest.FunSuite import org.apache.spark._ -import org.apache.spark.SparkContext._ -import org.apache.spark.util.Utils - import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.rdd.RDDSuiteUtils._ +import org.apache.spark.util.Utils class RDDSuite extends FunSuite with SharedSparkContext { @@ -38,8 +36,8 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(nums.toLocalIterator.toList === List(1, 2, 3, 4)) val dups = sc.makeRDD(Array(1, 1, 2, 2, 3, 3, 4, 4), 2) assert(dups.distinct().count() === 4) - assert(dups.distinct.count === 4) // Can distinct and count be called without parentheses? - assert(dups.distinct.collect === dups.distinct().collect) + assert(dups.distinct().count === 4) // Can distinct and count be called without parentheses? + assert(dups.distinct().collect === dups.distinct().collect) assert(dups.distinct(2).collect === dups.distinct().collect) assert(nums.reduce(_ + _) === 10) assert(nums.fold(0)(_ + _) === 10) @@ -97,7 +95,6 @@ class RDDSuite extends FunSuite with SharedSparkContext { } test("partitioner aware union") { - import SparkContext._ def makeRDDWithPartitioner(seq: Seq[Int]) = { sc.makeRDD(seq, 1) .map(x => (x, null)) @@ -297,7 +294,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("coalesced RDDs with locality") { val data3 = sc.makeRDD(List((1,List("a","c")), (2,List("a","b","c")), (3,List("b")))) val coal3 = data3.coalesce(3) - val list3 = coal3.partitions.map(p => p.asInstanceOf[CoalescedRDDPartition].preferredLocation) + val list3 = coal3.partitions.flatMap(_.asInstanceOf[CoalescedRDDPartition].preferredLocation) assert(list3.sorted === Array("a","b","c"), "Locality preferences are dropped") // RDD with locality preferences spread (non-randomly) over 6 machines, m0 through m5 @@ -619,9 +616,9 @@ class RDDSuite extends FunSuite with SharedSparkContext { for(seed <- 1 to 5) { val splits = data.randomSplit(Array(1.0, 2.0, 3.0), seed) assert(splits.size == 3, "wrong number of splits") - assert(splits.flatMap(_.collect).sorted.toList == data.collect.toList, + assert(splits.flatMap(_.collect()).sorted.toList == data.collect().toList, "incomplete or wrong split") - val s = splits.map(_.count) + val s = splits.map(_.count()) assert(math.abs(s(0) - 100) < 50) // std = 9.13 assert(math.abs(s(1) - 200) < 50) // std = 11.55 assert(math.abs(s(2) - 300) < 50) // std = 12.25 @@ -764,8 +761,8 @@ class RDDSuite extends FunSuite with SharedSparkContext { val rdd3 = rdd2.map(_ + 1) val rdd4 = new UnionRDD(sc, List(rdd1, rdd2, rdd3)) assert(rdd4.parent(0).isInstanceOf[ParallelCollectionRDD[_]]) - assert(rdd4.parent(1).isInstanceOf[FilteredRDD[_]]) - assert(rdd4.parent(2).isInstanceOf[MappedRDD[_, _]]) + assert(rdd4.parent[Int](1) === rdd2) + assert(rdd4.parent[Int](2) === rdd3) } test("getNarrowAncestors") { @@ -783,20 +780,18 @@ class RDDSuite extends FunSuite with SharedSparkContext { // Simple dependency tree with a single branch assert(ancestors1.size === 0) assert(ancestors2.size === 2) - assert(ancestors2.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 1) - assert(ancestors2.count(_.isInstanceOf[FilteredRDD[_]]) === 1) + assert(ancestors2.count(_ === rdd1) === 1) + assert(ancestors2.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 1) assert(ancestors3.size === 5) - assert(ancestors3.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 1) - assert(ancestors3.count(_.isInstanceOf[FilteredRDD[_]]) === 2) - assert(ancestors3.count(_.isInstanceOf[MappedRDD[_, _]]) === 2) + assert(ancestors3.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 4) // Any ancestors before the shuffle are not considered assert(ancestors4.size === 0) assert(ancestors4.count(_.isInstanceOf[ShuffledRDD[_, _, _]]) === 0) assert(ancestors5.size === 3) assert(ancestors5.count(_.isInstanceOf[ShuffledRDD[_, _, _]]) === 1) - assert(ancestors5.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 0) - assert(ancestors5.count(_.isInstanceOf[MappedValuesRDD[_, _, _]]) === 2) + assert(ancestors5.count(_ === rdd3) === 0) + assert(ancestors5.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 2) } test("getNarrowAncestors with multiple parents") { @@ -817,16 +812,16 @@ class RDDSuite extends FunSuite with SharedSparkContext { // Simple dependency tree with multiple branches assert(ancestors6.size === 3) assert(ancestors6.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 2) - assert(ancestors6.count(_.isInstanceOf[MappedRDD[_, _]]) === 1) + assert(ancestors6.count(_ === rdd2) === 1) assert(ancestors7.size === 5) assert(ancestors7.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 3) - assert(ancestors7.count(_.isInstanceOf[MappedRDD[_, _]]) === 1) - assert(ancestors7.count(_.isInstanceOf[FilteredRDD[_]]) === 1) + assert(ancestors7.count(_ === rdd2) === 1) + assert(ancestors7.count(_ === rdd3) === 1) // Dependency tree with duplicate nodes (e.g. rdd1 should not be reported twice) assert(ancestors8.size === 7) - assert(ancestors8.count(_.isInstanceOf[MappedRDD[_, _]]) === 1) - assert(ancestors8.count(_.isInstanceOf[FilteredRDD[_]]) === 1) + assert(ancestors8.count(_ === rdd2) === 1) + assert(ancestors8.count(_ === rdd3) === 1) assert(ancestors8.count(_.isInstanceOf[UnionRDD[_]]) === 2) assert(ancestors8.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 3) assert(ancestors8.count(_ == rdd1) === 1) @@ -836,7 +831,6 @@ class RDDSuite extends FunSuite with SharedSparkContext { // Any ancestors before the shuffle are not considered assert(ancestors9.size === 2) assert(ancestors9.count(_.isInstanceOf[CoGroupedRDD[_]]) === 1) - assert(ancestors9.count(_.isInstanceOf[MappedValuesRDD[_, _, _]]) === 1) } /** @@ -870,12 +864,10 @@ class RDDSuite extends FunSuite with SharedSparkContext { val ancestors3 = rdd3.getNarrowAncestors val ancestors4 = rdd4.getNarrowAncestors assert(ancestors3.size === 4) - assert(ancestors3.count(_.isInstanceOf[MappedRDD[_, _]]) === 2) - assert(ancestors3.count(_.isInstanceOf[FilteredRDD[_]]) === 2) + assert(ancestors3.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 4) assert(ancestors3.count(_ == rdd3) === 0) assert(ancestors4.size === 4) - assert(ancestors4.count(_.isInstanceOf[MappedRDD[_, _]]) === 2) - assert(ancestors4.count(_.isInstanceOf[FilteredRDD[_]]) === 1) + assert(ancestors4.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 3) assert(ancestors4.count(_.isInstanceOf[CyclicalDependencyRDD[_]]) === 1) assert(ancestors4.count(_ == rdd3) === 1) assert(ancestors4.count(_ == rdd4) === 0) @@ -883,8 +875,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { // Cycles that do not involve the root val ancestors5 = rdd5.getNarrowAncestors assert(ancestors5.size === 6) - assert(ancestors5.count(_.isInstanceOf[MappedRDD[_, _]]) === 3) - assert(ancestors5.count(_.isInstanceOf[FilteredRDD[_]]) === 2) + assert(ancestors5.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 5) assert(ancestors5.count(_.isInstanceOf[CyclicalDependencyRDD[_]]) === 1) assert(ancestors4.count(_ == rdd3) === 1) @@ -892,8 +883,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { val ancestors6 = rdd6.getNarrowAncestors assert(ancestors6.size === 12) assert(ancestors6.count(_.isInstanceOf[UnionRDD[_]]) === 2) - assert(ancestors6.count(_.isInstanceOf[MappedRDD[_, _]]) === 4) - assert(ancestors6.count(_.isInstanceOf[FilteredRDD[_]]) === 3) + assert(ancestors6.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 7) assert(ancestors6.count(_.isInstanceOf[CyclicalDependencyRDD[_]]) === 3) } diff --git a/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala b/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala index 656917628f7a8..a40f2ffeffdf9 100644 --- a/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala @@ -21,7 +21,6 @@ import org.scalatest.FunSuite import org.scalatest.Matchers import org.apache.spark.{Logging, SharedSparkContext} -import org.apache.spark.SparkContext._ class SortingSuite extends FunSuite with SharedSparkContext with Matchers with Logging { diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index bdd721dc7eaf7..436eea4f1fdcf 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -27,7 +27,6 @@ import org.scalatest.concurrent.Timeouts import org.scalatest.time.SpanSugar._ import org.apache.spark._ -import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index abc300fcffaf9..1de7e130039a5 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -17,69 +17,59 @@ package org.apache.spark.scheduler +import java.io.{File, FileOutputStream, InputStream, IOException} + import scala.collection.mutable import scala.io.Source -import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.fs.Path import org.json4s.jackson.JsonMethods._ import org.scalatest.{BeforeAndAfter, FunSuite} -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.{Logging, SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.io.CompressionCodec -import org.apache.spark.SPARK_VERSION +import org.apache.spark.io._ import org.apache.spark.util.{JsonProtocol, Utils} -import java.io.File - /** * Test whether EventLoggingListener logs events properly. * - * This tests whether EventLoggingListener actually creates special files while logging events, - * whether the parsing of these special files is correct, and whether the logged events can be - * read and deserialized into actual SparkListenerEvents. + * This tests whether EventLoggingListener actually log files with expected name patterns while + * logging events, whether the parsing of the file names is correct, and whether the logged events + * can be read and deserialized into actual SparkListenerEvents. */ -class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { +class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Logging { + import EventLoggingListenerSuite._ + private val fileSystem = Utils.getHadoopFileSystem("/", SparkHadoopUtil.get.newConfiguration(new SparkConf())) - private val allCompressionCodecs = Seq[String]( - "org.apache.spark.io.LZFCompressionCodec", - "org.apache.spark.io.SnappyCompressionCodec" - ) private var testDir: File = _ - private var logDirPath: Path = _ + private var testDirPath: Path = _ before { testDir = Utils.createTempDir() - logDirPath = Utils.getFilePath(testDir, "spark-events") + testDir.deleteOnExit() + testDirPath = new Path(testDir.getAbsolutePath()) } after { Utils.deleteRecursively(testDir) } - test("Parse names of special files") { - testParsingFileName() - } - - test("Verify special files exist") { - testSpecialFilesExist() - } - - test("Verify special files exist with compression") { - allCompressionCodecs.foreach { codec => - testSpecialFilesExist(compressionCodec = Some(codec)) - } - } + test("Verify log file exist") { + // Verify logging directory exists + val conf = getLoggingConf(testDirPath) + val eventLogger = new EventLoggingListener("test", testDirPath.toUri().toString(), conf) + eventLogger.start() - test("Parse event logging info") { - testParsingLogInfo() - } + val logPath = new Path(eventLogger.logPath + EventLoggingListener.IN_PROGRESS) + assert(fileSystem.exists(logPath)) + val logStatus = fileSystem.getFileStatus(logPath) + assert(!logStatus.isDir) - test("Parse event logging info with compression") { - allCompressionCodecs.foreach { codec => - testParsingLogInfo(compressionCodec = Some(codec)) - } + // Verify log is renamed after stop() + eventLogger.stop() + assert(!fileSystem.getFileStatus(new Path(eventLogger.logPath)).isDir) } test("Basic event logging") { @@ -87,7 +77,7 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { } test("Basic event logging with compression") { - allCompressionCodecs.foreach { codec => + CompressionCodec.ALL_COMPRESSION_CODECS.foreach { codec => testEventLogging(compressionCodec = Some(codec)) } } @@ -97,11 +87,25 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { } test("End-to-end event logging with compression") { - allCompressionCodecs.foreach { codec => + CompressionCodec.ALL_COMPRESSION_CODECS.foreach { codec => testApplicationEventLogging(compressionCodec = Some(codec)) } } + test("Log overwriting") { + val log = new FileOutputStream(new File(testDir, "test")) + log.close() + try { + testEventLogging() + assert(false) + } catch { + case e: IOException => + // Expected, since we haven't enabled log overwrite. + } + + // Try again, but enable overwriting. + testEventLogging(extraConf = Map("spark.eventLog.overwrite" -> "true")) + } /* ----------------- * * Actual test logic * @@ -109,130 +113,19 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { import EventLoggingListenerSuite._ - /** - * Test whether names of special files are correctly identified and parsed. - */ - private def testParsingFileName() { - val logPrefix = EventLoggingListener.LOG_PREFIX - val sparkVersionPrefix = EventLoggingListener.SPARK_VERSION_PREFIX - val compressionCodecPrefix = EventLoggingListener.COMPRESSION_CODEC_PREFIX - val applicationComplete = EventLoggingListener.APPLICATION_COMPLETE - assert(EventLoggingListener.isEventLogFile(logPrefix + "0")) - assert(EventLoggingListener.isEventLogFile(logPrefix + "100")) - assert(EventLoggingListener.isEventLogFile(logPrefix + "ANYTHING")) - assert(EventLoggingListener.isSparkVersionFile(sparkVersionPrefix + "0.9.1")) - assert(EventLoggingListener.isSparkVersionFile(sparkVersionPrefix + "1.0.0")) - assert(EventLoggingListener.isSparkVersionFile(sparkVersionPrefix + "ANYTHING")) - assert(EventLoggingListener.isApplicationCompleteFile(applicationComplete)) - allCompressionCodecs.foreach { codec => - assert(EventLoggingListener.isCompressionCodecFile(compressionCodecPrefix + codec)) - } - - // Negatives - assert(!EventLoggingListener.isEventLogFile("The greatest man of all mankind")) - assert(!EventLoggingListener.isSparkVersionFile("Will never falter in the face of death!")) - assert(!EventLoggingListener.isCompressionCodecFile("Unless he chooses to leave behind")) - assert(!EventLoggingListener.isApplicationCompleteFile("The very treasure he calls Macbeth")) - - // Verify that parsing is correct - assert(EventLoggingListener.parseSparkVersion(sparkVersionPrefix + "1.0.0") === "1.0.0") - allCompressionCodecs.foreach { codec => - assert(EventLoggingListener.parseCompressionCodec(compressionCodecPrefix + codec) === codec) - } - } - - /** - * Test whether the special files produced by EventLoggingListener exist. - * - * There should be exactly one event log and one spark version file throughout the entire - * execution. If a compression codec is specified, then the compression codec file should - * also exist. Only after the application has completed does the test expect the application - * completed file to be present. - */ - private def testSpecialFilesExist(compressionCodec: Option[String] = None) { - - def assertFilesExist(logFiles: Array[FileStatus], loggerStopped: Boolean) { - val numCompressionCodecFiles = if (compressionCodec.isDefined) 1 else 0 - val numApplicationCompleteFiles = if (loggerStopped) 1 else 0 - assert(logFiles.size === 2 + numCompressionCodecFiles + numApplicationCompleteFiles) - assert(eventLogsExist(logFiles)) - assert(sparkVersionExists(logFiles)) - assert(compressionCodecExists(logFiles) === compressionCodec.isDefined) - assert(applicationCompleteExists(logFiles) === loggerStopped) - assertSparkVersionIsValid(logFiles) - compressionCodec.foreach { codec => - assertCompressionCodecIsValid(logFiles, codec) - } - } - - // Verify logging directory exists - val conf = getLoggingConf(logDirPath, compressionCodec) - val logBaseDir = conf.get("spark.eventLog.dir") - val appId = EventLoggingListenerSuite.getUniqueApplicationId - val eventLogger = new EventLoggingListener(appId, logBaseDir, conf) - eventLogger.start() - val logPath = new Path(eventLogger.logDir) - assert(fileSystem.exists(logPath)) - val logDir = fileSystem.getFileStatus(logPath) - assert(logDir.isDir) - - // Verify special files are as expected before stop() - var logFiles = fileSystem.listStatus(logPath) - assert(logFiles != null) - assertFilesExist(logFiles, loggerStopped = false) - - // Verify special files are as expected after stop() - eventLogger.stop() - logFiles = fileSystem.listStatus(logPath) - assertFilesExist(logFiles, loggerStopped = true) - } - - /** - * Test whether EventLoggingListener correctly parses the correct information from the logs. - * - * This includes whether it returns the correct Spark version, compression codec (if any), - * and the application's completion status. - */ - private def testParsingLogInfo(compressionCodec: Option[String] = None) { - - def assertInfoCorrect(info: EventLoggingInfo, loggerStopped: Boolean) { - assert(info.logPaths.size > 0) - assert(info.sparkVersion === SPARK_VERSION) - assert(info.compressionCodec.isDefined === compressionCodec.isDefined) - info.compressionCodec.foreach { codec => - assert(compressionCodec.isDefined) - val expectedCodec = compressionCodec.get.split('.').last - assert(codec.getClass.getSimpleName === expectedCodec) - } - assert(info.applicationComplete === loggerStopped) - } - - // Verify that all information is correctly parsed before stop() - val conf = getLoggingConf(logDirPath, compressionCodec) - val logBaseDir = conf.get("spark.eventLog.dir") - val appId = EventLoggingListenerSuite.getUniqueApplicationId - val eventLogger = new EventLoggingListener(appId, logBaseDir, conf) - eventLogger.start() - var eventLoggingInfo = EventLoggingListener.parseLoggingInfo(eventLogger.logDir, fileSystem) - assertInfoCorrect(eventLoggingInfo, loggerStopped = false) - - // Verify that all information is correctly parsed after stop() - eventLogger.stop() - eventLoggingInfo = EventLoggingListener.parseLoggingInfo(eventLogger.logDir, fileSystem) - assertInfoCorrect(eventLoggingInfo, loggerStopped = true) - } - /** * Test basic event logging functionality. * * This creates two simple events, posts them to the EventLoggingListener, and verifies that * exactly these two events are logged in the expected file. */ - private def testEventLogging(compressionCodec: Option[String] = None) { - val conf = getLoggingConf(logDirPath, compressionCodec) - val logBaseDir = conf.get("spark.eventLog.dir") - val appId = EventLoggingListenerSuite.getUniqueApplicationId - val eventLogger = new EventLoggingListener(appId, logBaseDir, conf) + private def testEventLogging( + compressionCodec: Option[String] = None, + extraConf: Map[String, String] = Map()) { + val conf = getLoggingConf(testDirPath, compressionCodec) + extraConf.foreach { case (k, v) => conf.set(k, v) } + val logName = compressionCodec.map("test-" + _).getOrElse("test") + val eventLogger = new EventLoggingListener(logName, testDirPath.toUri().toString(), conf) val listenerBus = new LiveListenerBus val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None, 125L, "Mickey") @@ -244,17 +137,21 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { listenerBus.addListener(eventLogger) listenerBus.postToAll(applicationStart) listenerBus.postToAll(applicationEnd) + eventLogger.stop() // Verify file contains exactly the two events logged - val eventLoggingInfo = EventLoggingListener.parseLoggingInfo(eventLogger.logDir, fileSystem) - assert(eventLoggingInfo.logPaths.size > 0) - val lines = readFileLines(eventLoggingInfo.logPaths.head, eventLoggingInfo.compressionCodec) - assert(lines.size === 2) - assert(lines(0).contains("SparkListenerApplicationStart")) - assert(lines(1).contains("SparkListenerApplicationEnd")) - assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === applicationStart) - assert(JsonProtocol.sparkEventFromJson(parse(lines(1))) === applicationEnd) - eventLogger.stop() + val (logData, version) = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), + fileSystem) + try { + val lines = readLines(logData) + assert(lines.size === 2) + assert(lines(0).contains("SparkListenerApplicationStart")) + assert(lines(1).contains("SparkListenerApplicationEnd")) + assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === applicationStart) + assert(JsonProtocol.sparkEventFromJson(parse(lines(1))) === applicationEnd) + } finally { + logData.close() + } } /** @@ -262,12 +159,12 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { * This runs a simple Spark job and asserts that the expected events are logged when expected. */ private def testApplicationEventLogging(compressionCodec: Option[String] = None) { - val conf = getLoggingConf(logDirPath, compressionCodec) + val conf = getLoggingConf(testDirPath, compressionCodec) val sc = new SparkContext("local", "test", conf) assert(sc.eventLogger.isDefined) val eventLogger = sc.eventLogger.get - val expectedLogDir = logDirPath.toString - assert(eventLogger.logDir.contains(expectedLogDir)) + val expectedLogDir = testDir.toURI().toString() + assert(eventLogger.logPath.startsWith(expectedLogDir + "/")) // Begin listening for events that trigger asserts val eventExistenceListener = new EventExistenceListener(eventLogger) @@ -279,16 +176,22 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { // Ensure all asserts have actually been triggered eventExistenceListener.assertAllCallbacksInvoked() - } - /** - * Assert that all of the specified events are logged by the given EventLoggingListener. - */ - private def assertEventsExist(eventLogger: EventLoggingListener, events: Seq[String]) { - val eventLoggingInfo = EventLoggingListener.parseLoggingInfo(eventLogger.logDir, fileSystem) - assert(eventLoggingInfo.logPaths.size > 0) - val lines = readFileLines(eventLoggingInfo.logPaths.head, eventLoggingInfo.compressionCodec) - val eventSet = mutable.Set(events: _*) + // Make sure expected events exist in the log file. + val (logData, version) = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), + fileSystem) + val lines = readLines(logData) + val eventSet = mutable.Set( + SparkListenerApplicationStart, + SparkListenerBlockManagerAdded, + SparkListenerEnvironmentUpdate, + SparkListenerJobStart, + SparkListenerJobEnd, + SparkListenerStageSubmitted, + SparkListenerStageCompleted, + SparkListenerTaskStart, + SparkListenerTaskEnd, + SparkListenerApplicationEnd).map(Utils.getFormattedClassName) lines.foreach { line => eventSet.foreach { event => if (line.contains(event)) { @@ -303,19 +206,8 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { assert(eventSet.isEmpty, "The following events are missing: " + eventSet.toSeq) } - /** - * Read all lines from the file specified by the given path. - * If a compression codec is specified, use it to read the file. - */ - private def readFileLines( - filePath: Path, - compressionCodec: Option[CompressionCodec]): Seq[String] = { - val fstream = fileSystem.open(filePath) - val cstream = - compressionCodec.map { codec => - codec.compressedInputStream(fstream) - }.getOrElse(fstream) - Source.fromInputStream(cstream).getLines().toSeq + private def readLines(in: InputStream): Seq[String] = { + Source.fromInputStream(in).getLines().toSeq } /** @@ -328,30 +220,14 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { var appEnded = false override def onJobStart(jobStart: SparkListenerJobStart) { - assertEventsExist(eventLogger, Seq[String]( - Utils.getFormattedClassName(SparkListenerApplicationStart), - Utils.getFormattedClassName(SparkListenerBlockManagerAdded), - Utils.getFormattedClassName(SparkListenerEnvironmentUpdate) - )) jobStarted = true } override def onJobEnd(jobEnd: SparkListenerJobEnd) { - assertEventsExist(eventLogger, Seq[String]( - Utils.getFormattedClassName(SparkListenerJobStart), - Utils.getFormattedClassName(SparkListenerJobEnd), - Utils.getFormattedClassName(SparkListenerStageSubmitted), - Utils.getFormattedClassName(SparkListenerStageCompleted), - Utils.getFormattedClassName(SparkListenerTaskStart), - Utils.getFormattedClassName(SparkListenerTaskEnd) - )) jobEnded = true } override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { - assertEventsExist(eventLogger, Seq[String]( - Utils.getFormattedClassName(SparkListenerApplicationEnd) - )) appEnded = true } @@ -362,39 +238,6 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { } } - - /* -------------------------------------------------------- * - * Helper methods for validating state of the special files * - * -------------------------------------------------------- */ - - private def eventLogsExist(logFiles: Array[FileStatus]): Boolean = { - logFiles.map(_.getPath.getName).exists(EventLoggingListener.isEventLogFile) - } - - private def sparkVersionExists(logFiles: Array[FileStatus]): Boolean = { - logFiles.map(_.getPath.getName).exists(EventLoggingListener.isSparkVersionFile) - } - - private def compressionCodecExists(logFiles: Array[FileStatus]): Boolean = { - logFiles.map(_.getPath.getName).exists(EventLoggingListener.isCompressionCodecFile) - } - - private def applicationCompleteExists(logFiles: Array[FileStatus]): Boolean = { - logFiles.map(_.getPath.getName).exists(EventLoggingListener.isApplicationCompleteFile) - } - - private def assertSparkVersionIsValid(logFiles: Array[FileStatus]) { - val file = logFiles.map(_.getPath.getName).find(EventLoggingListener.isSparkVersionFile) - assert(file.isDefined) - assert(EventLoggingListener.parseSparkVersion(file.get) === SPARK_VERSION) - } - - private def assertCompressionCodecIsValid(logFiles: Array[FileStatus], compressionCodec: String) { - val file = logFiles.map(_.getPath.getName).find(EventLoggingListener.isCompressionCodecFile) - assert(file.isDefined) - assert(EventLoggingListener.parseCompressionCodec(file.get) === compressionCodec) - } - } diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index e05f373392d4a..7e360cc6082ec 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -22,7 +22,7 @@ import java.io.{File, PrintWriter} import org.json4s.jackson.JsonMethods._ import org.scalatest.{BeforeAndAfter, FunSuite} -import org.apache.spark.SparkContext._ +import org.apache.spark.{SparkConf, SparkContext, SPARK_VERSION} import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec @@ -34,7 +34,6 @@ import org.apache.spark.util.{JsonProtocol, Utils} class ReplayListenerSuite extends FunSuite with BeforeAndAfter { private val fileSystem = Utils.getHadoopFileSystem("/", SparkHadoopUtil.get.newConfiguration(new SparkConf())) - private val allCompressionCodecs = CompressionCodec.ALL_COMPRESSION_CODECS private var testDir: File = _ before { @@ -46,13 +45,29 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { } test("Simple replay") { - testSimpleReplay() - } + val logFilePath = Utils.getFilePath(testDir, "events.txt") + val fstream = fileSystem.create(logFilePath) + val writer = new PrintWriter(fstream) + val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None, + 125L, "Mickey") + val applicationEnd = SparkListenerApplicationEnd(1000L) + writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationStart)))) + writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationEnd)))) + writer.close() - test("Simple replay with compression") { - allCompressionCodecs.foreach { codec => - testSimpleReplay(Some(codec)) + val conf = EventLoggingListenerSuite.getLoggingConf(logFilePath) + val logData = fileSystem.open(logFilePath) + val eventMonster = new EventMonster(conf) + try { + val replayer = new ReplayListenerBus() + replayer.addListener(eventMonster) + replayer.replay(logData, SPARK_VERSION) + } finally { + logData.close() } + assert(eventMonster.loggedEvents.size === 2) + assert(eventMonster.loggedEvents(0) === JsonProtocol.sparkEventToJson(applicationStart)) + assert(eventMonster.loggedEvents(1) === JsonProtocol.sparkEventToJson(applicationEnd)) } // This assumes the correctness of EventLoggingListener @@ -62,7 +77,7 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { // This assumes the correctness of EventLoggingListener test("End-to-end replay with compression") { - allCompressionCodecs.foreach { codec => + CompressionCodec.ALL_COMPRESSION_CODECS.foreach { codec => testApplicationReplay(Some(codec)) } } @@ -72,31 +87,6 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { * Actual test logic * * ----------------- */ - /** - * Test simple replaying of events. - */ - private def testSimpleReplay(codecName: Option[String] = None) { - val logFilePath = Utils.getFilePath(testDir, "events.txt") - val codec = codecName.map(getCompressionCodec) - val fstream = fileSystem.create(logFilePath) - val cstream = codec.map(_.compressedOutputStream(fstream)).getOrElse(fstream) - val writer = new PrintWriter(cstream) - val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None, - 125L, "Mickey") - val applicationEnd = SparkListenerApplicationEnd(1000L) - writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationStart)))) - writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationEnd)))) - writer.close() - val replayer = new ReplayListenerBus(Seq(logFilePath), fileSystem, codec) - val conf = EventLoggingListenerSuite.getLoggingConf(logFilePath, codecName) - val eventMonster = new EventMonster(conf) - replayer.addListener(eventMonster) - replayer.replay() - assert(eventMonster.loggedEvents.size === 2) - assert(eventMonster.loggedEvents(0) === JsonProtocol.sparkEventToJson(applicationStart)) - assert(eventMonster.loggedEvents(1) === JsonProtocol.sparkEventToJson(applicationEnd)) - } - /** * Test end-to-end replaying of events. * @@ -106,6 +96,8 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { */ private def testApplicationReplay(codecName: Option[String] = None) { val logDirPath = Utils.getFilePath(testDir, "test-replay") + fileSystem.mkdirs(logDirPath) + val conf = EventLoggingListenerSuite.getLoggingConf(logDirPath, codecName) val sc = new SparkContext("local-cluster[2,1,512]", "Test replay", conf) @@ -117,22 +109,21 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { sc.stop() // Prepare information needed for replay - val codec = codecName.map(getCompressionCodec) val applications = fileSystem.listStatus(logDirPath) assert(applications != null && applications.size > 0) - val eventLogDir = applications.sortBy(_.getAccessTime).last - assert(eventLogDir.isDir) - val logFiles = fileSystem.listStatus(eventLogDir.getPath) - assert(logFiles != null && logFiles.size > 0) - val logFile = logFiles.find(_.getPath.getName.startsWith("EVENT_LOG_")) - assert(logFile.isDefined) - val logFilePath = logFile.get.getPath + val eventLog = applications.sortBy(_.getModificationTime).last + assert(!eventLog.isDir) // Replay events - val replayer = new ReplayListenerBus(Seq(logFilePath), fileSystem, codec) + val (logData, version) = EventLoggingListener.openEventLog(eventLog.getPath(), fileSystem) val eventMonster = new EventMonster(conf) - replayer.addListener(eventMonster) - replayer.replay() + try { + val replayer = new ReplayListenerBus() + replayer.addListener(eventMonster) + replayer.replay(logData, version) + } finally { + logData.close() + } // Verify the same events are replayed in the same order assert(sc.eventLogger.isDefined) @@ -155,7 +146,9 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { */ private class EventMonster(conf: SparkConf) extends EventLoggingListener("test", "testdir", conf) { - logger.close() + + override def start() { } + } private def getCompressionCodec(codecName: String) = { diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index abe0dc35b07e2..b276343cb412c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -25,7 +25,6 @@ import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} import org.scalatest.Matchers import org.apache.spark.{LocalSparkContext, SparkContext} -import org.apache.spark.SparkContext._ import org.apache.spark.executor.TaskMetrics class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala index 11e8c9c4cb37f..855f1b6276089 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala @@ -23,7 +23,6 @@ import com.esotericsoftware.kryo.Kryo import org.scalatest.FunSuite import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv, TestUtils} -import org.apache.spark.SparkContext._ import org.apache.spark.serializer.KryoDistributedTest._ class KryoSerializerDistributedSuite extends FunSuite { diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleMemoryManagerSuite.scala index d31bc22ee74f7..e0e646f0a3652 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleMemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleMemoryManagerSuite.scala @@ -159,7 +159,7 @@ class ShuffleMemoryManagerSuite extends FunSuite with Timeouts { test("threads can block to get at least 1 / 2N memory") { // t1 grabs 1000 bytes and then waits until t2 is ready to make a request. It sleeps - // for a bit and releases 250 bytes, which should then be greanted to t2. Further requests + // for a bit and releases 250 bytes, which should then be granted to t2. Further requests // by t2 will return false right away because it now has 1 / 2N of the memory. val manager = new ShuffleMemoryManager(1000L) @@ -291,4 +291,19 @@ class ShuffleMemoryManagerSuite extends FunSuite with Timeouts { assert(state.t2WaitTime > 200, s"t2 waited less than 200 ms (${state.t2WaitTime})") } } + + test("threads should not be granted a negative size") { + val manager = new ShuffleMemoryManager(1000L) + manager.tryToAcquire(700L) + + val latch = new CountDownLatch(1) + startThread("t1") { + manager.tryToAcquire(300L) + latch.countDown() + } + latch.await() // Wait until `t1` calls `tryToAcquire` + + val granted = manager.tryToAcquire(300L) + assert(0 === granted, "granted is negative") + } } diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index d2857b8b55664..787f4c2b5a8b2 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -27,7 +27,6 @@ import org.scalatest.selenium.WebBrowser import org.scalatest.time.SpanSugar._ import org.apache.spark._ -import org.apache.spark.SparkContext._ import org.apache.spark.LocalSparkContext._ import org.apache.spark.api.java.StorageLevels import org.apache.spark.shuffle.FetchFailedException diff --git a/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala b/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala deleted file mode 100644 index 72466a3aa1130..0000000000000 --- a/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala +++ /dev/null @@ -1,169 +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.util - -import java.io.{File, IOException} - -import scala.io.Source - -import org.apache.hadoop.fs.Path -import org.scalatest.{BeforeAndAfter, FunSuite} - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.io.CompressionCodec - -/** - * Test writing files through the FileLogger. - */ -class FileLoggerSuite extends FunSuite with BeforeAndAfter { - private val fileSystem = Utils.getHadoopFileSystem("/", - SparkHadoopUtil.get.newConfiguration(new SparkConf())) - private val allCompressionCodecs = Seq[String]( - "org.apache.spark.io.LZFCompressionCodec", - "org.apache.spark.io.SnappyCompressionCodec" - ) - private var testDir: File = _ - private var logDirPath: Path = _ - private var logDirPathString: String = _ - - before { - testDir = Utils.createTempDir() - logDirPath = Utils.getFilePath(testDir, "test-file-logger") - logDirPathString = logDirPath.toString - } - - after { - Utils.deleteRecursively(testDir) - } - - test("Simple logging") { - testSingleFile() - } - - test ("Simple logging with compression") { - allCompressionCodecs.foreach { codec => - testSingleFile(Some(codec)) - } - } - - test("Logging multiple files") { - testMultipleFiles() - } - - test("Logging multiple files with compression") { - allCompressionCodecs.foreach { codec => - testMultipleFiles(Some(codec)) - } - } - - test("Logging when directory already exists") { - // Create the logging directory multiple times - new FileLogger(logDirPathString, new SparkConf, compress = false, overwrite = true).start() - new FileLogger(logDirPathString, new SparkConf, compress = false, overwrite = true).start() - new FileLogger(logDirPathString, new SparkConf, compress = false, overwrite = true).start() - - // If overwrite is not enabled, an exception should be thrown - intercept[IOException] { - new FileLogger(logDirPathString, new SparkConf, compress = false, overwrite = false).start() - } - } - - - /* ----------------- * - * Actual test logic * - * ----------------- */ - - /** - * Test logging to a single file. - */ - private def testSingleFile(codecName: Option[String] = None) { - val conf = getLoggingConf(codecName) - val codec = codecName.map { c => CompressionCodec.createCodec(conf) } - val logger = - if (codecName.isDefined) { - new FileLogger(logDirPathString, conf, compress = true) - } else { - new FileLogger(logDirPathString, conf) - } - logger.start() - assert(fileSystem.exists(logDirPath)) - assert(fileSystem.getFileStatus(logDirPath).isDir) - assert(fileSystem.listStatus(logDirPath).size === 0) - - logger.newFile() - val files = fileSystem.listStatus(logDirPath) - assert(files.size === 1) - val firstFile = files.head - val firstFilePath = firstFile.getPath - - logger.log("hello") - logger.flush() - assert(readFileContent(firstFilePath, codec) === "hello") - - logger.log(" world") - logger.close() - assert(readFileContent(firstFilePath, codec) === "hello world") - } - - /** - * Test logging to multiple files. - */ - private def testMultipleFiles(codecName: Option[String] = None) { - val conf = getLoggingConf(codecName) - val codec = codecName.map { c => CompressionCodec.createCodec(conf) } - val logger = - if (codecName.isDefined) { - new FileLogger(logDirPathString, conf, compress = true) - } else { - new FileLogger(logDirPathString, conf) - } - logger.start() - logger.newFile("Jean_Valjean") - logger.logLine("Who am I?") - logger.logLine("Destiny?") - logger.newFile("John_Valjohn") - logger.logLine("One") - logger.logLine("Two three...") - logger.newFile("Wolverine") - logger.logLine("There was a time") - logger.logLine("A time when our enemies knew honor.") - logger.close() - assert(readFileContent(new Path(logDirPath, "Jean_Valjean"), codec) === "Who am I?\nDestiny?") - assert(readFileContent(new Path(logDirPath, "John_Valjohn"), codec) === "One\nTwo three...") - assert(readFileContent(new Path(logDirPath, "Wolverine"), codec) === - "There was a time\nA time when our enemies knew honor.") - } - - /** - * Read the content of the file specified by the given path. - * If a compression codec is specified, use it to read the file. - */ - private def readFileContent(logPath: Path, codec: Option[CompressionCodec] = None): String = { - val fstream = fileSystem.open(logPath) - val cstream = codec.map(_.compressedInputStream(fstream)).getOrElse(fstream) - Source.fromInputStream(cstream).getLines().mkString("\n") - } - - private def getLoggingConf(codecName: Option[String]) = { - val conf = new SparkConf - codecName.foreach { c => conf.set("spark.io.compression.codec", c) } - conf - } - -} diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index 511d76c9144cc..48f79ea651018 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -22,7 +22,6 @@ import scala.collection.mutable.ArrayBuffer import org.scalatest.FunSuite import org.apache.spark._ -import org.apache.spark.SparkContext._ import org.apache.spark.io.CompressionCodec class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala index 3cb42d416de4f..72d96798b1141 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala @@ -22,7 +22,6 @@ import scala.collection.mutable.ArrayBuffer import org.scalatest.{PrivateMethodTester, FunSuite} import org.apache.spark._ -import org.apache.spark.SparkContext._ import scala.util.Random diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index e0aca467ac949..3b89aaba29609 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -39,7 +39,6 @@ RC_NAME=${RC_NAME:-rc2} M2_REPO=~/.m2/repository SPARK_REPO=$M2_REPO/org/apache/spark NEXUS_ROOT=https://repository.apache.org/service/local/staging -NEXUS_UPLOAD=$NEXUS_ROOT/deploy/maven2 NEXUS_PROFILE=d63f592e7eac0 # Profile for Spark staging uploads if [ -z "$JAVA_HOME" ]; then @@ -64,19 +63,28 @@ if [[ ! "$@" =~ --package-only ]]; then # NOTE: This is done "eagerly" i.e. we don't check if we can succesfully build # or before we coin the release commit. This helps avoid races where # other people add commits to this branch while we are in the middle of building. - old=" ${RELEASE_VERSION}-SNAPSHOT<\/version>" - new=" ${RELEASE_VERSION}<\/version>" - find . -name pom.xml -o -name package.scala | grep -v dev | xargs -I {} sed -i \ - -e "s/$old/$new/" {} + cur_ver="${RELEASE_VERSION}-SNAPSHOT" + rel_ver="${RELEASE_VERSION}" + next_ver="${NEXT_VERSION}-SNAPSHOT" + + old="^\( \{2,4\}\)${cur_ver}<\/version>$" + new="\1${rel_ver}<\/version>" + find . -name pom.xml | grep -v dev | xargs -I {} sed -i \ + -e "s/${old}/${new}/" {} + find . -name package.scala | grep -v dev | xargs -I {} sed -i \ + -e "s/${old}/${new}/" {} + git commit -a -m "Preparing Spark release $GIT_TAG" echo "Creating tag $GIT_TAG at the head of $GIT_BRANCH" git tag $GIT_TAG - old=" ${RELEASE_VERSION}<\/version>" - new=" ${NEXT_VERSION}-SNAPSHOT<\/version>" - find . -name pom.xml -o -name package.scala | grep -v dev | xargs -I {} sed -i \ + old="^\( \{2,4\}\)${rel_ver}<\/version>$" + new="\1${next_ver}<\/version>" + find . -name pom.xml | grep -v dev | xargs -I {} sed -i \ -e "s/$old/$new/" {} - git commit -a -m "Preparing development version ${NEXT_VERSION}-SNAPSHOT" + find . -name package.scala | grep -v dev | xargs -I {} sed -i \ + -e "s/${old}/${new}/" {} + git commit -a -m "Preparing development version $next_ver" git push origin $GIT_TAG git push origin HEAD:$GIT_BRANCH git checkout -f $GIT_TAG @@ -118,12 +126,13 @@ if [[ ! "$@" =~ --package-only ]]; then gpg --print-md SHA1 $file > $file.sha1 done - echo "Uplading files to $NEXUS_UPLOAD" + nexus_upload=$NEXUS_ROOT/deployByRepositoryId/$staged_repo_id + echo "Uplading files to $nexus_upload" for file in $(find . -type f) do # strip leading ./ file_short=$(echo $file | sed -e "s/\.\///") - dest_url="$NEXUS_UPLOAD/org/apache/spark/$file_short" + dest_url="$nexus_upload/org/apache/spark/$file_short" echo " Uploading $file_short" curl -u $ASF_USERNAME:$ASF_PASSWORD --upload-file $file_short $dest_url done diff --git a/dev/create-release/generate-contributors.py b/dev/create-release/generate-contributors.py index f4bf734081583..8aaa250bd7e29 100755 --- a/dev/create-release/generate-contributors.py +++ b/dev/create-release/generate-contributors.py @@ -26,94 +26,112 @@ # You must set the following before use! JIRA_API_BASE = os.environ.get("JIRA_API_BASE", "https://issues.apache.org/jira") -START_COMMIT = os.environ.get("START_COMMIT", "37b100") -END_COMMIT = os.environ.get("END_COMMIT", "3693ae") - -try: - from jira.client import JIRA -except ImportError: - print "This tool requires the jira-python library" - print "Install using 'sudo pip install jira-python'" - sys.exit(-1) - -try: - import unidecode -except ImportError: - print "This tool requires the unidecode library to decode obscure github usernames" - print "Install using 'sudo pip install unidecode'" - sys.exit(-1) - -# If commit range is not specified, prompt the user to provide it -if not START_COMMIT or not END_COMMIT: - print "A commit range is required to proceed." - if not START_COMMIT: - START_COMMIT = raw_input("Please specify starting commit hash (inclusive): ") - if not END_COMMIT: - END_COMMIT = raw_input("Please specify ending commit hash (non-inclusive): ") - -# Verify provided arguments -start_commit_line = get_one_line(START_COMMIT) -end_commit_line = get_one_line(END_COMMIT) -num_commits = num_commits_in_range(START_COMMIT, END_COMMIT) -if not start_commit_line: sys.exit("Start commit %s not found!" % START_COMMIT) -if not end_commit_line: sys.exit("End commit %s not found!" % END_COMMIT) -if num_commits == 0: - sys.exit("There are no commits in the provided range [%s, %s)" % (START_COMMIT, END_COMMIT)) +RELEASE_TAG = os.environ.get("RELEASE_TAG", "v1.2.0-rc2") +PREVIOUS_RELEASE_TAG = os.environ.get("PREVIOUS_RELEASE_TAG", "v1.1.0") + +# If the release tags are not provided, prompt the user to provide them +while not tag_exists(RELEASE_TAG): + RELEASE_TAG = raw_input("Please provide a valid release tag: ") +while not tag_exists(PREVIOUS_RELEASE_TAG): + print "Please specify the previous release tag." + PREVIOUS_RELEASE_TAG = raw_input(\ + "For instance, if you are releasing v1.2.0, you should specify v1.1.0: ") + +# Gather commits found in the new tag but not in the old tag. +# This filters commits based on both the git hash and the PR number. +# If either is present in the old tag, then we ignore the commit. +print "Gathering new commits between tags %s and %s" % (PREVIOUS_RELEASE_TAG, RELEASE_TAG) +release_commits = get_commits(RELEASE_TAG) +previous_release_commits = get_commits(PREVIOUS_RELEASE_TAG) +previous_release_hashes = set() +previous_release_prs = set() +for old_commit in previous_release_commits: + previous_release_hashes.add(old_commit.get_hash()) + if old_commit.get_pr_number(): + previous_release_prs.add(old_commit.get_pr_number()) +new_commits = [] +for this_commit in release_commits: + this_hash = this_commit.get_hash() + this_pr_number = this_commit.get_pr_number() + if this_hash in previous_release_hashes: + continue + if this_pr_number and this_pr_number in previous_release_prs: + continue + new_commits.append(this_commit) +if not new_commits: + sys.exit("There are no new commits between %s and %s!" % (PREVIOUS_RELEASE_TAG, RELEASE_TAG)) + +# Prompt the user for confirmation that the commit range is correct print "\n==================================================================================" print "JIRA server: %s" % JIRA_API_BASE -print "Start commit (inclusive): %s" % start_commit_line -print "End commit (non-inclusive): %s" % end_commit_line -print "Number of commits in this range: %s" % num_commits +print "Release tag: %s" % RELEASE_TAG +print "Previous release tag: %s" % PREVIOUS_RELEASE_TAG +print "Number of commits in this range: %s" % len(new_commits) print -response = raw_input("Is this correct? [Y/n] ") -if response.lower() != "y" and response: - sys.exit("Ok, exiting") +def print_indented(_list): + for x in _list: print " %s" % x +if yesOrNoPrompt("Show all commits?"): + print_indented(new_commits) print "==================================================================================\n" - -# Find all commits within this range -print "Gathering commits within range [%s..%s)" % (START_COMMIT, END_COMMIT) -commits = get_one_line_commits(START_COMMIT, END_COMMIT) -if not commits: sys.exit("Error: No commits found within this range!") -commits = commits.split("\n") +if not yesOrNoPrompt("Does this look correct?"): + sys.exit("Ok, exiting") # Filter out special commits releases = [] +maintenance = [] reverts = [] nojiras = [] filtered_commits = [] -def is_release(commit): - return re.findall("\[release\]", commit.lower()) or\ - "maven-release-plugin" in commit or "CHANGES.txt" in commit -def has_no_jira(commit): - return not re.findall("SPARK-[0-9]+", commit.upper()) -def is_revert(commit): - return "revert" in commit.lower() -def is_docs(commit): - return re.findall("docs*", commit.lower()) or "programming guide" in commit.lower() -for c in commits: - if not c: continue - elif is_release(c): releases.append(c) - elif is_revert(c): reverts.append(c) - elif is_docs(c): filtered_commits.append(c) # docs may not have JIRA numbers - elif has_no_jira(c): nojiras.append(c) +def is_release(commit_title): + return re.findall("\[release\]", commit_title.lower()) or\ + "preparing spark release" in commit_title.lower() or\ + "preparing development version" in commit_title.lower() or\ + "CHANGES.txt" in commit_title +def is_maintenance(commit_title): + return "maintenance" in commit_title.lower() or\ + "manually close" in commit_title.lower() +def has_no_jira(commit_title): + return not re.findall("SPARK-[0-9]+", commit_title.upper()) +def is_revert(commit_title): + return "revert" in commit_title.lower() +def is_docs(commit_title): + return re.findall("docs*", commit_title.lower()) or\ + "programming guide" in commit_title.lower() +for c in new_commits: + t = c.get_title() + if not t: continue + elif is_release(t): releases.append(c) + elif is_maintenance(t): maintenance.append(c) + elif is_revert(t): reverts.append(c) + elif is_docs(t): filtered_commits.append(c) # docs may not have JIRA numbers + elif has_no_jira(t): nojiras.append(c) else: filtered_commits.append(c) # Warn against ignored commits -def print_indented(_list): - for x in _list: print " %s" % x -if releases or reverts or nojiras: +if releases or maintenance or reverts or nojiras: print "\n==================================================================================" - if releases: print "Releases (%d)" % len(releases); print_indented(releases) - if reverts: print "Reverts (%d)" % len(reverts); print_indented(reverts) - if nojiras: print "No JIRA (%d)" % len(nojiras); print_indented(nojiras) + if releases: print "Found %d release commits" % len(releases) + if maintenance: print "Found %d maintenance commits" % len(maintenance) + if reverts: print "Found %d revert commits" % len(reverts) + if nojiras: print "Found %d commits with no JIRA" % len(nojiras) + print "* Warning: these commits will be ignored.\n" + if yesOrNoPrompt("Show ignored commits?"): + if releases: print "Release (%d)" % len(releases); print_indented(releases) + if maintenance: print "Maintenance (%d)" % len(maintenance); print_indented(maintenance) + if reverts: print "Revert (%d)" % len(reverts); print_indented(reverts) + if nojiras: print "No JIRA (%d)" % len(nojiras); print_indented(nojiras) print "==================== Warning: the above commits will be ignored ==================\n" -response = raw_input("%d commits left to process. Ok to proceed? [y/N] " % len(filtered_commits)) -if response.lower() != "y": +prompt_msg = "%d commits left to process after filtering. Ok to proceed?" % len(filtered_commits) +if not yesOrNoPrompt(prompt_msg): sys.exit("Ok, exiting.") # Keep track of warnings to tell the user at the end warnings = [] +# Mapping from the invalid author name to its associated JIRA issues +# E.g. andrewor14 -> set("SPARK-2413", "SPARK-3551", "SPARK-3471") +invalid_authors = {} + # Populate a map that groups issues and components by author # It takes the form: Author name -> { Contribution type -> Spark components } # For instance, @@ -130,16 +148,25 @@ def print_indented(_list): # author_info = {} jira_options = { "server": JIRA_API_BASE } -jira = JIRA(jira_options) +jira_client = JIRA(options = jira_options) print "\n=========================== Compiling contributor list ===========================" for commit in filtered_commits: - commit_hash = re.findall("^[a-z0-9]+", commit)[0] - issues = re.findall("SPARK-[0-9]+", commit.upper()) - author = get_author(commit_hash) - author = unidecode.unidecode(unicode(author, "UTF-8")) # guard against special characters - date = get_date(commit_hash) - # Parse components from the commit message, if any - commit_components = find_components(commit, commit_hash) + _hash = commit.get_hash() + title = commit.get_title() + issues = re.findall("SPARK-[0-9]+", title.upper()) + author = commit.get_author() + date = get_date(_hash) + # If the author name is invalid, keep track of it along + # with all associated issues so we can translate it later + if is_valid_author(author): + author = capitalize_author(author) + else: + if author not in invalid_authors: + invalid_authors[author] = set() + for issue in issues: + invalid_authors[author].add(issue) + # Parse components from the commit title, if any + commit_components = find_components(title, _hash) # Populate or merge an issue into author_info[author] def populate(issue_type, components): components = components or [CORE_COMPONENT] # assume core if no components provided @@ -147,28 +174,27 @@ def populate(issue_type, components): author_info[author] = {} if issue_type not in author_info[author]: author_info[author][issue_type] = set() - for component in all_components: + for component in components: author_info[author][issue_type].add(component) # Find issues and components associated with this commit for issue in issues: - jira_issue = jira.issue(issue) + jira_issue = jira_client.issue(issue) jira_type = jira_issue.fields.issuetype.name jira_type = translate_issue_type(jira_type, issue, warnings) - jira_components = [translate_component(c.name, commit_hash, warnings)\ - for c in jira_issue.fields.components] + jira_components = [translate_component(c.name, _hash, warnings)\ + for c in jira_issue.fields.components] all_components = set(jira_components + commit_components) populate(jira_type, all_components) # For docs without an associated JIRA, manually add it ourselves - if is_docs(commit) and not issues: + if is_docs(title) and not issues: populate("documentation", commit_components) - print " Processed commit %s authored by %s on %s" % (commit_hash, author, date) + print " Processed commit %s authored by %s on %s" % (_hash, author, date) print "==================================================================================\n" # Write to contributors file ordered by author names -# Each line takes the format "Author name - semi-colon delimited contributions" -# e.g. Andrew Or - Bug fixes in Windows, Core, and Web UI; improvements in Core -# e.g. Tathagata Das - Bug fixes and new features in Streaming -contributors_file_name = "contributors.txt" +# Each line takes the format " * Author name -- semi-colon delimited contributions" +# e.g. * Andrew Or -- Bug fixes in Windows, Core, and Web UI; improvements in Core +# e.g. * Tathagata Das -- Bug fixes and new features in Streaming contributors_file = open(contributors_file_name, "w") authors = author_info.keys() authors.sort() @@ -187,16 +213,28 @@ def populate(issue_type, components): # e.g. Bug fixes in MLlib, Core, and Streaming; documentation in YARN else: contributions = ["%s in %s" % (issue_type, nice_join(comps)) \ - for issue_type, comps in author_info[author].items()] + for issue_type, comps in author_info[author].items()] contribution = "; ".join(contributions) # Do not use python's capitalize() on the whole string to preserve case assert contribution contribution = contribution[0].capitalize() + contribution[1:] - line = "%s - %s" % (author, contribution) + # If the author name is invalid, use an intermediate format that + # can be translated through translate-contributors.py later + # E.g. andrewor14/SPARK-3425/SPARK-1157/SPARK-6672 + if author in invalid_authors and invalid_authors[author]: + author = author + "/" + "/".join(invalid_authors[author]) + line = " * %s -- %s" % (author, contribution) contributors_file.write(line + "\n") contributors_file.close() print "Contributors list is successfully written to %s!" % contributors_file_name +# Prompt the user to translate author names if necessary +if invalid_authors: + warnings.append("Found the following invalid authors:") + for a in invalid_authors: + warnings.append("\t%s" % a) + warnings.append("Please run './translate-contributors.py' to translate them.") + # Log any warnings encountered in the process if warnings: print "\n============ Warnings encountered while creating the contributor list ============" diff --git a/dev/create-release/known_translations b/dev/create-release/known_translations new file mode 100644 index 0000000000000..b74e4ee8a330b --- /dev/null +++ b/dev/create-release/known_translations @@ -0,0 +1,59 @@ +# This is a mapping of names to be translated through translate-contributors.py +# The format expected on each line should be: - +CodingCat - Nan Zhu +CrazyJvm - Chao Chen +EugenCepoi - Eugen Cepoi +GraceH - Jie Huang +JerryLead - Lijie Xu +Leolh - Liu Hao +Lewuathe - Kai Sasaki +RongGu - Rong Gu +Shiti - Shiti Saxena +Victsm - Min Shen +WangTaoTheTonic - Wang Tao +XuTingjun - Tingjun Xu +YanTangZhai - Yantang Zhai +alexdebrie - Alex DeBrie +alokito - Alok Saldanha +anantasty - Anant Asthana +andrewor14 - Andrew Or +aniketbhatnagar - Aniket Bhatnagar +arahuja - Arun Ahuja +brkyvz - Burak Yavuz +chesterxgchen - Chester Chen +chiragaggarwal - Chirag Aggarwal +chouqin - Qiping Li +cocoatomo - Tomohiko K. +coderfi - Fairiz Azizi +coderxiang - Shuo Xiang +davies - Davies Liu +epahomov - Egor Pahomov +falaki - Hossein Falaki +freeman-lab - Jeremy Freeman +industrial-sloth - Jascha Swisher +jackylk - Jacky Li +jayunit100 - Jay Vyas +jerryshao - Saisai Shao +jkbradley - Joseph Bradley +lianhuiwang - Lianhui Wang +lirui-intel - Rui Li +luluorta - Lu Lu +luogankun - Gankun Luo +maji2014 - Derek Ma +mccheah - Matthew Cheah +mengxr - Xiangrui Meng +nartz - Nathan Artz +odedz - Oded Zimerman +ravipesala - Ravindra Pesala +roxchkplusony - Victor Tso +scwf - Wang Fei +shimingfei - Shiming Fei +surq - Surong Quan +suyanNone - Su Yan +tedyu - Ted Yu +tigerquoll - Dale Richardson +wangxiaojing - Xiaojing Wang +watermen - Yadong Qi +witgo - Guoqiang Li +xinyunh - Xinyun Huang +zsxwing - Shixiong Zhu diff --git a/dev/create-release/releaseutils.py b/dev/create-release/releaseutils.py index e56d7fa58fa2c..26221b270394e 100755 --- a/dev/create-release/releaseutils.py +++ b/dev/create-release/releaseutils.py @@ -19,22 +19,114 @@ # This file contains helper methods used in creating a release. import re +import sys from subprocess import Popen, PIPE +try: + from jira.client import JIRA + from jira.exceptions import JIRAError +except ImportError: + print "This tool requires the jira-python library" + print "Install using 'sudo pip install jira-python'" + sys.exit(-1) + +try: + from github import Github + from github import GithubException +except ImportError: + print "This tool requires the PyGithub library" + print "Install using 'sudo pip install PyGithub'" + sys.exit(-1) + +try: + import unidecode +except ImportError: + print "This tool requires the unidecode library to decode obscure github usernames" + print "Install using 'sudo pip install unidecode'" + sys.exit(-1) + +# Contributors list file name +contributors_file_name = "contributors.txt" + +# Prompt the user to answer yes or no until they do so +def yesOrNoPrompt(msg): + response = raw_input("%s [y/n]: " % msg) + while response != "y" and response != "n": + return yesOrNoPrompt(msg) + return response == "y" + # Utility functions run git commands (written with Git 1.8.5) def run_cmd(cmd): return Popen(cmd, stdout=PIPE).communicate()[0] -def get_author(commit_hash): - return run_cmd(["git", "show", "--quiet", "--pretty=format:%an", commit_hash]) +def run_cmd_error(cmd): return Popen(cmd, stdout=PIPE, stderr=PIPE).communicate()[1] def get_date(commit_hash): return run_cmd(["git", "show", "--quiet", "--pretty=format:%cd", commit_hash]) -def get_one_line(commit_hash): - return run_cmd(["git", "show", "--quiet", "--pretty=format:\"%h %cd %s\"", commit_hash]) -def get_one_line_commits(start_hash, end_hash): - return run_cmd(["git", "log", "--oneline", "%s..%s" % (start_hash, end_hash)]) -def num_commits_in_range(start_hash, end_hash): - output = run_cmd(["git", "log", "--oneline", "%s..%s" % (start_hash, end_hash)]) - lines = [line for line in output.split("\n") if line] # filter out empty lines - return len(lines) +def tag_exists(tag): + stderr = run_cmd_error(["git", "show", tag]) + return "error" not in stderr + +# A type-safe representation of a commit +class Commit: + def __init__(self, _hash, author, title, pr_number = None): + self._hash = _hash + self.author = author + self.title = title + self.pr_number = pr_number + def get_hash(self): return self._hash + def get_author(self): return self.author + def get_title(self): return self.title + def get_pr_number(self): return self.pr_number + def __str__(self): + closes_pr = "(Closes #%s)" % self.pr_number if self.pr_number else "" + return "%s %s %s %s" % (self._hash, self.author, self.title, closes_pr) + +# Return all commits that belong to the specified tag. +# +# Under the hood, this runs a `git log` on that tag and parses the fields +# from the command output to construct a list of Commit objects. Note that +# because certain fields reside in the commit description and cannot be parsed +# through the Github API itself, we need to do some intelligent regex parsing +# to extract those fields. +# +# This is written using Git 1.8.5. +def get_commits(tag): + commit_start_marker = "|=== COMMIT START MARKER ===|" + commit_end_marker = "|=== COMMIT END MARKER ===|" + field_end_marker = "|=== COMMIT FIELD END MARKER ===|" + log_format =\ + commit_start_marker + "%h" +\ + field_end_marker + "%an" +\ + field_end_marker + "%s" +\ + commit_end_marker + "%b" + output = run_cmd(["git", "log", "--quiet", "--pretty=format:" + log_format, tag]) + commits = [] + raw_commits = [c for c in output.split(commit_start_marker) if c] + for commit in raw_commits: + if commit.count(commit_end_marker) != 1: + print "Commit end marker not found in commit: " + for line in commit.split("\n"): print line + sys.exit(1) + # Separate commit digest from the body + # From the digest we extract the hash, author and the title + # From the body, we extract the PR number and the github username + [commit_digest, commit_body] = commit.split(commit_end_marker) + if commit_digest.count(field_end_marker) != 2: + sys.exit("Unexpected format in commit: %s" % commit_digest) + [_hash, author, title] = commit_digest.split(field_end_marker) + # The PR number and github username is in the commit message + # itself and cannot be accessed through any Github API + pr_number = None + match = re.search("Closes #([0-9]+) from ([^/\\s]+)/", commit_body) + if match: + [pr_number, github_username] = match.groups() + # If the author name is not valid, use the github + # username so we can translate it properly later + if not is_valid_author(author): + author = github_username + # Guard against special characters + author = unidecode.unidecode(unicode(author, "UTF-8")).strip() + commit = Commit(_hash, author, title, pr_number) + commits.append(commit) + return commits # Maintain a mapping for translating issue types to contributions in the release notes # This serves an additional function of warning the user against unknown issue types @@ -44,9 +136,13 @@ def num_commits_in_range(start_hash, end_hash): known_issue_types = { "bug": "bug fixes", "build": "build fixes", + "dependency upgrade": "build fixes", "improvement": "improvements", "new feature": "new features", - "documentation": "documentation" + "documentation": "documentation", + "test": "test", + "task": "improvement", + "sub-task": "improvement" } # Maintain a mapping for translating component names when creating the release notes @@ -122,3 +218,39 @@ def nice_join(str_list): else: return ", ".join(str_list[:-1]) + ", and " + str_list[-1] +# Return the full name of the specified user on Github +# If the user doesn't exist, return None +def get_github_name(author, github_client): + if github_client: + try: + return github_client.get_user(author).name + except GithubException as e: + # If this is not a "not found" exception + if e.status != 404: + raise e + return None + +# Return the full name of the specified user on JIRA +# If the user doesn't exist, return None +def get_jira_name(author, jira_client): + if jira_client: + try: + return jira_client.user(author).displayName + except JIRAError as e: + # If this is not a "not found" exception + if e.status_code != 404: + raise e + return None + +# Return whether the given name is in the form +def is_valid_author(author): + if not author: return False + return " " in author and not re.findall("[0-9]", author) + +# Capitalize the first letter of each word in the given author name +def capitalize_author(author): + if not author: return None + words = author.split(" ") + words = [w[0].capitalize() + w[1:] for w in words if w] + return " ".join(words) + diff --git a/dev/create-release/translate-contributors.py b/dev/create-release/translate-contributors.py new file mode 100755 index 0000000000000..86fa02d87b9a0 --- /dev/null +++ b/dev/create-release/translate-contributors.py @@ -0,0 +1,253 @@ +#!/usr/bin/env python +# +# 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. + +# This script translates invalid authors in the contributors list generated +# by generate-contributors.py. When the script encounters an author name that +# is considered invalid, it searches Github and JIRA in an attempt to search +# for replacements. This tool runs in two modes: +# +# (1) Interactive mode: For each invalid author name, this script presents +# all candidate replacements to the user and awaits user response. In this +# mode, the user may also input a custom name. This is the default. +# +# (2) Non-interactive mode: For each invalid author name, this script replaces +# the name with the first valid candidate it can find. If there is none, it +# uses the original name. This can be enabled through the --non-interactive flag. + +import os +import sys + +from releaseutils import * + +# You must set the following before use! +JIRA_API_BASE = os.environ.get("JIRA_API_BASE", "https://issues.apache.org/jira") +JIRA_USERNAME = os.environ.get("JIRA_USERNAME", None) +JIRA_PASSWORD = os.environ.get("JIRA_PASSWORD", None) +GITHUB_API_TOKEN = os.environ.get("GITHUB_API_TOKEN", None) +if not JIRA_USERNAME or not JIRA_PASSWORD: + sys.exit("Both JIRA_USERNAME and JIRA_PASSWORD must be set") +if not GITHUB_API_TOKEN: + sys.exit("GITHUB_API_TOKEN must be set") + +# Write new contributors list to .final +if not os.path.isfile(contributors_file_name): + print "Contributors file %s does not exist!" % contributors_file_name + print "Have you run ./generate-contributors.py yet?" + sys.exit(1) +contributors_file = open(contributors_file_name, "r") +warnings = [] + +# In non-interactive mode, this script will choose the first replacement that is valid +INTERACTIVE_MODE = True +if len(sys.argv) > 1: + options = set(sys.argv[1:]) + if "--non-interactive" in options: + INTERACTIVE_MODE = False +if INTERACTIVE_MODE: + print "Running in interactive mode. To disable this, provide the --non-interactive flag." + +# Setup Github and JIRA clients +jira_options = { "server": JIRA_API_BASE } +jira_client = JIRA(options = jira_options, basic_auth = (JIRA_USERNAME, JIRA_PASSWORD)) +github_client = Github(GITHUB_API_TOKEN) + +# Load known author translations that are cached locally +known_translations = {} +known_translations_file_name = "known_translations" +known_translations_file = open(known_translations_file_name, "r") +for line in known_translations_file: + if line.startswith("#"): continue + [old_name, new_name] = line.strip("\n").split(" - ") + known_translations[old_name] = new_name +known_translations_file.close() + +# Open again in case the user adds new mappings +known_translations_file = open(known_translations_file_name, "a") + +# Generate candidates for the given author. This should only be called if the given author +# name does not represent a full name as this operation is somewhat expensive. Under the +# hood, it makes several calls to the Github and JIRA API servers to find the candidates. +# +# This returns a list of (candidate name, source) 2-tuples. E.g. +# [ +# (NOT_FOUND, "No full name found for Github user andrewor14"), +# ("Andrew Or", "Full name of JIRA user andrewor14"), +# ("Andrew Orso", "Full name of SPARK-1444 assignee andrewor14"), +# ("Andrew Ordall", "Full name of SPARK-1663 assignee andrewor14"), +# (NOT_FOUND, "No assignee found for SPARK-1763") +# ] +NOT_FOUND = "Not found" +def generate_candidates(author, issues): + candidates = [] + # First check for full name of Github user + github_name = get_github_name(author, github_client) + if github_name: + candidates.append((github_name, "Full name of Github user %s" % author)) + else: + candidates.append((NOT_FOUND, "No full name found for Github user %s" % author)) + # Then do the same for JIRA user + jira_name = get_jira_name(author, jira_client) + if jira_name: + candidates.append((jira_name, "Full name of JIRA user %s" % author)) + else: + candidates.append((NOT_FOUND, "No full name found for JIRA user %s" % author)) + # Then do the same for the assignee of each of the associated JIRAs + # Note that a given issue may not have an assignee, or the assignee may not have a full name + for issue in issues: + try: + jira_issue = jira_client.issue(issue) + except JIRAError as e: + # Do not exit just because an issue is not found! + if e.status_code == 404: + warnings.append("Issue %s not found!" % issue) + continue + raise e + jira_assignee = jira_issue.fields.assignee + if jira_assignee: + user_name = jira_assignee.name + display_name = jira_assignee.displayName + if display_name: + candidates.append((display_name, "Full name of %s assignee %s" % (issue, user_name))) + else: + candidates.append((NOT_FOUND, "No full name found for %s assignee %" % (issue, user_name))) + else: + candidates.append((NOT_FOUND, "No assignee found for %s" % issue)) + # Guard against special characters in candidate names + # Note that the candidate name may already be in unicode (JIRA returns this) + for i, (candidate, source) in enumerate(candidates): + try: + candidate = unicode(candidate, "UTF-8") + except TypeError: + # already in unicode + pass + candidate = unidecode.unidecode(candidate).strip() + candidates[i] = (candidate, source) + return candidates + +# Translate each invalid author by searching for possible candidates from Github and JIRA +# In interactive mode, this script presents the user with a list of choices and have the user +# select from this list. Additionally, the user may also choose to enter a custom name. +# In non-interactive mode, this script picks the first valid author name from the candidates +# If no such name exists, the original name is used (without the JIRA numbers). +print "\n========================== Translating contributor list ==========================" +lines = contributors_file.readlines() +contributions = [] +for i, line in enumerate(lines): + temp_author = line.strip(" * ").split(" -- ")[0] + print "Processing author %s (%d/%d)" % (temp_author, i + 1, len(lines)) + if not temp_author: + error_msg = " ERROR: Expected the following format \" * -- \"\n" + error_msg += " ERROR: Actual = %s" % line + print error_msg + warnings.append(error_msg) + contributions.append(line) + continue + author = temp_author.split("/")[0] + # Use the local copy of known translations where possible + if author in known_translations: + line = line.replace(temp_author, known_translations[author]) + elif not is_valid_author(author): + new_author = author + issues = temp_author.split("/")[1:] + candidates = generate_candidates(author, issues) + # Print out potential replacement candidates along with the sources, e.g. + # [X] No full name found for Github user andrewor14 + # [X] No assignee found for SPARK-1763 + # [0] Andrew Or - Full name of JIRA user andrewor14 + # [1] Andrew Orso - Full name of SPARK-1444 assignee andrewor14 + # [2] Andrew Ordall - Full name of SPARK-1663 assignee andrewor14 + # [3] andrewor14 - Raw Github username + # [4] Custom + candidate_names = [] + bad_prompts = [] # Prompts that can't actually be selected; print these first. + good_prompts = [] # Prompts that contain valid choices + for candidate, source in candidates: + if candidate == NOT_FOUND: + bad_prompts.append(" [X] %s" % source) + else: + index = len(candidate_names) + candidate_names.append(candidate) + good_prompts.append(" [%d] %s - %s" % (index, candidate, source)) + raw_index = len(candidate_names) + custom_index = len(candidate_names) + 1 + for p in bad_prompts: print p + if bad_prompts: print " ---" + for p in good_prompts: print p + # In interactive mode, additionally provide "custom" option and await user response + if INTERACTIVE_MODE: + print " [%d] %s - Raw Github username" % (raw_index, author) + print " [%d] Custom" % custom_index + response = raw_input(" Your choice: ") + last_index = custom_index + while not response.isdigit() or int(response) > last_index: + response = raw_input(" Please enter an integer between 0 and %d: " % last_index) + response = int(response) + if response == custom_index: + new_author = raw_input(" Please type a custom name for this author: ") + elif response != raw_index: + new_author = candidate_names[response] + # In non-interactive mode, just pick the first candidate + else: + valid_candidate_names = [name for name, _ in candidates\ + if is_valid_author(name) and name != NOT_FOUND] + if valid_candidate_names: + new_author = valid_candidate_names[0] + # Finally, capitalize the author and replace the original one with it + # If the final replacement is still invalid, log a warning + if is_valid_author(new_author): + new_author = capitalize_author(new_author) + else: + warnings.append("Unable to find a valid name %s for author %s" % (author, temp_author)) + print " * Replacing %s with %s" % (author, new_author) + # If we are in interactive mode, prompt the user whether we want to remember this new mapping + if INTERACTIVE_MODE and\ + author not in known_translations and\ + yesOrNoPrompt(" Add mapping %s -> %s to known translations file?" % (author, new_author)): + known_translations_file.write("%s - %s\n" % (author, new_author)) + known_translations_file.flush() + line = line.replace(temp_author, author) + contributions.append(line) +print "==================================================================================\n" +contributors_file.close() +known_translations_file.close() + +# Sort the contributions before writing them to the new file. +# Additionally, check if there are any duplicate author rows. +# This could happen if the same user has both a valid full +# name (e.g. Andrew Or) and an invalid one (andrewor14). +# If so, warn the user about this at the end. +contributions.sort() +all_authors = set() +new_contributors_file_name = contributors_file_name + ".final" +new_contributors_file = open(new_contributors_file_name, "w") +for line in contributions: + author = line.strip(" * ").split(" -- ")[0] + if author in all_authors: + warnings.append("Detected duplicate author name %s. Please merge these manually." % author) + all_authors.add(author) + new_contributors_file.write(line) +new_contributors_file.close() + +print "Translated contributors list successfully written to %s!" % new_contributors_file_name + +# Log any warnings encountered in the process +if warnings: + print "\n========== Warnings encountered while translating the contributor list ===========" + for w in warnings: print w + print "Please manually correct these in the final contributors list at %s." % new_contributors_file_name + print "==================================================================================\n" + diff --git a/dev/run-tests b/dev/run-tests index 328a73bd8b26d..9192cb7e169f3 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -141,20 +141,22 @@ CURRENT_BLOCK=$BLOCK_BUILD { # NOTE: echo "q" is needed because sbt on encountering a build file with failure - #+ (either resolution or compilation) prompts the user for input either q, r, etc - #+ to quit or retry. This echo is there to make it not block. + # (either resolution or compilation) prompts the user for input either q, r, etc + # to quit or retry. This echo is there to make it not block. # NOTE: Do not quote $BUILD_MVN_PROFILE_ARGS or else it will be interpreted as a - #+ single argument! + # single argument! # QUESTION: Why doesn't 'yes "q"' work? # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? - # First build with 0.12 to ensure patches do not break the hive 12 build + # First build with Hive 0.12.0 to ensure patches do not break the Hive 0.12.0 build HIVE_12_BUILD_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver -Phive-0.12.0" - echo "[info] Compile with hive 0.12" + echo "[info] Compile with Hive 0.12.0" echo -e "q\n" \ | sbt/sbt $HIVE_12_BUILD_ARGS clean hive/compile hive-thriftserver/compile \ | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" - # Then build with default version(0.13.1) because tests are based on this version + # Then build with default Hive version (0.13.1) because tests are based on this version + echo "[info] Compile with Hive 0.13.1" + rm -rf lib_managed echo "[info] Building Spark with these arguments: $SBT_MAVEN_PROFILES_ARGS"\ " -Phive -Phive-thriftserver" echo -e "q\n" \ @@ -178,7 +180,7 @@ CURRENT_BLOCK=$BLOCK_SPARK_UNIT_TESTS if [ -n "$_SQL_TESTS_ONLY" ]; then # This must be an array of individual arguments. Otherwise, having one long string - #+ will be interpreted as a single test, which doesn't work. + # will be interpreted as a single test, which doesn't work. SBT_MAVEN_TEST_ARGS=("catalyst/test" "sql/test" "hive/test" "mllib/test") else SBT_MAVEN_TEST_ARGS=("test") @@ -187,11 +189,11 @@ CURRENT_BLOCK=$BLOCK_SPARK_UNIT_TESTS echo "[info] Running Spark tests with these arguments: $SBT_MAVEN_PROFILES_ARGS ${SBT_MAVEN_TEST_ARGS[@]}" # NOTE: echo "q" is needed because sbt on encountering a build file with failure - #+ (either resolution or compilation) prompts the user for input either q, r, etc - #+ to quit or retry. This echo is there to make it not block. + # (either resolution or compilation) prompts the user for input either q, r, etc + # to quit or retry. This echo is there to make it not block. # NOTE: Do not quote $SBT_MAVEN_PROFILES_ARGS or else it will be interpreted as a - #+ single argument! - #+ "${SBT_MAVEN_TEST_ARGS[@]}" is cool because it's an array. + # single argument! + # "${SBT_MAVEN_TEST_ARGS[@]}" is cool because it's an array. # QUESTION: Why doesn't 'yes "q"' work? # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? echo -e "q\n" \ @@ -210,7 +212,7 @@ CURRENT_BLOCK=$BLOCK_PYSPARK_UNIT_TESTS echo "" echo "=========================================================================" -echo "Detecting binary incompatibilites with MiMa" +echo "Detecting binary incompatibilities with MiMa" echo "=========================================================================" CURRENT_BLOCK=$BLOCK_MIMA diff --git a/dev/scalastyle b/dev/scalastyle index c3c6012e74ffa..3a4df6e4bf1bc 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -18,11 +18,8 @@ # echo -e "q\n" | sbt/sbt -Phive -Phive-thriftserver scalastyle > scalastyle.txt -# Check style with YARN alpha built too -echo -e "q\n" | sbt/sbt -Pyarn-alpha -Phadoop-0.23 -Dhadoop.version=0.23.9 yarn-alpha/scalastyle \ - >> scalastyle.txt # Check style with YARN built too -echo -e "q\n" | sbt/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 yarn/scalastyle \ +echo -e "q\n" | sbt/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 scalastyle \ >> scalastyle.txt ERRORS=$(cat scalastyle.txt | awk '{if($1~/error/)print}') diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 627ed37de4a9c..8841f7675d35e 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -33,7 +33,7 @@