From 85a35c68401e171df0b72b172a689d8c4e412199 Mon Sep 17 00:00:00 2001 From: Christoph Grothaus Date: Fri, 15 Feb 2013 14:11:34 +0100 Subject: [PATCH 01/94] Fix SPARK-698. From ExecutorRunner, launch java directly instead via the run scripts. --- .../spark/deploy/worker/ExecutorRunner.scala | 43 +++++++++++++++++-- run | 3 ++ run2.cmd | 3 ++ 3 files changed, 45 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index de11771c8e62d..214c44fc888c0 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -75,9 +75,45 @@ private[spark] class ExecutorRunner( def buildCommandSeq(): Seq[String] = { val command = appDesc.command - val script = if (System.getProperty("os.name").startsWith("Windows")) "run.cmd" else "run" - val runScript = new File(sparkHome, script).getCanonicalPath - Seq(runScript, command.mainClass) ++ (command.arguments ++ Seq(appId)).map(substituteVariables) + val runner = if (getEnvOrEmpty("JAVA_HOME") == "") { + "java" + } else { + getEnvOrEmpty("JAVA_HOME") + "/bin/java" + } + // SPARK-698: do not call the run.cmd script, as process.destroy() + // fails to kill a process tree on Windows + Seq(runner) ++ buildJavaOpts() ++ Seq(command.mainClass) ++ + command.arguments.map(substituteVariables) + } + + /* + * Attention: this must always be aligned with the environment variables in the run scripts and the + * way the JAVA_OPTS are assembled there. + */ + def buildJavaOpts(): Seq[String] = { + val _javaLibPath = if (getEnvOrEmpty("SPARK_LIBRARY_PATH") == "") { + "" + } else { + "-Djava.library.path=" + getEnvOrEmpty("SPARK_LIBRARY_PATH") + } + + Seq("-cp", + getEnvOrEmpty("CLASSPATH"), + // SPARK_JAVA_OPTS is overwritten with SPARK_DAEMON_JAVA_OPTS for running the worker + getEnvOrEmpty("SPARK_NONDAEMON_JAVA_OPTS"), + _javaLibPath, + "-Xms" + memory.toString + "M", + "-Xmx" + memory.toString + "M") + .filter(_ != "") + } + + def getEnvOrEmpty(key: String): String = { + val result = System.getenv(key) + if (result == null) { + "" + } else { + result + } } /** Spawn a thread that will redirect a given stream to a file */ @@ -113,7 +149,6 @@ private[spark] class ExecutorRunner( for ((key, value) <- appDesc.command.environment) { env.put(key, value) } - env.put("SPARK_MEM", memory.toString + "m") // In case we are running this from within the Spark Shell, avoid creating a "scala" // parent process for the executor command env.put("SPARK_LAUNCH_WITH_SCALA", "0") diff --git a/run b/run index 82b1da005a447..b5f693f1faee3 100755 --- a/run +++ b/run @@ -22,6 +22,8 @@ fi # values for that; it doesn't need a lot if [ "$1" = "spark.deploy.master.Master" -o "$1" = "spark.deploy.worker.Worker" ]; then SPARK_MEM=${SPARK_DAEMON_MEMORY:-512m} + # Backup current SPARK_JAVA_OPTS for use in ExecutorRunner.scala + SPARK_NONDAEMON_JAVA_OPTS=$SPARK_JAVA_OPTS SPARK_JAVA_OPTS=$SPARK_DAEMON_JAVA_OPTS # Empty by default fi @@ -70,6 +72,7 @@ if [ -e $FWDIR/conf/java-opts ] ; then JAVA_OPTS+=" `cat $FWDIR/conf/java-opts`" fi export JAVA_OPTS +# Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala! CORE_DIR="$FWDIR/core" REPL_DIR="$FWDIR/repl" diff --git a/run2.cmd b/run2.cmd index c913a5195ef95..a93bbad0b9908 100644 --- a/run2.cmd +++ b/run2.cmd @@ -22,6 +22,8 @@ if "%1"=="spark.deploy.master.Master" set RUNNING_DAEMON=1 if "%1"=="spark.deploy.worker.Worker" set RUNNING_DAEMON=1 if "x%SPARK_DAEMON_MEMORY%" == "x" set SPARK_DAEMON_MEMORY=512m if "%RUNNING_DAEMON%"=="1" set SPARK_MEM=%SPARK_DAEMON_MEMORY% +rem Backup current SPARK_JAVA_OPTS for use in ExecutorRunner.scala +if "%RUNNING_DAEMON%"=="1" set SPARK_NONDAEMON_JAVA_OPTS=%SPARK_JAVA_OPTS% if "%RUNNING_DAEMON%"=="1" set SPARK_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% rem Check that SCALA_HOME has been specified @@ -42,6 +44,7 @@ rem Set JAVA_OPTS to be able to load native libraries and to set heap size set JAVA_OPTS=%SPARK_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%SPARK_MEM% -Xmx%SPARK_MEM% rem Load extra JAVA_OPTS from conf/java-opts, if it exists if exist "%FWDIR%conf\java-opts.cmd" call "%FWDIR%conf\java-opts.cmd" +rem Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala! set CORE_DIR=%FWDIR%core set REPL_DIR=%FWDIR%repl From f39f2b7636f52568a556987c8b7f7393299b0351 Mon Sep 17 00:00:00 2001 From: Christoph Grothaus Date: Sun, 24 Feb 2013 21:24:30 +0100 Subject: [PATCH 02/94] Incorporate feedback from mateiz: - we do not need getEnvOrEmpty - Instead of saving SPARK_NONDAEMON_JAVA_OPTS, it would be better to modify the scripts to use a different variable name for the JAVA_OPTS they do eventually use --- .../spark/deploy/worker/ExecutorRunner.scala | 24 ++++++------------- run | 9 +++---- run2.cmd | 8 +++---- 3 files changed, 16 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index 214c44fc888c0..38216ce62f526 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -75,10 +75,10 @@ private[spark] class ExecutorRunner( def buildCommandSeq(): Seq[String] = { val command = appDesc.command - val runner = if (getEnvOrEmpty("JAVA_HOME") == "") { + val runner = if (System.getenv("JAVA_HOME") == null) { "java" } else { - getEnvOrEmpty("JAVA_HOME") + "/bin/java" + System.getenv("JAVA_HOME") + "/bin/java" } // SPARK-698: do not call the run.cmd script, as process.destroy() // fails to kill a process tree on Windows @@ -91,29 +91,19 @@ private[spark] class ExecutorRunner( * way the JAVA_OPTS are assembled there. */ def buildJavaOpts(): Seq[String] = { - val _javaLibPath = if (getEnvOrEmpty("SPARK_LIBRARY_PATH") == "") { + val _javaLibPath = if (System.getenv("SPARK_LIBRARY_PATH") == null) { "" } else { - "-Djava.library.path=" + getEnvOrEmpty("SPARK_LIBRARY_PATH") + "-Djava.library.path=" + System.getenv("SPARK_LIBRARY_PATH") } Seq("-cp", - getEnvOrEmpty("CLASSPATH"), - // SPARK_JAVA_OPTS is overwritten with SPARK_DAEMON_JAVA_OPTS for running the worker - getEnvOrEmpty("SPARK_NONDAEMON_JAVA_OPTS"), + System.getenv("CLASSPATH"), + System.getenv("SPARK_JAVA_OPTS"), _javaLibPath, "-Xms" + memory.toString + "M", "-Xmx" + memory.toString + "M") - .filter(_ != "") - } - - def getEnvOrEmpty(key: String): String = { - val result = System.getenv(key) - if (result == null) { - "" - } else { - result - } + .filter(_ != null) } /** Spawn a thread that will redirect a given stream to a file */ diff --git a/run b/run index b5f693f1faee3..e1482dafbe71d 100755 --- a/run +++ b/run @@ -22,9 +22,10 @@ fi # values for that; it doesn't need a lot if [ "$1" = "spark.deploy.master.Master" -o "$1" = "spark.deploy.worker.Worker" ]; then SPARK_MEM=${SPARK_DAEMON_MEMORY:-512m} - # Backup current SPARK_JAVA_OPTS for use in ExecutorRunner.scala - SPARK_NONDAEMON_JAVA_OPTS=$SPARK_JAVA_OPTS - SPARK_JAVA_OPTS=$SPARK_DAEMON_JAVA_OPTS # Empty by default + # Do not overwrite SPARK_JAVA_OPTS environment variable in this script + OUR_JAVA_OPTS=$SPARK_DAEMON_JAVA_OPTS # Empty by default +else + OUR_JAVA_OPTS=$SPARK_JAVA_OPTS fi if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then @@ -64,7 +65,7 @@ fi export SPARK_MEM # Set JAVA_OPTS to be able to load native libraries and to set heap size -JAVA_OPTS="$SPARK_JAVA_OPTS" +JAVA_OPTS="$OUR_JAVA_OPTS" JAVA_OPTS+=" -Djava.library.path=$SPARK_LIBRARY_PATH" JAVA_OPTS+=" -Xms$SPARK_MEM -Xmx$SPARK_MEM" # Load extra JAVA_OPTS from conf/java-opts, if it exists diff --git a/run2.cmd b/run2.cmd index a93bbad0b9908..8648c0380a3d4 100644 --- a/run2.cmd +++ b/run2.cmd @@ -22,9 +22,9 @@ if "%1"=="spark.deploy.master.Master" set RUNNING_DAEMON=1 if "%1"=="spark.deploy.worker.Worker" set RUNNING_DAEMON=1 if "x%SPARK_DAEMON_MEMORY%" == "x" set SPARK_DAEMON_MEMORY=512m if "%RUNNING_DAEMON%"=="1" set SPARK_MEM=%SPARK_DAEMON_MEMORY% -rem Backup current SPARK_JAVA_OPTS for use in ExecutorRunner.scala -if "%RUNNING_DAEMON%"=="1" set SPARK_NONDAEMON_JAVA_OPTS=%SPARK_JAVA_OPTS% -if "%RUNNING_DAEMON%"=="1" set SPARK_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% +rem Do not overwrite SPARK_JAVA_OPTS environment variable in this script +if "%RUNNING_DAEMON%"=="0" set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% +if "%RUNNING_DAEMON%"=="1" set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% rem Check that SCALA_HOME has been specified if not "x%SCALA_HOME%"=="x" goto scala_exists @@ -41,7 +41,7 @@ rem variable so that our process sees it and can report it to Mesos if "x%SPARK_MEM%"=="x" set SPARK_MEM=512m rem Set JAVA_OPTS to be able to load native libraries and to set heap size -set JAVA_OPTS=%SPARK_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%SPARK_MEM% -Xmx%SPARK_MEM% +set JAVA_OPTS=%OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%SPARK_MEM% -Xmx%SPARK_MEM% rem Load extra JAVA_OPTS from conf/java-opts, if it exists if exist "%FWDIR%conf\java-opts.cmd" call "%FWDIR%conf\java-opts.cmd" rem Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala! From 4aa1205202f26663f59347f25a7d1f03c755545d Mon Sep 17 00:00:00 2001 From: seanm Date: Mon, 11 Mar 2013 12:37:29 -0600 Subject: [PATCH 03/94] adding typesafe repo to streaming resolvers so that akka-zeromq is found --- project/SparkBuild.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b0b6e21681b29..44c8058e9d673 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -162,6 +162,9 @@ object SparkBuild extends Build { def streamingSettings = sharedSettings ++ Seq( name := "spark-streaming", + resolvers ++= Seq( + "Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/" + ), libraryDependencies ++= Seq( "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile", "com.github.sgroschupf" % "zkclient" % "0.1", From c1c3682c984c83f75352fc22dcadd3e46058cfaf Mon Sep 17 00:00:00 2001 From: seanm Date: Mon, 11 Mar 2013 12:40:44 -0600 Subject: [PATCH 04/94] adding checkpoint dir to .gitignore --- .gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitignore b/.gitignore index 155e785b01beb..6c9ffa5426593 100644 --- a/.gitignore +++ b/.gitignore @@ -36,3 +36,4 @@ streaming-tests.log dependency-reduced-pom.xml .ensime .ensime_lucene +checkpoint From 42822cf95de71039988e22d8690ba6a4bd639227 Mon Sep 17 00:00:00 2001 From: seanm Date: Wed, 13 Mar 2013 11:40:42 -0600 Subject: [PATCH 05/94] changing streaming resolver for akka --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 44c8058e9d673..7e65979a5d6bb 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -163,7 +163,7 @@ object SparkBuild extends Build { def streamingSettings = sharedSettings ++ Seq( name := "spark-streaming", resolvers ++= Seq( - "Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/" + "Akka Repository" at "http://repo.akka.io/releases/" ), libraryDependencies ++= Seq( "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile", From cfa8e769a86664722f47182fa572179e8beadcb7 Mon Sep 17 00:00:00 2001 From: seanm Date: Mon, 11 Mar 2013 17:16:15 -0600 Subject: [PATCH 06/94] KafkaInputDStream improvements. Allows more Kafka configurability --- .../spark/streaming/StreamingContext.scala | 22 ++++++++- .../streaming/dstream/KafkaInputDStream.scala | 48 ++++++++++++------- 2 files changed, 51 insertions(+), 19 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index 25c67b279b7d2..4e1732adf5608 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -199,7 +199,7 @@ class StreamingContext private ( } /** - * Create an input stream that pulls messages form a Kafka Broker. + * Create an input stream that pulls messages from a Kafka Broker. * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..). * @param groupId The group id for this consumer. * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed @@ -216,7 +216,25 @@ class StreamingContext private ( initialOffsets: Map[KafkaPartitionKey, Long] = Map[KafkaPartitionKey, Long](), storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2 ): DStream[T] = { - val inputStream = new KafkaInputDStream[T](this, zkQuorum, groupId, topics, initialOffsets, storageLevel) + val kafkaParams = Map[String, String]("zk.connect" -> zkQuorum, "groupid" -> groupId, "zk.connectiontimeout.ms" -> "10000"); + kafkaStream[T](kafkaParams, topics, initialOffsets, storageLevel) + } + + /** + * Create an input stream that pulls messages from a Kafka Broker. + * @param kafkaParams Map of kafka configuration paramaters. See: http://kafka.apache.org/configuration.html + * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * in its own thread. + * @param initialOffsets Optional initial offsets for each of the partitions to consume. + * @param storageLevel Storage level to use for storing the received objects + */ + def kafkaStream[T: ClassManifest]( + kafkaParams: Map[String, String], + topics: Map[String, Int], + initialOffsets: Map[KafkaPartitionKey, Long], + storageLevel: StorageLevel + ): DStream[T] = { + val inputStream = new KafkaInputDStream[T](this, kafkaParams, topics, initialOffsets, storageLevel) registerInputStream(inputStream) inputStream } diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index dc7139cc273cf..f769fc1cc35e0 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -12,6 +12,8 @@ import kafka.message.{Message, MessageSet, MessageAndMetadata} import kafka.serializer.StringDecoder import kafka.utils.{Utils, ZKGroupTopicDirs} import kafka.utils.ZkUtils._ +import kafka.utils.ZKStringSerializer +import org.I0Itec.zkclient._ import scala.collection.mutable.HashMap import scala.collection.JavaConversions._ @@ -23,8 +25,7 @@ case class KafkaPartitionKey(brokerId: Int, topic: String, groupId: String, part /** * Input stream that pulls messages from a Kafka Broker. * - * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..). - * @param groupId The group id for this consumer. + * @param kafkaParams Map of kafka configuration paramaters. See: http://kafka.apache.org/configuration.html * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed * in its own thread. * @param initialOffsets Optional initial offsets for each of the partitions to consume. @@ -34,8 +35,7 @@ case class KafkaPartitionKey(brokerId: Int, topic: String, groupId: String, part private[streaming] class KafkaInputDStream[T: ClassManifest]( @transient ssc_ : StreamingContext, - zkQuorum: String, - groupId: String, + kafkaParams: Map[String, String], topics: Map[String, Int], initialOffsets: Map[KafkaPartitionKey, Long], storageLevel: StorageLevel @@ -43,19 +43,16 @@ class KafkaInputDStream[T: ClassManifest]( def getReceiver(): NetworkReceiver[T] = { - new KafkaReceiver(zkQuorum, groupId, topics, initialOffsets, storageLevel) + new KafkaReceiver(kafkaParams, topics, initialOffsets, storageLevel) .asInstanceOf[NetworkReceiver[T]] } } private[streaming] -class KafkaReceiver(zkQuorum: String, groupId: String, +class KafkaReceiver(kafkaParams: Map[String, String], topics: Map[String, Int], initialOffsets: Map[KafkaPartitionKey, Long], storageLevel: StorageLevel) extends NetworkReceiver[Any] { - // Timeout for establishing a connection to Zookeper in ms. - val ZK_TIMEOUT = 10000 - // Handles pushing data into the BlockManager lazy protected val blockGenerator = new BlockGenerator(storageLevel) // Connection to Kafka @@ -72,20 +69,24 @@ class KafkaReceiver(zkQuorum: String, groupId: String, // In case we are using multiple Threads to handle Kafka Messages val executorPool = Executors.newFixedThreadPool(topics.values.reduce(_ + _)) - logInfo("Starting Kafka Consumer Stream with group: " + groupId) + logInfo("Starting Kafka Consumer Stream with group: " + kafkaParams("groupid")) logInfo("Initial offsets: " + initialOffsets.toString) - // Zookeper connection properties + // Kafka connection properties val props = new Properties() - props.put("zk.connect", zkQuorum) - props.put("zk.connectiontimeout.ms", ZK_TIMEOUT.toString) - props.put("groupid", groupId) + kafkaParams.foreach(param => props.put(param._1, param._2)) // Create the connection to the cluster - logInfo("Connecting to Zookeper: " + zkQuorum) + logInfo("Connecting to Zookeper: " + kafkaParams("zk.connect")) val consumerConfig = new ConsumerConfig(props) consumerConnector = Consumer.create(consumerConfig).asInstanceOf[ZookeeperConsumerConnector] - logInfo("Connected to " + zkQuorum) + logInfo("Connected to " + kafkaParams("zk.connect")) + + // When autooffset.reset is 'smallest', it is our responsibility to try and whack the + // consumer group zk node. + if (kafkaParams.get("autooffset.reset").exists(_ == "smallest")) { + tryZookeeperConsumerGroupCleanup(kafkaParams("zk.connect"), kafkaParams("groupid")) + } // If specified, set the topic offset setOffsets(initialOffsets) @@ -97,7 +98,6 @@ class KafkaReceiver(zkQuorum: String, groupId: String, topicMessageStreams.values.foreach { streams => streams.foreach { stream => executorPool.submit(new MessageHandler(stream)) } } - } // Overwrites the offets in Zookeper. @@ -122,4 +122,18 @@ class KafkaReceiver(zkQuorum: String, groupId: String, } } } + + // Handles cleanup of consumer group znode. Lifted with love from Kafka's + // ConsumerConsole.scala tryCleanupZookeeper() + private def tryZookeeperConsumerGroupCleanup(zkUrl: String, groupId: String) { + try { + val dir = "/consumers/" + groupId + logInfo("Cleaning up temporary zookeeper data under " + dir + ".") + val zk = new ZkClient(zkUrl, 30*1000, 30*1000, ZKStringSerializer) + zk.deleteRecursive(dir) + zk.close() + } catch { + case _ => // swallow + } + } } From d06928321194b11e082986cd2bb2737d9bc3b698 Mon Sep 17 00:00:00 2001 From: seanm Date: Thu, 14 Mar 2013 23:25:35 -0600 Subject: [PATCH 07/94] fixing memory leak in kafka MessageHandler --- .../scala/spark/streaming/dstream/KafkaInputDStream.scala | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index f769fc1cc35e0..d674b6ee87a78 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -114,11 +114,8 @@ class KafkaReceiver(kafkaParams: Map[String, String], private class MessageHandler(stream: KafkaStream[String]) extends Runnable { def run() { logInfo("Starting MessageHandler.") - stream.takeWhile { msgAndMetadata => + for (msgAndMetadata <- stream) { blockGenerator += msgAndMetadata.message - // Keep on handling messages - - true } } } From 33fa1e7e4aca4d9e0edf65d2b768b569305fd044 Mon Sep 17 00:00:00 2001 From: seanm Date: Thu, 14 Mar 2013 23:32:52 -0600 Subject: [PATCH 08/94] removing dependency on ZookeeperConsumerConnector + purging last relic of kafka reliability that never solidified (ie- setOffsets) --- .../spark/streaming/StreamingContext.scala | 9 ++---- .../api/java/JavaStreamingContext.scala | 28 ------------------- .../streaming/dstream/KafkaInputDStream.scala | 28 +++---------------- 3 files changed, 6 insertions(+), 59 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index 4e1732adf5608..bb7f216ca7dcd 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -204,8 +204,6 @@ class StreamingContext private ( * @param groupId The group id for this consumer. * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed * in its own thread. - * @param initialOffsets Optional initial offsets for each of the partitions to consume. - * By default the value is pulled from zookeper. * @param storageLevel Storage level to use for storing the received objects * (default: StorageLevel.MEMORY_AND_DISK_SER_2) */ @@ -213,11 +211,10 @@ class StreamingContext private ( zkQuorum: String, groupId: String, topics: Map[String, Int], - initialOffsets: Map[KafkaPartitionKey, Long] = Map[KafkaPartitionKey, Long](), storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2 ): DStream[T] = { val kafkaParams = Map[String, String]("zk.connect" -> zkQuorum, "groupid" -> groupId, "zk.connectiontimeout.ms" -> "10000"); - kafkaStream[T](kafkaParams, topics, initialOffsets, storageLevel) + kafkaStream[T](kafkaParams, topics, storageLevel) } /** @@ -225,16 +222,14 @@ class StreamingContext private ( * @param kafkaParams Map of kafka configuration paramaters. See: http://kafka.apache.org/configuration.html * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed * in its own thread. - * @param initialOffsets Optional initial offsets for each of the partitions to consume. * @param storageLevel Storage level to use for storing the received objects */ def kafkaStream[T: ClassManifest]( kafkaParams: Map[String, String], topics: Map[String, Int], - initialOffsets: Map[KafkaPartitionKey, Long], storageLevel: StorageLevel ): DStream[T] = { - val inputStream = new KafkaInputDStream[T](this, kafkaParams, topics, initialOffsets, storageLevel) + val inputStream = new KafkaInputDStream[T](this, kafkaParams, topics, storageLevel) registerInputStream(inputStream) inputStream } diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index f3b40b5b88ef9..2373f4824a89a 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -84,39 +84,12 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param groupId The group id for this consumer. * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed * in its own thread. - * @param initialOffsets Optional initial offsets for each of the partitions to consume. - * By default the value is pulled from zookeper. - */ - def kafkaStream[T]( - zkQuorum: String, - groupId: String, - topics: JMap[String, JInt], - initialOffsets: JMap[KafkaPartitionKey, JLong]) - : JavaDStream[T] = { - implicit val cmt: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] - ssc.kafkaStream[T]( - zkQuorum, - groupId, - Map(topics.mapValues(_.intValue()).toSeq: _*), - Map(initialOffsets.mapValues(_.longValue()).toSeq: _*)) - } - - /** - * Create an input stream that pulls messages form a Kafka Broker. - * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..). - * @param groupId The group id for this consumer. - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread. - * @param initialOffsets Optional initial offsets for each of the partitions to consume. - * By default the value is pulled from zookeper. * @param storageLevel RDD storage level. Defaults to memory-only */ def kafkaStream[T]( zkQuorum: String, groupId: String, topics: JMap[String, JInt], - initialOffsets: JMap[KafkaPartitionKey, JLong], storageLevel: StorageLevel) : JavaDStream[T] = { implicit val cmt: ClassManifest[T] = @@ -125,7 +98,6 @@ class JavaStreamingContext(val ssc: StreamingContext) { zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), - Map(initialOffsets.mapValues(_.longValue()).toSeq: _*), storageLevel) } diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index d674b6ee87a78..c6da1a7f70b7f 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -19,17 +19,12 @@ import scala.collection.mutable.HashMap import scala.collection.JavaConversions._ -// Key for a specific Kafka Partition: (broker, topic, group, part) -case class KafkaPartitionKey(brokerId: Int, topic: String, groupId: String, partId: Int) - /** * Input stream that pulls messages from a Kafka Broker. * * @param kafkaParams Map of kafka configuration paramaters. See: http://kafka.apache.org/configuration.html * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed * in its own thread. - * @param initialOffsets Optional initial offsets for each of the partitions to consume. - * By default the value is pulled from zookeper. * @param storageLevel RDD storage level. */ private[streaming] @@ -37,26 +32,25 @@ class KafkaInputDStream[T: ClassManifest]( @transient ssc_ : StreamingContext, kafkaParams: Map[String, String], topics: Map[String, Int], - initialOffsets: Map[KafkaPartitionKey, Long], storageLevel: StorageLevel ) extends NetworkInputDStream[T](ssc_ ) with Logging { def getReceiver(): NetworkReceiver[T] = { - new KafkaReceiver(kafkaParams, topics, initialOffsets, storageLevel) + new KafkaReceiver(kafkaParams, topics, storageLevel) .asInstanceOf[NetworkReceiver[T]] } } private[streaming] class KafkaReceiver(kafkaParams: Map[String, String], - topics: Map[String, Int], initialOffsets: Map[KafkaPartitionKey, Long], + topics: Map[String, Int], storageLevel: StorageLevel) extends NetworkReceiver[Any] { // Handles pushing data into the BlockManager lazy protected val blockGenerator = new BlockGenerator(storageLevel) // Connection to Kafka - var consumerConnector : ZookeeperConsumerConnector = null + var consumerConnector : ConsumerConnector = null def onStop() { blockGenerator.stop() @@ -70,7 +64,6 @@ class KafkaReceiver(kafkaParams: Map[String, String], val executorPool = Executors.newFixedThreadPool(topics.values.reduce(_ + _)) logInfo("Starting Kafka Consumer Stream with group: " + kafkaParams("groupid")) - logInfo("Initial offsets: " + initialOffsets.toString) // Kafka connection properties val props = new Properties() @@ -79,7 +72,7 @@ class KafkaReceiver(kafkaParams: Map[String, String], // Create the connection to the cluster logInfo("Connecting to Zookeper: " + kafkaParams("zk.connect")) val consumerConfig = new ConsumerConfig(props) - consumerConnector = Consumer.create(consumerConfig).asInstanceOf[ZookeeperConsumerConnector] + consumerConnector = Consumer.create(consumerConfig) logInfo("Connected to " + kafkaParams("zk.connect")) // When autooffset.reset is 'smallest', it is our responsibility to try and whack the @@ -88,9 +81,6 @@ class KafkaReceiver(kafkaParams: Map[String, String], tryZookeeperConsumerGroupCleanup(kafkaParams("zk.connect"), kafkaParams("groupid")) } - // If specified, set the topic offset - setOffsets(initialOffsets) - // Create Threads for each Topic/Message Stream we are listening val topicMessageStreams = consumerConnector.createMessageStreams(topics, new StringDecoder()) @@ -100,16 +90,6 @@ class KafkaReceiver(kafkaParams: Map[String, String], } } - // Overwrites the offets in Zookeper. - private def setOffsets(offsets: Map[KafkaPartitionKey, Long]) { - offsets.foreach { case(key, offset) => - val topicDirs = new ZKGroupTopicDirs(key.groupId, key.topic) - val partitionName = key.brokerId + "-" + key.partId - updatePersistentPath(consumerConnector.zkClient, - topicDirs.consumerOffsetDir + "/" + partitionName, offset.toString) - } - } - // Handles Kafka Messages private class MessageHandler(stream: KafkaStream[String]) extends Runnable { def run() { From d61978d0abad30a148680c8a63df33e40e469525 Mon Sep 17 00:00:00 2001 From: seanm Date: Fri, 15 Mar 2013 23:36:52 -0600 Subject: [PATCH 09/94] keeping JavaStreamingContext in sync with StreamingContext + adding comments for better clarity --- .../spark/streaming/api/java/JavaStreamingContext.scala | 7 +++---- .../scala/spark/streaming/dstream/KafkaInputDStream.scala | 6 ++++-- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index 2373f4824a89a..7a8864614c95f 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -80,6 +80,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { /** * Create an input stream that pulls messages form a Kafka Broker. + * @param kafkaParams Map of kafka configuration paramaters. See: http://kafka.apache.org/configuration.html * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..). * @param groupId The group id for this consumer. * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed @@ -87,16 +88,14 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param storageLevel RDD storage level. Defaults to memory-only */ def kafkaStream[T]( - zkQuorum: String, - groupId: String, + kafkaParams: JMap[String, String], topics: JMap[String, JInt], storageLevel: StorageLevel) : JavaDStream[T] = { implicit val cmt: ClassManifest[T] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] ssc.kafkaStream[T]( - zkQuorum, - groupId, + kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) } diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index c6da1a7f70b7f..85693808d1046 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -100,8 +100,10 @@ class KafkaReceiver(kafkaParams: Map[String, String], } } - // Handles cleanup of consumer group znode. Lifted with love from Kafka's - // ConsumerConsole.scala tryCleanupZookeeper() + // Delete consumer group from zookeeper. This effectivly resets the group so we can consume from the beginning again. + // The kafka high level consumer doesn't expose setting offsets currently, this is a trick copied from Kafkas' + // ConsoleConsumer. See code related to 'autooffset.reset' when it is set to 'smallest': + // https://github.com/apache/kafka/blob/0.7.2/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala private def tryZookeeperConsumerGroupCleanup(zkUrl: String, groupId: String) { try { val dir = "/consumers/" + groupId From 329ef34c2e04d28c2ad150cf6674d6e86d7511ce Mon Sep 17 00:00:00 2001 From: seanm Date: Tue, 26 Mar 2013 23:56:15 -0600 Subject: [PATCH 10/94] fixing autooffset.reset behavior when set to 'largest' --- .../spark/streaming/dstream/KafkaInputDStream.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index 85693808d1046..17a5be3420d41 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -75,9 +75,9 @@ class KafkaReceiver(kafkaParams: Map[String, String], consumerConnector = Consumer.create(consumerConfig) logInfo("Connected to " + kafkaParams("zk.connect")) - // When autooffset.reset is 'smallest', it is our responsibility to try and whack the + // When autooffset.reset is defined, it is our responsibility to try and whack the // consumer group zk node. - if (kafkaParams.get("autooffset.reset").exists(_ == "smallest")) { + if (kafkaParams.contains("autooffset.reset")) { tryZookeeperConsumerGroupCleanup(kafkaParams("zk.connect"), kafkaParams("groupid")) } @@ -100,9 +100,11 @@ class KafkaReceiver(kafkaParams: Map[String, String], } } - // Delete consumer group from zookeeper. This effectivly resets the group so we can consume from the beginning again. + // It is our responsibility to delete the consumer group when specifying autooffset.reset. This is because + // Kafka 0.7.2 only honors this param when the group is not in zookeeper. + // // The kafka high level consumer doesn't expose setting offsets currently, this is a trick copied from Kafkas' - // ConsoleConsumer. See code related to 'autooffset.reset' when it is set to 'smallest': + // ConsoleConsumer. See code related to 'autooffset.reset' when it is set to 'smallest'/'largest': // https://github.com/apache/kafka/blob/0.7.2/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala private def tryZookeeperConsumerGroupCleanup(zkUrl: String, groupId: String) { try { From b42d68c8ce9f63513969297b65f4b5a2b06e6078 Mon Sep 17 00:00:00 2001 From: seanm Date: Mon, 15 Apr 2013 12:54:55 -0600 Subject: [PATCH 11/94] fixing Spark Streaming count() so that 0 will be emitted when there is nothing to count --- streaming/src/main/scala/spark/streaming/DStream.scala | 5 ++++- .../test/scala/spark/streaming/BasicOperationsSuite.scala | 4 ++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/spark/streaming/DStream.scala index e1be5ef51cc9c..e3a9247924f12 100644 --- a/streaming/src/main/scala/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/spark/streaming/DStream.scala @@ -441,7 +441,10 @@ abstract class DStream[T: ClassManifest] ( * Return a new DStream in which each RDD has a single element generated by counting each RDD * of this DStream. */ - def count(): DStream[Long] = this.map(_ => 1L).reduce(_ + _) + def count(): DStream[Long] = { + val zero = new ConstantInputDStream(context, context.sparkContext.makeRDD(Seq((null, 0L)), 1)) + this.map(_ => (null, 1L)).union(zero).reduceByKey(_ + _).map(_._2) + } /** * Return a new DStream in which each RDD contains the counts of each distinct value in diff --git a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala index 8fce91853c77e..168e1b7a557c9 100644 --- a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala @@ -90,9 +90,9 @@ class BasicOperationsSuite extends TestSuiteBase { test("count") { testOperation( - Seq(1 to 1, 1 to 2, 1 to 3, 1 to 4), + Seq(Seq(), 1 to 1, 1 to 2, 1 to 3, 1 to 4), (s: DStream[Int]) => s.count(), - Seq(Seq(1L), Seq(2L), Seq(3L), Seq(4L)) + Seq(Seq(0L), Seq(1L), Seq(2L), Seq(3L), Seq(4L)) ) } From ab0f834dbb509d323577572691293b74368a9d86 Mon Sep 17 00:00:00 2001 From: seanm Date: Tue, 16 Apr 2013 11:57:05 -0600 Subject: [PATCH 12/94] adding spark.streaming.blockInterval property --- docs/configuration.md | 7 +++++++ .../spark/streaming/dstream/NetworkInputDStream.scala | 2 +- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index 04eb6daaa5d01..55f1962b18bd2 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -253,6 +253,13 @@ Apart from these, the following properties are also available, and may be useful applications). Note that any RDD that persists in memory for more than this duration will be cleared as well. + + spark.streaming.blockInterval + 200 + + Duration (milliseconds) of how long to batch new objects coming from network receivers. + + diff --git a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala index 7385474963ebb..26805e96210ab 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala @@ -198,7 +198,7 @@ abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Log case class Block(id: String, iterator: Iterator[T], metadata: Any = null) val clock = new SystemClock() - val blockInterval = 200L + val blockInterval = System.getProperty("spark.streaming.blockInterval", "200").toLong val blockIntervalTimer = new RecurringTimer(clock, blockInterval, updateCurrentBuffer) val blockStorageLevel = storageLevel val blocksForPushing = new ArrayBlockingQueue[Block](1000) From 7e56e99573b4cf161293e648aeb159375c9c0fcb Mon Sep 17 00:00:00 2001 From: seanm Date: Sun, 24 Mar 2013 13:40:19 -0600 Subject: [PATCH 13/94] Surfacing decoders on KafkaInputDStream --- .../streaming/examples/KafkaWordCount.scala | 2 +- .../spark/streaming/StreamingContext.scala | 11 ++++--- .../api/java/JavaStreamingContext.scala | 33 ++++++++++++++----- .../streaming/dstream/KafkaInputDStream.scala | 17 ++++++---- .../java/spark/streaming/JavaAPISuite.java | 6 ++-- 5 files changed, 44 insertions(+), 25 deletions(-) diff --git a/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala index 9b135a5c54cf3..e0c3555f21b29 100644 --- a/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala @@ -37,7 +37,7 @@ object KafkaWordCount { ssc.checkpoint("checkpoint") val topicpMap = topics.split(",").map((_,numThreads.toInt)).toMap - val lines = ssc.kafkaStream[String](zkQuorum, group, topicpMap) + val lines = ssc.kafkaStream(zkQuorum, group, topicpMap) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1l)).reduceByKeyAndWindow(add _, subtract _, Minutes(10), Seconds(2), 2) wordCounts.print() diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index bb7f216ca7dcd..2c6326943dc99 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -28,6 +28,7 @@ import org.apache.hadoop.fs.Path import java.util.UUID import twitter4j.Status + /** * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic * information (such as, cluster URL and job name) to internally create a SparkContext, it provides @@ -207,14 +208,14 @@ class StreamingContext private ( * @param storageLevel Storage level to use for storing the received objects * (default: StorageLevel.MEMORY_AND_DISK_SER_2) */ - def kafkaStream[T: ClassManifest]( + def kafkaStream( zkQuorum: String, groupId: String, topics: Map[String, Int], storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2 - ): DStream[T] = { + ): DStream[String] = { val kafkaParams = Map[String, String]("zk.connect" -> zkQuorum, "groupid" -> groupId, "zk.connectiontimeout.ms" -> "10000"); - kafkaStream[T](kafkaParams, topics, storageLevel) + kafkaStream[String, kafka.serializer.StringDecoder](kafkaParams, topics, storageLevel) } /** @@ -224,12 +225,12 @@ class StreamingContext private ( * in its own thread. * @param storageLevel Storage level to use for storing the received objects */ - def kafkaStream[T: ClassManifest]( + def kafkaStream[T: ClassManifest, D <: kafka.serializer.Decoder[_]: Manifest]( kafkaParams: Map[String, String], topics: Map[String, Int], storageLevel: StorageLevel ): DStream[T] = { - val inputStream = new KafkaInputDStream[T](this, kafkaParams, topics, storageLevel) + val inputStream = new KafkaInputDStream[T, D](this, kafkaParams, topics, storageLevel) registerInputStream(inputStream) inputStream } diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index 7a8864614c95f..13427873ff530 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -68,33 +68,50 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed * in its own thread. */ - def kafkaStream[T]( + def kafkaStream( zkQuorum: String, groupId: String, topics: JMap[String, JInt]) - : JavaDStream[T] = { - implicit val cmt: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] - ssc.kafkaStream[T](zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*)) + : JavaDStream[String] = { + implicit val cmt: ClassManifest[String] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[String]] + ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), StorageLevel.MEMORY_ONLY_SER_2) } /** * Create an input stream that pulls messages form a Kafka Broker. - * @param kafkaParams Map of kafka configuration paramaters. See: http://kafka.apache.org/configuration.html * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..). * @param groupId The group id for this consumer. * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * @param storageLevel RDD storage level. Defaults to memory-only + * in its own thread. + */ + def kafkaStream( + zkQuorum: String, + groupId: String, + topics: JMap[String, JInt], + storageLevel: StorageLevel) + : JavaDStream[String] = { + implicit val cmt: ClassManifest[String] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[String]] + ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) + } + + /** + * Create an input stream that pulls messages form a Kafka Broker. + * @param kafkaParams Map of kafka configuration paramaters. See: http://kafka.apache.org/configuration.html + * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed * in its own thread. * @param storageLevel RDD storage level. Defaults to memory-only */ - def kafkaStream[T]( + def kafkaStream[T, D <: kafka.serializer.Decoder[_]: Manifest]( kafkaParams: JMap[String, String], topics: JMap[String, JInt], storageLevel: StorageLevel) : JavaDStream[T] = { implicit val cmt: ClassManifest[T] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] - ssc.kafkaStream[T]( + ssc.kafkaStream[T, D]( kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index 17a5be3420d41..7bd53fb6ddcb9 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -9,7 +9,7 @@ import java.util.concurrent.Executors import kafka.consumer._ import kafka.message.{Message, MessageSet, MessageAndMetadata} -import kafka.serializer.StringDecoder +import kafka.serializer.Decoder import kafka.utils.{Utils, ZKGroupTopicDirs} import kafka.utils.ZkUtils._ import kafka.utils.ZKStringSerializer @@ -28,7 +28,7 @@ import scala.collection.JavaConversions._ * @param storageLevel RDD storage level. */ private[streaming] -class KafkaInputDStream[T: ClassManifest]( +class KafkaInputDStream[T: ClassManifest, D <: Decoder[_]: Manifest]( @transient ssc_ : StreamingContext, kafkaParams: Map[String, String], topics: Map[String, Int], @@ -37,15 +37,17 @@ class KafkaInputDStream[T: ClassManifest]( def getReceiver(): NetworkReceiver[T] = { - new KafkaReceiver(kafkaParams, topics, storageLevel) + new KafkaReceiver[T, D](kafkaParams, topics, storageLevel) .asInstanceOf[NetworkReceiver[T]] } } private[streaming] -class KafkaReceiver(kafkaParams: Map[String, String], +class KafkaReceiver[T: ClassManifest, D <: Decoder[_]: Manifest]( + kafkaParams: Map[String, String], topics: Map[String, Int], - storageLevel: StorageLevel) extends NetworkReceiver[Any] { + storageLevel: StorageLevel + ) extends NetworkReceiver[Any] { // Handles pushing data into the BlockManager lazy protected val blockGenerator = new BlockGenerator(storageLevel) @@ -82,7 +84,8 @@ class KafkaReceiver(kafkaParams: Map[String, String], } // Create Threads for each Topic/Message Stream we are listening - val topicMessageStreams = consumerConnector.createMessageStreams(topics, new StringDecoder()) + val decoder = manifest[D].erasure.newInstance.asInstanceOf[Decoder[T]] + val topicMessageStreams = consumerConnector.createMessageStreams(topics, decoder) // Start the messages handler for each partition topicMessageStreams.values.foreach { streams => @@ -91,7 +94,7 @@ class KafkaReceiver(kafkaParams: Map[String, String], } // Handles Kafka Messages - private class MessageHandler(stream: KafkaStream[String]) extends Runnable { + private class MessageHandler[T: ClassManifest](stream: KafkaStream[T]) extends Runnable { def run() { logInfo("Starting MessageHandler.") for (msgAndMetadata <- stream) { diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java index 3bed500f73e84..61e4c0a207cef 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java @@ -23,7 +23,6 @@ import spark.streaming.api.java.JavaStreamingContext; import spark.streaming.JavaTestUtils; import spark.streaming.JavaCheckpointTestUtils; -import spark.streaming.dstream.KafkaPartitionKey; import spark.streaming.InputStreamsSuite; import java.io.*; @@ -1203,10 +1202,9 @@ public Integer call(String s) throws Exception { @Test public void testKafkaStream() { HashMap topics = Maps.newHashMap(); - HashMap offsets = Maps.newHashMap(); JavaDStream test1 = ssc.kafkaStream("localhost:12345", "group", topics); - JavaDStream test2 = ssc.kafkaStream("localhost:12345", "group", topics, offsets); - JavaDStream test3 = ssc.kafkaStream("localhost:12345", "group", topics, offsets, + JavaDStream test2 = ssc.kafkaStream("localhost:12345", "group", topics); + JavaDStream test3 = ssc.kafkaStream("localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK()); } From 0dc1e2d60f89f07f54e0985d37cdcd32ad388f6a Mon Sep 17 00:00:00 2001 From: unknown Date: Mon, 22 Apr 2013 09:22:45 -0600 Subject: [PATCH 14/94] Examaple of cumulative counting using updateStateByKey --- ...kWordCumulativeCountUpdateStateByKey.scala | 63 +++++++++++++++++++ 1 file changed, 63 insertions(+) create mode 100644 examples/src/main/scala/spark/streaming/examples/NetworkWordCumulativeCountUpdateStateByKey.scala diff --git a/examples/src/main/scala/spark/streaming/examples/NetworkWordCumulativeCountUpdateStateByKey.scala b/examples/src/main/scala/spark/streaming/examples/NetworkWordCumulativeCountUpdateStateByKey.scala new file mode 100644 index 0000000000000..db62246387e85 --- /dev/null +++ b/examples/src/main/scala/spark/streaming/examples/NetworkWordCumulativeCountUpdateStateByKey.scala @@ -0,0 +1,63 @@ +package spark.streaming.examples + +import spark.streaming._ +import spark.streaming.StreamingContext._ + +/** + * Counts words in UTF8 encoded, '\n' delimited text received from the network every second. + * Usage: NetworkWordCumulativeCountUpdateStateByKey + * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. + * and describe the TCP server that Spark Streaming would connect to receive data. + * + * To run this on your local machine, you need to first run a Netcat server + * `$ nc -lk 9999` + * and then run the example + * `$ ./run spark.streaming.examples.NetworkWordCumulativeCountUpdateStateByKey local[2] localhost 9999` + */ +object NetworkWordCumulativeCountUpdateStateByKey { + private def className[A](a: A)(implicit m: Manifest[A]) = m.toString + + def main(args: Array[String]) { + if (args.length < 3) { + System.err.println("Usage: NetworkWordCountUpdateStateByKey \n" + + "In local mode, should be 'local[n]' with n > 1") + System.exit(1) + } + + val updateFunc = (values: Seq[Int], state: Option[Int]) => { + val currentCount = values.foldLeft(0)(_ + _) + //println("currentCount: " + currentCount) + + val previousCount = state.getOrElse(0) + //println("previousCount: " + previousCount) + + val cumulative = Some(currentCount + previousCount) + //println("Cumulative: " + cumulative) + + cumulative + } + + // Create the context with a 10 second batch size + val ssc = new StreamingContext(args(0), "NetworkWordCumulativeCountUpdateStateByKey", Seconds(10), + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + ssc.checkpoint(".") + + // Create a NetworkInputDStream on target ip:port and count the + // words in input stream of \n delimited test (eg. generated by 'nc') + val lines = ssc.socketTextStream(args(1), args(2).toInt) + val words = lines.flatMap(_.split(" ")) + val wordDstream = words.map(x => (x, 1)) + + // Update the cumulative count using updateStateByKey + // This will give a Dstream made of state (which is the cumulative count of the words) + val stateDstream = wordDstream.updateStateByKey[Int](updateFunc) + + stateDstream.foreach(rdd => { + rdd.foreach(rddVal => { + println("Current Count: " + rddVal) + }) + }) + + ssc.start() + } +} From 1d54401d7e41095d8cbeeefd42c9d39ee500cd9f Mon Sep 17 00:00:00 2001 From: unknown Date: Tue, 30 Apr 2013 23:01:32 -0600 Subject: [PATCH 15/94] Modified as per TD's suggestions --- ...y.scala => StatefulNetworkWordCount.scala} | 23 +++++-------------- 1 file changed, 6 insertions(+), 17 deletions(-) rename examples/src/main/scala/spark/streaming/examples/{NetworkWordCumulativeCountUpdateStateByKey.scala => StatefulNetworkWordCount.scala} (71%) diff --git a/examples/src/main/scala/spark/streaming/examples/NetworkWordCumulativeCountUpdateStateByKey.scala b/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala similarity index 71% rename from examples/src/main/scala/spark/streaming/examples/NetworkWordCumulativeCountUpdateStateByKey.scala rename to examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala index db62246387e85..b662cb116281b 100644 --- a/examples/src/main/scala/spark/streaming/examples/NetworkWordCumulativeCountUpdateStateByKey.scala +++ b/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala @@ -5,36 +5,31 @@ import spark.streaming.StreamingContext._ /** * Counts words in UTF8 encoded, '\n' delimited text received from the network every second. - * Usage: NetworkWordCumulativeCountUpdateStateByKey + * Usage: StatefulNetworkWordCount * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. * and describe the TCP server that Spark Streaming would connect to receive data. * * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./run spark.streaming.examples.NetworkWordCumulativeCountUpdateStateByKey local[2] localhost 9999` + * `$ ./run spark.streaming.examples.StatefulNetworkWordCount local[2] localhost 9999` */ -object NetworkWordCumulativeCountUpdateStateByKey { +object StatefulNetworkWordCount { private def className[A](a: A)(implicit m: Manifest[A]) = m.toString def main(args: Array[String]) { if (args.length < 3) { - System.err.println("Usage: NetworkWordCountUpdateStateByKey \n" + + System.err.println("Usage: StatefulNetworkWordCount \n" + "In local mode, should be 'local[n]' with n > 1") System.exit(1) } val updateFunc = (values: Seq[Int], state: Option[Int]) => { val currentCount = values.foldLeft(0)(_ + _) - //println("currentCount: " + currentCount) val previousCount = state.getOrElse(0) - //println("previousCount: " + previousCount) - val cumulative = Some(currentCount + previousCount) - //println("Cumulative: " + cumulative) - - cumulative + Some(currentCount + previousCount) } // Create the context with a 10 second batch size @@ -51,13 +46,7 @@ object NetworkWordCumulativeCountUpdateStateByKey { // Update the cumulative count using updateStateByKey // This will give a Dstream made of state (which is the cumulative count of the words) val stateDstream = wordDstream.updateStateByKey[Int](updateFunc) - - stateDstream.foreach(rdd => { - rdd.foreach(rddVal => { - println("Current Count: " + rddVal) - }) - }) - + stateDstream.print() ssc.start() } } From cbf6a5ee1e7d290d04a0c5dac78d360266d415a4 Mon Sep 17 00:00:00 2001 From: unknown Date: Mon, 6 May 2013 08:05:45 -0600 Subject: [PATCH 16/94] Removed unused code, clarified intent of the program, batch size to 1 second --- .../streaming/examples/StatefulNetworkWordCount.scala | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala index b662cb116281b..51c3c9f9b4b0c 100644 --- a/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala @@ -4,7 +4,7 @@ import spark.streaming._ import spark.streaming.StreamingContext._ /** - * Counts words in UTF8 encoded, '\n' delimited text received from the network every second. + * Counts words cumulatively in UTF8 encoded, '\n' delimited text received from the network every second. * Usage: StatefulNetworkWordCount * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. * and describe the TCP server that Spark Streaming would connect to receive data. @@ -15,8 +15,6 @@ import spark.streaming.StreamingContext._ * `$ ./run spark.streaming.examples.StatefulNetworkWordCount local[2] localhost 9999` */ object StatefulNetworkWordCount { - private def className[A](a: A)(implicit m: Manifest[A]) = m.toString - def main(args: Array[String]) { if (args.length < 3) { System.err.println("Usage: StatefulNetworkWordCount \n" + @@ -32,8 +30,8 @@ object StatefulNetworkWordCount { Some(currentCount + previousCount) } - // Create the context with a 10 second batch size - val ssc = new StreamingContext(args(0), "NetworkWordCumulativeCountUpdateStateByKey", Seconds(10), + // Create the context with a 1 second batch size + val ssc = new StreamingContext(args(0), "NetworkWordCumulativeCountUpdateStateByKey", Seconds(1), System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) ssc.checkpoint(".") From d761e7359deb7ca864d33b8f2e4380b57448630b Mon Sep 17 00:00:00 2001 From: seanm Date: Fri, 10 May 2013 12:05:10 -0600 Subject: [PATCH 17/94] adding kafkaStream API tests --- .../src/test/java/spark/streaming/JavaAPISuite.java | 4 ++-- .../scala/spark/streaming/InputStreamsSuite.scala | 11 +++++++++++ 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java index 61e4c0a207cef..350d0888a3510 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java @@ -4,6 +4,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.io.Files; +import kafka.serializer.StringDecoder; import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; import org.junit.After; import org.junit.Assert; @@ -1203,8 +1204,7 @@ public Integer call(String s) throws Exception { public void testKafkaStream() { HashMap topics = Maps.newHashMap(); JavaDStream test1 = ssc.kafkaStream("localhost:12345", "group", topics); - JavaDStream test2 = ssc.kafkaStream("localhost:12345", "group", topics); - JavaDStream test3 = ssc.kafkaStream("localhost:12345", "group", topics, + JavaDStream test2 = ssc.kafkaStream("localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK()); } diff --git a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala index 1024d3ac9790e..595c766a216f0 100644 --- a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala @@ -240,6 +240,17 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { assert(output(i) === expectedOutput(i)) } } + + test("kafka input stream") { + val ssc = new StreamingContext(master, framework, batchDuration) + val topics = Map("my-topic" -> 1) + val test1 = ssc.kafkaStream("localhost:12345", "group", topics) + val test2 = ssc.kafkaStream("localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK) + + // Test specifying decoder + val kafkaParams = Map("zk.connect"->"localhost:12345","groupid"->"consumer-group") + val test3 = ssc.kafkaStream[String, kafka.serializer.StringDecoder](kafkaParams, topics, StorageLevel.MEMORY_AND_DISK) + } } From b95c1bdbbaeea86152e24b394a03bbbad95989d5 Mon Sep 17 00:00:00 2001 From: seanm Date: Fri, 10 May 2013 12:47:24 -0600 Subject: [PATCH 18/94] count() now uses a transform instead of ConstantInputDStream --- streaming/src/main/scala/spark/streaming/DStream.scala | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/spark/streaming/DStream.scala index e3a9247924f12..e125310861ebc 100644 --- a/streaming/src/main/scala/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/spark/streaming/DStream.scala @@ -441,10 +441,7 @@ abstract class DStream[T: ClassManifest] ( * Return a new DStream in which each RDD has a single element generated by counting each RDD * of this DStream. */ - def count(): DStream[Long] = { - val zero = new ConstantInputDStream(context, context.sparkContext.makeRDD(Seq((null, 0L)), 1)) - this.map(_ => (null, 1L)).union(zero).reduceByKey(_ + _).map(_._2) - } + def count(): DStream[Long] = this.map(_ => (null, 1L)).transform(_.union(context.sparkContext.makeRDD(Seq((null, 0L)), 1))).reduceByKey(_ + _).map(_._2) /** * Return a new DStream in which each RDD contains the counts of each distinct value in From 3632980b1b61dbb9ab9a3ab3d92fb415cb7173b9 Mon Sep 17 00:00:00 2001 From: seanm Date: Fri, 10 May 2013 15:54:26 -0600 Subject: [PATCH 19/94] fixing indentation --- .../scala/spark/streaming/api/java/JavaStreamingContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index 13427873ff530..4ad2bdf8a8296 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -105,7 +105,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param storageLevel RDD storage level. Defaults to memory-only */ def kafkaStream[T, D <: kafka.serializer.Decoder[_]: Manifest]( - kafkaParams: JMap[String, String], + kafkaParams: JMap[String, String], topics: JMap[String, JInt], storageLevel: StorageLevel) : JavaDStream[T] = { From f25282def5826fab6caabff28c82c57a7f3fdcb8 Mon Sep 17 00:00:00 2001 From: seanm Date: Fri, 10 May 2013 17:34:28 -0600 Subject: [PATCH 20/94] fixing kafkaStream Java API and adding test --- .../streaming/api/java/JavaStreamingContext.scala | 10 +++++++--- .../src/test/java/spark/streaming/JavaAPISuite.java | 6 ++++++ 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index 4ad2bdf8a8296..b35d9032f1d63 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -99,18 +99,22 @@ class JavaStreamingContext(val ssc: StreamingContext) { /** * Create an input stream that pulls messages form a Kafka Broker. + * @param typeClass Type of RDD + * @param decoderClass Type of kafka decoder * @param kafkaParams Map of kafka configuration paramaters. See: http://kafka.apache.org/configuration.html * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed * in its own thread. * @param storageLevel RDD storage level. Defaults to memory-only */ - def kafkaStream[T, D <: kafka.serializer.Decoder[_]: Manifest]( + def kafkaStream[T, D <: kafka.serializer.Decoder[_]]( + typeClass: Class[T], + decoderClass: Class[D], kafkaParams: JMap[String, String], topics: JMap[String, JInt], storageLevel: StorageLevel) : JavaDStream[T] = { - implicit val cmt: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cmt: ClassManifest[T] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cmd: Manifest[D] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[D]] ssc.kafkaStream[T, D]( kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java index 350d0888a3510..e5fdbe1b7af75 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java @@ -1206,6 +1206,12 @@ public void testKafkaStream() { JavaDStream test1 = ssc.kafkaStream("localhost:12345", "group", topics); JavaDStream test2 = ssc.kafkaStream("localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK()); + + HashMap kafkaParams = Maps.newHashMap(); + kafkaParams.put("zk.connect","localhost:12345"); + kafkaParams.put("groupid","consumer-group"); + JavaDStream test3 = ssc.kafkaStream(String.class, StringDecoder.class, kafkaParams, topics, + StorageLevel.MEMORY_AND_DISK()); } @Test From 91aca9224936da84b16ea789cb81914579a0db03 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Fri, 31 May 2013 23:21:38 -0700 Subject: [PATCH 21/94] Another round of Netty fixes. 1. Avoid race condition between stop and copier completion 2. Handle socket exceptions by reporting them and filling in a failed FetchResult --- .../java/spark/network/netty/FileClient.java | 24 ++----- .../network/netty/FileClientHandler.java | 8 +++ .../spark/network/netty/ShuffleCopier.scala | 62 ++++++++++++------- .../spark/storage/BlockFetcherIterator.scala | 9 +-- 4 files changed, 58 insertions(+), 45 deletions(-) diff --git a/core/src/main/java/spark/network/netty/FileClient.java b/core/src/main/java/spark/network/netty/FileClient.java index 3a62dacbc8e89..9c9b976ebe196 100644 --- a/core/src/main/java/spark/network/netty/FileClient.java +++ b/core/src/main/java/spark/network/netty/FileClient.java @@ -8,9 +8,12 @@ import io.netty.channel.oio.OioEventLoopGroup; import io.netty.channel.socket.oio.OioSocketChannel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; class FileClient { + private Logger LOG = LoggerFactory.getLogger(this.getClass().getName()); private FileClientHandler handler = null; private Channel channel = null; private Bootstrap bootstrap = null; @@ -25,25 +28,10 @@ public void init() { .channel(OioSocketChannel.class) .option(ChannelOption.SO_KEEPALIVE, true) .option(ChannelOption.TCP_NODELAY, true) + .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, 0) // Disable connect timeout .handler(new FileClientChannelInitializer(handler)); } - public static final class ChannelCloseListener implements ChannelFutureListener { - private FileClient fc = null; - - public ChannelCloseListener(FileClient fc){ - this.fc = fc; - } - - @Override - public void operationComplete(ChannelFuture future) { - if (fc.bootstrap!=null){ - fc.bootstrap.shutdown(); - fc.bootstrap = null; - } - } - } - public void connect(String host, int port) { try { // Start the connection attempt. @@ -58,8 +46,8 @@ public void connect(String host, int port) { public void waitForClose() { try { channel.closeFuture().sync(); - } catch (InterruptedException e){ - e.printStackTrace(); + } catch (InterruptedException e) { + LOG.warn("FileClient interrupted", e); } } diff --git a/core/src/main/java/spark/network/netty/FileClientHandler.java b/core/src/main/java/spark/network/netty/FileClientHandler.java index 2069dee5caa59..9fc9449827c87 100644 --- a/core/src/main/java/spark/network/netty/FileClientHandler.java +++ b/core/src/main/java/spark/network/netty/FileClientHandler.java @@ -9,7 +9,14 @@ abstract class FileClientHandler extends ChannelInboundByteHandlerAdapter { private FileHeader currentHeader = null; + private volatile boolean handlerCalled = false; + + public boolean isComplete() { + return handlerCalled; + } + public abstract void handle(ChannelHandlerContext ctx, ByteBuf in, FileHeader header); + public abstract void handleError(String blockId); @Override public ByteBuf newInboundBuffer(ChannelHandlerContext ctx) { @@ -26,6 +33,7 @@ public void inboundBufferUpdated(ChannelHandlerContext ctx, ByteBuf in) { // get file if(in.readableBytes() >= currentHeader.fileLen()) { handle(ctx, in, currentHeader); + handlerCalled = true; currentHeader = null; ctx.close(); } diff --git a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala index a91f5a886d732..8ec46d42fa6fa 100644 --- a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala +++ b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala @@ -9,19 +9,35 @@ import io.netty.util.CharsetUtil import spark.Logging import spark.network.ConnectionManagerId +import scala.collection.JavaConverters._ + private[spark] class ShuffleCopier extends Logging { - def getBlock(cmId: ConnectionManagerId, blockId: String, + def getBlock(host: String, port: Int, blockId: String, resultCollectCallback: (String, Long, ByteBuf) => Unit) { val handler = new ShuffleCopier.ShuffleClientHandler(resultCollectCallback) val fc = new FileClient(handler) - fc.init() - fc.connect(cmId.host, cmId.port) - fc.sendRequest(blockId) - fc.waitForClose() - fc.close() + try { + fc.init() + fc.connect(host, port) + fc.sendRequest(blockId) + fc.waitForClose() + fc.close() + } catch { + // Handle any socket-related exceptions in FileClient + case e: Exception => { + logError("Shuffle copy of block " + blockId + " from " + host + ":" + port + + " failed", e) + handler.handleError(blockId) + } + } + } + + def getBlock(cmId: ConnectionManagerId, blockId: String, + resultCollectCallback: (String, Long, ByteBuf) => Unit) { + getBlock(cmId.host, cmId.port, blockId, resultCollectCallback) } def getBlocks(cmId: ConnectionManagerId, @@ -44,20 +60,18 @@ private[spark] object ShuffleCopier extends Logging { logDebug("Received Block: " + header.blockId + " (" + header.fileLen + "B)"); resultCollectCallBack(header.blockId, header.fileLen.toLong, in.readBytes(header.fileLen)) } - } - def echoResultCollectCallBack(blockId: String, size: Long, content: ByteBuf) { - logInfo("File: " + blockId + " content is : \" " + content.toString(CharsetUtil.UTF_8) + "\"") + override def handleError(blockId: String) { + if (!isComplete) { + resultCollectCallBack(blockId, -1, null) + } + } } - def runGetBlock(host:String, port:Int, file:String){ - val handler = new ShuffleClientHandler(echoResultCollectCallBack) - val fc = new FileClient(handler) - fc.init(); - fc.connect(host, port) - fc.sendRequest(file) - fc.waitForClose(); - fc.close() + def echoResultCollectCallBack(blockId: String, size: Long, content: ByteBuf) { + if (size != -1) { + logInfo("File: " + blockId + " content is : \" " + content.toString(CharsetUtil.UTF_8) + "\"") + } } def main(args: Array[String]) { @@ -71,14 +85,16 @@ private[spark] object ShuffleCopier extends Logging { val threads = if (args.length > 3) args(3).toInt else 10 val copiers = Executors.newFixedThreadPool(80) - for (i <- Range(0, threads)) { - val runnable = new Runnable() { + val tasks = (for (i <- Range(0, threads)) yield { + Executors.callable(new Runnable() { def run() { - runGetBlock(host, port, file) + val copier = new ShuffleCopier() + copier.getBlock(host, port, file, echoResultCollectCallBack) } - } - copiers.execute(runnable) - } + }) + }).asJava + copiers.invokeAll(tasks) copiers.shutdown + System.exit(0) } } diff --git a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala index 1d69d658f7d8a..fac416a5b3d5d 100644 --- a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala @@ -265,7 +265,7 @@ object BlockFetcherIterator { }).toList } - //keep this to interrupt the threads when necessary + // keep this to interrupt the threads when necessary private def stopCopiers() { for (copier <- copiers) { copier.interrupt() @@ -312,9 +312,10 @@ object BlockFetcherIterator { resultsGotten += 1 val result = results.take() // if all the results has been retrieved, shutdown the copiers - if (resultsGotten == _totalBlocks && copiers != null) { - stopCopiers() - } + // NO need to stop the copiers if we got all the blocks ? + // if (resultsGotten == _totalBlocks && copiers != null) { + // stopCopiers() + // } (result.blockId, if (result.failed) None else Some(result.deserialize())) } } From 038cfc1a9acb32f8c17d883ea64f8cbb324ed82c Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Fri, 31 May 2013 23:32:18 -0700 Subject: [PATCH 22/94] Make connect timeout configurable --- core/src/main/java/spark/network/netty/FileClient.java | 6 ++++-- core/src/main/scala/spark/network/netty/ShuffleCopier.scala | 3 ++- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/spark/network/netty/FileClient.java b/core/src/main/java/spark/network/netty/FileClient.java index 9c9b976ebe196..517772202fe7a 100644 --- a/core/src/main/java/spark/network/netty/FileClient.java +++ b/core/src/main/java/spark/network/netty/FileClient.java @@ -17,9 +17,11 @@ class FileClient { private FileClientHandler handler = null; private Channel channel = null; private Bootstrap bootstrap = null; + private int connectTimeout = 60*1000; // 1 min - public FileClient(FileClientHandler handler) { + public FileClient(FileClientHandler handler, int connectTimeout) { this.handler = handler; + this.connectTimeout = connectTimeout; } public void init() { @@ -28,7 +30,7 @@ public void init() { .channel(OioSocketChannel.class) .option(ChannelOption.SO_KEEPALIVE, true) .option(ChannelOption.TCP_NODELAY, true) - .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, 0) // Disable connect timeout + .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, connectTimeout) // Disable connect timeout .handler(new FileClientChannelInitializer(handler)); } diff --git a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala index 8ec46d42fa6fa..afb2cdbb3a2d9 100644 --- a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala +++ b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala @@ -18,7 +18,8 @@ private[spark] class ShuffleCopier extends Logging { resultCollectCallback: (String, Long, ByteBuf) => Unit) { val handler = new ShuffleCopier.ShuffleClientHandler(resultCollectCallback) - val fc = new FileClient(handler) + val fc = new FileClient(handler, + System.getProperty("spark.shuffle.netty.connect.timeout", "60000").toInt) try { fc.init() fc.connect(host, port) From a058b0acf3e5ae41e64640feeace3d4e32f47401 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 3 Jun 2013 12:10:00 -0700 Subject: [PATCH 23/94] Delete a file for a block if it already exists. --- core/src/main/scala/spark/storage/DiskStore.scala | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index c7281200e7e00..2be5d01e312b9 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -195,9 +195,15 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) } private def createFile(blockId: String, allowAppendExisting: Boolean = false): File = { - val file = getFile(blockId) + var file = getFile(blockId) if (!allowAppendExisting && file.exists()) { - throw new Exception("File for block " + blockId + " already exists on disk: " + file) + // NOTE(shivaram): Delete the file if it exists. This might happen if a ShuffleMap task + // was rescheduled on the same machine as the old task ? + logWarning("File for block " + blockId + " already exists on disk: " + file + ". Deleting") + file.delete() + // Reopen the file + file = getFile(blockId) + // throw new Exception("File for block " + blockId + " already exists on disk: " + file) } file } From cd347f547a9a9b7bdd0d3f4734ae5c13be54f75d Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 3 Jun 2013 12:27:51 -0700 Subject: [PATCH 24/94] Reuse the file object as it is valid after delete --- core/src/main/scala/spark/storage/DiskStore.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index 2be5d01e312b9..e51d258a21e52 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -201,8 +201,6 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) // was rescheduled on the same machine as the old task ? logWarning("File for block " + blockId + " already exists on disk: " + file + ". Deleting") file.delete() - // Reopen the file - file = getFile(blockId) // throw new Exception("File for block " + blockId + " already exists on disk: " + file) } file From 96943a1cc054d7cf80eb8d3dfc7fb19ce48d3c0a Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 3 Jun 2013 12:29:38 -0700 Subject: [PATCH 25/94] var to val --- core/src/main/scala/spark/storage/DiskStore.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index e51d258a21e52..cd85fa1e9d29d 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -195,7 +195,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) } private def createFile(blockId: String, allowAppendExisting: Boolean = false): File = { - var file = getFile(blockId) + val file = getFile(blockId) if (!allowAppendExisting && file.exists()) { // NOTE(shivaram): Delete the file if it exists. This might happen if a ShuffleMap task // was rescheduled on the same machine as the old task ? From c851957fe4798d5dfb8deba7bf79a035a0543c74 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 5 Jun 2013 14:28:38 -0700 Subject: [PATCH 26/94] Don't write zero block files with java serializer --- .../spark/storage/BlockFetcherIterator.scala | 5 +- .../main/scala/spark/storage/DiskStore.scala | 46 ++++++++++++------- .../spark/storage/ShuffleBlockManager.scala | 2 +- core/src/test/scala/spark/ShuffleSuite.scala | 26 +++++++++++ 4 files changed, 61 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala index fac416a5b3d5d..843069239c21b 100644 --- a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala @@ -71,6 +71,7 @@ object BlockFetcherIterator { logDebug("Getting " + _totalBlocks + " blocks") protected var startTime = System.currentTimeMillis protected val localBlockIds = new ArrayBuffer[String]() + protected val localNonZeroBlocks = new ArrayBuffer[String]() protected val remoteBlockIds = new HashSet[String]() // A queue to hold our results. @@ -129,6 +130,8 @@ object BlockFetcherIterator { for ((address, blockInfos) <- blocksByAddress) { if (address == blockManagerId) { localBlockIds ++= blockInfos.map(_._1) + localNonZeroBlocks ++= blockInfos.filter(_._2 != 0).map(_._1) + _totalBlocks -= (localBlockIds.size - localNonZeroBlocks.size) } else { remoteBlockIds ++= blockInfos.map(_._1) // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them @@ -172,7 +175,7 @@ object BlockFetcherIterator { // Get the local blocks while remote blocks are being fetched. Note that it's okay to do // these all at once because they will just memory-map some files, so they won't consume // any memory that might exceed our maxBytesInFlight - for (id <- localBlockIds) { + for (id <- localNonZeroBlocks) { getLocalFromDisk(id, serializer) match { case Some(iter) => { // Pass 0 as size since it's not in flight diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index cd85fa1e9d29d..c1cff25552660 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -35,21 +35,25 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) private var bs: OutputStream = null private var objOut: SerializationStream = null private var lastValidPosition = 0L + private var initialized = false override def open(): DiskBlockObjectWriter = { val fos = new FileOutputStream(f, true) channel = fos.getChannel() bs = blockManager.wrapForCompression(blockId, new FastBufferedOutputStream(fos)) objOut = serializer.newInstance().serializeStream(bs) + initialized = true this } override def close() { - objOut.close() - bs.close() - channel = null - bs = null - objOut = null + if (initialized) { + objOut.close() + bs.close() + channel = null + bs = null + objOut = null + } // Invoke the close callback handler. super.close() } @@ -59,23 +63,33 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) // Flush the partial writes, and set valid length to be the length of the entire file. // Return the number of bytes written for this commit. override def commit(): Long = { - // NOTE: Flush the serializer first and then the compressed/buffered output stream - objOut.flush() - bs.flush() - val prevPos = lastValidPosition - lastValidPosition = channel.position() - lastValidPosition - prevPos + if (initialized) { + // NOTE: Flush the serializer first and then the compressed/buffered output stream + objOut.flush() + bs.flush() + val prevPos = lastValidPosition + lastValidPosition = channel.position() + lastValidPosition - prevPos + } else { + // lastValidPosition is zero if stream is uninitialized + lastValidPosition + } } override def revertPartialWrites() { - // Discard current writes. We do this by flushing the outstanding writes and - // truncate the file to the last valid position. - objOut.flush() - bs.flush() - channel.truncate(lastValidPosition) + if (initialized) { + // Discard current writes. We do this by flushing the outstanding writes and + // truncate the file to the last valid position. + objOut.flush() + bs.flush() + channel.truncate(lastValidPosition) + } } override def write(value: Any) { + if (!initialized) { + open() + } objOut.writeObject(value) } diff --git a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala index 49eabfb0d2150..44638e0c2d8f9 100644 --- a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala @@ -24,7 +24,7 @@ class ShuffleBlockManager(blockManager: BlockManager) { val bufferSize = System.getProperty("spark.shuffle.file.buffer.kb", "100").toInt * 1024 val writers = Array.tabulate[BlockObjectWriter](numBuckets) { bucketId => val blockId = ShuffleBlockManager.blockId(shuffleId, bucketId, mapId) - blockManager.getDiskBlockWriter(blockId, serializer, bufferSize).open() + blockManager.getDiskBlockWriter(blockId, serializer, bufferSize) } new ShuffleWriterGroup(mapId, writers) } diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index b967016cf7267..33b02fff80144 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -367,6 +367,32 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { assert(nonEmptyBlocks.size <= 4) } + test("zero sized blocks without kryo") { + // Use a local cluster with 2 processes to make sure there are both local and remote blocks + sc = new SparkContext("local-cluster[2,1,512]", "test") + + // 10 partitions from 4 keys + val NUM_BLOCKS = 10 + val a = sc.parallelize(1 to 4, NUM_BLOCKS) + val b = a.map(x => (x, x*2)) + + // NOTE: The default Java serializer doesn't create zero-sized blocks. + // So, use Kryo + val c = new ShuffledRDD(b, new HashPartitioner(10)) + + val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[Int, Int]].shuffleId + assert(c.count === 4) + + val blockSizes = (0 until NUM_BLOCKS).flatMap { id => + val statuses = SparkEnv.get.mapOutputTracker.getServerStatuses(shuffleId, id) + statuses.map(x => x._2) + } + val nonEmptyBlocks = blockSizes.filter(x => x > 0) + + // We should have at most 4 non-zero sized partitions + assert(nonEmptyBlocks.size <= 4) + } + } object ShuffleSuite { From cb2f5046ee99582a5038a78478c23468b14c134e Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 5 Jun 2013 15:09:02 -0700 Subject: [PATCH 27/94] Pass in bufferSize to BufferedOutputStream --- core/src/main/scala/spark/storage/DiskStore.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index c1cff25552660..0af6e4a35917f 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -40,7 +40,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) override def open(): DiskBlockObjectWriter = { val fos = new FileOutputStream(f, true) channel = fos.getChannel() - bs = blockManager.wrapForCompression(blockId, new FastBufferedOutputStream(fos)) + bs = blockManager.wrapForCompression(blockId, new FastBufferedOutputStream(fos, bufferSize)) objOut = serializer.newInstance().serializeStream(bs) initialized = true this From ac480fd977e0de97bcfe646e39feadbd239c1c29 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 6 Jun 2013 16:34:27 -0700 Subject: [PATCH 28/94] Clean up variables and counters in BlockFetcherIterator --- .../spark/storage/BlockFetcherIterator.scala | 54 +++++++++++-------- 1 file changed, 31 insertions(+), 23 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala index 843069239c21b..bb78207c9f3c2 100644 --- a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala @@ -67,12 +67,20 @@ object BlockFetcherIterator { throw new IllegalArgumentException("BlocksByAddress is null") } - protected var _totalBlocks = blocksByAddress.map(_._2.size).sum - logDebug("Getting " + _totalBlocks + " blocks") + // Total number blocks fetched (local + remote). Also number of FetchResults expected + protected var _numBlocksToFetch = 0 + protected var startTime = System.currentTimeMillis - protected val localBlockIds = new ArrayBuffer[String]() - protected val localNonZeroBlocks = new ArrayBuffer[String]() - protected val remoteBlockIds = new HashSet[String]() + + // This represents the number of local blocks, also counting zero-sized blocks + private var numLocal = 0 + // BlockIds for local blocks that need to be fetched. Excludes zero-sized blocks + protected val localBlocksToFetch = new ArrayBuffer[String]() + + // This represents the number of remote blocks, also counting zero-sized blocks + private var numRemote = 0 + // BlockIds for remote blocks that need to be fetched. Excludes zero-sized blocks + protected val remoteBlocksToFetch = new HashSet[String]() // A queue to hold our results. protected val results = new LinkedBlockingQueue[FetchResult] @@ -125,15 +133,15 @@ object BlockFetcherIterator { protected def splitLocalRemoteBlocks(): ArrayBuffer[FetchRequest] = { // Split local and remote blocks. Remote blocks are further split into FetchRequests of size // at most maxBytesInFlight in order to limit the amount of data in flight. - val originalTotalBlocks = _totalBlocks val remoteRequests = new ArrayBuffer[FetchRequest] for ((address, blockInfos) <- blocksByAddress) { if (address == blockManagerId) { - localBlockIds ++= blockInfos.map(_._1) - localNonZeroBlocks ++= blockInfos.filter(_._2 != 0).map(_._1) - _totalBlocks -= (localBlockIds.size - localNonZeroBlocks.size) + numLocal = blockInfos.size + // Filter out zero-sized blocks + localBlocksToFetch ++= blockInfos.filter(_._2 != 0).map(_._1) + _numBlocksToFetch += localBlocksToFetch.size } else { - remoteBlockIds ++= blockInfos.map(_._1) + numRemote += blockInfos.size // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 // nodes, rather than blocking on reading output from one node. @@ -147,10 +155,10 @@ object BlockFetcherIterator { // Skip empty blocks if (size > 0) { curBlocks += ((blockId, size)) + remoteBlocksToFetch += blockId + _numBlocksToFetch += 1 curRequestSize += size - } else if (size == 0) { - _totalBlocks -= 1 - } else { + } else if (size < 0) { throw new BlockException(blockId, "Negative block size " + size) } if (curRequestSize >= minRequestSize) { @@ -166,8 +174,8 @@ object BlockFetcherIterator { } } } - logInfo("Getting " + _totalBlocks + " non-zero-bytes blocks out of " + - originalTotalBlocks + " blocks") + logInfo("Getting " + _numBlocksToFetch + " non-zero-bytes blocks out of " + + totalBlocks + " blocks") remoteRequests } @@ -175,7 +183,7 @@ object BlockFetcherIterator { // Get the local blocks while remote blocks are being fetched. Note that it's okay to do // these all at once because they will just memory-map some files, so they won't consume // any memory that might exceed our maxBytesInFlight - for (id <- localNonZeroBlocks) { + for (id <- localBlocksToFetch) { getLocalFromDisk(id, serializer) match { case Some(iter) => { // Pass 0 as size since it's not in flight @@ -201,7 +209,7 @@ object BlockFetcherIterator { sendRequest(fetchRequests.dequeue()) } - val numGets = remoteBlockIds.size - fetchRequests.size + val numGets = remoteRequests.size - fetchRequests.size logInfo("Started " + numGets + " remote gets in " + Utils.getUsedTimeMs(startTime)) // Get Local Blocks @@ -213,7 +221,7 @@ object BlockFetcherIterator { //an iterator that will read fetched blocks off the queue as they arrive. @volatile protected var resultsGotten = 0 - override def hasNext: Boolean = resultsGotten < _totalBlocks + override def hasNext: Boolean = resultsGotten < _numBlocksToFetch override def next(): (String, Option[Iterator[Any]]) = { resultsGotten += 1 @@ -230,9 +238,9 @@ object BlockFetcherIterator { } // Implementing BlockFetchTracker trait. - override def totalBlocks: Int = _totalBlocks - override def numLocalBlocks: Int = localBlockIds.size - override def numRemoteBlocks: Int = remoteBlockIds.size + override def totalBlocks: Int = numLocal + numRemote + override def numLocalBlocks: Int = numLocal + override def numRemoteBlocks: Int = numRemote override def remoteFetchTime: Long = _remoteFetchTime override def fetchWaitTime: Long = _fetchWaitTime override def remoteBytesRead: Long = _remoteBytesRead @@ -294,7 +302,7 @@ object BlockFetcherIterator { private var copiers: List[_ <: Thread] = null override def initialize() { - // Split Local Remote Blocks and adjust totalBlocks to include only the non 0-byte blocks + // Split Local Remote Blocks and set numBlocksToFetch val remoteRequests = splitLocalRemoteBlocks() // Add the remote requests into our queue in a random order for (request <- Utils.randomize(remoteRequests)) { @@ -316,7 +324,7 @@ object BlockFetcherIterator { val result = results.take() // if all the results has been retrieved, shutdown the copiers // NO need to stop the copiers if we got all the blocks ? - // if (resultsGotten == _totalBlocks && copiers != null) { + // if (resultsGotten == _numBlocksToFetch && copiers != null) { // stopCopiers() // } (result.blockId, if (result.failed) None else Some(result.deserialize())) From 1d9f0df0652f455145d2dfed43a9407df6de6c43 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 13 Jun 2013 14:46:25 -0700 Subject: [PATCH 29/94] Fix some comments and style --- core/src/main/java/spark/network/netty/FileClient.java | 2 +- .../main/scala/spark/network/netty/ShuffleCopier.scala | 8 ++++---- .../main/scala/spark/storage/BlockFetcherIterator.scala | 6 +----- core/src/main/scala/spark/storage/DiskStore.scala | 3 +-- core/src/test/scala/spark/ShuffleSuite.scala | 3 +-- 5 files changed, 8 insertions(+), 14 deletions(-) diff --git a/core/src/main/java/spark/network/netty/FileClient.java b/core/src/main/java/spark/network/netty/FileClient.java index 517772202fe7a..a4bb4bc701af9 100644 --- a/core/src/main/java/spark/network/netty/FileClient.java +++ b/core/src/main/java/spark/network/netty/FileClient.java @@ -30,7 +30,7 @@ public void init() { .channel(OioSocketChannel.class) .option(ChannelOption.SO_KEEPALIVE, true) .option(ChannelOption.TCP_NODELAY, true) - .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, connectTimeout) // Disable connect timeout + .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, connectTimeout) .handler(new FileClientChannelInitializer(handler)); } diff --git a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala index afb2cdbb3a2d9..8d5194a7379eb 100644 --- a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala +++ b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala @@ -18,8 +18,9 @@ private[spark] class ShuffleCopier extends Logging { resultCollectCallback: (String, Long, ByteBuf) => Unit) { val handler = new ShuffleCopier.ShuffleClientHandler(resultCollectCallback) - val fc = new FileClient(handler, - System.getProperty("spark.shuffle.netty.connect.timeout", "60000").toInt) + val connectTimeout = System.getProperty("spark.shuffle.netty.connect.timeout", "60000").toInt + val fc = new FileClient(handler, connectTimeout) + try { fc.init() fc.connect(host, port) @@ -29,8 +30,7 @@ private[spark] class ShuffleCopier extends Logging { } catch { // Handle any socket-related exceptions in FileClient case e: Exception => { - logError("Shuffle copy of block " + blockId + " from " + host + ":" + port + - " failed", e) + logError("Shuffle copy of block " + blockId + " from " + host + ":" + port + " failed", e) handler.handleError(blockId) } } diff --git a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala index bb78207c9f3c2..bec876213e9f4 100644 --- a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala @@ -322,11 +322,7 @@ object BlockFetcherIterator { override def next(): (String, Option[Iterator[Any]]) = { resultsGotten += 1 val result = results.take() - // if all the results has been retrieved, shutdown the copiers - // NO need to stop the copiers if we got all the blocks ? - // if (resultsGotten == _numBlocksToFetch && copiers != null) { - // stopCopiers() - // } + // If all the results has been retrieved, copiers will exit automatically (result.blockId, if (result.failed) None else Some(result.deserialize())) } } diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index 0af6e4a35917f..15ab8401554bf 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -212,10 +212,9 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) val file = getFile(blockId) if (!allowAppendExisting && file.exists()) { // NOTE(shivaram): Delete the file if it exists. This might happen if a ShuffleMap task - // was rescheduled on the same machine as the old task ? + // was rescheduled on the same machine as the old task. logWarning("File for block " + blockId + " already exists on disk: " + file + ". Deleting") file.delete() - // throw new Exception("File for block " + blockId + " already exists on disk: " + file) } file } diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index 33b02fff80144..1916885a73837 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -376,8 +376,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { val a = sc.parallelize(1 to 4, NUM_BLOCKS) val b = a.map(x => (x, x*2)) - // NOTE: The default Java serializer doesn't create zero-sized blocks. - // So, use Kryo + // NOTE: The default Java serializer should create zero-sized blocks val c = new ShuffledRDD(b, new HashPartitioner(10)) val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[Int, Int]].shuffleId From 93a1643405d7c1a1fffe8210130341f34d64ea72 Mon Sep 17 00:00:00 2001 From: James Phillpotts Date: Fri, 21 Jun 2013 14:21:52 +0100 Subject: [PATCH 30/94] Allow other twitter authorizations than username/password --- .../scala/spark/streaming/StreamingContext.scala | 15 ++++++++++++++- .../streaming/dstream/TwitterInputDStream.scala | 14 ++++++-------- 2 files changed, 20 insertions(+), 9 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index b8b60aab43303..f97e47ada0d5e 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -27,6 +27,7 @@ import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.hadoop.mapreduce.lib.input.TextInputFormat import org.apache.hadoop.fs.Path import twitter4j.Status +import twitter4j.auth.{Authorization, BasicAuthorization} /** * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic @@ -372,8 +373,20 @@ class StreamingContext private ( password: String, filters: Seq[String] = Nil, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + ): DStream[Status] = twitterStream(new BasicAuthorization(username, password), filters, storageLevel) + + /** + * Create a input stream that returns tweets received from Twitter. + * @param twitterAuth Twitter4J authentication + * @param filters Set of filter strings to get only those tweets that match them + * @param storageLevel Storage level to use for storing the received objects + */ + def twitterStream( + twitterAuth: Authorization, + filters: Seq[String] = Nil, + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 ): DStream[Status] = { - val inputStream = new TwitterInputDStream(this, username, password, filters, storageLevel) + val inputStream = new TwitterInputDStream(this, twitterAuth, filters, storageLevel) registerInputStream(inputStream) inputStream } diff --git a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala index c69749886289c..0b01091a520b8 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala @@ -6,6 +6,7 @@ import storage.StorageLevel import twitter4j._ import twitter4j.auth.BasicAuthorization +import twitter4j.auth.Authorization /* A stream of Twitter statuses, potentially filtered by one or more keywords. * @@ -16,21 +17,19 @@ import twitter4j.auth.BasicAuthorization private[streaming] class TwitterInputDStream( @transient ssc_ : StreamingContext, - username: String, - password: String, + twitterAuth: Authorization, filters: Seq[String], storageLevel: StorageLevel ) extends NetworkInputDStream[Status](ssc_) { - + override def getReceiver(): NetworkReceiver[Status] = { - new TwitterReceiver(username, password, filters, storageLevel) + new TwitterReceiver(twitterAuth, filters, storageLevel) } } private[streaming] class TwitterReceiver( - username: String, - password: String, + twitterAuth: Authorization, filters: Seq[String], storageLevel: StorageLevel ) extends NetworkReceiver[Status] { @@ -40,8 +39,7 @@ class TwitterReceiver( protected override def onStart() { blockGenerator.start() - twitterStream = new TwitterStreamFactory() - .getInstance(new BasicAuthorization(username, password)) + twitterStream = new TwitterStreamFactory().getInstance(twitterAuth) twitterStream.addListener(new StatusListener { def onStatus(status: Status) = { blockGenerator += status From b5df1cd668e45fd0cc22c1666136d05548cae3e9 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 22 Jun 2013 17:12:39 -0700 Subject: [PATCH 31/94] ADD_JARS environment variable for spark-shell --- docs/scala-programming-guide.md | 10 ++++++++-- repl/src/main/scala/spark/repl/SparkILoop.scala | 9 +++++++-- 2 files changed, 15 insertions(+), 4 deletions(-) diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index b0da130fcb320..e9cf9ef36fcc4 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -43,12 +43,18 @@ new SparkContext(master, appName, [sparkHome], [jars]) The `master` parameter is a string specifying a [Spark or Mesos cluster URL](#master-urls) to connect to, or a special "local" string to run in local mode, as described below. `appName` is a name for your application, which will be shown in the cluster web UI. Finally, the last two parameters are needed to deploy your code to a cluster if running in distributed mode, as described later. -In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `MASTER` environment variable. For example, to run on four cores, use +In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `MASTER` environment variable, and you can add JARs to the classpath with the `ADD_JARS` variable. For example, to run `spark-shell` on four cores, use {% highlight bash %} $ MASTER=local[4] ./spark-shell {% endhighlight %} +Or, to also add `code.jar` to its classpath, use: + +{% highlight bash %} +$ MASTER=local[4] ADD_JARS=code.jar ./spark-shell +{% endhighlight %} + ### Master URLs The master URL passed to Spark can be in one of the following formats: @@ -78,7 +84,7 @@ If you want to run your job on a cluster, you will need to specify the two optio * `sparkHome`: The path at which Spark is installed on your worker machines (it should be the same on all of them). * `jars`: A list of JAR files on the local machine containing your job's code and any dependencies, which Spark will deploy to all the worker nodes. You'll need to package your job into a set of JARs using your build system. For example, if you're using SBT, the [sbt-assembly](https://github.com/sbt/sbt-assembly) plugin is a good way to make a single JAR with your code and dependencies. -If you run `spark-shell` on a cluster, any classes you define in the shell will automatically be distributed. +If you run `spark-shell` on a cluster, you can add JARs to it by specifying the `ADD_JARS` environment variable before you launch it. This variable should contain a comma-separated list of JARs. For example, `ADD_JARS=a.jar,b.jar ./spark-shell` will launch a shell with `a.jar` and `b.jar` on its classpath. In addition, any new classes you define in the shell will automatically be distributed. # Resilient Distributed Datasets (RDDs) diff --git a/repl/src/main/scala/spark/repl/SparkILoop.scala b/repl/src/main/scala/spark/repl/SparkILoop.scala index 23556dbc8f3b9..86eed090d0e42 100644 --- a/repl/src/main/scala/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/spark/repl/SparkILoop.scala @@ -822,7 +822,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: spark.repl.Main.interp.out.println("Spark context available as sc."); spark.repl.Main.interp.out.flush(); """) - command("import spark.SparkContext._"); + command("import spark.SparkContext._") } echo("Type in expressions to have them evaluated.") echo("Type :help for more information.") @@ -838,7 +838,8 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: if (prop != null) prop else "local" } } - sparkContext = new SparkContext(master, "Spark shell") + val jars = Option(System.getenv("ADD_JARS")).map(_.split(',')).getOrElse(new Array[String](0)) + sparkContext = new SparkContext(master, "Spark shell", System.getenv("SPARK_HOME"), jars) sparkContext } @@ -850,6 +851,10 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: printWelcome() echo("Initializing interpreter...") + // Add JARS specified in Spark's ADD_JARS variable to classpath + val jars = Option(System.getenv("ADD_JARS")).map(_.split(',')).getOrElse(new Array[String](0)) + jars.foreach(settings.classpath.append(_)) + this.settings = settings createInterpreter() From 0e0f9d3069039f03bbf5eefe3b0637c89fddf0f1 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 22 Jun 2013 17:44:04 -0700 Subject: [PATCH 32/94] Fix search path for REPL class loader to really find added JARs --- .../main/scala/spark/executor/Executor.scala | 38 +++++++++++-------- .../main/scala/spark/repl/SparkILoop.scala | 4 +- 2 files changed, 25 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 8bebfafce4421..2bf55ea9a9085 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -42,7 +42,8 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert // Create our ClassLoader and set it on this thread private val urlClassLoader = createClassLoader() - Thread.currentThread.setContextClassLoader(urlClassLoader) + private val replClassLoader = addReplClassLoaderIfNeeded(urlClassLoader) + Thread.currentThread.setContextClassLoader(replClassLoader) // Make any thread terminations due to uncaught exceptions kill the entire // executor process to avoid surprising stalls. @@ -88,7 +89,7 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert override def run() { val startTime = System.currentTimeMillis() SparkEnv.set(env) - Thread.currentThread.setContextClassLoader(urlClassLoader) + Thread.currentThread.setContextClassLoader(replClassLoader) val ser = SparkEnv.get.closureSerializer.newInstance() logInfo("Running task ID " + taskId) context.statusUpdate(taskId, TaskState.RUNNING, EMPTY_BYTE_BUFFER) @@ -153,26 +154,31 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert val urls = currentJars.keySet.map { uri => new File(uri.split("/").last).toURI.toURL }.toArray - loader = new URLClassLoader(urls, loader) + new ExecutorURLClassLoader(urls, loader) + } - // If the REPL is in use, add another ClassLoader that will read - // new classes defined by the REPL as the user types code + /** + * If the REPL is in use, add another ClassLoader that will read + * new classes defined by the REPL as the user types code + */ + private def addReplClassLoaderIfNeeded(parent: ClassLoader): ClassLoader = { val classUri = System.getProperty("spark.repl.class.uri") if (classUri != null) { logInfo("Using REPL class URI: " + classUri) - loader = { - try { - val klass = Class.forName("spark.repl.ExecutorClassLoader") - .asInstanceOf[Class[_ <: ClassLoader]] - val constructor = klass.getConstructor(classOf[String], classOf[ClassLoader]) - constructor.newInstance(classUri, loader) - } catch { - case _: ClassNotFoundException => loader - } + try { + val klass = Class.forName("spark.repl.ExecutorClassLoader") + .asInstanceOf[Class[_ <: ClassLoader]] + val constructor = klass.getConstructor(classOf[String], classOf[ClassLoader]) + return constructor.newInstance(classUri, parent) + } catch { + case _: ClassNotFoundException => + logError("Could not find spark.repl.ExecutorClassLoader on classpath!") + System.exit(1) + null } + } else { + return parent } - - return new ExecutorURLClassLoader(Array(), loader) } /** diff --git a/repl/src/main/scala/spark/repl/SparkILoop.scala b/repl/src/main/scala/spark/repl/SparkILoop.scala index 86eed090d0e42..59f9d05683dd0 100644 --- a/repl/src/main/scala/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/spark/repl/SparkILoop.scala @@ -838,7 +838,9 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: if (prop != null) prop else "local" } } - val jars = Option(System.getenv("ADD_JARS")).map(_.split(',')).getOrElse(new Array[String](0)) + val jars = Option(System.getenv("ADD_JARS")).map(_.split(',')) + .getOrElse(new Array[String](0)) + .map(new java.io.File(_).getAbsolutePath) sparkContext = new SparkContext(master, "Spark shell", System.getenv("SPARK_HOME"), jars) sparkContext } From 78ffe164b33c6b11a2e511442605acd2f795a1b5 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 23 Jun 2013 10:07:16 -0700 Subject: [PATCH 33/94] Clone the zero value for each key in foldByKey The old version reused the object within each task, leading to overwriting of the object when a mutable type is used, which is expected to be common in fold. Conflicts: core/src/test/scala/spark/ShuffleSuite.scala --- .../main/scala/spark/PairRDDFunctions.scala | 15 ++++++++++--- core/src/test/scala/spark/ShuffleSuite.scala | 22 +++++++++++++++++++ 2 files changed, 34 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index fa4bbfc76f306..7630fe7803575 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -1,5 +1,6 @@ package spark +import java.nio.ByteBuffer import java.util.{Date, HashMap => JHashMap} import java.text.SimpleDateFormat @@ -64,8 +65,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( 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](createCombiner, mergeValue, mergeCombiners) if (self.partitioner == Some(partitioner)) { self.mapPartitions(aggregator.combineValuesByKey(_), true) } else if (mapSideCombine) { @@ -97,7 +97,16 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * list concatenation, 0 for addition, or 1 for multiplication.). */ def foldByKey(zeroValue: V, partitioner: Partitioner)(func: (V, V) => V): RDD[(K, V)] = { - combineByKey[V]({v: V => func(zeroValue, v)}, func, func, partitioner) + // Serialize the zero value to a byte array so that we can get a new clone of it on each key + val zeroBuffer = SparkEnv.get.closureSerializer.newInstance().serialize(zeroValue) + val zeroArray = new Array[Byte](zeroBuffer.limit) + zeroBuffer.get(zeroArray) + + // When deserializing, use a lazy val to create just one instance of the serializer per task + lazy val cachedSerializer = SparkEnv.get.closureSerializer.newInstance() + def createZero() = cachedSerializer.deserialize[V](ByteBuffer.wrap(zeroArray)) + + combineByKey[V]((v: V) => func(createZero(), v), func, func, partitioner) } /** diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index 1916885a73837..0c1ec29f96c94 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -392,6 +392,28 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { assert(nonEmptyBlocks.size <= 4) } + test("foldByKey") { + sc = new SparkContext("local", "test") + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val sums = pairs.foldByKey(0)(_+_).collect() + assert(sums.toSet === Set((1, 7), (2, 1))) + } + + test("foldByKey with mutable result type") { + sc = new SparkContext("local", "test") + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val bufs = pairs.mapValues(v => ArrayBuffer(v)).cache() + // Fold the values using in-place mutation + val sums = bufs.foldByKey(new ArrayBuffer[Int])(_ ++= _).collect() + assert(sums.toSet === Set((1, ArrayBuffer(1, 2, 3, 1)), (2, ArrayBuffer(1)))) + // Check that the mutable objects in the original RDD were not changed + assert(bufs.collect().toSet === Set( + (1, ArrayBuffer(1)), + (1, ArrayBuffer(2)), + (1, ArrayBuffer(3)), + (1, ArrayBuffer(1)), + (2, ArrayBuffer(1)))) + } } object ShuffleSuite { From 8955787a596216a35ad4ec52b57331aa40444bef Mon Sep 17 00:00:00 2001 From: James Phillpotts Date: Mon, 24 Jun 2013 09:15:17 +0100 Subject: [PATCH 34/94] Twitter API v1 is retired - username/password auth no longer possible --- .../scala/spark/streaming/StreamingContext.scala | 16 +--------------- 1 file changed, 1 insertion(+), 15 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index f97e47ada0d5e..05be6bd58abd6 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -27,7 +27,7 @@ import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.hadoop.mapreduce.lib.input.TextInputFormat import org.apache.hadoop.fs.Path import twitter4j.Status -import twitter4j.auth.{Authorization, BasicAuthorization} +import twitter4j.auth.Authorization /** * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic @@ -361,20 +361,6 @@ class StreamingContext private ( fileStream[LongWritable, Text, TextInputFormat](directory).map(_._2.toString) } - /** - * Create a input stream that returns tweets received from Twitter. - * @param username Twitter username - * @param password Twitter password - * @param filters Set of filter strings to get only those tweets that match them - * @param storageLevel Storage level to use for storing the received objects - */ - def twitterStream( - username: String, - password: String, - filters: Seq[String] = Nil, - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[Status] = twitterStream(new BasicAuthorization(username, password), filters, storageLevel) - /** * Create a input stream that returns tweets received from Twitter. * @param twitterAuth Twitter4J authentication From 48c7e373c62b2e8cf48157ceb0d92c38c3a40652 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 24 Jun 2013 23:11:04 -0700 Subject: [PATCH 35/94] Minor formatting fixes --- .../main/scala/spark/streaming/DStream.scala | 9 ++++-- .../spark/streaming/StreamingContext.scala | 29 +++++++++++-------- .../api/java/JavaStreamingContext.scala | 15 ++++++---- 3 files changed, 34 insertions(+), 19 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/spark/streaming/DStream.scala index e125310861ebc..9be7926a4a800 100644 --- a/streaming/src/main/scala/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/spark/streaming/DStream.scala @@ -441,7 +441,12 @@ abstract class DStream[T: ClassManifest] ( * Return a new DStream in which each RDD has a single element generated by counting each RDD * of this DStream. */ - def count(): DStream[Long] = this.map(_ => (null, 1L)).transform(_.union(context.sparkContext.makeRDD(Seq((null, 0L)), 1))).reduceByKey(_ + _).map(_._2) + def count(): DStream[Long] = { + this.map(_ => (null, 1L)) + .transform(_.union(context.sparkContext.makeRDD(Seq((null, 0L)), 1))) + .reduceByKey(_ + _) + .map(_._2) + } /** * Return a new DStream in which each RDD contains the counts of each distinct value in @@ -457,7 +462,7 @@ abstract class DStream[T: ClassManifest] ( * this DStream will be registered as an output stream and therefore materialized. */ def foreach(foreachFunc: RDD[T] => Unit) { - foreach((r: RDD[T], t: Time) => foreachFunc(r)) + this.foreach((r: RDD[T], t: Time) => foreachFunc(r)) } /** diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index 2c6326943dc99..03d2907323c21 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -171,10 +171,11 @@ class StreamingContext private ( * should be same. */ def actorStream[T: ClassManifest]( - props: Props, - name: String, - storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2, - supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy): DStream[T] = { + props: Props, + name: String, + storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2, + supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy + ): DStream[T] = { networkStream(new ActorReceiver[T](props, name, storageLevel, supervisorStrategy)) } @@ -182,9 +183,10 @@ class StreamingContext private ( * Create an input stream that receives messages pushed by a zeromq publisher. * @param publisherUrl Url of remote zeromq publisher * @param subscribe topic to subscribe to - * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence - * of byte thus it needs the converter(which might be deserializer of bytes) - * to translate from sequence of sequence of bytes, where sequence refer to a frame + * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic + * and each frame has sequence of byte thus it needs the converter + * (which might be deserializer of bytes) to translate from sequence + * of sequence of bytes, where sequence refer to a frame * and sub sequence refer to its payload. * @param storageLevel RDD storage level. Defaults to memory-only. */ @@ -204,7 +206,7 @@ class StreamingContext private ( * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..). * @param groupId The group id for this consumer. * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread. + * in its own thread. * @param storageLevel Storage level to use for storing the received objects * (default: StorageLevel.MEMORY_AND_DISK_SER_2) */ @@ -214,15 +216,17 @@ class StreamingContext private ( topics: Map[String, Int], storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2 ): DStream[String] = { - val kafkaParams = Map[String, String]("zk.connect" -> zkQuorum, "groupid" -> groupId, "zk.connectiontimeout.ms" -> "10000"); + val kafkaParams = Map[String, String]( + "zk.connect" -> zkQuorum, "groupid" -> groupId, "zk.connectiontimeout.ms" -> "10000") kafkaStream[String, kafka.serializer.StringDecoder](kafkaParams, topics, storageLevel) } /** * Create an input stream that pulls messages from a Kafka Broker. - * @param kafkaParams Map of kafka configuration paramaters. See: http://kafka.apache.org/configuration.html + * @param kafkaParams Map of kafka configuration paramaters. + * See: http://kafka.apache.org/configuration.html * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread. + * in its own thread. * @param storageLevel Storage level to use for storing the received objects */ def kafkaStream[T: ClassManifest, D <: kafka.serializer.Decoder[_]: Manifest]( @@ -395,7 +399,8 @@ class StreamingContext private ( * it will process either one or all of the RDDs returned by the queue. * @param queue Queue of RDDs * @param oneAtATime Whether only one RDD should be consumed from the queue in every interval - * @param defaultRDD Default RDD is returned by the DStream when the queue is empty. Set as null if no RDD should be returned when empty + * @param defaultRDD Default RDD is returned by the DStream when the queue is empty. + * Set as null if no RDD should be returned when empty * @tparam T Type of objects in the RDD */ def queueStream[T: ClassManifest]( diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index b35d9032f1d63..fd5e06b50f9fd 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -75,7 +75,8 @@ class JavaStreamingContext(val ssc: StreamingContext) { : JavaDStream[String] = { implicit val cmt: ClassManifest[String] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[String]] - ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), StorageLevel.MEMORY_ONLY_SER_2) + ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), + StorageLevel.MEMORY_ONLY_SER_2) } /** @@ -83,8 +84,9 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..). * @param groupId The group id for this consumer. * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * in its own thread. * @param storageLevel RDD storage level. Defaults to memory-only - * in its own thread. + * */ def kafkaStream( zkQuorum: String, @@ -94,14 +96,16 @@ class JavaStreamingContext(val ssc: StreamingContext) { : JavaDStream[String] = { implicit val cmt: ClassManifest[String] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[String]] - ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) + ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), + storageLevel) } /** * Create an input stream that pulls messages form a Kafka Broker. * @param typeClass Type of RDD * @param decoderClass Type of kafka decoder - * @param kafkaParams Map of kafka configuration paramaters. See: http://kafka.apache.org/configuration.html + * @param kafkaParams Map of kafka configuration paramaters. + * See: http://kafka.apache.org/configuration.html * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed * in its own thread. * @param storageLevel RDD storage level. Defaults to memory-only @@ -113,7 +117,8 @@ class JavaStreamingContext(val ssc: StreamingContext) { topics: JMap[String, JInt], storageLevel: StorageLevel) : JavaDStream[T] = { - implicit val cmt: ClassManifest[T] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cmt: ClassManifest[T] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] implicit val cmd: Manifest[D] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[D]] ssc.kafkaStream[T, D]( kafkaParams.toMap, From 7680ce0bd65fc44716c5bc03d5909a3ddbd43501 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 25 Jun 2013 16:11:44 -0400 Subject: [PATCH 36/94] Fixed deprecated use of expect in SizeEstimatorSuite --- .../test/scala/spark/SizeEstimatorSuite.scala | 72 +++++++++---------- 1 file changed, 36 insertions(+), 36 deletions(-) diff --git a/core/src/test/scala/spark/SizeEstimatorSuite.scala b/core/src/test/scala/spark/SizeEstimatorSuite.scala index e235ef2f67a12..b5c8525f914d7 100644 --- a/core/src/test/scala/spark/SizeEstimatorSuite.scala +++ b/core/src/test/scala/spark/SizeEstimatorSuite.scala @@ -35,7 +35,7 @@ class SizeEstimatorSuite var oldOops: String = _ override def beforeAll() { - // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case + // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case oldArch = System.setProperty("os.arch", "amd64") oldOops = System.setProperty("spark.test.useCompressedOops", "true") } @@ -46,54 +46,54 @@ class SizeEstimatorSuite } test("simple classes") { - expect(16)(SizeEstimator.estimate(new DummyClass1)) - expect(16)(SizeEstimator.estimate(new DummyClass2)) - expect(24)(SizeEstimator.estimate(new DummyClass3)) - expect(24)(SizeEstimator.estimate(new DummyClass4(null))) - expect(48)(SizeEstimator.estimate(new DummyClass4(new DummyClass3))) + assert(SizeEstimator.estimate(new DummyClass1) === 16) + assert(SizeEstimator.estimate(new DummyClass2) === 16) + assert(SizeEstimator.estimate(new DummyClass3) === 24) + assert(SizeEstimator.estimate(new DummyClass4(null)) === 24) + assert(SizeEstimator.estimate(new DummyClass4(new DummyClass3)) === 48) } // NOTE: The String class definition varies across JDK versions (1.6 vs. 1.7) and vendors // (Sun vs IBM). Use a DummyString class to make tests deterministic. test("strings") { - expect(40)(SizeEstimator.estimate(DummyString(""))) - expect(48)(SizeEstimator.estimate(DummyString("a"))) - expect(48)(SizeEstimator.estimate(DummyString("ab"))) - expect(56)(SizeEstimator.estimate(DummyString("abcdefgh"))) + assert(SizeEstimator.estimate(DummyString("")) === 40) + assert(SizeEstimator.estimate(DummyString("a")) === 48) + assert(SizeEstimator.estimate(DummyString("ab")) === 48) + assert(SizeEstimator.estimate(DummyString("abcdefgh")) === 56) } test("primitive arrays") { - expect(32)(SizeEstimator.estimate(new Array[Byte](10))) - expect(40)(SizeEstimator.estimate(new Array[Char](10))) - expect(40)(SizeEstimator.estimate(new Array[Short](10))) - expect(56)(SizeEstimator.estimate(new Array[Int](10))) - expect(96)(SizeEstimator.estimate(new Array[Long](10))) - expect(56)(SizeEstimator.estimate(new Array[Float](10))) - expect(96)(SizeEstimator.estimate(new Array[Double](10))) - expect(4016)(SizeEstimator.estimate(new Array[Int](1000))) - expect(8016)(SizeEstimator.estimate(new Array[Long](1000))) + assert(SizeEstimator.estimate(new Array[Byte](10)) === 32) + assert(SizeEstimator.estimate(new Array[Char](10)) === 40) + assert(SizeEstimator.estimate(new Array[Short](10)) === 40) + assert(SizeEstimator.estimate(new Array[Int](10)) === 56) + assert(SizeEstimator.estimate(new Array[Long](10)) === 96) + assert(SizeEstimator.estimate(new Array[Float](10)) === 56) + assert(SizeEstimator.estimate(new Array[Double](10)) === 96) + assert(SizeEstimator.estimate(new Array[Int](1000)) === 4016) + assert(SizeEstimator.estimate(new Array[Long](1000)) === 8016) } test("object arrays") { // Arrays containing nulls should just have one pointer per element - expect(56)(SizeEstimator.estimate(new Array[String](10))) - expect(56)(SizeEstimator.estimate(new Array[AnyRef](10))) + assert(SizeEstimator.estimate(new Array[String](10)) === 56) + assert(SizeEstimator.estimate(new Array[AnyRef](10)) === 56) // For object arrays with non-null elements, each object should take one pointer plus // however many bytes that class takes. (Note that Array.fill calls the code in its // second parameter separately for each object, so we get distinct objects.) - expect(216)(SizeEstimator.estimate(Array.fill(10)(new DummyClass1))) - expect(216)(SizeEstimator.estimate(Array.fill(10)(new DummyClass2))) - expect(296)(SizeEstimator.estimate(Array.fill(10)(new DummyClass3))) - expect(56)(SizeEstimator.estimate(Array(new DummyClass1, new DummyClass2))) + assert(SizeEstimator.estimate(Array.fill(10)(new DummyClass1)) === 216) + assert(SizeEstimator.estimate(Array.fill(10)(new DummyClass2)) === 216) + assert(SizeEstimator.estimate(Array.fill(10)(new DummyClass3)) === 296) + assert(SizeEstimator.estimate(Array(new DummyClass1, new DummyClass2)) === 56) // Past size 100, our samples 100 elements, but we should still get the right size. - expect(28016)(SizeEstimator.estimate(Array.fill(1000)(new DummyClass3))) + assert(SizeEstimator.estimate(Array.fill(1000)(new DummyClass3)) === 28016) // If an array contains the *same* element many times, we should only count it once. val d1 = new DummyClass1 - expect(72)(SizeEstimator.estimate(Array.fill(10)(d1))) // 10 pointers plus 8-byte object - expect(432)(SizeEstimator.estimate(Array.fill(100)(d1))) // 100 pointers plus 8-byte object + assert(SizeEstimator.estimate(Array.fill(10)(d1)) === 72) // 10 pointers plus 8-byte object + assert(SizeEstimator.estimate(Array.fill(100)(d1)) === 432) // 100 pointers plus 8-byte object // Same thing with huge array containing the same element many times. Note that this won't // return exactly 4032 because it can't tell that *all* the elements will equal the first @@ -111,10 +111,10 @@ class SizeEstimatorSuite val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() - expect(40)(SizeEstimator.estimate(DummyString(""))) - expect(48)(SizeEstimator.estimate(DummyString("a"))) - expect(48)(SizeEstimator.estimate(DummyString("ab"))) - expect(56)(SizeEstimator.estimate(DummyString("abcdefgh"))) + assert(SizeEstimator.estimate(DummyString("")) === 40) + assert(SizeEstimator.estimate(DummyString("a")) === 48) + assert(SizeEstimator.estimate(DummyString("ab")) === 48) + assert(SizeEstimator.estimate(DummyString("abcdefgh")) === 56) resetOrClear("os.arch", arch) } @@ -128,10 +128,10 @@ class SizeEstimatorSuite val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() - expect(56)(SizeEstimator.estimate(DummyString(""))) - expect(64)(SizeEstimator.estimate(DummyString("a"))) - expect(64)(SizeEstimator.estimate(DummyString("ab"))) - expect(72)(SizeEstimator.estimate(DummyString("abcdefgh"))) + assert(SizeEstimator.estimate(DummyString("")) === 56) + assert(SizeEstimator.estimate(DummyString("a")) === 64) + assert(SizeEstimator.estimate(DummyString("ab")) === 64) + assert(SizeEstimator.estimate(DummyString("abcdefgh")) === 72) resetOrClear("os.arch", arch) resetOrClear("spark.test.useCompressedOops", oops) From 15b00914c53f1f4f00a3313968f68a8f032e7cb7 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 25 Jun 2013 17:17:27 -0400 Subject: [PATCH 37/94] Some fixes to the launch-java-directly change: - Split SPARK_JAVA_OPTS into multiple command-line arguments if it contains spaces; this splitting follows quoting rules in bash - Add the Scala JARs to the classpath if they're not in the CLASSPATH variable because the ExecutorRunner is launched with "scala" (this can happen when using local-cluster URLs in spark-shell) --- core/src/main/scala/spark/Utils.scala | 65 ++++++++++++++++++- .../spark/deploy/worker/ExecutorRunner.scala | 51 ++++++++++----- core/src/test/scala/spark/UtilsSuite.scala | 53 +++++++++++---- 3 files changed, 138 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index f3621c6beee1f..bdc1494cc9da1 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -522,7 +522,7 @@ private object Utils extends Logging { execute(command, new File(".")) } - private[spark] class CallSiteInfo(val lastSparkMethod: String, val firstUserFile: String, + private[spark] class CallSiteInfo(val lastSparkMethod: String, val firstUserFile: String, val firstUserLine: Int, val firstUserClass: String) /** * When called inside a class in the spark package, returns the name of the user code class @@ -610,4 +610,67 @@ private object Utils extends Logging { } return false } + + def isSpace(c: Char): Boolean = { + " \t\r\n".indexOf(c) != -1 + } + + /** + * Split a string of potentially quoted arguments from the command line the way that a shell + * would do it to determine arguments to a command. For example, if the string is 'a "b c" d', + * then it would be parsed as three arguments: 'a', 'b c' and 'd'. + */ + def splitCommandString(s: String): Seq[String] = { + val buf = new ArrayBuffer[String] + var inWord = false + var inSingleQuote = false + var inDoubleQuote = false + var curWord = new StringBuilder + def endWord() { + buf += curWord.toString + curWord.clear() + } + var i = 0 + while (i < s.length) { + var nextChar = s.charAt(i) + if (inDoubleQuote) { + if (nextChar == '"') { + inDoubleQuote = false + } else if (nextChar == '\\') { + if (i < s.length - 1) { + // Append the next character directly, because only " and \ may be escaped in + // double quotes after the shell's own expansion + curWord.append(s.charAt(i + 1)) + i += 1 + } + } else { + curWord.append(nextChar) + } + } else if (inSingleQuote) { + if (nextChar == '\'') { + inSingleQuote = false + } else { + curWord.append(nextChar) + } + // Backslashes are not treated specially in single quotes + } else if (nextChar == '"') { + inWord = true + inDoubleQuote = true + } else if (nextChar == '\'') { + inWord = true + inSingleQuote = true + } else if (!isSpace(nextChar)) { + curWord.append(nextChar) + inWord = true + } else if (inWord && isSpace(nextChar)) { + endWord() + inWord = false + } + i += 1 + } + if (inWord || inDoubleQuote || inSingleQuote) { + endWord() + } + return buf + } } diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index 4d31657d9e318..db580e39abd35 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -40,7 +40,7 @@ private[spark] class ExecutorRunner( workerThread.start() // Shutdown hook that kills actors on shutdown. - shutdownHook = new Thread() { + shutdownHook = new Thread() { override def run() { if (process != null) { logInfo("Shutdown hook killing child process.") @@ -87,25 +87,43 @@ private[spark] class ExecutorRunner( Seq(runner) ++ buildJavaOpts() ++ Seq(command.mainClass) ++ command.arguments.map(substituteVariables) } - - /* - * Attention: this must always be aligned with the environment variables in the run scripts and the - * way the JAVA_OPTS are assembled there. + + /** + * Attention: this must always be aligned with the environment variables in the run scripts and + * the way the JAVA_OPTS are assembled there. */ def buildJavaOpts(): Seq[String] = { - val _javaLibPath = if (System.getenv("SPARK_LIBRARY_PATH") == null) { - "" + val libraryOpts = if (System.getenv("SPARK_LIBRARY_PATH") == null) { + Nil + } else { + List("-Djava.library.path=" + System.getenv("SPARK_LIBRARY_PATH")) + } + + val userOpts = if (System.getenv("SPARK_JAVA_OPTS") == null) { + Nil } else { - "-Djava.library.path=" + System.getenv("SPARK_LIBRARY_PATH") + Utils.splitCommandString(System.getenv("SPARK_JAVA_OPTS")) } - - Seq("-cp", - System.getenv("CLASSPATH"), - System.getenv("SPARK_JAVA_OPTS"), - _javaLibPath, - "-Xms" + memory.toString + "M", - "-Xmx" + memory.toString + "M") - .filter(_ != null) + + val memoryOpts = Seq("-Xms" + memory + "M", "-Xmx" + memory + "M") + + var classPath = System.getenv("CLASSPATH") + if (System.getenv("SPARK_LAUNCH_WITH_SCALA") == "1") { + // Add the Scala library JARs to the classpath; this is needed when the ExecutorRunner + // was launched with "scala" as the runner (e.g. in spark-shell in local-cluster mode) + // and the Scala libraries won't be in the CLASSPATH environment variable by defalt. + if (System.getenv("SCALA_LIBRARY_PATH") == null && System.getenv("SCALA_HOME") == null) { + logError("Cloud not launch executors: neither SCALA_LIBRARY_PATH nor SCALA_HOME are set") + System.exit(1) + } + val scalaLib = Option(System.getenv("SCALA_LIBRARY_PATH")).getOrElse( + System.getenv("SCALA_HOME") + "/lib") + classPath += ":" + scalaLib + "/scala-library.jar" + + ":" + scalaLib + "/scala-compiler.jar" + + ":" + scalaLib + "/jline.jar" + } + + Seq("-cp", classPath) ++ libraryOpts ++ userOpts ++ memoryOpts } /** Spawn a thread that will redirect a given stream to a file */ @@ -136,6 +154,7 @@ private[spark] class ExecutorRunner( // Launch the process val command = buildCommandSeq() + println("COMMAND: " + command.mkString(" ")) val builder = new ProcessBuilder(command: _*).directory(executorDir) val env = builder.environment() for ((key, value) <- appDesc.command.environment) { diff --git a/core/src/test/scala/spark/UtilsSuite.scala b/core/src/test/scala/spark/UtilsSuite.scala index ed4701574fd4d..4a113e16bf5a5 100644 --- a/core/src/test/scala/spark/UtilsSuite.scala +++ b/core/src/test/scala/spark/UtilsSuite.scala @@ -27,24 +27,49 @@ class UtilsSuite extends FunSuite { assert(os.toByteArray.toList.equals(bytes.toList)) } - test("memoryStringToMb"){ - assert(Utils.memoryStringToMb("1") == 0) - assert(Utils.memoryStringToMb("1048575") == 0) - assert(Utils.memoryStringToMb("3145728") == 3) + test("memoryStringToMb") { + assert(Utils.memoryStringToMb("1") === 0) + assert(Utils.memoryStringToMb("1048575") === 0) + assert(Utils.memoryStringToMb("3145728") === 3) - assert(Utils.memoryStringToMb("1024k") == 1) - assert(Utils.memoryStringToMb("5000k") == 4) - assert(Utils.memoryStringToMb("4024k") == Utils.memoryStringToMb("4024K")) + assert(Utils.memoryStringToMb("1024k") === 1) + assert(Utils.memoryStringToMb("5000k") === 4) + assert(Utils.memoryStringToMb("4024k") === Utils.memoryStringToMb("4024K")) - assert(Utils.memoryStringToMb("1024m") == 1024) - assert(Utils.memoryStringToMb("5000m") == 5000) - assert(Utils.memoryStringToMb("4024m") == Utils.memoryStringToMb("4024M")) + assert(Utils.memoryStringToMb("1024m") === 1024) + assert(Utils.memoryStringToMb("5000m") === 5000) + assert(Utils.memoryStringToMb("4024m") === Utils.memoryStringToMb("4024M")) - assert(Utils.memoryStringToMb("2g") == 2048) - assert(Utils.memoryStringToMb("3g") == Utils.memoryStringToMb("3G")) + assert(Utils.memoryStringToMb("2g") === 2048) + assert(Utils.memoryStringToMb("3g") === Utils.memoryStringToMb("3G")) - assert(Utils.memoryStringToMb("2t") == 2097152) - assert(Utils.memoryStringToMb("3t") == Utils.memoryStringToMb("3T")) + assert(Utils.memoryStringToMb("2t") === 2097152) + assert(Utils.memoryStringToMb("3t") === Utils.memoryStringToMb("3T")) + } + + test("splitCommandString") { + assert(Utils.splitCommandString("") === Seq()) + assert(Utils.splitCommandString("a") === Seq("a")) + assert(Utils.splitCommandString("aaa") === Seq("aaa")) + assert(Utils.splitCommandString("a b c") === Seq("a", "b", "c")) + assert(Utils.splitCommandString(" a b\t c ") === Seq("a", "b", "c")) + assert(Utils.splitCommandString("a 'b c'") === Seq("a", "b c")) + assert(Utils.splitCommandString("a 'b c' d") === Seq("a", "b c", "d")) + assert(Utils.splitCommandString("'b c'") === Seq("b c")) + assert(Utils.splitCommandString("a \"b c\"") === Seq("a", "b c")) + assert(Utils.splitCommandString("a \"b c\" d") === Seq("a", "b c", "d")) + assert(Utils.splitCommandString("\"b c\"") === Seq("b c")) + assert(Utils.splitCommandString("a 'b\" c' \"d' e\"") === Seq("a", "b\" c", "d' e")) + assert(Utils.splitCommandString("a\t'b\nc'\nd") === Seq("a", "b\nc", "d")) + assert(Utils.splitCommandString("a \"b\\\\c\"") === Seq("a", "b\\c")) + assert(Utils.splitCommandString("a \"b\\\"c\"") === Seq("a", "b\"c")) + assert(Utils.splitCommandString("a 'b\\\"c'") === Seq("a", "b\\\"c")) + assert(Utils.splitCommandString("'a'b") === Seq("ab")) + assert(Utils.splitCommandString("'a''b'") === Seq("ab")) + assert(Utils.splitCommandString("\"a\"b") === Seq("ab")) + assert(Utils.splitCommandString("\"a\"\"b\"") === Seq("ab")) + assert(Utils.splitCommandString("''") === Seq("")) + assert(Utils.splitCommandString("\"\"") === Seq("")) } } From 366572edcab87701fd795ca0142ac9829b312d36 Mon Sep 17 00:00:00 2001 From: James Phillpotts Date: Tue, 25 Jun 2013 22:59:34 +0100 Subject: [PATCH 38/94] Include a default OAuth implementation, and update examples and JavaStreamingContext --- .../examples/TwitterAlgebirdCMS.scala | 2 +- .../examples/TwitterAlgebirdHLL.scala | 2 +- .../examples/TwitterPopularTags.scala | 2 +- .../spark/streaming/StreamingContext.scala | 2 +- .../api/java/JavaStreamingContext.scala | 69 +++++++++++++------ .../dstream/TwitterInputDStream.scala | 32 ++++++++- 6 files changed, 81 insertions(+), 28 deletions(-) diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala index a9642100e3d48..548190309e9d2 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala @@ -45,7 +45,7 @@ object TwitterAlgebirdCMS { val ssc = new StreamingContext(master, "TwitterAlgebirdCMS", Seconds(10), System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) - val stream = ssc.twitterStream(username, password, filters, StorageLevel.MEMORY_ONLY_SER) + val stream = ssc.twitterStream(None, filters, StorageLevel.MEMORY_ONLY_SER) val users = stream.map(status => status.getUser.getId) diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala index f3288bfb8547d..5a86c6318d6fe 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala @@ -34,7 +34,7 @@ object TwitterAlgebirdHLL { val ssc = new StreamingContext(master, "TwitterAlgebirdHLL", Seconds(5), System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) - val stream = ssc.twitterStream(username, password, filters, StorageLevel.MEMORY_ONLY_SER) + val stream = ssc.twitterStream(None, filters, StorageLevel.MEMORY_ONLY_SER) val users = stream.map(status => status.getUser.getId) diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala index 9d4494c6f2759..076c3878c81a1 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala @@ -23,7 +23,7 @@ object TwitterPopularTags { val ssc = new StreamingContext(master, "TwitterPopularTags", Seconds(2), System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) - val stream = ssc.twitterStream(username, password, filters) + val stream = ssc.twitterStream(None, filters) val hashTags = stream.flatMap(status => status.getText.split(" ").filter(_.startsWith("#"))) diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index 05be6bd58abd6..0f36504c0d83b 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -368,7 +368,7 @@ class StreamingContext private ( * @param storageLevel Storage level to use for storing the received objects */ def twitterStream( - twitterAuth: Authorization, + twitterAuth: Option[Authorization] = None, filters: Seq[String] = Nil, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 ): DStream[Status] = { diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index 3d149a742cb3c..85390ef57e1d6 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -4,23 +4,18 @@ import spark.streaming._ import receivers.{ActorReceiver, ReceiverSupervisorStrategy} import spark.streaming.dstream._ import spark.storage.StorageLevel - import spark.api.java.function.{Function => JFunction, Function2 => JFunction2} import spark.api.java.{JavaSparkContext, JavaRDD} - import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} - import twitter4j.Status - import akka.actor.Props import akka.actor.SupervisorStrategy import akka.zeromq.Subscribe - import scala.collection.JavaConversions._ - import java.lang.{Long => JLong, Integer => JInt} import java.io.InputStream import java.util.{Map => JMap} +import twitter4j.auth.Authorization /** * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic @@ -315,46 +310,78 @@ class JavaStreamingContext(val ssc: StreamingContext) { /** * Create a input stream that returns tweets received from Twitter. - * @param username Twitter username - * @param password Twitter password + * @param twitterAuth Twitter4J Authorization * @param filters Set of filter strings to get only those tweets that match them * @param storageLevel Storage level to use for storing the received objects */ def twitterStream( - username: String, - password: String, + twitterAuth: Authorization, filters: Array[String], storageLevel: StorageLevel ): JavaDStream[Status] = { - ssc.twitterStream(username, password, filters, storageLevel) + ssc.twitterStream(Some(twitterAuth), filters, storageLevel) + } + + /** + * Create a input stream that returns tweets received from Twitter using + * java.util.Preferences to store OAuth token. OAuth key and secret should + * be provided using system properties twitter4j.oauth.consumerKey and + * twitter4j.oauth.consumerSecret + * @param filters Set of filter strings to get only those tweets that match them + * @param storageLevel Storage level to use for storing the received objects + */ + def twitterStream( + filters: Array[String], + storageLevel: StorageLevel + ): JavaDStream[Status] = { + ssc.twitterStream(None, filters, storageLevel) } /** * Create a input stream that returns tweets received from Twitter. - * @param username Twitter username - * @param password Twitter password + * @param twitterAuth Twitter4J Authorization * @param filters Set of filter strings to get only those tweets that match them */ def twitterStream( - username: String, - password: String, + twitterAuth: Authorization, filters: Array[String] ): JavaDStream[Status] = { - ssc.twitterStream(username, password, filters) + ssc.twitterStream(Some(twitterAuth), filters) + } + + /** + * Create a input stream that returns tweets received from Twitter using + * java.util.Preferences to store OAuth token. OAuth key and secret should + * be provided using system properties twitter4j.oauth.consumerKey and + * twitter4j.oauth.consumerSecret + * @param filters Set of filter strings to get only those tweets that match them + */ + def twitterStream( + filters: Array[String] + ): JavaDStream[Status] = { + ssc.twitterStream(None, filters) } /** * Create a input stream that returns tweets received from Twitter. - * @param username Twitter username - * @param password Twitter password + * @param twitterAuth Twitter4J Authorization */ def twitterStream( - username: String, - password: String + twitterAuth: Authorization ): JavaDStream[Status] = { - ssc.twitterStream(username, password) + ssc.twitterStream(Some(twitterAuth)) } + /** + * Create a input stream that returns tweets received from Twitter using + * java.util.Preferences to store OAuth token. OAuth key and secret should + * be provided using system properties twitter4j.oauth.consumerKey and + * twitter4j.oauth.consumerSecret + */ + def twitterStream(): JavaDStream[Status] = { + ssc.twitterStream() + } + /** * Create an input stream with any arbitrary user implemented actor receiver. * @param props Props object defining creation of the actor diff --git a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala index 0b01091a520b8..e0c654d385f1e 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala @@ -3,27 +3,53 @@ package spark.streaming.dstream import spark._ import spark.streaming._ import storage.StorageLevel - import twitter4j._ import twitter4j.auth.BasicAuthorization import twitter4j.auth.Authorization +import java.util.prefs.Preferences +import twitter4j.conf.PropertyConfiguration +import twitter4j.auth.OAuthAuthorization +import twitter4j.auth.AccessToken /* A stream of Twitter statuses, potentially filtered by one or more keywords. * * @constructor create a new Twitter stream using the supplied username and password to authenticate. * An optional set of string filters can be used to restrict the set of tweets. The Twitter API is * such that this may return a sampled subset of all tweets during each interval. +* +* Includes a simple implementation of OAuth using consumer key and secret provided using system +* properties twitter4j.oauth.consumerKey and twitter4j.oauth.consumerSecret */ private[streaming] class TwitterInputDStream( @transient ssc_ : StreamingContext, - twitterAuth: Authorization, + twitterAuth: Option[Authorization], filters: Seq[String], storageLevel: StorageLevel ) extends NetworkInputDStream[Status](ssc_) { + lazy val createOAuthAuthorization: Authorization = { + val userRoot = Preferences.userRoot(); + val token = Option(userRoot.get(PropertyConfiguration.OAUTH_ACCESS_TOKEN, null)) + val tokenSecret = Option(userRoot.get(PropertyConfiguration.OAUTH_ACCESS_TOKEN_SECRET, null)) + val oAuth = new OAuthAuthorization(new PropertyConfiguration(System.getProperties())) + if (token.isEmpty || tokenSecret.isEmpty) { + val requestToken = oAuth.getOAuthRequestToken() + println("Authorize application using URL: "+requestToken.getAuthorizationURL()) + println("Enter PIN: ") + val pin = Console.readLine + val accessToken = if (pin.length() > 0) oAuth.getOAuthAccessToken(requestToken, pin) else oAuth.getOAuthAccessToken() + userRoot.put(PropertyConfiguration.OAUTH_ACCESS_TOKEN, accessToken.getToken()) + userRoot.put(PropertyConfiguration.OAUTH_ACCESS_TOKEN_SECRET, accessToken.getTokenSecret()) + userRoot.flush() + } else { + oAuth.setOAuthAccessToken(new AccessToken(token.get, tokenSecret.get)); + } + oAuth + } + override def getReceiver(): NetworkReceiver[Status] = { - new TwitterReceiver(twitterAuth, filters, storageLevel) + new TwitterReceiver(if (twitterAuth.isEmpty) createOAuthAuthorization else twitterAuth.get, filters, storageLevel) } } From 176193b1e8acdbe2f1cfaed16b8f42f89e226f79 Mon Sep 17 00:00:00 2001 From: James Phillpotts Date: Tue, 25 Jun 2013 23:06:15 +0100 Subject: [PATCH 39/94] Fix usage and parameter extraction --- .../spark/streaming/examples/TwitterAlgebirdCMS.scala | 7 +++---- .../spark/streaming/examples/TwitterAlgebirdHLL.scala | 7 +++---- .../spark/streaming/examples/TwitterPopularTags.scala | 7 +++---- 3 files changed, 9 insertions(+), 12 deletions(-) diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala index 548190309e9d2..528778ed722e5 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala @@ -26,8 +26,8 @@ import spark.SparkContext._ */ object TwitterAlgebirdCMS { def main(args: Array[String]) { - if (args.length < 3) { - System.err.println("Usage: TwitterAlgebirdCMS " + + if (args.length < 1) { + System.err.println("Usage: TwitterAlgebirdCMS " + " [filter1] [filter2] ... [filter n]") System.exit(1) } @@ -40,8 +40,7 @@ object TwitterAlgebirdCMS { // K highest frequency elements to take val TOPK = 10 - val Array(master, username, password) = args.slice(0, 3) - val filters = args.slice(3, args.length) + val (master, filters) = (args.head, args.tail) val ssc = new StreamingContext(master, "TwitterAlgebirdCMS", Seconds(10), System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala index 5a86c6318d6fe..896e9fd8af1fb 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala @@ -21,16 +21,15 @@ import spark.streaming.dstream.TwitterInputDStream */ object TwitterAlgebirdHLL { def main(args: Array[String]) { - if (args.length < 3) { - System.err.println("Usage: TwitterAlgebirdHLL " + + if (args.length < 1) { + System.err.println("Usage: TwitterAlgebirdHLL " + " [filter1] [filter2] ... [filter n]") System.exit(1) } /** Bit size parameter for HyperLogLog, trades off accuracy vs size */ val BIT_SIZE = 12 - val Array(master, username, password) = args.slice(0, 3) - val filters = args.slice(3, args.length) + val (master, filters) = (args.head, args.tail) val ssc = new StreamingContext(master, "TwitterAlgebirdHLL", Seconds(5), System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala index 076c3878c81a1..65f0b6d3525a4 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala @@ -12,14 +12,13 @@ import spark.SparkContext._ */ object TwitterPopularTags { def main(args: Array[String]) { - if (args.length < 3) { - System.err.println("Usage: TwitterPopularTags " + + if (args.length < 1) { + System.err.println("Usage: TwitterPopularTags " + " [filter1] [filter2] ... [filter n]") System.exit(1) } - val Array(master, username, password) = args.slice(0, 3) - val filters = args.slice(3, args.length) + val (master, filters) = (args.head, args.tail) val ssc = new StreamingContext(master, "TwitterPopularTags", Seconds(2), System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) From 6c8d1b2ca618a1a17566ede46821c0807a1b11f5 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 25 Jun 2013 18:21:00 -0400 Subject: [PATCH 40/94] Fix computation of classpath when we launch java directly The previous version assumed that a CLASSPATH environment variable was set by the "run" script when launching the process that starts the ExecutorRunner, but unfortunately this is not true in tests. Instead, we factor the classpath calculation into an extenral script and call that. NOTE: This includes a Windows version but hasn't yet been tested there. --- bin/compute-classpath.cmd | 52 +++++++++++ bin/compute-classpath.sh | 89 +++++++++++++++++++ core/src/main/scala/spark/Utils.scala | 31 +++++++ .../spark/deploy/worker/ExecutorRunner.scala | 19 +--- run | 67 +++----------- run2.cmd | 38 +------- 6 files changed, 189 insertions(+), 107 deletions(-) create mode 100644 bin/compute-classpath.cmd create mode 100755 bin/compute-classpath.sh diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd new file mode 100644 index 0000000000000..1dff8fea220ed --- /dev/null +++ b/bin/compute-classpath.cmd @@ -0,0 +1,52 @@ +@echo off + +rem This script computes Spark's classpath and prints it to stdout; it's used by both the "run" +rem script and the ExecutorRunner in standalone cluster mode. + +set SCALA_VERSION=2.9.3 + +rem Figure out where the Spark framework is installed +set FWDIR=%~dp0\.. + +rem Load environment variables from conf\spark-env.cmd, if it exists +if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" + +set CORE_DIR=%FWDIR%core +set REPL_DIR=%FWDIR%repl +set EXAMPLES_DIR=%FWDIR%examples +set BAGEL_DIR=%FWDIR%bagel +set STREAMING_DIR=%FWDIR%streaming +set PYSPARK_DIR=%FWDIR%python + +rem Build up classpath +set CLASSPATH=%SPARK_CLASSPATH%;%MESOS_CLASSPATH%;%FWDIR%conf;%CORE_DIR%\target\scala-%SCALA_VERSION%\classes +set CLASSPATH=%CLASSPATH%;%CORE_DIR%\target\scala-%SCALA_VERSION%\test-classes;%CORE_DIR%\src\main\resources +set CLASSPATH=%CLASSPATH%;%STREAMING_DIR%\target\scala-%SCALA_VERSION%\classes;%STREAMING_DIR%\target\scala-%SCALA_VERSION%\test-classes +set CLASSPATH=%CLASSPATH%;%STREAMING_DIR%\lib\org\apache\kafka\kafka\0.7.2-spark\* +set CLASSPATH=%CLASSPATH%;%REPL_DIR%\target\scala-%SCALA_VERSION%\classes;%EXAMPLES_DIR%\target\scala-%SCALA_VERSION%\classes +set CLASSPATH=%CLASSPATH%;%FWDIR%lib_managed\jars\* +set CLASSPATH=%CLASSPATH%;%FWDIR%lib_managed\bundles\* +set CLASSPATH=%CLASSPATH%;%FWDIR%repl\lib\* +set CLASSPATH=%CLASSPATH%;%FWDIR%python\lib\* +set CLASSPATH=%CLASSPATH%;%BAGEL_DIR%\target\scala-%SCALA_VERSION%\classes + +rem Add hadoop conf dir - else FileSystem.*, etc fail +rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts +rem the configurtion files. +if "x%HADOOP_CONF_DIR%"=="x" goto no_hadoop_conf_dir + set CLASSPATH=%CLASSPATH%;%HADOOP_CONF_DIR% +:no_hadoop_conf_dir + +if "x%YARN_CONF_DIR%"=="x" goto no_yarn_conf_dir + set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR% +:no_yarn_conf_dir + +rem Add Scala standard library +set CLASSPATH=%CLASSPATH%;%SCALA_HOME%\lib\scala-library.jar;%SCALA_HOME%\lib\scala-compiler.jar;%SCALA_HOME%\lib\jline.jar + +rem A bit of a hack to allow calling this script within run2.cmd without seeing output +if "x%DONT_PRINT_CLASSPATH%"=="x1" goto exit + +echo %CLASSPATH% + +:exit diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh new file mode 100755 index 0000000000000..3a78880290455 --- /dev/null +++ b/bin/compute-classpath.sh @@ -0,0 +1,89 @@ +#!/bin/bash + +# This script computes Spark's classpath and prints it to stdout; it's used by both the "run" +# script and the ExecutorRunner in standalone cluster mode. + +SCALA_VERSION=2.9.3 + +# Figure out where Spark is installed +FWDIR="$(cd `dirname $0`/..; pwd)" + +# Load environment variables from conf/spark-env.sh, if it exists +if [ -e $FWDIR/conf/spark-env.sh ] ; then + . $FWDIR/conf/spark-env.sh +fi + +CORE_DIR="$FWDIR/core" +REPL_DIR="$FWDIR/repl" +REPL_BIN_DIR="$FWDIR/repl-bin" +EXAMPLES_DIR="$FWDIR/examples" +BAGEL_DIR="$FWDIR/bagel" +STREAMING_DIR="$FWDIR/streaming" +PYSPARK_DIR="$FWDIR/python" + +# Build up classpath +CLASSPATH="$SPARK_CLASSPATH" +CLASSPATH="$CLASSPATH:$FWDIR/conf" +CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/classes" +if [ -n "$SPARK_TESTING" ] ; then + CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/test-classes" + CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/test-classes" +fi +CLASSPATH="$CLASSPATH:$CORE_DIR/src/main/resources" +CLASSPATH="$CLASSPATH:$REPL_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH="$CLASSPATH:$EXAMPLES_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH="$CLASSPATH:$STREAMING_DIR/lib/org/apache/kafka/kafka/0.7.2-spark/*" # <-- our in-project Kafka Jar +if [ -e "$FWDIR/lib_managed" ]; then + CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/jars/*" + CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/bundles/*" +fi +CLASSPATH="$CLASSPATH:$REPL_DIR/lib/*" +# Add the shaded JAR for Maven builds +if [ -e $REPL_BIN_DIR/target ]; then + for jar in `find "$REPL_BIN_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do + CLASSPATH="$CLASSPATH:$jar" + done + # The shaded JAR doesn't contain examples, so include those separately + EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar` + CLASSPATH+=":$EXAMPLES_JAR" +fi +CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" +for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do + CLASSPATH="$CLASSPATH:$jar" +done + +# Figure out the JAR file that our examples were packaged into. This includes a bit of a hack +# to avoid the -sources and -doc packages that are built by publish-local. +if [ -e "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar ]; then + # Use the JAR from the SBT build + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar` +fi +if [ -e "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar ]; then + # Use the JAR from the Maven build + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar` +fi + +# Add hadoop conf dir - else FileSystem.*, etc fail ! +# Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts +# the configurtion files. +if [ "x" != "x$HADOOP_CONF_DIR" ]; then + CLASSPATH="$CLASSPATH:$HADOOP_CONF_DIR" +fi +if [ "x" != "x$YARN_CONF_DIR" ]; then + CLASSPATH="$CLASSPATH:$YARN_CONF_DIR" +fi + +# Add Scala standard library +if [ -z "$SCALA_LIBRARY_PATH" ]; then + if [ -z "$SCALA_HOME" ]; then + echo "SCALA_HOME is not set" >&2 + exit 1 + fi + SCALA_LIBRARY_PATH="$SCALA_HOME/lib" +fi +CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-library.jar" +CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-compiler.jar" +CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/jline.jar" + +echo "$CLASSPATH" diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index bdc1494cc9da1..f41efa9d29929 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -522,6 +522,37 @@ private object Utils extends Logging { execute(command, new File(".")) } + /** + * Execute a command and get its output, throwing an exception if it yields a code other than 0. + */ + def executeAndGetOutput(command: Seq[String], workingDir: File = new File(".")): String = { + val process = new ProcessBuilder(command: _*) + .directory(workingDir) + .start() + new Thread("read stderr for " + command(0)) { + override def run() { + for (line <- Source.fromInputStream(process.getErrorStream).getLines) { + System.err.println(line) + } + } + }.start() + val output = new StringBuffer + val stdoutThread = new Thread("read stdout for " + command(0)) { + override def run() { + for (line <- Source.fromInputStream(process.getInputStream).getLines) { + output.append(line) + } + } + } + stdoutThread.start() + val exitCode = process.waitFor() + stdoutThread.join() // Wait for it to finish reading output + if (exitCode != 0) { + throw new SparkException("Process " + command + " exited with code " + exitCode) + } + output.toString + } + private[spark] class CallSiteInfo(val lastSparkMethod: String, val firstUserFile: String, val firstUserLine: Int, val firstUserClass: String) /** diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index db580e39abd35..4f8e1dcb260a1 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -107,21 +107,9 @@ private[spark] class ExecutorRunner( val memoryOpts = Seq("-Xms" + memory + "M", "-Xmx" + memory + "M") - var classPath = System.getenv("CLASSPATH") - if (System.getenv("SPARK_LAUNCH_WITH_SCALA") == "1") { - // Add the Scala library JARs to the classpath; this is needed when the ExecutorRunner - // was launched with "scala" as the runner (e.g. in spark-shell in local-cluster mode) - // and the Scala libraries won't be in the CLASSPATH environment variable by defalt. - if (System.getenv("SCALA_LIBRARY_PATH") == null && System.getenv("SCALA_HOME") == null) { - logError("Cloud not launch executors: neither SCALA_LIBRARY_PATH nor SCALA_HOME are set") - System.exit(1) - } - val scalaLib = Option(System.getenv("SCALA_LIBRARY_PATH")).getOrElse( - System.getenv("SCALA_HOME") + "/lib") - classPath += ":" + scalaLib + "/scala-library.jar" + - ":" + scalaLib + "/scala-compiler.jar" + - ":" + scalaLib + "/jline.jar" - } + // Figure out our classpath with the external compute-classpath script + val ext = if (System.getProperty("os.name").startsWith("Windows")) ".cmd" else ".sh" + val classPath = Utils.executeAndGetOutput(Seq(sparkHome + "/bin/compute-classpath" + ext)) Seq("-cp", classPath) ++ libraryOpts ++ userOpts ++ memoryOpts } @@ -154,7 +142,6 @@ private[spark] class ExecutorRunner( // Launch the process val command = buildCommandSeq() - println("COMMAND: " + command.mkString(" ")) val builder = new ProcessBuilder(command: _*).directory(executorDir) val env = builder.environment() for ((key, value) <- appDesc.command.environment) { diff --git a/run b/run index 0fb15f8b24c89..7c06a55062212 100755 --- a/run +++ b/run @@ -49,6 +49,12 @@ case "$1" in ;; esac +# Figure out whether to run our class with java or with the scala launcher. +# In most cases, we'd prefer to execute our process with java because scala +# creates a shell script as the parent of its Java process, which makes it +# hard to kill the child with stuff like Process.destroy(). However, for +# the Spark shell, the wrapper is necessary to properly reset the terminal +# when we exit, so we allow it to set a variable to launch with scala. if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then if [ "$SCALA_HOME" ]; then RUNNER="${SCALA_HOME}/bin/scala" @@ -98,12 +104,8 @@ export JAVA_OPTS # Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala! CORE_DIR="$FWDIR/core" -REPL_DIR="$FWDIR/repl" -REPL_BIN_DIR="$FWDIR/repl-bin" EXAMPLES_DIR="$FWDIR/examples" -BAGEL_DIR="$FWDIR/bagel" -STREAMING_DIR="$FWDIR/streaming" -PYSPARK_DIR="$FWDIR/python" +REPL_DIR="$FWDIR/repl" # Exit if the user hasn't compiled Spark if [ ! -e "$CORE_DIR/target" ]; then @@ -118,37 +120,9 @@ if [[ "$@" = *repl* && ! -e "$REPL_DIR/target" ]]; then exit 1 fi -# Build up classpath -CLASSPATH="$SPARK_CLASSPATH" -CLASSPATH="$CLASSPATH:$FWDIR/conf" -CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/classes" -if [ -n "$SPARK_TESTING" ] ; then - CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/test-classes" -fi -CLASSPATH="$CLASSPATH:$CORE_DIR/src/main/resources" -CLASSPATH="$CLASSPATH:$REPL_DIR/target/scala-$SCALA_VERSION/classes" -CLASSPATH="$CLASSPATH:$EXAMPLES_DIR/target/scala-$SCALA_VERSION/classes" -CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/classes" -CLASSPATH="$CLASSPATH:$STREAMING_DIR/lib/org/apache/kafka/kafka/0.7.2-spark/*" # <-- our in-project Kafka Jar -if [ -e "$FWDIR/lib_managed" ]; then - CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/jars/*" - CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/bundles/*" -fi -CLASSPATH="$CLASSPATH:$REPL_DIR/lib/*" -# Add the shaded JAR for Maven builds -if [ -e $REPL_BIN_DIR/target ]; then - for jar in `find "$REPL_BIN_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do - CLASSPATH="$CLASSPATH:$jar" - done - # The shaded JAR doesn't contain examples, so include those separately - EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar` - CLASSPATH+=":$EXAMPLES_JAR" -fi -CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" -for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do - CLASSPATH="$CLASSPATH:$jar" -done +# Compute classpath using external script +CLASSPATH=`$FWDIR/bin/compute-classpath.sh` +export CLASSPATH # Figure out the JAR file that our examples were packaged into. This includes a bit of a hack # to avoid the -sources and -doc packages that are built by publish-local. @@ -161,32 +135,11 @@ if [ -e "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar ]; then export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar` fi -# Add hadoop conf dir - else FileSystem.*, etc fail ! -# Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts -# the configurtion files. -if [ "x" != "x$HADOOP_CONF_DIR" ]; then - CLASSPATH="$CLASSPATH:$HADOOP_CONF_DIR" -fi -if [ "x" != "x$YARN_CONF_DIR" ]; then - CLASSPATH="$CLASSPATH:$YARN_CONF_DIR" -fi - - -# Figure out whether to run our class with java or with the scala launcher. -# In most cases, we'd prefer to execute our process with java because scala -# creates a shell script as the parent of its Java process, which makes it -# hard to kill the child with stuff like Process.destroy(). However, for -# the Spark shell, the wrapper is necessary to properly reset the terminal -# when we exit, so we allow it to set a variable to launch with scala. if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then EXTRA_ARGS="" # Java options will be passed to scala as JAVA_OPTS else - CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-library.jar" - CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-compiler.jar" - CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/jline.jar" # The JVM doesn't read JAVA_OPTS by default so we need to pass it in EXTRA_ARGS="$JAVA_OPTS" fi -export CLASSPATH # Needed for spark-shell exec "$RUNNER" -cp "$CLASSPATH" $EXTRA_ARGS "$@" diff --git a/run2.cmd b/run2.cmd index bf76844d11336..25e4f3b57c1b5 100644 --- a/run2.cmd +++ b/run2.cmd @@ -33,51 +33,21 @@ if not "x%SCALA_HOME%"=="x" goto scala_exists goto exit :scala_exists -rem If the user specifies a Mesos JAR, put it before our included one on the classpath -set MESOS_CLASSPATH= -if not "x%MESOS_JAR%"=="x" set MESOS_CLASSPATH=%MESOS_JAR% - rem Figure out how much memory to use per executor and set it as an environment rem variable so that our process sees it and can report it to Mesos if "x%SPARK_MEM%"=="x" set SPARK_MEM=512m rem Set JAVA_OPTS to be able to load native libraries and to set heap size set JAVA_OPTS=%OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%SPARK_MEM% -Xmx%SPARK_MEM% -rem Load extra JAVA_OPTS from conf/java-opts, if it exists -if exist "%FWDIR%conf\java-opts.cmd" call "%FWDIR%conf\java-opts.cmd" rem Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala! set CORE_DIR=%FWDIR%core -set REPL_DIR=%FWDIR%repl set EXAMPLES_DIR=%FWDIR%examples -set BAGEL_DIR=%FWDIR%bagel -set STREAMING_DIR=%FWDIR%streaming -set PYSPARK_DIR=%FWDIR%python - -rem Build up classpath -set CLASSPATH=%SPARK_CLASSPATH%;%MESOS_CLASSPATH%;%FWDIR%conf;%CORE_DIR%\target\scala-%SCALA_VERSION%\classes -set CLASSPATH=%CLASSPATH%;%CORE_DIR%\target\scala-%SCALA_VERSION%\test-classes;%CORE_DIR%\src\main\resources -set CLASSPATH=%CLASSPATH%;%STREAMING_DIR%\target\scala-%SCALA_VERSION%\classes;%STREAMING_DIR%\target\scala-%SCALA_VERSION%\test-classes -set CLASSPATH=%CLASSPATH%;%STREAMING_DIR%\lib\org\apache\kafka\kafka\0.7.2-spark\* -set CLASSPATH=%CLASSPATH%;%REPL_DIR%\target\scala-%SCALA_VERSION%\classes;%EXAMPLES_DIR%\target\scala-%SCALA_VERSION%\classes -set CLASSPATH=%CLASSPATH%;%FWDIR%lib_managed\jars\* -set CLASSPATH=%CLASSPATH%;%FWDIR%lib_managed\bundles\* -set CLASSPATH=%CLASSPATH%;%FWDIR%repl\lib\* -set CLASSPATH=%CLASSPATH%;%FWDIR%python\lib\* -set CLASSPATH=%CLASSPATH%;%BAGEL_DIR%\target\scala-%SCALA_VERSION%\classes - -rem Add hadoop conf dir - else FileSystem.*, etc fail -rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts -rem the configurtion files. -if "x%HADOOP_CONF_DIR%"=="x" goto no_hadoop_conf_dir - set CLASSPATH=%CLASSPATH%;%HADOOP_CONF_DIR% -:no_hadoop_conf_dir - -if "x%YARN_CONF_DIR%"=="x" goto no_yarn_conf_dir - set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR% -:no_yarn_conf_dir - +set REPL_DIR=%FWDIR%repl +rem Compute classpath using external script +set DONT_PRINT_CLASSPATH=1 +call "%FWDIR%bin\compute-classpath.cmd" rem Figure out the JAR file that our examples were packaged into. rem First search in the build path from SBT: From f2263350eda780aba45f383b722e20702c310e6a Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 25 Jun 2013 18:35:35 -0400 Subject: [PATCH 41/94] Added a local-cluster mode test to ReplSuite --- .../src/test/scala/spark/repl/ReplSuite.scala | 31 ++++++++++++++++--- 1 file changed, 26 insertions(+), 5 deletions(-) diff --git a/repl/src/test/scala/spark/repl/ReplSuite.scala b/repl/src/test/scala/spark/repl/ReplSuite.scala index 1c64f9b98d099..72ed8aca5b6b7 100644 --- a/repl/src/test/scala/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/spark/repl/ReplSuite.scala @@ -35,17 +35,17 @@ class ReplSuite extends FunSuite { System.clearProperty("spark.hostPort") return out.toString } - + def assertContains(message: String, output: String) { assert(output contains message, "Interpreter output did not contain '" + message + "':\n" + output) } - + def assertDoesNotContain(message: String, output: String) { assert(!(output contains message), "Interpreter output contained '" + message + "':\n" + output) } - + test ("simple foreach with accumulator") { val output = runInterpreter("local", """ val accum = sc.accumulator(0) @@ -56,7 +56,7 @@ class ReplSuite extends FunSuite { assertDoesNotContain("Exception", output) assertContains("res1: Int = 55", output) } - + test ("external vars") { val output = runInterpreter("local", """ var v = 7 @@ -105,7 +105,7 @@ class ReplSuite extends FunSuite { assertContains("res0: Int = 70", output) assertContains("res1: Int = 100", output) } - + test ("broadcast vars") { // Test that the value that a broadcast var had when it was created is used, // even if that variable is then modified in the driver program @@ -143,6 +143,27 @@ class ReplSuite extends FunSuite { assertContains("res2: Long = 3", output) } + test ("local-cluster mode") { + val output = runInterpreter("local-cluster[1,1,512]", """ + var v = 7 + def getV() = v + sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + v = 10 + sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + var array = new Array[Int](5) + val broadcastArray = sc.broadcast(array) + sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + array(0) = 5 + sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + """) + assertDoesNotContain("error:", output) + assertDoesNotContain("Exception", output) + assertContains("res0: Int = 70", output) + assertContains("res1: Int = 100", output) + assertContains("res2: Array[Int] = Array(0, 0, 0, 0, 0)", output) + assertContains("res4: Array[Int] = Array(0, 0, 0, 0, 0)", output) + } + if (System.getenv("MESOS_NATIVE_LIBRARY") != null) { test ("running on Mesos") { val output = runInterpreter("localquiet", """ From 2bd04c3513ffa6deabc290a3931be946b1c18713 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 25 Jun 2013 18:37:14 -0400 Subject: [PATCH 42/94] Formatting --- repl/src/test/scala/spark/repl/ReplSuite.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/repl/src/test/scala/spark/repl/ReplSuite.scala b/repl/src/test/scala/spark/repl/ReplSuite.scala index 72ed8aca5b6b7..f46e6d8be4724 100644 --- a/repl/src/test/scala/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/spark/repl/ReplSuite.scala @@ -28,8 +28,9 @@ class ReplSuite extends FunSuite { val separator = System.getProperty("path.separator") interp.process(Array("-classpath", paths.mkString(separator))) spark.repl.Main.interp = null - if (interp.sparkContext != null) + if (interp.sparkContext != null) { interp.sparkContext.stop() + } // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown System.clearProperty("spark.driver.port") System.clearProperty("spark.hostPort") @@ -37,12 +38,12 @@ class ReplSuite extends FunSuite { } def assertContains(message: String, output: String) { - assert(output contains message, + assert(output.contains(message), "Interpreter output did not contain '" + message + "':\n" + output) } def assertDoesNotContain(message: String, output: String) { - assert(!(output contains message), + assert(!output.contains(message), "Interpreter output contained '" + message + "':\n" + output) } From 9f0d91329516c829c86fc8e95d02071ca7d1f186 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 25 Jun 2013 19:18:30 -0400 Subject: [PATCH 43/94] Refactored tests to share SparkContexts in some of them Creating these seems to take a while and clutters the output with Akka stuff, so it would be nice to share them. --- .../test/scala/spark/CheckpointSuite.scala | 10 + .../scala/spark/PairRDDFunctionsSuite.scala | 287 +++++++++++++++++ .../test/scala/spark/PartitioningSuite.scala | 19 +- core/src/test/scala/spark/PipedRDDSuite.scala | 16 +- core/src/test/scala/spark/RDDSuite.scala | 87 +---- .../test/scala/spark/SharedSparkContext.scala | 25 ++ core/src/test/scala/spark/ShuffleSuite.scala | 298 +----------------- core/src/test/scala/spark/SortingSuite.scala | 23 +- .../src/test/scala/spark/UnpersistSuite.scala | 30 ++ .../scala/spark/ZippedPartitionsSuite.scala | 3 +- 10 files changed, 373 insertions(+), 425 deletions(-) create mode 100644 core/src/test/scala/spark/PairRDDFunctionsSuite.scala create mode 100644 core/src/test/scala/spark/SharedSparkContext.scala create mode 100644 core/src/test/scala/spark/UnpersistSuite.scala diff --git a/core/src/test/scala/spark/CheckpointSuite.scala b/core/src/test/scala/spark/CheckpointSuite.scala index ca385972fb2eb..28a7b21b92bb5 100644 --- a/core/src/test/scala/spark/CheckpointSuite.scala +++ b/core/src/test/scala/spark/CheckpointSuite.scala @@ -27,6 +27,16 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { } } + test("basic checkpointing") { + val parCollection = sc.makeRDD(1 to 4) + val flatMappedRDD = parCollection.flatMap(x => 1 to x) + flatMappedRDD.checkpoint() + assert(flatMappedRDD.dependencies.head.rdd == parCollection) + val result = flatMappedRDD.collect() + assert(flatMappedRDD.dependencies.head.rdd != parCollection) + assert(flatMappedRDD.collect() === result) + } + test("RDDs with one-to-one dependencies") { testCheckpointing(_.map(x => x.toString)) testCheckpointing(_.flatMap(x => 1 to x)) diff --git a/core/src/test/scala/spark/PairRDDFunctionsSuite.scala b/core/src/test/scala/spark/PairRDDFunctionsSuite.scala new file mode 100644 index 0000000000000..682d2745bffc4 --- /dev/null +++ b/core/src/test/scala/spark/PairRDDFunctionsSuite.scala @@ -0,0 +1,287 @@ +package spark + +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashSet + +import org.scalatest.FunSuite +import org.scalatest.prop.Checkers +import org.scalacheck.Arbitrary._ +import org.scalacheck.Gen +import org.scalacheck.Prop._ + +import com.google.common.io.Files + +import spark.rdd.ShuffledRDD +import spark.SparkContext._ + +class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { + test("groupByKey") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))) + val groups = pairs.groupByKey().collect() + assert(groups.size === 2) + val valuesFor1 = groups.find(_._1 == 1).get._2 + assert(valuesFor1.toList.sorted === List(1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } + + test("groupByKey with duplicates") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val groups = pairs.groupByKey().collect() + assert(groups.size === 2) + val valuesFor1 = groups.find(_._1 == 1).get._2 + assert(valuesFor1.toList.sorted === List(1, 1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } + + test("groupByKey with negative key hash codes") { + val pairs = sc.parallelize(Array((-1, 1), (-1, 2), (-1, 3), (2, 1))) + val groups = pairs.groupByKey().collect() + assert(groups.size === 2) + val valuesForMinus1 = groups.find(_._1 == -1).get._2 + assert(valuesForMinus1.toList.sorted === List(1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } + + test("groupByKey with many output partitions") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))) + val groups = pairs.groupByKey(10).collect() + assert(groups.size === 2) + val valuesFor1 = groups.find(_._1 == 1).get._2 + assert(valuesFor1.toList.sorted === List(1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } + + test("reduceByKey") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val sums = pairs.reduceByKey(_+_).collect() + assert(sums.toSet === Set((1, 7), (2, 1))) + } + + test("reduceByKey with collectAsMap") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val sums = pairs.reduceByKey(_+_).collectAsMap() + assert(sums.size === 2) + assert(sums(1) === 7) + assert(sums(2) === 1) + } + + test("reduceByKey with many output partitons") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val sums = pairs.reduceByKey(_+_, 10).collect() + assert(sums.toSet === Set((1, 7), (2, 1))) + } + + test("reduceByKey with partitioner") { + val p = new Partitioner() { + def numPartitions = 2 + def getPartition(key: Any) = key.asInstanceOf[Int] + } + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 1), (0, 1))).partitionBy(p) + val sums = pairs.reduceByKey(_+_) + assert(sums.collect().toSet === Set((1, 4), (0, 1))) + assert(sums.partitioner === Some(p)) + // count the dependencies to make sure there is only 1 ShuffledRDD + val deps = new HashSet[RDD[_]]() + def visit(r: RDD[_]) { + for (dep <- r.dependencies) { + deps += dep.rdd + visit(dep.rdd) + } + } + visit(sums) + assert(deps.size === 2) // ShuffledRDD, ParallelCollection + } + + test("join") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (2, 'x')), + (2, (1, 'y')), + (2, (1, 'z')) + )) + } + + test("join all-to-all") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (1, 3))) + val rdd2 = sc.parallelize(Array((1, 'x'), (1, 'y'))) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 6) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (1, 'y')), + (1, (2, 'x')), + (1, (2, 'y')), + (1, (3, 'x')), + (1, (3, 'y')) + )) + } + + test("leftOuterJoin") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.leftOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (1, Some('x'))), + (1, (2, Some('x'))), + (2, (1, Some('y'))), + (2, (1, Some('z'))), + (3, (1, None)) + )) + } + + test("rightOuterJoin") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.rightOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (Some(1), 'x')), + (1, (Some(2), 'x')), + (2, (Some(1), 'y')), + (2, (Some(1), 'z')), + (4, (None, 'w')) + )) + } + + test("join with no matches") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((4, 'x'), (5, 'y'), (5, 'z'), (6, 'w'))) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 0) + } + + test("join with many output partitions") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.join(rdd2, 10).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (2, 'x')), + (2, (1, 'y')), + (2, (1, 'z')) + )) + } + + test("groupWith") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.groupWith(rdd2).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (ArrayBuffer(1, 2), ArrayBuffer('x'))), + (2, (ArrayBuffer(1), ArrayBuffer('y', 'z'))), + (3, (ArrayBuffer(1), ArrayBuffer())), + (4, (ArrayBuffer(), ArrayBuffer('w'))) + )) + } + + test("zero-partition RDD") { + val emptyDir = Files.createTempDir() + val file = sc.textFile(emptyDir.getAbsolutePath) + assert(file.partitions.size == 0) + assert(file.collect().toList === Nil) + // Test that a shuffle on the file works, because this used to be a bug + assert(file.map(line => (line, 1)).reduceByKey(_ + _).collect().toList === Nil) + } + + test("keys and values") { + val rdd = sc.parallelize(Array((1, "a"), (2, "b"))) + assert(rdd.keys.collect().toList === List(1, 2)) + assert(rdd.values.collect().toList === List("a", "b")) + } + + test("default partitioner uses partition size") { + // specify 2000 partitions + val a = sc.makeRDD(Array(1, 2, 3, 4), 2000) + // do a map, which loses the partitioner + val b = a.map(a => (a, (a * 2).toString)) + // then a group by, and see we didn't revert to 2 partitions + val c = b.groupByKey() + assert(c.partitions.size === 2000) + } + + test("default partitioner uses largest partitioner") { + val a = sc.makeRDD(Array((1, "a"), (2, "b")), 2) + val b = sc.makeRDD(Array((1, "a"), (2, "b")), 2000) + val c = a.join(b) + assert(c.partitions.size === 2000) + } + + test("subtract") { + val a = sc.parallelize(Array(1, 2, 3), 2) + val b = sc.parallelize(Array(2, 3, 4), 4) + val c = a.subtract(b) + assert(c.collect().toSet === Set(1)) + assert(c.partitions.size === a.partitions.size) + } + + test("subtract with narrow dependency") { + // use a deterministic partitioner + val p = new Partitioner() { + def numPartitions = 5 + def getPartition(key: Any) = key.asInstanceOf[Int] + } + // partitionBy so we have a narrow dependency + val a = sc.parallelize(Array((1, "a"), (2, "b"), (3, "c"))).partitionBy(p) + // more partitions/no partitioner so a shuffle dependency + val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) + val c = a.subtract(b) + assert(c.collect().toSet === Set((1, "a"), (3, "c"))) + // Ideally we could keep the original partitioner... + assert(c.partitioner === None) + } + + test("subtractByKey") { + val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 2) + val b = sc.parallelize(Array((2, 20), (3, 30), (4, 40)), 4) + val c = a.subtractByKey(b) + assert(c.collect().toSet === Set((1, "a"), (1, "a"))) + assert(c.partitions.size === a.partitions.size) + } + + test("subtractByKey with narrow dependency") { + // use a deterministic partitioner + val p = new Partitioner() { + def numPartitions = 5 + def getPartition(key: Any) = key.asInstanceOf[Int] + } + // partitionBy so we have a narrow dependency + val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c"))).partitionBy(p) + // more partitions/no partitioner so a shuffle dependency + val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) + val c = a.subtractByKey(b) + assert(c.collect().toSet === Set((1, "a"), (1, "a"))) + assert(c.partitioner.get === p) + } + + test("foldByKey") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val sums = pairs.foldByKey(0)(_+_).collect() + assert(sums.toSet === Set((1, 7), (2, 1))) + } + + test("foldByKey with mutable result type") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val bufs = pairs.mapValues(v => ArrayBuffer(v)).cache() + // Fold the values using in-place mutation + val sums = bufs.foldByKey(new ArrayBuffer[Int])(_ ++= _).collect() + assert(sums.toSet === Set((1, ArrayBuffer(1, 2, 3, 1)), (2, ArrayBuffer(1)))) + // Check that the mutable objects in the original RDD were not changed + assert(bufs.collect().toSet === Set( + (1, ArrayBuffer(1)), + (1, ArrayBuffer(2)), + (1, ArrayBuffer(3)), + (1, ArrayBuffer(1)), + (2, ArrayBuffer(1)))) + } +} diff --git a/core/src/test/scala/spark/PartitioningSuite.scala b/core/src/test/scala/spark/PartitioningSuite.scala index 16f93e71a3721..99e433e3bd61c 100644 --- a/core/src/test/scala/spark/PartitioningSuite.scala +++ b/core/src/test/scala/spark/PartitioningSuite.scala @@ -6,8 +6,8 @@ import SparkContext._ import spark.util.StatCounter import scala.math.abs -class PartitioningSuite extends FunSuite with LocalSparkContext { - +class PartitioningSuite extends FunSuite with SharedSparkContext { + test("HashPartitioner equality") { val p2 = new HashPartitioner(2) val p4 = new HashPartitioner(4) @@ -21,8 +21,6 @@ class PartitioningSuite extends FunSuite with LocalSparkContext { } test("RangePartitioner equality") { - sc = new SparkContext("local", "test") - // Make an RDD where all the elements are the same so that the partition range bounds // are deterministically all the same. val rdd = sc.parallelize(Seq(1, 1, 1, 1)).map(x => (x, x)) @@ -50,7 +48,6 @@ class PartitioningSuite extends FunSuite with LocalSparkContext { } test("HashPartitioner not equal to RangePartitioner") { - sc = new SparkContext("local", "test") val rdd = sc.parallelize(1 to 10).map(x => (x, x)) val rangeP2 = new RangePartitioner(2, rdd) val hashP2 = new HashPartitioner(2) @@ -61,8 +58,6 @@ class PartitioningSuite extends FunSuite with LocalSparkContext { } test("partitioner preservation") { - sc = new SparkContext("local", "test") - val rdd = sc.parallelize(1 to 10, 4).map(x => (x, x)) val grouped2 = rdd.groupByKey(2) @@ -101,7 +96,6 @@ class PartitioningSuite extends FunSuite with LocalSparkContext { } test("partitioning Java arrays should fail") { - sc = new SparkContext("local", "test") val arrs: RDD[Array[Int]] = sc.parallelize(Array(1, 2, 3, 4), 2).map(x => Array(x)) val arrPairs: RDD[(Array[Int], Int)] = sc.parallelize(Array(1, 2, 3, 4), 2).map(x => (Array(x), x)) @@ -120,21 +114,20 @@ class PartitioningSuite extends FunSuite with LocalSparkContext { assert(intercept[SparkException]{ arrPairs.reduceByKeyLocally(_ + _) }.getMessage.contains("array")) assert(intercept[SparkException]{ arrPairs.reduceByKey(_ + _) }.getMessage.contains("array")) } - - test("Zero-length partitions should be correctly handled") { + + test("zero-length partitions should be correctly handled") { // Create RDD with some consecutive empty partitions (including the "first" one) - sc = new SparkContext("local", "test") val rdd: RDD[Double] = sc .parallelize(Array(-1.0, -1.0, -1.0, -1.0, 2.0, 4.0, -1.0, -1.0), 8) .filter(_ >= 0.0) - + // Run the partitions, including the consecutive empty ones, through StatCounter val stats: StatCounter = rdd.stats(); assert(abs(6.0 - stats.sum) < 0.01); assert(abs(6.0/2 - rdd.mean) < 0.01); assert(abs(1.0 - rdd.variance) < 0.01); assert(abs(1.0 - rdd.stdev) < 0.01); - + // Add other tests here for classes that should be able to handle empty partitions correctly } } diff --git a/core/src/test/scala/spark/PipedRDDSuite.scala b/core/src/test/scala/spark/PipedRDDSuite.scala index ed075f93ec550..1c9ca5081120a 100644 --- a/core/src/test/scala/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/spark/PipedRDDSuite.scala @@ -3,10 +3,9 @@ package spark import org.scalatest.FunSuite import SparkContext._ -class PipedRDDSuite extends FunSuite with LocalSparkContext { - +class PipedRDDSuite extends FunSuite with SharedSparkContext { + test("basic pipe") { - sc = new SparkContext("local", "test") val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) val piped = nums.pipe(Seq("cat")) @@ -20,12 +19,11 @@ class PipedRDDSuite extends FunSuite with LocalSparkContext { } test("advanced pipe") { - sc = new SparkContext("local", "test") val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) val bl = sc.broadcast(List("0")) - val piped = nums.pipe(Seq("cat"), - Map[String, String](), + val piped = nums.pipe(Seq("cat"), + Map[String, String](), (f: String => Unit) => {bl.value.map(f(_));f("\u0001")}, (i:Int, f: String=> Unit) => f(i + "_")) @@ -43,8 +41,8 @@ class PipedRDDSuite extends FunSuite with LocalSparkContext { val nums1 = sc.makeRDD(Array("a\t1", "b\t2", "a\t3", "b\t4"), 2) val d = nums1.groupBy(str=>str.split("\t")(0)). - pipe(Seq("cat"), - Map[String, String](), + pipe(Seq("cat"), + Map[String, String](), (f: String => Unit) => {bl.value.map(f(_));f("\u0001")}, (i:Tuple2[String, Seq[String]], f: String=> Unit) => {for (e <- i._2){ f(e + "_")}}).collect() assert(d.size === 8) @@ -59,7 +57,6 @@ class PipedRDDSuite extends FunSuite with LocalSparkContext { } test("pipe with env variable") { - sc = new SparkContext("local", "test") val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) val piped = nums.pipe(Seq("printenv", "MY_TEST_ENV"), Map("MY_TEST_ENV" -> "LALALA")) val c = piped.collect() @@ -69,7 +66,6 @@ class PipedRDDSuite extends FunSuite with LocalSparkContext { } test("pipe with non-zero exit status") { - sc = new SparkContext("local", "test") val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) val piped = nums.pipe("cat nonexistent_file") intercept[SparkException] { diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 67f3332d444d8..d8db69b1c918c 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -7,10 +7,9 @@ import org.scalatest.time.{Span, Millis} import spark.SparkContext._ import spark.rdd.{CoalescedRDD, CoGroupedRDD, EmptyRDD, PartitionPruningRDD, ShuffledRDD} -class RDDSuite extends FunSuite with LocalSparkContext { +class RDDSuite extends FunSuite with SharedSparkContext { test("basic operations") { - sc = new SparkContext("local", "test") val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) assert(nums.collect().toList === List(1, 2, 3, 4)) val dups = sc.makeRDD(Array(1, 1, 2, 2, 3, 3, 4, 4), 2) @@ -46,7 +45,6 @@ class RDDSuite extends FunSuite with LocalSparkContext { } test("SparkContext.union") { - sc = new SparkContext("local", "test") val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) assert(sc.union(nums).collect().toList === List(1, 2, 3, 4)) assert(sc.union(nums, nums).collect().toList === List(1, 2, 3, 4, 1, 2, 3, 4)) @@ -55,7 +53,6 @@ class RDDSuite extends FunSuite with LocalSparkContext { } test("aggregate") { - sc = new SparkContext("local", "test") val pairs = sc.makeRDD(Array(("a", 1), ("b", 2), ("a", 2), ("c", 5), ("a", 3))) type StringMap = HashMap[String, Int] val emptyMap = new StringMap { @@ -75,57 +72,14 @@ class RDDSuite extends FunSuite with LocalSparkContext { assert(result.toSet === Set(("a", 6), ("b", 2), ("c", 5))) } - test("basic checkpointing") { - import java.io.File - val checkpointDir = File.createTempFile("temp", "") - checkpointDir.delete() - - sc = new SparkContext("local", "test") - sc.setCheckpointDir(checkpointDir.toString) - val parCollection = sc.makeRDD(1 to 4) - val flatMappedRDD = parCollection.flatMap(x => 1 to x) - flatMappedRDD.checkpoint() - assert(flatMappedRDD.dependencies.head.rdd == parCollection) - val result = flatMappedRDD.collect() - Thread.sleep(1000) - assert(flatMappedRDD.dependencies.head.rdd != parCollection) - assert(flatMappedRDD.collect() === result) - - checkpointDir.deleteOnExit() - } - test("basic caching") { - sc = new SparkContext("local", "test") val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() assert(rdd.collect().toList === List(1, 2, 3, 4)) assert(rdd.collect().toList === List(1, 2, 3, 4)) assert(rdd.collect().toList === List(1, 2, 3, 4)) } - test("unpersist RDD") { - sc = new SparkContext("local", "test") - val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() - rdd.count - assert(sc.persistentRdds.isEmpty === false) - rdd.unpersist() - assert(sc.persistentRdds.isEmpty === true) - - failAfter(Span(3000, Millis)) { - try { - while (! sc.getRDDStorageInfo.isEmpty) { - Thread.sleep(200) - } - } catch { - case _ => { Thread.sleep(10) } - // Do nothing. We might see exceptions because block manager - // is racing this thread to remove entries from the driver. - } - } - assert(sc.getRDDStorageInfo.isEmpty === true) - } - test("caching with failures") { - sc = new SparkContext("local", "test") val onlySplit = new Partition { override def index: Int = 0 } var shouldFail = true val rdd = new RDD[Int](sc, Nil) { @@ -148,7 +102,6 @@ class RDDSuite extends FunSuite with LocalSparkContext { } test("empty RDD") { - sc = new SparkContext("local", "test") val empty = new EmptyRDD[Int](sc) assert(empty.count === 0) assert(empty.collect().size === 0) @@ -168,37 +121,6 @@ class RDDSuite extends FunSuite with LocalSparkContext { } test("cogrouped RDDs") { - sc = new SparkContext("local", "test") - val rdd1 = sc.makeRDD(Array((1, "one"), (1, "another one"), (2, "two"), (3, "three")), 2) - val rdd2 = sc.makeRDD(Array((1, "one1"), (1, "another one1"), (2, "two1")), 2) - - // Use cogroup function - val cogrouped = rdd1.cogroup(rdd2).collectAsMap() - assert(cogrouped(1) === (Seq("one", "another one"), Seq("one1", "another one1"))) - assert(cogrouped(2) === (Seq("two"), Seq("two1"))) - assert(cogrouped(3) === (Seq("three"), Seq())) - - // Construct CoGroupedRDD directly, with map side combine enabled - val cogrouped1 = new CoGroupedRDD[Int]( - Seq(rdd1.asInstanceOf[RDD[(Int, Any)]], rdd2.asInstanceOf[RDD[(Int, Any)]]), - new HashPartitioner(3), - true).collectAsMap() - assert(cogrouped1(1).toSeq === Seq(Seq("one", "another one"), Seq("one1", "another one1"))) - assert(cogrouped1(2).toSeq === Seq(Seq("two"), Seq("two1"))) - assert(cogrouped1(3).toSeq === Seq(Seq("three"), Seq())) - - // Construct CoGroupedRDD directly, with map side combine disabled - val cogrouped2 = new CoGroupedRDD[Int]( - Seq(rdd1.asInstanceOf[RDD[(Int, Any)]], rdd2.asInstanceOf[RDD[(Int, Any)]]), - new HashPartitioner(3), - false).collectAsMap() - assert(cogrouped2(1).toSeq === Seq(Seq("one", "another one"), Seq("one1", "another one1"))) - assert(cogrouped2(2).toSeq === Seq(Seq("two"), Seq("two1"))) - assert(cogrouped2(3).toSeq === Seq(Seq("three"), Seq())) - } - - test("coalesced RDDs") { - sc = new SparkContext("local", "test") val data = sc.parallelize(1 to 10, 10) val coalesced1 = data.coalesce(2) @@ -236,7 +158,6 @@ class RDDSuite extends FunSuite with LocalSparkContext { } test("zipped RDDs") { - sc = new SparkContext("local", "test") val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) val zipped = nums.zip(nums.map(_ + 1.0)) assert(zipped.glom().map(_.toList).collect().toList === @@ -248,7 +169,6 @@ class RDDSuite extends FunSuite with LocalSparkContext { } test("partition pruning") { - sc = new SparkContext("local", "test") val data = sc.parallelize(1 to 10, 10) // Note that split number starts from 0, so > 8 means only 10th partition left. val prunedRdd = new PartitionPruningRDD(data, splitNum => splitNum > 8) @@ -260,7 +180,6 @@ class RDDSuite extends FunSuite with LocalSparkContext { test("mapWith") { import java.util.Random - sc = new SparkContext("local", "test") val ones = sc.makeRDD(Array(1, 1, 1, 1, 1, 1), 2) val randoms = ones.mapWith( (index: Int) => new Random(index + 42)) @@ -279,7 +198,6 @@ class RDDSuite extends FunSuite with LocalSparkContext { test("flatMapWith") { import java.util.Random - sc = new SparkContext("local", "test") val ones = sc.makeRDD(Array(1, 1, 1, 1, 1, 1), 2) val randoms = ones.flatMapWith( (index: Int) => new Random(index + 42)) @@ -301,7 +219,6 @@ class RDDSuite extends FunSuite with LocalSparkContext { test("filterWith") { import java.util.Random - sc = new SparkContext("local", "test") val ints = sc.makeRDD(Array(1, 2, 3, 4, 5, 6), 2) val sample = ints.filterWith( (index: Int) => new Random(index + 42)) @@ -319,7 +236,6 @@ class RDDSuite extends FunSuite with LocalSparkContext { } test("top with predefined ordering") { - sc = new SparkContext("local", "test") val nums = Array.range(1, 100000) val ints = sc.makeRDD(scala.util.Random.shuffle(nums), 2) val topK = ints.top(5) @@ -328,7 +244,6 @@ class RDDSuite extends FunSuite with LocalSparkContext { } test("top with custom ordering") { - sc = new SparkContext("local", "test") val words = Vector("a", "b", "c", "d") implicit val ord = implicitly[Ordering[String]].reverse val rdd = sc.makeRDD(words, 2) diff --git a/core/src/test/scala/spark/SharedSparkContext.scala b/core/src/test/scala/spark/SharedSparkContext.scala new file mode 100644 index 0000000000000..1da79f9824d11 --- /dev/null +++ b/core/src/test/scala/spark/SharedSparkContext.scala @@ -0,0 +1,25 @@ +package spark + +import org.scalatest.Suite +import org.scalatest.BeforeAndAfterAll + +/** Shares a local `SparkContext` between all tests in a suite and closes it at the end */ +trait SharedSparkContext extends BeforeAndAfterAll { self: Suite => + + @transient private var _sc: SparkContext = _ + + def sc: SparkContext = _sc + + override def beforeAll() { + _sc = new SparkContext("local", "test") + super.beforeAll() + } + + override def afterAll() { + if (_sc != null) { + LocalSparkContext.stop(_sc) + _sc = null + } + super.afterAll() + } +} diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index 0c1ec29f96c94..950218fa28077 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -16,54 +16,9 @@ import spark.rdd.ShuffledRDD import spark.SparkContext._ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { - - test("groupByKey") { - sc = new SparkContext("local", "test") - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))) - val groups = pairs.groupByKey().collect() - assert(groups.size === 2) - val valuesFor1 = groups.find(_._1 == 1).get._2 - assert(valuesFor1.toList.sorted === List(1, 2, 3)) - val valuesFor2 = groups.find(_._1 == 2).get._2 - assert(valuesFor2.toList.sorted === List(1)) - } - - test("groupByKey with duplicates") { - sc = new SparkContext("local", "test") - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val groups = pairs.groupByKey().collect() - assert(groups.size === 2) - val valuesFor1 = groups.find(_._1 == 1).get._2 - assert(valuesFor1.toList.sorted === List(1, 1, 2, 3)) - val valuesFor2 = groups.find(_._1 == 2).get._2 - assert(valuesFor2.toList.sorted === List(1)) - } - - test("groupByKey with negative key hash codes") { - sc = new SparkContext("local", "test") - val pairs = sc.parallelize(Array((-1, 1), (-1, 2), (-1, 3), (2, 1))) - val groups = pairs.groupByKey().collect() - assert(groups.size === 2) - val valuesForMinus1 = groups.find(_._1 == -1).get._2 - assert(valuesForMinus1.toList.sorted === List(1, 2, 3)) - val valuesFor2 = groups.find(_._1 == 2).get._2 - assert(valuesFor2.toList.sorted === List(1)) - } - - test("groupByKey with many output partitions") { - sc = new SparkContext("local", "test") - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))) - val groups = pairs.groupByKey(10).collect() - assert(groups.size === 2) - val valuesFor1 = groups.find(_._1 == 1).get._2 - assert(valuesFor1.toList.sorted === List(1, 2, 3)) - val valuesFor2 = groups.find(_._1 == 2).get._2 - assert(valuesFor2.toList.sorted === List(1)) - } - test("groupByKey with compression") { try { - System.setProperty("spark.blockManager.compress", "true") + System.setProperty("spark.shuffle.compress", "true") sc = new SparkContext("local", "test") val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1)), 4) val groups = pairs.groupByKey(4).collect() @@ -77,234 +32,6 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { } } - test("reduceByKey") { - sc = new SparkContext("local", "test") - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val sums = pairs.reduceByKey(_+_).collect() - assert(sums.toSet === Set((1, 7), (2, 1))) - } - - test("reduceByKey with collectAsMap") { - sc = new SparkContext("local", "test") - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val sums = pairs.reduceByKey(_+_).collectAsMap() - assert(sums.size === 2) - assert(sums(1) === 7) - assert(sums(2) === 1) - } - - test("reduceByKey with many output partitons") { - sc = new SparkContext("local", "test") - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val sums = pairs.reduceByKey(_+_, 10).collect() - assert(sums.toSet === Set((1, 7), (2, 1))) - } - - test("reduceByKey with partitioner") { - sc = new SparkContext("local", "test") - val p = new Partitioner() { - def numPartitions = 2 - def getPartition(key: Any) = key.asInstanceOf[Int] - } - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 1), (0, 1))).partitionBy(p) - val sums = pairs.reduceByKey(_+_) - assert(sums.collect().toSet === Set((1, 4), (0, 1))) - assert(sums.partitioner === Some(p)) - // count the dependencies to make sure there is only 1 ShuffledRDD - val deps = new HashSet[RDD[_]]() - def visit(r: RDD[_]) { - for (dep <- r.dependencies) { - deps += dep.rdd - visit(dep.rdd) - } - } - visit(sums) - assert(deps.size === 2) // ShuffledRDD, ParallelCollection - } - - test("join") { - sc = new SparkContext("local", "test") - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) - val joined = rdd1.join(rdd2).collect() - assert(joined.size === 4) - assert(joined.toSet === Set( - (1, (1, 'x')), - (1, (2, 'x')), - (2, (1, 'y')), - (2, (1, 'z')) - )) - } - - test("join all-to-all") { - sc = new SparkContext("local", "test") - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (1, 3))) - val rdd2 = sc.parallelize(Array((1, 'x'), (1, 'y'))) - val joined = rdd1.join(rdd2).collect() - assert(joined.size === 6) - assert(joined.toSet === Set( - (1, (1, 'x')), - (1, (1, 'y')), - (1, (2, 'x')), - (1, (2, 'y')), - (1, (3, 'x')), - (1, (3, 'y')) - )) - } - - test("leftOuterJoin") { - sc = new SparkContext("local", "test") - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) - val joined = rdd1.leftOuterJoin(rdd2).collect() - assert(joined.size === 5) - assert(joined.toSet === Set( - (1, (1, Some('x'))), - (1, (2, Some('x'))), - (2, (1, Some('y'))), - (2, (1, Some('z'))), - (3, (1, None)) - )) - } - - test("rightOuterJoin") { - sc = new SparkContext("local", "test") - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) - val joined = rdd1.rightOuterJoin(rdd2).collect() - assert(joined.size === 5) - assert(joined.toSet === Set( - (1, (Some(1), 'x')), - (1, (Some(2), 'x')), - (2, (Some(1), 'y')), - (2, (Some(1), 'z')), - (4, (None, 'w')) - )) - } - - test("join with no matches") { - sc = new SparkContext("local", "test") - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) - val rdd2 = sc.parallelize(Array((4, 'x'), (5, 'y'), (5, 'z'), (6, 'w'))) - val joined = rdd1.join(rdd2).collect() - assert(joined.size === 0) - } - - test("join with many output partitions") { - sc = new SparkContext("local", "test") - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) - val joined = rdd1.join(rdd2, 10).collect() - assert(joined.size === 4) - assert(joined.toSet === Set( - (1, (1, 'x')), - (1, (2, 'x')), - (2, (1, 'y')), - (2, (1, 'z')) - )) - } - - test("groupWith") { - sc = new SparkContext("local", "test") - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) - val joined = rdd1.groupWith(rdd2).collect() - assert(joined.size === 4) - assert(joined.toSet === Set( - (1, (ArrayBuffer(1, 2), ArrayBuffer('x'))), - (2, (ArrayBuffer(1), ArrayBuffer('y', 'z'))), - (3, (ArrayBuffer(1), ArrayBuffer())), - (4, (ArrayBuffer(), ArrayBuffer('w'))) - )) - } - - test("zero-partition RDD") { - sc = new SparkContext("local", "test") - val emptyDir = Files.createTempDir() - val file = sc.textFile(emptyDir.getAbsolutePath) - assert(file.partitions.size == 0) - assert(file.collect().toList === Nil) - // Test that a shuffle on the file works, because this used to be a bug - assert(file.map(line => (line, 1)).reduceByKey(_ + _).collect().toList === Nil) - } - - test("keys and values") { - sc = new SparkContext("local", "test") - val rdd = sc.parallelize(Array((1, "a"), (2, "b"))) - assert(rdd.keys.collect().toList === List(1, 2)) - assert(rdd.values.collect().toList === List("a", "b")) - } - - test("default partitioner uses partition size") { - sc = new SparkContext("local", "test") - // specify 2000 partitions - val a = sc.makeRDD(Array(1, 2, 3, 4), 2000) - // do a map, which loses the partitioner - val b = a.map(a => (a, (a * 2).toString)) - // then a group by, and see we didn't revert to 2 partitions - val c = b.groupByKey() - assert(c.partitions.size === 2000) - } - - test("default partitioner uses largest partitioner") { - sc = new SparkContext("local", "test") - val a = sc.makeRDD(Array((1, "a"), (2, "b")), 2) - val b = sc.makeRDD(Array((1, "a"), (2, "b")), 2000) - val c = a.join(b) - assert(c.partitions.size === 2000) - } - - test("subtract") { - sc = new SparkContext("local", "test") - val a = sc.parallelize(Array(1, 2, 3), 2) - val b = sc.parallelize(Array(2, 3, 4), 4) - val c = a.subtract(b) - assert(c.collect().toSet === Set(1)) - assert(c.partitions.size === a.partitions.size) - } - - test("subtract with narrow dependency") { - sc = new SparkContext("local", "test") - // use a deterministic partitioner - val p = new Partitioner() { - def numPartitions = 5 - def getPartition(key: Any) = key.asInstanceOf[Int] - } - // partitionBy so we have a narrow dependency - val a = sc.parallelize(Array((1, "a"), (2, "b"), (3, "c"))).partitionBy(p) - // more partitions/no partitioner so a shuffle dependency - val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) - val c = a.subtract(b) - assert(c.collect().toSet === Set((1, "a"), (3, "c"))) - // Ideally we could keep the original partitioner... - assert(c.partitioner === None) - } - - test("subtractByKey") { - sc = new SparkContext("local", "test") - val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 2) - val b = sc.parallelize(Array((2, 20), (3, 30), (4, 40)), 4) - val c = a.subtractByKey(b) - assert(c.collect().toSet === Set((1, "a"), (1, "a"))) - assert(c.partitions.size === a.partitions.size) - } - - test("subtractByKey with narrow dependency") { - sc = new SparkContext("local", "test") - // use a deterministic partitioner - val p = new Partitioner() { - def numPartitions = 5 - def getPartition(key: Any) = key.asInstanceOf[Int] - } - // partitionBy so we have a narrow dependency - val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c"))).partitionBy(p) - // more partitions/no partitioner so a shuffle dependency - val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) - val c = a.subtractByKey(b) - assert(c.collect().toSet === Set((1, "a"), (1, "a"))) - assert(c.partitioner.get === p) - } - test("shuffle non-zero block size") { sc = new SparkContext("local-cluster[2,1,512]", "test") val NUM_BLOCKS = 3 @@ -391,29 +118,6 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { // We should have at most 4 non-zero sized partitions assert(nonEmptyBlocks.size <= 4) } - - test("foldByKey") { - sc = new SparkContext("local", "test") - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val sums = pairs.foldByKey(0)(_+_).collect() - assert(sums.toSet === Set((1, 7), (2, 1))) - } - - test("foldByKey with mutable result type") { - sc = new SparkContext("local", "test") - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val bufs = pairs.mapValues(v => ArrayBuffer(v)).cache() - // Fold the values using in-place mutation - val sums = bufs.foldByKey(new ArrayBuffer[Int])(_ ++= _).collect() - assert(sums.toSet === Set((1, ArrayBuffer(1, 2, 3, 1)), (2, ArrayBuffer(1)))) - // Check that the mutable objects in the original RDD were not changed - assert(bufs.collect().toSet === Set( - (1, ArrayBuffer(1)), - (1, ArrayBuffer(2)), - (1, ArrayBuffer(3)), - (1, ArrayBuffer(1)), - (2, ArrayBuffer(1)))) - } } object ShuffleSuite { diff --git a/core/src/test/scala/spark/SortingSuite.scala b/core/src/test/scala/spark/SortingSuite.scala index 495f957e53f25..f7bf207c68c70 100644 --- a/core/src/test/scala/spark/SortingSuite.scala +++ b/core/src/test/scala/spark/SortingSuite.scala @@ -5,16 +5,14 @@ import org.scalatest.BeforeAndAfter import org.scalatest.matchers.ShouldMatchers import SparkContext._ -class SortingSuite extends FunSuite with LocalSparkContext with ShouldMatchers with Logging { - +class SortingSuite extends FunSuite with SharedSparkContext with ShouldMatchers with Logging { + test("sortByKey") { - sc = new SparkContext("local", "test") val pairs = sc.parallelize(Array((1, 0), (2, 0), (0, 0), (3, 0)), 2) - assert(pairs.sortByKey().collect() === Array((0,0), (1,0), (2,0), (3,0))) + assert(pairs.sortByKey().collect() === Array((0,0), (1,0), (2,0), (3,0))) } test("large array") { - sc = new SparkContext("local", "test") val rand = new scala.util.Random() val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } val pairs = sc.parallelize(pairArr, 2) @@ -24,7 +22,6 @@ class SortingSuite extends FunSuite with LocalSparkContext with ShouldMatchers w } test("large array with one split") { - sc = new SparkContext("local", "test") val rand = new scala.util.Random() val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } val pairs = sc.parallelize(pairArr, 2) @@ -32,9 +29,8 @@ class SortingSuite extends FunSuite with LocalSparkContext with ShouldMatchers w assert(sorted.partitions.size === 1) assert(sorted.collect() === pairArr.sortBy(_._1)) } - + test("large array with many partitions") { - sc = new SparkContext("local", "test") val rand = new scala.util.Random() val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } val pairs = sc.parallelize(pairArr, 2) @@ -42,9 +38,8 @@ class SortingSuite extends FunSuite with LocalSparkContext with ShouldMatchers w assert(sorted.partitions.size === 20) assert(sorted.collect() === pairArr.sortBy(_._1)) } - + test("sort descending") { - sc = new SparkContext("local", "test") val rand = new scala.util.Random() val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } val pairs = sc.parallelize(pairArr, 2) @@ -52,15 +47,13 @@ class SortingSuite extends FunSuite with LocalSparkContext with ShouldMatchers w } test("sort descending with one split") { - sc = new SparkContext("local", "test") val rand = new scala.util.Random() val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } val pairs = sc.parallelize(pairArr, 1) assert(pairs.sortByKey(false, 1).collect() === pairArr.sortWith((x, y) => x._1 > y._1)) } - + test("sort descending with many partitions") { - sc = new SparkContext("local", "test") val rand = new scala.util.Random() val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } val pairs = sc.parallelize(pairArr, 2) @@ -68,7 +61,6 @@ class SortingSuite extends FunSuite with LocalSparkContext with ShouldMatchers w } test("more partitions than elements") { - sc = new SparkContext("local", "test") val rand = new scala.util.Random() val pairArr = Array.fill(10) { (rand.nextInt(), rand.nextInt()) } val pairs = sc.parallelize(pairArr, 30) @@ -76,14 +68,12 @@ class SortingSuite extends FunSuite with LocalSparkContext with ShouldMatchers w } test("empty RDD") { - sc = new SparkContext("local", "test") val pairArr = new Array[(Int, Int)](0) val pairs = sc.parallelize(pairArr, 2) assert(pairs.sortByKey().collect() === pairArr.sortBy(_._1)) } test("partition balancing") { - sc = new SparkContext("local", "test") val pairArr = (1 to 1000).map(x => (x, x)).toArray val sorted = sc.parallelize(pairArr, 4).sortByKey() assert(sorted.collect() === pairArr.sortBy(_._1)) @@ -99,7 +89,6 @@ class SortingSuite extends FunSuite with LocalSparkContext with ShouldMatchers w } test("partition balancing for descending sort") { - sc = new SparkContext("local", "test") val pairArr = (1 to 1000).map(x => (x, x)).toArray val sorted = sc.parallelize(pairArr, 4).sortByKey(false) assert(sorted.collect() === pairArr.sortBy(_._1).reverse) diff --git a/core/src/test/scala/spark/UnpersistSuite.scala b/core/src/test/scala/spark/UnpersistSuite.scala new file mode 100644 index 0000000000000..94776e75720b8 --- /dev/null +++ b/core/src/test/scala/spark/UnpersistSuite.scala @@ -0,0 +1,30 @@ +package spark + +import org.scalatest.FunSuite +import org.scalatest.concurrent.Timeouts._ +import org.scalatest.time.{Span, Millis} +import spark.SparkContext._ + +class UnpersistSuite extends FunSuite with LocalSparkContext { + test("unpersist RDD") { + sc = new SparkContext("local", "test") + val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() + rdd.count + assert(sc.persistentRdds.isEmpty === false) + rdd.unpersist() + assert(sc.persistentRdds.isEmpty === true) + + failAfter(Span(3000, Millis)) { + try { + while (! sc.getRDDStorageInfo.isEmpty) { + Thread.sleep(200) + } + } catch { + case _ => { Thread.sleep(10) } + // Do nothing. We might see exceptions because block manager + // is racing this thread to remove entries from the driver. + } + } + assert(sc.getRDDStorageInfo.isEmpty === true) + } +} diff --git a/core/src/test/scala/spark/ZippedPartitionsSuite.scala b/core/src/test/scala/spark/ZippedPartitionsSuite.scala index 5f60aa75d7f03..96cb295f45b0c 100644 --- a/core/src/test/scala/spark/ZippedPartitionsSuite.scala +++ b/core/src/test/scala/spark/ZippedPartitionsSuite.scala @@ -17,9 +17,8 @@ object ZippedPartitionsSuite { } } -class ZippedPartitionsSuite extends FunSuite with LocalSparkContext { +class ZippedPartitionsSuite extends FunSuite with SharedSparkContext { test("print sizes") { - sc = new SparkContext("local", "test") val data1 = sc.makeRDD(Array(1, 2, 3, 4), 2) val data2 = sc.makeRDD(Array("1", "2", "3", "4", "5", "6"), 2) val data3 = sc.makeRDD(Array(1.0, 2.0), 2) From 32370da4e40062b88c921417cd7418d804e16f23 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 25 Jun 2013 22:08:19 -0400 Subject: [PATCH 44/94] Don't use forward slash in exclusion for JAR signature files --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 484f97d99220b..07572201dedc9 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -236,7 +236,7 @@ object SparkBuild extends Build { def extraAssemblySettings() = Seq(test in assembly := {}) ++ Seq( mergeStrategy in assembly := { case m if m.toLowerCase.endsWith("manifest.mf") => MergeStrategy.discard - case m if m.toLowerCase.matches("meta-inf/.*\\.sf$") => MergeStrategy.discard + case m if m.toLowerCase.matches("meta-inf.*\\.sf$") => MergeStrategy.discard case "reference.conf" => MergeStrategy.concat case _ => MergeStrategy.first } From d11025dc6aedd9763cfd2390e8daf24747d17258 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Wed, 26 Jun 2013 09:53:35 -0500 Subject: [PATCH 45/94] Be cute with Option and getenv. --- .../spark/deploy/worker/ExecutorRunner.scala | 21 ++++--------------- 1 file changed, 4 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index 4f8e1dcb260a1..a9b12421e6cca 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -1,6 +1,7 @@ package spark.deploy.worker import java.io._ +import java.lang.System.getenv import spark.deploy.{ExecutorState, ExecutorStateChanged, ApplicationDescription} import akka.actor.ActorRef import spark.{Utils, Logging} @@ -77,11 +78,7 @@ private[spark] class ExecutorRunner( def buildCommandSeq(): Seq[String] = { val command = appDesc.command - val runner = if (System.getenv("JAVA_HOME") == null) { - "java" - } else { - System.getenv("JAVA_HOME") + "/bin/java" - } + val runner = Option(getenv("JAVA_HOME")).map(_ + "/bin/java").getOrElse("java") // SPARK-698: do not call the run.cmd script, as process.destroy() // fails to kill a process tree on Windows Seq(runner) ++ buildJavaOpts() ++ Seq(command.mainClass) ++ @@ -93,18 +90,8 @@ private[spark] class ExecutorRunner( * the way the JAVA_OPTS are assembled there. */ def buildJavaOpts(): Seq[String] = { - val libraryOpts = if (System.getenv("SPARK_LIBRARY_PATH") == null) { - Nil - } else { - List("-Djava.library.path=" + System.getenv("SPARK_LIBRARY_PATH")) - } - - val userOpts = if (System.getenv("SPARK_JAVA_OPTS") == null) { - Nil - } else { - Utils.splitCommandString(System.getenv("SPARK_JAVA_OPTS")) - } - + val libraryOpts = Option(getenv("SPARK_LIBRARY_PATH")).map(p => List("-Djava.library.path=" + p)).getOrElse(Nil) + val userOpts = Option(getenv("SPARK_JAVA_OPTS")).map(Utils.splitCommandString).getOrElse(Nil) val memoryOpts = Seq("-Xms" + memory + "M", "-Xmx" + memory + "M") // Figure out our classpath with the external compute-classpath script From d7011632d15b6de2129d360277e304fddb8f2aac Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Wed, 26 Jun 2013 12:35:57 -0500 Subject: [PATCH 46/94] Wrap lines. --- core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index a9b12421e6cca..d7f58b2cb1bc1 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -90,7 +90,9 @@ private[spark] class ExecutorRunner( * the way the JAVA_OPTS are assembled there. */ def buildJavaOpts(): Seq[String] = { - val libraryOpts = Option(getenv("SPARK_LIBRARY_PATH")).map(p => List("-Djava.library.path=" + p)).getOrElse(Nil) + val libraryOpts = Option(getenv("SPARK_LIBRARY_PATH")) + .map(p => List("-Djava.library.path=" + p)) + .getOrElse(Nil) val userOpts = Option(getenv("SPARK_JAVA_OPTS")).map(Utils.splitCommandString).getOrElse(Nil) val memoryOpts = Seq("-Xms" + memory + "M", "-Xmx" + memory + "M") From 03906f7f0a8c93b09e3b47ccaad5b5f72c29302b Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 26 Jun 2013 17:40:22 -0700 Subject: [PATCH 47/94] Fixes to compute-classpath on Windows --- bin/compute-classpath.cmd | 4 ++-- run2.cmd | 1 + 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd index 1dff8fea220ed..6e7efbd334987 100644 --- a/bin/compute-classpath.cmd +++ b/bin/compute-classpath.cmd @@ -6,7 +6,7 @@ rem script and the ExecutorRunner in standalone cluster mode. set SCALA_VERSION=2.9.3 rem Figure out where the Spark framework is installed -set FWDIR=%~dp0\.. +set FWDIR=%~dp0..\ rem Load environment variables from conf\spark-env.cmd, if it exists if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" @@ -45,7 +45,7 @@ rem Add Scala standard library set CLASSPATH=%CLASSPATH%;%SCALA_HOME%\lib\scala-library.jar;%SCALA_HOME%\lib\scala-compiler.jar;%SCALA_HOME%\lib\jline.jar rem A bit of a hack to allow calling this script within run2.cmd without seeing output -if "x%DONT_PRINT_CLASSPATH%"=="x1" goto exit +if "%DONT_PRINT_CLASSPATH%"=="1" goto exit echo %CLASSPATH% diff --git a/run2.cmd b/run2.cmd index 25e4f3b57c1b5..a9c4df180ff59 100644 --- a/run2.cmd +++ b/run2.cmd @@ -48,6 +48,7 @@ set REPL_DIR=%FWDIR%repl rem Compute classpath using external script set DONT_PRINT_CLASSPATH=1 call "%FWDIR%bin\compute-classpath.cmd" +set DONT_PRINT_CLASSPATH=0 rem Figure out the JAR file that our examples were packaged into. rem First search in the build path from SBT: From aea727f68d5fe5e81fc04ece97ad94c6f12c7270 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 26 Jun 2013 21:14:38 -0700 Subject: [PATCH 48/94] Simplify Python docs a little to do substring search --- docs/python-programming-guide.md | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 3a7a8db4a6ee4..7f1e7cf93dc9f 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -27,14 +27,14 @@ Short functions can be passed to RDD methods using Python's [`lambda`](http://ww {% highlight python %} logData = sc.textFile(logFile).cache() -errors = logData.filter(lambda s: 'ERROR' in s.split()) +errors = logData.filter(lambda line: "ERROR" in line) {% endhighlight %} You can also pass functions that are defined using the `def` keyword; this is useful for more complicated functions that cannot be expressed using `lambda`: {% highlight python %} def is_error(line): - return 'ERROR' in line.split() + return "ERROR" in line errors = logData.filter(is_error) {% endhighlight %} @@ -43,8 +43,7 @@ Functions can access objects in enclosing scopes, although modifications to thos {% highlight python %} error_keywords = ["Exception", "Error"] def is_error(line): - words = line.split() - return any(keyword in words for keyword in error_keywords) + return any(keyword in line for keyword in error_keywords) errors = logData.filter(is_error) {% endhighlight %} From c767e7437059aa35bca3e8bb93264b35853c7a8f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 27 Jun 2013 21:48:58 -0700 Subject: [PATCH 49/94] Removing incorrect test statement --- core/src/test/scala/spark/scheduler/JobLoggerSuite.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala index 4000c4d5209b6..699901f1a1369 100644 --- a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala @@ -41,7 +41,6 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers val rootStage = new Stage(0, rootRdd, None, List(shuffleMapStage), jobID) joblogger.onStageSubmitted(SparkListenerStageSubmitted(rootStage, 4)) - joblogger.getEventQueue.size should be (1) joblogger.getRddNameTest(parentRdd) should be (parentRdd.getClass.getName) parentRdd.setName("MyRDD") joblogger.getRddNameTest(parentRdd) should be ("MyRDD") From 4974b658edf2716ff3c6f2e6863cddb2a4ddf891 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 27 Jun 2013 22:16:40 -0700 Subject: [PATCH 50/94] Look at JAVA_HOME before PATH to determine Java executable --- run | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/run b/run index 7c06a55062212..805466ea2c086 100755 --- a/run +++ b/run @@ -67,14 +67,15 @@ if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then fi fi else - if [ `command -v java` ]; then - RUNNER="java" + if [ -n "${JAVA_HOME}" ]; then + RUNNER="${JAVA_HOME}/bin/java" else - if [ -z "$JAVA_HOME" ]; then + if [ `command -v java` ]; then + RUNNER="java" + else echo "JAVA_HOME is not set" >&2 exit 1 fi - RUNNER="${JAVA_HOME}/bin/java" fi if [ -z "$SCALA_LIBRARY_PATH" ]; then if [ -z "$SCALA_HOME" ]; then From 4358acfe07e991090fbe009aafe3f5110fbf0c40 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 29 Jun 2013 15:25:06 -0700 Subject: [PATCH 51/94] Initialize Twitter4J OAuth from system properties instead of prompting --- .../spark/streaming/StreamingContext.scala | 4 ++- .../api/java/JavaStreamingContext.scala | 23 ++++++------- .../dstream/TwitterInputDStream.scala | 32 ++++++------------- .../java/spark/streaming/JavaAPISuite.java | 2 +- 4 files changed, 23 insertions(+), 38 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index e61438fe3ac45..36b841af8fa19 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -381,7 +381,9 @@ class StreamingContext private ( /** * Create a input stream that returns tweets received from Twitter. - * @param twitterAuth Twitter4J authentication + * @param twitterAuth Twitter4J authentication, or None to use Twitter4J's default OAuth + * authorization; this uses the system properties twitter4j.oauth.consumerKey, + * .consumerSecret, .accessToken and .accessTokenSecret. * @param filters Set of filter strings to get only those tweets that match them * @param storageLevel Storage level to use for storing the received objects */ diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index c4a223b419518..ed7b789d981a1 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -307,7 +307,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { /** * Create a input stream that returns tweets received from Twitter. - * @param twitterAuth Twitter4J Authorization + * @param twitterAuth Twitter4J Authorization object * @param filters Set of filter strings to get only those tweets that match them * @param storageLevel Storage level to use for storing the received objects */ @@ -320,10 +320,9 @@ class JavaStreamingContext(val ssc: StreamingContext) { } /** - * Create a input stream that returns tweets received from Twitter using - * java.util.Preferences to store OAuth token. OAuth key and secret should - * be provided using system properties twitter4j.oauth.consumerKey and - * twitter4j.oauth.consumerSecret + * Create a input stream that returns tweets received from Twitter using Twitter4J's default + * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, + * .consumerSecret, .accessToken and .accessTokenSecret to be set. * @param filters Set of filter strings to get only those tweets that match them * @param storageLevel Storage level to use for storing the received objects */ @@ -347,10 +346,9 @@ class JavaStreamingContext(val ssc: StreamingContext) { } /** - * Create a input stream that returns tweets received from Twitter using - * java.util.Preferences to store OAuth token. OAuth key and secret should - * be provided using system properties twitter4j.oauth.consumerKey and - * twitter4j.oauth.consumerSecret + * Create a input stream that returns tweets received from Twitter using Twitter4J's default + * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, + * .consumerSecret, .accessToken and .accessTokenSecret to be set. * @param filters Set of filter strings to get only those tweets that match them */ def twitterStream( @@ -370,10 +368,9 @@ class JavaStreamingContext(val ssc: StreamingContext) { } /** - * Create a input stream that returns tweets received from Twitter using - * java.util.Preferences to store OAuth token. OAuth key and secret should - * be provided using system properties twitter4j.oauth.consumerKey and - * twitter4j.oauth.consumerSecret + * Create a input stream that returns tweets received from Twitter using Twitter4J's default + * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, + * .consumerSecret, .accessToken and .accessTokenSecret to be set. */ def twitterStream(): JavaDStream[Status] = { ssc.twitterStream() diff --git a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala index e0c654d385f1e..ff7a58be4522a 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala @@ -4,21 +4,21 @@ import spark._ import spark.streaming._ import storage.StorageLevel import twitter4j._ -import twitter4j.auth.BasicAuthorization import twitter4j.auth.Authorization import java.util.prefs.Preferences +import twitter4j.conf.ConfigurationBuilder import twitter4j.conf.PropertyConfiguration import twitter4j.auth.OAuthAuthorization import twitter4j.auth.AccessToken /* A stream of Twitter statuses, potentially filtered by one or more keywords. * -* @constructor create a new Twitter stream using the supplied username and password to authenticate. +* @constructor create a new Twitter stream using the supplied Twitter4J authentication credentials. * An optional set of string filters can be used to restrict the set of tweets. The Twitter API is * such that this may return a sampled subset of all tweets during each interval. * -* Includes a simple implementation of OAuth using consumer key and secret provided using system -* properties twitter4j.oauth.consumerKey and twitter4j.oauth.consumerSecret +* If no Authorization object is provided, initializes OAuth authorization using the system +* properties twitter4j.oauth.consumerKey, .consumerSecret, .accessToken and .accessTokenSecret. */ private[streaming] class TwitterInputDStream( @@ -28,28 +28,14 @@ class TwitterInputDStream( storageLevel: StorageLevel ) extends NetworkInputDStream[Status](ssc_) { - lazy val createOAuthAuthorization: Authorization = { - val userRoot = Preferences.userRoot(); - val token = Option(userRoot.get(PropertyConfiguration.OAUTH_ACCESS_TOKEN, null)) - val tokenSecret = Option(userRoot.get(PropertyConfiguration.OAUTH_ACCESS_TOKEN_SECRET, null)) - val oAuth = new OAuthAuthorization(new PropertyConfiguration(System.getProperties())) - if (token.isEmpty || tokenSecret.isEmpty) { - val requestToken = oAuth.getOAuthRequestToken() - println("Authorize application using URL: "+requestToken.getAuthorizationURL()) - println("Enter PIN: ") - val pin = Console.readLine - val accessToken = if (pin.length() > 0) oAuth.getOAuthAccessToken(requestToken, pin) else oAuth.getOAuthAccessToken() - userRoot.put(PropertyConfiguration.OAUTH_ACCESS_TOKEN, accessToken.getToken()) - userRoot.put(PropertyConfiguration.OAUTH_ACCESS_TOKEN_SECRET, accessToken.getTokenSecret()) - userRoot.flush() - } else { - oAuth.setOAuthAccessToken(new AccessToken(token.get, tokenSecret.get)); - } - oAuth + private def createOAuthAuthorization(): Authorization = { + new OAuthAuthorization(new ConfigurationBuilder().build()) } + + private val authorization = twitterAuth.getOrElse(createOAuthAuthorization()) override def getReceiver(): NetworkReceiver[Status] = { - new TwitterReceiver(if (twitterAuth.isEmpty) createOAuthAuthorization else twitterAuth.get, filters, storageLevel) + new TwitterReceiver(authorization, filters, storageLevel) } } diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java index e5fdbe1b7af75..4cf10582a951f 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java @@ -1267,7 +1267,7 @@ public void testFileStream() { @Test public void testTwitterStream() { String[] filters = new String[] { "good", "bad", "ugly" }; - JavaDStream test = ssc.twitterStream("username", "password", filters, StorageLevel.MEMORY_ONLY()); + JavaDStream test = ssc.twitterStream(filters, StorageLevel.MEMORY_ONLY()); } @Test From 5cfcd3c336cc13e9fd448ae122216e4b583b77b4 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 29 Jun 2013 15:37:27 -0700 Subject: [PATCH 52/94] Remove Twitter4J specific repo since it's in Maven central --- pom.xml | 11 ----------- project/SparkBuild.scala | 3 +-- 2 files changed, 1 insertion(+), 13 deletions(-) diff --git a/pom.xml b/pom.xml index 3bcb2a3f345dc..7a31be98b2b01 100644 --- a/pom.xml +++ b/pom.xml @@ -109,17 +109,6 @@ false - - twitter4j-repo - Twitter4J Repository - http://twitter4j.org/maven2/ - - true - - - false - - diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 07572201dedc9..5e4692162e900 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -138,8 +138,7 @@ object SparkBuild extends Build { resolvers ++= Seq( "JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/", "Spray Repository" at "http://repo.spray.cc/", - "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/", - "Twitter4J Repository" at "http://twitter4j.org/maven2/" + "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/" ), libraryDependencies ++= Seq( From 03d0b858c807339b4221bedffa29ac76eef5352e Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 30 Jun 2013 15:38:58 -0700 Subject: [PATCH 53/94] Made use of spark.executor.memory setting consistent and documented it Conflicts: core/src/main/scala/spark/SparkContext.scala --- core/src/main/scala/spark/SparkContext.scala | 24 +++++++++----- .../scheduler/cluster/SchedulerBackend.scala | 11 ++----- docs/configuration.md | 31 ++++++++++++------- docs/ec2-scripts.md | 5 ++- docs/tuning.md | 6 ++-- 5 files changed, 43 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 70a9d7698c490..366afb2a2a7e2 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -115,13 +115,14 @@ class SparkContext( // Environment variables to pass to our executors private[spark] val executorEnvs = HashMap[String, String]() // Note: SPARK_MEM is included for Mesos, but overwritten for standalone mode in ExecutorRunner - for (key <- Seq("SPARK_MEM", "SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS", - "SPARK_TESTING")) { + for (key <- Seq("SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS", "SPARK_TESTING")) { val value = System.getenv(key) if (value != null) { executorEnvs(key) = value } } + // Since memory can be set with a system property too, use that + executorEnvs("SPARK_MEM") = SparkContext.executorMemoryRequested + "m" if (environment != null) { executorEnvs ++= environment } @@ -156,14 +157,12 @@ class SparkContext( scheduler case LOCAL_CLUSTER_REGEX(numSlaves, coresPerSlave, memoryPerSlave) => - // Check to make sure SPARK_MEM <= memoryPerSlave. Otherwise Spark will just hang. + // Check to make sure memory requested <= memoryPerSlave. Otherwise Spark will just hang. val memoryPerSlaveInt = memoryPerSlave.toInt - val sparkMemEnv = System.getenv("SPARK_MEM") - val sparkMemEnvInt = if (sparkMemEnv != null) Utils.memoryStringToMb(sparkMemEnv) else 512 - if (sparkMemEnvInt > memoryPerSlaveInt) { + if (SparkContext.executorMemoryRequested > memoryPerSlaveInt) { throw new SparkException( - "Slave memory (%d MB) cannot be smaller than SPARK_MEM (%d MB)".format( - memoryPerSlaveInt, sparkMemEnvInt)) + "Asked to launch cluster with %d MB RAM / worker but requested %d MB/worker".format( + memoryPerSlaveInt, SparkContext.executorMemoryRequested)) } val scheduler = new ClusterScheduler(this) @@ -881,6 +880,15 @@ object SparkContext { /** Find the JAR that contains the class of a particular object */ def jarOfObject(obj: AnyRef): Seq[String] = jarOfClass(obj.getClass) + + /** Get the amount of memory per executor requested through system properties or SPARK_MEM */ + private[spark] val executorMemoryRequested = { + // TODO: Might need to add some extra memory for the non-heap parts of the JVM + Option(System.getProperty("spark.executor.memory")) + .orElse(Option(System.getenv("SPARK_MEM"))) + .map(Utils.memoryStringToMb) + .getOrElse(512) + } } diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala index 9ac875de3a16a..8844057a5c33d 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala @@ -1,6 +1,6 @@ package spark.scheduler.cluster -import spark.Utils +import spark.{SparkContext, Utils} /** * A backend interface for cluster scheduling systems that allows plugging in different ones under @@ -14,14 +14,7 @@ private[spark] trait SchedulerBackend { def defaultParallelism(): Int // Memory used by each executor (in megabytes) - protected val executorMemory = { - // TODO: Might need to add some extra memory for the non-heap parts of the JVM - Option(System.getProperty("spark.executor.memory")) - .orElse(Option(System.getenv("SPARK_MEM"))) - .map(Utils.memoryStringToMb) - .getOrElse(512) - } - + protected val executorMemory: Int = SparkContext.executorMemoryRequested // TODO: Probably want to add a killTask too } diff --git a/docs/configuration.md b/docs/configuration.md index 2de512f8965fe..ae61769e3154b 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -25,23 +25,25 @@ Inside `spark-env.sh`, you *must* set at least the following two variables: * `SCALA_HOME`, to point to your Scala installation. * `MESOS_NATIVE_LIBRARY`, if you are [running on a Mesos cluster](running-on-mesos.html). -In addition, there are four other variables that control execution. These can be set *either in `spark-env.sh` -or in each job's driver program*, because they will automatically be propagated to workers from the driver. -For a multi-user environment, we recommend setting the in the driver program instead of `spark-env.sh`, so -that different user jobs can use different amounts of memory, JVM options, etc. +In addition, there are four other variables that control execution. These should be set *in the environment that +launches the job's driver program* instead of `spark-env.sh`, because they will be automatically propagated to +workers. Setting these per-job instead of in `spark-env.sh` ensures that different jobs can have different settings +for these variables. -* `SPARK_MEM`, to set the amount of memory used per node (this should be in the same format as the - JVM's -Xmx option, e.g. `300m` or `1g`) * `SPARK_JAVA_OPTS`, to add JVM options. This includes any system properties that you'd like to pass with `-D`. * `SPARK_CLASSPATH`, to add elements to Spark's classpath. * `SPARK_LIBRARY_PATH`, to add search directories for native libraries. +* `SPARK_MEM`, to set the amount of memory used per node. This should be in the same format as the + JVM's -Xmx option, e.g. `300m` or `1g`. Note that this option will soon be deprecated in favor of + the `spark.executor.memory` system property, so we recommend using that in new code. -Note that if you do set these in `spark-env.sh`, they will override the values set by user programs, which -is undesirable; you can choose to have `spark-env.sh` set them only if the user program hasn't, as follows: +Beware that if you do set these variables in `spark-env.sh`, they will override the values set by user programs, +which is undesirable; if you prefer, you can choose to have `spark-env.sh` set them only if the user program +hasn't, as follows: {% highlight bash %} -if [ -z "$SPARK_MEM" ] ; then - SPARK_MEM="1g" +if [ -z "$SPARK_JAVA_OPTS" ] ; then + SPARK_JAVA_OPTS="-verbose:gc" fi {% endhighlight %} @@ -55,10 +57,17 @@ val sc = new SparkContext(...) {% endhighlight %} Most of the configurable system properties control internal settings that have reasonable default values. However, -there are at least four properties that you will commonly want to control: +there are at least five properties that you will commonly want to control: + + + + + diff --git a/docs/ec2-scripts.md b/docs/ec2-scripts.md index dc57035ebaf6c..eab8a0ff20403 100644 --- a/docs/ec2-scripts.md +++ b/docs/ec2-scripts.md @@ -106,9 +106,8 @@ permissions on your private key file, you can run `launch` with the # Configuration You can edit `/root/spark/conf/spark-env.sh` on each machine to set Spark configuration options, such -as JVM options and, most crucially, the amount of memory to use per machine (`SPARK_MEM`). -This file needs to be copied to **every machine** to reflect the change. The easiest way to do this -is to use a script we provide called `copy-dir`. First edit your `spark-env.sh` file on the master, +as JVM options. This file needs to be copied to **every machine** to reflect the change. The easiest way to +do this is to use a script we provide called `copy-dir`. First edit your `spark-env.sh` file on the master, then run `~/spark-ec2/copy-dir /root/spark/conf` to RSYNC it to all the workers. The [configuration guide](configuration.html) describes the available configuration options. diff --git a/docs/tuning.md b/docs/tuning.md index 32c7ab86e9991..5ffca54481b80 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -157,9 +157,9 @@ their work directories), *not* on your driver program. **Cache Size Tuning** -One important configuration parameter for GC is the amount of memory that should be used for -caching RDDs. By default, Spark uses 66% of the configured memory (`SPARK_MEM`) to cache RDDs. This means that - 33% of memory is available for any objects created during task execution. +One important configuration parameter for GC is the amount of memory that should be used for caching RDDs. +By default, Spark uses 66% of the configured executor memory (`spark.executor.memory` or `SPARK_MEM`) to +cache RDDs. This means that 33% of memory is available for any objects created during task execution. In case your tasks slow down and you find that your JVM is garbage-collecting frequently or running out of memory, lowering this value will help reduce the memory consumption. To change this to say 50%, you can call From 5bbd0eec84867937713ceb8438f25a943765a084 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 30 Jun 2013 17:00:26 -0700 Subject: [PATCH 54/94] Update docs on SCALA_LIBRARY_PATH --- conf/spark-env.sh.template | 18 ++++++------------ docs/configuration.md | 4 +++- 2 files changed, 9 insertions(+), 13 deletions(-) diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index 37565ca827980..b8936314ecce2 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -3,8 +3,10 @@ # This file contains environment variables required to run Spark. Copy it as # spark-env.sh and edit that to configure Spark for your site. At a minimum, # the following two variables should be set: -# - MESOS_NATIVE_LIBRARY, to point to your Mesos native library (libmesos.so) -# - SCALA_HOME, to point to your Scala installation +# - SCALA_HOME, to point to your Scala installation, or SCALA_LIBRARY_PATH to +# point to the directory for Scala library JARs (if you install Scala as a +# Debian or RPM package, these are in a separate path, often /usr/share/java) +# - MESOS_NATIVE_LIBRARY, to point to your libmesos.so if you use Mesos # # If using the standalone deploy mode, you can also set variables for it: # - SPARK_MASTER_IP, to bind the master to a different IP address @@ -12,14 +14,6 @@ # - SPARK_WORKER_CORES, to set the number of cores to use on this machine # - SPARK_WORKER_MEMORY, to set how much memory to use (e.g. 1000m, 2g) # - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT -# - SPARK_WORKER_INSTANCES, to set the number of worker instances/processes to be spawned on every slave machine -# -# Finally, Spark also relies on the following variables, but these can be set -# on just the *master* (i.e. in your driver program), and will automatically -# be propagated to workers: -# - SPARK_MEM, to change the amount of memory used per node (this should -# be in the same format as the JVM's -Xmx option, e.g. 300m or 1g) -# - SPARK_CLASSPATH, to add elements to Spark's classpath -# - SPARK_JAVA_OPTS, to add JVM options -# - SPARK_LIBRARY_PATH, to add extra search paths for native libraries. +# - SPARK_WORKER_INSTANCES, to set the number of worker instances/processes +# to be spawned on every slave machine diff --git a/docs/configuration.md b/docs/configuration.md index ae61769e3154b..3266db7af149e 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -22,7 +22,9 @@ the copy executable. Inside `spark-env.sh`, you *must* set at least the following two variables: -* `SCALA_HOME`, to point to your Scala installation. +* `SCALA_HOME`, to point to your Scala installation, or `SCALA_LIBRARY_PATH` to point to the directory for Scala + library JARs (if you install Scala as a Debian or RPM package, there is no `SCALA_HOME`, but these libraries + are in a separate path, typically /usr/share/java; look for `scala-library.jar`). * `MESOS_NATIVE_LIBRARY`, if you are [running on a Mesos cluster](running-on-mesos.html). In addition, there are four other variables that control execution. These should be set *in the environment that From 39ae073b5cd0dcfe4a00d9f205c88bad9df37870 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 30 Jun 2013 17:11:14 -0700 Subject: [PATCH 55/94] Increase SLF4j version in Maven too --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 7a31be98b2b01..48e623fa1cb17 100644 --- a/pom.xml +++ b/pom.xml @@ -56,7 +56,7 @@ 2.0.3 1.0-M2.1 1.1.1 - 1.6.1 + 1.7.2 4.1.2 1.2.17 From 3296d132b6ce042843de6e7384800e089b49e5fa Mon Sep 17 00:00:00 2001 From: root Date: Mon, 1 Jul 2013 02:45:00 +0000 Subject: [PATCH 56/94] Fix performance bug with new Python code not using buffered streams --- core/src/main/scala/spark/SparkEnv.scala | 3 +- .../scala/spark/api/python/PythonRDD.scala | 33 ++++++++++--------- 2 files changed, 19 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 7ccde2e8182f3..ec59b4f48fddd 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -59,7 +59,8 @@ class SparkEnv ( def createPythonWorker(pythonExec: String, envVars: Map[String, String]): java.net.Socket = { synchronized { - pythonWorkers.getOrElseUpdate((pythonExec, envVars), new PythonWorkerFactory(pythonExec, envVars)).create() + val key = (pythonExec, envVars) + pythonWorkers.getOrElseUpdate(key, new PythonWorkerFactory(pythonExec, envVars)).create() } } diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 63140cf37f5bb..3f283afa62f27 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -45,37 +45,38 @@ private[spark] class PythonRDD[T: ClassManifest]( new Thread("stdin writer for " + pythonExec) { override def run() { SparkEnv.set(env) - val out = new PrintWriter(worker.getOutputStream) - val dOut = new DataOutputStream(worker.getOutputStream) + val stream = new BufferedOutputStream(worker.getOutputStream) + val dataOut = new DataOutputStream(stream) + val printOut = new PrintWriter(stream) // Partition index - dOut.writeInt(split.index) + dataOut.writeInt(split.index) // sparkFilesDir - PythonRDD.writeAsPickle(SparkFiles.getRootDirectory, dOut) + PythonRDD.writeAsPickle(SparkFiles.getRootDirectory, dataOut) // Broadcast variables - dOut.writeInt(broadcastVars.length) + dataOut.writeInt(broadcastVars.length) for (broadcast <- broadcastVars) { - dOut.writeLong(broadcast.id) - dOut.writeInt(broadcast.value.length) - dOut.write(broadcast.value) - dOut.flush() + dataOut.writeLong(broadcast.id) + dataOut.writeInt(broadcast.value.length) + dataOut.write(broadcast.value) } + dataOut.flush() // Serialized user code for (elem <- command) { - out.println(elem) + printOut.println(elem) } - out.flush() + printOut.flush() // Data values for (elem <- parent.iterator(split, context)) { - PythonRDD.writeAsPickle(elem, dOut) + PythonRDD.writeAsPickle(elem, dataOut) } - dOut.flush() - out.flush() + dataOut.flush() + printOut.flush() worker.shutdownOutput() } }.start() // Return an iterator that read lines from the process's stdout - val stream = new DataInputStream(worker.getInputStream) + val stream = new DataInputStream(new BufferedInputStream(worker.getInputStream)) return new Iterator[Array[Byte]] { def next(): Array[Byte] = { val obj = _nextObj @@ -288,7 +289,7 @@ class PythonAccumulatorParam(@transient serverHost: String, serverPort: Int) // This happens on the master, where we pass the updates to Python through a socket val socket = new Socket(serverHost, serverPort) val in = socket.getInputStream - val out = new DataOutputStream(socket.getOutputStream) + val out = new DataOutputStream(new BufferedOutputStream(socket.getOutputStream)) out.writeInt(val2.size) for (array <- val2) { out.writeInt(array.length) From ec31e68d5df259e6df001529235d8c906ff02a6f Mon Sep 17 00:00:00 2001 From: root Date: Mon, 1 Jul 2013 06:20:14 +0000 Subject: [PATCH 57/94] Fixed PySpark perf regression by not using socket.makefile(), and improved debuggability by letting "print" statements show up in the executor's stderr Conflicts: core/src/main/scala/spark/api/python/PythonRDD.scala --- .../scala/spark/api/python/PythonRDD.scala | 10 +++-- .../api/python/PythonWorkerFactory.scala | 20 ++++++++- python/pyspark/daemon.py | 42 +++++++++++-------- 3 files changed, 50 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 3f283afa62f27..31d8ea89d43e4 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -22,6 +22,8 @@ private[spark] class PythonRDD[T: ClassManifest]( accumulator: Accumulator[JList[Array[Byte]]]) extends RDD[Array[Byte]](parent) { + val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt + // Similar to Runtime.exec(), if we are given a single string, split it into words // using a standard StringTokenizer (i.e. by spaces) def this(parent: RDD[T], command: String, envVars: JMap[String, String], @@ -45,7 +47,7 @@ private[spark] class PythonRDD[T: ClassManifest]( new Thread("stdin writer for " + pythonExec) { override def run() { SparkEnv.set(env) - val stream = new BufferedOutputStream(worker.getOutputStream) + val stream = new BufferedOutputStream(worker.getOutputStream, bufferSize) val dataOut = new DataOutputStream(stream) val printOut = new PrintWriter(stream) // Partition index @@ -76,7 +78,7 @@ private[spark] class PythonRDD[T: ClassManifest]( }.start() // Return an iterator that read lines from the process's stdout - val stream = new DataInputStream(new BufferedInputStream(worker.getInputStream)) + val stream = new DataInputStream(new BufferedInputStream(worker.getInputStream, bufferSize)) return new Iterator[Array[Byte]] { def next(): Array[Byte] = { val obj = _nextObj @@ -276,6 +278,8 @@ class PythonAccumulatorParam(@transient serverHost: String, serverPort: Int) extends AccumulatorParam[JList[Array[Byte]]] { Utils.checkHost(serverHost, "Expected hostname") + + val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt override def zero(value: JList[Array[Byte]]): JList[Array[Byte]] = new JArrayList @@ -289,7 +293,7 @@ class PythonAccumulatorParam(@transient serverHost: String, serverPort: Int) // This happens on the master, where we pass the updates to Python through a socket val socket = new Socket(serverHost, serverPort) val in = socket.getInputStream - val out = new DataOutputStream(new BufferedOutputStream(socket.getOutputStream)) + val out = new DataOutputStream(new BufferedOutputStream(socket.getOutputStream, bufferSize)) out.writeInt(val2.size) for (array <- val2) { out.writeInt(array.length) diff --git a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala index 8844411d738af..85d1dfeac8bb4 100644 --- a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala @@ -51,7 +51,6 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String val workerEnv = pb.environment() workerEnv.putAll(envVars) daemon = pb.start() - daemonPort = new DataInputStream(daemon.getInputStream).readInt() // Redirect the stderr to ours new Thread("stderr reader for " + pythonExec) { @@ -69,6 +68,25 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String } } }.start() + + val in = new DataInputStream(daemon.getInputStream) + daemonPort = in.readInt() + + // Redirect further stdout output to our stderr + new Thread("stdout reader for " + pythonExec) { + override def run() { + scala.util.control.Exception.ignoring(classOf[IOException]) { + // FIXME HACK: We copy the stream on the level of bytes to + // attempt to dodge encoding problems. + var buf = new Array[Byte](1024) + var len = in.read(buf) + while (len != -1) { + System.err.write(buf, 0, len) + len = in.read(buf) + } + } + } + }.start() } catch { case e => { stopDaemon() diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index 78a2da1e18ea4..78c9457b84d9c 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -1,10 +1,13 @@ import os +import signal +import socket import sys +import traceback import multiprocessing from ctypes import c_bool from errno import EINTR, ECHILD -from socket import socket, AF_INET, SOCK_STREAM, SOMAXCONN -from signal import signal, SIGHUP, SIGTERM, SIGCHLD, SIG_DFL, SIG_IGN +from socket import AF_INET, SOCK_STREAM, SOMAXCONN +from signal import SIGHUP, SIGTERM, SIGCHLD, SIG_DFL, SIG_IGN from pyspark.worker import main as worker_main from pyspark.serializers import write_int @@ -33,11 +36,12 @@ def compute_real_exit_code(exit_code): def worker(listen_sock): # Redirect stdout to stderr os.dup2(2, 1) + sys.stdout = sys.stderr # The sys.stdout object is different from file descriptor 1 # Manager sends SIGHUP to request termination of workers in the pool def handle_sighup(*args): assert should_exit() - signal(SIGHUP, handle_sighup) + signal.signal(SIGHUP, handle_sighup) # Cleanup zombie children def handle_sigchld(*args): @@ -51,7 +55,7 @@ def handle_sigchld(*args): handle_sigchld() elif err.errno != ECHILD: raise - signal(SIGCHLD, handle_sigchld) + signal.signal(SIGCHLD, handle_sigchld) # Handle clients while not should_exit(): @@ -70,19 +74,22 @@ def handle_sigchld(*args): # never receives SIGCHLD unless a worker crashes. if os.fork() == 0: # Leave the worker pool - signal(SIGHUP, SIG_DFL) + signal.signal(SIGHUP, SIG_DFL) listen_sock.close() - # Handle the client then exit - sockfile = sock.makefile() + # Read the socket using fdopen instead of socket.makefile() because the latter + # seems to be very slow; note that we need to dup() the file descriptor because + # otherwise writes also cause a seek that makes us miss data on the read side. + infile = os.fdopen(os.dup(sock.fileno()), "a+", 65536) + outfile = os.fdopen(os.dup(sock.fileno()), "a+", 65536) exit_code = 0 try: - worker_main(sockfile, sockfile) + worker_main(infile, outfile) except SystemExit as exc: - exit_code = exc.code + exit_code = exc.code finally: - sockfile.close() - sock.close() - os._exit(compute_real_exit_code(exit_code)) + outfile.flush() + sock.close() + os._exit(compute_real_exit_code(exit_code)) else: sock.close() @@ -92,7 +99,6 @@ def launch_worker(listen_sock): try: worker(listen_sock) except Exception as err: - import traceback traceback.print_exc() os._exit(1) else: @@ -105,7 +111,7 @@ def manager(): os.setpgid(0, 0) # Create a listening socket on the AF_INET loopback interface - listen_sock = socket(AF_INET, SOCK_STREAM) + listen_sock = socket.socket(AF_INET, SOCK_STREAM) listen_sock.bind(('127.0.0.1', 0)) listen_sock.listen(max(1024, 2 * POOLSIZE, SOMAXCONN)) listen_host, listen_port = listen_sock.getsockname() @@ -121,8 +127,8 @@ def shutdown(): exit_flag.value = True # Gracefully exit on SIGTERM, don't die on SIGHUP - signal(SIGTERM, lambda signum, frame: shutdown()) - signal(SIGHUP, SIG_IGN) + signal.signal(SIGTERM, lambda signum, frame: shutdown()) + signal.signal(SIGHUP, SIG_IGN) # Cleanup zombie children def handle_sigchld(*args): @@ -133,7 +139,7 @@ def handle_sigchld(*args): except EnvironmentError as err: if err.errno not in (ECHILD, EINTR): raise - signal(SIGCHLD, handle_sigchld) + signal.signal(SIGCHLD, handle_sigchld) # Initialization complete sys.stdout.close() @@ -148,7 +154,7 @@ def handle_sigchld(*args): shutdown() raise finally: - signal(SIGTERM, SIG_DFL) + signal.signal(SIGTERM, SIG_DFL) exit_flag.value = True # Send SIGHUP to notify workers of shutdown os.kill(0, SIGHUP) From 7cd490ef5ba28df31f5e061eff83c855731dfca4 Mon Sep 17 00:00:00 2001 From: root Date: Mon, 1 Jul 2013 06:25:17 +0000 Subject: [PATCH 58/94] Clarify that PySpark is not supported on Windows --- docs/python-programming-guide.md | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 7f1e7cf93dc9f..e8aaac74d030e 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -17,10 +17,9 @@ There are a few key differences between the Python and Scala APIs: * Python is dynamically typed, so RDDs can hold objects of different types. * PySpark does not currently support the following Spark features: - Special functions on RDDs of doubles, such as `mean` and `stdev` - - `lookup` + - `lookup`, `sample` and `sort` - `persist` at storage levels other than `MEMORY_ONLY` - - `sample` - - `sort` + - Execution on Windows -- this is slated for a future release In PySpark, RDDs support the same methods as their Scala counterparts but take Python functions and return Python collection types. Short functions can be passed to RDD methods using Python's [`lambda`](http://www.diveintopython.net/power_of_introspection/lambda_functions.html) syntax: From 6fdbc68f2c5b220d1618d5a78d46aa0f844cae45 Mon Sep 17 00:00:00 2001 From: Konstantin Boudnik Date: Mon, 1 Jul 2013 16:05:55 -0700 Subject: [PATCH 59/94] Fixing missed hbase dependency in examples hadoop2-yarn profile --- examples/pom.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/examples/pom.xml b/examples/pom.xml index 3e5271ec2f963..78ec58729b3c3 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -193,6 +193,11 @@ hadoop-yarn-common provided + + org.apache.hbase + hbase + 0.94.6 + From 9d7026dc11b3b30381bb7f9c0449696a39403916 Mon Sep 17 00:00:00 2001 From: Thomas Dudziak Date: Thu, 3 Jan 2013 16:09:10 -0800 Subject: [PATCH 60/94] Added csd-specific version and distribution settings --- bagel/pom.xml | 4 ++-- core/pom.xml | 4 ++-- examples/pom.xml | 4 ++-- pom.xml | 19 +++++++++++++++---- repl-bin/pom.xml | 4 ++-- repl/pom.xml | 4 ++-- 6 files changed, 25 insertions(+), 14 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index b83a0ef6c0f37..03187dd2e2510 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -3,8 +3,8 @@ 4.0.0 org.spark-project - spark-parent - 0.8.0-SNAPSHOT + parent + 0.7.0-csd-1-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 88f0ed70f3c03..2f718f02b62df 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -3,8 +3,8 @@ 4.0.0 org.spark-project - spark-parent - 0.8.0-SNAPSHOT + parent + 0.7.0-csd-1-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 78ec58729b3c3..39a65beab1b83 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -3,8 +3,8 @@ 4.0.0 org.spark-project - spark-parent - 0.8.0-SNAPSHOT + parent + 0.7.0-csd-1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 48e623fa1cb17..d541f72e168c3 100644 --- a/pom.xml +++ b/pom.xml @@ -2,8 +2,8 @@ 4.0.0 org.spark-project - spark-parent - 0.8.0-SNAPSHOT + parent + 0.7.0-csd-1-SNAPSHOT pom Spark Project Parent POM http://spark-project.org/ @@ -15,8 +15,8 @@ - scm:git:git@github.com:mesos/spark.git - scm:git:git@github.com:mesos/spark.git + scm:git:git@github.com:clearstorydata/spark.git + scm:git:git@github.com:clearstorydata/spark.git @@ -146,6 +146,17 @@ + + + ${distRepo.snapshots.id} + ${distRepo.snapshots.url} + + + ${distRepo.releases.id} + ${distRepo.releases.url} + + + diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 7a7280313edb7..a8597e380689a 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -3,8 +3,8 @@ 4.0.0 org.spark-project - spark-parent - 0.8.0-SNAPSHOT + parent + 0.7.0-csd-1-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 92a2020b48d0b..a51dfeaa7b378 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -3,8 +3,8 @@ 4.0.0 org.spark-project - spark-parent - 0.8.0-SNAPSHOT + parent + 0.7.0-csd-1-SNAPSHOT ../pom.xml From a11ebbe0f9753a5e8b7fcfa9dc456f355c848f71 Mon Sep 17 00:00:00 2001 From: Thomas Dudziak Date: Thu, 3 Jan 2013 16:27:08 -0800 Subject: [PATCH 61/94] [maven-release-plugin] prepare release parent-0.7.0-csd-1 --- bagel/pom.xml | 2 +- core/pom.xml | 10 +++++----- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 4 ++-- repl/pom.xml | 2 +- 6 files changed, 11 insertions(+), 11 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index 03187dd2e2510..fee9f1742a0b4 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-1-SNAPSHOT + 0.7.0-csd-1 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 2f718f02b62df..3f0e20f833bd1 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-1-SNAPSHOT + 0.7.0-csd-1 ../pom.xml @@ -143,14 +143,14 @@ true - - + + - - + + diff --git a/examples/pom.xml b/examples/pom.xml index 39a65beab1b83..1766be4581341 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-1-SNAPSHOT + 0.7.0-csd-1 ../pom.xml diff --git a/pom.xml b/pom.xml index d541f72e168c3..37cf566c365a9 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project parent - 0.7.0-csd-1-SNAPSHOT + 0.7.0-csd-1 pom Spark Project Parent POM http://spark-project.org/ diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index a8597e380689a..e5c64c8e4e5b7 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-1-SNAPSHOT + 0.7.0-csd-1 ../pom.xml @@ -52,7 +52,7 @@ - + reference.conf diff --git a/repl/pom.xml b/repl/pom.xml index a51dfeaa7b378..0665b4b590669 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-1-SNAPSHOT + 0.7.0-csd-1 ../pom.xml From 1223c1f00a953d9908c09aa4ac29c928a3918b8e Mon Sep 17 00:00:00 2001 From: Thomas Dudziak Date: Thu, 3 Jan 2013 16:27:15 -0800 Subject: [PATCH 62/94] [maven-release-plugin] prepare for next development iteration --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index fee9f1742a0b4..feb86e0226883 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-1 + 0.7.0-csd-2-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 3f0e20f833bd1..ffe9c2ffe1c2a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-1 + 0.7.0-csd-2-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 1766be4581341..85ee378cdcddd 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-1 + 0.7.0-csd-2-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 37cf566c365a9..c4620ab82fbc5 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project parent - 0.7.0-csd-1 + 0.7.0-csd-2-SNAPSHOT pom Spark Project Parent POM http://spark-project.org/ diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index e5c64c8e4e5b7..cb0a2b87f2a70 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-1 + 0.7.0-csd-2-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 0665b4b590669..d1042f279ad5b 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-1 + 0.7.0-csd-2-SNAPSHOT ../pom.xml From 921adf7d2b34e495ac85c92cae600bb5062646b0 Mon Sep 17 00:00:00 2001 From: Mikhail Bautin Date: Tue, 22 Jan 2013 16:38:05 -0800 Subject: [PATCH 63/94] Update streaming version and Kafka dependency --- streaming/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/streaming/pom.xml b/streaming/pom.xml index 4dc9a19d51fc1..511d76b88062d 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -3,8 +3,8 @@ 4.0.0 org.spark-project - spark-parent - 0.8.0-SNAPSHOT + parent + 0.7.0-csd-2-SNAPSHOT ../pom.xml From 61affcb0bdac04e9bfb972253c3977eafcba3c0f Mon Sep 17 00:00:00 2001 From: Mikhail Bautin Date: Tue, 22 Jan 2013 17:31:11 -0800 Subject: [PATCH 64/94] Add an Avro dependency to REPL to make it compile with Hadoop 2 --- repl/pom.xml | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/repl/pom.xml b/repl/pom.xml index d1042f279ad5b..189eab15ff272 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -231,6 +231,16 @@ avro-ipc provided + + org.apache.avro + avro + provided + + + org.apache.avro + avro-ipc + provided + From 2da06aa81807f1403f62e9d0c13c3b50f199b708 Mon Sep 17 00:00:00 2001 From: mbautin Date: Tue, 22 Jan 2013 18:43:26 -0800 Subject: [PATCH 65/94] [maven-release-plugin] prepare release parent-0.7.0-csd-2 --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index feb86e0226883..c6f151cf61062 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-2-SNAPSHOT + 0.7.0-csd-2 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index ffe9c2ffe1c2a..ae799f7d3ebe2 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-2-SNAPSHOT + 0.7.0-csd-2 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 85ee378cdcddd..4b4b72dfa905d 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-2-SNAPSHOT + 0.7.0-csd-2 ../pom.xml diff --git a/pom.xml b/pom.xml index c4620ab82fbc5..c78b1468355ce 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project parent - 0.7.0-csd-2-SNAPSHOT + 0.7.0-csd-2 pom Spark Project Parent POM http://spark-project.org/ diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index cb0a2b87f2a70..e97c91fe9d64b 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-2-SNAPSHOT + 0.7.0-csd-2 ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 189eab15ff272..66c82535e48ae 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-2-SNAPSHOT + 0.7.0-csd-2 ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 511d76b88062d..1445f43262a90 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-2-SNAPSHOT + 0.7.0-csd-2 ../pom.xml From dbfd14dc843c4b9efcd5160c086261b1773abd8c Mon Sep 17 00:00:00 2001 From: mbautin Date: Tue, 22 Jan 2013 18:43:33 -0800 Subject: [PATCH 66/94] [maven-release-plugin] prepare for next development iteration --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index c6f151cf61062..65eae2d474b18 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-2 + 0.7.0-csd-3-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index ae799f7d3ebe2..8e3d779839800 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-2 + 0.7.0-csd-3-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 4b4b72dfa905d..96cc742f227bc 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-2 + 0.7.0-csd-3-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index c78b1468355ce..3cd6dc6455865 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project parent - 0.7.0-csd-2 + 0.7.0-csd-3-SNAPSHOT pom Spark Project Parent POM http://spark-project.org/ diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index e97c91fe9d64b..287703d24c8c9 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-2 + 0.7.0-csd-3-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 66c82535e48ae..76191edc19ee4 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-2 + 0.7.0-csd-3-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 1445f43262a90..132951c9adafa 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-2 + 0.7.0-csd-3-SNAPSHOT ../pom.xml From a92fe5d1f4d074fa192afa1162614eab8bfe203e Mon Sep 17 00:00:00 2001 From: Jon Hartlaub Date: Wed, 23 Jan 2013 10:39:52 -0800 Subject: [PATCH 67/94] Fixed memory leak in DAGScheduler- needs more tests under error conditions. --- .../scala/spark/scheduler/DAGScheduler.scala | 30 ++++-- .../test/scala/spark/DAGSchedulerSuite.scala | 94 +++++++++++++++++++ 2 files changed, 118 insertions(+), 6 deletions(-) create mode 100644 core/src/test/scala/spark/DAGSchedulerSuite.scala diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index f7d60be5dbfc2..bdbeb70c25bb1 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -74,11 +74,13 @@ class DAGScheduler( val nextRunId = new AtomicInteger(0) + val runIdToStageIds = new HashMap[Int, HashSet[Int]] + val nextStageId = new AtomicInteger(0) - val idToStage = new TimeStampedHashMap[Int, Stage] + val idToStage = new HashMap[Int, Stage] - val shuffleToMapStage = new TimeStampedHashMap[Int, Stage] + val shuffleToMapStage = new HashMap[Int, Stage] private[spark] val stageToInfos = new TimeStampedHashMap[Stage, StageInfo] @@ -99,14 +101,12 @@ class DAGScheduler( val waiting = new HashSet[Stage] // Stages we need to run whose parents aren't done val running = new HashSet[Stage] // Stages we are running right now val failed = new HashSet[Stage] // Stages that must be resubmitted due to fetch failures - val pendingTasks = new TimeStampedHashMap[Stage, HashSet[Task[_]]] // Missing tasks from each stage + val pendingTasks = new HashMap[Stage, HashSet[Task[_]]] // Missing tasks from each stage var lastFetchFailureTime: Long = 0 // Used to wait a bit to avoid repeated resubmits val activeJobs = new HashSet[ActiveJob] val resultStageToJob = new HashMap[Stage, ActiveJob] - val metadataCleaner = new MetadataCleaner("DAGScheduler", this.cleanup) - // Start a thread to run the DAGScheduler event loop def start() { new Thread("DAGScheduler") { @@ -161,6 +161,8 @@ class DAGScheduler( val stage = new Stage(id, rdd, shuffleDep, getParentStages(rdd, priority), priority) idToStage(id) = stage stageToInfos(stage) = StageInfo(stage) + val stageIdSet = runIdToStageIds.getOrElseUpdate(priority, new HashSet) + stageIdSet += id stage } @@ -260,6 +262,7 @@ class DAGScheduler( logInfo("Failed to run " + callSite) throw exception } + r } def runApproximateJob[T, U, R]( @@ -528,6 +531,7 @@ class DAGScheduler( resultStageToJob -= stage markStageAsFinished(stage) sparkListeners.foreach(_.onJobEnd(SparkListenerJobEnd(job, JobSucceeded))) + removeStages(job) } job.listener.taskSucceeded(rt.outputId, event.result) } @@ -673,6 +677,8 @@ class DAGScheduler( sparkListeners.foreach(_.onJobEnd(SparkListenerJobEnd(job, JobFailed(error)))) activeJobs -= job resultStageToJob -= resultStage + removeStages(job) + job.listener.jobFailed(new SparkException("Job failed: " + reason)) } if (dependentStages.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") @@ -749,9 +755,21 @@ class DAGScheduler( logInfo("pendingTasks " + sizeBefore + " --> " + pendingTasks.size) } + def removeStages(job: ActiveJob) = { + runIdToStageIds(job.runId).foreach(stageId => { + idToStage.get(stageId).map( stage => { + pendingTasks -= stage + waiting -= stage + running -= stage + failed -= stage + }) + idToStage -= stageId + }) + runIdToStageIds -= job.runId + } + def stop() { eventQueue.put(StopDAGScheduler) - metadataCleaner.cancel() taskSched.stop() } } diff --git a/core/src/test/scala/spark/DAGSchedulerSuite.scala b/core/src/test/scala/spark/DAGSchedulerSuite.scala new file mode 100644 index 0000000000000..4af17f2ef7afd --- /dev/null +++ b/core/src/test/scala/spark/DAGSchedulerSuite.scala @@ -0,0 +1,94 @@ +/* +* Copyright (c) Clear Story Data, Inc. All Rights Reserved. +* +* Please see the COPYRIGHT file in the root of this repository for more +* details. +*/ +package spark + +import org.scalatest.FunSuite +import scheduler.{DAGScheduler, TaskSchedulerListener, TaskSet, TaskScheduler} +import collection.mutable + +class TaskSchedulerMock(f: (Int) => TaskEndReason ) extends TaskScheduler { + // Listener object to pass upcalls into + var listener: TaskSchedulerListener = null + var taskCount = 0 + + override def start(): Unit = {} + + // Disconnect from the cluster. + override def stop(): Unit = {} + + // Submit a sequence of tasks to run. + override def submitTasks(taskSet: TaskSet): Unit = { + taskSet.tasks.foreach( task => { + val m = new mutable.HashMap[Long, Any]() + m.put(task.stageId, 1) + taskCount += 1 + listener.taskEnded(task, f(taskCount), 1, m) + }) + } + + // Set a listener for upcalls. This is guaranteed to be set before submitTasks is called. + override def setListener(listener: TaskSchedulerListener) { + this.listener = listener + } + + // Get the default level of parallelism to use in the cluster, as a hint for sizing jobs. + override def defaultParallelism(): Int = { + 2 + } +} + +class DAGSchedulerSuite extends FunSuite { + def assertDagSchedulerEmpty(dagScheduler: DAGScheduler) = { + assert(dagScheduler.pendingTasks.isEmpty) + assert(dagScheduler.activeJobs.isEmpty) + assert(dagScheduler.failed.isEmpty) + assert(dagScheduler.runIdToStageIds.isEmpty) + assert(dagScheduler.idToStage.isEmpty) + assert(dagScheduler.resultStageToJob.isEmpty) + assert(dagScheduler.running.isEmpty) + assert(dagScheduler.shuffleToMapStage.isEmpty) + assert(dagScheduler.waiting.isEmpty) + } + + test("oneGoodJob") { + val sc = new SparkContext("local", "test") + val dagScheduler = new DAGScheduler(new TaskSchedulerMock(count => Success)) + try { + val rdd = new ParallelCollection(sc, 1.to(100).toSeq, 5) + val func = (tc: TaskContext, iter: Iterator[Int]) => 1 + val callSite = Utils.getSparkCallSite + + val result = dagScheduler.runJob(rdd, func, 0 until rdd.splits.size, callSite, false) + assertDagSchedulerEmpty(dagScheduler) + } finally { + dagScheduler.stop() + sc.stop() + // pause to let dagScheduler stop (separate thread) + Thread.sleep(10) + } + } + + test("manyGoodJobs") { + val sc = new SparkContext("local", "test") + val dagScheduler = new DAGScheduler(new TaskSchedulerMock(count => Success)) + try { + val rdd = new ParallelCollection(sc, 1.to(100).toSeq, 5) + val func = (tc: TaskContext, iter: Iterator[Int]) => 1 + val callSite = Utils.getSparkCallSite + + 1.to(100).foreach( v => { + val result = dagScheduler.runJob(rdd, func, 0 until rdd.splits.size, callSite, false) + }) + assertDagSchedulerEmpty(dagScheduler) + } finally { + dagScheduler.stop() + sc.stop() + // pause to let dagScheduler stop (separate thread) + Thread.sleep(10) + } + } +} From 8677580cdab81efd78e9cb37ca2a6a659f6a0e29 Mon Sep 17 00:00:00 2001 From: Jon Hartlaub Date: Wed, 23 Jan 2013 14:40:43 -0800 Subject: [PATCH 68/94] Fixed a merge issue. --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 9 ++++++--- core/src/test/scala/spark/DAGSchedulerSuite.scala | 4 ++-- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index bdbeb70c25bb1..ac70c0f34139b 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -78,9 +78,9 @@ class DAGScheduler( val nextStageId = new AtomicInteger(0) - val idToStage = new HashMap[Int, Stage] + val idToStage = new TimeStampedHashMap[Int, Stage] - val shuffleToMapStage = new HashMap[Int, Stage] + val shuffleToMapStage = new TimeStampedHashMap[Int, Stage] private[spark] val stageToInfos = new TimeStampedHashMap[Stage, StageInfo] @@ -101,12 +101,14 @@ class DAGScheduler( val waiting = new HashSet[Stage] // Stages we need to run whose parents aren't done val running = new HashSet[Stage] // Stages we are running right now val failed = new HashSet[Stage] // Stages that must be resubmitted due to fetch failures - val pendingTasks = new HashMap[Stage, HashSet[Task[_]]] // Missing tasks from each stage + val pendingTasks = new TimeStampedHashMap[Stage, HashSet[Task[_]]] // Missing tasks from each stage var lastFetchFailureTime: Long = 0 // Used to wait a bit to avoid repeated resubmits val activeJobs = new HashSet[ActiveJob] val resultStageToJob = new HashMap[Stage, ActiveJob] + val metadataCleaner = new MetadataCleaner("DAGScheduler", this.cleanup) + // Start a thread to run the DAGScheduler event loop def start() { new Thread("DAGScheduler") { @@ -770,6 +772,7 @@ class DAGScheduler( def stop() { eventQueue.put(StopDAGScheduler) + metadataCleaner.cancel() taskSched.stop() } } diff --git a/core/src/test/scala/spark/DAGSchedulerSuite.scala b/core/src/test/scala/spark/DAGSchedulerSuite.scala index 4af17f2ef7afd..99fd83e347f87 100644 --- a/core/src/test/scala/spark/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/DAGSchedulerSuite.scala @@ -58,7 +58,7 @@ class DAGSchedulerSuite extends FunSuite { val sc = new SparkContext("local", "test") val dagScheduler = new DAGScheduler(new TaskSchedulerMock(count => Success)) try { - val rdd = new ParallelCollection(sc, 1.to(100).toSeq, 5) + val rdd = new ParallelCollection(sc, 1.to(100).toSeq, 5, Map.empty) val func = (tc: TaskContext, iter: Iterator[Int]) => 1 val callSite = Utils.getSparkCallSite @@ -76,7 +76,7 @@ class DAGSchedulerSuite extends FunSuite { val sc = new SparkContext("local", "test") val dagScheduler = new DAGScheduler(new TaskSchedulerMock(count => Success)) try { - val rdd = new ParallelCollection(sc, 1.to(100).toSeq, 5) + val rdd = new ParallelCollection(sc, 1.to(100).toSeq, 5, Map.empty) val func = (tc: TaskContext, iter: Iterator[Int]) => 1 val callSite = Utils.getSparkCallSite From 027730e786e0ce76a8ea4669241b0a3d11390840 Mon Sep 17 00:00:00 2001 From: Jon Hartlaub Date: Thu, 24 Jan 2013 15:23:29 -0800 Subject: [PATCH 69/94] Removed copyright --- core/src/test/scala/spark/DAGSchedulerSuite.scala | 6 ------ 1 file changed, 6 deletions(-) diff --git a/core/src/test/scala/spark/DAGSchedulerSuite.scala b/core/src/test/scala/spark/DAGSchedulerSuite.scala index 99fd83e347f87..2a3b30ae429b3 100644 --- a/core/src/test/scala/spark/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/DAGSchedulerSuite.scala @@ -1,9 +1,3 @@ -/* -* Copyright (c) Clear Story Data, Inc. All Rights Reserved. -* -* Please see the COPYRIGHT file in the root of this repository for more -* details. -*/ package spark import org.scalatest.FunSuite From 0fff70db37019e338de873c90c03858b28676ae8 Mon Sep 17 00:00:00 2001 From: Mikhail Bautin Date: Thu, 24 Jan 2013 15:16:53 -0800 Subject: [PATCH 70/94] Remove activation of the hadoop1 profile by default --- bagel/pom.xml | 1 + core/pom.xml | 1 + examples/pom.xml | 1 + repl-bin/pom.xml | 1 + repl/pom.xml | 1 + streaming/pom.xml | 1 + 6 files changed, 6 insertions(+) diff --git a/bagel/pom.xml b/bagel/pom.xml index 65eae2d474b18..ef32cddac340a 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -45,6 +45,7 @@ hadoop1 + org.spark-project diff --git a/core/pom.xml b/core/pom.xml index 8e3d779839800..833ec589931c2 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -181,6 +181,7 @@ hadoop1 + org.apache.hadoop diff --git a/examples/pom.xml b/examples/pom.xml index 96cc742f227bc..dc9c1fe39b93e 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -84,6 +84,7 @@ hadoop1 + org.spark-project diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 287703d24c8c9..9001a03a77cf4 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -70,6 +70,7 @@ hadoop1 + hadoop1 diff --git a/repl/pom.xml b/repl/pom.xml index 76191edc19ee4..33f818d9e1cba 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -72,6 +72,7 @@ hadoop1 + hadoop1 diff --git a/streaming/pom.xml b/streaming/pom.xml index 132951c9adafa..d103abbf0010f 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -98,6 +98,7 @@ hadoop1 + org.spark-project From 9915d293d2f857c0ff18e959a30bc5986b813cac Mon Sep 17 00:00:00 2001 From: Mikhail Bautin Date: Thu, 24 Jan 2013 17:54:23 -0800 Subject: [PATCH 71/94] Remove hadoop2 activation as well --- bagel/pom.xml | 1 + core/pom.xml | 1 + examples/pom.xml | 1 + repl-bin/pom.xml | 1 + repl/pom.xml | 1 + streaming/pom.xml | 1 + 6 files changed, 6 insertions(+) diff --git a/bagel/pom.xml b/bagel/pom.xml index ef32cddac340a..2e14e8b701f92 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -73,6 +73,7 @@ hadoop2 + org.spark-project diff --git a/core/pom.xml b/core/pom.xml index 833ec589931c2..1114b851e7e15 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -234,6 +234,7 @@ hadoop2 + org.apache.hadoop diff --git a/examples/pom.xml b/examples/pom.xml index dc9c1fe39b93e..c61506a73ecd1 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -123,6 +123,7 @@ hadoop2 + org.spark-project diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 9001a03a77cf4..dce92c19f2ccd 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -111,6 +111,7 @@ hadoop2 + hadoop2 diff --git a/repl/pom.xml b/repl/pom.xml index 33f818d9e1cba..32b05000be377 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -117,6 +117,7 @@ hadoop2 + hadoop2 diff --git a/streaming/pom.xml b/streaming/pom.xml index d103abbf0010f..bb5f5a2632cd7 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -126,6 +126,7 @@ hadoop2 + org.spark-project From 976aca05246ae8a40d1705a0837a5d45d9349ae3 Mon Sep 17 00:00:00 2001 From: mbautin Date: Thu, 24 Jan 2013 18:01:24 -0800 Subject: [PATCH 72/94] [maven-release-plugin] prepare release parent-0.7.0-csd-3 --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index 2e14e8b701f92..149d1feda461b 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-3-SNAPSHOT + 0.7.0-csd-3 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 1114b851e7e15..7512dee80482d 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-3-SNAPSHOT + 0.7.0-csd-3 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index c61506a73ecd1..ca078df5d494c 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-3-SNAPSHOT + 0.7.0-csd-3 ../pom.xml diff --git a/pom.xml b/pom.xml index 3cd6dc6455865..f25f8eb233f3c 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project parent - 0.7.0-csd-3-SNAPSHOT + 0.7.0-csd-3 pom Spark Project Parent POM http://spark-project.org/ diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index dce92c19f2ccd..4481c5b7dc5bf 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-3-SNAPSHOT + 0.7.0-csd-3 ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 32b05000be377..18650536a5843 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-3-SNAPSHOT + 0.7.0-csd-3 ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index bb5f5a2632cd7..c699ca774f055 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-3-SNAPSHOT + 0.7.0-csd-3 ../pom.xml From f160a975714037a1a5d6145dee2fa6195c7056e1 Mon Sep 17 00:00:00 2001 From: mbautin Date: Thu, 24 Jan 2013 18:01:32 -0800 Subject: [PATCH 73/94] [maven-release-plugin] prepare for next development iteration --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index 149d1feda461b..5ba3941786eea 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-3 + 0.7.0-csd-4-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 7512dee80482d..3497739d117cd 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-3 + 0.7.0-csd-4-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index ca078df5d494c..3b2ffeb8570f7 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-3 + 0.7.0-csd-4-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index f25f8eb233f3c..fdbefecf900fa 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project parent - 0.7.0-csd-3 + 0.7.0-csd-4-SNAPSHOT pom Spark Project Parent POM http://spark-project.org/ diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 4481c5b7dc5bf..77a137e40a130 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-3 + 0.7.0-csd-4-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 18650536a5843..f441ef4b5ce25 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-3 + 0.7.0-csd-4-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index c699ca774f055..bb80dc37c44bf 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project parent - 0.7.0-csd-3 + 0.7.0-csd-4-SNAPSHOT ../pom.xml From 2ab6742cabd6179a9f6d5be47c8add1cae5e2001 Mon Sep 17 00:00:00 2001 From: Mikhail Bautin Date: Fri, 15 Mar 2013 11:41:51 -0700 Subject: [PATCH 74/94] Add a log4j compile dependency to fix build in IntelliJ Also rename parent project to spark-parent (otherwise it shows up as "parent" in IntelliJ, which is very confusing). --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index 5ba3941786eea..b01488b000bbd 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project - parent + spark-parent 0.7.0-csd-4-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 3497739d117cd..284fcae328b7a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project - parent + spark-parent 0.7.0-csd-4-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 3b2ffeb8570f7..5018b2de33e7f 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project - parent + spark-parent 0.7.0-csd-4-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index fdbefecf900fa..4653c1d057a99 100644 --- a/pom.xml +++ b/pom.xml @@ -2,7 +2,7 @@ 4.0.0 org.spark-project - parent + spark-parent 0.7.0-csd-4-SNAPSHOT pom Spark Project Parent POM diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 77a137e40a130..9ccc729fee139 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project - parent + spark-parent 0.7.0-csd-4-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index f441ef4b5ce25..bcbd70efdccbd 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project - parent + spark-parent 0.7.0-csd-4-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index bb80dc37c44bf..4cea28b8a4637 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project - parent + spark-parent 0.7.0-csd-4-SNAPSHOT ../pom.xml From 6adc746c6e4c99a46460c5ebdc5014205165f815 Mon Sep 17 00:00:00 2001 From: Jon Hartlaub Date: Wed, 23 Jan 2013 10:39:52 -0800 Subject: [PATCH 75/94] Fixed memory leak in DAGScheduler- needs more tests under error conditions. --- .../scala/spark/scheduler/DAGScheduler.scala | 6 +- .../test/scala/spark/DAGSchedulerSuite.scala | 88 ------------------- 2 files changed, 1 insertion(+), 93 deletions(-) delete mode 100644 core/src/test/scala/spark/DAGSchedulerSuite.scala diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index ac70c0f34139b..e5ecc899c485c 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -383,11 +383,7 @@ class DAGScheduler( logDebug("Got event of type " + event.getClass.getName) } - if (event != null) { - if (processEvent(event)) { - return - } - } + if (event != null && processEvent(event)) return val time = System.currentTimeMillis() // TODO: use a pluggable clock for testability // Periodically resubmit failed stages if some map output fetches have failed and we have diff --git a/core/src/test/scala/spark/DAGSchedulerSuite.scala b/core/src/test/scala/spark/DAGSchedulerSuite.scala deleted file mode 100644 index 2a3b30ae429b3..0000000000000 --- a/core/src/test/scala/spark/DAGSchedulerSuite.scala +++ /dev/null @@ -1,88 +0,0 @@ -package spark - -import org.scalatest.FunSuite -import scheduler.{DAGScheduler, TaskSchedulerListener, TaskSet, TaskScheduler} -import collection.mutable - -class TaskSchedulerMock(f: (Int) => TaskEndReason ) extends TaskScheduler { - // Listener object to pass upcalls into - var listener: TaskSchedulerListener = null - var taskCount = 0 - - override def start(): Unit = {} - - // Disconnect from the cluster. - override def stop(): Unit = {} - - // Submit a sequence of tasks to run. - override def submitTasks(taskSet: TaskSet): Unit = { - taskSet.tasks.foreach( task => { - val m = new mutable.HashMap[Long, Any]() - m.put(task.stageId, 1) - taskCount += 1 - listener.taskEnded(task, f(taskCount), 1, m) - }) - } - - // Set a listener for upcalls. This is guaranteed to be set before submitTasks is called. - override def setListener(listener: TaskSchedulerListener) { - this.listener = listener - } - - // Get the default level of parallelism to use in the cluster, as a hint for sizing jobs. - override def defaultParallelism(): Int = { - 2 - } -} - -class DAGSchedulerSuite extends FunSuite { - def assertDagSchedulerEmpty(dagScheduler: DAGScheduler) = { - assert(dagScheduler.pendingTasks.isEmpty) - assert(dagScheduler.activeJobs.isEmpty) - assert(dagScheduler.failed.isEmpty) - assert(dagScheduler.runIdToStageIds.isEmpty) - assert(dagScheduler.idToStage.isEmpty) - assert(dagScheduler.resultStageToJob.isEmpty) - assert(dagScheduler.running.isEmpty) - assert(dagScheduler.shuffleToMapStage.isEmpty) - assert(dagScheduler.waiting.isEmpty) - } - - test("oneGoodJob") { - val sc = new SparkContext("local", "test") - val dagScheduler = new DAGScheduler(new TaskSchedulerMock(count => Success)) - try { - val rdd = new ParallelCollection(sc, 1.to(100).toSeq, 5, Map.empty) - val func = (tc: TaskContext, iter: Iterator[Int]) => 1 - val callSite = Utils.getSparkCallSite - - val result = dagScheduler.runJob(rdd, func, 0 until rdd.splits.size, callSite, false) - assertDagSchedulerEmpty(dagScheduler) - } finally { - dagScheduler.stop() - sc.stop() - // pause to let dagScheduler stop (separate thread) - Thread.sleep(10) - } - } - - test("manyGoodJobs") { - val sc = new SparkContext("local", "test") - val dagScheduler = new DAGScheduler(new TaskSchedulerMock(count => Success)) - try { - val rdd = new ParallelCollection(sc, 1.to(100).toSeq, 5, Map.empty) - val func = (tc: TaskContext, iter: Iterator[Int]) => 1 - val callSite = Utils.getSparkCallSite - - 1.to(100).foreach( v => { - val result = dagScheduler.runJob(rdd, func, 0 until rdd.splits.size, callSite, false) - }) - assertDagSchedulerEmpty(dagScheduler) - } finally { - dagScheduler.stop() - sc.stop() - // pause to let dagScheduler stop (separate thread) - Thread.sleep(10) - } - } -} From 0796048928b824a848b9671340142c07a7995141 Mon Sep 17 00:00:00 2001 From: Thomas Dudziak Date: Thu, 3 Jan 2013 16:09:10 -0800 Subject: [PATCH 76/94] CSD version numbering and distribution management --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index b01488b000bbd..17729a57e9be6 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.7.0-csd-4-SNAPSHOT + 0.8.0-candidate-csd-1-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 284fcae328b7a..16c1a0f6656aa 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.7.0-csd-4-SNAPSHOT + 0.8.0-candidate-csd-1-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 5018b2de33e7f..239ec8f60c98d 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.7.0-csd-4-SNAPSHOT + 0.8.0-candidate-csd-1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 4653c1d057a99..9bd3a69da04c1 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project spark-parent - 0.7.0-csd-4-SNAPSHOT + 0.8.0-candidate-csd-1-SNAPSHOT pom Spark Project Parent POM http://spark-project.org/ diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 9ccc729fee139..a8ba3f10b3840 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.7.0-csd-4-SNAPSHOT + 0.8.0-candidate-csd-1-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index bcbd70efdccbd..a37f28351782c 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.7.0-csd-4-SNAPSHOT + 0.8.0-candidate-csd-1-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 4cea28b8a4637..be6bc0c6cf964 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.7.0-csd-4-SNAPSHOT + 0.8.0-candidate-csd-1-SNAPSHOT ../pom.xml From 59eb09810b0a787e69d196f030824c6b7446c7e7 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 12 Apr 2013 15:37:13 -0700 Subject: [PATCH 77/94] Bump some dependencies, pin others, for Scala 2.9.2 --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 9bd3a69da04c1..57a6689d64742 100644 --- a/pom.xml +++ b/pom.xml @@ -162,7 +162,7 @@ org.eclipse.jetty jetty-server - 7.5.3.v20111011 + 7.6.8.v20121106 com.google.guava @@ -309,7 +309,7 @@ com.novocode junit-interface - 0.8 + 0.9 test From ff4fa9011991b34965add3a83b66833045164bf9 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 12 Apr 2013 16:07:47 -0700 Subject: [PATCH 78/94] wip: Bringing Jon's DAGScheduler tests into the new DAGSchedulerSuite --- .../spark/scheduler/DAGSchedulerSuite.scala | 81 ++++++++++++++++++- 1 file changed, 80 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index 30e6fef9505c7..e475976f9f342 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -14,12 +14,13 @@ import spark.{Dependency, ShuffleDependency, OneToOneDependency} import spark.FetchFailedException import spark.MapOutputTracker import spark.RDD +import spark.rdd.ParallelCollectionRDD import spark.SparkContext import spark.SparkException import spark.Partition import spark.TaskContext import spark.TaskEndReason - +import spark.Utils import spark.{FetchFailed, Success} /** @@ -385,6 +386,42 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont assert(results === Map(0 -> 42)) } + // test("oneGoodJob") { + // val dagScheduler = new DAGScheduler(new TaskSchedulerMock(count => Success)) + // try { + // val rdd = new ParallelCollectionRDD(sc, 1.to(100).toSeq, 5, Map.empty) + // val func = (tc: TaskContext, iter: Iterator[Int]) => 1 + // val callSite = Utils.getSparkCallSite + + // dagScheduler.runJob(rdd, func, 0 until rdd.partitions.size, callSite, false, {(_: Int, _: Int) => {}}) + // assertDagSchedulerEmpty(dagScheduler) + // } finally { + // dagScheduler.stop() + // sc.stop() + // // pause to let dagScheduler stop (separate thread) + // Thread.sleep(10) + // } + // } + + // test("manyGoodJobs") { + // val dagScheduler = new DAGScheduler(new TaskSchedulerMock(count => Success)) + // try { + // val rdd = new ParallelCollectionRDD(sc, 1.to(100).toSeq, 5, Map.empty) + // val func = (tc: TaskContext, iter: Iterator[Int]) => 1 + // val callSite = Utils.getSparkCallSite + + // 1.to(100).foreach( v => { + // dagScheduler.runJob(rdd, func, 0 until rdd.partitions.size, callSite, false, {(_: Int, _: Int) => {}}) + // }) + // assertDagSchedulerEmpty(dagScheduler) + // } finally { + // dagScheduler.stop() + // sc.stop() + // // pause to let dagScheduler stop (separate thread) + // Thread.sleep(10) + // } + // } + /** Assert that the supplied TaskSet has exactly the given preferredLocations. Note, converts taskSet's locations to host only. */ private def assertLocations(taskSet: TaskSet, locations: Seq[Seq[String]]) { assert(locations.size === taskSet.tasks.size) @@ -400,4 +437,46 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont private def makeBlockManagerId(host: String): BlockManagerId = BlockManagerId("exec-" + host, host, 12345, 0) + private def assertDagSchedulerEmpty(dagScheduler: DAGScheduler) = { + assert(dagScheduler.pendingTasks.isEmpty) + assert(dagScheduler.activeJobs.isEmpty) + assert(dagScheduler.failed.isEmpty) + assert(dagScheduler.runIdToStageIds.isEmpty) + assert(dagScheduler.idToStage.isEmpty) + assert(dagScheduler.resultStageToJob.isEmpty) + assert(dagScheduler.running.isEmpty) + assert(dagScheduler.shuffleToMapStage.isEmpty) + assert(dagScheduler.waiting.isEmpty) + } +} + +class TaskSchedulerMock(f: (Int) => TaskEndReason ) extends TaskScheduler { + // Listener object to pass upcalls into + var listener: TaskSchedulerListener = null + var taskCount = 0 + + override def start(): Unit = {} + + // Disconnect from the cluster. + override def stop(): Unit = {} + + // Submit a sequence of tasks to run. + override def submitTasks(taskSet: TaskSet): Unit = { + taskSet.tasks.foreach( task => { + val m = new scala.collection.mutable.HashMap[Long, Any]() + m.put(task.stageId, 1) + taskCount += 1 + listener.taskEnded(task, f(taskCount), 1, m, null, task.metrics.getOrElse(null)) // TODO: TaskInfo + }) + } + + // Set a listener for upcalls. This is guaranteed to be set before submitTasks is called. + override def setListener(listener: TaskSchedulerListener) { + this.listener = listener + } + + // Get the default level of parallelism to use in the cluster, as a hint for sizing jobs. + override def defaultParallelism(): Int = { + 2 + } } From f443225a403a1591c1a5f54b896fa9dd6a355147 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 12 Apr 2013 16:22:34 -0700 Subject: [PATCH 79/94] [maven-release-plugin] prepare release spark-parent-0.8.0-candidate-csd-1 --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index 17729a57e9be6..26e12f56feb5c 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-1-SNAPSHOT + 0.8.0-candidate-csd-1 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 16c1a0f6656aa..129368ca759f8 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-1-SNAPSHOT + 0.8.0-candidate-csd-1 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 239ec8f60c98d..00b7dfed5dfc2 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-1-SNAPSHOT + 0.8.0-candidate-csd-1 ../pom.xml diff --git a/pom.xml b/pom.xml index 57a6689d64742..0439da29995c8 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project spark-parent - 0.8.0-candidate-csd-1-SNAPSHOT + 0.8.0-candidate-csd-1 pom Spark Project Parent POM http://spark-project.org/ diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index a8ba3f10b3840..580e5e045d55e 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-1-SNAPSHOT + 0.8.0-candidate-csd-1 ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index a37f28351782c..91055cd25560a 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-1-SNAPSHOT + 0.8.0-candidate-csd-1 ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index be6bc0c6cf964..3a7e0253cf13d 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-1-SNAPSHOT + 0.8.0-candidate-csd-1 ../pom.xml From d1de6ef85dc4eb306395eb9622653efd2c9773af Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 12 Apr 2013 16:22:42 -0700 Subject: [PATCH 80/94] [maven-release-plugin] prepare for next development iteration --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index 26e12f56feb5c..fe61c8d5772a0 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-1 + 0.8.0-candidate-csd-2-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 129368ca759f8..bb22e33bc66d9 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-1 + 0.8.0-candidate-csd-2-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 00b7dfed5dfc2..0e4f52c173a6c 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-1 + 0.8.0-candidate-csd-2-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 0439da29995c8..72adf7397e30c 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project spark-parent - 0.8.0-candidate-csd-1 + 0.8.0-candidate-csd-2-SNAPSHOT pom Spark Project Parent POM http://spark-project.org/ diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 580e5e045d55e..b5972e8bd3f52 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-1 + 0.8.0-candidate-csd-2-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 91055cd25560a..8a106baa78a90 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-1 + 0.8.0-candidate-csd-2-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 3a7e0253cf13d..55fdfeb6cd629 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-1 + 0.8.0-candidate-csd-2-SNAPSHOT ../pom.xml From 701762c171adddf6937d6b0b66d74cd32a0af15e Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 17 May 2013 15:13:52 -0700 Subject: [PATCH 81/94] Specified commons-io version for spark-core --- core/pom.xml | 4 ++++ pom.xml | 10 ++++++++++ 2 files changed, 14 insertions(+) diff --git a/core/pom.xml b/core/pom.xml index bb22e33bc66d9..47becb7e538e2 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -15,6 +15,10 @@ http://spark-project.org/ + + commons-io + commons-io + org.eclipse.jetty jetty-server diff --git a/pom.xml b/pom.xml index 72adf7397e30c..b0e752516fef0 100644 --- a/pom.xml +++ b/pom.xml @@ -159,6 +159,11 @@ + + commons-io + commons-io + 2.4 + org.eclipse.jetty jetty-server @@ -184,6 +189,11 @@ jul-to-slf4j ${slf4j.version} + + commons-daemon + commons-daemon + 1.0.10 + com.ning compress-lzf From 4069c17d96db639533c18c6507a28a2e8963f133 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Wed, 22 May 2013 15:14:45 -0700 Subject: [PATCH 82/94] [maven-release-plugin] prepare release spark-parent-0.8.0-candidate-csd-2 --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 8 insertions(+), 8 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index fe61c8d5772a0..d0d20bb8c5599 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-2-SNAPSHOT + 0.8.0-candidate-csd-2 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 47becb7e538e2..c6d413a44cd54 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-2-SNAPSHOT + 0.8.0-candidate-csd-2 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 0e4f52c173a6c..42a40b32f9072 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-2-SNAPSHOT + 0.8.0-candidate-csd-2 ../pom.xml diff --git a/pom.xml b/pom.xml index b0e752516fef0..dd83f7d320099 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project spark-parent - 0.8.0-candidate-csd-2-SNAPSHOT + 0.8.0-candidate-csd-2 pom Spark Project Parent POM http://spark-project.org/ @@ -445,7 +445,7 @@ . ${project.build.directory}/SparkTestSuite.txt -Xms64m -Xmx1024m - + diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index b5972e8bd3f52..801cf9e4ec2a9 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-2-SNAPSHOT + 0.8.0-candidate-csd-2 ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 8a106baa78a90..8283405e24979 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-2-SNAPSHOT + 0.8.0-candidate-csd-2 ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 55fdfeb6cd629..130f2ba4f8d90 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-2-SNAPSHOT + 0.8.0-candidate-csd-2 ../pom.xml From e4cf0e32a043c6e2deccf661c01a8861f1985122 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Wed, 22 May 2013 15:18:41 -0700 Subject: [PATCH 83/94] [maven-release-plugin] prepare for next development iteration --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index d0d20bb8c5599..85de58137d700 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-2 + 0.8.0-candidate-csd-3-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index c6d413a44cd54..1341b4da48f1c 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-2 + 0.8.0-candidate-csd-3-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 42a40b32f9072..15fcb948985b8 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-2 + 0.8.0-candidate-csd-3-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index dd83f7d320099..e7568f442b21a 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project spark-parent - 0.8.0-candidate-csd-2 + 0.8.0-candidate-csd-3-SNAPSHOT pom Spark Project Parent POM http://spark-project.org/ diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 801cf9e4ec2a9..39a0b5255f181 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-2 + 0.8.0-candidate-csd-3-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 8283405e24979..a5bec44ab16ef 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-2 + 0.8.0-candidate-csd-3-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 130f2ba4f8d90..d37a01e8afcb4 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-2 + 0.8.0-candidate-csd-3-SNAPSHOT ../pom.xml From 47a029446cefa1a40c1bbd4e7fe803acc93186a6 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Tue, 28 May 2013 10:54:41 -0700 Subject: [PATCH 84/94] [maven-release-plugin] prepare release spark-parent-0.8.0-candidate-csd-3 --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index 85de58137d700..0e51e5f8ac449 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-3-SNAPSHOT + 0.8.0-candidate-csd-3 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 1341b4da48f1c..08f6c967eedbf 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-3-SNAPSHOT + 0.8.0-candidate-csd-3 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 15fcb948985b8..de032a44939fe 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-3-SNAPSHOT + 0.8.0-candidate-csd-3 ../pom.xml diff --git a/pom.xml b/pom.xml index e7568f442b21a..60ce245b11ff1 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project spark-parent - 0.8.0-candidate-csd-3-SNAPSHOT + 0.8.0-candidate-csd-3 pom Spark Project Parent POM http://spark-project.org/ diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 39a0b5255f181..2600d1e7619f9 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-3-SNAPSHOT + 0.8.0-candidate-csd-3 ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index a5bec44ab16ef..31e4e42eb8021 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-3-SNAPSHOT + 0.8.0-candidate-csd-3 ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index d37a01e8afcb4..011d3dfee6c81 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-3-SNAPSHOT + 0.8.0-candidate-csd-3 ../pom.xml From ecde46685357038a011b866f0cee0eeb318f4a4d Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Tue, 28 May 2013 10:54:47 -0700 Subject: [PATCH 85/94] [maven-release-plugin] prepare for next development iteration --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index 0e51e5f8ac449..a8c55ed27205d 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-3 + 0.8.0-candidate-csd-4-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 08f6c967eedbf..61b3f2f27c1b6 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-3 + 0.8.0-candidate-csd-4-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index de032a44939fe..ca0d4639de4b1 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-3 + 0.8.0-candidate-csd-4-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 60ce245b11ff1..f68123e31086e 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project spark-parent - 0.8.0-candidate-csd-3 + 0.8.0-candidate-csd-4-SNAPSHOT pom Spark Project Parent POM http://spark-project.org/ diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 2600d1e7619f9..4c6387e680b17 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-3 + 0.8.0-candidate-csd-4-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 31e4e42eb8021..4459db0661959 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-3 + 0.8.0-candidate-csd-4-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 011d3dfee6c81..f932213c09b00 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-3 + 0.8.0-candidate-csd-4-SNAPSHOT ../pom.xml From c2cb18f3e9ef2de0a2b86744d502589b042a3e28 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Thu, 30 May 2013 10:15:09 -0700 Subject: [PATCH 86/94] [maven-release-plugin] prepare release spark-parent-0.8.0-candidate-csd-4 --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index a8c55ed27205d..b5dd7e938ee9c 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-4-SNAPSHOT + 0.8.0-candidate-csd-4 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 61b3f2f27c1b6..54840533c2eb1 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-4-SNAPSHOT + 0.8.0-candidate-csd-4 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index ca0d4639de4b1..86a7b543d5c5e 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-4-SNAPSHOT + 0.8.0-candidate-csd-4 ../pom.xml diff --git a/pom.xml b/pom.xml index f68123e31086e..ab329fdd5267b 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project spark-parent - 0.8.0-candidate-csd-4-SNAPSHOT + 0.8.0-candidate-csd-4 pom Spark Project Parent POM http://spark-project.org/ diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 4c6387e680b17..ddd019e301028 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-4-SNAPSHOT + 0.8.0-candidate-csd-4 ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 4459db0661959..3bce5dfed5b8f 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-4-SNAPSHOT + 0.8.0-candidate-csd-4 ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index f932213c09b00..897d248373361 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-4-SNAPSHOT + 0.8.0-candidate-csd-4 ../pom.xml From e6d1041d9f5a1d491b728ece5183546e2c6efb28 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Thu, 30 May 2013 10:15:15 -0700 Subject: [PATCH 87/94] [maven-release-plugin] prepare for next development iteration --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index b5dd7e938ee9c..8ed56d352a1ca 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-4 + 0.8.0-candidate-csd-5-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 54840533c2eb1..7db58f70ccc78 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-4 + 0.8.0-candidate-csd-5-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 86a7b543d5c5e..b83b1698a27be 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-4 + 0.8.0-candidate-csd-5-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index ab329fdd5267b..b3f60c6fabc11 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project spark-parent - 0.8.0-candidate-csd-4 + 0.8.0-candidate-csd-5-SNAPSHOT pom Spark Project Parent POM http://spark-project.org/ diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index ddd019e301028..c3e451054a816 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-4 + 0.8.0-candidate-csd-5-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 3bce5dfed5b8f..3c09fcbbe0a74 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-4 + 0.8.0-candidate-csd-5-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 897d248373361..10c48efcac5c8 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-4 + 0.8.0-candidate-csd-5-SNAPSHOT ../pom.xml From 1762e97132fb9dd0573eed542d9fb230f5cc5c79 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Thu, 30 May 2013 17:21:04 -0700 Subject: [PATCH 88/94] [maven-release-plugin] prepare release spark-parent-0.8.0-candidate-csd-5 --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index 8ed56d352a1ca..c95c8cf188071 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-5-SNAPSHOT + 0.8.0-candidate-csd-5 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 7db58f70ccc78..0a368e3816e1c 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-5-SNAPSHOT + 0.8.0-candidate-csd-5 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index b83b1698a27be..54ea828e2b1d8 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-5-SNAPSHOT + 0.8.0-candidate-csd-5 ../pom.xml diff --git a/pom.xml b/pom.xml index b3f60c6fabc11..cc23bbb742645 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project spark-parent - 0.8.0-candidate-csd-5-SNAPSHOT + 0.8.0-candidate-csd-5 pom Spark Project Parent POM http://spark-project.org/ diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index c3e451054a816..6316b1b66a19e 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-5-SNAPSHOT + 0.8.0-candidate-csd-5 ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 3c09fcbbe0a74..e2a40b34c2bf0 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-5-SNAPSHOT + 0.8.0-candidate-csd-5 ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 10c48efcac5c8..309ed4b2b5a31 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-5-SNAPSHOT + 0.8.0-candidate-csd-5 ../pom.xml From e25914e9a00e0c617023b5f4c789fc39fc6bf65f Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Thu, 30 May 2013 17:21:11 -0700 Subject: [PATCH 89/94] [maven-release-plugin] prepare for next development iteration --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index c95c8cf188071..64ee1084b2c96 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-5 + 0.8.0-candidate-csd-6-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 0a368e3816e1c..89b27880a3269 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-5 + 0.8.0-candidate-csd-6-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 54ea828e2b1d8..ccdd7f69d3dc8 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-5 + 0.8.0-candidate-csd-6-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index cc23bbb742645..e907353b4c2b7 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project spark-parent - 0.8.0-candidate-csd-5 + 0.8.0-candidate-csd-6-SNAPSHOT pom Spark Project Parent POM http://spark-project.org/ diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 6316b1b66a19e..86a72e5a20674 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-5 + 0.8.0-candidate-csd-6-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index e2a40b34c2bf0..0337b680f4efd 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-5 + 0.8.0-candidate-csd-6-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 309ed4b2b5a31..a00eeb09049fb 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-5 + 0.8.0-candidate-csd-6-SNAPSHOT ../pom.xml From 5005a599fa6bcc84463e50b55e82faa45bcf7905 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Wed, 12 Jun 2013 16:01:20 -0700 Subject: [PATCH 90/94] Removed commons-io dependency from core --- core/pom.xml | 4 ---- 1 file changed, 4 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 89b27880a3269..5f93a7f131839 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -15,10 +15,6 @@ http://spark-project.org/ - - commons-io - commons-io - org.eclipse.jetty jetty-server From 39baf7689a07e4079672b01c0812150cc50133d0 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 14 Jun 2013 11:51:48 -0700 Subject: [PATCH 91/94] [maven-release-plugin] prepare release spark-parent-0.8.0-candidate-csd-6 --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index 64ee1084b2c96..b69b1bc761849 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-6-SNAPSHOT + 0.8.0-candidate-csd-6 ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 5f93a7f131839..026d86f86e769 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-6-SNAPSHOT + 0.8.0-candidate-csd-6 ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index ccdd7f69d3dc8..306e25fafc802 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-6-SNAPSHOT + 0.8.0-candidate-csd-6 ../pom.xml diff --git a/pom.xml b/pom.xml index e907353b4c2b7..141e9f70fa596 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project spark-parent - 0.8.0-candidate-csd-6-SNAPSHOT + 0.8.0-candidate-csd-6 pom Spark Project Parent POM http://spark-project.org/ diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 86a72e5a20674..85310149b1565 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-6-SNAPSHOT + 0.8.0-candidate-csd-6 ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 0337b680f4efd..dbf2037918a89 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-6-SNAPSHOT + 0.8.0-candidate-csd-6 ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index a00eeb09049fb..55d6984387583 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-6-SNAPSHOT + 0.8.0-candidate-csd-6 ../pom.xml From 3d8bc99353fc6a8489b663e071419fdeeea9e636 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 14 Jun 2013 11:51:55 -0700 Subject: [PATCH 92/94] [maven-release-plugin] prepare for next development iteration --- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- pom.xml | 2 +- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index b69b1bc761849..188efb588e188 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-6 + 0.8.0-candidate-csd-7-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 026d86f86e769..3935464160306 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-6 + 0.8.0-candidate-csd-7-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 306e25fafc802..4ca367003d4ee 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-6 + 0.8.0-candidate-csd-7-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 141e9f70fa596..a0d017daa6b76 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ 4.0.0 org.spark-project spark-parent - 0.8.0-candidate-csd-6 + 0.8.0-candidate-csd-7-SNAPSHOT pom Spark Project Parent POM http://spark-project.org/ diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 85310149b1565..95700435a0da7 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-6 + 0.8.0-candidate-csd-7-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index dbf2037918a89..517a92885ec48 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-6 + 0.8.0-candidate-csd-7-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 55d6984387583..58a553cd5f9ce 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -4,7 +4,7 @@ org.spark-project spark-parent - 0.8.0-candidate-csd-6 + 0.8.0-candidate-csd-7-SNAPSHOT ../pom.xml From 5c119324a0bc540d5746c7e712933c623fc3ae66 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Sat, 22 Jun 2013 13:01:52 -0700 Subject: [PATCH 93/94] Removed redundant job.listener.jobFailed notification --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index e5ecc899c485c..1c91fd0aed2c1 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -676,7 +676,6 @@ class DAGScheduler( activeJobs -= job resultStageToJob -= resultStage removeStages(job) - job.listener.jobFailed(new SparkException("Job failed: " + reason)) } if (dependentStages.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") From 4e5860db44df778ce561918601c6e0282c72a28e Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 28 Jun 2013 18:28:12 -0700 Subject: [PATCH 94/94] Don't try to return a value from DAGScheduler.runJob --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 1c91fd0aed2c1..f9c880558eb15 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -264,7 +264,6 @@ class DAGScheduler( logInfo("Failed to run " + callSite) throw exception } - r } def runApproximateJob[T, U, R](
Property NameDefaultMeaning
spark.executor.memory512m + Amount of memory to use per executor process, in the same format as JVM memory strings (e.g. `512m`, `2g`). +
spark.serializer spark.JavaSerializer