diff --git a/.gitignore b/.gitignore index c67cffa1c4375..30b1e12bf1b03 100644 --- a/.gitignore +++ b/.gitignore @@ -5,6 +5,7 @@ *.ipr *.iml *.iws +*.pyc .idea/ .idea_modules/ sbt/*.jar @@ -49,6 +50,8 @@ dependency-reduced-pom.xml checkpoint derby.log dist/ +dev/create-release/*txt +dev/create-release/*final spark-*-bin-*.tgz unit-tests.log /lib/ 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..685051eeed9f1 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -68,14 +68,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 +108,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/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..68c52ac09b55a 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 @@ -171,6 +171,6 @@ span.additional-metric-title { /* 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/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/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..8e5378ecc08de 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -357,8 +357,12 @@ 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)) { + if (dynamicAllocationEnabled) { + assert(master.contains("yarn") || dynamicAllocationTesting, + "Dynamic allocation of executors is currently only supported in YARN mode") Some(new ExecutorAllocationManager(this)) } else { None @@ -989,6 +993,8 @@ class SparkContext(config: SparkConf) extends Logging { */ @DeveloperApi 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) @@ -1005,6 +1011,8 @@ class SparkContext(config: SparkConf) extends Logging { */ @DeveloperApi 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) @@ -1630,28 +1638,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 +1670,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 +1730,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 +2025,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..f4215f268a0d3 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 + ) } /** @@ -171,8 +179,16 @@ object SparkEnv extends Logging { numCores: Int, isLocal: Boolean, actorSystem: ActorSystem = null): SparkEnv = { - create(conf, executorId, hostname, port, false, isLocal, defaultActorSystem = actorSystem, - numUsableCores = numCores) + create( + conf, + executorId, + hostname, + port, + isDriver = false, + isLocal = isLocal, + defaultActorSystem = actorSystem, + numUsableCores = numCores + ) } /** 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/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/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/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/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 8ba6a01bbcb97..f4fedc6327ab9 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 @@ -144,8 +144,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/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 835157fc520aa..52de6980ecbf8 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -172,7 +172,6 @@ private[spark] class Executor( val startGCTime = gcTime try { - Accumulators.clear() val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(serializedTask) updateDependencies(taskFiles, taskJars) task = ser.deserialize[Task[Any]](taskBytes, Thread.currentThread.getContextClassLoader) @@ -278,6 +277,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) } } 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..1b1131b9b8831 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 @@ -27,6 +27,18 @@ import org.apache.hadoop.mapreduce.lib.input.{CombineFileSplit, CombineFileRecor import org.apache.hadoop.mapreduce.RecordReader import org.apache.hadoop.mapreduce.TaskAttemptContext + +/** + * 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 * out in a key-value pair, where the key is the file path and the value is the entire content of @@ -38,12 +50,6 @@ 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) @@ -87,29 +93,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/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..b0434c9a3b6b3 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() @@ -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..214f22bc5b603 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 @@ -1386,6 +1398,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 +1413,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/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/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..29cd34429b881 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 @@ -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 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/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/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 315327c3c6b7c..d970fa30c1c35 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) 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 ""} 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/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/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index e05f373392d4a..90bdfe07f61c9 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,6 @@ 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} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec 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/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/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 @@