From 644e31524a6a9a22c671a368aeb3b4eaeb61cf29 Mon Sep 17 00:00:00 2001 From: Prudhvi Krishna Date: Tue, 2 Sep 2014 17:36:53 -0700 Subject: [PATCH 01/39] SPARK-3328 fixed make-distribution script --with-tachyon option. Directory path for dependencies jar and resources in Tachyon 0.5.0 has been changed. Author: Prudhvi Krishna Closes #2228 from prudhvije/SPARK-3328/make-dist-fix and squashes the following commits: d1d2c22 [Prudhvi Krishna] SPARK-3328 fixed make-distribution script --with-tachyon option. --- make-distribution.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/make-distribution.sh b/make-distribution.sh index ee1399071112d..f030d3f430581 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -219,10 +219,10 @@ if [ "$SPARK_TACHYON" == "true" ]; then wget "$TACHYON_URL" tar xf "tachyon-${TACHYON_VERSION}-bin.tar.gz" - cp "tachyon-${TACHYON_VERSION}/target/tachyon-${TACHYON_VERSION}-jar-with-dependencies.jar" "$DISTDIR/lib" + cp "tachyon-${TACHYON_VERSION}/core/target/tachyon-${TACHYON_VERSION}-jar-with-dependencies.jar" "$DISTDIR/lib" mkdir -p "$DISTDIR/tachyon/src/main/java/tachyon/web" cp -r "tachyon-${TACHYON_VERSION}"/{bin,conf,libexec} "$DISTDIR/tachyon" - cp -r "tachyon-${TACHYON_VERSION}"/src/main/java/tachyon/web/resources "$DISTDIR/tachyon/src/main/java/tachyon/web" + cp -r "tachyon-${TACHYON_VERSION}"/core/src/main/java/tachyon/web "$DISTDIR/tachyon/src/main/java/tachyon/web" if [[ `uname -a` == Darwin* ]]; then # need to run sed differently on osx From 7c92b49d6b62f88fcde883aacb60c5e32ae54b30 Mon Sep 17 00:00:00 2001 From: Larry Xiao Date: Tue, 2 Sep 2014 18:29:08 -0700 Subject: [PATCH 02/39] [SPARK-1986][GraphX]move lib.Analytics to org.apache.spark.examples to support ~/spark/bin/run-example GraphXAnalytics triangles /soc-LiveJournal1.txt --numEPart=256 Author: Larry Xiao Closes #1766 from larryxiao/1986 and squashes the following commits: bb77cd9 [Larry Xiao] [SPARK-1986][GraphX]move lib.Analytics to org.apache.spark.examples --- .../scala/org/apache/spark/examples/graphx}/Analytics.scala | 3 ++- .../org/apache/spark/examples/graphx/LiveJournalPageRank.scala | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) rename {graphx/src/main/scala/org/apache/spark/graphx/lib => examples/src/main/scala/org/apache/spark/examples/graphx}/Analytics.scala (98%) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala similarity index 98% rename from graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala rename to examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala index c1513a00453cf..c4317a6aec798 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala @@ -15,12 +15,13 @@ * limitations under the License. */ -package org.apache.spark.graphx.lib +package org.apache.spark.examples.graphx import scala.collection.mutable import org.apache.spark._ import org.apache.spark.storage.StorageLevel import org.apache.spark.graphx._ +import org.apache.spark.graphx.lib._ import org.apache.spark.graphx.PartitionStrategy._ /** diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala index 6ef3b62dcbedc..bdc8fa7f99f2e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala @@ -20,7 +20,7 @@ package org.apache.spark.examples.graphx import org.apache.spark.SparkContext._ import org.apache.spark._ import org.apache.spark.graphx._ -import org.apache.spark.graphx.lib.Analytics +import org.apache.spark.examples.graphx.Analytics /** * Uses GraphX to run PageRank on a LiveJournal social network graph. Download the dataset from From 7c9bbf172512701c75992671bcb2f4b6d9e5034b Mon Sep 17 00:00:00 2001 From: uncleGen Date: Tue, 2 Sep 2014 18:41:54 -0700 Subject: [PATCH 03/39] [SPARK-3123][GraphX]: override the "setName" function to set EdgeRDD's name manually just as VertexRDD does. Author: uncleGen Closes #2033 from uncleGen/master_origin and squashes the following commits: 801994b [uncleGen] Update EdgeRDD.scala --- .../main/scala/org/apache/spark/graphx/EdgeRDD.scala | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala index 899a3cbd62b60..5bcb96b136ed7 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala @@ -37,7 +37,15 @@ class EdgeRDD[@specialized ED: ClassTag, VD: ClassTag]( val targetStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY) extends RDD[Edge[ED]](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { - partitionsRDD.setName("EdgeRDD") + override def setName(_name: String): this.type = { + if (partitionsRDD.name != null) { + partitionsRDD.setName(partitionsRDD.name + ", " + _name) + } else { + partitionsRDD.setName(_name) + } + this + } + setName("EdgeRDD") override protected def getPartitions: Array[Partition] = partitionsRDD.partitions From aa7de128c5987fd2e134736f07ae913ad1f5eb26 Mon Sep 17 00:00:00 2001 From: Larry Xiao Date: Tue, 2 Sep 2014 18:50:52 -0700 Subject: [PATCH 04/39] [SPARK-2981][GraphX] EdgePartition1D Int overflow minor fix detail is here: https://issues.apache.org/jira/browse/SPARK-2981 Author: Larry Xiao Closes #1902 from larryxiao/2981 and squashes the following commits: 88059a2 [Larry Xiao] [SPARK-2981][GraphX] EdgePartition1D Int overflow --- .../main/scala/org/apache/spark/graphx/PartitionStrategy.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala index 5e7e72a764cc8..13033fee0e6b5 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala @@ -91,7 +91,7 @@ object PartitionStrategy { case object EdgePartition1D extends PartitionStrategy { override def getPartition(src: VertexId, dst: VertexId, numParts: PartitionID): PartitionID = { val mixingPrime: VertexId = 1125899906842597L - (math.abs(src) * mixingPrime).toInt % numParts + (math.abs(src * mixingPrime) % numParts).toInt } } From e9bb12bea9fbef94332fbec88e3cd9197a27b7ad Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 2 Sep 2014 19:02:48 -0700 Subject: [PATCH 05/39] [SPARK-1981][Streaming][Hotfix] Fixed docs related to kinesis - Include kinesis in the unidocs - Hide non-public classes from docs Author: Tathagata Das Closes #2239 from tdas/kinesis-doc-fix and squashes the following commits: 156e20c [Tathagata Das] More fixes, based on PR comments. e9a6c01 [Tathagata Das] Fixed docs related to kinesis --- docs/_plugins/copy_api_dirs.rb | 4 ++-- .../examples/streaming/JavaKinesisWordCountASL.java | 10 +++++----- .../spark/examples/streaming/KinesisWordCountASL.scala | 6 +++--- .../apache/spark/streaming/kinesis/KinesisUtils.scala | 7 ++++--- project/SparkBuild.scala | 6 +++--- 5 files changed, 17 insertions(+), 16 deletions(-) diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index 2dbbbf6feb4b8..3b02e090aec28 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -25,8 +25,8 @@ curr_dir = pwd cd("..") - puts "Running 'sbt/sbt compile unidoc' from " + pwd + "; this may take a few minutes..." - puts `sbt/sbt compile unidoc` + puts "Running 'sbt/sbt -Pkinesis-asl compile unidoc' from " + pwd + "; this may take a few minutes..." + puts `sbt/sbt -Pkinesis-asl compile unidoc` puts "Moving back into docs dir." cd("docs") diff --git a/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java b/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java index 1a710d7b18c6f..aa917d0575c4c 100644 --- a/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java +++ b/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java @@ -75,7 +75,7 @@ * onto the Kinesis stream. * Usage instructions for KinesisWordCountProducerASL are provided in the class definition. */ -public final class JavaKinesisWordCountASL { +public final class JavaKinesisWordCountASL { // needs to be public for access from run-example private static final Pattern WORD_SEPARATOR = Pattern.compile(" "); private static final Logger logger = Logger.getLogger(JavaKinesisWordCountASL.class); @@ -87,10 +87,10 @@ public static void main(String[] args) { /* Check that all required args were passed in. */ if (args.length < 2) { System.err.println( - "|Usage: KinesisWordCount \n" + - "| is the name of the Kinesis stream\n" + - "| is the endpoint of the Kinesis service\n" + - "| (e.g. https://kinesis.us-east-1.amazonaws.com)\n"); + "Usage: JavaKinesisWordCountASL \n" + + " is the name of the Kinesis stream\n" + + " is the endpoint of the Kinesis service\n" + + " (e.g. https://kinesis.us-east-1.amazonaws.com)\n"); System.exit(1); } diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala index d03edf8b30a9f..fffd90de08240 100644 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala @@ -69,7 +69,7 @@ import org.apache.log4j.Level * dummy data onto the Kinesis stream. * Usage instructions for KinesisWordCountProducerASL are provided in that class definition. */ -object KinesisWordCountASL extends Logging { +private object KinesisWordCountASL extends Logging { def main(args: Array[String]) { /* Check that all required args were passed in. */ if (args.length < 2) { @@ -154,7 +154,7 @@ object KinesisWordCountASL extends Logging { * org.apache.spark.examples.streaming.KinesisWordCountProducerASL mySparkStream \ * https://kinesis.us-east-1.amazonaws.com 10 5 */ -object KinesisWordCountProducerASL { +private object KinesisWordCountProducerASL { def main(args: Array[String]) { if (args.length < 4) { System.err.println("Usage: KinesisWordCountProducerASL " + @@ -235,7 +235,7 @@ object KinesisWordCountProducerASL { * Utility functions for Spark Streaming examples. * This has been lifted from the examples/ project to remove the circular dependency. */ -object StreamingExamples extends Logging { +private[streaming] object StreamingExamples extends Logging { /** Set reasonable logging levels for streaming if the user has not configured log4j. */ def setStreamingLogLevels() { diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala index 713cac0e293c0..96f4399accd3a 100644 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala @@ -35,7 +35,7 @@ import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionIn object KinesisUtils { /** * Create an InputDStream that pulls messages from a Kinesis stream. - * + * :: Experimental :: * @param ssc StreamingContext object * @param streamName Kinesis stream name * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com) @@ -52,6 +52,7 @@ object KinesisUtils { * * @return ReceiverInputDStream[Array[Byte]] */ + @Experimental def createStream( ssc: StreamingContext, streamName: String, @@ -65,9 +66,8 @@ object KinesisUtils { /** * Create a Java-friendly InputDStream that pulls messages from a Kinesis stream. - * + * :: Experimental :: * @param jssc Java StreamingContext object - * @param ssc StreamingContext object * @param streamName Kinesis stream name * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com) * @param checkpointInterval Checkpoint interval for Kinesis checkpointing. @@ -83,6 +83,7 @@ object KinesisUtils { * * @return JavaReceiverInputDStream[Array[Byte]] */ + @Experimental def createStream( jssc: JavaStreamingContext, streamName: String, diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 4c696d3d385fb..a26c2c90cb321 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -290,9 +290,9 @@ object Unidoc { publish := {}, unidocProjectFilter in(ScalaUnidoc, unidoc) := - inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, catalyst, yarn, yarnAlpha), + inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, catalyst, streamingFlumeSink, yarn, yarnAlpha), unidocProjectFilter in(JavaUnidoc, unidoc) := - inAnyProject -- inProjects(OldDeps.project, repl, bagel, graphx, examples, tools, catalyst, yarn, yarnAlpha), + inAnyProject -- inProjects(OldDeps.project, repl, bagel, graphx, examples, tools, catalyst, streamingFlumeSink, yarn, yarnAlpha), // Skip class names containing $ and some internal packages in Javadocs unidocAllSources in (JavaUnidoc, unidoc) := { @@ -314,7 +314,7 @@ object Unidoc { "-group", "Core Java API", packageList("api.java", "api.java.function"), "-group", "Spark Streaming", packageList( "streaming.api.java", "streaming.flume", "streaming.kafka", - "streaming.mqtt", "streaming.twitter", "streaming.zeromq" + "streaming.mqtt", "streaming.twitter", "streaming.zeromq", "streaming.kinesis" ), "-group", "MLlib", packageList( "mllib.classification", "mllib.clustering", "mllib.evaluation.binary", "mllib.linalg", From 9b225ac3072de522b40b46aba6df1f1c231f13ef Mon Sep 17 00:00:00 2001 From: luluorta Date: Tue, 2 Sep 2014 19:25:52 -0700 Subject: [PATCH 06/39] [SPARK-2823][GraphX]fix GraphX EdgeRDD zipPartitions MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit If the users set “spark.default.parallelism” and the value is different with the EdgeRDD partition number, GraphX jobs will throw: java.lang.IllegalArgumentException: Can't zip RDDs with unequal numbers of partitions Author: luluorta Closes #1763 from luluorta/fix-graph-zip and squashes the following commits: 8338961 [luluorta] fix GraphX EdgeRDD zipPartitions --- .../scala/org/apache/spark/graphx/EdgeRDD.scala | 4 ++-- .../org/apache/spark/graphx/GraphSuite.scala | 16 ++++++++++++++++ 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala index 5bcb96b136ed7..35fbd47e6c2a5 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala @@ -19,7 +19,7 @@ package org.apache.spark.graphx import scala.reflect.{classTag, ClassTag} -import org.apache.spark.{OneToOneDependency, Partition, Partitioner, TaskContext} +import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -55,7 +55,7 @@ class EdgeRDD[@specialized ED: ClassTag, VD: ClassTag]( * partitioner that allows co-partitioning with `partitionsRDD`. */ override val partitioner = - partitionsRDD.partitioner.orElse(Some(Partitioner.defaultPartitioner(partitionsRDD))) + partitionsRDD.partitioner.orElse(Some(new HashPartitioner(partitionsRDD.partitions.size))) override def compute(part: Partition, context: TaskContext): Iterator[Edge[ED]] = { val p = firstParent[(PartitionID, EdgePartition[ED, VD])].iterator(part, context) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala index 6506bac73d71c..eaaa4499b6b93 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.graphx import org.scalatest.FunSuite +import org.apache.spark.SparkConf import org.apache.spark.SparkContext import org.apache.spark.graphx.Graph._ import org.apache.spark.graphx.PartitionStrategy._ @@ -350,4 +351,19 @@ class GraphSuite extends FunSuite with LocalSparkContext { } } + test("non-default number of edge partitions") { + val n = 10 + val defaultParallelism = 3 + val numEdgePartitions = 4 + assert(defaultParallelism != numEdgePartitions) + val conf = new SparkConf() + .set("spark.default.parallelism", defaultParallelism.toString) + val sc = new SparkContext("local", "test", conf) + val edges = sc.parallelize((1 to n).map(x => (x: VertexId, 0: VertexId)), + numEdgePartitions) + val graph = Graph.fromEdgeTuples(edges, 1) + val neighborAttrSums = graph.mapReduceTriplets[Int]( + et => Iterator((et.dstId, et.srcAttr)), _ + _) + assert(neighborAttrSums.collect.toSet === Set((0: VertexId, n))) + } } From 0cd91f666dfe318934d5d41d3cb7085b2f9ae278 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Tue, 2 Sep 2014 20:31:15 -0700 Subject: [PATCH 07/39] [SPARK-3341][SQL] The dataType of Sqrt expression should be DoubleType. Author: Takuya UESHIN Closes #2233 from ueshin/issues/SPARK-3341 and squashes the following commits: e497320 [Takuya UESHIN] Fix data type of Sqrt expression. --- .../org/apache/spark/sql/catalyst/expressions/arithmetic.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 56f042891a2e6..f988fb010b107 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -36,7 +36,7 @@ case class UnaryMinus(child: Expression) extends UnaryExpression { case class Sqrt(child: Expression) extends UnaryExpression { type EvaluatedType = Any - def dataType = child.dataType + def dataType = DoubleType override def foldable = child.foldable def nullable = child.nullable override def toString = s"SQRT($child)" From 19d3e1e8e9c25c0936c0ad6efbc2092b473723aa Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 2 Sep 2014 20:49:36 -0700 Subject: [PATCH 08/39] [SQL] Renamed ColumnStat to ColumnMetrics to avoid confusion between ColumnStats Class names of these two are just too similar. Author: Cheng Lian Closes #2189 from liancheng/column-metrics and squashes the following commits: 8bb3b21 [Cheng Lian] Renamed ColumnStat to ColumnMetrics to avoid confusion between ColumnStats --- .../org/apache/spark/sql/execution/debug/package.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index 8ff757bbe3508..a9535a750bcd7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -74,22 +74,22 @@ package object debug { } /** - * A collection of stats for each column of output. + * A collection of metrics for each column of output. * @param elementTypes the actual runtime types for the output. Useful when there are bugs * causing the wrong data to be projected. */ - case class ColumnStat( + case class ColumnMetrics( elementTypes: Accumulator[HashSet[String]] = sparkContext.accumulator(HashSet.empty)) val tupleCount = sparkContext.accumulator[Int](0) val numColumns = child.output.size - val columnStats = Array.fill(child.output.size)(new ColumnStat()) + val columnStats = Array.fill(child.output.size)(new ColumnMetrics()) def dumpStats(): Unit = { println(s"== ${child.simpleString} ==") println(s"Tuples output: ${tupleCount.value}") - child.output.zip(columnStats).foreach { case(attr, stat) => - val actualDataTypes =stat.elementTypes.value.mkString("{", ",", "}") + child.output.zip(columnStats).foreach { case(attr, metric) => + val actualDataTypes = metric.elementTypes.value.mkString("{", ",", "}") println(s" ${attr.name} ${attr.dataType}: $actualDataTypes") } } From 24ab384018270e4f7af7eb8ca7192f337498eaf5 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 2 Sep 2014 20:51:25 -0700 Subject: [PATCH 09/39] [SPARK-3300][SQL] No need to call clear() and shorten build() The function `ensureFreeSpace` in object `ColumnBuilder` clears old buffer before copying its content to new buffer. This PR fixes it. Author: Liang-Chi Hsieh Closes #2195 from viirya/fix_buffer_clear and squashes the following commits: 792f009 [Liang-Chi Hsieh] no need to call clear(). use flip() instead of calling limit(), position() and rewind(). df2169f [Liang-Chi Hsieh] should clean old buffer after copying its content. --- .../scala/org/apache/spark/sql/columnar/ColumnBuilder.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index 7e7bb2859bbcd..247337a875c75 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -75,8 +75,7 @@ private[sql] class BasicColumnBuilder[T <: DataType, JvmType]( } override def build() = { - buffer.limit(buffer.position()).rewind() - buffer + buffer.flip().asInstanceOf[ByteBuffer] } } @@ -129,7 +128,6 @@ private[sql] object ColumnBuilder { val newSize = capacity + size.max(capacity / 8 + 1) val pos = orig.position() - orig.clear() ByteBuffer .allocate(newSize) .order(ByteOrder.nativeOrder()) From c64cc435e2a29c6f0ff66022fd4d5b4cb5011718 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 2 Sep 2014 21:30:09 -0700 Subject: [PATCH 10/39] SPARK-3358: [EC2] Switch back to HVM instances for m3.X. During regression tests of Spark 1.1 we discovered perf issues with PVM instances when running PySpark. This reverts a change added in #1156 which changed the default type for m3 instances to PVM. Author: Patrick Wendell Closes #2244 from pwendell/ec2-hvm and squashes the following commits: 1342d7e [Patrick Wendell] SPARK-3358: [EC2] Switch back to HVM instances for m3.X. --- ec2/spark_ec2.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 7e25df57ee45b..eed6eb8485183 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -234,10 +234,10 @@ def get_spark_ami(opts): "cg1.4xlarge": "hvm", "hs1.8xlarge": "pvm", "hi1.4xlarge": "pvm", - "m3.medium": "pvm", - "m3.large": "pvm", - "m3.xlarge": "pvm", - "m3.2xlarge": "pvm", + "m3.medium": "hvm", + "m3.large": "hvm", + "m3.xlarge": "hvm", + "m3.2xlarge": "hvm", "cr1.8xlarge": "hvm", "i2.xlarge": "hvm", "i2.2xlarge": "hvm", From 6a72a36940311fcb3429bd34c8818bc7d513115c Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 3 Sep 2014 08:22:50 -0500 Subject: [PATCH 11/39] [SPARK-3187] [yarn] Cleanup allocator code. Move all shared logic to the base YarnAllocator class, and leave the version-specific logic in the version-specific module. Author: Marcelo Vanzin Closes #2169 from vanzin/SPARK-3187 and squashes the following commits: 46c2826 [Marcelo Vanzin] Hide the privates. 4dc9c83 [Marcelo Vanzin] Actually release containers. 8b1a077 [Marcelo Vanzin] Changes to the Yarn alpha allocator. f3f5f1d [Marcelo Vanzin] [SPARK-3187] [yarn] Cleanup allocator code. --- .../deploy/yarn/YarnAllocationHandler.scala | 462 +++--------------- .../spark/deploy/yarn/YarnAllocator.scala | 425 +++++++++++++++- .../deploy/yarn/YarnAllocationHandler.scala | 402 +-------------- 3 files changed, 495 insertions(+), 794 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 629cd13f67145..9f9e16c06452b 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -17,35 +17,21 @@ package org.apache.spark.deploy.yarn -import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap} +import java.util.concurrent.CopyOnWriteArrayList import java.util.concurrent.atomic.AtomicInteger import scala.collection.JavaConversions._ -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.collection.mutable.{ArrayBuffer, HashMap} -import org.apache.spark.{Logging, SparkConf, SparkEnv} -import org.apache.spark.scheduler.{SplitInfo, TaskSchedulerImpl} -import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend -import org.apache.spark.util.Utils +import org.apache.spark.SparkConf +import org.apache.spark.scheduler.SplitInfo import org.apache.hadoop.conf.Configuration import org.apache.hadoop.yarn.api.AMRMProtocol -import org.apache.hadoop.yarn.api.records.{AMResponse, ApplicationAttemptId} -import org.apache.hadoop.yarn.api.records.{Container, ContainerId} -import org.apache.hadoop.yarn.api.records.{Priority, Resource, ResourceRequest} -import org.apache.hadoop.yarn.api.protocolrecords.{AllocateRequest, AllocateResponse} +import org.apache.hadoop.yarn.api.records._ +import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest import org.apache.hadoop.yarn.util.Records -// TODO: -// Too many params. -// Needs to be mt-safe -// Need to refactor this to make it 'cleaner' ... right now, all computation is reactive - should -// make it more proactive and decoupled. - -// Note that right now, we assume all node asks as uniform in terms of capabilities and priority -// Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for -// more info on how we are requesting for containers. - /** * Acquires resources for executors from a ResourceManager and launches executors in new containers. */ @@ -56,357 +42,20 @@ private[yarn] class YarnAllocationHandler( appAttemptId: ApplicationAttemptId, args: ApplicationMasterArguments, preferredNodes: collection.Map[String, collection.Set[SplitInfo]]) - extends YarnAllocator with Logging { - - // These three are locked on allocatedHostToContainersMap. Complementary data structures - // allocatedHostToContainersMap : containers which are running : host, Set - // allocatedContainerToHostMap: container to host mapping. - private val allocatedHostToContainersMap = - new HashMap[String, collection.mutable.Set[ContainerId]]() - - private val allocatedContainerToHostMap = new HashMap[ContainerId, String]() - - // allocatedRackCount is populated ONLY if allocation happens (or decremented if this is an - // allocated node) - // As with the two data structures above, tightly coupled with them, and to be locked on - // allocatedHostToContainersMap - private val allocatedRackCount = new HashMap[String, Int]() - - // Containers which have been released. - private val releasedContainerList = new CopyOnWriteArrayList[ContainerId]() - // Containers to be released in next request to RM - private val pendingReleaseContainers = new ConcurrentHashMap[ContainerId, Boolean] - - // Additional memory overhead - in mb. - private def memoryOverhead: Int = sparkConf.getInt("spark.yarn.executor.memoryOverhead", - YarnSparkHadoopUtil.DEFAULT_MEMORY_OVERHEAD) - - private val numExecutorsRunning = new AtomicInteger() - // Used to generate a unique id per executor - private val executorIdCounter = new AtomicInteger() - private val lastResponseId = new AtomicInteger() - private val numExecutorsFailed = new AtomicInteger() - - private val maxExecutors = args.numExecutors - private val executorMemory = args.executorMemory - private val executorCores = args.executorCores - private val (preferredHostToCount, preferredRackToCount) = - generateNodeToWeight(conf, preferredNodes) - - def getNumExecutorsRunning: Int = numExecutorsRunning.intValue - - def getNumExecutorsFailed: Int = numExecutorsFailed.intValue - - def isResourceConstraintSatisfied(container: Container): Boolean = { - container.getResource.getMemory >= (executorMemory + memoryOverhead) - } - - override def allocateResources() = { - // We need to send the request only once from what I understand ... but for now, not modifying - // this much. - val executorsToRequest = Math.max(maxExecutors - numExecutorsRunning.get(), 0) - - // Keep polling the Resource Manager for containers - val amResp = allocateExecutorResources(executorsToRequest).getAMResponse - - val _allocatedContainers = amResp.getAllocatedContainers() - - if (_allocatedContainers.size > 0) { - logDebug(""" - Allocated containers: %d - Current executor count: %d - Containers released: %s - Containers to be released: %s - Cluster resources: %s - """.format( - _allocatedContainers.size, - numExecutorsRunning.get(), - releasedContainerList, - pendingReleaseContainers, - amResp.getAvailableResources)) - - val hostToContainers = new HashMap[String, ArrayBuffer[Container]]() - - // Ignore if not satisfying constraints { - for (container <- _allocatedContainers) { - if (isResourceConstraintSatisfied(container)) { - // allocatedContainers += container - - val host = container.getNodeId.getHost - val containers = hostToContainers.getOrElseUpdate(host, new ArrayBuffer[Container]()) - - containers += container - } else { - // Add all ignored containers to released list - releasedContainerList.add(container.getId()) - } - } - - // Find the appropriate containers to use. Slightly non trivial groupBy ... - val dataLocalContainers = new HashMap[String, ArrayBuffer[Container]]() - val rackLocalContainers = new HashMap[String, ArrayBuffer[Container]]() - val offRackContainers = new HashMap[String, ArrayBuffer[Container]]() - - for (candidateHost <- hostToContainers.keySet) - { - val maxExpectedHostCount = preferredHostToCount.getOrElse(candidateHost, 0) - val requiredHostCount = maxExpectedHostCount - allocatedContainersOnHost(candidateHost) - - var remainingContainers = hostToContainers.get(candidateHost).getOrElse(null) - assert(remainingContainers != null) - - if (requiredHostCount >= remainingContainers.size){ - // Since we got <= required containers, add all to dataLocalContainers - dataLocalContainers.put(candidateHost, remainingContainers) - // all consumed - remainingContainers = null - } else if (requiredHostCount > 0) { - // Container list has more containers than we need for data locality. - // Split into two : data local container count of (remainingContainers.size - - // requiredHostCount) and rest as remainingContainer - val (dataLocal, remaining) = remainingContainers.splitAt( - remainingContainers.size - requiredHostCount) - dataLocalContainers.put(candidateHost, dataLocal) - // remainingContainers = remaining - - // yarn has nasty habit of allocating a tonne of containers on a host - discourage this : - // add remaining to release list. If we have insufficient containers, next allocation - // cycle will reallocate (but wont treat it as data local) - for (container <- remaining) releasedContainerList.add(container.getId()) - remainingContainers = null - } - - // Now rack local - if (remainingContainers != null){ - val rack = YarnSparkHadoopUtil.lookupRack(conf, candidateHost) - - if (rack != null){ - val maxExpectedRackCount = preferredRackToCount.getOrElse(rack, 0) - val requiredRackCount = maxExpectedRackCount - allocatedContainersOnRack(rack) - - rackLocalContainers.get(rack).getOrElse(List()).size - - - if (requiredRackCount >= remainingContainers.size){ - // Add all to dataLocalContainers - dataLocalContainers.put(rack, remainingContainers) - // All consumed - remainingContainers = null - } else if (requiredRackCount > 0) { - // container list has more containers than we need for data locality. - // Split into two : data local container count of (remainingContainers.size - - // requiredRackCount) and rest as remainingContainer - val (rackLocal, remaining) = remainingContainers.splitAt( - remainingContainers.size - requiredRackCount) - val existingRackLocal = rackLocalContainers.getOrElseUpdate(rack, - new ArrayBuffer[Container]()) - - existingRackLocal ++= rackLocal - remainingContainers = remaining - } - } - } - - // If still not consumed, then it is off rack host - add to that list. - if (remainingContainers != null){ - offRackContainers.put(candidateHost, remainingContainers) - } - } - - // Now that we have split the containers into various groups, go through them in order : - // first host local, then rack local and then off rack (everything else). - // Note that the list we create below tries to ensure that not all containers end up within a - // host if there are sufficiently large number of hosts/containers. - - val allocatedContainers = new ArrayBuffer[Container](_allocatedContainers.size) - allocatedContainers ++= TaskSchedulerImpl.prioritizeContainers(dataLocalContainers) - allocatedContainers ++= TaskSchedulerImpl.prioritizeContainers(rackLocalContainers) - allocatedContainers ++= TaskSchedulerImpl.prioritizeContainers(offRackContainers) - - // Run each of the allocated containers - for (container <- allocatedContainers) { - val numExecutorsRunningNow = numExecutorsRunning.incrementAndGet() - val executorHostname = container.getNodeId.getHost - val containerId = container.getId - - assert( container.getResource.getMemory >= - (executorMemory + memoryOverhead)) - - if (numExecutorsRunningNow > maxExecutors) { - logInfo("""Ignoring container %s at host %s, since we already have the required number of - containers for it.""".format(containerId, executorHostname)) - releasedContainerList.add(containerId) - // reset counter back to old value. - numExecutorsRunning.decrementAndGet() - } else { - // Deallocate + allocate can result in reusing id's wrongly - so use a different counter - // (executorIdCounter) - val executorId = executorIdCounter.incrementAndGet().toString - val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( - SparkEnv.driverActorSystemName, - sparkConf.get("spark.driver.host"), - sparkConf.get("spark.driver.port"), - CoarseGrainedSchedulerBackend.ACTOR_NAME) - - logInfo("launching container on " + containerId + " host " + executorHostname) - // Just to be safe, simply remove it from pendingReleaseContainers. - // Should not be there, but .. - pendingReleaseContainers.remove(containerId) - - val rack = YarnSparkHadoopUtil.lookupRack(conf, executorHostname) - allocatedHostToContainersMap.synchronized { - val containerSet = allocatedHostToContainersMap.getOrElseUpdate(executorHostname, - new HashSet[ContainerId]()) - - containerSet += containerId - allocatedContainerToHostMap.put(containerId, executorHostname) - if (rack != null) { - allocatedRackCount.put(rack, allocatedRackCount.getOrElse(rack, 0) + 1) - } - } - - new Thread( - new ExecutorRunnable(container, conf, sparkConf, driverUrl, executorId, - executorHostname, executorMemory, executorCores) - ).start() - } - } - logDebug(""" - Finished processing %d containers. - Current number of executors running: %d, - releasedContainerList: %s, - pendingReleaseContainers: %s - """.format( - allocatedContainers.size, - numExecutorsRunning.get(), - releasedContainerList, - pendingReleaseContainers)) - } - - - val completedContainers = amResp.getCompletedContainersStatuses() - if (completedContainers.size > 0){ - logDebug("Completed %d containers, to-be-released: %s".format( - completedContainers.size, releasedContainerList)) - for (completedContainer <- completedContainers){ - val containerId = completedContainer.getContainerId - - // Was this released by us ? If yes, then simply remove from containerSet and move on. - if (pendingReleaseContainers.containsKey(containerId)) { - pendingReleaseContainers.remove(containerId) - } else { - // Simply decrement count - next iteration of ReporterThread will take care of allocating. - numExecutorsRunning.decrementAndGet() - logInfo("Completed container %s (state: %s, exit status: %s)".format( - containerId, - completedContainer.getState, - completedContainer.getExitStatus())) - // Hadoop 2.2.X added a ContainerExitStatus we should switch to use - // there are some exit status' we shouldn't necessarily count against us, but for - // now I think its ok as none of the containers are expected to exit - if (completedContainer.getExitStatus() != 0) { - logInfo("Container marked as failed: " + containerId) - numExecutorsFailed.incrementAndGet() - } - } - - allocatedHostToContainersMap.synchronized { - if (allocatedContainerToHostMap.containsKey(containerId)) { - val host = allocatedContainerToHostMap.get(containerId).getOrElse(null) - assert (host != null) - - val containerSet = allocatedHostToContainersMap.get(host).getOrElse(null) - assert (containerSet != null) - - containerSet -= containerId - if (containerSet.isEmpty) { - allocatedHostToContainersMap.remove(host) - } else { - allocatedHostToContainersMap.update(host, containerSet) - } - - allocatedContainerToHostMap -= containerId - - // Doing this within locked context, sigh ... move to outside ? - val rack = YarnSparkHadoopUtil.lookupRack(conf, host) - if (rack != null) { - val rackCount = allocatedRackCount.getOrElse(rack, 0) - 1 - if (rackCount > 0) { - allocatedRackCount.put(rack, rackCount) - } else { - allocatedRackCount.remove(rack) - } - } - } - } - } - logDebug(""" - Finished processing %d completed containers. - Current number of executors running: %d, - releasedContainerList: %s, - pendingReleaseContainers: %s - """.format( - completedContainers.size, - numExecutorsRunning.get(), - releasedContainerList, - pendingReleaseContainers)) - } - } - - def createRackResourceRequests(hostContainers: List[ResourceRequest]): List[ResourceRequest] = { - // First generate modified racks and new set of hosts under it : then issue requests - val rackToCounts = new HashMap[String, Int]() - - // Within this lock - used to read/write to the rack related maps too. - for (container <- hostContainers) { - val candidateHost = container.getHostName - val candidateNumContainers = container.getNumContainers - assert(YarnSparkHadoopUtil.ANY_HOST != candidateHost) - - val rack = YarnSparkHadoopUtil.lookupRack(conf, candidateHost) - if (rack != null) { - var count = rackToCounts.getOrElse(rack, 0) - count += candidateNumContainers - rackToCounts.put(rack, count) - } - } - - val requestedContainers: ArrayBuffer[ResourceRequest] = - new ArrayBuffer[ResourceRequest](rackToCounts.size) - for ((rack, count) <- rackToCounts){ - requestedContainers += - createResourceRequest(AllocationType.RACK, rack, count, - YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) - } - - requestedContainers.toList - } - - def allocatedContainersOnHost(host: String): Int = { - var retval = 0 - allocatedHostToContainersMap.synchronized { - retval = allocatedHostToContainersMap.getOrElse(host, Set()).size - } - retval - } + extends YarnAllocator(conf, sparkConf, args, preferredNodes) { - def allocatedContainersOnRack(rack: String): Int = { - var retval = 0 - allocatedHostToContainersMap.synchronized { - retval = allocatedRackCount.getOrElse(rack, 0) - } - retval - } - - private def allocateExecutorResources(numExecutors: Int): AllocateResponse = { + private val lastResponseId = new AtomicInteger() + private val releaseList: CopyOnWriteArrayList[ContainerId] = new CopyOnWriteArrayList() + override protected def allocateContainers(count: Int): YarnAllocateResponse = { var resourceRequests: List[ResourceRequest] = null - // default. - if (numExecutors <= 0 || preferredHostToCount.isEmpty) { - logDebug("numExecutors: " + numExecutors + ", host preferences: " + + // default. + if (count <= 0 || preferredHostToCount.isEmpty) { + logDebug("numExecutors: " + count + ", host preferences: " + preferredHostToCount.isEmpty) resourceRequests = List(createResourceRequest( - AllocationType.ANY, null, numExecutors, YarnSparkHadoopUtil.RM_REQUEST_PRIORITY)) + AllocationType.ANY, null, count, YarnSparkHadoopUtil.RM_REQUEST_PRIORITY)) } else { // request for all hosts in preferred nodes and for numExecutors - // candidates.size, request by default allocation policy. @@ -429,7 +78,7 @@ private[yarn] class YarnAllocationHandler( val anyContainerRequests: ResourceRequest = createResourceRequest( AllocationType.ANY, resource = null, - numExecutors, + count, YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) val containerRequests: ArrayBuffer[ResourceRequest] = new ArrayBuffer[ResourceRequest]( @@ -451,8 +100,8 @@ private[yarn] class YarnAllocationHandler( val releasedContainerList = createReleasedContainerList() req.addAllReleases(releasedContainerList) - if (numExecutors > 0) { - logInfo("Allocating %d executor containers with %d of memory each.".format(numExecutors, + if (count > 0) { + logInfo("Allocating %d executor containers with %d of memory each.".format(count, executorMemory + memoryOverhead)) } else { logDebug("Empty allocation req .. release : " + releasedContainerList) @@ -466,9 +115,42 @@ private[yarn] class YarnAllocationHandler( request.getPriority, request.getCapability)) } - resourceManager.allocate(req) + new AlphaAllocateResponse(resourceManager.allocate(req).getAMResponse()) } + override protected def releaseContainer(container: Container) = { + releaseList.add(container.getId()) + } + + private def createRackResourceRequests(hostContainers: List[ResourceRequest]): + List[ResourceRequest] = { + // First generate modified racks and new set of hosts under it : then issue requests + val rackToCounts = new HashMap[String, Int]() + + // Within this lock - used to read/write to the rack related maps too. + for (container <- hostContainers) { + val candidateHost = container.getHostName + val candidateNumContainers = container.getNumContainers + assert(YarnSparkHadoopUtil.ANY_HOST != candidateHost) + + val rack = YarnSparkHadoopUtil.lookupRack(conf, candidateHost) + if (rack != null) { + var count = rackToCounts.getOrElse(rack, 0) + count += candidateNumContainers + rackToCounts.put(rack, count) + } + } + + val requestedContainers: ArrayBuffer[ResourceRequest] = + new ArrayBuffer[ResourceRequest](rackToCounts.size) + for ((rack, count) <- rackToCounts){ + requestedContainers += + createResourceRequest(AllocationType.RACK, rack, count, + YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) + } + + requestedContainers.toList + } private def createResourceRequest( requestType: AllocationType.AllocationType, @@ -521,48 +203,24 @@ private[yarn] class YarnAllocationHandler( rsrcRequest } - def createReleasedContainerList(): ArrayBuffer[ContainerId] = { - + private def createReleasedContainerList(): ArrayBuffer[ContainerId] = { val retval = new ArrayBuffer[ContainerId](1) // Iterator on COW list ... - for (container <- releasedContainerList.iterator()){ + for (container <- releaseList.iterator()){ retval += container } // Remove from the original list. - if (! retval.isEmpty) { - releasedContainerList.removeAll(retval) - for (v <- retval) pendingReleaseContainers.put(v, true) - logInfo("Releasing " + retval.size + " containers. pendingReleaseContainers : " + - pendingReleaseContainers) + if (!retval.isEmpty) { + releaseList.removeAll(retval) + logInfo("Releasing " + retval.size + " containers.") } - retval } - // A simple method to copy the split info map. - private def generateNodeToWeight( - conf: Configuration, - input: collection.Map[String, collection.Set[SplitInfo]]) : - // host to count, rack to count - (Map[String, Int], Map[String, Int]) = { - - if (input == null) return (Map[String, Int](), Map[String, Int]()) - - val hostToCount = new HashMap[String, Int] - val rackToCount = new HashMap[String, Int] - - for ((host, splits) <- input) { - val hostCount = hostToCount.getOrElse(host, 0) - hostToCount.put(host, hostCount + splits.size) - - val rack = YarnSparkHadoopUtil.lookupRack(conf, host) - if (rack != null){ - val rackCount = rackToCount.getOrElse(host, 0) - rackToCount.put(host, rackCount + splits.size) - } - } - - (hostToCount.toMap, rackToCount.toMap) + private class AlphaAllocateResponse(response: AMResponse) extends YarnAllocateResponse { + override def getAllocatedContainers() = response.getAllocatedContainers() + override def getAvailableResources() = response.getAvailableResources() + override def getCompletedContainersStatuses() = response.getCompletedContainersStatuses() } } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index cad94e5e19e1f..c74dd1c2b21dc 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -17,18 +17,431 @@ package org.apache.spark.deploy.yarn +import java.util.{List => JList} +import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.atomic.AtomicInteger + +import scala.collection.JavaConversions._ +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.yarn.api.records._ +import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse + +import org.apache.spark.{Logging, SparkConf, SparkEnv} +import org.apache.spark.scheduler.{SplitInfo, TaskSchedulerImpl} +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend + object AllocationType extends Enumeration { type AllocationType = Value val HOST, RACK, ANY = Value } +// TODO: +// Too many params. +// Needs to be mt-safe +// Need to refactor this to make it 'cleaner' ... right now, all computation is reactive - should +// make it more proactive and decoupled. + +// Note that right now, we assume all node asks as uniform in terms of capabilities and priority +// Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for +// more info on how we are requesting for containers. + /** - * Interface that defines a Yarn allocator. + * Common code for the Yarn container allocator. Contains all the version-agnostic code to + * manage container allocation for a running Spark application. */ -trait YarnAllocator { +private[yarn] abstract class YarnAllocator( + conf: Configuration, + sparkConf: SparkConf, + args: ApplicationMasterArguments, + preferredNodes: collection.Map[String, collection.Set[SplitInfo]]) + extends Logging { - def allocateResources(): Unit - def getNumExecutorsFailed: Int - def getNumExecutorsRunning: Int + // These three are locked on allocatedHostToContainersMap. Complementary data structures + // allocatedHostToContainersMap : containers which are running : host, Set + // allocatedContainerToHostMap: container to host mapping. + private val allocatedHostToContainersMap = + new HashMap[String, collection.mutable.Set[ContainerId]]() -} + private val allocatedContainerToHostMap = new HashMap[ContainerId, String]() + + // allocatedRackCount is populated ONLY if allocation happens (or decremented if this is an + // allocated node) + // As with the two data structures above, tightly coupled with them, and to be locked on + // allocatedHostToContainersMap + private val allocatedRackCount = new HashMap[String, Int]() + + // Containers to be released in next request to RM + private val releasedContainers = new ConcurrentHashMap[ContainerId, Boolean] + + // Additional memory overhead - in mb. + protected val memoryOverhead: Int = sparkConf.getInt("spark.yarn.executor.memoryOverhead", + YarnSparkHadoopUtil.DEFAULT_MEMORY_OVERHEAD) + + // Number of container requests that have been sent to, but not yet allocated by the + // ApplicationMaster. + private val numPendingAllocate = new AtomicInteger() + private val numExecutorsRunning = new AtomicInteger() + // Used to generate a unique id per executor + private val executorIdCounter = new AtomicInteger() + private val numExecutorsFailed = new AtomicInteger() + + private val maxExecutors = args.numExecutors + + protected val executorMemory = args.executorMemory + protected val executorCores = args.executorCores + protected val (preferredHostToCount, preferredRackToCount) = + generateNodeToWeight(conf, preferredNodes) + + def getNumExecutorsRunning: Int = numExecutorsRunning.intValue + + def getNumExecutorsFailed: Int = numExecutorsFailed.intValue + + def allocateResources() = { + val missing = maxExecutors - numPendingAllocate.get() - numExecutorsRunning.get() + + if (missing > 0) { + numPendingAllocate.addAndGet(missing) + logInfo("Will Allocate %d executor containers, each with %d memory".format( + missing, + (executorMemory + memoryOverhead))) + } else { + logDebug("Empty allocation request ...") + } + + val allocateResponse = allocateContainers(missing) + val allocatedContainers = allocateResponse.getAllocatedContainers() + + if (allocatedContainers.size > 0) { + var numPendingAllocateNow = numPendingAllocate.addAndGet(-1 * allocatedContainers.size) + + if (numPendingAllocateNow < 0) { + numPendingAllocateNow = numPendingAllocate.addAndGet(-1 * numPendingAllocateNow) + } + + logDebug(""" + Allocated containers: %d + Current executor count: %d + Containers released: %s + Cluster resources: %s + """.format( + allocatedContainers.size, + numExecutorsRunning.get(), + releasedContainers, + allocateResponse.getAvailableResources)) + + val hostToContainers = new HashMap[String, ArrayBuffer[Container]]() + + for (container <- allocatedContainers) { + if (isResourceConstraintSatisfied(container)) { + // Add the accepted `container` to the host's list of already accepted, + // allocated containers + val host = container.getNodeId.getHost + val containersForHost = hostToContainers.getOrElseUpdate(host, + new ArrayBuffer[Container]()) + containersForHost += container + } else { + // Release container, since it doesn't satisfy resource constraints. + internalReleaseContainer(container) + } + } + + // Find the appropriate containers to use. + // TODO: Cleanup this group-by... + val dataLocalContainers = new HashMap[String, ArrayBuffer[Container]]() + val rackLocalContainers = new HashMap[String, ArrayBuffer[Container]]() + val offRackContainers = new HashMap[String, ArrayBuffer[Container]]() + + for (candidateHost <- hostToContainers.keySet) { + val maxExpectedHostCount = preferredHostToCount.getOrElse(candidateHost, 0) + val requiredHostCount = maxExpectedHostCount - allocatedContainersOnHost(candidateHost) + + val remainingContainersOpt = hostToContainers.get(candidateHost) + assert(remainingContainersOpt.isDefined) + var remainingContainers = remainingContainersOpt.get + + if (requiredHostCount >= remainingContainers.size) { + // Since we have <= required containers, add all remaining containers to + // `dataLocalContainers`. + dataLocalContainers.put(candidateHost, remainingContainers) + // There are no more free containers remaining. + remainingContainers = null + } else if (requiredHostCount > 0) { + // Container list has more containers than we need for data locality. + // Split the list into two: one based on the data local container count, + // (`remainingContainers.size` - `requiredHostCount`), and the other to hold remaining + // containers. + val (dataLocal, remaining) = remainingContainers.splitAt( + remainingContainers.size - requiredHostCount) + dataLocalContainers.put(candidateHost, dataLocal) + + // Invariant: remainingContainers == remaining + + // YARN has a nasty habit of allocating a ton of containers on a host - discourage this. + // Add each container in `remaining` to list of containers to release. If we have an + // insufficient number of containers, then the next allocation cycle will reallocate + // (but won't treat it as data local). + // TODO(harvey): Rephrase this comment some more. + for (container <- remaining) internalReleaseContainer(container) + remainingContainers = null + } + + // For rack local containers + if (remainingContainers != null) { + val rack = YarnSparkHadoopUtil.lookupRack(conf, candidateHost) + if (rack != null) { + val maxExpectedRackCount = preferredRackToCount.getOrElse(rack, 0) + val requiredRackCount = maxExpectedRackCount - allocatedContainersOnRack(rack) - + rackLocalContainers.getOrElse(rack, List()).size + + if (requiredRackCount >= remainingContainers.size) { + // Add all remaining containers to to `dataLocalContainers`. + dataLocalContainers.put(rack, remainingContainers) + remainingContainers = null + } else if (requiredRackCount > 0) { + // Container list has more containers that we need for data locality. + // Split the list into two: one based on the data local container count, + // (`remainingContainers.size` - `requiredHostCount`), and the other to hold remaining + // containers. + val (rackLocal, remaining) = remainingContainers.splitAt( + remainingContainers.size - requiredRackCount) + val existingRackLocal = rackLocalContainers.getOrElseUpdate(rack, + new ArrayBuffer[Container]()) + + existingRackLocal ++= rackLocal + + remainingContainers = remaining + } + } + } + + if (remainingContainers != null) { + // Not all containers have been consumed - add them to the list of off-rack containers. + offRackContainers.put(candidateHost, remainingContainers) + } + } + + // Now that we have split the containers into various groups, go through them in order: + // first host-local, then rack-local, and finally off-rack. + // Note that the list we create below tries to ensure that not all containers end up within + // a host if there is a sufficiently large number of hosts/containers. + val allocatedContainersToProcess = new ArrayBuffer[Container](allocatedContainers.size) + allocatedContainersToProcess ++= TaskSchedulerImpl.prioritizeContainers(dataLocalContainers) + allocatedContainersToProcess ++= TaskSchedulerImpl.prioritizeContainers(rackLocalContainers) + allocatedContainersToProcess ++= TaskSchedulerImpl.prioritizeContainers(offRackContainers) + + // Run each of the allocated containers. + for (container <- allocatedContainersToProcess) { + val numExecutorsRunningNow = numExecutorsRunning.incrementAndGet() + val executorHostname = container.getNodeId.getHost + val containerId = container.getId + + val executorMemoryOverhead = (executorMemory + memoryOverhead) + assert(container.getResource.getMemory >= executorMemoryOverhead) + + if (numExecutorsRunningNow > maxExecutors) { + logInfo("""Ignoring container %s at host %s, since we already have the required number of + containers for it.""".format(containerId, executorHostname)) + internalReleaseContainer(container) + numExecutorsRunning.decrementAndGet() + } else { + val executorId = executorIdCounter.incrementAndGet().toString + val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + SparkEnv.driverActorSystemName, + sparkConf.get("spark.driver.host"), + sparkConf.get("spark.driver.port"), + CoarseGrainedSchedulerBackend.ACTOR_NAME) + + logInfo("Launching container %s for on host %s".format(containerId, executorHostname)) + + // To be safe, remove the container from `releasedContainers`. + releasedContainers.remove(containerId) + + val rack = YarnSparkHadoopUtil.lookupRack(conf, executorHostname) + allocatedHostToContainersMap.synchronized { + val containerSet = allocatedHostToContainersMap.getOrElseUpdate(executorHostname, + new HashSet[ContainerId]()) + + containerSet += containerId + allocatedContainerToHostMap.put(containerId, executorHostname) + + if (rack != null) { + allocatedRackCount.put(rack, allocatedRackCount.getOrElse(rack, 0) + 1) + } + } + logInfo("Launching ExecutorRunnable. driverUrl: %s, executorHostname: %s".format( + driverUrl, executorHostname)) + val executorRunnable = new ExecutorRunnable( + container, + conf, + sparkConf, + driverUrl, + executorId, + executorHostname, + executorMemory, + executorCores) + new Thread(executorRunnable).start() + } + } + logDebug(""" + Finished allocating %s containers (from %s originally). + Current number of executors running: %d, + Released containers: %s + """.format( + allocatedContainersToProcess, + allocatedContainers, + numExecutorsRunning.get(), + releasedContainers)) + } + + val completedContainers = allocateResponse.getCompletedContainersStatuses() + if (completedContainers.size > 0) { + logDebug("Completed %d containers".format(completedContainers.size)) + + for (completedContainer <- completedContainers) { + val containerId = completedContainer.getContainerId + + if (releasedContainers.containsKey(containerId)) { + // YarnAllocationHandler already marked the container for release, so remove it from + // `releasedContainers`. + releasedContainers.remove(containerId) + } else { + // Decrement the number of executors running. The next iteration of + // the ApplicationMaster's reporting thread will take care of allocating. + numExecutorsRunning.decrementAndGet() + logInfo("Completed container %s (state: %s, exit status: %s)".format( + containerId, + completedContainer.getState, + completedContainer.getExitStatus())) + // Hadoop 2.2.X added a ContainerExitStatus we should switch to use + // there are some exit status' we shouldn't necessarily count against us, but for + // now I think its ok as none of the containers are expected to exit + if (completedContainer.getExitStatus() != 0) { + logInfo("Container marked as failed: " + containerId) + numExecutorsFailed.incrementAndGet() + } + } + + allocatedHostToContainersMap.synchronized { + if (allocatedContainerToHostMap.containsKey(containerId)) { + val hostOpt = allocatedContainerToHostMap.get(containerId) + assert(hostOpt.isDefined) + val host = hostOpt.get + + val containerSetOpt = allocatedHostToContainersMap.get(host) + assert(containerSetOpt.isDefined) + val containerSet = containerSetOpt.get + + containerSet.remove(containerId) + if (containerSet.isEmpty) { + allocatedHostToContainersMap.remove(host) + } else { + allocatedHostToContainersMap.update(host, containerSet) + } + + allocatedContainerToHostMap.remove(containerId) + + // TODO: Move this part outside the synchronized block? + val rack = YarnSparkHadoopUtil.lookupRack(conf, host) + if (rack != null) { + val rackCount = allocatedRackCount.getOrElse(rack, 0) - 1 + if (rackCount > 0) { + allocatedRackCount.put(rack, rackCount) + } else { + allocatedRackCount.remove(rack) + } + } + } + } + } + logDebug(""" + Finished processing %d completed containers. + Current number of executors running: %d, + Released containers: %s + """.format( + completedContainers.size, + numExecutorsRunning.get(), + releasedContainers)) + } + } + + protected def allocatedContainersOnHost(host: String): Int = { + var retval = 0 + allocatedHostToContainersMap.synchronized { + retval = allocatedHostToContainersMap.getOrElse(host, Set()).size + } + retval + } + + protected def allocatedContainersOnRack(rack: String): Int = { + var retval = 0 + allocatedHostToContainersMap.synchronized { + retval = allocatedRackCount.getOrElse(rack, 0) + } + retval + } + + private def isResourceConstraintSatisfied(container: Container): Boolean = { + container.getResource.getMemory >= (executorMemory + memoryOverhead) + } + + // A simple method to copy the split info map. + private def generateNodeToWeight( + conf: Configuration, + input: collection.Map[String, collection.Set[SplitInfo]] + ): (Map[String, Int], Map[String, Int]) = { + + if (input == null) { + return (Map[String, Int](), Map[String, Int]()) + } + + val hostToCount = new HashMap[String, Int] + val rackToCount = new HashMap[String, Int] + + for ((host, splits) <- input) { + val hostCount = hostToCount.getOrElse(host, 0) + hostToCount.put(host, hostCount + splits.size) + + val rack = YarnSparkHadoopUtil.lookupRack(conf, host) + if (rack != null) { + val rackCount = rackToCount.getOrElse(host, 0) + rackToCount.put(host, rackCount + splits.size) + } + } + + (hostToCount.toMap, rackToCount.toMap) + } + + private def internalReleaseContainer(container: Container) = { + releasedContainers.put(container.getId(), true) + releaseContainer(container) + } + + /** + * Called to allocate containers in the cluster. + * + * @param count Number of containers to allocate. + * If zero, should still contact RM (as a heartbeat). + * @return Response to the allocation request. + */ + protected def allocateContainers(count: Int): YarnAllocateResponse + + /** Called to release a previously allocated container. */ + protected def releaseContainer(container: Container): Unit + + /** + * Defines the interface for an allocate response from the RM. This is needed since the alpha + * and stable interfaces differ here in ways that cannot be fixed using other routes. + */ + protected trait YarnAllocateResponse { + + def getAllocatedContainers(): JList[Container] + + def getAvailableResources(): Resource + + def getCompletedContainersStatuses(): JList[ContainerStatus] + + } + +} \ No newline at end of file diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 4d5144989991f..ed31457b61571 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -17,36 +17,19 @@ package org.apache.spark.deploy.yarn -import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap} -import java.util.concurrent.atomic.AtomicInteger - import scala.collection.JavaConversions._ -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.collection.mutable.{ArrayBuffer, HashMap} -import org.apache.spark.{Logging, SparkConf, SparkEnv} -import org.apache.spark.scheduler.{SplitInfo,TaskSchedulerImpl} -import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend -import org.apache.spark.util.Utils +import org.apache.spark.SparkConf +import org.apache.spark.scheduler.SplitInfo import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId -import org.apache.hadoop.yarn.api.records.{Container, ContainerId} -import org.apache.hadoop.yarn.api.records.{Priority, Resource, ResourceRequest} -import org.apache.hadoop.yarn.api.protocolrecords.{AllocateRequest, AllocateResponse} +import org.apache.hadoop.yarn.api.records._ +import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse import org.apache.hadoop.yarn.client.api.AMRMClient import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest import org.apache.hadoop.yarn.util.Records -// TODO: -// Too many params. -// Needs to be mt-safe -// Need to refactor this to make it 'cleaner' ... right now, all computation is reactive - should -// make it more proactive and decoupled. - -// Note that right now, we assume all node asks as uniform in terms of capabilities and priority -// Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for -// more info on how we are requesting for containers. - /** * Acquires resources for executors from a ResourceManager and launches executors in new containers. */ @@ -57,329 +40,22 @@ private[yarn] class YarnAllocationHandler( appAttemptId: ApplicationAttemptId, args: ApplicationMasterArguments, preferredNodes: collection.Map[String, collection.Set[SplitInfo]]) - extends YarnAllocator with Logging { - - // These three are locked on allocatedHostToContainersMap. Complementary data structures - // allocatedHostToContainersMap : containers which are running : host, Set - // allocatedContainerToHostMap: container to host mapping. - private val allocatedHostToContainersMap = - new HashMap[String, collection.mutable.Set[ContainerId]]() - - private val allocatedContainerToHostMap = new HashMap[ContainerId, String]() - - // allocatedRackCount is populated ONLY if allocation happens (or decremented if this is an - // allocated node) - // As with the two data structures above, tightly coupled with them, and to be locked on - // allocatedHostToContainersMap - private val allocatedRackCount = new HashMap[String, Int]() - - // Containers which have been released. - private val releasedContainerList = new CopyOnWriteArrayList[ContainerId]() - // Containers to be released in next request to RM - private val pendingReleaseContainers = new ConcurrentHashMap[ContainerId, Boolean] - - // Additional memory overhead - in mb. - private def memoryOverhead: Int = sparkConf.getInt("spark.yarn.executor.memoryOverhead", - YarnSparkHadoopUtil.DEFAULT_MEMORY_OVERHEAD) - - // Number of container requests that have been sent to, but not yet allocated by the - // ApplicationMaster. - private val numPendingAllocate = new AtomicInteger() - private val numExecutorsRunning = new AtomicInteger() - // Used to generate a unique id per executor - private val executorIdCounter = new AtomicInteger() - private val lastResponseId = new AtomicInteger() - private val numExecutorsFailed = new AtomicInteger() - - private val maxExecutors = args.numExecutors - private val executorMemory = args.executorMemory - private val executorCores = args.executorCores - private val (preferredHostToCount, preferredRackToCount) = - generateNodeToWeight(conf, preferredNodes) - - override def getNumExecutorsRunning: Int = numExecutorsRunning.intValue + extends YarnAllocator(conf, sparkConf, args, preferredNodes) { - override def getNumExecutorsFailed: Int = numExecutorsFailed.intValue - - def isResourceConstraintSatisfied(container: Container): Boolean = { - container.getResource.getMemory >= (executorMemory + memoryOverhead) - } - - def releaseContainer(container: Container) { - val containerId = container.getId - pendingReleaseContainers.put(containerId, true) - amClient.releaseAssignedContainer(containerId) + override protected def releaseContainer(container: Container) = { + amClient.releaseAssignedContainer(container.getId()) } - override def allocateResources() = { - addResourceRequests(maxExecutors - numPendingAllocate.get() - numExecutorsRunning.get()) + override protected def allocateContainers(count: Int): YarnAllocateResponse = { + addResourceRequests(count) // We have already set the container request. Poll the ResourceManager for a response. // This doubles as a heartbeat if there are no pending container requests. val progressIndicator = 0.1f - val allocateResponse = amClient.allocate(progressIndicator) - - val allocatedContainers = allocateResponse.getAllocatedContainers() - if (allocatedContainers.size > 0) { - var numPendingAllocateNow = numPendingAllocate.addAndGet(-1 * allocatedContainers.size) - - if (numPendingAllocateNow < 0) { - numPendingAllocateNow = numPendingAllocate.addAndGet(-1 * numPendingAllocateNow) - } - - logDebug(""" - Allocated containers: %d - Current executor count: %d - Containers released: %s - Containers to-be-released: %s - Cluster resources: %s - """.format( - allocatedContainers.size, - numExecutorsRunning.get(), - releasedContainerList, - pendingReleaseContainers, - allocateResponse.getAvailableResources)) - - val hostToContainers = new HashMap[String, ArrayBuffer[Container]]() - - for (container <- allocatedContainers) { - if (isResourceConstraintSatisfied(container)) { - // Add the accepted `container` to the host's list of already accepted, - // allocated containers - val host = container.getNodeId.getHost - val containersForHost = hostToContainers.getOrElseUpdate(host, - new ArrayBuffer[Container]()) - containersForHost += container - } else { - // Release container, since it doesn't satisfy resource constraints. - releaseContainer(container) - } - } - - // Find the appropriate containers to use. - // TODO: Cleanup this group-by... - val dataLocalContainers = new HashMap[String, ArrayBuffer[Container]]() - val rackLocalContainers = new HashMap[String, ArrayBuffer[Container]]() - val offRackContainers = new HashMap[String, ArrayBuffer[Container]]() - - for (candidateHost <- hostToContainers.keySet) { - val maxExpectedHostCount = preferredHostToCount.getOrElse(candidateHost, 0) - val requiredHostCount = maxExpectedHostCount - allocatedContainersOnHost(candidateHost) - - val remainingContainersOpt = hostToContainers.get(candidateHost) - assert(remainingContainersOpt.isDefined) - var remainingContainers = remainingContainersOpt.get - - if (requiredHostCount >= remainingContainers.size) { - // Since we have <= required containers, add all remaining containers to - // `dataLocalContainers`. - dataLocalContainers.put(candidateHost, remainingContainers) - // There are no more free containers remaining. - remainingContainers = null - } else if (requiredHostCount > 0) { - // Container list has more containers than we need for data locality. - // Split the list into two: one based on the data local container count, - // (`remainingContainers.size` - `requiredHostCount`), and the other to hold remaining - // containers. - val (dataLocal, remaining) = remainingContainers.splitAt( - remainingContainers.size - requiredHostCount) - dataLocalContainers.put(candidateHost, dataLocal) - - // Invariant: remainingContainers == remaining - - // YARN has a nasty habit of allocating a ton of containers on a host - discourage this. - // Add each container in `remaining` to list of containers to release. If we have an - // insufficient number of containers, then the next allocation cycle will reallocate - // (but won't treat it as data local). - // TODO(harvey): Rephrase this comment some more. - for (container <- remaining) releaseContainer(container) - remainingContainers = null - } - - // For rack local containers - if (remainingContainers != null) { - val rack = YarnSparkHadoopUtil.lookupRack(conf, candidateHost) - if (rack != null) { - val maxExpectedRackCount = preferredRackToCount.getOrElse(rack, 0) - val requiredRackCount = maxExpectedRackCount - allocatedContainersOnRack(rack) - - rackLocalContainers.getOrElse(rack, List()).size - - if (requiredRackCount >= remainingContainers.size) { - // Add all remaining containers to to `dataLocalContainers`. - dataLocalContainers.put(rack, remainingContainers) - remainingContainers = null - } else if (requiredRackCount > 0) { - // Container list has more containers that we need for data locality. - // Split the list into two: one based on the data local container count, - // (`remainingContainers.size` - `requiredHostCount`), and the other to hold remaining - // containers. - val (rackLocal, remaining) = remainingContainers.splitAt( - remainingContainers.size - requiredRackCount) - val existingRackLocal = rackLocalContainers.getOrElseUpdate(rack, - new ArrayBuffer[Container]()) - - existingRackLocal ++= rackLocal - - remainingContainers = remaining - } - } - } - - if (remainingContainers != null) { - // Not all containers have been consumed - add them to the list of off-rack containers. - offRackContainers.put(candidateHost, remainingContainers) - } - } - - // Now that we have split the containers into various groups, go through them in order: - // first host-local, then rack-local, and finally off-rack. - // Note that the list we create below tries to ensure that not all containers end up within - // a host if there is a sufficiently large number of hosts/containers. - val allocatedContainersToProcess = new ArrayBuffer[Container](allocatedContainers.size) - allocatedContainersToProcess ++= TaskSchedulerImpl.prioritizeContainers(dataLocalContainers) - allocatedContainersToProcess ++= TaskSchedulerImpl.prioritizeContainers(rackLocalContainers) - allocatedContainersToProcess ++= TaskSchedulerImpl.prioritizeContainers(offRackContainers) - - // Run each of the allocated containers. - for (container <- allocatedContainersToProcess) { - val numExecutorsRunningNow = numExecutorsRunning.incrementAndGet() - val executorHostname = container.getNodeId.getHost - val containerId = container.getId - - val executorMemoryOverhead = (executorMemory + memoryOverhead) - assert(container.getResource.getMemory >= executorMemoryOverhead) - - if (numExecutorsRunningNow > maxExecutors) { - logInfo("""Ignoring container %s at host %s, since we already have the required number of - containers for it.""".format(containerId, executorHostname)) - releaseContainer(container) - numExecutorsRunning.decrementAndGet() - } else { - val executorId = executorIdCounter.incrementAndGet().toString - val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( - SparkEnv.driverActorSystemName, - sparkConf.get("spark.driver.host"), - sparkConf.get("spark.driver.port"), - CoarseGrainedSchedulerBackend.ACTOR_NAME) - - logInfo("Launching container %s for on host %s".format(containerId, executorHostname)) - - // To be safe, remove the container from `pendingReleaseContainers`. - pendingReleaseContainers.remove(containerId) - - val rack = YarnSparkHadoopUtil.lookupRack(conf, executorHostname) - allocatedHostToContainersMap.synchronized { - val containerSet = allocatedHostToContainersMap.getOrElseUpdate(executorHostname, - new HashSet[ContainerId]()) - - containerSet += containerId - allocatedContainerToHostMap.put(containerId, executorHostname) - - if (rack != null) { - allocatedRackCount.put(rack, allocatedRackCount.getOrElse(rack, 0) + 1) - } - } - logInfo("Launching ExecutorRunnable. driverUrl: %s, executorHostname: %s".format( - driverUrl, executorHostname)) - val executorRunnable = new ExecutorRunnable( - container, - conf, - sparkConf, - driverUrl, - executorId, - executorHostname, - executorMemory, - executorCores) - new Thread(executorRunnable).start() - } - } - logDebug(""" - Finished allocating %s containers (from %s originally). - Current number of executors running: %d, - releasedContainerList: %s, - pendingReleaseContainers: %s - """.format( - allocatedContainersToProcess, - allocatedContainers, - numExecutorsRunning.get(), - releasedContainerList, - pendingReleaseContainers)) - } - - val completedContainers = allocateResponse.getCompletedContainersStatuses() - if (completedContainers.size > 0) { - logDebug("Completed %d containers".format(completedContainers.size)) - - for (completedContainer <- completedContainers) { - val containerId = completedContainer.getContainerId - - if (pendingReleaseContainers.containsKey(containerId)) { - // YarnAllocationHandler already marked the container for release, so remove it from - // `pendingReleaseContainers`. - pendingReleaseContainers.remove(containerId) - } else { - // Decrement the number of executors running. The next iteration of - // the ApplicationMaster's reporting thread will take care of allocating. - numExecutorsRunning.decrementAndGet() - logInfo("Completed container %s (state: %s, exit status: %s)".format( - containerId, - completedContainer.getState, - completedContainer.getExitStatus())) - // Hadoop 2.2.X added a ContainerExitStatus we should switch to use - // there are some exit status' we shouldn't necessarily count against us, but for - // now I think its ok as none of the containers are expected to exit - if (completedContainer.getExitStatus() != 0) { - logInfo("Container marked as failed: " + containerId) - numExecutorsFailed.incrementAndGet() - } - } - - allocatedHostToContainersMap.synchronized { - if (allocatedContainerToHostMap.containsKey(containerId)) { - val hostOpt = allocatedContainerToHostMap.get(containerId) - assert(hostOpt.isDefined) - val host = hostOpt.get - - val containerSetOpt = allocatedHostToContainersMap.get(host) - assert(containerSetOpt.isDefined) - val containerSet = containerSetOpt.get - - containerSet.remove(containerId) - if (containerSet.isEmpty) { - allocatedHostToContainersMap.remove(host) - } else { - allocatedHostToContainersMap.update(host, containerSet) - } - - allocatedContainerToHostMap.remove(containerId) - - // TODO: Move this part outside the synchronized block? - val rack = YarnSparkHadoopUtil.lookupRack(conf, host) - if (rack != null) { - val rackCount = allocatedRackCount.getOrElse(rack, 0) - 1 - if (rackCount > 0) { - allocatedRackCount.put(rack, rackCount) - } else { - allocatedRackCount.remove(rack) - } - } - } - } - } - logDebug(""" - Finished processing %d completed containers. - Current number of executors running: %d, - releasedContainerList: %s, - pendingReleaseContainers: %s - """.format( - completedContainers.size, - numExecutorsRunning.get(), - releasedContainerList, - pendingReleaseContainers)) - } + new StableAllocateResponse(amClient.allocate(progressIndicator)) } - def createRackResourceRequests( + private def createRackResourceRequests( hostContainers: ArrayBuffer[ContainerRequest] ): ArrayBuffer[ContainerRequest] = { // Generate modified racks and new set of hosts under it before issuing requests. @@ -409,22 +85,6 @@ private[yarn] class YarnAllocationHandler( requestedContainers } - def allocatedContainersOnHost(host: String): Int = { - var retval = 0 - allocatedHostToContainersMap.synchronized { - retval = allocatedHostToContainersMap.getOrElse(host, Set()).size - } - retval - } - - def allocatedContainersOnRack(rack: String): Int = { - var retval = 0 - allocatedHostToContainersMap.synchronized { - retval = allocatedRackCount.getOrElse(rack, 0) - } - retval - } - private def addResourceRequests(numExecutors: Int) { val containerRequests: List[ContainerRequest] = if (numExecutors <= 0 || preferredHostToCount.isEmpty) { @@ -472,15 +132,6 @@ private[yarn] class YarnAllocationHandler( amClient.addContainerRequest(request) } - if (numExecutors > 0) { - numPendingAllocate.addAndGet(numExecutors) - logInfo("Will Allocate %d executor containers, each with %d memory".format( - numExecutors, - (executorMemory + memoryOverhead))) - } else { - logDebug("Empty allocation request ...") - } - for (request <- containerRequests) { val nodes = request.getNodes var hostStr = if (nodes == null || nodes.isEmpty) { @@ -549,31 +200,10 @@ private[yarn] class YarnAllocationHandler( requests } - // A simple method to copy the split info map. - private def generateNodeToWeight( - conf: Configuration, - input: collection.Map[String, collection.Set[SplitInfo]] - ): (Map[String, Int], Map[String, Int]) = { - - if (input == null) { - return (Map[String, Int](), Map[String, Int]()) - } - - val hostToCount = new HashMap[String, Int] - val rackToCount = new HashMap[String, Int] - - for ((host, splits) <- input) { - val hostCount = hostToCount.getOrElse(host, 0) - hostToCount.put(host, hostCount + splits.size) - - val rack = YarnSparkHadoopUtil.lookupRack(conf, host) - if (rack != null){ - val rackCount = rackToCount.getOrElse(host, 0) - rackToCount.put(host, rackCount + splits.size) - } - } - - (hostToCount.toMap, rackToCount.toMap) + private class StableAllocateResponse(response: AllocateResponse) extends YarnAllocateResponse { + override def getAllocatedContainers() = response.getAllocatedContainers() + override def getAvailableResources() = response.getAvailableResources() + override def getCompletedContainersStatuses() = response.getCompletedContainersStatuses() } } From 6481d27425f6d42ead36663c9a4ef7ee13b3a8c9 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 3 Sep 2014 11:49:45 -0700 Subject: [PATCH 12/39] [SPARK-3309] [PySpark] Put all public API in __all__ Put all public API in __all__, also put them all in pyspark.__init__.py, then we can got all the documents for public API by `pydoc pyspark`. It also can be used by other programs (such as Sphinx or Epydoc) to generate only documents for public APIs. Author: Davies Liu Closes #2205 from davies/public and squashes the following commits: c6c5567 [Davies Liu] fix message f7b35be [Davies Liu] put SchemeRDD, Row in pyspark.sql module 7e3016a [Davies Liu] add __all__ in mllib 6281b48 [Davies Liu] fix doc for SchemaRDD 6caab21 [Davies Liu] add public interfaces into pyspark.__init__.py --- python/pyspark/__init__.py | 14 +++++++++----- python/pyspark/accumulators.py | 3 +++ python/pyspark/broadcast.py | 24 ++++++++++++++---------- python/pyspark/conf.py | 2 ++ python/pyspark/context.py | 3 +++ python/pyspark/files.py | 3 +++ python/pyspark/mllib/classification.py | 4 ++++ python/pyspark/mllib/clustering.py | 2 ++ python/pyspark/mllib/linalg.py | 3 +++ python/pyspark/mllib/random.py | 3 +++ python/pyspark/mllib/recommendation.py | 2 ++ python/pyspark/mllib/regression.py | 10 +++++----- python/pyspark/mllib/stat.py | 6 ++++-- python/pyspark/mllib/tree.py | 4 ++++ python/pyspark/rdd.py | 1 + python/pyspark/serializers.py | 2 +- python/pyspark/sql.py | 21 ++++++++++++++++++--- 17 files changed, 81 insertions(+), 26 deletions(-) diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py index c58555fc9d2c5..1a2e774738fe7 100644 --- a/python/pyspark/__init__.py +++ b/python/pyspark/__init__.py @@ -61,13 +61,17 @@ from pyspark.conf import SparkConf from pyspark.context import SparkContext -from pyspark.sql import SQLContext from pyspark.rdd import RDD -from pyspark.sql import SchemaRDD -from pyspark.sql import Row from pyspark.files import SparkFiles from pyspark.storagelevel import StorageLevel +from pyspark.accumulators import Accumulator, AccumulatorParam +from pyspark.broadcast import Broadcast +from pyspark.serializers import MarshalSerializer, PickleSerializer +# for back compatibility +from pyspark.sql import SQLContext, HiveContext, SchemaRDD, Row -__all__ = ["SparkConf", "SparkContext", "SQLContext", "RDD", "SchemaRDD", - "SparkFiles", "StorageLevel", "Row"] +__all__ = [ + "SparkConf", "SparkContext", "SparkFiles", "RDD", "StorageLevel", "Broadcast", + "Accumulator", "AccumulatorParam", "MarshalSerializer", "PickleSerializer", +] diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index f133cf6f7befc..ccbca67656c8d 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -94,6 +94,9 @@ from pyspark.serializers import read_int, PickleSerializer +__all__ = ['Accumulator', 'AccumulatorParam'] + + pickleSer = PickleSerializer() # Holds accumulators registered on the current machine, keyed by ID. This is then used to send diff --git a/python/pyspark/broadcast.py b/python/pyspark/broadcast.py index 675a2fcd2ff4e..5c7c9cc161dff 100644 --- a/python/pyspark/broadcast.py +++ b/python/pyspark/broadcast.py @@ -31,6 +31,10 @@ from pyspark.serializers import CompressedSerializer, PickleSerializer + +__all__ = ['Broadcast'] + + # Holds broadcasted data received from Java, keyed by its id. _broadcastRegistry = {} @@ -59,11 +63,20 @@ def __init__(self, bid, value, java_broadcast=None, """ self.bid = bid if path is None: - self.value = value + self._value = value self._jbroadcast = java_broadcast self._pickle_registry = pickle_registry self.path = path + @property + def value(self): + """ Return the broadcasted value + """ + if not hasattr(self, "_value") and self.path is not None: + ser = CompressedSerializer(PickleSerializer()) + self._value = ser.load_stream(open(self.path)).next() + return self._value + def unpersist(self, blocking=False): self._jbroadcast.unpersist(blocking) os.unlink(self.path) @@ -72,15 +85,6 @@ def __reduce__(self): self._pickle_registry.add(self) return (_from_id, (self.bid, )) - def __getattr__(self, item): - if item == 'value' and self.path is not None: - ser = CompressedSerializer(PickleSerializer()) - value = ser.load_stream(open(self.path)).next() - self.value = value - return value - - raise AttributeError(item) - if __name__ == "__main__": import doctest diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py index fb716f6753a45..b64875a3f495a 100644 --- a/python/pyspark/conf.py +++ b/python/pyspark/conf.py @@ -54,6 +54,8 @@ (u'spark.executorEnv.VAR4', u'value4'), (u'spark.home', u'/path')] """ +__all__ = ['SparkConf'] + class SparkConf(object): diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 6e4fdaa6eec9d..5a30431568b16 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -37,6 +37,9 @@ from py4j.java_collections import ListConverter +__all__ = ['SparkContext'] + + # These are special default configs for PySpark, they will overwrite # the default ones for Spark if they are not configured by user. DEFAULT_CONFIGS = { diff --git a/python/pyspark/files.py b/python/pyspark/files.py index 331de9a9b2212..797573f49dac8 100644 --- a/python/pyspark/files.py +++ b/python/pyspark/files.py @@ -18,6 +18,9 @@ import os +__all__ = ['SparkFiles'] + + class SparkFiles(object): """ diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index ffdda7ee19302..71ab46b61d7fa 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -30,6 +30,10 @@ from math import exp, log +__all__ = ['LogisticRegressionModel', 'LogisticRegressionWithSGD', 'SVMModel', + 'SVMWithSGD', 'NaiveBayesModel', 'NaiveBayes'] + + class LogisticRegressionModel(LinearModel): """A linear binary classification model derived from logistic regression. diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index a0630d1d5c58b..f3e952a1d842a 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -25,6 +25,8 @@ _get_initial_weights, _serialize_rating, _regression_train_wrapper from pyspark.mllib.linalg import SparseVector +__all__ = ['KMeansModel', 'KMeans'] + class KMeansModel(object): diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index f485a69db1fa2..e69051c104e37 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -27,6 +27,9 @@ from numpy import array, array_equal, ndarray, float64, int32 +__all__ = ['SparseVector', 'Vectors'] + + class SparseVector(object): """ diff --git a/python/pyspark/mllib/random.py b/python/pyspark/mllib/random.py index 4dc1a4a912421..3e59c73db85e3 100644 --- a/python/pyspark/mllib/random.py +++ b/python/pyspark/mllib/random.py @@ -25,6 +25,9 @@ from pyspark.serializers import NoOpSerializer +__all__ = ['RandomRDDs', ] + + class RandomRDDs: """ Generator methods for creating RDDs comprised of i.i.d samples from diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index e863fc249ec36..2df23394da6f8 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -24,6 +24,8 @@ _serialize_tuple, RatingDeserializer from pyspark.rdd import RDD +__all__ = ['MatrixFactorizationModel', 'ALS'] + class MatrixFactorizationModel(object): diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index d8792cf44872f..f572dcfb840b6 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -17,15 +17,15 @@ from numpy import array, ndarray from pyspark import SparkContext -from pyspark.mllib._common import \ - _dot, _get_unmangled_rdd, _get_unmangled_double_vector_rdd, \ - _serialize_double_matrix, _deserialize_double_matrix, \ - _serialize_double_vector, _deserialize_double_vector, \ - _get_initial_weights, _serialize_rating, _regression_train_wrapper, \ +from pyspark.mllib._common import _dot, _regression_train_wrapper, \ _linear_predictor_typecheck, _have_scipy, _scipy_issparse from pyspark.mllib.linalg import SparseVector, Vectors +__all__ = ['LabeledPoint', 'LinearModel', 'LinearRegressionModel', 'RidgeRegressionModel' + 'LinearRegressionWithSGD', 'LassoWithSGD', 'RidgeRegressionWithSGD'] + + class LabeledPoint(object): """ diff --git a/python/pyspark/mllib/stat.py b/python/pyspark/mllib/stat.py index feef0d16cd644..8c726f171c978 100644 --- a/python/pyspark/mllib/stat.py +++ b/python/pyspark/mllib/stat.py @@ -21,8 +21,10 @@ from pyspark.mllib._common import \ _get_unmangled_double_vector_rdd, _get_unmangled_rdd, \ - _serialize_double, _serialize_double_vector, \ - _deserialize_double, _deserialize_double_matrix, _deserialize_double_vector + _serialize_double, _deserialize_double_matrix, _deserialize_double_vector + + +__all__ = ['MultivariateStatisticalSummary', 'Statistics'] class MultivariateStatisticalSummary(object): diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index e9d778df5a24b..a2fade61e9a71 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -26,6 +26,9 @@ from pyspark.serializers import NoOpSerializer +__all__ = ['DecisionTreeModel', 'DecisionTree'] + + class DecisionTreeModel(object): """ @@ -88,6 +91,7 @@ class DecisionTree(object): It will probably be modified for Spark v1.2. Example usage: + >>> from numpy import array >>> import sys >>> from pyspark.mllib.regression import LabeledPoint diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 6fc9f66bc5a94..dff6fc26fcb18 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -48,6 +48,7 @@ from py4j.java_collections import ListConverter, MapConverter + __all__ = ["RDD"] diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index fc49aa42dbaf9..55e6cf3308611 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -409,7 +409,7 @@ def loads(self, obj): class CompressedSerializer(FramedSerializer): """ - compress the serialized data + Compress the serialized data """ def __init__(self, serializer): diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 0ff6a548a85f1..44316926ba334 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -40,8 +40,7 @@ "StringType", "BinaryType", "BooleanType", "TimestampType", "DecimalType", "DoubleType", "FloatType", "ByteType", "IntegerType", "LongType", "ShortType", "ArrayType", "MapType", "StructField", "StructType", - "SQLContext", "HiveContext", "LocalHiveContext", "TestHiveContext", - "SchemaRDD", "Row"] + "SQLContext", "HiveContext", "SchemaRDD", "Row"] class DataType(object): @@ -1037,7 +1036,7 @@ def inferSchema(self, rdd): "can not infer schema") if type(first) is dict: warnings.warn("Using RDD of dict to inferSchema is deprecated," - "please use pyspark.Row instead") + "please use pyspark.sql.Row instead") schema = _infer_schema(first) rdd = rdd.mapPartitions(lambda rows: _drop_schema(rows, schema)) @@ -1487,6 +1486,21 @@ def __repr__(self): return "" % ", ".join(self) +def inherit_doc(cls): + for name, func in vars(cls).items(): + # only inherit docstring for public functions + if name.startswith("_"): + continue + if not func.__doc__: + for parent in cls.__bases__: + parent_func = getattr(parent, name, None) + if parent_func and getattr(parent_func, "__doc__", None): + func.__doc__ = parent_func.__doc__ + break + return cls + + +@inherit_doc class SchemaRDD(RDD): """An RDD of L{Row} objects that has an associated schema. @@ -1563,6 +1577,7 @@ def registerTempTable(self, name): self._jschema_rdd.registerTempTable(name) def registerAsTable(self, name): + """DEPRECATED: use registerTempTable() instead""" warnings.warn("Use registerTempTable instead of registerAsTable.", DeprecationWarning) self.registerTempTable(name) From e5d376801d57dffb0791980a1786a0a9b45bc491 Mon Sep 17 00:00:00 2001 From: RJ Nowling Date: Wed, 3 Sep 2014 14:15:22 -0700 Subject: [PATCH 13/39] [SPARK-3263][GraphX] Fix changes made to GraphGenerator.logNormalGraph in PR #720 PR #720 made multiple changes to GraphGenerator.logNormalGraph including: * Replacing the call to functions for generating random vertices and edges with in-line implementations with different equations. Based on reading the Pregel paper, I believe the in-line functions are incorrect. * Hard-coding of RNG seeds so that method now generates the same graph for a given number of vertices, edges, mu, and sigma -- user is not able to override seed or specify that seed should be randomly generated. * Backwards-incompatible change to logNormalGraph signature with introduction of new required parameter. * Failed to update scala docs and programming guide for API changes * Added a Synthetic Benchmark in the examples. This PR: * Removes the in-line calls and calls original vertex / edge generation functions again * Adds an optional seed parameter for deterministic behavior (when desired) * Keeps the number of partitions parameter that was added. * Keeps compatibility with the synthetic benchmark example * Maintains backwards-compatible API Author: RJ Nowling Author: Ankur Dave Closes #2168 from rnowling/graphgenrand and squashes the following commits: f1cd79f [Ankur Dave] Style fixes e11918e [RJ Nowling] Fix bad comparisons in unit tests 785ac70 [RJ Nowling] Fix style error c70868d [RJ Nowling] Fix logNormalGraph scala doc for seed 41fd1f8 [RJ Nowling] Fix logNormalGraph scala doc for seed 799f002 [RJ Nowling] Added test for different seeds for sampleLogNormal 43949ad [RJ Nowling] Added test for different seeds for generateRandomEdges 2faf75f [RJ Nowling] Added unit test for logNormalGraph 82f22397 [RJ Nowling] Add unit test for sampleLogNormal b99cba9 [RJ Nowling] Make sampleLogNormal private to Spark (vs private) for unit testing 6803da1 [RJ Nowling] Add GraphGeneratorsSuite with test for generateRandomEdges 1c8fc44 [RJ Nowling] Connected components part of SynthBenchmark was failing to call count on RDD before printing dfbb6dd [RJ Nowling] Fix parameter name in SynthBenchmark docs b5eeb80 [RJ Nowling] Add optional seed parameter to SynthBenchmark and set default to randomly generate a seed 1ff8d30 [RJ Nowling] Fix bug in generateRandomEdges where numVertices instead of numEdges was used to control number of edges to generate 98bb73c [RJ Nowling] Add documentation for logNormalGraph parameters d40141a [RJ Nowling] Fix style error 684804d [RJ Nowling] revert PR #720 which introduce errors in logNormalGraph and messed up seeding of RNGs. Add user-defined optional seed for deterministic behavior c183136 [RJ Nowling] Fix to deterministic GraphGenerators.logNormalGraph that allows generating graphs randomly using optional seed. 015010c [RJ Nowling] Fixed GraphGenerator logNormalGraph API to make backward-incompatible change in commit 894ecde04 --- .../examples/graphx/SynthBenchmark.scala | 9 +- .../spark/graphx/util/GraphGenerators.scala | 65 ++++++----- .../graphx/util/GraphGeneratorsSuite.scala | 110 ++++++++++++++++++ 3 files changed, 152 insertions(+), 32 deletions(-) create mode 100644 graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala index 551c339b19523..5f35a5836462e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala @@ -38,12 +38,13 @@ object SynthBenchmark { * Options: * -app "pagerank" or "cc" for pagerank or connected components. (Default: pagerank) * -niters the number of iterations of pagerank to use (Default: 10) - * -numVertices the number of vertices in the graph (Default: 1000000) + * -nverts the number of vertices in the graph (Default: 1000000) * -numEPart the number of edge partitions in the graph (Default: number of cores) * -partStrategy the graph partitioning strategy to use * -mu the mean parameter for the log-normal graph (Default: 4.0) * -sigma the stdev parameter for the log-normal graph (Default: 1.3) * -degFile the local file to save the degree information (Default: Empty) + * -seed seed to use for RNGs (Default: -1, picks seed randomly) */ def main(args: Array[String]) { val options = args.map { @@ -62,6 +63,7 @@ object SynthBenchmark { var mu: Double = 4.0 var sigma: Double = 1.3 var degFile: String = "" + var seed: Int = -1 options.foreach { case ("app", v) => app = v @@ -72,6 +74,7 @@ object SynthBenchmark { case ("mu", v) => mu = v.toDouble case ("sigma", v) => sigma = v.toDouble case ("degFile", v) => degFile = v + case ("seed", v) => seed = v.toInt case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) } @@ -85,7 +88,7 @@ object SynthBenchmark { // Create the graph println(s"Creating graph...") val unpartitionedGraph = GraphGenerators.logNormalGraph(sc, numVertices, - numEPart.getOrElse(sc.defaultParallelism), mu, sigma) + numEPart.getOrElse(sc.defaultParallelism), mu, sigma, seed) // Repartition the graph val graph = partitionStrategy.foldLeft(unpartitionedGraph)(_.partitionBy(_)).cache() @@ -113,7 +116,7 @@ object SynthBenchmark { println(s"Total PageRank = $totalPR") } else if (app == "cc") { println("Running Connected Components") - val numComponents = graph.connectedComponents.vertices.map(_._2).distinct() + val numComponents = graph.connectedComponents.vertices.map(_._2).distinct().count() println(s"Number of components = $numComponents") } val runTime = System.currentTimeMillis() - startTime diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala index 60149548ab852..b8309289fe475 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala @@ -40,7 +40,7 @@ object GraphGenerators { val RMATd = 0.25 /** - * Generate a graph whose vertex out degree is log normal. + * Generate a graph whose vertex out degree distribution is log normal. * * The default values for mu and sigma are taken from the Pregel paper: * @@ -48,33 +48,36 @@ object GraphGenerators { * Ilan Horn, Naty Leiser, and Grzegorz Czajkowski. 2010. * Pregel: a system for large-scale graph processing. SIGMOD '10. * - * @param sc - * @param numVertices - * @param mu - * @param sigma - * @return + * If the seed is -1 (default), a random seed is chosen. Otherwise, use + * the user-specified seed. + * + * @param sc Spark Context + * @param numVertices number of vertices in generated graph + * @param numEParts (optional) number of partitions + * @param mu (optional, default: 4.0) mean of out-degree distribution + * @param sigma (optional, default: 1.3) standard deviation of out-degree distribution + * @param seed (optional, default: -1) seed for RNGs, -1 causes a random seed to be chosen + * @return Graph object */ - def logNormalGraph(sc: SparkContext, numVertices: Int, numEParts: Int, - mu: Double = 4.0, sigma: Double = 1.3): Graph[Long, Int] = { - val vertices = sc.parallelize(0 until numVertices, numEParts).map { src => - // Initialize the random number generator with the source vertex id - val rand = new Random(src) - val degree = math.min(numVertices.toLong, math.exp(rand.nextGaussian() * sigma + mu).toLong) - (src.toLong, degree) + def logNormalGraph( + sc: SparkContext, numVertices: Int, numEParts: Int = 0, mu: Double = 4.0, + sigma: Double = 1.3, seed: Long = -1): Graph[Long, Int] = { + + val evalNumEParts = if (numEParts == 0) sc.defaultParallelism else numEParts + + // Enable deterministic seeding + val seedRand = if (seed == -1) new Random() else new Random(seed) + val seed1 = seedRand.nextInt() + val seed2 = seedRand.nextInt() + + val vertices: RDD[(VertexId, Long)] = sc.parallelize(0 until numVertices, evalNumEParts).map { + src => (src, sampleLogNormal(mu, sigma, numVertices, seed = (seed1 ^ src))) } + val edges = vertices.flatMap { case (src, degree) => - new Iterator[Edge[Int]] { - // Initialize the random number generator with the source vertex id - val rand = new Random(src) - var i = 0 - override def hasNext(): Boolean = { i < degree } - override def next(): Edge[Int] = { - val nextEdge = Edge[Int](src, rand.nextInt(numVertices), i) - i += 1 - nextEdge - } - } + generateRandomEdges(src.toInt, degree.toInt, numVertices, seed = (seed2 ^ src)) } + Graph(vertices, edges, 0) } @@ -82,9 +85,10 @@ object GraphGenerators { // the edge data is the weight (default 1) val RMATc = 0.15 - def generateRandomEdges(src: Int, numEdges: Int, maxVertexId: Int): Array[Edge[Int]] = { - val rand = new Random() - Array.fill(maxVertexId) { Edge[Int](src, rand.nextInt(maxVertexId), 1) } + def generateRandomEdges( + src: Int, numEdges: Int, maxVertexId: Int, seed: Long = -1): Array[Edge[Int]] = { + val rand = if (seed == -1) new Random() else new Random(seed) + Array.fill(numEdges) { Edge[Int](src, rand.nextInt(maxVertexId), 1) } } /** @@ -97,9 +101,12 @@ object GraphGenerators { * @param mu the mean of the normal distribution * @param sigma the standard deviation of the normal distribution * @param maxVal exclusive upper bound on the value of the sample + * @param seed optional seed */ - private def sampleLogNormal(mu: Double, sigma: Double, maxVal: Int): Int = { - val rand = new Random() + private[spark] def sampleLogNormal( + mu: Double, sigma: Double, maxVal: Int, seed: Long = -1): Int = { + val rand = if (seed == -1) new Random() else new Random(seed) + val sigmaSq = sigma * sigma val m = math.exp(mu + sigmaSq / 2.0) // expm1 is exp(m)-1 with better accuracy for tiny m diff --git a/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala new file mode 100644 index 0000000000000..b346d4db2ef96 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphx.util + +import org.scalatest.FunSuite + +import org.apache.spark.graphx.LocalSparkContext + +class GraphGeneratorsSuite extends FunSuite with LocalSparkContext { + + test("GraphGenerators.generateRandomEdges") { + val src = 5 + val numEdges10 = 10 + val numEdges20 = 20 + val maxVertexId = 100 + + val edges10 = GraphGenerators.generateRandomEdges(src, numEdges10, maxVertexId) + assert(edges10.length == numEdges10) + + val correctSrc = edges10.forall(e => e.srcId == src) + assert(correctSrc) + + val correctWeight = edges10.forall(e => e.attr == 1) + assert(correctWeight) + + val correctRange = edges10.forall(e => e.dstId >= 0 && e.dstId <= maxVertexId) + assert(correctRange) + + val edges20 = GraphGenerators.generateRandomEdges(src, numEdges20, maxVertexId) + assert(edges20.length == numEdges20) + + val edges10_round1 = + GraphGenerators.generateRandomEdges(src, numEdges10, maxVertexId, seed = 12345) + val edges10_round2 = + GraphGenerators.generateRandomEdges(src, numEdges10, maxVertexId, seed = 12345) + assert(edges10_round1.zip(edges10_round2).forall { case (e1, e2) => + e1.srcId == e2.srcId && e1.dstId == e2.dstId && e1.attr == e2.attr + }) + + val edges10_round3 = + GraphGenerators.generateRandomEdges(src, numEdges10, maxVertexId, seed = 3467) + assert(!edges10_round1.zip(edges10_round3).forall { case (e1, e2) => + e1.srcId == e2.srcId && e1.dstId == e2.dstId && e1.attr == e2.attr + }) + } + + test("GraphGenerators.sampleLogNormal") { + val mu = 4.0 + val sigma = 1.3 + val maxVal = 100 + + val dstId = GraphGenerators.sampleLogNormal(mu, sigma, maxVal) + assert(dstId < maxVal) + + val dstId_round1 = GraphGenerators.sampleLogNormal(mu, sigma, maxVal, 12345) + val dstId_round2 = GraphGenerators.sampleLogNormal(mu, sigma, maxVal, 12345) + assert(dstId_round1 == dstId_round2) + + val dstId_round3 = GraphGenerators.sampleLogNormal(mu, sigma, maxVal, 789) + assert(dstId_round1 != dstId_round3) + } + + test("GraphGenerators.logNormalGraph") { + withSpark { sc => + val mu = 4.0 + val sigma = 1.3 + val numVertices100 = 100 + + val graph = GraphGenerators.logNormalGraph(sc, numVertices100, mu = mu, sigma = sigma) + assert(graph.vertices.count() == numVertices100) + + val graph_round1 = + GraphGenerators.logNormalGraph(sc, numVertices100, mu = mu, sigma = sigma, seed = 12345) + val graph_round2 = + GraphGenerators.logNormalGraph(sc, numVertices100, mu = mu, sigma = sigma, seed = 12345) + + val graph_round1_edges = graph_round1.edges.collect() + val graph_round2_edges = graph_round2.edges.collect() + + assert(graph_round1_edges.zip(graph_round2_edges).forall { case (e1, e2) => + e1.srcId == e2.srcId && e1.dstId == e2.dstId && e1.attr == e2.attr + }) + + val graph_round3 = + GraphGenerators.logNormalGraph(sc, numVertices100, mu = mu, sigma = sigma, seed = 567) + + val graph_round3_edges = graph_round3.edges.collect() + + assert(!graph_round1_edges.zip(graph_round3_edges).forall { case (e1, e2) => + e1.srcId == e2.srcId && e1.dstId == e2.dstId && e1.attr == e2.attr + }) + } + } + +} From ccc69e26ec2fadd90886990b90a5a600efd08aba Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 3 Sep 2014 14:47:11 -0700 Subject: [PATCH 14/39] [SPARK-2845] Add timestamps to block manager events. These are not used by the UI but are useful when analysing the logs from a spark job. Author: Marcelo Vanzin Closes #654 from vanzin/bm-event-tstamp and squashes the following commits: d5d6e66 [Marcelo Vanzin] Fix tests. ec06218 [Marcelo Vanzin] Review feedback. f134dbc [Marcelo Vanzin] Merge branch 'master' into bm-event-tstamp b495b7c [Marcelo Vanzin] Merge branch 'master' into bm-event-tstamp 7d2fe9e [Marcelo Vanzin] Review feedback. d6f381c [Marcelo Vanzin] Update tests added after patch was created. 45e3bf8 [Marcelo Vanzin] Fix unit test after merge. b37a10f [Marcelo Vanzin] Use === in test assertions. ef72824 [Marcelo Vanzin] Handle backwards compatibility with 1.0.0. aca1151 [Marcelo Vanzin] Fix unit test to check new fields. efdda8e [Marcelo Vanzin] Add timestamps to block manager events. --- .../spark/scheduler/SparkListener.scala | 4 +- .../storage/BlockManagerMasterActor.scala | 7 ++-- .../org/apache/spark/util/JsonProtocol.scala | 12 ++++-- .../storage/StorageStatusListenerSuite.scala | 18 ++++----- .../spark/ui/storage/StorageTabSuite.scala | 4 +- .../apache/spark/util/JsonProtocolSuite.scala | 37 +++++++++++++++++-- 6 files changed, 58 insertions(+), 24 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 86ca8445a1124..f33c2e065a200 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -67,11 +67,11 @@ case class SparkListenerEnvironmentUpdate(environmentDetails: Map[String, Seq[(S extends SparkListenerEvent @DeveloperApi -case class SparkListenerBlockManagerAdded(blockManagerId: BlockManagerId, maxMem: Long) +case class SparkListenerBlockManagerAdded(time: Long, blockManagerId: BlockManagerId, maxMem: Long) extends SparkListenerEvent @DeveloperApi -case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId) +case class SparkListenerBlockManagerRemoved(time: Long, blockManagerId: BlockManagerId) extends SparkListenerEvent @DeveloperApi diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 3ab07703b6f85..1a6c7cb24f9ac 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -203,7 +203,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus blockLocations.remove(blockId) } } - listenerBus.post(SparkListenerBlockManagerRemoved(blockManagerId)) + listenerBus.post(SparkListenerBlockManagerRemoved(System.currentTimeMillis(), blockManagerId)) } private def expireDeadHosts() { @@ -325,6 +325,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus } private def register(id: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { + val time = System.currentTimeMillis() if (!blockManagerInfo.contains(id)) { blockManagerIdByExecutor.get(id.executorId) match { case Some(manager) => @@ -340,9 +341,9 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus id.hostPort, Utils.bytesToString(maxMemSize))) blockManagerInfo(id) = - new BlockManagerInfo(id, System.currentTimeMillis(), maxMemSize, slaveActor) + new BlockManagerInfo(id, time, maxMemSize, slaveActor) } - listenerBus.post(SparkListenerBlockManagerAdded(id, maxMemSize)) + listenerBus.post(SparkListenerBlockManagerAdded(time, id, maxMemSize)) } private def updateBlockInfo( diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index a7543454eca1f..1fc536b096996 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -152,13 +152,15 @@ private[spark] object JsonProtocol { val blockManagerId = blockManagerIdToJson(blockManagerAdded.blockManagerId) ("Event" -> Utils.getFormattedClassName(blockManagerAdded)) ~ ("Block Manager ID" -> blockManagerId) ~ - ("Maximum Memory" -> blockManagerAdded.maxMem) + ("Maximum Memory" -> blockManagerAdded.maxMem) ~ + ("Timestamp" -> blockManagerAdded.time) } def blockManagerRemovedToJson(blockManagerRemoved: SparkListenerBlockManagerRemoved): JValue = { val blockManagerId = blockManagerIdToJson(blockManagerRemoved.blockManagerId) ("Event" -> Utils.getFormattedClassName(blockManagerRemoved)) ~ - ("Block Manager ID" -> blockManagerId) + ("Block Manager ID" -> blockManagerId) ~ + ("Timestamp" -> blockManagerRemoved.time) } def unpersistRDDToJson(unpersistRDD: SparkListenerUnpersistRDD): JValue = { @@ -466,12 +468,14 @@ private[spark] object JsonProtocol { def blockManagerAddedFromJson(json: JValue): SparkListenerBlockManagerAdded = { val blockManagerId = blockManagerIdFromJson(json \ "Block Manager ID") val maxMem = (json \ "Maximum Memory").extract[Long] - SparkListenerBlockManagerAdded(blockManagerId, maxMem) + val time = Utils.jsonOption(json \ "Timestamp").map(_.extract[Long]).getOrElse(-1L) + SparkListenerBlockManagerAdded(time, blockManagerId, maxMem) } def blockManagerRemovedFromJson(json: JValue): SparkListenerBlockManagerRemoved = { val blockManagerId = blockManagerIdFromJson(json \ "Block Manager ID") - SparkListenerBlockManagerRemoved(blockManagerId) + val time = Utils.jsonOption(json \ "Timestamp").map(_.extract[Long]).getOrElse(-1L) + SparkListenerBlockManagerRemoved(time, blockManagerId) } def unpersistRDDFromJson(json: JValue): SparkListenerUnpersistRDD = { diff --git a/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala index 4e022a69c8212..3a45875391e29 100644 --- a/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala @@ -36,13 +36,13 @@ class StorageStatusListenerSuite extends FunSuite { // Block manager add assert(listener.executorIdToStorageStatus.size === 0) - listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(bm1, 1000L)) + listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) assert(listener.executorIdToStorageStatus.size === 1) assert(listener.executorIdToStorageStatus.get("big").isDefined) assert(listener.executorIdToStorageStatus("big").blockManagerId === bm1) assert(listener.executorIdToStorageStatus("big").maxMem === 1000L) assert(listener.executorIdToStorageStatus("big").numBlocks === 0) - listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(bm2, 2000L)) + listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm2, 2000L)) assert(listener.executorIdToStorageStatus.size === 2) assert(listener.executorIdToStorageStatus.get("fat").isDefined) assert(listener.executorIdToStorageStatus("fat").blockManagerId === bm2) @@ -50,11 +50,11 @@ class StorageStatusListenerSuite extends FunSuite { assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) // Block manager remove - listener.onBlockManagerRemoved(SparkListenerBlockManagerRemoved(bm1)) + listener.onBlockManagerRemoved(SparkListenerBlockManagerRemoved(1L, bm1)) assert(listener.executorIdToStorageStatus.size === 1) assert(!listener.executorIdToStorageStatus.get("big").isDefined) assert(listener.executorIdToStorageStatus.get("fat").isDefined) - listener.onBlockManagerRemoved(SparkListenerBlockManagerRemoved(bm2)) + listener.onBlockManagerRemoved(SparkListenerBlockManagerRemoved(1L, bm2)) assert(listener.executorIdToStorageStatus.size === 0) assert(!listener.executorIdToStorageStatus.get("big").isDefined) assert(!listener.executorIdToStorageStatus.get("fat").isDefined) @@ -62,8 +62,8 @@ class StorageStatusListenerSuite extends FunSuite { test("task end without updated blocks") { val listener = new StorageStatusListener - listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(bm1, 1000L)) - listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(bm2, 2000L)) + listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) + listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm2, 2000L)) val taskMetrics = new TaskMetrics // Task end with no updated blocks @@ -79,8 +79,8 @@ class StorageStatusListenerSuite extends FunSuite { test("task end with updated blocks") { val listener = new StorageStatusListener - listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(bm1, 1000L)) - listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(bm2, 2000L)) + listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) + listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm2, 2000L)) val taskMetrics1 = new TaskMetrics val taskMetrics2 = new TaskMetrics val block1 = (RDDBlockId(1, 1), BlockStatus(StorageLevel.DISK_ONLY, 0L, 100L, 0L)) @@ -128,7 +128,7 @@ class StorageStatusListenerSuite extends FunSuite { test("unpersist RDD") { val listener = new StorageStatusListener - listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(bm1, 1000L)) + listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) val taskMetrics1 = new TaskMetrics val taskMetrics2 = new TaskMetrics val block1 = (RDDBlockId(1, 1), BlockStatus(StorageLevel.DISK_ONLY, 0L, 100L, 0L)) diff --git a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala index d9e9c70a8a9e7..e1bc1379b5d80 100644 --- a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala @@ -108,7 +108,7 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { val myRddInfo1 = rddInfo1 val myRddInfo2 = rddInfo2 val stageInfo0 = new StageInfo(0, 0, "0", 100, Seq(myRddInfo0, myRddInfo1, myRddInfo2), "details") - bus.postToAll(SparkListenerBlockManagerAdded(bm1, 1000L)) + bus.postToAll(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) assert(storageListener._rddInfoMap.size === 3) assert(storageListener.rddInfoList.size === 0) // not cached @@ -175,7 +175,7 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { val block1 = (RDDBlockId(1, 1), BlockStatus(memOnly, 200L, 0L, 0L)) taskMetrics0.updatedBlocks = Some(Seq(block0)) taskMetrics1.updatedBlocks = Some(Seq(block1)) - bus.postToAll(SparkListenerBlockManagerAdded(bm1, 1000L)) + bus.postToAll(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) assert(storageListener.rddInfoList.size === 0) bus.postToAll(SparkListenerTaskEnd(0, 0, "big", Success, taskInfo, taskMetrics0)) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 66a17de9ec9ce..c84bafce37f70 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -21,6 +21,9 @@ import java.util.Properties import scala.collection.Map +import org.json4s.DefaultFormats +import org.json4s.JsonDSL._ +import org.json4s.JsonAST._ import org.json4s.jackson.JsonMethods._ import org.scalatest.FunSuite @@ -52,9 +55,9 @@ class JsonProtocolSuite extends FunSuite { "System Properties" -> Seq(("Username", "guest"), ("Password", "guest")), "Classpath Entries" -> Seq(("Super library", "/tmp/super_library")) )) - val blockManagerAdded = SparkListenerBlockManagerAdded( + val blockManagerAdded = SparkListenerBlockManagerAdded(1L, BlockManagerId("Stars", "In your multitude...", 300), 500) - val blockManagerRemoved = SparkListenerBlockManagerRemoved( + val blockManagerRemoved = SparkListenerBlockManagerRemoved(2L, BlockManagerId("Scarce", "to be counted...", 100)) val unpersistRdd = SparkListenerUnpersistRDD(12345) val applicationStart = SparkListenerApplicationStart("The winner of all", 42L, "Garfield") @@ -151,6 +154,28 @@ class JsonProtocolSuite extends FunSuite { assert(newMetrics.inputMetrics.isEmpty) } + test("BlockManager events backward compatibility") { + // SparkListenerBlockManagerAdded/Removed in Spark 1.0.0 do not have a "time" property. + val blockManagerAdded = SparkListenerBlockManagerAdded(1L, + BlockManagerId("Stars", "In your multitude...", 300), 500) + val blockManagerRemoved = SparkListenerBlockManagerRemoved(2L, + BlockManagerId("Scarce", "to be counted...", 100)) + + val oldBmAdded = JsonProtocol.blockManagerAddedToJson(blockManagerAdded) + .removeField({ _._1 == "Timestamp" }) + + val deserializedBmAdded = JsonProtocol.blockManagerAddedFromJson(oldBmAdded) + assert(SparkListenerBlockManagerAdded(-1L, blockManagerAdded.blockManagerId, + blockManagerAdded.maxMem) === deserializedBmAdded) + + val oldBmRemoved = JsonProtocol.blockManagerRemovedToJson(blockManagerRemoved) + .removeField({ _._1 == "Timestamp" }) + + val deserializedBmRemoved = JsonProtocol.blockManagerRemovedFromJson(oldBmRemoved) + assert(SparkListenerBlockManagerRemoved(-1L, blockManagerRemoved.blockManagerId) === + deserializedBmRemoved) + } + /** -------------------------- * | Helper test running methods | @@ -242,8 +267,10 @@ class JsonProtocolSuite extends FunSuite { assertEquals(e1.environmentDetails, e2.environmentDetails) case (e1: SparkListenerBlockManagerAdded, e2: SparkListenerBlockManagerAdded) => assert(e1.maxMem === e2.maxMem) + assert(e1.time === e2.time) assertEquals(e1.blockManagerId, e2.blockManagerId) case (e1: SparkListenerBlockManagerRemoved, e2: SparkListenerBlockManagerRemoved) => + assert(e1.time === e2.time) assertEquals(e1.blockManagerId, e2.blockManagerId) case (e1: SparkListenerUnpersistRDD, e2: SparkListenerUnpersistRDD) => assert(e1.rddId == e2.rddId) @@ -945,7 +972,8 @@ class JsonProtocolSuite extends FunSuite { | "Host": "In your multitude...", | "Port": 300 | }, - | "Maximum Memory": 500 + | "Maximum Memory": 500, + | "Timestamp": 1 |} """ @@ -957,7 +985,8 @@ class JsonProtocolSuite extends FunSuite { | "Executor ID": "Scarce", | "Host": "to be counted...", | "Port": 100 - | } + | }, + | "Timestamp": 2 |} """ From f2b5b619a9efee91573c0e546792e68e72afce21 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 3 Sep 2014 14:57:38 -0700 Subject: [PATCH 15/39] [SPARK-3388] Expose aplication ID in ApplicationStart event, use it in history server. This change exposes the application ID generated by the Spark Master, Mesos or Yarn via the SparkListenerApplicationStart event. It then uses that information to expose the application via its ID in the history server, instead of using the internal directory name generated by the event logger as an application id. This allows someone who knows the application ID to easily figure out the URL for the application's entry in the HS, aside from looking better. In Yarn mode, this is used to generate a direct link from the RM application list to the Spark history server entry (thus providing a fix for SPARK-2150). Note this sort of assumes that the different managers will generate app ids that are sufficiently different from each other that clashes will not occur. Author: Marcelo Vanzin This patch had conflicts when merged, resolved by Committer: Andrew Or Closes #1218 from vanzin/yarn-hs-link-2 and squashes the following commits: 2d19f3c [Marcelo Vanzin] Review feedback. 6706d3a [Marcelo Vanzin] Implement applicationId() in base classes. 56fe42e [Marcelo Vanzin] Fix cluster mode history address, plus a cleanup. 44112a8 [Marcelo Vanzin] Merge branch 'master' into yarn-hs-link-2 8278316 [Marcelo Vanzin] Merge branch 'master' into yarn-hs-link-2 a86bbcf [Marcelo Vanzin] Merge branch 'master' into yarn-hs-link-2 a0056e6 [Marcelo Vanzin] Unbreak test. 4b10cfd [Marcelo Vanzin] Merge branch 'master' into yarn-hs-link-2 cb0cab2 [Marcelo Vanzin] Merge branch 'master' into yarn-hs-link-2 25f2826 [Marcelo Vanzin] Add MIMA excludes. f0ba90f [Marcelo Vanzin] Use BufferedIterator. c90a08d [Marcelo Vanzin] Remove unused code. 3f8ec66 [Marcelo Vanzin] Review feedback. 21aa71b [Marcelo Vanzin] Fix JSON test. b022bae [Marcelo Vanzin] Undo SparkContext cleanup. c6d7478 [Marcelo Vanzin] Merge branch 'master' into yarn-hs-link-2 4e3483f [Marcelo Vanzin] Fix test. 57517b8 [Marcelo Vanzin] Review feedback. Mostly, more consistent use of Scala's Option. 311e49d [Marcelo Vanzin] Merge branch 'master' into yarn-hs-link-2 d35d86f [Marcelo Vanzin] Fix yarn backend after rebase. 36dc362 [Marcelo Vanzin] Don't use Iterator::takeWhile(). 0afd696 [Marcelo Vanzin] Wait until master responds before returning from start(). abc4697 [Marcelo Vanzin] Make FsHistoryProvider keep a map of applications by id. 26b266e [Marcelo Vanzin] Use Mesos framework ID as Spark application ID. b3f3664 [Marcelo Vanzin] [yarn] Make the RM link point to the app direcly in the HS. 2fb7de4 [Marcelo Vanzin] Expose the application ID in the ApplicationStart event. ed10348 [Marcelo Vanzin] Expose application id to spark context. --- .../scala/org/apache/spark/SparkContext.scala | 5 +- .../history/ApplicationHistoryProvider.scala | 6 +- .../deploy/history/FsHistoryProvider.scala | 176 +++++++++++------- .../spark/deploy/history/HistoryServer.scala | 5 +- .../scheduler/ApplicationEventListener.scala | 35 ++-- .../spark/scheduler/SchedulerBackend.scala | 8 + .../spark/scheduler/SparkListener.scala | 4 +- .../spark/scheduler/TaskScheduler.scala | 8 + .../spark/scheduler/TaskSchedulerImpl.scala | 4 + .../CoarseGrainedSchedulerBackend.scala | 4 +- .../cluster/SimrSchedulerBackend.scala | 1 + .../cluster/SparkDeploySchedulerBackend.scala | 28 +++ .../mesos/CoarseMesosSchedulerBackend.scala | 1 + .../cluster/mesos/MesosSchedulerBackend.scala | 1 + .../spark/scheduler/local/LocalBackend.scala | 1 + .../scala/org/apache/spark/ui/UIUtils.scala | 2 +- .../org/apache/spark/util/JsonProtocol.scala | 4 +- .../scheduler/EventLoggingListenerSuite.scala | 3 +- .../spark/scheduler/ReplayListenerSuite.scala | 3 +- .../apache/spark/util/JsonProtocolSuite.scala | 9 +- project/MimaExcludes.scala | 6 +- .../spark/deploy/yarn/ApplicationMaster.scala | 24 ++- .../deploy/yarn/YarnSparkHadoopUtil.scala | 14 -- .../cluster/YarnClientSchedulerBackend.scala | 4 +- .../cluster/YarnClusterSchedulerBackend.scala | 5 +- 25 files changed, 228 insertions(+), 133 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index cb4fb7cfbd32f..529febff94196 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1261,7 +1261,10 @@ class SparkContext(config: SparkConf) extends Logging { /** Post the application start event */ private def postApplicationStart() { - listenerBus.post(SparkListenerApplicationStart(appName, startTime, sparkUser)) + // Note: this code assumes that the task scheduler has been initialized and has contacted + // the cluster manager to get an application ID (in case the cluster manager provides one). + listenerBus.post(SparkListenerApplicationStart(appName, taskScheduler.applicationId(), + startTime, sparkUser)) } /** Post the application end event */ diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index a0e8bd403a41d..fbe39b27649f6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -34,15 +34,15 @@ private[spark] abstract class ApplicationHistoryProvider { * * @return List of all know applications. */ - def getListing(): Seq[ApplicationHistoryInfo] + def getListing(): Iterable[ApplicationHistoryInfo] /** * Returns the Spark UI for a specific application. * * @param appId The application ID. - * @return The application's UI, or null if application is not found. + * @return The application's UI, or None if application is not found. */ - def getAppUI(appId: String): SparkUI + def getAppUI(appId: String): Option[SparkUI] /** * Called when the server is shutting down. diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 05c8a90782c74..481f6c93c6a8d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -32,6 +32,8 @@ import org.apache.spark.util.Utils private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider with Logging { + private val NOT_STARTED = "" + // Interval between each check for event log updates private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", conf.getInt("spark.history.updateInterval", 10)) * 1000 @@ -47,8 +49,15 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis // A timestamp of when the disk was last accessed to check for log updates private var lastLogCheckTimeMs = -1L - // List of applications, in order from newest to oldest. - @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil + // The modification time of the newest log detected during the last scan. This is used + // to ignore logs that are older during subsequent scans, to avoid processing data that + // is already known. + private var lastModifiedTime = -1L + + // Mapping of application IDs to their metadata, in descending end time order. Apps are inserted + // into the map in order, so the LinkedHashMap maintains the correct ordering. + @volatile private var applications: mutable.LinkedHashMap[String, FsApplicationHistoryInfo] + = new mutable.LinkedHashMap() /** * A background thread that periodically checks for event log updates on disk. @@ -93,15 +102,35 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis logCheckingThread.start() } - override def getListing() = appList + override def getListing() = applications.values - override def getAppUI(appId: String): SparkUI = { + override def getAppUI(appId: String): Option[SparkUI] = { try { - val appLogDir = fs.getFileStatus(new Path(resolvedLogDir.toString, appId)) - val (_, ui) = loadAppInfo(appLogDir, renderUI = true) - ui + applications.get(appId).map { info => + val (replayBus, appListener) = createReplayBus(fs.getFileStatus( + new Path(logDir, info.logDir))) + val ui = { + val conf = this.conf.clone() + val appSecManager = new SecurityManager(conf) + new SparkUI(conf, appSecManager, replayBus, appId, + s"${HistoryServer.UI_PATH_PREFIX}/$appId") + // Do not call ui.bind() to avoid creating a new server for each application + } + + replayBus.replay() + + ui.setAppName(s"${appListener.appName.getOrElse(NOT_STARTED)} ($appId)") + + val uiAclsEnabled = conf.getBoolean("spark.history.ui.acls.enable", false) + ui.getSecurityManager.setAcls(uiAclsEnabled) + // make sure to set admin acls before view acls so they are properly picked up + ui.getSecurityManager.setAdminAcls(appListener.adminAcls.getOrElse("")) + ui.getSecurityManager.setViewAcls(appListener.sparkUser.getOrElse(NOT_STARTED), + appListener.viewAcls.getOrElse("")) + ui + } } catch { - case e: FileNotFoundException => null + case e: FileNotFoundException => None } } @@ -119,84 +148,79 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis try { val logStatus = fs.listStatus(new Path(resolvedLogDir)) val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]() - val logInfos = logDirs.filter { dir => - fs.isFile(new Path(dir.getPath, EventLoggingListener.APPLICATION_COMPLETE)) - } - val currentApps = Map[String, ApplicationHistoryInfo]( - appList.map(app => app.id -> app):_*) - - // For any application that either (i) is not listed or (ii) has changed since the last time - // the listing was created (defined by the log dir's modification time), load the app's info. - // Otherwise just reuse what's already in memory. - val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size) - for (dir <- logInfos) { - val curr = currentApps.getOrElse(dir.getPath().getName(), null) - if (curr == null || curr.lastUpdated < getModificationTime(dir)) { + // Load all new logs from the log directory. Only directories that have a modification time + // later than the last known log directory will be loaded. + var newLastModifiedTime = lastModifiedTime + val logInfos = logDirs + .filter { dir => + if (fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))) { + val modTime = getModificationTime(dir) + newLastModifiedTime = math.max(newLastModifiedTime, modTime) + modTime > lastModifiedTime + } else { + false + } + } + .flatMap { dir => try { - val (app, _) = loadAppInfo(dir, renderUI = false) - newApps += app + val (replayBus, appListener) = createReplayBus(dir) + replayBus.replay() + Some(new FsApplicationHistoryInfo( + dir.getPath().getName(), + appListener.appId.getOrElse(dir.getPath().getName()), + appListener.appName.getOrElse(NOT_STARTED), + appListener.startTime.getOrElse(-1L), + appListener.endTime.getOrElse(-1L), + getModificationTime(dir), + appListener.sparkUser.getOrElse(NOT_STARTED))) } catch { - case e: Exception => logError(s"Failed to load app info from directory $dir.") + case e: Exception => + logInfo(s"Failed to load application log data from $dir.", e) + None + } + } + .sortBy { info => -info.endTime } + + lastModifiedTime = newLastModifiedTime + + // When there are new logs, merge the new list with the existing one, maintaining + // the expected ordering (descending end time). Maintaining the order is important + // to avoid having to sort the list every time there is a request for the log list. + if (!logInfos.isEmpty) { + val newApps = new mutable.LinkedHashMap[String, FsApplicationHistoryInfo]() + def addIfAbsent(info: FsApplicationHistoryInfo) = { + if (!newApps.contains(info.id)) { + newApps += (info.id -> info) } - } else { - newApps += curr } - } - appList = newApps.sortBy { info => -info.endTime } + val newIterator = logInfos.iterator.buffered + val oldIterator = applications.values.iterator.buffered + while (newIterator.hasNext && oldIterator.hasNext) { + if (newIterator.head.endTime > oldIterator.head.endTime) { + addIfAbsent(newIterator.next) + } else { + addIfAbsent(oldIterator.next) + } + } + newIterator.foreach(addIfAbsent) + oldIterator.foreach(addIfAbsent) + + applications = newApps + } } catch { case t: Throwable => logError("Exception in checking for event log updates", t) } } - /** - * Parse the application's logs to find out the information we need to build the - * listing page. - * - * When creating the listing of available apps, there is no need to load the whole UI for the - * application. The UI is requested by the HistoryServer (by calling getAppInfo()) when the user - * clicks on a specific application. - * - * @param logDir Directory with application's log files. - * @param renderUI Whether to create the SparkUI for the application. - * @return A 2-tuple `(app info, ui)`. `ui` will be null if `renderUI` is false. - */ - private def loadAppInfo(logDir: FileStatus, renderUI: Boolean) = { - val path = logDir.getPath - val appId = path.getName + private def createReplayBus(logDir: FileStatus): (ReplayListenerBus, ApplicationEventListener) = { + val path = logDir.getPath() val elogInfo = EventLoggingListener.parseLoggingInfo(path, fs) val replayBus = new ReplayListenerBus(elogInfo.logPaths, fs, elogInfo.compressionCodec) val appListener = new ApplicationEventListener replayBus.addListener(appListener) - - val ui: SparkUI = if (renderUI) { - val conf = this.conf.clone() - val appSecManager = new SecurityManager(conf) - new SparkUI(conf, appSecManager, replayBus, appId, - HistoryServer.UI_PATH_PREFIX + s"/$appId") - // Do not call ui.bind() to avoid creating a new server for each application - } else { - null - } - - replayBus.replay() - val appInfo = ApplicationHistoryInfo( - appId, - appListener.appName, - appListener.startTime, - appListener.endTime, - getModificationTime(logDir), - appListener.sparkUser) - - if (ui != null) { - val uiAclsEnabled = conf.getBoolean("spark.history.ui.acls.enable", false) - ui.getSecurityManager.setAcls(uiAclsEnabled) - // make sure to set admin acls before view acls so properly picked up - ui.getSecurityManager.setAdminAcls(appListener.adminAcls) - ui.getSecurityManager.setViewAcls(appListener.sparkUser, appListener.viewAcls) - } - (appInfo, ui) + (replayBus, appListener) } /** Return when this directory was last modified. */ @@ -219,3 +243,13 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis private def getMonotonicTimeMs() = System.nanoTime() / (1000 * 1000) } + +private class FsApplicationHistoryInfo( + val logDir: String, + id: String, + name: String, + startTime: Long, + endTime: Long, + lastUpdated: Long, + sparkUser: String) + extends ApplicationHistoryInfo(id, name, startTime, endTime, lastUpdated, sparkUser) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index d1a64c1912cb8..ce00c0ffd21e0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -52,10 +52,7 @@ class HistoryServer( private val appLoader = new CacheLoader[String, SparkUI] { override def load(key: String): SparkUI = { - val ui = provider.getAppUI(key) - if (ui == null) { - throw new NoSuchElementException() - } + val ui = provider.getAppUI(key).getOrElse(throw new NoSuchElementException()) attachSparkUI(ui) ui } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala index 162158babc35b..6d39a5e3fa64c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala @@ -24,38 +24,31 @@ package org.apache.spark.scheduler * from multiple applications are seen, the behavior is unspecified. */ private[spark] class ApplicationEventListener extends SparkListener { - var appName = "" - var sparkUser = "" - var startTime = -1L - var endTime = -1L - var viewAcls = "" - var adminAcls = "" - - def applicationStarted = startTime != -1 - - def applicationCompleted = endTime != -1 - - def applicationDuration: Long = { - val difference = endTime - startTime - if (applicationStarted && applicationCompleted && difference > 0) difference else -1L - } + var appName: Option[String] = None + var appId: Option[String] = None + var sparkUser: Option[String] = None + var startTime: Option[Long] = None + var endTime: Option[Long] = None + var viewAcls: Option[String] = None + var adminAcls: Option[String] = None override def onApplicationStart(applicationStart: SparkListenerApplicationStart) { - appName = applicationStart.appName - startTime = applicationStart.time - sparkUser = applicationStart.sparkUser + appName = Some(applicationStart.appName) + appId = applicationStart.appId + startTime = Some(applicationStart.time) + sparkUser = Some(applicationStart.sparkUser) } override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { - endTime = applicationEnd.time + endTime = Some(applicationEnd.time) } override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { synchronized { val environmentDetails = environmentUpdate.environmentDetails val allProperties = environmentDetails("Spark Properties").toMap - viewAcls = allProperties.getOrElse("spark.ui.view.acls", "") - adminAcls = allProperties.getOrElse("spark.admin.acls", "") + viewAcls = allProperties.get("spark.ui.view.acls") + adminAcls = allProperties.get("spark.admin.acls") } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index e41e0a9841691..a0be8307eff27 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -31,4 +31,12 @@ private[spark] trait SchedulerBackend { def killTask(taskId: Long, executorId: String, interruptThread: Boolean): Unit = throw new UnsupportedOperationException def isReady(): Boolean = true + + /** + * The application ID associated with the job, if any. + * + * @return The application ID, or None if the backend does not provide an ID. + */ + def applicationId(): Option[String] = None + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index f33c2e065a200..86afe3bd5265f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -89,8 +89,8 @@ case class SparkListenerExecutorMetricsUpdate( extends SparkListenerEvent @DeveloperApi -case class SparkListenerApplicationStart(appName: String, time: Long, sparkUser: String) - extends SparkListenerEvent +case class SparkListenerApplicationStart(appName: String, appId: Option[String], time: Long, + sparkUser: String) extends SparkListenerEvent @DeveloperApi case class SparkListenerApplicationEnd(time: Long) extends SparkListenerEvent diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 1a0b877c8a5e1..1c1ce666eab0f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -64,4 +64,12 @@ private[spark] trait TaskScheduler { */ def executorHeartbeatReceived(execId: String, taskMetrics: Array[(Long, TaskMetrics)], blockManagerId: BlockManagerId): Boolean + + /** + * The application ID associated with the job, if any. + * + * @return The application ID, or None if the backend does not provide an ID. + */ + def applicationId(): Option[String] = None + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index ad051e59af86d..633e892554c50 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -491,6 +491,9 @@ private[spark] class TaskSchedulerImpl( } } } + + override def applicationId(): Option[String] = backend.applicationId() + } @@ -535,4 +538,5 @@ private[spark] object TaskSchedulerImpl { retval.toList } + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 2a3711ae2a78c..5b5257269d92f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -51,12 +51,12 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A val conf = scheduler.sc.conf private val timeout = AkkaUtils.askTimeout(conf) private val akkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) - // Submit tasks only after (registered resources / total expected resources) + // Submit tasks only after (registered resources / total expected resources) // is equal to at least this value, that is double between 0 and 1. var minRegisteredRatio = math.min(1, conf.getDouble("spark.scheduler.minRegisteredResourcesRatio", 0)) // Submit tasks after maxRegisteredWaitingTime milliseconds - // if minRegisteredRatio has not yet been reached + // if minRegisteredRatio has not yet been reached val maxRegisteredWaitingTime = conf.getInt("spark.scheduler.maxRegisteredResourcesWaitingTime", 30000) val createTime = System.currentTimeMillis() diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala index bc7670f4a804d..513d74a08a47f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala @@ -69,4 +69,5 @@ private[spark] class SimrSchedulerBackend( fs.delete(new Path(driverFilePath), false) super.stop() } + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 32138e5246700..06872ace2ecf4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -34,6 +34,10 @@ private[spark] class SparkDeploySchedulerBackend( var client: AppClient = null var stopping = false var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _ + var appId: String = _ + + val registrationLock = new Object() + var registrationDone = false val maxCores = conf.getOption("spark.cores.max").map(_.toInt) val totalExpectedCores = maxCores.getOrElse(0) @@ -68,6 +72,8 @@ private[spark] class SparkDeploySchedulerBackend( client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf) client.start() + + waitForRegistration() } override def stop() { @@ -81,15 +87,19 @@ private[spark] class SparkDeploySchedulerBackend( override def connected(appId: String) { logInfo("Connected to Spark cluster with app ID " + appId) + this.appId = appId + notifyContext() } override def disconnected() { + notifyContext() if (!stopping) { logWarning("Disconnected from Spark cluster! Waiting for reconnection...") } } override def dead(reason: String) { + notifyContext() if (!stopping) { logError("Application has been killed. Reason: " + reason) scheduler.error(reason) @@ -116,4 +126,22 @@ private[spark] class SparkDeploySchedulerBackend( override def sufficientResourcesRegistered(): Boolean = { totalCoreCount.get() >= totalExpectedCores * minRegisteredRatio } + + override def applicationId(): Option[String] = Option(appId) + + private def waitForRegistration() = { + registrationLock.synchronized { + while (!registrationDone) { + registrationLock.wait() + } + } + } + + private def notifyContext() = { + registrationLock.synchronized { + registrationDone = true + registrationLock.notifyAll() + } + } + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 87e181e773fdf..da43ef567608c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -309,4 +309,5 @@ private[spark] class CoarseMesosSchedulerBackend( logInfo("Executor lost: %s, marking slave %s as lost".format(e.getValue, s.getValue)) slaveLost(d, s) } + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 67ee4d66f151b..a9ef126f5de0e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -349,4 +349,5 @@ private[spark] class MesosSchedulerBackend( // TODO: query Mesos for number of cores override def defaultParallelism() = sc.conf.getInt("spark.default.parallelism", 8) + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index bec9502f20466..9ea25c2bc7090 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -114,4 +114,5 @@ private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: override def statusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) { localActor ! StatusUpdate(taskId, state, serializedData) } + } diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index bee6dad3387e5..f0006b42aee4f 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -232,7 +232,7 @@ private[spark] object UIUtils extends Logging { def listingTable[T]( headers: Seq[String], generateDataRow: T => Seq[Node], - data: Seq[T], + data: Iterable[T], fixedWidth: Boolean = false): Seq[Node] = { var listingTableClass = TABLE_CLASS diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 1fc536b096996..b0754e3ce10db 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -171,6 +171,7 @@ private[spark] object JsonProtocol { def applicationStartToJson(applicationStart: SparkListenerApplicationStart): JValue = { ("Event" -> Utils.getFormattedClassName(applicationStart)) ~ ("App Name" -> applicationStart.appName) ~ + ("App ID" -> applicationStart.appId.map(JString(_)).getOrElse(JNothing)) ~ ("Timestamp" -> applicationStart.time) ~ ("User" -> applicationStart.sparkUser) } @@ -484,9 +485,10 @@ private[spark] object JsonProtocol { def applicationStartFromJson(json: JValue): SparkListenerApplicationStart = { val appName = (json \ "App Name").extract[String] + val appId = Utils.jsonOption(json \ "App ID").map(_.extract[String]) val time = (json \ "Timestamp").extract[Long] val sparkUser = (json \ "User").extract[String] - SparkListenerApplicationStart(appName, time, sparkUser) + SparkListenerApplicationStart(appName, appId, time, sparkUser) } def applicationEndFromJson(json: JValue): SparkListenerApplicationEnd = { diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 41e58a008c533..fead883793430 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -229,7 +229,8 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { val conf = getLoggingConf(logDirPath, compressionCodec) val eventLogger = new EventLoggingListener("test", conf) val listenerBus = new LiveListenerBus - val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", 125L, "Mickey") + val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None, + 125L, "Mickey") val applicationEnd = SparkListenerApplicationEnd(1000L) // A comprehensive test on JSON de/serialization of all events is in JsonProtocolSuite diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index 8f0ee9f4dbafd..7ab351d1b4d24 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -83,7 +83,8 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { val fstream = fileSystem.create(logFilePath) val cstream = codec.map(_.compressedOutputStream(fstream)).getOrElse(fstream) val writer = new PrintWriter(cstream) - val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", 125L, "Mickey") + val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None, + 125L, "Mickey") val applicationEnd = SparkListenerApplicationEnd(1000L) writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationStart)))) writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationEnd)))) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index c84bafce37f70..2b45d8b695853 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -60,7 +60,7 @@ class JsonProtocolSuite extends FunSuite { val blockManagerRemoved = SparkListenerBlockManagerRemoved(2L, BlockManagerId("Scarce", "to be counted...", 100)) val unpersistRdd = SparkListenerUnpersistRDD(12345) - val applicationStart = SparkListenerApplicationStart("The winner of all", 42L, "Garfield") + val applicationStart = SparkListenerApplicationStart("The winner of all", None, 42L, "Garfield") val applicationEnd = SparkListenerApplicationEnd(42L) testEvent(stageSubmitted, stageSubmittedJsonString) @@ -176,6 +176,13 @@ class JsonProtocolSuite extends FunSuite { deserializedBmRemoved) } + test("SparkListenerApplicationStart backwards compatibility") { + // SparkListenerApplicationStart in Spark 1.0.0 do not have an "appId" property. + val applicationStart = SparkListenerApplicationStart("test", None, 1L, "user") + val oldEvent = JsonProtocol.applicationStartToJson(applicationStart) + .removeField({ _._1 == "App ID" }) + assert(applicationStart === JsonProtocol.applicationStartFromJson(oldEvent)) + } /** -------------------------- * | Helper test running methods | diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index a2f1b3582ab71..855d5cc8cf3fd 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -111,6 +111,8 @@ object MimaExcludes { MimaBuild.excludeSparkClass("storage.Values") ++ MimaBuild.excludeSparkClass("storage.Entry") ++ MimaBuild.excludeSparkClass("storage.MemoryStore$Entry") ++ + // Class was missing "@DeveloperApi" annotation in 1.0. + MimaBuild.excludeSparkClass("scheduler.SparkListenerApplicationStart") ++ Seq( ProblemFilters.exclude[IncompatibleMethTypeProblem]( "org.apache.spark.mllib.tree.impurity.Gini.calculate"), @@ -119,14 +121,14 @@ object MimaExcludes { ProblemFilters.exclude[IncompatibleMethTypeProblem]( "org.apache.spark.mllib.tree.impurity.Variance.calculate") ) ++ - Seq ( // Package-private classes removed in SPARK-2341 + Seq( // Package-private classes removed in SPARK-2341 ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.BinaryLabelParser"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.BinaryLabelParser$"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.LabelParser"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.LabelParser$"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.MulticlassLabelParser"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.MulticlassLabelParser$") - ) ++ + ) ++ Seq( // package-private classes removed in MLlib ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.mllib.regression.GeneralizedLinearAlgorithm.org$apache$spark$mllib$regression$GeneralizedLinearAlgorithm$$prependOne") diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 8c548409719da..98039a20de245 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkEnv} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.AddWebUIFilter import org.apache.spark.util.{AkkaUtils, SignalLogger, Utils} @@ -70,6 +71,8 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, private val sparkContextRef = new AtomicReference[SparkContext](null) final def run(): Int = { + val appAttemptId = client.getAttemptId() + if (isDriver) { // Set the web ui port to be ephemeral for yarn so we don't conflict with // other spark processes running on the same box @@ -77,9 +80,12 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, // Set the master property to match the requested mode. System.setProperty("spark.master", "yarn-cluster") + + // Propagate the application ID so that YarnClusterSchedulerBackend can pick it up. + System.setProperty("spark.yarn.app.id", appAttemptId.getApplicationId().toString()) } - logInfo("ApplicationAttemptId: " + client.getAttemptId()) + logInfo("ApplicationAttemptId: " + appAttemptId) val cleanupHook = new Runnable { override def run() { @@ -151,13 +157,20 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, sparkContextRef.compareAndSet(sc, null) } - private def registerAM(uiAddress: String, uiHistoryAddress: String) = { + private def registerAM(uiAddress: String) = { val sc = sparkContextRef.get() + + val appId = client.getAttemptId().getApplicationId().toString() + val historyAddress = + sparkConf.getOption("spark.yarn.historyServer.address") + .map { address => s"${address}${HistoryServer.UI_PATH_PREFIX}/${appId}" } + .getOrElse("") + allocator = client.register(yarnConf, if (sc != null) sc.getConf else sparkConf, if (sc != null) sc.preferredNodeLocationData else Map(), uiAddress, - uiHistoryAddress) + historyAddress) allocator.allocateResources() reporterThread = launchReporterThread() @@ -175,7 +188,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, if (sc == null) { finish(FinalApplicationStatus.FAILED, "Timed out waiting for SparkContext.") } else { - registerAM(sc.ui.appUIHostPort, YarnSparkHadoopUtil.getUIHistoryAddress(sc, sparkConf)) + registerAM(sc.ui.appUIHostPort) try { userThread.join() } finally { @@ -190,8 +203,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, conf = sparkConf, securityManager = securityMgr)._1 actor = waitForSparkDriver() addAmIpFilter() - registerAM(sparkConf.get("spark.driver.appUIAddress", ""), - sparkConf.get("spark.driver.appUIHistoryAddress", "")) + registerAM(sparkConf.get("spark.driver.appUIAddress", "")) // In client mode the actor will stop the reporter thread. reporterThread.join() diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index ffe2731ca1d17..dc77f1236492d 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -36,7 +36,6 @@ import org.apache.hadoop.yarn.util.RackResolver import org.apache.hadoop.conf.Configuration import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.Utils @@ -156,19 +155,6 @@ object YarnSparkHadoopUtil { } } - def getUIHistoryAddress(sc: SparkContext, conf: SparkConf) : String = { - val eventLogDir = sc.eventLogger match { - case Some(logger) => logger.getApplicationLogDir() - case None => "" - } - val historyServerAddress = conf.get("spark.yarn.historyServer.address", "") - if (historyServerAddress != "" && eventLogDir != "") { - historyServerAddress + HistoryServer.UI_PATH_PREFIX + s"/$eventLogDir" - } else { - "" - } - } - /** * Escapes a string for inclusion in a command line executed by Yarn. Yarn executes commands * using `bash -c "command arg1 arg2"` and that means plain quoting doesn't really work. The diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index a5f537dd9de30..41c662cd7a6de 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -56,7 +56,6 @@ private[spark] class YarnClientSchedulerBackend( val driverPort = conf.get("spark.driver.port") val hostport = driverHost + ":" + driverPort conf.set("spark.driver.appUIAddress", sc.ui.appUIHostPort) - conf.set("spark.driver.appUIHistoryAddress", YarnSparkHadoopUtil.getUIHistoryAddress(sc, conf)) val argsArrayBuf = new ArrayBuffer[String]() argsArrayBuf += ( @@ -150,4 +149,7 @@ private[spark] class YarnClientSchedulerBackend( override def sufficientResourcesRegistered(): Boolean = { totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio } + + override def applicationId(): Option[String] = Option(appId).map(_.toString()) + } diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala index 55665220a6f96..39436d0999663 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala @@ -28,7 +28,7 @@ private[spark] class YarnClusterSchedulerBackend( extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) { var totalExpectedExecutors = 0 - + if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) { minRegisteredRatio = 0.8 } @@ -47,4 +47,7 @@ private[spark] class YarnClusterSchedulerBackend( override def sufficientResourcesRegistered(): Boolean = { totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio } + + override def applicationId(): Option[String] = sc.getConf.getOption("spark.yarn.app.id") + } From 2784822e4c63083a647cc2d6c7089065ef3b947d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 3 Sep 2014 16:58:19 -0700 Subject: [PATCH 16/39] [Minor] Fix outdated Spark version This is causing the event logs to include a file called SPARK_VERSION_1.0.0, which is not accurate. Author: Andrew Or Author: andrewor14 Closes #2255 from andrewor14/spark-version and squashes the following commits: 1fbdfe9 [andrewor14] Snapshot 805a1c8 [Andrew Or] JK. Update Spark version to 1.2.0 instead. bffbaab [Andrew Or] Update Spark version to 1.1.0 --- core/src/main/scala/org/apache/spark/SparkContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 529febff94196..6eaf6794764c7 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1297,7 +1297,7 @@ class SparkContext(config: SparkConf) extends Logging { */ object SparkContext extends Logging { - private[spark] val SPARK_VERSION = "1.0.0" + private[spark] val SPARK_VERSION = "1.2.0-SNAPSHOT" private[spark] val SPARK_JOB_DESCRIPTION = "spark.job.description" From 996b7434ee0d0c7c26987eb9cf050c139fdd2db2 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 3 Sep 2014 17:04:53 -0700 Subject: [PATCH 17/39] [SPARK-3345] Do correct parameters for ShuffleFileGroup In the method `newFileGroup` of class `FileShuffleBlockManager`, the parameters for creating new `ShuffleFileGroup` object is in wrong order. Because in current codes, the parameters `shuffleId` and `fileId` are not used. So it doesn't cause problem now. However it should be corrected for readability and avoid future problem. Author: Liang-Chi Hsieh Closes #2235 from viirya/correct_shufflefilegroup_params and squashes the following commits: fe72567 [Liang-Chi Hsieh] Do correct parameters for ShuffleFileGroup. --- .../org/apache/spark/shuffle/FileShuffleBlockManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala index 76e3932a9bb91..96facccd52373 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala @@ -155,7 +155,7 @@ class FileShuffleBlockManager(conf: SparkConf) val filename = physicalFileName(shuffleId, bucketId, fileId) blockManager.diskBlockManager.getFile(filename) } - val fileGroup = new ShuffleFileGroup(fileId, shuffleId, files) + val fileGroup = new ShuffleFileGroup(shuffleId, fileId, files) shuffleState.allFileGroups.add(fileGroup) fileGroup } From a5224079286d1777864cf9fa77330aadae10cd7b Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 3 Sep 2014 17:38:01 -0700 Subject: [PATCH 18/39] [SPARK-2419][Streaming][Docs] Updates to the streaming programming guide Updated the main streaming programming guide, and also added source-specific guides for Kafka, Flume, Kinesis. Author: Tathagata Das Author: Jacek Laskowski Closes #2254 from tdas/streaming-doc-fix and squashes the following commits: e45c6d7 [Jacek Laskowski] More fixes from an old PR 5125316 [Tathagata Das] Fixed links dc02f26 [Tathagata Das] Refactored streaming kinesis guide and made many other changes. acbc3e3 [Tathagata Das] Fixed links between streaming guides. cb7007f [Tathagata Das] Added Streaming + Flume integration guide. 9bd9407 [Tathagata Das] Updated streaming programming guide with additional information from SPARK-2419. --- docs/streaming-flume-integration.md | 132 +++++++ docs/streaming-kafka-integration.md | 42 +++ docs/streaming-kinesis-integration.md | 110 ++++++ docs/streaming-kinesis.md | 59 --- docs/streaming-programming-guide.md | 518 +++++++++++++++++--------- 5 files changed, 622 insertions(+), 239 deletions(-) create mode 100644 docs/streaming-flume-integration.md create mode 100644 docs/streaming-kafka-integration.md create mode 100644 docs/streaming-kinesis-integration.md delete mode 100644 docs/streaming-kinesis.md diff --git a/docs/streaming-flume-integration.md b/docs/streaming-flume-integration.md new file mode 100644 index 0000000000000..d57c3e0ef9ba0 --- /dev/null +++ b/docs/streaming-flume-integration.md @@ -0,0 +1,132 @@ +--- +layout: global +title: Spark Streaming + Flume Integration Guide +--- + +[Apache Flume](https://flume.apache.org/) is a distributed, reliable, and available service for efficiently collecting, aggregating, and moving large amounts of log data. Here we explain how to configure Flume and Spark Streaming to receive data from Flume. There are two approaches to this. + +## Approach 1: Flume-style Push-based Approach +Flume is designed to push data between Flume agents. In this approach, Spark Streaming essentially sets up a receiver that acts an Avro agent for Flume, to which Flume can push the data. Here are the configuration steps. + +#### General Requirements +Choose a machine in your cluster such that + +- When your Flume + Spark Streaming application is launched, one of the Spark workers must run on that machine. + +- Flume can be configured to push data to a port on that machine. + +Due to the push model, the streaming application needs to be up, with the receiver scheduled and listening on the chosen port, for Flume to be able push data. + +#### Configuring Flume +Configure Flume agent to send data to an Avro sink by having the following in the configuration file. + + agent.sinks = avroSink + agent.sinks.avroSink.type = avro + agent.sinks.avroSink.channel = memoryChannel + agent.sinks.avroSink.hostname = + agent.sinks.avroSink.port = + +See the [Flume's documentation](https://flume.apache.org/documentation.html) for more information about +configuring Flume agents. + +#### Configuring Spark Streaming Application +1. **Linking:** In your SBT/Maven projrect definition, link your streaming application against the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). + + groupId = org.apache.spark + artifactId = spark-streaming-flume_{{site.SCALA_BINARY_VERSION}} + version = {{site.SPARK_VERSION_SHORT}} + +2. **Programming:** In the streaming application code, import `FlumeUtils` and create input DStream as follows. + +
+
+ import org.apache.spark.streaming.flume._ + + val flumeStream = FlumeUtils.createStream(streamingContext, [chosen machine's hostname], [chosen port]) + + See the [API docs](api/scala/index.html#org.apache.spark.streaming.flume.FlumeUtils$) + and the [example]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala). +
+
+ import org.apache.spark.streaming.flume.*; + + JavaReceiverInputDStream flumeStream = + FlumeUtils.createStream(streamingContext, [chosen machine's hostname], [chosen port]); + + See the [API docs](api/java/index.html?org/apache/spark/streaming/flume/FlumeUtils.html) + and the [example]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java). +
+
+ + Note that the hostname should be the same as the one used by the resource manager in the + cluster (Mesos, YARN or Spark Standalone), so that resource allocation can match the names and launch + the receiver in the right machine. + +3. **Deploying:** Package `spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}` and its dependencies (except `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` which are provided by `spark-submit`) into the application JAR. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). + +## Approach 2 (Experimental): Pull-based Approach using a Custom Sink +Instead of Flume pushing data directly to Spark Streaming, this approach runs a custom Flume sink that allows the following. +- Flume pushes data into the sink, and the data stays buffered. +- Spark Streaming uses transactions to pull data from the sink. Transactions succeed only after data is received and replicated by Spark Streaming. +This ensures that better reliability and fault-tolerance than the previous approach. However, this requires configuring Flume to run a custom sink. Here are the configuration steps. + +#### General Requirements +Choose a machine that will run the custom sink in a Flume agent. The rest of the Flume pipeline is configured to send data to that agent. Machines in the Spark cluster should have access to the chosen machine running the custom sink. + +#### Configuring Flume +Configuring Flume on the chosen machine requires the following two steps. + +1. **Sink JARs**: Add the following JARs to Flume's classpath (see [Flume's documentation](https://flume.apache.org/documentation.html) to see how) in the machine designated to run the custom sink . + + (i) *Custom sink JAR*: Download the JAR corresponding to the following artifact (or [direct link](http://search.maven.org/remotecontent?filepath=org/apache/spark/spark-streaming-flume-sink_{{site.SCALA_BINARY_VERSION}}/{{site.SPARK_VERSION_SHORT}}/spark-streaming-flume-sink_{{site.SCALA_BINARY_VERSION}}-{{site.SPARK_VERSION_SHORT}}.jar)). + + groupId = org.apache.spark + artifactId = spark-streaming-flume-sink_{{site.SCALA_BINARY_VERSION}} + version = {{site.SPARK_VERSION_SHORT}} + + (ii) *Scala library JAR*: Download the Scala library JAR for Scala {{site.SCALA_VERSION}}. It can be found with the following artifact detail (or, [direct link](http://search.maven.org/remotecontent?filepath=org/scala-lang/scala-library/{{site.SCALA_VERSION}}/scala-library-{{site.SCALA_VERSION}}.jar)). + + groupId = org.scala-lang + artifactId = scala-library + version = {{site.SCALA_VERSION}} + +2. **Configuration file**: On that machine, configure Flume agent to send data to an Avro sink by having the following in the configuration file. + + agent.sinks = spark + agent.sinks.spark.type = org.apache.spark.streaming.flume.sink.SparkSink + agent.sinks.spark.hostname = + agent.sinks.spark.port = + agent.sinks.spark.channel = memoryChannel + + Also make sure that the upstream Flume pipeline is configured to send the data to the Flume agent running this sink. + +See the [Flume's documentation](https://flume.apache.org/documentation.html) for more information about +configuring Flume agents. + +#### Configuring Spark Streaming Application +1. **Linking:** In your SBT/Maven projrect definition, link your streaming application against the `spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}` (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide). + +2. **Programming:** In the streaming application code, import `FlumeUtils` and create input DStream as follows. + +
+
+ import org.apache.spark.streaming.flume._ + + val flumeStream = FlumeUtils.createPollingStream(streamingContext, [sink machine hostname], [sink port]) +
+
+ import org.apache.spark.streaming.flume.*; + + JavaReceiverInputDStreamflumeStream = + FlumeUtils.createPollingStream(streamingContext, [sink machine hostname], [sink port]); +
+
+ + See the Scala example [FlumePollingEventCount]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/streaming/FlumePollingEventCount.scala). + + Note that each input DStream can be configured to receive data from multiple sinks. + +3. **Deploying:** Package `spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}` and its dependencies (except `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` which are provided by `spark-submit`) into the application JAR. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). + + + diff --git a/docs/streaming-kafka-integration.md b/docs/streaming-kafka-integration.md new file mode 100644 index 0000000000000..a3b705d4c31d0 --- /dev/null +++ b/docs/streaming-kafka-integration.md @@ -0,0 +1,42 @@ +--- +layout: global +title: Spark Streaming + Kafka Integration Guide +--- +[Apache Kafka](http://kafka.apache.org/) is publish-subscribe messaging rethought as a distributed, partitioned, replicated commit log service. Here we explain how to configure Spark Streaming to receive data from Kafka. + +1. **Linking:** In your SBT/Maven projrect definition, link your streaming application against the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). + + groupId = org.apache.spark + artifactId = spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}} + version = {{site.SPARK_VERSION_SHORT}} + +2. **Programming:** In the streaming application code, import `KafkaUtils` and create input DStream as follows. + +
+
+ import org.apache.spark.streaming.kafka._ + + val kafkaStream = KafkaUtils.createStream( + streamingContext, [zookeeperQuorum], [group id of the consumer], [per-topic number of Kafka partitions to consume]) + + See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$) + and the [example]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala). +
+
+ import org.apache.spark.streaming.kafka.*; + + JavaPairReceiverInputDStream kafkaStream = KafkaUtils.createStream( + streamingContext, [zookeeperQuorum], [group id of the consumer], [per-topic number of Kafka partitions to consume]); + + See the [API docs](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html) + and the [example]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java). +
+
+ + *Points to remember:* + + - Topic partitions in Kafka does not correlate to partitions of RDDs generated in Spark Streaming. So increasing the number of topic-specific partitions in the `KafkaUtils.createStream()` only increases the number of threads using which topics that are consumed within a single receiver. It does not increase the parallelism of Spark in processing the data. Refer to the main document for more information on that. + + - Multiple Kafka input DStreams can be created with different groups and topics for parallel receiving of data using multiple receivers. + +3. **Deploying:** Package `spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}` and its dependencies (except `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` which are provided by `spark-submit`) into the application JAR. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). diff --git a/docs/streaming-kinesis-integration.md b/docs/streaming-kinesis-integration.md new file mode 100644 index 0000000000000..079d4c5550537 --- /dev/null +++ b/docs/streaming-kinesis-integration.md @@ -0,0 +1,110 @@ +--- +layout: global +title: Spark Streaming + Kinesis Integration +--- +[Amazon Kinesis](http://aws.amazon.com/kinesis/) is a fully managed service for real-time processing of streaming data at massive scale. +The Kinesis input DStream and receiver uses the Kinesis Client Library (KCL) provided by Amazon under the Amazon Software License (ASL). +The KCL builds on top of the Apache 2.0 licensed AWS Java SDK and provides load-balancing, fault-tolerance, checkpointing through the concept of Workers, Checkpoints, and Shard Leases. +Here we explain how to configure Spark Streaming to receive data from Kinesis. + +#### Configuring Kinesis + +A Kinesis stream can be set up at one of the valid Kinesis endpoints with 1 or more shards per the following +[guide](http://docs.aws.amazon.com/kinesis/latest/dev/step-one-create-stream.html). + + +#### Configuring Spark Streaming Application + +1. **Linking:** In your SBT/Maven projrect definition, link your streaming application against the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). + + groupId = org.apache.spark + artifactId = spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}} + version = {{site.SPARK_VERSION_SHORT}} + + **Note that by linking to this library, you will include [ASL](https://aws.amazon.com/asl/)-licensed code in your application.** + +2. **Programming:** In the streaming application code, import `KinesisUtils` and create input DStream as follows. + +
+
+ import org.apache.spark.streaming.kinesis._ + import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream + + val kinesisStream = KinesisUtils.createStream( + streamingContext, [Kinesis stream name], [endpoint URL], [checkpoint interval], [initial position]) + + See the [API docs](api/scala/index.html#org.apache.spark.streaming.kinesis.KinesisUtils$) + and the [example]({{site.SPARK_GITHUB_URL}}/tree/master/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala). Refer to the next subsection for instructions to run the example. + +
+
+ import org.apache.spark.streaming.flume.*; + + JavaReceiverInputDStream kinesisStream = KinesisUtils.createStream( + streamingContext, [Kinesis stream name], [endpoint URL], [checkpoint interval], [initial position]); + + See the [API docs](api/java/index.html?org/apache/spark/streaming/kinesis/KinesisUtils.html) + and the [example]({{site.SPARK_GITHUB_URL}}/tree/master/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java). Refer to the next subsection for instructions to run the example. + +
+
+ + `[endpoint URL]`: Valid Kinesis endpoints URL can be found [here](http://docs.aws.amazon.com/general/latest/gr/rande.html#ak_region). + + `[checkpoint interval]`: The interval at which the Kinesis client library is going to save its position in the stream. For starters, set it to the same as the batch interval of the streaming application. + + `[initial position]`: Can be either `InitialPositionInStream.TRIM_HORIZON` or `InitialPositionInStream.LATEST` (see later section and Amazon Kinesis API documentation for more details). + + *Points to remember:* + + - The name used in the context of the streaming application must be unique for a given account and region. Changing the app name or stream name could lead to Kinesis errors as only a single logical application can process a single stream. + - A single Kinesis input DStream can receive many Kinesis shards by spinning up multiple KinesisRecordProcessor threads. Note that there is no correlation between number of shards in Kinesis and the number of partitions in the generated RDDs that is used for processing the data. + - You never need more KinesisReceivers than the number of shards in your stream as each will spin up at least one KinesisRecordProcessor thread. + - Horizontal scaling is achieved by autoscaling additional Kinesis input DStreams (separate processes) up to the number of current shards for a given stream, of course. + +3. **Deploying:** Package `spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}` and its dependencies (except `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` which are provided by `spark-submit`) into the application JAR. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). + + - A DynamoDB table and CloudWatch namespace are created during KCL initialization using this Kinesis application name. This DynamoDB table lives in the us-east-1 region regardless of the Kinesis endpoint URL. It is used to store KCL's checkpoint information. + + - If you are seeing errors after changing the app name or stream name, it may be necessary to manually delete the DynamoDB table and start from scratch. + +#### Running the Example +To run the example, +- Download Spark source and follow the [instructions](building-with-maven.html) to build Spark with profile *-Pkinesis-asl*. + + mvn -Pkinesis-asl -DskipTests clean package + +- Set up Kinesis stream (see earlier section). Note the name of the Kinesis stream, and the endpoint URL corresponding to the region the stream is based on. + +- Set up the environment variables AWS_ACCESS_KEY_ID and AWS_SECRET_KEY with your AWS credentials. + +- In the Spark root directory, run the example as +
+
+ + bin/run-example streaming.KinesisWordCountASL [Kinesis stream name] [endpoint URL] + +
+
+ + bin/run-example streaming.JavaKinesisWordCountASL [Kinesis stream name] [endpoint URL] + +
+
+ + This will wait for data to be received from Kinesis. + +- To generate random string data, in another terminal, run the associated Kinesis data producer. + + bin/run-example streaming.KinesisWordCountProducerASL [Kinesis stream name] [endpoint URL] 1000 10 + + This will push random words to the Kinesis stream, which should then be received and processed by the running example. + +#### Kinesis Checkpointing +The Kinesis receiver checkpoints the position of the stream that has been read periodically, so that the system can recover from failures and continue processing where it had left off. Checkpointing too frequently will cause excess load on the AWS checkpoint storage layer and may lead to AWS throttling. The provided example handles this throttling with a random-backoff-retry strategy. + +- If no Kinesis checkpoint info exists, the KinesisReceiver will start either from the oldest record available (InitialPositionInStream.TRIM_HORIZON) or from the latest tip (InitialPostitionInStream.LATEST). This is configurable. + +- InitialPositionInStream.LATEST could lead to missed records if data is added to the stream while no KinesisReceivers are running (and no checkpoint info is being stored). In production, you'll want to switch to InitialPositionInStream.TRIM_HORIZON which will read up to 24 hours (Kinesis limit) of previous stream data. + +- InitialPositionInStream.TRIM_HORIZON may lead to duplicate processing of records where the impact is dependent on checkpoint frequency. diff --git a/docs/streaming-kinesis.md b/docs/streaming-kinesis.md deleted file mode 100644 index 16ad3222105a2..0000000000000 --- a/docs/streaming-kinesis.md +++ /dev/null @@ -1,59 +0,0 @@ ---- -layout: global -title: Spark Streaming Kinesis Receiver ---- - -## Kinesis -###Design -
  • The KinesisReceiver uses the Kinesis Client Library (KCL) provided by Amazon under the Amazon Software License.
  • -
  • The KCL builds on top of the Apache 2.0 licensed AWS Java SDK and provides load-balancing, fault-tolerance, checkpointing through the concept of Workers, Checkpoints, and Shard Leases.
  • -
  • The KCL uses DynamoDB to maintain all state. A DynamoDB table is created in the us-east-1 region (regardless of Kinesis stream region) during KCL initialization for each Kinesis application name.
  • -
  • A single KinesisReceiver can process many shards of a stream by spinning up multiple KinesisRecordProcessor threads.
  • -
  • You never need more KinesisReceivers than the number of shards in your stream as each will spin up at least one KinesisRecordProcessor thread.
  • -
  • Horizontal scaling is achieved by autoscaling additional KinesisReceiver (separate processes) or spinning up new KinesisRecordProcessor threads within each KinesisReceiver - up to the number of current shards for a given stream, of course. Don't forget to autoscale back down!
  • - -### Build -
  • Spark supports a Streaming KinesisReceiver, but it is not included in the default build due to Amazon Software Licensing (ASL) restrictions.
  • -
  • To build with the Kinesis Streaming Receiver and supporting ASL-licensed code, you must run the maven or sbt builds with the **-Pkinesis-asl** profile.
  • -
  • All KinesisReceiver-related code, examples, tests, and artifacts live in **$SPARK_HOME/extras/kinesis-asl/**.
  • -
  • Kinesis-based Spark Applications will need to link to the **spark-streaming-kinesis-asl** artifact that is built when **-Pkinesis-asl** is specified.
  • -
  • _**Note that by linking to this library, you will include [ASL](https://aws.amazon.com/asl/)-licensed code in your Spark package**_.
  • - -###Example -
  • To build the Kinesis example, you must run the maven or sbt builds with the **-Pkinesis-asl** profile.
  • -
  • You need to setup a Kinesis stream at one of the valid Kinesis endpoints with 1 or more shards per the following: http://docs.aws.amazon.com/kinesis/latest/dev/step-one-create-stream.html
  • -
  • Valid Kinesis endpoints can be found here: http://docs.aws.amazon.com/general/latest/gr/rande.html#ak_region
  • -
  • When running **locally**, the example automatically determines the number of threads and KinesisReceivers to spin up based on the number of shards configured for the stream. Therefore, **local[n]** is not needed when starting the example as with other streaming examples.
  • -
  • While this example could use a single KinesisReceiver which spins up multiple KinesisRecordProcessor threads to process multiple shards, I wanted to demonstrate unioning multiple KinesisReceivers as a single DStream. (It's a bit confusing in local mode.)
  • -
  • **KinesisWordCountProducerASL** is provided to generate random records into the Kinesis stream for testing.
  • -
  • The example has been configured to immediately replicate incoming stream data to another node by using (StorageLevel.MEMORY_AND_DISK_2) -
  • Spark checkpointing is disabled because the example does not use any stateful or window-based DStream operations such as updateStateByKey and reduceByWindow. If those operations are introduced, you would need to enable checkpointing or risk losing data in the case of a failure.
  • -
  • Kinesis checkpointing is enabled. This means that the example will recover from a Kinesis failure.
  • -
  • The example uses InitialPositionInStream.LATEST strategy to pull from the latest tip of the stream if no Kinesis checkpoint info exists.
  • -
  • In our example, **KinesisWordCount** is the Kinesis application name for both the Scala and Java versions. The use of this application name is described next.
  • - -###Deployment and Runtime -
  • A Kinesis application name must be unique for a given account and region.
  • -
  • A DynamoDB table and CloudWatch namespace are created during KCL initialization using this Kinesis application name. http://docs.aws.amazon.com/kinesis/latest/dev/kinesis-record-processor-implementation-app.html#kinesis-record-processor-initialization
  • -
  • This DynamoDB table lives in the us-east-1 region regardless of the Kinesis endpoint URL.
  • -
  • Changing the app name or stream name could lead to Kinesis errors as only a single logical application can process a single stream.
  • -
  • If you are seeing errors after changing the app name or stream name, it may be necessary to manually delete the DynamoDB table and start from scratch.
  • -
  • The Kinesis libraries must be present on all worker nodes, as they will need access to the KCL.
  • -
  • The KinesisReceiver uses the DefaultAWSCredentialsProviderChain for AWS credentials which searches for credentials in the following order of precedence:
    -1) Environment Variables - AWS_ACCESS_KEY_ID and AWS_SECRET_KEY
    -2) Java System Properties - aws.accessKeyId and aws.secretKey
    -3) Credential profiles file - default location (~/.aws/credentials) shared by all AWS SDKs
    -4) Instance profile credentials - delivered through the Amazon EC2 metadata service -
  • - -###Fault-Tolerance -
  • The combination of Spark Streaming and Kinesis creates 2 different checkpoints that may occur at different intervals.
  • -
  • Checkpointing too frequently against Kinesis will cause excess load on the AWS checkpoint storage layer and may lead to AWS throttling. The provided example handles this throttling with a random backoff retry strategy.
  • -
  • Upon startup, a KinesisReceiver will begin processing records with sequence numbers greater than the last Kinesis checkpoint sequence number recorded per shard (stored in the DynamoDB table).
  • -
  • If no Kinesis checkpoint info exists, the KinesisReceiver will start either from the oldest record available (InitialPositionInStream.TRIM_HORIZON) or from the latest tip (InitialPostitionInStream.LATEST). This is configurable.
  • -
  • InitialPositionInStream.LATEST could lead to missed records if data is added to the stream while no KinesisReceivers are running (and no checkpoint info is being stored.)
  • -
  • In production, you'll want to switch to InitialPositionInStream.TRIM_HORIZON which will read up to 24 hours (Kinesis limit) of previous stream data.
  • -
  • InitialPositionInStream.TRIM_HORIZON may lead to duplicate processing of records where the impact is dependent on checkpoint frequency.
  • -
  • Record processing should be idempotent when possible.
  • -
  • A failed or latent KinesisRecordProcessor within the KinesisReceiver will be detected and automatically restarted by the KCL.
  • -
  • If possible, the KinesisReceiver should be shutdown cleanly in order to trigger a final checkpoint of all KinesisRecordProcessors to avoid duplicate record processing.
  • \ No newline at end of file diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 9f331ed50d2a4..3d4bce49666ed 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -7,12 +7,12 @@ title: Spark Streaming Programming Guide {:toc} # Overview -Spark Streaming is an extension of the core Spark API that allows enables high-throughput, +Spark Streaming is an extension of the core Spark API that allows enables scalable, high-throughput, fault-tolerant stream processing of live data streams. Data can be ingested from many sources like Kafka, Flume, Twitter, ZeroMQ, Kinesis or plain old TCP sockets and be processed using complex algorithms expressed with high-level functions like `map`, `reduce`, `join` and `window`. Finally, processed data can be pushed out to filesystems, databases, -and live dashboards. In fact, you can apply Spark's in-built +and live dashboards. In fact, you can apply Spark's [machine learning](mllib-guide.html) algorithms, and [graph processing](graphx-programming-guide.html) algorithms on data streams. @@ -60,35 +60,24 @@ do is as follows.
    First, we import the names of the Spark Streaming classes, and some implicit conversions from StreamingContext into our environment, to add useful methods to -other classes we need (like DStream). - -[StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) is the -main entry point for all streaming functionality. +other classes we need (like DStream). [StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) is the +main entry point for all streaming functionality. We create a local StreamingContext with two execution threads, and batch interval of 1 second. {% highlight scala %} +import org.apache.spark._ import org.apache.spark.streaming._ import org.apache.spark.streaming.StreamingContext._ -{% endhighlight %} - -Then we create a -[StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) object. -Besides Spark's configuration, we specify that any DStream will be processed -in 1 second batches. -{% highlight scala %} -import org.apache.spark.api.java.function._ -import org.apache.spark.streaming._ -import org.apache.spark.streaming.api._ -// Create a StreamingContext with a local master -// Spark Streaming needs at least two working thread -val ssc = new StreamingContext("local[2]", "NetworkWordCount", Seconds(1)) +// Create a local StreamingContext with two working thread and batch interval of 1 second +val conf = new SparkConf().setMaster("local[2]").setAppName("NetworkWordCount") +val ssc = new StreamingContext(conf, Seconds(1)) {% endhighlight %} -Using this context, we then create a new DStream -by specifying the IP address and port of the data server. +Using this context, we can create a DStream that represents streaming data from a TCP +source hostname, e.g. `localhost`, and port, e.g. `9999` {% highlight scala %} -// Create a DStream that will connect to serverIP:serverPort, like localhost:9999 +// Create a DStream that will connect to hostname:port, like localhost:9999 val lines = ssc.socketTextStream("localhost", 9999) {% endhighlight %} @@ -112,7 +101,7 @@ import org.apache.spark.streaming.StreamingContext._ val pairs = words.map(word => (word, 1)) val wordCounts = pairs.reduceByKey(_ + _) -// Print a few of the counts to the console +// Print the first ten elements of each RDD generated in this DStream to the console wordCounts.print() {% endhighlight %} @@ -139,23 +128,25 @@ The complete code can be found in the Spark Streaming example First, we create a [JavaStreamingContext](api/java/index.html?org/apache/spark/streaming/api/java/JavaStreamingContext.html) object, which is the main entry point for all streaming -functionality. Besides Spark's configuration, we specify that any DStream would be processed -in 1 second batches. +functionality. We create a local StreamingContext with two execution threads, and a batch interval of 1 second. {% highlight java %} +import org.apache.spark.*; import org.apache.spark.api.java.function.*; import org.apache.spark.streaming.*; import org.apache.spark.streaming.api.java.*; import scala.Tuple2; -// Create a StreamingContext with a local master -JavaStreamingContext jssc = new JavaStreamingContext("local[2]", "JavaNetworkWordCount", new Duration(1000)) + +// Create a local StreamingContext with two working thread and batch interval of 1 second +val conf = new SparkConf().setMaster("local[2]").setAppName("NetworkWordCount") +JavaStreamingContext jssc = new JavaStreamingContext(conf, new Duration(1000)) {% endhighlight %} -Using this context, we then create a new DStream -by specifying the IP address and port of the data server. +Using this context, we can create a DStream that represents streaming data from a TCP +source hostname, e.g. `localhost`, and port, e.g. `9999` {% highlight java %} -// Create a DStream that will connect to serverIP:serverPort, like localhost:9999 +// Create a DStream that will connect to hostname:port, like localhost:9999 JavaReceiverInputDStream lines = jssc.socketTextStream("localhost", 9999); {% endhighlight %} @@ -197,7 +188,9 @@ JavaPairDStream wordCounts = pairs.reduceByKey( return i1 + i2; } }); -wordCounts.print(); // Print a few of the counts to the console + +// Print the first ten elements of each RDD generated in this DStream to the console +wordCounts.print(); {% endhighlight %} The `words` DStream is further mapped (one-to-one transformation) to a DStream of `(word, @@ -207,8 +200,8 @@ using a [Function2](api/scala/index.html#org.apache.spark.api.java.function.Func Finally, `wordCounts.print()` will print a few of the counts generated every second. Note that when these lines are executed, Spark Streaming only sets up the computation it -will perform when it is started, and no real processing has started yet. To start the processing -after all the transformations have been setup, we finally call +will perform after it is started, and no real processing has started yet. To start the processing +after all the transformations have been setup, we finally call `start` method. {% highlight java %} jssc.start(); // Start the computation @@ -235,12 +228,12 @@ Then, in a different terminal, you can start the example by using
    {% highlight bash %} -$ ./bin/run-example org.apache.spark.examples.streaming.NetworkWordCount localhost 9999 +$ ./bin/run-example streaming.NetworkWordCount localhost 9999 {% endhighlight %}
    {% highlight bash %} -$ ./bin/run-example org.apache.spark.examples.streaming.JavaNetworkWordCount localhost 9999 +$ ./bin/run-example JavaNetworkWordCount localhost 9999 {% endhighlight %}
    @@ -281,25 +274,11 @@ Time: 1357008430000 ms -You can also use Spark Streaming directly from the Spark shell: - -{% highlight bash %} -$ bin/spark-shell -{% endhighlight %} - -... and create your StreamingContext by wrapping the existing interactive shell -SparkContext object, `sc`: - -{% highlight scala %} -val ssc = new StreamingContext(sc, Seconds(1)) -{% endhighlight %} -When working with the shell, you may also need to send a `^D` to your netcat session -to force the pipeline to print the word counts to the console at the sink. - -*************************************************************************************************** +*************************************************************************************************** +*************************************************************************************************** -# Basics +# Basic Concepts Next, we move beyond the simple example and elaborate on the basics of Spark Streaming that you need to know to write your streaming applications. @@ -319,68 +298,120 @@ Streaming core artifact `spark-streaming-xyz_{{site.SCALA_BINARY_VERSION}}` to the dependencies. For example, some of the common ones are as follows. - + - - +
    SourceArtifact
    Kafka spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}
    Flume spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}
    Kinesis
    spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}}
    Twitter spark-streaming-twitter_{{site.SCALA_BINARY_VERSION}}
    ZeroMQ spark-streaming-zeromq_{{site.SCALA_BINARY_VERSION}}
    MQTT spark-streaming-mqtt_{{site.SCALA_BINARY_VERSION}}
    Kinesis
    (built separately)
    kinesis-asl_{{site.SCALA_BINARY_VERSION}}
    For an up-to-date list, please refer to the -[Apache repository](http://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.apache.spark%22%20AND%20v%3A%22{{site.SPARK_VERSION}}%22) +[Apache repository](http://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.apache.spark%22%20AND%20v%3A%22{{site.SPARK_VERSION_SHORT}}%22) for the full list of supported sources and artifacts. -## Initializing +*** + +## Initializing StreamingContext + +To initialize a Spark Streaming program, a **StreamingContext** object has to be created which is the main entry point of all Spark Streaming functionality.
    -To initialize a Spark Streaming program in Scala, a -[`StreamingContext`](api/scala/index.html#org.apache.spark.streaming.StreamingContext) -object has to be created, which is the main entry point of all Spark Streaming functionality. -A `StreamingContext` object can be created by using +A [StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) object can be created from a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) object. {% highlight scala %} -new StreamingContext(master, appName, batchDuration, [sparkHome], [jars]) +import org.apache.spark._ +import org.apache.spark.streaming._ + +val conf = new SparkConf().setAppName(appName).setMaster(master) +val ssc = new StreamingContext(conf, Seconds(1)) {% endhighlight %} -
    -
    -To initialize a Spark Streaming program in Java, a -[`JavaStreamingContext`](api/scala/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) -object has to be created, which is the main entry point of all Spark Streaming functionality. -A `JavaStreamingContext` object can be created by using +The `appName` parameter is a name for your application to show on the cluster UI. +`master` is a [Spark, Mesos or YARN cluster URL](submitting-applications.html#master-urls), +or a special __"local[\*]"__ string to run in local mode. In practice, when running on a cluster, +you will not want to hardcode `master` in the program, +but rather [launch the application with `spark-submit`](submitting-applications.html) and +receive it there. However, for local testing and unit tests, you can pass "local[\*]" to run Spark Streaming +in-process (detects the number of cores in the local system). Note that this internally creates a [SparkContext](api/scala/index.html#org.apache.spark.SparkContext) (starting point of all Spark functionality) which can be accessed as `ssc.sparkContext`. + +The batch interval must be set based on the latency requirements of your application +and available cluster resources. See the [Performance Tuning](#setting-the-right-batch-size) +section for more details. + +A `StreamingContext` object can also be created from an existing `SparkContext` object. {% highlight scala %} -new JavaStreamingContext(master, appName, batchInterval, [sparkHome], [jars]) +import org.apache.spark.streaming._ + +val sc = ... // existing SparkContext +val ssc = new StreamingContext(sc, Seconds(1)) {% endhighlight %} + +
    -
    +
    -The `master` parameter is a standard [Spark cluster URL](programming-guide.html#master-urls) -and can be "local" for local testing. The `appName` is a name of your program, -which will be shown on your cluster's web UI. The `batchInterval` is the size of the batches, -as explained earlier. Finally, the last two parameters are needed to deploy your code to a cluster - if running in distributed mode, as described in the - [Spark programming guide](programming-guide.html#deploying-code-on-a-cluster). - Additionally, the underlying SparkContext can be accessed as -`ssc.sparkContext`. +A [JavaStreamingContext](api/java/index.html?org/apache/spark/streaming/api/java/JavaStreamingContext.html) object can be created from a [SparkConf](api/java/index.html?org/apache/spark/SparkConf.html) object. + +{% highlight java %} +import org.apache.spark.*; +import org.apache.spark.streaming.api.java.*; + +SparkConf conf = new SparkConf().setAppName(appName).setMaster(master); +JavaStreamingContext ssc = new JavaStreamingContext(conf, Duration(1000)); +{% endhighlight %} + +The `appName` parameter is a name for your application to show on the cluster UI. +`master` is a [Spark, Mesos or YARN cluster URL](submitting-applications.html#master-urls), +or a special __"local[\*]"__ string to run in local mode. In practice, when running on a cluster, +you will not want to hardcode `master` in the program, +but rather [launch the application with `spark-submit`](submitting-applications.html) and +receive it there. However, for local testing and unit tests, you can pass "local[*]" to run Spark Streaming +in-process. Note that this internally creates a [JavaSparkContext](api/java/index.html?org/apache/spark/api/java/JavaSparkContext.html) (starting point of all Spark functionality) which can be accessed as `ssc.sparkContext`. The batch interval must be set based on the latency requirements of your application and available cluster resources. See the [Performance Tuning](#setting-the-right-batch-size) section for more details. -## DStreams -*Discretized Stream* or *DStream* is the basic abstraction provided by Spark Streaming. +A `JavaStreamingContext` object can also be created from an existing `JavaSparkContext`. + +{% highlight java %} +import org.apache.spark.streaming.api.java.*; + +JavaSparkContext sc = ... //existing JavaSparkContext +JavaStreamingContext ssc = new JavaStreamingContext(sc, new Duration(1000)); +{% endhighlight %} +
    +
    + +After a context is defined, you have to do the follow steps. +1. Define the input sources. +1. Setup the streaming computations. +1. Start the receiving and procesing of data using `streamingContext.start()`. +1. The processing will continue until `streamingContext.stop()` is called. + +##### Points to remember: +{:.no_toc} +- Once a context has been started, no new streaming computations can be setup or added to it. +- Once a context has been stopped, it cannot be started (that is, re-used) again. +- Only one StreamingContext can be active in a JVM at the same time. +- stop() on StreamingContext also stops the SparkContext. To stop only the StreamingContext, set optional parameter of `stop()` called `stopSparkContext` to false. +- A SparkContext can be re-used to create multiple StreamingContexts, as long as the previous StreamingContext is stopped (without stopping the SparkContext) before the next StreamingContext is created. + +*** + +## Discretized Streams (DStreams) +**Discretized Stream** or **DStream** is the basic abstraction provided by Spark Streaming. It represents a continuous stream of data, either the input data stream received from source, or the processed data stream generated by transforming the input stream. Internally, -it is represented by a continuous sequence of RDDs, which is Spark's abstraction of an immutable, -distributed dataset. Each RDD in a DStream contains data from a certain interval, +a DStream is represented by a continuous series of RDDs, which is Spark's abstraction of an immutable, +distributed dataset (see [Spark Programming Guide](programming-guide.html#resilient-distributed-datasets-rdds) for more details). Each RDD in a DStream contains data from a certain interval, as shown in the following figure.

    @@ -392,8 +423,8 @@ as shown in the following figure. Any operation applied on a DStream translates to operations on the underlying RDDs. For example, in the [earlier example](#a-quick-example) of converting a stream of lines to words, -the `flatmap` operation is applied on each RDD in the `lines` DStream to generate the RDDs of the - `words` DStream. This is shown the following figure. +the `flatMap` operation is applied on each RDD in the `lines` DStream to generate the RDDs of the + `words` DStream. This is shown in the following figure.

    -

    -{% highlight scala %} -ssc.fileStream(dataDirectory) -{% endhighlight %} -
    -
    -{% highlight java %} -jssc.fileStream(dataDirectory); -{% endhighlight %} -
    - +
    +
    + streamingContext.fileStream[keyClass, valueClass, inputFormatClass](dataDirectory) +
    +
    + streamingContext.fileStream(dataDirectory); +
    +
    -Spark Streaming will monitor the directory `dataDirectory` for any Hadoop-compatible filesystem -and process any files created in that directory. Note that + Spark Streaming will monitor the directory `dataDirectory` and process any files created in that directory (files written in nested directories not supported). Note that - * The files must have the same data format. - * The files must be created in the `dataDirectory` by atomically *moving* or *renaming* them into - the data directory. - * Once moved the files must not be changed. + + The files must have the same data format. + + The files must be created in the `dataDirectory` by atomically *moving* or *renaming* them into + the data directory. + + Once moved, the files must not be changed. So if the files are being continuously appended, the new data will not be read. -For more details on streams from files, Akka actors and sockets, + For simple text files, there is an easier method `streamingContext.textFileStream(dataDirectory)`. And file streams do not require running a receiver, hence does not require allocating cores. + +- **Streams based on Custom Actors:** DStreams can be created with data streams received through Akka actors by using `streamingContext.actorStream(actorProps, actor-name)`. See the [Custom Receiver Guide](#implementing-and-using-a-custom-actor-based-receiver) for more details. + +- **Queue of RDDs as a Stream:** For testing a Spark Streaming application with test data, one can also create a DStream based on a queue of RDDs, using `streamingContext.queueStream(queueOfRDDs)`. Each RDD pushed into the queue will be treated as a batch of data in the DStream, and processed like a stream. + +For more details on streams from sockets, files, and actors, see the API documentations of the relevant functions in [StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) for -Scala and [JavaStreamingContext](api/scala/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) - for Java. +Scala and [JavaStreamingContext](api/java/index.html?org/apache/spark/streaming/api/java/JavaStreamingContext.html) for Java. + +### Advanced Sources +{:.no_toc} +This category of sources require interfacing with external non-Spark libraries, some of them with complex dependencies (e.g., Kafka and Flume). Hence, to minimize issues related to version conflicts of dependencies, the functionality to create DStreams from these sources have been moved to separate libraries, that can be [linked to](#linking) explicitly as necessary. For example, if you want to create a DStream using data from Twitter's stream of tweets, you have to do the following. -Additional functionality for creating DStreams from sources such as Kafka, Flume, Kinesis, and Twitter -can be imported by adding the right dependencies as explained in an -[earlier](#linking) section. To take the -case of Kafka, after adding the artifact `spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}` to the -project dependencies, you can create a DStream from Kafka as +1. *Linking*: Add the artifact `spark-streaming-twitter_{{site.SCALA_BINARY_VERSION}}` to the SBT/Maven project dependencies. +1. *Programming*: Import the `TwitterUtils` class and create a DStream with `TwitterUtils.createStream` as shown below. +1. *Deploying*: Generate an uber JAR with all the dependencies (including the dependency `spark-streaming-twitter_{{site.SCALA_BINARY_VERSION}}` and its transitive dependencies) and then deploy the application. This is further explained in the [Deploying section](#deploying-applications).
    {% highlight scala %} -import org.apache.spark.streaming.kafka._ -KafkaUtils.createStream(ssc, kafkaParams, ...) +import org.apache.spark.streaming.twitter._ + +TwitterUtils.createStream(ssc) {% endhighlight %}
    {% highlight java %} -import org.apache.spark.streaming.kafka.*; -KafkaUtils.createStream(jssc, kafkaParams, ...); +import org.apache.spark.streaming.twitter.*; + +TwitterUtils.createStream(jssc); {% endhighlight %}
    -For more details on these additional sources, see the corresponding [API documentation](#where-to-go-from-here). -Furthermore, you can also implement your own custom receiver for your sources. See the -[Custom Receiver Guide](streaming-custom-receivers.html). +Note that these advanced sources are not available in the `spark-shell`, hence applications based on these +advanced sources cannot be tested in the shell. + +Some of these advanced sources are as follows. + +- **Twitter:** Spark Streaming's TwitterUtils uses Twitter4j 3.0.3 to get the public stream of tweets using + [Twitter's Streaming API](https://dev.twitter.com/docs/streaming-apis). Authentication information + can be provided by any of the [methods](http://twitter4j.org/en/configuration.html) supported by + Twitter4J library. You can either get the public stream, or get the filtered stream based on a + keywords. See the API documentation ([Scala](api/scala/index.html#org.apache.spark.streaming.twitter.TwitterUtils$), [Java](api/java/index.html?org/apache/spark/streaming/twitter/TwitterUtils.html)) and examples ([TwitterPopularTags]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala) and + [TwitterAlgebirdCMS]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala)). + +- **Flume:** Spark Streaming {{site.SPARK_VERSION_SHORT}} can received data from Flume 1.4.0. See the [Flume Integration Guide](streaming-flume-integration.html) for more details. -### Kinesis -[Kinesis](streaming-kinesis.html) +- **Kafka:** Spark Streaming {{site.SPARK_VERSION_SHORT}} can receive data from Kafka 0.8.0. See the [Kafka Integration Guide](streaming-kafka-integration.html) for more details. -## Operations -There are two kinds of DStream operations - _transformations_ and _output operations_. Similar to -RDD transformations, DStream transformations operate on one or more DStreams to create new DStreams -with transformed data. After applying a sequence of transformations to the input streams, output -operations need to called, which write data out to an external data sink, such as a filesystem or a -database. +- **Kinesis:** See the [Kinesis Integration Guide](streaming-kinesis-integration.html) for more details. -### Transformations -DStreams support many of the transformations available on normal Spark RDD's. Some of the -common ones are as follows. +### Custom Sources +{:.no_toc} +Input DStreams can also be created out of custom data sources. All you have to do is implement an user-defined **receiver** (see next section to understand what that is) that can receive data from the custom sources and push it into Spark. See the +[Custom Receiver Guide](streaming-custom-receivers.html) for details. + +*** + +## Transformations on DStreams +Similar to that of RDDs, transformations allow the data from the input DStream to be modified. +DStreams support many of the transformations available on normal Spark RDD's. +Some of the common ones are as follows. @@ -557,8 +622,8 @@ common ones are as follows. The last two transformations are worth highlighting again. -

    UpdateStateByKey Operation

    - +#### UpdateStateByKey Operation +{:.no_toc} The `updateStateByKey` operation allows you to maintain arbitrary state while continuously updating it with new information. To use this, you will have to do two steps. @@ -616,8 +681,8 @@ the `(word, 1)` pairs) and the `runningCount` having the previous count. For the Scala code, take a look at the example [StatefulNetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala). -

    Transform Operation

    - +#### Transform Operation +{:.no_toc} The `transform` operation (along with its variations like `transformWith`) allows arbitrary RDD-to-RDD functions to be applied on a DStream. It can be used to apply any RDD operation that is not exposed in the DStream API. @@ -662,8 +727,8 @@ JavaPairDStream cleanedDStream = wordCounts.transform( In fact, you can also use [machine learning](mllib-guide.html) and [graph computation](graphx-programming-guide.html) algorithms in the `transform` method. -

    Window Operations

    - +#### Window Operations +{:.no_toc} Finally, Spark Streaming also provides *windowed computations*, which allow you to apply transformations over a sliding window of data. This following figure illustrates this sliding window. @@ -678,11 +743,11 @@ window. As shown in the figure, every time the window *slides* over a source DStream, the source RDDs that fall within the window are combined and operated upon to produce the RDDs of the windowed DStream. In this specific case, the operation is applied over last 3 time -units of data, and slides by 2 time units. This shows that any window-based operation needs to +units of data, and slides by 2 time units. This shows that any window operation needs to specify two parameters. * window length - The duration of the window (3 in the figure) - * slide interval - The interval at which the window-based operation is performed (2 in + * sliding interval - The interval at which the window operation is performed (2 in the figure). These two parameters must be multiples of the batch interval of the source DStream (1 in the @@ -720,7 +785,7 @@ JavaPairDStream windowedWordCounts = pairs.reduceByKeyAndWindow -Some of the common window-based operations are as follows. All of these operations take the +Some of the common window operations are as follows. All of these operations take the said two parameters - windowLength and slideInterval.
    TransformationMeaning
    @@ -778,21 +843,27 @@ said two parameters - windowLength and slideInterval.
    -### Output Operations -When an output operator is called, it triggers the computation of a stream. Currently the following -output operators are defined: + +The complete list of DStream transformations is available in the API documentation. For the Scala API, +see [DStream](api/scala/index.html#org.apache.spark.streaming.dstream.DStream) +and [PairDStreamFunctions](api/scala/index.html#org.apache.spark.streaming.dstream.PairDStreamFunctions). +For the Java API, see [JavaDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaDStream.html) +and [JavaPairDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaPairDStream.html). + +*** + +## Output Operations on DStreams +Output operations allow DStream's data to be pushed out external systems like a database or a file systems. +Since the output operations actually allow the transformed data to be consumed by external systems, +they trigger the actual execution of all the DStream transformations (similar to actions for RDDs). +Currently, the following output operations are defined: - - - - - + @@ -811,17 +882,84 @@ output operators are defined: + + + +
    Output OperationMeaning
    print() Prints first ten elements of every batch of data in a DStream on the driver.
    foreachRDD(func) The fundamental output operator. Applies a function, func, to each RDD generated from - the stream. This function should have side effects, such as printing output, saving the RDD to - external files, or writing it over the network to an external system. Prints first ten elements of every batch of data in a DStream on the driver. + This is useful for development and debugging.
    saveAsObjectFiles(prefix, [suffix]) Save this DStream's contents as a Hadoop file. The file name at each batch interval is generated based on prefix and suffix: "prefix-TIME_IN_MS[.suffix]".
    foreachRDD(func) The most generic output operator that applies a function, func, to each RDD generated from + the stream. This function should push the data in each RDD to a external system, like saving the RDD to + files, or writing it over the network to a database. Note that the function func is executed + at the driver, and will usually have RDD actions in it that will force the computation of the streaming RDDs.
    +### Design Patterns for using foreachRDD +{:.no_toc} +`dstream.foreachRDD` is a powerful primitive that allows data to sent out to external systems. +However, it is important to understand how to use this primitive correctly and efficiently. +Some of the common mistakes to avoid are as follows. -The complete list of DStream operations is available in the API documentation. For the Scala API, -see [DStream](api/scala/index.html#org.apache.spark.streaming.dstream.DStream) -and [PairDStreamFunctions](api/scala/index.html#org.apache.spark.streaming.dstream.PairDStreamFunctions). -For the Java API, see [JavaDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaDStream.html) -and [JavaPairDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaPairDStream.html). +- Often writing data to external system requires creating a connection object +(e.g. TCP connection to a remote server) and using it to send data to a remote system. +For this purpose, a developer may inadvertantly try creating a connection object at +the Spark driver, but try to use it in a Spark worker to save records in the RDDs. +For example (in Scala), + + dstream.foreachRDD(rdd => { + val connection = createNewConnection() // executed at the driver + rdd.foreach(record => { + connection.send(record) // executed at the worker + }) + }) + + This is incorrect as this requires the connection object to be serialized and sent from the driver to the worker. Such connection objects are rarely transferrable across machines. This error may manifest as serialization errors (connection object not serializable), initialization errors (connection object needs to be initialized at the workers), etc. The correct solution is to create the connection object at the worker. + +- However, this can lead to another common mistake - creating a new connection for every record. For example, + + dstream.foreachRDD(rdd => { + rdd.foreach(record => { + val connection = createNewConnection() + connection.send(record) + connection.close() + }) + }) + + Typically, creating a connection object has time and resource overheads. Therefore, creating and destroying a connection object for each record can incur unnecessarily high overheads and can significantly reduce the overall throughput of the system. A better solution is to use `rdd.foreachPartition` - create a single connection object and send all the records in a RDD partition using that connection. + + dstream.foreachRDD(rdd => { + rdd.foreachPartition(partitionOfRecords => { + val connection = createNewConnection() + partitionOfRecords.foreach(record => connection.send(record)) + connection.close() + }) + }) + + This amortizes the connection creation overheads over many records. + +- Finally, this can be further optimized by reusing connection objects across multiple RDDs/batches. + One can maintain a static pool of connection objects than can be reused as + RDDs of multiple batches are pushed to the external system, thus further reducing the overheads. -## Persistence + dstream.foreachRDD(rdd => { + rdd.foreachPartition(partitionOfRecords => { + // ConnectionPool is a static, lazily initialized pool of connections + val connection = ConnectionPool.getConnection() + partitionOfRecords.foreach(record => connection.send(record)) + ConnectionPool.returnConnection(connection) // return to the pool for future reuse + }) + }) + + Note that the connections in the pool should be lazily created on demand and timed out if not used for a while. This achieves the most efficient sending of data to external systems. + + +##### Other points to remember: +{:.no_toc} +- DStreams are executed lazily by the output operations, just like RDDs are lazily executed by RDD actions. Specifically, RDD actions inside the DStream output operations force the processing of the received data. Hence, if your application does not have any output operation, or has output operations like `dstream.foreachRDD()` without any RDD action inside them, then nothing will get executed. The system will simply receive the data and discard it. + +- By default, output operations are executed one-at-a-time. And they are executed in the order they are defined in the application. + +*** + +## Caching / Persistence Similar to RDDs, DStreams also allow developers to persist the stream's data in memory. That is, using `persist()` method on a DStream would automatically persist every RDD of that DStream in memory. This is useful if the data in the DStream will be computed multiple times (e.g., multiple @@ -838,7 +976,9 @@ memory. This is further discussed in the [Performance Tuning](#memory-tuning) se information on different persistence levels can be found in [Spark Programming Guide](programming-guide.html#rdd-persistence). -## RDD Checkpointing +*** + +## Checkpointing A _stateful operation_ is one which operates over multiple batches of data. This includes all window-based operations and the `updateStateByKey` operation. Since stateful operations have a dependency on previous batches of data, they continuously accumulate metadata over time. @@ -867,10 +1007,19 @@ For DStreams that must be checkpointed (that is, DStreams created by `updateStat `reduceByKeyAndWindow` with inverse function), the checkpoint interval of the DStream is by default set to a multiple of the DStream's sliding interval such that its at least 10 seconds. -## Deployment +*** + +## Deploying Applications A Spark Streaming application is deployed on a cluster in the same way as any other Spark application. Please refer to the [deployment guide](cluster-overview.html) for more details. +Note that the applications +that use [advanced sources](#advanced-sources) (e.g. Kafka, Flume, Twitter) are also required to package the +extra artifact they link to, along with their dependencies, in the JAR that is used to deploy the application. +For example, an application using `TwitterUtils` will have to include +`spark-streaming-twitter_{{site.SCALA_BINARY_VERSION}}` and all its transitive +dependencies in the application JAR. + If a running Spark Streaming application needs to be upgraded (with new application code), then there are two possible mechanism. @@ -889,7 +1038,9 @@ application left off. Note that this can be done only with input sources that su (like Kafka, and Flume) as data needs to be buffered while the previous application down and the upgraded application is not yet up. -## Monitoring +*** + +## Monitoring Applications Beyond Spark's [monitoring capabilities](monitoring.html), there are additional capabilities specific to Spark Streaming. When a StreamingContext is used, the [Spark web UI](monitoring.html#web-interfaces) shows @@ -912,22 +1063,18 @@ The progress of a Spark Streaming program can also be monitored using the which allows you to get receiver status and processing times. Note that this is a developer API and it is likely to be improved upon (i.e., more information reported) in the future. -*************************************************************************************************** +*************************************************************************************************** +*************************************************************************************************** # Performance Tuning Getting the best performance of a Spark Streaming application on a cluster requires a bit of tuning. This section explains a number of the parameters and configurations that can tuned to improve the performance of you application. At a high level, you need to consider two things: -
      -
    1. - Reducing the processing time of each batch of data by efficiently using cluster resources. -
    2. -
    3. - Setting the right batch size such that the batches of data can be processed as fast as they - are received (that is, data processing keeps up with the data ingestion). -
    4. -
    +1. Reducing the processing time of each batch of data by efficiently using cluster resources. + +2. Setting the right batch size such that the batches of data can be processed as fast as they + are received (that is, data processing keeps up with the data ingestion). ## Reducing the Processing Time of each Batch There are a number of optimizations that can be done in Spark to minimize the processing time of @@ -935,6 +1082,7 @@ each batch. These have been discussed in detail in [Tuning Guide](tuning.html). highlights some of the most important ones. ### Level of Parallelism in Data Receiving +{:.no_toc} Receiving data over the network (like Kafka, Flume, socket, etc.) requires the data to deserialized and stored in Spark. If the data receiving becomes a bottleneck in the system, then consider parallelizing the data receiving. Note that each input DStream @@ -958,6 +1106,7 @@ This distributes the received batches of data across specified number of machine before further processing. ### Level of Parallelism in Data Processing +{:.no_toc} Cluster resources maybe under-utilized if the number of parallel tasks used in any stage of the computation is not high enough. For example, for distributed reduce operations like `reduceByKey` and `reduceByKeyAndWindow`, the default number of parallel tasks is decided by the [config property] @@ -968,6 +1117,7 @@ documentation), or set the [config property](configuration.html#spark-properties `spark.default.parallelism` to change the default. ### Data Serialization +{:.no_toc} The overhead of data serialization can be significant, especially when sub-second batch sizes are to be achieved. There are two aspects to it. @@ -980,6 +1130,7 @@ The overhead of data serialization can be significant, especially when sub-secon serialization format. Hence, the deserialization overhead of input data may be a bottleneck. ### Task Launching Overheads +{:.no_toc} If the number of tasks launched per second is high (say, 50 or more per second), then the overhead of sending out tasks to the slaves maybe significant and will make it hard to achieve sub-second latencies. The overhead can be reduced by the following changes: @@ -994,6 +1145,8 @@ latencies. The overhead can be reduced by the following changes: These changes may reduce batch processing time by 100s of milliseconds, thus allowing sub-second batch size to be viable. +*** + ## Setting the Right Batch Size For a Spark Streaming application running on a cluster to be stable, the system should be able to process data as fast as it is being received. In other words, batches of data should be processed @@ -1022,6 +1175,8 @@ data rate and/or reducing the batch size. Note that momentary increase in the de temporary data rate increases maybe fine as long as the delay reduces back to a low value (i.e., less than batch size). +*** + ## Memory Tuning Tuning the memory usage and GC behavior of Spark applications have been discussed in great detail in the [Tuning Guide](tuning.html). It is recommended that you read that. In this section, @@ -1037,7 +1192,7 @@ Even though keeping the data serialized incurs higher serialization/deserializat it significantly reduces GC pauses. * **Clearing persistent RDDs**: By default, all persistent RDDs generated by Spark Streaming will - be cleared from memory based on Spark's in-built policy (LRU). If `spark.cleaner.ttl` is set, + be cleared from memory based on Spark's built-in policy (LRU). If `spark.cleaner.ttl` is set, then persistent RDDs that are older than that value are periodically cleared. As mentioned [earlier](#operation), this needs to be careful set based on operations used in the Spark Streaming program. However, a smarter unpersisting of RDDs can be enabled by setting the @@ -1051,7 +1206,8 @@ minimizes the variability of GC pauses. Even though concurrent GC is known to re overall processing throughput of the system, its use is still recommended to achieve more consistent batch processing times. -*************************************************************************************************** +*************************************************************************************************** +*************************************************************************************************** # Fault-tolerance Properties In this section, we are going to discuss the behavior of Spark Streaming application in the event @@ -1124,7 +1280,7 @@ def functionToCreateContext(): StreamingContext = { ssc } -// Get StreaminContext from checkpoint data or create a new one +// Get StreamingContext from checkpoint data or create a new one val context = StreamingContext.getOrCreate(checkpointDirectory, functionToCreateContext _) // Do additional setup on context that needs to be done, @@ -1178,10 +1334,7 @@ context.awaitTermination(); If the `checkpointDirectory` exists, then the context will be recreated from the checkpoint data. If the directory does not exist (i.e., running for the first time), then the function `contextFactory` will be called to create a new -context and set up the DStreams. See the Scala example -[JavaRecoverableWordCount]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/streaming/JavaRecoverableWordCount.scala) -(note that this example is missing in the 0.9 release, so you can test it using the master branch). -This example appends the word counts of network data into a file. +context and set up the DStreams. You can also explicitly create a `JavaStreamingContext` from the checkpoint data and start the computation by using `new JavaStreamingContext(checkpointDirectory)`. @@ -1208,7 +1361,8 @@ automatically restarted, and the word counts will cont For other deployment environments like Mesos and Yarn, you have to restart the driver through other mechanisms. -

    Recovery Semantics

    +#### Recovery Semantics +{:.no_toc} There are two different failure behaviors based on which input sources are used. @@ -1306,7 +1460,8 @@ in the file. This is what the sequence of outputs would be with and without a dr If the driver had crashed in the middle of the processing of time 3, then it will process time 3 and output 30 after recovery. -*************************************************************************************************** +*************************************************************************************************** +*************************************************************************************************** # Migration Guide from 0.9.1 or below to 1.x Between Spark 0.9.1 and Spark 1.0, there were a few API changes made to ensure future API stability. @@ -1332,7 +1487,7 @@ replaced by [Receiver](api/scala/index.html#org.apache.spark.streaming.receiver. the following advantages. * Methods like `stop` and `restart` have been added to for better control of the lifecycle of a receiver. See -the [custom receiver guide](streaming-custom-receiver.html) for more details. +the [custom receiver guide](streaming-custom-receivers.html) for more details. * Custom receivers can be implemented using both Scala and Java. To migrate your existing custom receivers from the earlier NetworkReceiver to the new Receiver, you have @@ -1356,6 +1511,7 @@ the `org.apache.spark.streaming.receivers` package were also moved to [`org.apache.spark.streaming.receiver`](api/scala/index.html#org.apache.spark.streaming.receiver.package) package and renamed for better clarity. +*************************************************************************************************** *************************************************************************************************** # Where to Go from Here @@ -1366,6 +1522,7 @@ package and renamed for better clarity. [DStream](api/scala/index.html#org.apache.spark.streaming.dstream.DStream) * [KafkaUtils](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$), [FlumeUtils](api/scala/index.html#org.apache.spark.streaming.flume.FlumeUtils$), + [KinesisUtils](api/scala/index.html#org.apache.spark.streaming.kinesis.KinesisUtils$), [TwitterUtils](api/scala/index.html#org.apache.spark.streaming.twitter.TwitterUtils$), [ZeroMQUtils](api/scala/index.html#org.apache.spark.streaming.zeromq.ZeroMQUtils$), and [MQTTUtils](api/scala/index.html#org.apache.spark.streaming.mqtt.MQTTUtils$) @@ -1375,6 +1532,7 @@ package and renamed for better clarity. [PairJavaDStream](api/java/index.html?org/apache/spark/streaming/api/java/PairJavaDStream.html) * [KafkaUtils](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html), [FlumeUtils](api/java/index.html?org/apache/spark/streaming/flume/FlumeUtils.html), + [KinesisUtils](api/java/index.html?org/apache/spark/streaming/kinesis/KinesisUtils.html) [TwitterUtils](api/java/index.html?org/apache/spark/streaming/twitter/TwitterUtils.html), [ZeroMQUtils](api/java/index.html?org/apache/spark/streaming/zeromq/ZeroMQUtils.html), and [MQTTUtils](api/java/index.html?org/apache/spark/streaming/mqtt/MQTTUtils.html) From e08ea7393df46567f552aa67c60a690c231775e4 Mon Sep 17 00:00:00 2001 From: scwf Date: Wed, 3 Sep 2014 18:39:13 -0700 Subject: [PATCH 19/39] [SPARK-3303][core] fix SparkContextSchedulerCreationSuite test error run test with the master branch with this command when mesos native lib is set sbt/sbt -Phive "test-only org.apache.spark.SparkContextSchedulerCreationSuite" get this error: [info] SparkContextSchedulerCreationSuite: [info] - bad-master [info] - local [info] - local-* [info] - local-n [info] - local--n-failures [info] - local-n-failures [info] - bad-local-n [info] - bad-local-n-failures [info] - local-default-parallelism [info] - simr [info] - local-cluster [info] - yarn-cluster [info] - yarn-standalone [info] - yarn-client [info] - mesos fine-grained [info] - mesos coarse-grained ** FAILED *** [info] Executor Spark home `spark.mesos.executor.home` is not set! Since `executorSparkHome` only used in `createCommand`, move `val executorSparkHome...` to `createCommand` to fix this issue. Author: scwf Author: wangfei Closes #2199 from scwf/SparkContextSchedulerCreationSuite and squashes the following commits: ef1de22 [scwf] fix code fomate 19d26f3 [scwf] fix conflict d9a8a60 [wangfei] fix SparkContextSchedulerCreationSuite test error --- .../cluster/mesos/CoarseMesosSchedulerBackend.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index da43ef567608c..64568409dbafd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -71,11 +71,6 @@ private[spark] class CoarseMesosSchedulerBackend( val taskIdToSlaveId = new HashMap[Int, String] val failuresBySlaveId = new HashMap[String, Int] // How many times tasks on each slave failed - val executorSparkHome = conf.getOption("spark.mesos.executor.home") - .orElse(sc.getSparkHome()) - .getOrElse { - throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!") - } val extraCoresPerSlave = conf.getInt("spark.mesos.extra.cores", 0) @@ -112,6 +107,11 @@ private[spark] class CoarseMesosSchedulerBackend( } def createCommand(offer: Offer, numCores: Int): CommandInfo = { + val executorSparkHome = conf.getOption("spark.mesos.executor.home") + .orElse(sc.getSparkHome()) + .getOrElse { + throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!") + } val environment = Environment.newBuilder() val extraClassPath = conf.getOption("spark.executor.extraClassPath") extraClassPath.foreach { cp => From 4bba10c41acaf84a1c4a8e2db467c22f5ab7cbb9 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 3 Sep 2014 18:42:01 -0700 Subject: [PATCH 20/39] [SPARK-3233] Executor never stop its SparnEnv, BlockManager, ConnectionManager etc. Author: Kousuke Saruta Closes #2138 from sarutak/SPARK-3233 and squashes the following commits: c0205b7 [Kousuke Saruta] Merge branch 'SPARK-3233' of github.com:sarutak/spark into SPARK-3233 064679d [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3233 d3005fd [Kousuke Saruta] Modified Class definition format of BlockManagerMaster 039b747 [Kousuke Saruta] Modified style 889e2d1 [Kousuke Saruta] Modified BlockManagerMaster to be able to be past isDriver flag 4da8535 [Kousuke Saruta] Modified BlockManagerMaster#stop to send StopBlockManagerMaster message when sender is Driver 6518c3a [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3233 d5ab19a [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3233 6bce25c [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3233 6058a58 [Kousuke Saruta] Modified Executor not to invoke SparkEnv#stop in local mode e5ad9d3 [Kousuke Saruta] Modified Executor to stop SparnEnv at the end of itself --- core/src/main/scala/org/apache/spark/SparkEnv.scala | 2 +- .../main/scala/org/apache/spark/executor/Executor.scala | 3 +++ .../org/apache/spark/storage/BlockManagerMaster.scala | 8 ++++++-- .../scala/org/apache/spark/storage/ThreadingTest.scala | 2 +- .../org/apache/spark/scheduler/DAGSchedulerSuite.scala | 2 +- .../org/apache/spark/storage/BlockManagerSuite.scala | 2 +- 6 files changed, 13 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 72716567ca99b..2973d002cc428 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -225,7 +225,7 @@ object SparkEnv extends Logging { val blockManagerMaster = new BlockManagerMaster(registerOrLookup( "BlockManagerMaster", - new BlockManagerMasterActor(isLocal, conf, listenerBus)), conf) + new BlockManagerMasterActor(isLocal, conf, listenerBus)), conf, isDriver) val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer, conf, securityManager, mapOutputTracker, shuffleManager) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index d7d19f6fa3b96..dd903dc65d204 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -123,6 +123,9 @@ private[spark] class Executor( env.metricsSystem.report() isStopped = true threadPool.shutdown() + if (!isLocal) { + env.stop() + } } class TaskRunner( diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index e67b3dc5ce02e..2e262594b3538 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -27,7 +27,11 @@ import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.AkkaUtils private[spark] -class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Logging { +class BlockManagerMaster( + var driverActor: ActorRef, + conf: SparkConf, + isDriver: Boolean) + extends Logging { private val AKKA_RETRY_ATTEMPTS: Int = AkkaUtils.numRetries(conf) private val AKKA_RETRY_INTERVAL_MS: Int = AkkaUtils.retryWaitMs(conf) @@ -196,7 +200,7 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log /** Stop the driver actor, called only on the Spark driver node */ def stop() { - if (driverActor != null) { + if (driverActor != null && isDriver) { tell(StopBlockManagerMaster) driverActor = null logInfo("BlockManagerMaster stopped") diff --git a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala index aa83ea90ee9ee..7540f0d5e2a5a 100644 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala @@ -99,7 +99,7 @@ private[spark] object ThreadingTest { val serializer = new KryoSerializer(conf) val blockManagerMaster = new BlockManagerMaster( actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), - conf) + conf, true) val blockManager = new BlockManager( "", actorSystem, blockManagerMaster, serializer, 1024 * 1024, conf, new SecurityManager(conf), new MapOutputTrackerMaster(conf), new HashShuffleManager(conf)) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 1a42fc1b233ba..0bb91febde9d7 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -120,7 +120,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F */ val cacheLocations = new HashMap[(Int, Int), Seq[BlockManagerId]] // stub out BlockManagerMaster.getLocations to use our cacheLocations - val blockManagerMaster = new BlockManagerMaster(null, conf) { + val blockManagerMaster = new BlockManagerMaster(null, conf, true) { override def getLocations(blockIds: Array[BlockId]): Seq[Seq[BlockManagerId]] = { blockIds.map { _.asRDDId.map(id => (id.rddId -> id.splitIndex)).flatMap(key => cacheLocations.get(key)). diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 14ffadab99cae..c200654162268 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -93,7 +93,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter master = new BlockManagerMaster( actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), - conf) + conf, true) val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() From f48420fde58d554480cc8830d2f8c4d17618f283 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 3 Sep 2014 18:57:20 -0700 Subject: [PATCH 21/39] [SPARK-2973][SQL] Lightweight SQL commands without distributed jobs when calling .collect() By overriding `executeCollect()` in physical plan classes of all commands, we can avoid to kick off a distributed job when collecting result of a SQL command, e.g. `sql("SET").collect()`. Previously, `Command.sideEffectResult` returns a `Seq[Any]`, and the `execute()` method in sub-classes of `Command` typically convert that to a `Seq[Row]` then parallelize it to an RDD. Now with this PR, `sideEffectResult` is required to return a `Seq[Row]` directly, so that `executeCollect()` can directly leverage that and be factored to the `Command` parent class. Author: Cheng Lian Closes #2215 from liancheng/lightweight-commands and squashes the following commits: 3fbef60 [Cheng Lian] Factored execute() method of physical commands to parent class Command 5a0e16c [Cheng Lian] Passes test suites e0e12e9 [Cheng Lian] Refactored Command.sideEffectResult and Command.executeCollect 995bdd8 [Cheng Lian] Cleaned up DescribeHiveTableCommand 542977c [Cheng Lian] Avoids confusion between logical and physical plan by adding package prefixes 55b2aa5 [Cheng Lian] Avoids distributed jobs when execution SQL commands --- .../apache/spark/sql/execution/commands.scala | 63 +++++++------------ .../apache/spark/sql/hive/HiveContext.scala | 4 +- .../spark/sql/hive/HiveStrategies.scala | 14 +++-- .../execution/DescribeHiveTableCommand.scala | 30 +++------ .../sql/hive/execution/NativeCommand.scala | 11 +--- .../spark/sql/hive/execution/commands.scala | 20 ++---- 6 files changed, 48 insertions(+), 94 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 031b695169cea..286c6d264f86a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -21,11 +21,13 @@ import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.errors.TreeNodeException -import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRow} +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.{Row, SQLConf, SQLContext} trait Command { + this: SparkPlan => + /** * A concrete command should override this lazy field to wrap up any side effects caused by the * command or any other computation that should be evaluated exactly once. The value of this field @@ -35,7 +37,11 @@ trait Command { * The `execute()` method of all the physical command classes should reference `sideEffectResult` * so that the command can be executed eagerly right after the command query is created. */ - protected[sql] lazy val sideEffectResult: Seq[Any] = Seq.empty[Any] + protected[sql] lazy val sideEffectResult: Seq[Row] = Seq.empty[Row] + + override def executeCollect(): Array[Row] = sideEffectResult.toArray + + override def execute(): RDD[Row] = sqlContext.sparkContext.parallelize(sideEffectResult, 1) } /** @@ -47,17 +53,17 @@ case class SetCommand( @transient context: SQLContext) extends LeafNode with Command with Logging { - override protected[sql] lazy val sideEffectResult: Seq[String] = (key, value) match { + override protected[sql] lazy val sideEffectResult: Seq[Row] = (key, value) match { // Set value for key k. case (Some(k), Some(v)) => if (k == SQLConf.Deprecated.MAPRED_REDUCE_TASKS) { logWarning(s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + s"automatically converted to ${SQLConf.SHUFFLE_PARTITIONS} instead.") context.setConf(SQLConf.SHUFFLE_PARTITIONS, v) - Array(s"${SQLConf.SHUFFLE_PARTITIONS}=$v") + Array(Row(s"${SQLConf.SHUFFLE_PARTITIONS}=$v")) } else { context.setConf(k, v) - Array(s"$k=$v") + Array(Row(s"$k=$v")) } // Query the value bound to key k. @@ -73,28 +79,22 @@ case class SetCommand( "hive-0.12.0.jar").mkString(":") Array( - "system:java.class.path=" + hiveJars, - "system:sun.java.command=shark.SharkServer2") - } - else { - Array(s"$k=${context.getConf(k, "")}") + Row("system:java.class.path=" + hiveJars), + Row("system:sun.java.command=shark.SharkServer2")) + } else { + Array(Row(s"$k=${context.getConf(k, "")}")) } // Query all key-value pairs that are set in the SQLConf of the context. case (None, None) => context.getAllConfs.map { case (k, v) => - s"$k=$v" + Row(s"$k=$v") }.toSeq case _ => throw new IllegalArgumentException() } - def execute(): RDD[Row] = { - val rows = sideEffectResult.map { line => new GenericRow(Array[Any](line)) } - context.sparkContext.parallelize(rows, 1) - } - override def otherCopyArgs = context :: Nil } @@ -113,19 +113,14 @@ case class ExplainCommand( extends LeafNode with Command { // Run through the optimizer to generate the physical plan. - override protected[sql] lazy val sideEffectResult: Seq[String] = try { + override protected[sql] lazy val sideEffectResult: Seq[Row] = try { // TODO in Hive, the "extended" ExplainCommand prints the AST as well, and detailed properties. val queryExecution = context.executePlan(logicalPlan) val outputString = if (extended) queryExecution.toString else queryExecution.simpleString - outputString.split("\n") + outputString.split("\n").map(Row(_)) } catch { case cause: TreeNodeException[_] => - ("Error occurred during query planning: \n" + cause.getMessage).split("\n") - } - - def execute(): RDD[Row] = { - val explanation = sideEffectResult.map(row => new GenericRow(Array[Any](row))) - context.sparkContext.parallelize(explanation, 1) + ("Error occurred during query planning: \n" + cause.getMessage).split("\n").map(Row(_)) } override def otherCopyArgs = context :: Nil @@ -144,12 +139,7 @@ case class CacheCommand(tableName: String, doCache: Boolean)(@transient context: } else { context.uncacheTable(tableName) } - Seq.empty[Any] - } - - override def execute(): RDD[Row] = { - sideEffectResult - context.emptyResult + Seq.empty[Row] } override def output: Seq[Attribute] = Seq.empty @@ -163,15 +153,8 @@ case class DescribeCommand(child: SparkPlan, output: Seq[Attribute])( @transient context: SQLContext) extends LeafNode with Command { - override protected[sql] lazy val sideEffectResult: Seq[(String, String, String)] = { - Seq(("# Registered as a temporary table", null, null)) ++ - child.output.map(field => (field.name, field.dataType.toString, null)) - } - - override def execute(): RDD[Row] = { - val rows = sideEffectResult.map { - case (name, dataType, comment) => new GenericRow(Array[Any](name, dataType, comment)) - } - context.sparkContext.parallelize(rows, 1) + override protected[sql] lazy val sideEffectResult: Seq[Row] = { + Row("# Registered as a temporary table", null, null) +: + child.output.map(field => Row(field.name, field.dataType.toString, null)) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index d9b2bc7348ad2..ced8397972fbd 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -389,7 +389,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { }.mkString("{", ",", "}") case (seq: Seq[_], ArrayType(typ, _)) => seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") - case (map: Map[_,_], MapType(kType, vType, _)) => + case (map: Map[_, _], MapType(kType, vType, _)) => map.map { case (key, value) => toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) @@ -409,7 +409,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { // be similar with Hive. describeHiveTableCommand.hiveString case command: PhysicalCommand => - command.sideEffectResult.map(_.toString) + command.sideEffectResult.map(_.head.toString) case other => val result: Seq[Seq[Any]] = toRdd.collect().toSeq diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 47e24f0dec146..24abb1b5bd1a8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -18,17 +18,19 @@ package org.apache.spark.sql.hive import org.apache.spark.annotation.Experimental -import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LowerCaseSchema} -import org.apache.spark.sql.execution._ -import org.apache.spark.sql.hive.execution._ +import org.apache.spark.sql.catalyst.types.StringType import org.apache.spark.sql.columnar.InMemoryRelation -import org.apache.spark.sql.parquet.{ParquetRelation, ParquetTableScan} +import org.apache.spark.sql.execution.{DescribeCommand, OutputFaker, SparkPlan} +import org.apache.spark.sql.hive +import org.apache.spark.sql.hive.execution._ +import org.apache.spark.sql.parquet.ParquetRelation +import org.apache.spark.sql.{SQLContext, SchemaRDD} import scala.collection.JavaConversions._ @@ -196,9 +198,9 @@ private[hive] trait HiveStrategies { case logical.NativeCommand(sql) => NativeCommand(sql, plan.output)(context) :: Nil - case DropTable(tableName, ifExists) => execution.DropTable(tableName, ifExists) :: Nil + case hive.DropTable(tableName, ifExists) => execution.DropTable(tableName, ifExists) :: Nil - case AnalyzeTable(tableName) => execution.AnalyzeTable(tableName) :: Nil + case hive.AnalyzeTable(tableName) => execution.AnalyzeTable(tableName) :: Nil case describe: logical.DescribeCommand => val resolvedTable = context.executePlan(describe.table).analyzed diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala index a40e89e0d382b..317801001c7a4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRow, Row} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Row} import org.apache.spark.sql.execution.{Command, LeafNode} import org.apache.spark.sql.hive.{HiveContext, MetastoreRelation} @@ -41,26 +41,21 @@ case class DescribeHiveTableCommand( extends LeafNode with Command { // Strings with the format like Hive. It is used for result comparison in our unit tests. - lazy val hiveString: Seq[String] = { - val alignment = 20 - val delim = "\t" - - sideEffectResult.map { - case (name, dataType, comment) => - String.format("%-" + alignment + "s", name) + delim + - String.format("%-" + alignment + "s", dataType) + delim + - String.format("%-" + alignment + "s", Option(comment).getOrElse("None")) - } + lazy val hiveString: Seq[String] = sideEffectResult.map { + case Row(name: String, dataType: String, comment) => + Seq(name, dataType, Option(comment.asInstanceOf[String]).getOrElse("None")) + .map(s => String.format(s"%-20s", s)) + .mkString("\t") } - override protected[sql] lazy val sideEffectResult: Seq[(String, String, String)] = { + override protected[sql] lazy val sideEffectResult: Seq[Row] = { // Trying to mimic the format of Hive's output. But not exactly the same. var results: Seq[(String, String, String)] = Nil val columns: Seq[FieldSchema] = table.hiveQlTable.getCols val partitionColumns: Seq[FieldSchema] = table.hiveQlTable.getPartCols results ++= columns.map(field => (field.getName, field.getType, field.getComment)) - if (!partitionColumns.isEmpty) { + if (partitionColumns.nonEmpty) { val partColumnInfo = partitionColumns.map(field => (field.getName, field.getType, field.getComment)) results ++= @@ -74,14 +69,9 @@ case class DescribeHiveTableCommand( results ++= Seq(("Detailed Table Information", table.hiveQlTable.getTTable.toString, "")) } - results - } - - override def execute(): RDD[Row] = { - val rows = sideEffectResult.map { - case (name, dataType, comment) => new GenericRow(Array[Any](name, dataType, comment)) + results.map { case (name, dataType, comment) => + Row(name, dataType, comment) } - context.sparkContext.parallelize(rows, 1) } override def otherCopyArgs = context :: Nil diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala index fe6031678f70f..8f10e1ba7f426 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala @@ -32,16 +32,7 @@ case class NativeCommand( @transient context: HiveContext) extends LeafNode with Command { - override protected[sql] lazy val sideEffectResult: Seq[String] = context.runSqlHive(sql) - - override def execute(): RDD[Row] = { - if (sideEffectResult.size == 0) { - context.emptyResult - } else { - val rows = sideEffectResult.map(r => new GenericRow(Array[Any](r))) - context.sparkContext.parallelize(rows, 1) - } - } + override protected[sql] lazy val sideEffectResult: Seq[Row] = context.runSqlHive(sql).map(Row(_)) override def otherCopyArgs = context :: Nil } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index 2985169da033c..a1a4aa7de7bf7 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -33,19 +33,13 @@ import org.apache.spark.sql.hive.HiveContext */ @DeveloperApi case class AnalyzeTable(tableName: String) extends LeafNode with Command { - def hiveContext = sqlContext.asInstanceOf[HiveContext] def output = Seq.empty - override protected[sql] lazy val sideEffectResult = { + override protected[sql] lazy val sideEffectResult: Seq[Row] = { hiveContext.analyze(tableName) - Seq.empty[Any] - } - - override def execute(): RDD[Row] = { - sideEffectResult - sparkContext.emptyRDD[Row] + Seq.empty[Row] } } @@ -55,20 +49,14 @@ case class AnalyzeTable(tableName: String) extends LeafNode with Command { */ @DeveloperApi case class DropTable(tableName: String, ifExists: Boolean) extends LeafNode with Command { - def hiveContext = sqlContext.asInstanceOf[HiveContext] def output = Seq.empty - override protected[sql] lazy val sideEffectResult: Seq[Any] = { + override protected[sql] lazy val sideEffectResult: Seq[Row] = { val ifExistsClause = if (ifExists) "IF EXISTS " else "" hiveContext.runSqlHive(s"DROP TABLE $ifExistsClause$tableName") hiveContext.catalog.unregisterTable(None, tableName) - Seq.empty - } - - override def execute(): RDD[Row] = { - sideEffectResult - sparkContext.emptyRDD[Row] + Seq.empty[Row] } } From 248067adbe90f93c7d5e23aa61b3072dfdf48a8a Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 3 Sep 2014 18:59:26 -0700 Subject: [PATCH 22/39] [SPARK-2961][SQL] Use statistics to prune batches within cached partitions This PR is based on #1883 authored by marmbrus. Key differences: 1. Batch pruning instead of partition pruning When #1883 was authored, batched column buffer building (#1880) hadn't been introduced. This PR combines these two and provide partition batch level pruning, which leads to smaller memory footprints and can generally skip more elements. The cost is that the pruning predicates are evaluated more frequently (partition number multiplies batch number per partition). 1. More filters are supported Filter predicates consist of `=`, `<`, `<=`, `>`, `>=` and their conjunctions and disjunctions are supported. Author: Cheng Lian Closes #2188 from liancheng/in-mem-batch-pruning and squashes the following commits: 68cf019 [Cheng Lian] Marked sqlContext as @transient 4254f6c [Cheng Lian] Enables in-memory partition pruning in PartitionBatchPruningSuite 3784105 [Cheng Lian] Overrides InMemoryColumnarTableScan.sqlContext d2a1d66 [Cheng Lian] Disables in-memory partition pruning by default 062c315 [Cheng Lian] HiveCompatibilitySuite code cleanup 16b77bf [Cheng Lian] Fixed pruning predication conjunctions and disjunctions 16195c5 [Cheng Lian] Enabled both disjunction and conjunction 89950d0 [Cheng Lian] Worked around Scala style check 9c167f6 [Cheng Lian] Minor code cleanup 3c4d5c7 [Cheng Lian] Minor code cleanup ea59ee5 [Cheng Lian] Renamed PartitionSkippingSuite to PartitionBatchPruningSuite fc517d0 [Cheng Lian] More test cases 1868c18 [Cheng Lian] Code cleanup, bugfix, and adding tests cb76da4 [Cheng Lian] Added more predicate filters, fixed table scan stats for testing purposes 385474a [Cheng Lian] Merge branch 'inMemStats' into in-mem-batch-pruning --- .../catalyst/expressions/AttributeMap.scala | 41 ++ .../catalyst/expressions/BoundAttribute.scala | 12 +- .../scala/org/apache/spark/sql/SQLConf.scala | 7 + .../spark/sql/columnar/ColumnBuilder.scala | 10 +- .../spark/sql/columnar/ColumnStats.scala | 434 +++++------------- .../columnar/InMemoryColumnarTableScan.scala | 131 +++++- .../sql/columnar/NullableColumnBuilder.scala | 1 + .../spark/sql/execution/SparkStrategies.scala | 4 +- .../spark/sql/columnar/ColumnStatsSuite.scala | 39 +- .../columnar/NullableColumnBuilderSuite.scala | 2 +- .../columnar/PartitionBatchPruningSuite.scala | 95 ++++ .../compression/BooleanBitSetSuite.scala | 4 +- .../compression/DictionaryEncodingSuite.scala | 2 +- .../compression/IntegralDeltaSuite.scala | 2 +- .../compression/RunLengthEncodingSuite.scala | 4 +- .../TestCompressibleColumnBuilder.scala | 4 +- .../execution/HiveCompatibilitySuite.scala | 13 +- 17 files changed, 446 insertions(+), 359 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala new file mode 100644 index 0000000000000..8364379644c90 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +/** + * Builds a map that is keyed by an Attribute's expression id. Using the expression id allows values + * to be looked up even when the attributes used differ cosmetically (i.e., the capitalization + * of the name, or the expected nullability). + */ +object AttributeMap { + def apply[A](kvs: Seq[(Attribute, A)]) = + new AttributeMap(kvs.map(kv => (kv._1.exprId, (kv._1, kv._2))).toMap) +} + +class AttributeMap[A](baseMap: Map[ExprId, (Attribute, A)]) + extends Map[Attribute, A] with Serializable { + + override def get(k: Attribute): Option[A] = baseMap.get(k.exprId).map(_._2) + + override def + [B1 >: A](kv: (Attribute, B1)): Map[Attribute, B1] = + (baseMap.map(_._2) + kv).toMap + + override def iterator: Iterator[(Attribute, A)] = baseMap.map(_._2).iterator + + override def -(key: Attribute): Map[Attribute, A] = (baseMap.map(_._2) - key).toMap +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index 54c6baf1af3bf..fa80b07f8e6be 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -38,12 +38,20 @@ case class BoundReference(ordinal: Int, dataType: DataType, nullable: Boolean) } object BindReferences extends Logging { - def bindReference[A <: Expression](expression: A, input: Seq[Attribute]): A = { + + def bindReference[A <: Expression]( + expression: A, + input: Seq[Attribute], + allowFailures: Boolean = false): A = { expression.transform { case a: AttributeReference => attachTree(a, "Binding attribute") { val ordinal = input.indexWhere(_.exprId == a.exprId) if (ordinal == -1) { - sys.error(s"Couldn't find $a in ${input.mkString("[", ",", "]")}") + if (allowFailures) { + a + } else { + sys.error(s"Couldn't find $a in ${input.mkString("[", ",", "]")}") + } } else { BoundReference(ordinal, a.dataType, a.nullable) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 64d49354dadcd..4137ac7663739 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -26,6 +26,7 @@ import java.util.Properties private[spark] object SQLConf { val COMPRESS_CACHED = "spark.sql.inMemoryColumnarStorage.compressed" val COLUMN_BATCH_SIZE = "spark.sql.inMemoryColumnarStorage.batchSize" + val IN_MEMORY_PARTITION_PRUNING = "spark.sql.inMemoryColumnarStorage.partitionPruning" val AUTO_BROADCASTJOIN_THRESHOLD = "spark.sql.autoBroadcastJoinThreshold" val DEFAULT_SIZE_IN_BYTES = "spark.sql.defaultSizeInBytes" val SHUFFLE_PARTITIONS = "spark.sql.shuffle.partitions" @@ -124,6 +125,12 @@ trait SQLConf { private[spark] def isParquetBinaryAsString: Boolean = getConf(PARQUET_BINARY_AS_STRING, "false").toBoolean + /** + * When set to true, partition pruning for in-memory columnar tables is enabled. + */ + private[spark] def inMemoryPartitionPruning: Boolean = + getConf(IN_MEMORY_PARTITION_PRUNING, "false").toBoolean + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index 247337a875c75..b3ec5ded22422 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -38,7 +38,7 @@ private[sql] trait ColumnBuilder { /** * Column statistics information */ - def columnStats: ColumnStats[_, _] + def columnStats: ColumnStats /** * Returns the final columnar byte buffer. @@ -47,7 +47,7 @@ private[sql] trait ColumnBuilder { } private[sql] class BasicColumnBuilder[T <: DataType, JvmType]( - val columnStats: ColumnStats[T, JvmType], + val columnStats: ColumnStats, val columnType: ColumnType[T, JvmType]) extends ColumnBuilder { @@ -81,18 +81,18 @@ private[sql] class BasicColumnBuilder[T <: DataType, JvmType]( private[sql] abstract class ComplexColumnBuilder[T <: DataType, JvmType]( columnType: ColumnType[T, JvmType]) - extends BasicColumnBuilder[T, JvmType](new NoopColumnStats[T, JvmType], columnType) + extends BasicColumnBuilder[T, JvmType](new NoopColumnStats, columnType) with NullableColumnBuilder private[sql] abstract class NativeColumnBuilder[T <: NativeType]( - override val columnStats: NativeColumnStats[T], + override val columnStats: ColumnStats, override val columnType: NativeColumnType[T]) extends BasicColumnBuilder[T, T#JvmType](columnStats, columnType) with NullableColumnBuilder with AllCompressionSchemes with CompressibleColumnBuilder[T] -private[sql] class BooleanColumnBuilder extends NativeColumnBuilder(new BooleanColumnStats, BOOLEAN) +private[sql] class BooleanColumnBuilder extends NativeColumnBuilder(new NoopColumnStats, BOOLEAN) private[sql] class IntColumnBuilder extends NativeColumnBuilder(new IntColumnStats, INT) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala index 6502110e903fe..fc343ccb995c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala @@ -17,381 +17,193 @@ package org.apache.spark.sql.columnar +import java.sql.Timestamp + import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions.{AttributeMap, Attribute, AttributeReference} import org.apache.spark.sql.catalyst.types._ +private[sql] class ColumnStatisticsSchema(a: Attribute) extends Serializable { + val upperBound = AttributeReference(a.name + ".upperBound", a.dataType, nullable = false)() + val lowerBound = AttributeReference(a.name + ".lowerBound", a.dataType, nullable = false)() + val nullCount = AttributeReference(a.name + ".nullCount", IntegerType, nullable = false)() + + val schema = Seq(lowerBound, upperBound, nullCount) +} + +private[sql] class PartitionStatistics(tableSchema: Seq[Attribute]) extends Serializable { + val (forAttribute, schema) = { + val allStats = tableSchema.map(a => a -> new ColumnStatisticsSchema(a)) + (AttributeMap(allStats), allStats.map(_._2.schema).foldLeft(Seq.empty[Attribute])(_ ++ _)) + } +} + /** * Used to collect statistical information when building in-memory columns. * * NOTE: we intentionally avoid using `Ordering[T]` to compare values here because `Ordering[T]` * brings significant performance penalty. */ -private[sql] sealed abstract class ColumnStats[T <: DataType, JvmType] extends Serializable { - /** - * Closed lower bound of this column. - */ - def lowerBound: JvmType - - /** - * Closed upper bound of this column. - */ - def upperBound: JvmType - +private[sql] sealed trait ColumnStats extends Serializable { /** * Gathers statistics information from `row(ordinal)`. */ - def gatherStats(row: Row, ordinal: Int) - - /** - * Returns `true` if `lower <= row(ordinal) <= upper`. - */ - def contains(row: Row, ordinal: Int): Boolean + def gatherStats(row: Row, ordinal: Int): Unit /** - * Returns `true` if `row(ordinal) < upper` holds. + * Column statistics represented as a single row, currently including closed lower bound, closed + * upper bound and null count. */ - def isAbove(row: Row, ordinal: Int): Boolean - - /** - * Returns `true` if `lower < row(ordinal)` holds. - */ - def isBelow(row: Row, ordinal: Int): Boolean - - /** - * Returns `true` if `row(ordinal) <= upper` holds. - */ - def isAtOrAbove(row: Row, ordinal: Int): Boolean - - /** - * Returns `true` if `lower <= row(ordinal)` holds. - */ - def isAtOrBelow(row: Row, ordinal: Int): Boolean -} - -private[sql] sealed abstract class NativeColumnStats[T <: NativeType] - extends ColumnStats[T, T#JvmType] { - - type JvmType = T#JvmType - - protected var (_lower, _upper) = initialBounds - - def initialBounds: (JvmType, JvmType) - - protected def columnType: NativeColumnType[T] - - override def lowerBound: T#JvmType = _lower - - override def upperBound: T#JvmType = _upper - - override def isAtOrAbove(row: Row, ordinal: Int) = { - contains(row, ordinal) || isAbove(row, ordinal) - } - - override def isAtOrBelow(row: Row, ordinal: Int) = { - contains(row, ordinal) || isBelow(row, ordinal) - } + def collectedStatistics: Row } -private[sql] class NoopColumnStats[T <: DataType, JvmType] extends ColumnStats[T, JvmType] { - override def isAtOrBelow(row: Row, ordinal: Int) = true - - override def isAtOrAbove(row: Row, ordinal: Int) = true - - override def isBelow(row: Row, ordinal: Int) = true - - override def isAbove(row: Row, ordinal: Int) = true +private[sql] class NoopColumnStats extends ColumnStats { - override def contains(row: Row, ordinal: Int) = true + override def gatherStats(row: Row, ordinal: Int): Unit = {} - override def gatherStats(row: Row, ordinal: Int) {} - - override def upperBound = null.asInstanceOf[JvmType] - - override def lowerBound = null.asInstanceOf[JvmType] + override def collectedStatistics = Row() } -private[sql] abstract class BasicColumnStats[T <: NativeType]( - protected val columnType: NativeColumnType[T]) - extends NativeColumnStats[T] - -private[sql] class BooleanColumnStats extends BasicColumnStats(BOOLEAN) { - override def initialBounds = (true, false) - - override def isBelow(row: Row, ordinal: Int) = { - lowerBound < columnType.getField(row, ordinal) - } - - override def isAbove(row: Row, ordinal: Int) = { - columnType.getField(row, ordinal) < upperBound - } - - override def contains(row: Row, ordinal: Int) = { - val field = columnType.getField(row, ordinal) - lowerBound <= field && field <= upperBound - } +private[sql] class ByteColumnStats extends ColumnStats { + var upper = Byte.MinValue + var lower = Byte.MaxValue + var nullCount = 0 override def gatherStats(row: Row, ordinal: Int) { - val field = columnType.getField(row, ordinal) - if (field > upperBound) _upper = field - if (field < lowerBound) _lower = field - } -} - -private[sql] class ByteColumnStats extends BasicColumnStats(BYTE) { - override def initialBounds = (Byte.MaxValue, Byte.MinValue) - - override def isBelow(row: Row, ordinal: Int) = { - lowerBound < columnType.getField(row, ordinal) - } - - override def isAbove(row: Row, ordinal: Int) = { - columnType.getField(row, ordinal) < upperBound - } - - override def contains(row: Row, ordinal: Int) = { - val field = columnType.getField(row, ordinal) - lowerBound <= field && field <= upperBound + if (!row.isNullAt(ordinal)) { + val value = row.getByte(ordinal) + if (value > upper) upper = value + if (value < lower) lower = value + } else { + nullCount += 1 + } } - override def gatherStats(row: Row, ordinal: Int) { - val field = columnType.getField(row, ordinal) - if (field > upperBound) _upper = field - if (field < lowerBound) _lower = field - } + def collectedStatistics = Row(lower, upper, nullCount) } -private[sql] class ShortColumnStats extends BasicColumnStats(SHORT) { - override def initialBounds = (Short.MaxValue, Short.MinValue) - - override def isBelow(row: Row, ordinal: Int) = { - lowerBound < columnType.getField(row, ordinal) - } - - override def isAbove(row: Row, ordinal: Int) = { - columnType.getField(row, ordinal) < upperBound - } - - override def contains(row: Row, ordinal: Int) = { - val field = columnType.getField(row, ordinal) - lowerBound <= field && field <= upperBound - } +private[sql] class ShortColumnStats extends ColumnStats { + var upper = Short.MinValue + var lower = Short.MaxValue + var nullCount = 0 override def gatherStats(row: Row, ordinal: Int) { - val field = columnType.getField(row, ordinal) - if (field > upperBound) _upper = field - if (field < lowerBound) _lower = field - } -} - -private[sql] class LongColumnStats extends BasicColumnStats(LONG) { - override def initialBounds = (Long.MaxValue, Long.MinValue) - - override def isBelow(row: Row, ordinal: Int) = { - lowerBound < columnType.getField(row, ordinal) - } - - override def isAbove(row: Row, ordinal: Int) = { - columnType.getField(row, ordinal) < upperBound - } - - override def contains(row: Row, ordinal: Int) = { - val field = columnType.getField(row, ordinal) - lowerBound <= field && field <= upperBound + if (!row.isNullAt(ordinal)) { + val value = row.getShort(ordinal) + if (value > upper) upper = value + if (value < lower) lower = value + } else { + nullCount += 1 + } } - override def gatherStats(row: Row, ordinal: Int) { - val field = columnType.getField(row, ordinal) - if (field > upperBound) _upper = field - if (field < lowerBound) _lower = field - } + def collectedStatistics = Row(lower, upper, nullCount) } -private[sql] class DoubleColumnStats extends BasicColumnStats(DOUBLE) { - override def initialBounds = (Double.MaxValue, Double.MinValue) - - override def isBelow(row: Row, ordinal: Int) = { - lowerBound < columnType.getField(row, ordinal) - } - - override def isAbove(row: Row, ordinal: Int) = { - columnType.getField(row, ordinal) < upperBound - } - - override def contains(row: Row, ordinal: Int) = { - val field = columnType.getField(row, ordinal) - lowerBound <= field && field <= upperBound - } +private[sql] class LongColumnStats extends ColumnStats { + var upper = Long.MinValue + var lower = Long.MaxValue + var nullCount = 0 override def gatherStats(row: Row, ordinal: Int) { - val field = columnType.getField(row, ordinal) - if (field > upperBound) _upper = field - if (field < lowerBound) _lower = field - } -} - -private[sql] class FloatColumnStats extends BasicColumnStats(FLOAT) { - override def initialBounds = (Float.MaxValue, Float.MinValue) - - override def isBelow(row: Row, ordinal: Int) = { - lowerBound < columnType.getField(row, ordinal) + if (!row.isNullAt(ordinal)) { + val value = row.getLong(ordinal) + if (value > upper) upper = value + if (value < lower) lower = value + } else { + nullCount += 1 + } } - override def isAbove(row: Row, ordinal: Int) = { - columnType.getField(row, ordinal) < upperBound - } + def collectedStatistics = Row(lower, upper, nullCount) +} - override def contains(row: Row, ordinal: Int) = { - val field = columnType.getField(row, ordinal) - lowerBound <= field && field <= upperBound - } +private[sql] class DoubleColumnStats extends ColumnStats { + var upper = Double.MinValue + var lower = Double.MaxValue + var nullCount = 0 override def gatherStats(row: Row, ordinal: Int) { - val field = columnType.getField(row, ordinal) - if (field > upperBound) _upper = field - if (field < lowerBound) _lower = field + if (!row.isNullAt(ordinal)) { + val value = row.getDouble(ordinal) + if (value > upper) upper = value + if (value < lower) lower = value + } else { + nullCount += 1 + } } -} -private[sql] object IntColumnStats { - val UNINITIALIZED = 0 - val INITIALIZED = 1 - val ASCENDING = 2 - val DESCENDING = 3 - val UNORDERED = 4 + def collectedStatistics = Row(lower, upper, nullCount) } -/** - * Statistical information for `Int` columns. More information is collected since `Int` is - * frequently used. Extra information include: - * - * - Ordering state (ascending/descending/unordered), may be used to decide whether binary search - * is applicable when searching elements. - * - Maximum delta between adjacent elements, may be used to guide the `IntDelta` compression - * scheme. - * - * (This two kinds of information are not used anywhere yet and might be removed later.) - */ -private[sql] class IntColumnStats extends BasicColumnStats(INT) { - import IntColumnStats._ - - private var orderedState = UNINITIALIZED - private var lastValue: Int = _ - private var _maxDelta: Int = _ - - def isAscending = orderedState != DESCENDING && orderedState != UNORDERED - def isDescending = orderedState != ASCENDING && orderedState != UNORDERED - def isOrdered = isAscending || isDescending - def maxDelta = _maxDelta - - override def initialBounds = (Int.MaxValue, Int.MinValue) +private[sql] class FloatColumnStats extends ColumnStats { + var upper = Float.MinValue + var lower = Float.MaxValue + var nullCount = 0 - override def isBelow(row: Row, ordinal: Int) = { - lowerBound < columnType.getField(row, ordinal) + override def gatherStats(row: Row, ordinal: Int) { + if (!row.isNullAt(ordinal)) { + val value = row.getFloat(ordinal) + if (value > upper) upper = value + if (value < lower) lower = value + } else { + nullCount += 1 + } } - override def isAbove(row: Row, ordinal: Int) = { - columnType.getField(row, ordinal) < upperBound - } + def collectedStatistics = Row(lower, upper, nullCount) +} - override def contains(row: Row, ordinal: Int) = { - val field = columnType.getField(row, ordinal) - lowerBound <= field && field <= upperBound - } +private[sql] class IntColumnStats extends ColumnStats { + var upper = Int.MinValue + var lower = Int.MaxValue + var nullCount = 0 override def gatherStats(row: Row, ordinal: Int) { - val field = columnType.getField(row, ordinal) - - if (field > upperBound) _upper = field - if (field < lowerBound) _lower = field - - orderedState = orderedState match { - case UNINITIALIZED => - lastValue = field - INITIALIZED - - case INITIALIZED => - // If all the integers in the column are the same, ordered state is set to Ascending. - // TODO (lian) Confirm whether this is the standard behaviour. - val nextState = if (field >= lastValue) ASCENDING else DESCENDING - _maxDelta = math.abs(field - lastValue) - lastValue = field - nextState - - case ASCENDING if field < lastValue => - UNORDERED - - case DESCENDING if field > lastValue => - UNORDERED - - case state @ (ASCENDING | DESCENDING) => - _maxDelta = _maxDelta.max(field - lastValue) - lastValue = field - state - - case _ => - orderedState + if (!row.isNullAt(ordinal)) { + val value = row.getInt(ordinal) + if (value > upper) upper = value + if (value < lower) lower = value + } else { + nullCount += 1 } } + + def collectedStatistics = Row(lower, upper, nullCount) } -private[sql] class StringColumnStats extends BasicColumnStats(STRING) { - override def initialBounds = (null, null) +private[sql] class StringColumnStats extends ColumnStats { + var upper: String = null + var lower: String = null + var nullCount = 0 override def gatherStats(row: Row, ordinal: Int) { - val field = columnType.getField(row, ordinal) - if ((upperBound eq null) || field.compareTo(upperBound) > 0) _upper = field - if ((lowerBound eq null) || field.compareTo(lowerBound) < 0) _lower = field - } - - override def contains(row: Row, ordinal: Int) = { - (upperBound ne null) && { - val field = columnType.getField(row, ordinal) - lowerBound.compareTo(field) <= 0 && field.compareTo(upperBound) <= 0 - } - } - - override def isAbove(row: Row, ordinal: Int) = { - (upperBound ne null) && { - val field = columnType.getField(row, ordinal) - field.compareTo(upperBound) < 0 + if (!row.isNullAt(ordinal)) { + val value = row.getString(ordinal) + if (upper == null || value.compareTo(upper) > 0) upper = value + if (lower == null || value.compareTo(lower) < 0) lower = value + } else { + nullCount += 1 } } - override def isBelow(row: Row, ordinal: Int) = { - (lowerBound ne null) && { - val field = columnType.getField(row, ordinal) - lowerBound.compareTo(field) < 0 - } - } + def collectedStatistics = Row(lower, upper, nullCount) } -private[sql] class TimestampColumnStats extends BasicColumnStats(TIMESTAMP) { - override def initialBounds = (null, null) +private[sql] class TimestampColumnStats extends ColumnStats { + var upper: Timestamp = null + var lower: Timestamp = null + var nullCount = 0 override def gatherStats(row: Row, ordinal: Int) { - val field = columnType.getField(row, ordinal) - if ((upperBound eq null) || field.compareTo(upperBound) > 0) _upper = field - if ((lowerBound eq null) || field.compareTo(lowerBound) < 0) _lower = field - } - - override def contains(row: Row, ordinal: Int) = { - (upperBound ne null) && { - val field = columnType.getField(row, ordinal) - lowerBound.compareTo(field) <= 0 && field.compareTo(upperBound) <= 0 + if (!row.isNullAt(ordinal)) { + val value = row(ordinal).asInstanceOf[Timestamp] + if (upper == null || value.compareTo(upper) > 0) upper = value + if (lower == null || value.compareTo(lower) < 0) lower = value + } else { + nullCount += 1 } } - override def isAbove(row: Row, ordinal: Int) = { - (lowerBound ne null) && { - val field = columnType.getField(row, ordinal) - field.compareTo(upperBound) < 0 - } - } - - override def isBelow(row: Row, ordinal: Int) = { - (lowerBound ne null) && { - val field = columnType.getField(row, ordinal) - lowerBound.compareTo(field) < 0 - } - } + def collectedStatistics = Row(lower, upper, nullCount) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala index cb055cd74a5e5..dc668e7dc934c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala @@ -19,10 +19,12 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer +import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation -import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericMutableRow} +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.{LeafNode, SparkPlan} @@ -31,23 +33,27 @@ object InMemoryRelation { new InMemoryRelation(child.output, useCompression, batchSize, child)() } +private[sql] case class CachedBatch(buffers: Array[ByteBuffer], stats: Row) + private[sql] case class InMemoryRelation( output: Seq[Attribute], useCompression: Boolean, batchSize: Int, child: SparkPlan) - (private var _cachedColumnBuffers: RDD[Array[ByteBuffer]] = null) + (private var _cachedColumnBuffers: RDD[CachedBatch] = null) extends LogicalPlan with MultiInstanceRelation { override lazy val statistics = Statistics(sizeInBytes = child.sqlContext.defaultSizeInBytes) + val partitionStatistics = new PartitionStatistics(output) + // If the cached column buffers were not passed in, we calculate them in the constructor. // As in Spark, the actual work of caching is lazy. if (_cachedColumnBuffers == null) { val output = child.output val cached = child.execute().mapPartitions { baseIterator => - new Iterator[Array[ByteBuffer]] { + new Iterator[CachedBatch] { def next() = { val columnBuilders = output.map { attribute => val columnType = ColumnType(attribute.dataType) @@ -68,7 +74,10 @@ private[sql] case class InMemoryRelation( rowCount += 1 } - columnBuilders.map(_.build()) + val stats = Row.fromSeq( + columnBuilders.map(_.columnStats.collectedStatistics).foldLeft(Seq.empty[Any])(_ ++ _)) + + CachedBatch(columnBuilders.map(_.build()), stats) } def hasNext = baseIterator.hasNext @@ -79,7 +88,6 @@ private[sql] case class InMemoryRelation( _cachedColumnBuffers = cached } - override def children = Seq.empty override def newInstance() = { @@ -96,13 +104,98 @@ private[sql] case class InMemoryRelation( private[sql] case class InMemoryColumnarTableScan( attributes: Seq[Attribute], + predicates: Seq[Expression], relation: InMemoryRelation) extends LeafNode { + @transient override val sqlContext = relation.child.sqlContext + override def output: Seq[Attribute] = attributes + // Returned filter predicate should return false iff it is impossible for the input expression + // to evaluate to `true' based on statistics collected about this partition batch. + val buildFilter: PartialFunction[Expression, Expression] = { + case And(lhs: Expression, rhs: Expression) + if buildFilter.isDefinedAt(lhs) && buildFilter.isDefinedAt(rhs) => + buildFilter(lhs) && buildFilter(rhs) + + case Or(lhs: Expression, rhs: Expression) + if buildFilter.isDefinedAt(lhs) && buildFilter.isDefinedAt(rhs) => + buildFilter(lhs) || buildFilter(rhs) + + case EqualTo(a: AttributeReference, l: Literal) => + val aStats = relation.partitionStatistics.forAttribute(a) + aStats.lowerBound <= l && l <= aStats.upperBound + + case EqualTo(l: Literal, a: AttributeReference) => + val aStats = relation.partitionStatistics.forAttribute(a) + aStats.lowerBound <= l && l <= aStats.upperBound + + case LessThan(a: AttributeReference, l: Literal) => + val aStats = relation.partitionStatistics.forAttribute(a) + aStats.lowerBound < l + + case LessThan(l: Literal, a: AttributeReference) => + val aStats = relation.partitionStatistics.forAttribute(a) + l < aStats.upperBound + + case LessThanOrEqual(a: AttributeReference, l: Literal) => + val aStats = relation.partitionStatistics.forAttribute(a) + aStats.lowerBound <= l + + case LessThanOrEqual(l: Literal, a: AttributeReference) => + val aStats = relation.partitionStatistics.forAttribute(a) + l <= aStats.upperBound + + case GreaterThan(a: AttributeReference, l: Literal) => + val aStats = relation.partitionStatistics.forAttribute(a) + l < aStats.upperBound + + case GreaterThan(l: Literal, a: AttributeReference) => + val aStats = relation.partitionStatistics.forAttribute(a) + aStats.lowerBound < l + + case GreaterThanOrEqual(a: AttributeReference, l: Literal) => + val aStats = relation.partitionStatistics.forAttribute(a) + l <= aStats.upperBound + + case GreaterThanOrEqual(l: Literal, a: AttributeReference) => + val aStats = relation.partitionStatistics.forAttribute(a) + aStats.lowerBound <= l + } + + val partitionFilters = { + predicates.flatMap { p => + val filter = buildFilter.lift(p) + val boundFilter = + filter.map( + BindReferences.bindReference( + _, + relation.partitionStatistics.schema, + allowFailures = true)) + + boundFilter.foreach(_ => + filter.foreach(f => logInfo(s"Predicate $p generates partition filter: $f"))) + + // If the filter can't be resolved then we are missing required statistics. + boundFilter.filter(_.resolved) + } + } + + val readPartitions = sparkContext.accumulator(0) + val readBatches = sparkContext.accumulator(0) + + private val inMemoryPartitionPruningEnabled = sqlContext.inMemoryPartitionPruning + override def execute() = { + readPartitions.setValue(0) + readBatches.setValue(0) + relation.cachedColumnBuffers.mapPartitions { iterator => + val partitionFilter = newPredicate( + partitionFilters.reduceOption(And).getOrElse(Literal(true)), + relation.partitionStatistics.schema) + // Find the ordinals of the requested columns. If none are requested, use the first. val requestedColumns = if (attributes.isEmpty) { Seq(0) @@ -110,8 +203,26 @@ private[sql] case class InMemoryColumnarTableScan( attributes.map(a => relation.output.indexWhere(_.exprId == a.exprId)) } - iterator - .map(batch => requestedColumns.map(batch(_)).map(ColumnAccessor(_))) + val rows = iterator + // Skip pruned batches + .filter { cachedBatch => + if (inMemoryPartitionPruningEnabled && !partitionFilter(cachedBatch.stats)) { + def statsString = relation.partitionStatistics.schema + .zip(cachedBatch.stats) + .map { case (a, s) => s"${a.name}: $s" } + .mkString(", ") + logInfo(s"Skipping partition based on stats $statsString") + false + } else { + readBatches += 1 + true + } + } + // Build column accessors + .map { cachedBatch => + requestedColumns.map(cachedBatch.buffers(_)).map(ColumnAccessor(_)) + } + // Extract rows via column accessors .flatMap { columnAccessors => val nextRow = new GenericMutableRow(columnAccessors.length) new Iterator[Row] { @@ -127,6 +238,12 @@ private[sql] case class InMemoryColumnarTableScan( override def hasNext = columnAccessors.head.hasNext } } + + if (rows.hasNext) { + readPartitions += 1 + } + + rows } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala index f631ee76fcd78..a72970eef7aa4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala @@ -49,6 +49,7 @@ private[sql] trait NullableColumnBuilder extends ColumnBuilder { } abstract override def appendFrom(row: Row, ordinal: Int) { + columnStats.gatherStats(row, ordinal) if (row.isNullAt(ordinal)) { nulls = ColumnBuilder.ensureFreeSpace(nulls, 4) nulls.putInt(pos) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 8dacb84c8a17e..7943d6e1b6fb5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -243,8 +243,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { pruneFilterProject( projectList, filters, - identity[Seq[Expression]], // No filters are pushed down. - InMemoryColumnarTableScan(_, mem)) :: Nil + identity[Seq[Expression]], // All filters still need to be evaluated. + InMemoryColumnarTableScan(_, filters, mem)) :: Nil case _ => Nil } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala index 5f61fb5e16ea3..cde91ceb68c98 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala @@ -19,29 +19,30 @@ package org.apache.spark.sql.columnar import org.scalatest.FunSuite +import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.catalyst.types._ class ColumnStatsSuite extends FunSuite { - testColumnStats(classOf[BooleanColumnStats], BOOLEAN) - testColumnStats(classOf[ByteColumnStats], BYTE) - testColumnStats(classOf[ShortColumnStats], SHORT) - testColumnStats(classOf[IntColumnStats], INT) - testColumnStats(classOf[LongColumnStats], LONG) - testColumnStats(classOf[FloatColumnStats], FLOAT) - testColumnStats(classOf[DoubleColumnStats], DOUBLE) - testColumnStats(classOf[StringColumnStats], STRING) - testColumnStats(classOf[TimestampColumnStats], TIMESTAMP) - - def testColumnStats[T <: NativeType, U <: NativeColumnStats[T]]( + testColumnStats(classOf[ByteColumnStats], BYTE, Row(Byte.MaxValue, Byte.MinValue, 0)) + testColumnStats(classOf[ShortColumnStats], SHORT, Row(Short.MaxValue, Short.MinValue, 0)) + testColumnStats(classOf[IntColumnStats], INT, Row(Int.MaxValue, Int.MinValue, 0)) + testColumnStats(classOf[LongColumnStats], LONG, Row(Long.MaxValue, Long.MinValue, 0)) + testColumnStats(classOf[FloatColumnStats], FLOAT, Row(Float.MaxValue, Float.MinValue, 0)) + testColumnStats(classOf[DoubleColumnStats], DOUBLE, Row(Double.MaxValue, Double.MinValue, 0)) + testColumnStats(classOf[StringColumnStats], STRING, Row(null, null, 0)) + testColumnStats(classOf[TimestampColumnStats], TIMESTAMP, Row(null, null, 0)) + + def testColumnStats[T <: NativeType, U <: ColumnStats]( columnStatsClass: Class[U], - columnType: NativeColumnType[T]) { + columnType: NativeColumnType[T], + initialStatistics: Row) { val columnStatsName = columnStatsClass.getSimpleName test(s"$columnStatsName: empty") { val columnStats = columnStatsClass.newInstance() - assertResult(columnStats.initialBounds, "Wrong initial bounds") { - (columnStats.lowerBound, columnStats.upperBound) + columnStats.collectedStatistics.zip(initialStatistics).foreach { case (actual, expected) => + assert(actual === expected) } } @@ -49,14 +50,16 @@ class ColumnStatsSuite extends FunSuite { import ColumnarTestUtils._ val columnStats = columnStatsClass.newInstance() - val rows = Seq.fill(10)(makeRandomRow(columnType)) + val rows = Seq.fill(10)(makeRandomRow(columnType)) ++ Seq.fill(10)(makeNullRow(1)) rows.foreach(columnStats.gatherStats(_, 0)) - val values = rows.map(_.head.asInstanceOf[T#JvmType]) + val values = rows.take(10).map(_.head.asInstanceOf[T#JvmType]) val ordering = columnType.dataType.ordering.asInstanceOf[Ordering[T#JvmType]] + val stats = columnStats.collectedStatistics - assertResult(values.min(ordering), "Wrong lower bound")(columnStats.lowerBound) - assertResult(values.max(ordering), "Wrong upper bound")(columnStats.upperBound) + assertResult(values.min(ordering), "Wrong lower bound")(stats(0)) + assertResult(values.max(ordering), "Wrong upper bound")(stats(1)) + assertResult(10, "Wrong null count")(stats(2)) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala index dc813fe146c47..a77262534a352 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.execution.SparkSqlSerializer class TestNullableColumnBuilder[T <: DataType, JvmType](columnType: ColumnType[T, JvmType]) - extends BasicColumnBuilder[T, JvmType](new NoopColumnStats[T, JvmType], columnType) + extends BasicColumnBuilder[T, JvmType](new NoopColumnStats, columnType) with NullableColumnBuilder object TestNullableColumnBuilder { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala new file mode 100644 index 0000000000000..5d2fd4959197c --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.columnar + +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} + +import org.apache.spark.sql._ +import org.apache.spark.sql.test.TestSQLContext._ + +case class IntegerData(i: Int) + +class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfter { + val originalColumnBatchSize = columnBatchSize + val originalInMemoryPartitionPruning = inMemoryPartitionPruning + + override protected def beforeAll() { + // Make a table with 5 partitions, 2 batches per partition, 10 elements per batch + setConf(SQLConf.COLUMN_BATCH_SIZE, "10") + val rawData = sparkContext.makeRDD(1 to 100, 5).map(IntegerData) + rawData.registerTempTable("intData") + + // Enable in-memory partition pruning + setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, "true") + } + + override protected def afterAll() { + setConf(SQLConf.COLUMN_BATCH_SIZE, originalColumnBatchSize.toString) + setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, originalInMemoryPartitionPruning.toString) + } + + before { + cacheTable("intData") + } + + after { + uncacheTable("intData") + } + + // Comparisons + checkBatchPruning("i = 1", Seq(1), 1, 1) + checkBatchPruning("1 = i", Seq(1), 1, 1) + checkBatchPruning("i < 12", 1 to 11, 1, 2) + checkBatchPruning("i <= 11", 1 to 11, 1, 2) + checkBatchPruning("i > 88", 89 to 100, 1, 2) + checkBatchPruning("i >= 89", 89 to 100, 1, 2) + checkBatchPruning("12 > i", 1 to 11, 1, 2) + checkBatchPruning("11 >= i", 1 to 11, 1, 2) + checkBatchPruning("88 < i", 89 to 100, 1, 2) + checkBatchPruning("89 <= i", 89 to 100, 1, 2) + + // Conjunction and disjunction + checkBatchPruning("i > 8 AND i <= 21", 9 to 21, 2, 3) + checkBatchPruning("i < 2 OR i > 99", Seq(1, 100), 2, 2) + checkBatchPruning("i < 2 OR (i > 78 AND i < 92)", Seq(1) ++ (79 to 91), 3, 4) + + // With unsupported predicate + checkBatchPruning("i < 12 AND i IS NOT NULL", 1 to 11, 1, 2) + checkBatchPruning("NOT (i < 88)", 88 to 100, 5, 10) + + def checkBatchPruning( + filter: String, + expectedQueryResult: Seq[Int], + expectedReadPartitions: Int, + expectedReadBatches: Int) { + + test(filter) { + val query = sql(s"SELECT * FROM intData WHERE $filter") + assertResult(expectedQueryResult.toArray, "Wrong query result") { + query.collect().map(_.head).toArray + } + + val (readPartitions, readBatches) = query.queryExecution.executedPlan.collect { + case in: InMemoryColumnarTableScan => (in.readPartitions.value, in.readBatches.value) + }.head + + assert(readBatches === expectedReadBatches, "Wrong number of read batches") + assert(readPartitions === expectedReadPartitions, "Wrong number of read partitions") + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala index 5fba00480967c..e01cc8b4d20f2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.columnar.compression import org.scalatest.FunSuite import org.apache.spark.sql.Row -import org.apache.spark.sql.columnar.{BOOLEAN, BooleanColumnStats} +import org.apache.spark.sql.columnar.{NoopColumnStats, BOOLEAN} import org.apache.spark.sql.columnar.ColumnarTestUtils._ class BooleanBitSetSuite extends FunSuite { @@ -31,7 +31,7 @@ class BooleanBitSetSuite extends FunSuite { // Tests encoder // ------------- - val builder = TestCompressibleColumnBuilder(new BooleanColumnStats, BOOLEAN, BooleanBitSet) + val builder = TestCompressibleColumnBuilder(new NoopColumnStats, BOOLEAN, BooleanBitSet) val rows = Seq.fill[Row](count)(makeRandomRow(BOOLEAN)) val values = rows.map(_.head) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala index d8ae2a26778c9..d2969d906c943 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala @@ -31,7 +31,7 @@ class DictionaryEncodingSuite extends FunSuite { testDictionaryEncoding(new StringColumnStats, STRING) def testDictionaryEncoding[T <: NativeType]( - columnStats: NativeColumnStats[T], + columnStats: ColumnStats, columnType: NativeColumnType[T]) { val typeName = columnType.getClass.getSimpleName.stripSuffix("$") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala index 17619dcf974e3..322f447c24840 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala @@ -29,7 +29,7 @@ class IntegralDeltaSuite extends FunSuite { testIntegralDelta(new LongColumnStats, LONG, LongDelta) def testIntegralDelta[I <: IntegralType]( - columnStats: NativeColumnStats[I], + columnStats: ColumnStats, columnType: NativeColumnType[I], scheme: IntegralDelta[I]) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala index 40115beb98899..218c09ac26362 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.columnar._ import org.apache.spark.sql.columnar.ColumnarTestUtils._ class RunLengthEncodingSuite extends FunSuite { - testRunLengthEncoding(new BooleanColumnStats, BOOLEAN) + testRunLengthEncoding(new NoopColumnStats, BOOLEAN) testRunLengthEncoding(new ByteColumnStats, BYTE) testRunLengthEncoding(new ShortColumnStats, SHORT) testRunLengthEncoding(new IntColumnStats, INT) @@ -32,7 +32,7 @@ class RunLengthEncodingSuite extends FunSuite { testRunLengthEncoding(new StringColumnStats, STRING) def testRunLengthEncoding[T <: NativeType]( - columnStats: NativeColumnStats[T], + columnStats: ColumnStats, columnType: NativeColumnType[T]) { val typeName = columnType.getClass.getSimpleName.stripSuffix("$") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala index 72c19fa31d980..7db723d648d80 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala @@ -21,7 +21,7 @@ import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar._ class TestCompressibleColumnBuilder[T <: NativeType]( - override val columnStats: NativeColumnStats[T], + override val columnStats: ColumnStats, override val columnType: NativeColumnType[T], override val schemes: Seq[CompressionScheme]) extends NativeColumnBuilder(columnStats, columnType) @@ -33,7 +33,7 @@ class TestCompressibleColumnBuilder[T <: NativeType]( object TestCompressibleColumnBuilder { def apply[T <: NativeType]( - columnStats: NativeColumnStats[T], + columnStats: ColumnStats, columnType: NativeColumnType[T], scheme: CompressionScheme) = { diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index b589994bd25fa..ab487d673e813 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -35,26 +35,29 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { private val originalTimeZone = TimeZone.getDefault private val originalLocale = Locale.getDefault - private val originalUseCompression = TestHive.useCompression + private val originalColumnBatchSize = TestHive.columnBatchSize + private val originalInMemoryPartitionPruning = TestHive.inMemoryPartitionPruning def testCases = hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) override def beforeAll() { - // Enable in-memory columnar caching TestHive.cacheTables = true // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) // Add Locale setting Locale.setDefault(Locale.US) - // Enable in-memory columnar compression - TestHive.setConf(SQLConf.COMPRESS_CACHED, "true") + // Set a relatively small column batch size for testing purposes + TestHive.setConf(SQLConf.COLUMN_BATCH_SIZE, "5") + // Enable in-memory partition pruning for testing purposes + TestHive.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, "true") } override def afterAll() { TestHive.cacheTables = false TimeZone.setDefault(originalTimeZone) Locale.setDefault(originalLocale) - TestHive.setConf(SQLConf.COMPRESS_CACHED, originalUseCompression.toString) + TestHive.setConf(SQLConf.COLUMN_BATCH_SIZE, originalColumnBatchSize.toString) + TestHive.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, originalInMemoryPartitionPruning.toString) } /** A list of tests deemed out of scope currently and thus completely disregarded. */ From c5cbc49233193836b321cb6b77ce69dae798570b Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 3 Sep 2014 19:08:39 -0700 Subject: [PATCH 23/39] [SPARK-3335] [SQL] [PySpark] support broadcast in Python UDF After this patch, broadcast can be used in Python UDF. Author: Davies Liu Closes #2243 from davies/udf_broadcast and squashes the following commits: 7b88861 [Davies Liu] support broadcast in UDF --- python/pyspark/sql.py | 17 +++++++------- python/pyspark/tests.py | 22 +++++++++++++++++++ .../apache/spark/sql/UdfRegistration.scala | 3 +++ .../spark/sql/execution/pythonUdfs.scala | 3 ++- 4 files changed, 36 insertions(+), 9 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 44316926ba334..aaa35dadc203e 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -942,9 +942,7 @@ def __init__(self, sparkContext, sqlContext=None): self._jsc = self._sc._jsc self._jvm = self._sc._jvm self._pythonToJava = self._jvm.PythonRDD.pythonToJavaArray - - if sqlContext: - self._scala_SQLContext = sqlContext + self._scala_SQLContext = sqlContext @property def _ssql_ctx(self): @@ -953,7 +951,7 @@ def _ssql_ctx(self): Subclasses can override this property to provide their own JVM Contexts. """ - if not hasattr(self, '_scala_SQLContext'): + if self._scala_SQLContext is None: self._scala_SQLContext = self._jvm.SQLContext(self._jsc.sc()) return self._scala_SQLContext @@ -970,23 +968,26 @@ def registerFunction(self, name, f, returnType=StringType()): >>> sqlCtx.registerFunction("stringLengthInt", lambda x: len(x), IntegerType()) >>> sqlCtx.sql("SELECT stringLengthInt('test')").collect() [Row(c0=4)] - >>> sqlCtx.registerFunction("twoArgs", lambda x, y: len(x) + y, IntegerType()) - >>> sqlCtx.sql("SELECT twoArgs('test', 1)").collect() - [Row(c0=5)] """ func = lambda _, it: imap(lambda x: f(*x), it) command = (func, BatchedSerializer(PickleSerializer(), 1024), BatchedSerializer(PickleSerializer(), 1024)) + pickled_command = CloudPickleSerializer().dumps(command) + broadcast_vars = ListConverter().convert( + [x._jbroadcast for x in self._sc._pickled_broadcast_vars], + self._sc._gateway._gateway_client) + self._sc._pickled_broadcast_vars.clear() env = MapConverter().convert(self._sc.environment, self._sc._gateway._gateway_client) includes = ListConverter().convert(self._sc._python_includes, self._sc._gateway._gateway_client) self._ssql_ctx.registerPython(name, - bytearray(CloudPickleSerializer().dumps(command)), + bytearray(pickled_command), env, includes, self._sc.pythonExec, + broadcast_vars, self._sc._javaAccumulator, str(returnType)) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index f1a75cbff5c19..3e74799e82845 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -43,6 +43,7 @@ from pyspark.files import SparkFiles from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, ExternalSorter +from pyspark.sql import SQLContext, IntegerType _have_scipy = False _have_numpy = False @@ -525,6 +526,27 @@ def test_histogram(self): self.assertRaises(TypeError, lambda: rdd.histogram(2)) +class TestSQL(PySparkTestCase): + + def setUp(self): + PySparkTestCase.setUp(self) + self.sqlCtx = SQLContext(self.sc) + + def test_udf(self): + self.sqlCtx.registerFunction("twoArgs", lambda x, y: len(x) + y, IntegerType()) + [row] = self.sqlCtx.sql("SELECT twoArgs('test', 1)").collect() + self.assertEqual(row[0], 5) + + def test_broadcast_in_udf(self): + bar = {"a": "aa", "b": "bb", "c": "abc"} + foo = self.sc.broadcast(bar) + self.sqlCtx.registerFunction("MYUDF", lambda x: foo.value[x] if x else '') + [res] = self.sqlCtx.sql("SELECT MYUDF('c')").collect() + self.assertEqual("abc", res[0]) + [res] = self.sqlCtx.sql("SELECT MYUDF('')").collect() + self.assertEqual("", res[0]) + + class TestIO(PySparkTestCase): def test_stdout_redirection(self): diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala index 0b48e9e659faa..0ea1105f082a4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql import java.util.{List => JList, Map => JMap} import org.apache.spark.Accumulator +import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.expressions.{Expression, ScalaUdf} import org.apache.spark.sql.execution.PythonUDF @@ -38,6 +39,7 @@ protected[sql] trait UDFRegistration { envVars: JMap[String, String], pythonIncludes: JList[String], pythonExec: String, + broadcastVars: JList[Broadcast[Array[Byte]]], accumulator: Accumulator[JList[Array[Byte]]], stringDataType: String): Unit = { log.debug( @@ -61,6 +63,7 @@ protected[sql] trait UDFRegistration { envVars, pythonIncludes, pythonExec, + broadcastVars, accumulator, dataType, e) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala index 3dc8be2456781..0977da3e8577c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala @@ -42,6 +42,7 @@ private[spark] case class PythonUDF( envVars: JMap[String, String], pythonIncludes: JList[String], pythonExec: String, + broadcastVars: JList[Broadcast[Array[Byte]]], accumulator: Accumulator[JList[Array[Byte]]], dataType: DataType, children: Seq[Expression]) extends Expression with SparkLogging { @@ -145,7 +146,7 @@ case class BatchPythonEvaluation(udf: PythonUDF, output: Seq[Attribute], child: udf.pythonIncludes, false, udf.pythonExec, - Seq[Broadcast[Array[Byte]]](), + udf.broadcastVars, udf.accumulator ).mapPartitions { iter => val pickle = new Unpickler From 7c6e71f05f4f5e0cd2d038ee81d1cda4a3e5cb39 Mon Sep 17 00:00:00 2001 From: Matthew Farrellee Date: Wed, 3 Sep 2014 19:37:37 -0700 Subject: [PATCH 24/39] [SPARK-2435] Add shutdown hook to pyspark Author: Matthew Farrellee Closes #2183 from mattf/SPARK-2435 and squashes the following commits: ee0ee99 [Matthew Farrellee] [SPARK-2435] Add shutdown hook to pyspark --- python/pyspark/shell.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index e1e7cd954189f..fde3c29e5e790 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -28,6 +28,7 @@ sys.exit(1) +import atexit import os import platform import pyspark @@ -42,6 +43,7 @@ SparkContext.setSystemProperty("spark.executor.uri", os.environ["SPARK_EXECUTOR_URI"]) sc = SparkContext(appName="PySparkShell", pyFiles=add_files) +atexit.register(lambda: sc.stop()) print("""Welcome to ____ __ From 1bed0a3869a526241381d2a74ba064e5b3721336 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 3 Sep 2014 20:47:00 -0700 Subject: [PATCH 25/39] [SPARK-3372] [MLlib] MLlib doesn't pass maven build / checkstyle due to multi-byte character contained in Gradient.scala Author: Kousuke Saruta Closes #2248 from sarutak/SPARK-3372 and squashes the following commits: 73a28b8 [Kousuke Saruta] Replaced UTF-8 hyphen with ascii hyphen --- .../scala/org/apache/spark/mllib/optimization/Gradient.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala index fdd67160114ca..45dbf6044fcc5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala @@ -128,7 +128,7 @@ class LeastSquaresGradient extends Gradient { class HingeGradient extends Gradient { override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { val dotProduct = dot(data, weights) - // Our loss function with {0, 1} labels is max(0, 1 - (2y – 1) (f_w(x))) + // Our loss function with {0, 1} labels is max(0, 1 - (2y - 1) (f_w(x))) // Therefore the gradient is -(2y - 1)*x val labelScaled = 2 * label - 1.0 if (1.0 > labelScaled * dotProduct) { @@ -146,7 +146,7 @@ class HingeGradient extends Gradient { weights: Vector, cumGradient: Vector): Double = { val dotProduct = dot(data, weights) - // Our loss function with {0, 1} labels is max(0, 1 - (2y – 1) (f_w(x))) + // Our loss function with {0, 1} labels is max(0, 1 - (2y - 1) (f_w(x))) // Therefore the gradient is -(2y - 1)*x val labelScaled = 2 * label - 1.0 if (1.0 > labelScaled * dotProduct) { From 00362dac976cd05b06638deb11d990d612429e0b Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Wed, 3 Sep 2014 23:49:47 -0700 Subject: [PATCH 26/39] [HOTFIX] [SPARK-3400] Revert 9b225ac "fix GraphX EdgeRDD zipPartitions" 9b225ac3072de522b40b46aba6df1f1c231f13ef has been causing GraphX tests to fail nondeterministically, which is blocking development for others. Author: Ankur Dave Closes #2271 from ankurdave/SPARK-3400 and squashes the following commits: 10c2a97 [Ankur Dave] [HOTFIX] [SPARK-3400] Revert 9b225ac "fix GraphX EdgeRDD zipPartitions" --- .../scala/org/apache/spark/graphx/EdgeRDD.scala | 4 ++-- .../org/apache/spark/graphx/GraphSuite.scala | 16 ---------------- 2 files changed, 2 insertions(+), 18 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala index 35fbd47e6c2a5..5bcb96b136ed7 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala @@ -19,7 +19,7 @@ package org.apache.spark.graphx import scala.reflect.{classTag, ClassTag} -import org.apache.spark._ +import org.apache.spark.{OneToOneDependency, Partition, Partitioner, TaskContext} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -55,7 +55,7 @@ class EdgeRDD[@specialized ED: ClassTag, VD: ClassTag]( * partitioner that allows co-partitioning with `partitionsRDD`. */ override val partitioner = - partitionsRDD.partitioner.orElse(Some(new HashPartitioner(partitionsRDD.partitions.size))) + partitionsRDD.partitioner.orElse(Some(Partitioner.defaultPartitioner(partitionsRDD))) override def compute(part: Partition, context: TaskContext): Iterator[Edge[ED]] = { val p = firstParent[(PartitionID, EdgePartition[ED, VD])].iterator(part, context) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala index eaaa4499b6b93..6506bac73d71c 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.graphx import org.scalatest.FunSuite -import org.apache.spark.SparkConf import org.apache.spark.SparkContext import org.apache.spark.graphx.Graph._ import org.apache.spark.graphx.PartitionStrategy._ @@ -351,19 +350,4 @@ class GraphSuite extends FunSuite with LocalSparkContext { } } - test("non-default number of edge partitions") { - val n = 10 - val defaultParallelism = 3 - val numEdgePartitions = 4 - assert(defaultParallelism != numEdgePartitions) - val conf = new SparkConf() - .set("spark.default.parallelism", defaultParallelism.toString) - val sc = new SparkContext("local", "test", conf) - val edges = sc.parallelize((1 to n).map(x => (x: VertexId, 0: VertexId)), - numEdgePartitions) - val graph = Graph.fromEdgeTuples(edges, 1) - val neighborAttrSums = graph.mapReduceTriplets[Int]( - et => Iterator((et.dstId, et.srcAttr)), _ + _) - assert(neighborAttrSums.collect.toSet === Set((0: VertexId, n))) - } } From 905861906e250a0d6e6c91d9b8fd1aaa366781d3 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Thu, 4 Sep 2014 10:28:23 -0700 Subject: [PATCH 27/39] [Minor]Remove extra semicolon in FlumeStreamSuite.scala Author: GuoQiang Li Closes #2265 from witgo/FlumeStreamSuite and squashes the following commits: 6c99e6e [GuoQiang Li] Remove extra semicolon in FlumeStreamSuite.scala --- .../spark/streaming/flume/FlumeStreamSuite.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala index 73dffef953309..6ee7ac974b4a0 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala @@ -109,11 +109,11 @@ class FlumeStreamSuite extends TestSuiteBase { } class CompressionChannelFactory(compressionLevel: Int) extends NioClientSocketChannelFactory { - override def newChannel(pipeline:ChannelPipeline) : SocketChannel = { - var encoder : ZlibEncoder = new ZlibEncoder(compressionLevel); - pipeline.addFirst("deflater", encoder); - pipeline.addFirst("inflater", new ZlibDecoder()); - super.newChannel(pipeline); + override def newChannel(pipeline: ChannelPipeline): SocketChannel = { + val encoder = new ZlibEncoder(compressionLevel) + pipeline.addFirst("deflater", encoder) + pipeline.addFirst("inflater", new ZlibDecoder()) + super.newChannel(pipeline) } } } From 4feb46c5feca8d48ec340dc9c8d0eccbcd41f505 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 4 Sep 2014 10:29:11 -0700 Subject: [PATCH 28/39] [SPARK-3401][PySpark] Wrong usage of tee command in python/run-tests Author: Kousuke Saruta Closes #2272 from sarutak/SPARK-3401 and squashes the following commits: 2b35a59 [Kousuke Saruta] Modified wrong usage of tee command in python/run-tests --- python/run-tests | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/run-tests b/python/run-tests index 7b1ee3e1cddba..d671da40031c8 100755 --- a/python/run-tests +++ b/python/run-tests @@ -33,7 +33,7 @@ rm -rf metastore warehouse function run_test() { echo "Running test: $1" - SPARK_TESTING=1 $FWDIR/bin/pyspark $1 2>&1 | tee -a > unit-tests.log + SPARK_TESTING=1 $FWDIR/bin/pyspark $1 2>&1 | tee -a unit-tests.log FAILED=$((PIPESTATUS[0]||$FAILED)) # Fail and exit on the first test failure. From dc1ba9e9fc169962a9282ea6644dce09281ff598 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 4 Sep 2014 15:06:08 -0700 Subject: [PATCH 29/39] [SPARK-3378] [DOCS] Replace the word "SparkSQL" with right word "Spark SQL" Author: Kousuke Saruta Closes #2251 from sarutak/SPARK-3378 and squashes the following commits: 0bfe234 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3378 bb5938f [Kousuke Saruta] Replaced rest of "SparkSQL" with "Spark SQL" 6df66de [Kousuke Saruta] Replaced "SparkSQL" with "Spark SQL" --- dev/run-tests | 2 +- docs/programming-guide.md | 2 +- python/pyspark/sql.py | 6 +++--- python/run-tests | 2 +- .../src/main/scala/org/apache/spark/sql/api/java/Row.scala | 2 +- .../apache/spark/sql/hive/parquet/FakeParquetSerDe.scala | 2 +- 6 files changed, 8 insertions(+), 8 deletions(-) diff --git a/dev/run-tests b/dev/run-tests index d751961605dfd..90a8ce16f0f06 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -89,7 +89,7 @@ echo "=========================================================================" echo "Running Spark unit tests" echo "=========================================================================" -# Build Spark; we always build with Hive because the PySpark SparkSQL tests need it. +# Build Spark; we always build with Hive because the PySpark Spark SQL tests need it. # echo "q" is needed because sbt on encountering a build file with failure # (either resolution or compilation) prompts the user for input either q, r, # etc to quit or retry. This echo is there to make it not block. diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 6ae780d94046a..624cc744dfd51 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -385,7 +385,7 @@ Apart from text files, Spark's Python API also supports several other data forma * SequenceFile and Hadoop Input/Output Formats -**Note** this feature is currently marked ```Experimental``` and is intended for advanced users. It may be replaced in future with read/write support based on SparkSQL, in which case SparkSQL is the preferred approach. +**Note** this feature is currently marked ```Experimental``` and is intended for advanced users. It may be replaced in future with read/write support based on Spark SQL, in which case Spark SQL is the preferred approach. **Writable Support** diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index aaa35dadc203e..e7f573cf6da44 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -900,7 +900,7 @@ def __reduce__(self): class SQLContext: - """Main entry point for SparkSQL functionality. + """Main entry point for Spark SQL functionality. A SQLContext can be used create L{SchemaRDD}s, register L{SchemaRDD}s as tables, execute SQL over tables, cache tables, and read parquet files. @@ -946,7 +946,7 @@ def __init__(self, sparkContext, sqlContext=None): @property def _ssql_ctx(self): - """Accessor for the JVM SparkSQL context. + """Accessor for the JVM Spark SQL context. Subclasses can override this property to provide their own JVM Contexts. @@ -1507,7 +1507,7 @@ class SchemaRDD(RDD): """An RDD of L{Row} objects that has an associated schema. The underlying JVM object is a SchemaRDD, not a PythonRDD, so we can - utilize the relational query api exposed by SparkSQL. + utilize the relational query api exposed by Spark SQL. For normal L{pyspark.rdd.RDD} operations (map, count, etc.) the L{SchemaRDD} is not operated on directly, as it's underlying diff --git a/python/run-tests b/python/run-tests index d671da40031c8..f2a80b4f1838b 100755 --- a/python/run-tests +++ b/python/run-tests @@ -28,7 +28,7 @@ FAILED=0 rm -f unit-tests.log -# Remove the metastore and warehouse directory created by the HiveContext tests in SparkSQL +# Remove the metastore and warehouse directory created by the HiveContext tests in Spark SQL rm -rf metastore warehouse function run_test() { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala index 6c67934bda5b8..e9d04ce7aae4c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala @@ -25,7 +25,7 @@ import scala.math.BigDecimal import org.apache.spark.sql.catalyst.expressions.{Row => ScalaRow} /** - * A result row from a SparkSQL query. + * A result row from a Spark SQL query. */ class Row(private[spark] val row: ScalaRow) extends Serializable { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala index 544abfc32423c..abed299cd957f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector import org.apache.hadoop.io.Writable /** - * A placeholder that allows SparkSQL users to create metastore tables that are stored as + * A placeholder that allows Spark SQL users to create metastore tables that are stored as * parquet files. It is only intended to pass the checks that the serde is valid and exists * when a CREATE TABLE is run. The actual work of decoding will be done by ParquetTableScan * when "spark.sql.hive.convertMetastoreParquet" is set to true. From 0fdf2f5a18d88a600ca1ab7b4bf02fb0537e9411 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 4 Sep 2014 17:47:16 -0700 Subject: [PATCH 30/39] Manually close old PR Closes #1588 From 90b17a70c703c403d397e24cfbc20da22a32102d Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 4 Sep 2014 17:51:14 -0700 Subject: [PATCH 31/39] Manually close old PR Closes #544 From 3eb6ef316c2a5ee43d5ecfcf9f10c2d7adc6b819 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 4 Sep 2014 18:46:09 -0700 Subject: [PATCH 32/39] [SPARK-3310][SQL] Directly use currentTable without unnecessary implicit conversion We can directly use currentTable there without unnecessary implicit conversion. Author: Liang-Chi Hsieh Closes #2203 from viirya/direct_use_inmemoryrelation and squashes the following commits: 4741d02 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into direct_use_inmemoryrelation b671f67 [Liang-Chi Hsieh] Can directly use currentTable there without unnecessary implicit conversion. --- sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index a75af94d29303..5acb45c155ba5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -272,7 +272,7 @@ class SQLContext(@transient val sparkContext: SparkContext) val currentTable = table(tableName).queryExecution.analyzed val asInMemoryRelation = currentTable match { case _: InMemoryRelation => - currentTable.logicalPlan + currentTable case _ => InMemoryRelation(useCompression, columnBatchSize, executePlan(currentTable).executedPlan) From ee575f12f2ab059d9c1b4fa8d6c1e62248c3d11b Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 4 Sep 2014 18:47:45 -0700 Subject: [PATCH 33/39] [SPARK-2219][SQL] Added support for the "add jar" command Adds logical and physical command classes for the "add jar" command. Note that this PR conflicts with and should be merged after #2215. Author: Cheng Lian Closes #2242 from liancheng/add-jar and squashes the following commits: e43a2f1 [Cheng Lian] Updates AddJar according to conventions introduced in #2215 b99107f [Cheng Lian] Added test case for ADD JAR command 095b2c7 [Cheng Lian] Also forward ADD JAR command to Hive 9be031b [Cheng Lian] Trims Jar path string 8195056 [Cheng Lian] Added support for the "add jar" command --- .../org/apache/spark/sql/hive/HiveQl.scala | 8 +++--- .../spark/sql/hive/HiveStrategies.scala | 5 ++-- .../spark/sql/hive/execution/commands.scala | 16 ++++++++++++ .../sql/hive/execution/HiveQuerySuite.scala | 25 ++++++++++++++++--- 4 files changed, 46 insertions(+), 8 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index a4dd6be5f9e35..c98287c6aa662 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -44,6 +44,8 @@ private[hive] case class SourceCommand(filePath: String) extends Command private[hive] case class AddFile(filePath: String) extends Command +private[hive] case class AddJar(path: String) extends Command + private[hive] case class DropTable(tableName: String, ifExists: Boolean) extends Command private[hive] case class AnalyzeTable(tableName: String) extends Command @@ -231,7 +233,7 @@ private[hive] object HiveQl { } else if (sql.trim.toLowerCase.startsWith("uncache table")) { CacheCommand(sql.trim.drop(14).trim, false) } else if (sql.trim.toLowerCase.startsWith("add jar")) { - NativeCommand(sql) + AddJar(sql.trim.drop(8).trim) } else if (sql.trim.toLowerCase.startsWith("add file")) { AddFile(sql.trim.drop(9)) } else if (sql.trim.toLowerCase.startsWith("dfs")) { @@ -1018,9 +1020,9 @@ private[hive] object HiveQl { /* Other functions */ case Token("TOK_FUNCTION", Token(RAND(), Nil) :: Nil) => Rand - case Token("TOK_FUNCTION", Token(SUBSTR(), Nil) :: string :: pos :: Nil) => + case Token("TOK_FUNCTION", Token(SUBSTR(), Nil) :: string :: pos :: Nil) => Substring(nodeToExpr(string), nodeToExpr(pos), Literal(Integer.MAX_VALUE, IntegerType)) - case Token("TOK_FUNCTION", Token(SUBSTR(), Nil) :: string :: pos :: length :: Nil) => + case Token("TOK_FUNCTION", Token(SUBSTR(), Nil) :: string :: pos :: length :: Nil) => Substring(nodeToExpr(string), nodeToExpr(pos), nodeToExpr(length)) /* UDFs - Must be last otherwise will preempt built in functions */ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 24abb1b5bd1a8..72cc01cdf4c84 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -195,11 +195,12 @@ private[hive] trait HiveStrategies { case class HiveCommandStrategy(context: HiveContext) extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case logical.NativeCommand(sql) => - NativeCommand(sql, plan.output)(context) :: Nil + case logical.NativeCommand(sql) => NativeCommand(sql, plan.output)(context) :: Nil case hive.DropTable(tableName, ifExists) => execution.DropTable(tableName, ifExists) :: Nil + case hive.AddJar(path) => execution.AddJar(path) :: Nil + case hive.AnalyzeTable(tableName) => execution.AnalyzeTable(tableName) :: Nil case describe: logical.DescribeCommand => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index a1a4aa7de7bf7..d61c5e274a596 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -60,3 +60,19 @@ case class DropTable(tableName: String, ifExists: Boolean) extends LeafNode with Seq.empty[Row] } } + +/** + * :: DeveloperApi :: + */ +@DeveloperApi +case class AddJar(path: String) extends LeafNode with Command { + def hiveContext = sqlContext.asInstanceOf[HiveContext] + + override def output = Seq.empty + + override protected[sql] lazy val sideEffectResult: Seq[Row] = { + hiveContext.runSqlHive(s"ADD JAR $path") + hiveContext.sparkContext.addJar(path) + Seq.empty[Row] + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index c4abb3eb4861f..f4217a52c3822 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -17,9 +17,11 @@ package org.apache.spark.sql.hive.execution +import java.io.File + import scala.util.Try -import org.apache.spark.sql.{SchemaRDD, Row} +import org.apache.spark.SparkException import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ @@ -313,7 +315,7 @@ class HiveQuerySuite extends HiveComparisonTest { "SELECT srcalias.KEY, SRCALIAS.value FROM sRc SrCAlias WHERE SrCAlias.kEy < 15") test("case sensitivity: registered table") { - val testData: SchemaRDD = + val testData = TestHive.sparkContext.parallelize( TestData(1, "str1") :: TestData(2, "str2") :: Nil) @@ -467,7 +469,7 @@ class HiveQuerySuite extends HiveComparisonTest { } // Describe a registered temporary table. - val testData: SchemaRDD = + val testData = TestHive.sparkContext.parallelize( TestData(1, "str1") :: TestData(1, "str2") :: Nil) @@ -495,6 +497,23 @@ class HiveQuerySuite extends HiveComparisonTest { } } + test("ADD JAR command") { + val testJar = TestHive.getHiveFile("data/files/TestSerDe.jar").getCanonicalPath + sql("CREATE TABLE alter1(a INT, b INT)") + intercept[Exception] { + sql( + """ALTER TABLE alter1 SET SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' + |WITH serdeproperties('s1'='9') + """.stripMargin) + } + sql(s"ADD JAR $testJar") + sql( + """ALTER TABLE alter1 SET SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' + |WITH serdeproperties('s1'='9') + """.stripMargin) + sql("DROP TABLE alter1") + } + test("parse HQL set commands") { // Adapted from its SQL counterpart. val testKey = "spark.sql.key.usedfortestonly" From 1904bac38d97df5ae9fb193e92a83c7f8ff6d255 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Thu, 4 Sep 2014 19:16:12 -0700 Subject: [PATCH 34/39] [SPARK-3392] [SQL] Show value spark.sql.shuffle.partitions for mapred.reduce.tasks This is a tiny fix for getting the value of "mapred.reduce.tasks", which make more sense for the hive user. As well as the command "set -v", which should output verbose information for all of the key/values. Author: Cheng Hao Closes #2261 from chenghao-intel/set_mapreduce_tasks and squashes the following commits: 653858a [Cheng Hao] show value spark.sql.shuffle.partitions for mapred.reduce.tasks --- .../apache/spark/sql/execution/commands.scala | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 286c6d264f86a..94543fc95b470 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -60,10 +60,10 @@ case class SetCommand( logWarning(s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + s"automatically converted to ${SQLConf.SHUFFLE_PARTITIONS} instead.") context.setConf(SQLConf.SHUFFLE_PARTITIONS, v) - Array(Row(s"${SQLConf.SHUFFLE_PARTITIONS}=$v")) + Seq(Row(s"${SQLConf.SHUFFLE_PARTITIONS}=$v")) } else { context.setConf(k, v) - Array(Row(s"$k=$v")) + Seq(Row(s"$k=$v")) } // Query the value bound to key k. @@ -78,11 +78,19 @@ case class SetCommand( "hive-hwi-0.12.0.jar", "hive-0.12.0.jar").mkString(":") - Array( + context.getAllConfs.map { case (k, v) => + Row(s"$k=$v") + }.toSeq ++ Seq( Row("system:java.class.path=" + hiveJars), Row("system:sun.java.command=shark.SharkServer2")) } else { - Array(Row(s"$k=${context.getConf(k, "")}")) + if (k == SQLConf.Deprecated.MAPRED_REDUCE_TASKS) { + logWarning(s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + + s"showing ${SQLConf.SHUFFLE_PARTITIONS} instead.") + Seq(Row(s"${SQLConf.SHUFFLE_PARTITIONS}=${context.numShufflePartitions}")) + } else { + Seq(Row(s"$k=${context.getConf(k, "")}")) + } } // Query all key-value pairs that are set in the SQLConf of the context. From 1725a1a5d10a53762bd80f391eddbf306f2841ee Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 4 Sep 2014 23:34:58 -0700 Subject: [PATCH 35/39] [SPARK-3391][EC2] Support attaching up to 8 EBS volumes. Please merge this at the same time as https://github.com/mesos/spark-ec2/pull/66 Author: Reynold Xin Closes #2260 from rxin/ec2-ebs-vol and squashes the following commits: b9527d9 [Reynold Xin] Removed io1 ebs type. bf9c403 [Reynold Xin] Made EBS volume type configurable. c8e25ea [Reynold Xin] Support up to 8 EBS volumes. adf4f2e [Reynold Xin] Revert git repo change. 020c542 [Reynold Xin] [SPARK-3391] Support attaching more than 1 EBS volumes. --- ec2/spark_ec2.py | 33 +++++++++++++++++++++++++-------- 1 file changed, 25 insertions(+), 8 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index eed6eb8485183..1670faca4a480 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -102,9 +102,17 @@ def parse_args(): "(for debugging)") parser.add_option( "--ebs-vol-size", metavar="SIZE", type="int", default=0, - help="Attach a new EBS volume of size SIZE (in GB) to each node as " + - "/vol. The volumes will be deleted when the instances terminate. " + - "Only possible on EBS-backed AMIs.") + help="Size (in GB) of each EBS volume.") + parser.add_option( + "--ebs-vol-type", default="standard", + help="EBS volume type (e.g. 'gp2', 'standard').") + parser.add_option( + "--ebs-vol-num", type="int", default=1, + help="Number of EBS volumes to attach to each node as /vol[x]. " + + "The volumes will be deleted when the instances terminate. " + + "Only possible on EBS-backed AMIs. " + + "EBS volumes are only attached if --ebs-vol-size > 0." + + "Only support up to 8 EBS volumes.") parser.add_option( "--swap", metavar="SWAP", type="int", default=1024, help="Swap space to set up per node, in MB (default: 1024)") @@ -348,13 +356,16 @@ def launch_cluster(conn, opts, cluster_name): print >> stderr, "Could not find AMI " + opts.ami sys.exit(1) - # Create block device mapping so that we can add an EBS volume if asked to + # Create block device mapping so that we can add EBS volumes if asked to. + # The first drive is attached as /dev/sds, 2nd as /dev/sdt, ... /dev/sdz block_map = BlockDeviceMapping() if opts.ebs_vol_size > 0: - device = EBSBlockDeviceType() - device.size = opts.ebs_vol_size - device.delete_on_termination = True - block_map["/dev/sdv"] = device + for i in range(opts.ebs_vol_num): + device = EBSBlockDeviceType() + device.size = opts.ebs_vol_size + device.volume_type=opts.ebs_vol_type + device.delete_on_termination = True + block_map["/dev/sd" + chr(ord('s') + i)] = device # AWS ignores the AMI-specified block device mapping for M3 (see SPARK-3342). if opts.instance_type.startswith('m3.'): @@ -828,6 +839,12 @@ def get_partition(total, num_partitions, current_partitions): def real_main(): (opts, action, cluster_name) = parse_args() + + # Input parameter validation + if opts.ebs_vol_num > 8: + print >> stderr, "ebs-vol-num cannot be greater than 8" + sys.exit(1) + try: conn = ec2.connect_to_region(opts.region) except Exception as e: From 6a37ed838b3cbf96f7a904f3d3dabf99141729f5 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Thu, 4 Sep 2014 23:37:06 -0700 Subject: [PATCH 36/39] [Docs] fix minor MLlib case typo Also make the list of features consistent in style. Author: Nicholas Chammas Closes #2278 from nchammas/patch-1 and squashes the following commits: 56df319 [Nicholas Chammas] [Docs] fix minor MLlib case typo --- README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 0a683a460ffac..5b09ad86849e7 100644 --- a/README.md +++ b/README.md @@ -4,8 +4,8 @@ Spark is a fast and general cluster computing system for Big Data. It provides high-level APIs in Scala, Java, and Python, and an optimized engine that supports general computation graphs for data analysis. It also supports a rich set of higher-level tools including Spark SQL for SQL and structured -data processing, MLLib for machine learning, GraphX for graph processing, -and Spark Streaming. +data processing, MLlib for machine learning, GraphX for graph processing, +and Spark Streaming for stream processing. From 51b53a758c85f2e20ad9bd73ed815fcfa9c7180b Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 5 Sep 2014 09:54:40 -0500 Subject: [PATCH 37/39] [SPARK-3260] yarn - pass acls along with executor launch Pass along the acl settings when we launch a container so that they can be applied to viewing the logs on a running NodeManager. Author: Thomas Graves Closes #2185 from tgravescs/SPARK-3260 and squashes the following commits: 6f94b5a [Thomas Graves] make unit test more robust 28b9dd3 [Thomas Graves] yarn - pass acls along with executor launch --- .../spark/deploy/yarn/ExecutorRunnable.scala | 7 +- .../deploy/yarn/YarnAllocationHandler.scala | 7 +- .../spark/deploy/yarn/YarnRMClientImpl.scala | 7 +- .../spark/deploy/yarn/ApplicationMaster.scala | 13 ++-- .../apache/spark/deploy/yarn/ClientBase.scala | 6 +- .../spark/deploy/yarn/YarnAllocator.scala | 10 ++- .../spark/deploy/yarn/YarnRMClient.scala | 5 +- .../deploy/yarn/YarnSparkHadoopUtil.scala | 11 ++- .../yarn/YarnSparkHadoopUtilSuite.scala | 76 ++++++++++++++++++- .../spark/deploy/yarn/ExecutorRunnable.scala | 7 +- .../deploy/yarn/YarnAllocationHandler.scala | 7 +- .../spark/deploy/yarn/YarnRMClientImpl.scala | 7 +- 12 files changed, 129 insertions(+), 34 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 7dae248e3e7db..10cbeb8b94325 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -35,7 +35,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records, ProtoUtils} -import org.apache.spark.{SparkConf, Logging} +import org.apache.spark.{SecurityManager, SparkConf, Logging} class ExecutorRunnable( @@ -46,7 +46,8 @@ class ExecutorRunnable( slaveId: String, hostname: String, executorMemory: Int, - executorCores: Int) + executorCores: Int, + securityMgr: SecurityManager) extends Runnable with ExecutorRunnableUtil with Logging { var rpc: YarnRPC = YarnRPC.create(conf) @@ -86,6 +87,8 @@ class ExecutorRunnable( logInfo("Setting up executor with commands: " + commands) ctx.setCommands(commands) + ctx.setApplicationACLs(YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr)) + // Send the start request to the ContainerManager val startReq = Records.newRecord(classOf[StartContainerRequest]) .asInstanceOf[StartContainerRequest] diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 9f9e16c06452b..85d6274df2fcb 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -23,7 +23,7 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} -import org.apache.spark.SparkConf +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.scheduler.SplitInfo import org.apache.hadoop.conf.Configuration @@ -41,8 +41,9 @@ private[yarn] class YarnAllocationHandler( resourceManager: AMRMProtocol, appAttemptId: ApplicationAttemptId, args: ApplicationMasterArguments, - preferredNodes: collection.Map[String, collection.Set[SplitInfo]]) - extends YarnAllocator(conf, sparkConf, args, preferredNodes) { + preferredNodes: collection.Map[String, collection.Set[SplitInfo]], + securityMgr: SecurityManager) + extends YarnAllocator(conf, sparkConf, args, preferredNodes, securityMgr) { private val lastResponseId = new AtomicInteger() private val releaseList: CopyOnWriteArrayList[ContainerId] = new CopyOnWriteArrayList() diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala index cc5392192ec51..ad27a9ab781d2 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala @@ -27,7 +27,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{ConverterUtils, Records} -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.scheduler.SplitInfo import org.apache.spark.util.Utils @@ -45,7 +45,8 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC sparkConf: SparkConf, preferredNodeLocations: Map[String, Set[SplitInfo]], uiAddress: String, - uiHistoryAddress: String) = { + uiHistoryAddress: String, + securityMgr: SecurityManager) = { this.rpc = YarnRPC.create(conf) this.uiHistoryAddress = uiHistoryAddress @@ -53,7 +54,7 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC registerApplicationMaster(uiAddress) new YarnAllocationHandler(conf, sparkConf, resourceManager, getAttemptId(), args, - preferredNodeLocations) + preferredNodeLocations, securityMgr) } override def getAttemptId() = { diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 98039a20de245..a879c833a014f 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -116,7 +116,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, val securityMgr = new SecurityManager(sparkConf) if (isDriver) { - runDriver() + runDriver(securityMgr) } else { runExecutorLauncher(securityMgr) } @@ -157,7 +157,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, sparkContextRef.compareAndSet(sc, null) } - private def registerAM(uiAddress: String) = { + private def registerAM(uiAddress: String, securityMgr: SecurityManager) = { val sc = sparkContextRef.get() val appId = client.getAttemptId().getApplicationId().toString() @@ -170,13 +170,14 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, if (sc != null) sc.getConf else sparkConf, if (sc != null) sc.preferredNodeLocationData else Map(), uiAddress, - historyAddress) + historyAddress, + securityMgr) allocator.allocateResources() reporterThread = launchReporterThread() } - private def runDriver(): Unit = { + private def runDriver(securityMgr: SecurityManager): Unit = { addAmIpFilter() val userThread = startUserClass() @@ -188,7 +189,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, if (sc == null) { finish(FinalApplicationStatus.FAILED, "Timed out waiting for SparkContext.") } else { - registerAM(sc.ui.appUIHostPort) + registerAM(sc.ui.appUIHostPort, securityMgr) try { userThread.join() } finally { @@ -203,7 +204,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, conf = sparkConf, securityManager = securityMgr)._1 actor = waitForSparkDriver() addAmIpFilter() - registerAM(sparkConf.get("spark.driver.appUIAddress", "")) + registerAM(sparkConf.get("spark.driver.appUIAddress", ""), securityMgr) // In client mode the actor will stop the reporter thread. reporterThread.join() diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 5d8e5e6dffe7f..8075b7a7fb837 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -430,10 +430,8 @@ trait ClientBase extends Logging { // send the acl settings into YARN to control who has access via YARN interfaces val securityManager = new SecurityManager(sparkConf) - val acls = Map[ApplicationAccessType, String] ( - ApplicationAccessType.VIEW_APP -> securityManager.getViewAcls, - ApplicationAccessType.MODIFY_APP -> securityManager.getModifyAcls) - amContainer.setApplicationACLs(acls) + amContainer.setApplicationACLs(YarnSparkHadoopUtil.getApplicationAclsForYarn(securityManager)) + amContainer } } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index c74dd1c2b21dc..02b9a81bf6b50 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -28,7 +28,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse -import org.apache.spark.{Logging, SparkConf, SparkEnv} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv} import org.apache.spark.scheduler.{SplitInfo, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend @@ -55,7 +55,8 @@ private[yarn] abstract class YarnAllocator( conf: Configuration, sparkConf: SparkConf, args: ApplicationMasterArguments, - preferredNodes: collection.Map[String, collection.Set[SplitInfo]]) + preferredNodes: collection.Map[String, collection.Set[SplitInfo]], + securityMgr: SecurityManager) extends Logging { // These three are locked on allocatedHostToContainersMap. Complementary data structures @@ -280,7 +281,8 @@ private[yarn] abstract class YarnAllocator( executorId, executorHostname, executorMemory, - executorCores) + executorCores, + securityMgr) new Thread(executorRunnable).start() } } @@ -444,4 +446,4 @@ private[yarn] abstract class YarnAllocator( } -} \ No newline at end of file +} diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala index 922d7d1a854a5..ed65e56b3e413 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala @@ -22,7 +22,7 @@ import scala.collection.{Map, Set} import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.api.records._ -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.{SecurityManager, SparkConf, SparkContext} import org.apache.spark.scheduler.SplitInfo /** @@ -45,7 +45,8 @@ trait YarnRMClient { sparkConf: SparkConf, preferredNodeLocations: Map[String, Set[SplitInfo]], uiAddress: String, - uiHistoryAddress: String): YarnAllocator + uiHistoryAddress: String, + securityMgr: SecurityManager): YarnAllocator /** * Shuts down the AM. Guaranteed to only be called once. diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index dc77f1236492d..4a33e34c3bfc7 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -32,10 +32,11 @@ import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.util.StringInterner import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.api.ApplicationConstants +import org.apache.hadoop.yarn.api.records.ApplicationAccessType import org.apache.hadoop.yarn.util.RackResolver import org.apache.hadoop.conf.Configuration -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.{SecurityManager, SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.Utils @@ -211,4 +212,12 @@ object YarnSparkHadoopUtil { } } + private[spark] def getApplicationAclsForYarn(securityMgr: SecurityManager): + Map[ApplicationAccessType, String] = { + Map[ApplicationAccessType, String] ( + ApplicationAccessType.VIEW_APP -> securityMgr.getViewAcls, + ApplicationAccessType.MODIFY_APP -> securityMgr.getModifyAcls + ) + } + } diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala index 75db8ee6d468f..2cc5abb3a890c 100644 --- a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala +++ b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala @@ -23,7 +23,10 @@ import com.google.common.io.{ByteStreams, Files} import org.apache.hadoop.yarn.conf.YarnConfiguration import org.scalatest.{FunSuite, Matchers} -import org.apache.spark.{Logging, SparkConf} +import org.apache.hadoop.yarn.api.records.ApplicationAccessType + +import org.apache.spark.{Logging, SecurityManager, SparkConf} + class YarnSparkHadoopUtilSuite extends FunSuite with Matchers with Logging { @@ -74,4 +77,75 @@ class YarnSparkHadoopUtilSuite extends FunSuite with Matchers with Logging { yarnConf.get(key) should not be default.get(key) } + + test("test getApplicationAclsForYarn acls on") { + + // spark acls on, just pick up default user + val sparkConf = new SparkConf() + sparkConf.set("spark.acls.enable", "true") + + val securityMgr = new SecurityManager(sparkConf) + val acls = YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr) + + val viewAcls = acls.get(ApplicationAccessType.VIEW_APP) + val modifyAcls = acls.get(ApplicationAccessType.MODIFY_APP) + + viewAcls match { + case Some(vacls) => { + val aclSet = vacls.split(',').map(_.trim).toSet + assert(aclSet.contains(System.getProperty("user.name", "invalid"))) + } + case None => { + fail() + } + } + modifyAcls match { + case Some(macls) => { + val aclSet = macls.split(',').map(_.trim).toSet + assert(aclSet.contains(System.getProperty("user.name", "invalid"))) + } + case None => { + fail() + } + } + } + + test("test getApplicationAclsForYarn acls on and specify users") { + + // default spark acls are on and specify acls + val sparkConf = new SparkConf() + sparkConf.set("spark.acls.enable", "true") + sparkConf.set("spark.ui.view.acls", "user1,user2") + sparkConf.set("spark.modify.acls", "user3,user4") + + val securityMgr = new SecurityManager(sparkConf) + val acls = YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr) + + val viewAcls = acls.get(ApplicationAccessType.VIEW_APP) + val modifyAcls = acls.get(ApplicationAccessType.MODIFY_APP) + + viewAcls match { + case Some(vacls) => { + val aclSet = vacls.split(',').map(_.trim).toSet + assert(aclSet.contains("user1")) + assert(aclSet.contains("user2")) + assert(aclSet.contains(System.getProperty("user.name", "invalid"))) + } + case None => { + fail() + } + } + modifyAcls match { + case Some(macls) => { + val aclSet = macls.split(',').map(_.trim).toSet + assert(aclSet.contains("user3")) + assert(aclSet.contains("user4")) + assert(aclSet.contains(System.getProperty("user.name", "invalid"))) + } + case None => { + fail() + } + } + + } } diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 07ba0a4b30bd7..833be12982e71 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -35,7 +35,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records} -import org.apache.spark.{SparkConf, Logging} +import org.apache.spark.{SecurityManager, SparkConf, Logging} class ExecutorRunnable( @@ -46,7 +46,8 @@ class ExecutorRunnable( slaveId: String, hostname: String, executorMemory: Int, - executorCores: Int) + executorCores: Int, + securityMgr: SecurityManager) extends Runnable with ExecutorRunnableUtil with Logging { var rpc: YarnRPC = YarnRPC.create(conf) @@ -85,6 +86,8 @@ class ExecutorRunnable( logInfo("Setting up executor with commands: " + commands) ctx.setCommands(commands) + ctx.setApplicationACLs(YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr)) + // Send the start request to the ContainerManager nmClient.startContainer(container, ctx) } diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index ed31457b61571..c887cb52dd9cf 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -20,7 +20,7 @@ package org.apache.spark.deploy.yarn import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} -import org.apache.spark.SparkConf +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.scheduler.SplitInfo import org.apache.hadoop.conf.Configuration @@ -39,8 +39,9 @@ private[yarn] class YarnAllocationHandler( amClient: AMRMClient[ContainerRequest], appAttemptId: ApplicationAttemptId, args: ApplicationMasterArguments, - preferredNodes: collection.Map[String, collection.Set[SplitInfo]]) - extends YarnAllocator(conf, sparkConf, args, preferredNodes) { + preferredNodes: collection.Map[String, collection.Set[SplitInfo]], + securityMgr: SecurityManager) + extends YarnAllocator(conf, sparkConf, args, preferredNodes, securityMgr) { override protected def releaseContainer(container: Container) = { amClient.releaseAssignedContainer(container.getId()) diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala index e8b8d9bc722bd..54bc6b14c44ce 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala @@ -28,7 +28,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.util.ConverterUtils import org.apache.hadoop.yarn.webapp.util.WebAppUtils -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.scheduler.SplitInfo import org.apache.spark.util.Utils @@ -46,7 +46,8 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC sparkConf: SparkConf, preferredNodeLocations: Map[String, Set[SplitInfo]], uiAddress: String, - uiHistoryAddress: String) = { + uiHistoryAddress: String, + securityMgr: SecurityManager) = { amClient = AMRMClient.createAMRMClient() amClient.init(conf) amClient.start() @@ -55,7 +56,7 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC logInfo("Registering the ApplicationMaster") amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress) new YarnAllocationHandler(conf, sparkConf, amClient, getAttemptId(), args, - preferredNodeLocations) + preferredNodeLocations, securityMgr) } override def shutdown(status: FinalApplicationStatus, diagnostics: String = "") = From 62c557609929982eeec170fe12f810bedfcf97f2 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 5 Sep 2014 09:56:22 -0500 Subject: [PATCH 38/39] [SPARK-3375] spark on yarn container allocation issues If yarn doesn't get the containers immediately it stops asking for them and the yarn application hangs with never getting any executors. The issue here is that we are sending the number of containers as 0 after we send the original one of X. on the yarn side this clears out the original request. For a ping we should just send empty asks. Author: Thomas Graves Closes #2275 from tgravescs/SPARK-3375 and squashes the following commits: 74b6820 [Thomas Graves] send empty resource requests when we aren't asking for containers --- .../spark/deploy/yarn/YarnAllocationHandler.scala | 13 +++++++------ .../spark/deploy/yarn/YarnAllocationHandler.scala | 8 +++++--- 2 files changed, 12 insertions(+), 9 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 85d6274df2fcb..5a1b42c1e17d5 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -51,12 +51,13 @@ private[yarn] class YarnAllocationHandler( override protected def allocateContainers(count: Int): YarnAllocateResponse = { var resourceRequests: List[ResourceRequest] = null - // default. - if (count <= 0 || preferredHostToCount.isEmpty) { - logDebug("numExecutors: " + count + ", host preferences: " + - preferredHostToCount.isEmpty) - resourceRequests = List(createResourceRequest( - AllocationType.ANY, null, count, YarnSparkHadoopUtil.RM_REQUEST_PRIORITY)) + logDebug("numExecutors: " + count) + if (count <= 0) { + resourceRequests = List() + } else if (preferredHostToCount.isEmpty) { + logDebug("host preferences is empty") + resourceRequests = List(createResourceRequest( + AllocationType.ANY, null, count, YarnSparkHadoopUtil.RM_REQUEST_PRIORITY)) } else { // request for all hosts in preferred nodes and for numExecutors - // candidates.size, request by default allocation policy. diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index c887cb52dd9cf..5438f151ac0ad 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -88,9 +88,11 @@ private[yarn] class YarnAllocationHandler( private def addResourceRequests(numExecutors: Int) { val containerRequests: List[ContainerRequest] = - if (numExecutors <= 0 || preferredHostToCount.isEmpty) { - logDebug("numExecutors: " + numExecutors + ", host preferences: " + - preferredHostToCount.isEmpty) + if (numExecutors <= 0) { + logDebug("numExecutors: " + numExecutors) + List() + } else if (preferredHostToCount.isEmpty) { + logDebug("host preferences is empty") createResourceRequests( AllocationType.ANY, resource = null, From 7ff8c45d714e0f2315910838b739c0c034672015 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 5 Sep 2014 11:07:00 -0700 Subject: [PATCH 39/39] [SPARK-3399][PySpark] Test for PySpark should ignore HADOOP_CONF_DIR and YARN_CONF_DIR Author: Kousuke Saruta Closes #2270 from sarutak/SPARK-3399 and squashes the following commits: 7613be6 [Kousuke Saruta] Modified pyspark script to ignore environment variables YARN_CONF_DIR and HADOOP_CONF_DIR while testing --- bin/pyspark | 2 ++ 1 file changed, 2 insertions(+) diff --git a/bin/pyspark b/bin/pyspark index f553b314c5991..26a16dd600b7a 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -85,6 +85,8 @@ export PYSPARK_SUBMIT_ARGS # For pyspark tests if [[ -n "$SPARK_TESTING" ]]; then + unset YARN_CONF_DIR + unset HADOOP_CONF_DIR if [[ -n "$PYSPARK_DOC_TEST" ]]; then exec "$PYSPARK_PYTHON" -m doctest $1 else