From 6f61e1f961826a6c9e98a66d10b271b7e3c7dd55 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 5 Dec 2014 10:27:40 -0800 Subject: [PATCH 001/155] [SPARK-4761][SQL] Enables Kryo by default in Spark SQL Thrift server Enables Kryo and disables reference tracking by default in Spark SQL Thrift server. Configurations explicitly defined by users in `spark-defaults.conf` are respected (the Thrift server is started by `spark-submit`, which handles configuration properties properly). [Review on Reviewable](https://reviewable.io/reviews/apache/spark/3621) Author: Cheng Lian Closes #3621 from liancheng/kryo-by-default and squashes the following commits: 70c2775 [Cheng Lian] Enables Kryo by default in Spark SQL Thrift server --- .../spark/sql/hive/thriftserver/SparkSQLEnv.scala | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala index 89732c939b0ec..158c225159720 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -32,11 +32,21 @@ private[hive] object SparkSQLEnv extends Logging { def init() { if (hiveContext == null) { - val sparkConf = new SparkConf() + val sparkConf = new SparkConf(loadDefaults = true) + val maybeSerializer = sparkConf.getOption("spark.serializer") + val maybeKryoReferenceTracking = sparkConf.getOption("spark.kryo.referenceTracking") + + sparkConf .setAppName(s"SparkSQL::${java.net.InetAddress.getLocalHost.getHostName}") .set("spark.sql.hive.version", HiveShim.version) - sparkContext = new SparkContext(sparkConf) + .set( + "spark.serializer", + maybeSerializer.getOrElse("org.apache.spark.serializer.KryoSerializer")) + .set( + "spark.kryo.referenceTracking", + maybeKryoReferenceTracking.getOrElse("false")) + sparkContext = new SparkContext(sparkConf) sparkContext.addSparkListener(new StatsReportListener()) hiveContext = new HiveContext(sparkContext) From 98a7d09978eeb775600ff41f9cc6ae8622026b71 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Fri, 5 Dec 2014 12:00:32 -0800 Subject: [PATCH 002/155] [SPARK-4005][CORE] handle message replies in receive instead of in the individual private methods In BlockManagermasterActor, when handling message type UpdateBlockInfo, the message replies is in handled in individual private methods, should handle it in receive of Akka. Author: Zhang, Liye Closes #2853 from liyezhang556520/akkaRecv and squashes the following commits: 9b06f0a [Zhang, Liye] remove the unreachable code bf518cd [Zhang, Liye] change the indent 242166b [Zhang, Liye] modified accroding to the comments d4b929b [Zhang, Liye] [SPARK-4005][CORE] handle message replies in receive instead of in the individual private methods --- .../spark/storage/BlockManagerMasterActor.scala | 17 +++++++---------- 1 file changed, 7 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 685b2e11440fb..9cbda41223a8b 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -73,9 +73,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus case UpdateBlockInfo( blockManagerId, blockId, storageLevel, deserializedSize, size, tachyonSize) => - // TODO: Ideally we want to handle all the message replies in receive instead of in the - // individual private methods. - updateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size, tachyonSize) + sender ! updateBlockInfo( + blockManagerId, blockId, storageLevel, deserializedSize, size, tachyonSize) case GetLocations(blockId) => sender ! getLocations(blockId) @@ -355,23 +354,21 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus storageLevel: StorageLevel, memSize: Long, diskSize: Long, - tachyonSize: Long) { + tachyonSize: Long): Boolean = { if (!blockManagerInfo.contains(blockManagerId)) { if (blockManagerId.isDriver && !isLocal) { // We intentionally do not register the master (except in local mode), // so we should not indicate failure. - sender ! true + return true } else { - sender ! false + return false } - return } if (blockId == null) { blockManagerInfo(blockManagerId).updateLastSeenMs() - sender ! true - return + return true } blockManagerInfo(blockManagerId).updateBlockInfo( @@ -395,7 +392,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus if (locations.size == 0) { blockLocations.remove(blockId) } - sender ! true + true } private def getLocations(blockId: BlockId): Seq[BlockManagerId] = { From 6eb1b6f6204ea3c8083af3fb9cd990d9f3dac89d Mon Sep 17 00:00:00 2001 From: CrazyJvm Date: Fri, 5 Dec 2014 13:42:13 -0800 Subject: [PATCH 003/155] Streaming doc : do you mean inadvertently? Author: CrazyJvm Closes #3620 from CrazyJvm/streaming-foreachRDD and squashes the following commits: b72886b [CrazyJvm] do you mean inadvertently? --- docs/streaming-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 44a1f3ad7560b..5ebe834a32d31 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -1081,7 +1081,7 @@ Some of the common mistakes to avoid are as follows. - 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 +For this purpose, a developer may inadvertently 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), From e895e0cbecbbec1b412ff21321e57826d2d0a982 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Sat, 6 Dec 2014 00:56:51 -0800 Subject: [PATCH 004/155] [SPARK-3623][GraphX] GraphX should support the checkpoint operation Author: GuoQiang Li Closes #2631 from witgo/SPARK-3623 and squashes the following commits: a70c500 [GuoQiang Li] Remove java related 4d1e249 [GuoQiang Li] Add comments e682724 [GuoQiang Li] Graph should support the checkpoint operation --- .../scala/org/apache/spark/graphx/Graph.scala | 8 +++++++ .../apache/spark/graphx/impl/GraphImpl.scala | 5 +++++ .../org/apache/spark/graphx/GraphSuite.scala | 21 +++++++++++++++++++ 3 files changed, 34 insertions(+) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala index 637791543514c..23538b71562de 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -96,6 +96,14 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab */ def cache(): Graph[VD, ED] + /** + * Mark this Graph for checkpointing. It will be saved to a file inside the checkpoint + * directory set with SparkContext.setCheckpointDir() and all references to its parent + * RDDs will be removed. It is strongly recommended that this Graph is persisted in + * memory, otherwise saving it on a file will require recomputation. + */ + def checkpoint(): Unit + /** * Uncaches only the vertices of this graph, leaving the edges alone. This is useful in iterative * algorithms that modify the vertex attributes but reuse the edges. This method can be used to diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala index 0eae2a673874a..a617d84aea9d4 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -65,6 +65,11 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( this } + override def checkpoint(): Unit = { + vertices.checkpoint() + replicatedVertexView.edges.checkpoint() + } + override def unpersistVertices(blocking: Boolean = true): Graph[VD, ED] = { vertices.unpersist(blocking) // TODO: unpersist the replicated vertices in `replicatedVertexView` but leave the edges alone 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 a05d1ddb21295..9da0064104fb6 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,8 @@ package org.apache.spark.graphx import org.scalatest.FunSuite +import com.google.common.io.Files + import org.apache.spark.SparkContext import org.apache.spark.graphx.Graph._ import org.apache.spark.graphx.PartitionStrategy._ @@ -365,4 +367,23 @@ class GraphSuite extends FunSuite with LocalSparkContext { } } + test("checkpoint") { + val checkpointDir = Files.createTempDir() + checkpointDir.deleteOnExit() + withSpark { sc => + sc.setCheckpointDir(checkpointDir.getAbsolutePath) + val ring = (0L to 100L).zip((1L to 99L) :+ 0L).map { case (a, b) => Edge(a, b, 1)} + val rdd = sc.parallelize(ring) + val graph = Graph.fromEdges(rdd, 1.0F) + graph.checkpoint() + graph.edges.map(_.attr).count() + graph.vertices.map(_._2).count() + + val edgesDependencies = graph.edges.partitionsRDD.dependencies + val verticesDependencies = graph.vertices.partitionsRDD.dependencies + assert(edgesDependencies.forall(_.rdd.isInstanceOf[CheckpointRDD[_]])) + assert(verticesDependencies.forall(_.rdd.isInstanceOf[CheckpointRDD[_]])) + } + } + } From 2e6b736b0e6e5920d0523533c87832a53211db42 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Sun, 7 Dec 2014 19:36:08 -0800 Subject: [PATCH 005/155] [SPARK-4646] Replace Scala.util.Sorting.quickSort with Sorter(TimSort) in Spark This patch just replaces a native quick sorter with Sorter(TimSort) in Spark. It could get performance gains by ~8% in my quick experiments. Author: Takeshi Yamamuro Closes #3507 from maropu/TimSortInEdgePartitionBuilderSpike and squashes the following commits: 8d4e5d2 [Takeshi Yamamuro] Remove a wildcard import 3527e00 [Takeshi Yamamuro] Replace Scala.util.Sorting.quickSort with Sorter(TimSort) in Spark --- .../scala/org/apache/spark/graphx/Edge.scala | 30 ++++++++++++++ .../graphx/impl/EdgePartitionBuilder.scala | 39 ++++++++++++++++--- 2 files changed, 64 insertions(+), 5 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala index 7e842ec4cc82f..ecc37dcaad1fe 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala @@ -17,6 +17,8 @@ package org.apache.spark.graphx +import org.apache.spark.util.collection.SortDataFormat + /** * A single directed edge consisting of a source id, target id, * and the data associated with the edge. @@ -65,4 +67,32 @@ object Edge { else 1 } } + + private[graphx] def edgeArraySortDataFormat[ED] = new SortDataFormat[Edge[ED], Array[Edge[ED]]] { + override def getKey(data: Array[Edge[ED]], pos: Int): Edge[ED] = { + data(pos) + } + + override def swap(data: Array[Edge[ED]], pos0: Int, pos1: Int): Unit = { + val tmp = data(pos0) + data(pos0) = data(pos1) + data(pos1) = tmp + } + + override def copyElement( + src: Array[Edge[ED]], srcPos: Int, + dst: Array[Edge[ED]], dstPos: Int) { + dst(dstPos) = src(srcPos) + } + + override def copyRange( + src: Array[Edge[ED]], srcPos: Int, + dst: Array[Edge[ED]], dstPos: Int, length: Int) { + System.arraycopy(src, srcPos, dst, dstPos, length) + } + + override def allocate(length: Int): Array[Edge[ED]] = { + new Array[Edge[ED]](length) + } + } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala index b0cb0fe47d461..409cf60977f6f 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala @@ -18,12 +18,10 @@ package org.apache.spark.graphx.impl import scala.reflect.ClassTag -import scala.util.Sorting - -import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveVector} import org.apache.spark.graphx._ import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.util.collection.{SortDataFormat, Sorter, PrimitiveVector} /** Constructs an EdgePartition from scratch. */ private[graphx] @@ -38,7 +36,8 @@ class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: Cla def toEdgePartition: EdgePartition[ED, VD] = { val edgeArray = edges.trim().array - Sorting.quickSort(edgeArray)(Edge.lexicographicOrdering) + new Sorter(Edge.edgeArraySortDataFormat[ED]) + .sort(edgeArray, 0, edgeArray.length, Edge.lexicographicOrdering) val localSrcIds = new Array[Int](edgeArray.size) val localDstIds = new Array[Int](edgeArray.size) val data = new Array[ED](edgeArray.size) @@ -97,7 +96,8 @@ class ExistingEdgePartitionBuilder[ def toEdgePartition: EdgePartition[ED, VD] = { val edgeArray = edges.trim().array - Sorting.quickSort(edgeArray)(EdgeWithLocalIds.lexicographicOrdering) + new Sorter(EdgeWithLocalIds.edgeArraySortDataFormat[ED]) + .sort(edgeArray, 0, edgeArray.length, EdgeWithLocalIds.lexicographicOrdering) val localSrcIds = new Array[Int](edgeArray.size) val localDstIds = new Array[Int](edgeArray.size) val data = new Array[ED](edgeArray.size) @@ -140,4 +140,33 @@ private[impl] object EdgeWithLocalIds { } } + private[graphx] def edgeArraySortDataFormat[ED] + = new SortDataFormat[EdgeWithLocalIds[ED], Array[EdgeWithLocalIds[ED]]] { + override def getKey( + data: Array[EdgeWithLocalIds[ED]], pos: Int): EdgeWithLocalIds[ED] = { + data(pos) + } + + override def swap(data: Array[EdgeWithLocalIds[ED]], pos0: Int, pos1: Int): Unit = { + val tmp = data(pos0) + data(pos0) = data(pos1) + data(pos1) = tmp + } + + override def copyElement( + src: Array[EdgeWithLocalIds[ED]], srcPos: Int, + dst: Array[EdgeWithLocalIds[ED]], dstPos: Int) { + dst(dstPos) = src(srcPos) + } + + override def copyRange( + src: Array[EdgeWithLocalIds[ED]], srcPos: Int, + dst: Array[EdgeWithLocalIds[ED]], dstPos: Int, length: Int) { + System.arraycopy(src, srcPos, dst, dstPos, length) + } + + override def allocate(length: Int): Array[EdgeWithLocalIds[ED]] = { + new Array[EdgeWithLocalIds[ED]](length) + } + } } From 8817fc7fe8785d7b11138ca744f22f7e70f1f0a0 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Sun, 7 Dec 2014 19:42:02 -0800 Subject: [PATCH 006/155] [SPARK-4620] Add unpersist in Graph and GraphImpl Add an IF to uncache both vertices and edges of Graph/GraphImpl. This IF is useful when iterative graph operations build a new graph in each iteration, and the vertices and edges of previous iterations are no longer needed for following iterations. Author: Takeshi Yamamuro This patch had conflicts when merged, resolved by Committer: Ankur Dave Closes #3476 from maropu/UnpersistInGraphSpike and squashes the following commits: 77a006a [Takeshi Yamamuro] Add unpersist in Graph and GraphImpl --- graphx/src/main/scala/org/apache/spark/graphx/Graph.scala | 6 ++++++ .../main/scala/org/apache/spark/graphx/impl/GraphImpl.scala | 6 ++++++ 2 files changed, 12 insertions(+) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala index 23538b71562de..84b72b390ca35 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -104,6 +104,12 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab */ def checkpoint(): Unit + /** + * Uncaches both vertices and edges of this graph. This is useful in iterative algorithms that + * build a new graph in each iteration. + */ + def unpersist(blocking: Boolean = true): Graph[VD, ED] + /** * Uncaches only the vertices of this graph, leaving the edges alone. This is useful in iterative * algorithms that modify the vertex attributes but reuse the edges. This method can be used to diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala index a617d84aea9d4..3f4a900d5b601 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -70,6 +70,12 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( replicatedVertexView.edges.checkpoint() } + override def unpersist(blocking: Boolean = true): Graph[VD, ED] = { + unpersistVertices(blocking) + replicatedVertexView.edges.unpersist(blocking) + this + } + override def unpersistVertices(blocking: Boolean = true): Graph[VD, ED] = { vertices.unpersist(blocking) // TODO: unpersist the replicated vertices in `replicatedVertexView` but leave the edges alone From ab2abcb5ef925f15fa0e08d34a79b94a7b6578ef Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christophe=20Pr=C3=A9aud?= Date: Mon, 8 Dec 2014 11:44:54 -0800 Subject: [PATCH 007/155] [SPARK-4764] Ensure that files are fetched atomically MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit tempFile is created in the same directory than targetFile, so that the move from tempFile to targetFile is always atomic Author: Christophe Préaud Closes #2855 from preaudc/master and squashes the following commits: 9ba89ca [Christophe Préaud] Ensure that files are fetched atomically 54419ae [Christophe Préaud] Merge remote-tracking branch 'upstream/master' c6a5590 [Christophe Préaud] Revert commit 8ea871f8130b2490f1bad7374a819bf56f0ccbbd 7456a33 [Christophe Préaud] Merge remote-tracking branch 'upstream/master' 8ea871f [Christophe Préaud] Ensure that files are fetched atomically --- core/src/main/scala/org/apache/spark/util/Utils.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 336b0798cade9..9c04e45a58479 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -425,8 +425,7 @@ private[spark] object Utils extends Logging { conf: SparkConf, securityMgr: SecurityManager, hadoopConf: Configuration) { - val tempDir = getLocalDir(conf) - val tempFile = File.createTempFile("fetchFileTemp", null, new File(tempDir)) + val tempFile = File.createTempFile("fetchFileTemp", null, new File(targetDir.getAbsolutePath)) val targetFile = new File(targetDir, filename) val uri = new URI(url) val fileOverwrite = conf.getBoolean("spark.files.overwrite", defaultValue = false) From d6a972b3e4dc35a2d95df47d256462b325f4bda6 Mon Sep 17 00:00:00 2001 From: Kostas Sakellis Date: Mon, 8 Dec 2014 15:44:18 -0800 Subject: [PATCH 008/155] [SPARK-4774] [SQL] Makes HiveFromSpark more portable HiveFromSpark read the kv1.txt file from SPARK_HOME/examples/src/main/resources/kv1.txt which assumed you had a source tree checked out. Now we copy the kv1.txt file to a temporary file and delete it when the jvm shuts down. This allows us to run this example outside of a spark source tree. Author: Kostas Sakellis Closes #3628 from ksakellis/kostas-spark-4774 and squashes the following commits: 6770f83 [Kostas Sakellis] [SPARK-4774] [SQL] Makes HiveFromSpark more portable --- .../spark/examples/sql/hive/HiveFromSpark.scala | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index 138923c4d7f2f..5725da1848114 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -17,6 +17,10 @@ package org.apache.spark.examples.sql.hive +import com.google.common.io.{ByteStreams, Files} + +import java.io.File + import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.sql._ import org.apache.spark.sql.hive.HiveContext @@ -24,10 +28,15 @@ import org.apache.spark.sql.hive.HiveContext object HiveFromSpark { case class Record(key: Int, value: String) + // Copy kv1.txt file from classpath to temporary directory + val kv1Stream = HiveFromSpark.getClass.getResourceAsStream("/kv1.txt") + val kv1File = File.createTempFile("kv1", "txt") + kv1File.deleteOnExit() + ByteStreams.copy(kv1Stream, Files.newOutputStreamSupplier(kv1File)) + def main(args: Array[String]) { val sparkConf = new SparkConf().setAppName("HiveFromSpark") val sc = new SparkContext(sparkConf) - val path = s"${System.getenv("SPARK_HOME")}/examples/src/main/resources/kv1.txt" // A hive context adds support for finding tables in the MetaStore and writing queries // using HiveQL. Users who do not have an existing Hive deployment can still create a @@ -37,7 +46,7 @@ object HiveFromSpark { import hiveContext._ sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") - sql(s"LOAD DATA LOCAL INPATH '$path' INTO TABLE src") + sql(s"LOAD DATA LOCAL INPATH '${kv1File.getAbsolutePath}' INTO TABLE src") // Queries are expressed in HiveQL println("Result of 'SELECT *': ") From 65f929d5b3a50a73cd6397bd4b72c3e7d94c99d7 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 8 Dec 2014 16:02:33 -0800 Subject: [PATCH 009/155] [SPARK-4750] Dynamic allocation - synchronize kills Simple omission on my part. Author: Andrew Or Closes #3612 from andrewor14/dynamic-allocation-synchronization and squashes the following commits: 1f03b60 [Andrew Or] Synchronize kills --- .../spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 88b196ac64368..29cd34429b881 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -334,7 +334,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste * Request that the cluster manager kill the specified executors. * Return whether the kill request is acknowledged. */ - final def killExecutors(executorIds: Seq[String]): Boolean = { + final def killExecutors(executorIds: Seq[String]): Boolean = synchronized { logInfo(s"Requesting to kill executor(s) ${executorIds.mkString(", ")}") val filteredExecutorIds = new ArrayBuffer[String] executorIds.foreach { id => From e829bfa1ab9b68f44c489d26efb042f793fd9362 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 8 Dec 2014 16:13:03 -0800 Subject: [PATCH 010/155] SPARK-3926 [CORE] Reopened: result of JavaRDD collectAsMap() is not serializable My original 'fix' didn't fix at all. Now, there's a unit test to check whether it works. Of the two options to really fix it -- copy the `Map` to a `java.util.HashMap`, or copy and modify Scala's implementation in `Wrappers.MapWrapper`, I went with the latter. Author: Sean Owen Closes #3587 from srowen/SPARK-3926 and squashes the following commits: 8586bb9 [Sean Owen] Remove unneeded no-arg constructor, and add additional note about copied code in LICENSE 7bb0e66 [Sean Owen] Make SerializableMapWrapper actually serialize, and add unit test --- LICENSE | 3 +- .../org/apache/spark/api/java/JavaUtils.scala | 62 ++++++++++++++++++- .../java/org/apache/spark/JavaAPISuite.java | 13 ++++ 3 files changed, 75 insertions(+), 3 deletions(-) diff --git a/LICENSE b/LICENSE index 3c667bf45059a..0a42d389e4c3c 100644 --- a/LICENSE +++ b/LICENSE @@ -646,7 +646,8 @@ THE SOFTWARE. ======================================================================== For Scala Interpreter classes (all .scala files in repl/src/main/scala -except for Main.Scala, SparkHelper.scala and ExecutorClassLoader.scala): +except for Main.Scala, SparkHelper.scala and ExecutorClassLoader.scala), +and for SerializableMapWrapper in JavaUtils.scala: ======================================================================== Copyright (c) 2002-2013 EPFL diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala index b52d0a5028e84..86e94931300f8 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala @@ -19,7 +19,8 @@ package org.apache.spark.api.java import com.google.common.base.Optional -import scala.collection.convert.Wrappers.MapWrapper +import java.{util => ju} +import scala.collection.mutable private[spark] object JavaUtils { def optionToOptional[T](option: Option[T]): Optional[T] = @@ -32,7 +33,64 @@ private[spark] object JavaUtils { def mapAsSerializableJavaMap[A, B](underlying: collection.Map[A, B]) = new SerializableMapWrapper(underlying) + // Implementation is copied from scala.collection.convert.Wrappers.MapWrapper, + // but implements java.io.Serializable. It can't just be subclassed to make it + // Serializable since the MapWrapper class has no no-arg constructor. This class + // doesn't need a no-arg constructor though. class SerializableMapWrapper[A, B](underlying: collection.Map[A, B]) - extends MapWrapper(underlying) with java.io.Serializable + extends ju.AbstractMap[A, B] with java.io.Serializable { self => + override def size = underlying.size + + override def get(key: AnyRef): B = try { + underlying get key.asInstanceOf[A] match { + case None => null.asInstanceOf[B] + case Some(v) => v + } + } catch { + case ex: ClassCastException => null.asInstanceOf[B] + } + + override def entrySet: ju.Set[ju.Map.Entry[A, B]] = new ju.AbstractSet[ju.Map.Entry[A, B]] { + def size = self.size + + def iterator = new ju.Iterator[ju.Map.Entry[A, B]] { + val ui = underlying.iterator + var prev : Option[A] = None + + def hasNext = ui.hasNext + + def next() = { + val (k, v) = ui.next + prev = Some(k) + new ju.Map.Entry[A, B] { + import scala.util.hashing.byteswap32 + def getKey = k + def getValue = v + def setValue(v1 : B) = self.put(k, v1) + override def hashCode = byteswap32(k.hashCode) + (byteswap32(v.hashCode) << 16) + override def equals(other: Any) = other match { + case e: ju.Map.Entry[_, _] => k == e.getKey && v == e.getValue + case _ => false + } + } + } + + def remove() { + prev match { + case Some(k) => + underlying match { + case mm: mutable.Map[a, _] => + mm remove k + prev = None + case _ => + throw new UnsupportedOperationException("remove") + } + case _ => + throw new IllegalStateException("next must be called at least once before remove") + } + } + } + } + } } diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 3ad4f2f193af4..e5bdad6bda2fa 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -1357,6 +1357,19 @@ public Tuple2 call(Integer x) { pairRDD.collectAsMap(); // Used to crash with ClassCastException } + @SuppressWarnings("unchecked") + @Test + public void collectAsMapAndSerialize() throws Exception { + JavaPairRDD rdd = + sc.parallelizePairs(Arrays.asList(new Tuple2("foo", 1))); + Map map = rdd.collectAsMap(); + ByteArrayOutputStream bytes = new ByteArrayOutputStream(); + new ObjectOutputStream(bytes).writeObject(map); + Map deserializedMap = (Map) + new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray())).readObject(); + Assert.assertEquals(1, deserializedMap.get("foo").intValue()); + } + @Test @SuppressWarnings("unchecked") public void sampleByKey() { From cda94d15ea2a70ed3f0651ba2766b1e2f80308c1 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Mon, 8 Dec 2014 16:28:36 -0800 Subject: [PATCH 011/155] SPARK-4770. [DOC] [YARN] spark.scheduler.minRegisteredResourcesRatio doc... ...umented default is incorrect for YARN Author: Sandy Ryza Closes #3624 from sryza/sandy-spark-4770 and squashes the following commits: bd81a3a [Sandy Ryza] SPARK-4770. [DOC] [YARN] spark.scheduler.minRegisteredResourcesRatio documented default is incorrect for YARN --- docs/configuration.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 4b4bbea564d3a..d50b04642b6a2 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -939,11 +939,11 @@ Apart from these, the following properties are also available, and may be useful spark.scheduler.minRegisteredResourcesRatio - 0 + 0.0 for Mesos and Standalone mode, 0.8 for YARN The minimum ratio of registered resources (registered resources / total expected resources) (resources are executors in yarn mode, CPU cores in standalone mode) - to wait for before scheduling begins. Specified as a double between 0 and 1. + to wait for before scheduling begins. Specified as a double between 0.0 and 1.0. Regardless of whether the minimum ratio of resources has been reached, the maximum amount of time it will wait before scheduling begins is controlled by config spark.scheduler.maxRegisteredResourcesWaitingTime. From 944384363d390a133529c08a1d0ac70aa8e778b5 Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Mon, 8 Dec 2014 17:27:46 -0800 Subject: [PATCH 012/155] [SQL] remove unnecessary import in spark-sql Author: Jacky Li Closes #3630 from jackylk/remove and squashes the following commits: 150e7e0 [Jacky Li] remove unnecessary import --- sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala | 2 +- .../scala/org/apache/spark/sql/execution/ExistingRDD.scala | 4 +--- .../org/apache/spark/sql/execution/GeneratedAggregate.scala | 1 - .../main/scala/org/apache/spark/sql/execution/SparkPlan.scala | 3 --- .../org/apache/spark/sql/types/util/DataTypeConversions.scala | 2 +- 5 files changed, 3 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index c6d4dabf83bc4..95d73c1711523 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql import java.util.{Map => JMap, List => JList} -import java.io.StringWriter + import scala.collection.JavaConversions._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index ed6b95dc6d9d0..d2d8cb1c62d40 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -19,13 +19,11 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataType, StructType, Row, SQLContext} +import org.apache.spark.sql.{StructType, Row, SQLContext} import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.catalyst.ScalaReflection.Schema import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericMutableRow} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics} -import org.apache.spark.sql.catalyst.types.UserDefinedType /** * :: DeveloperApi :: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala index 18afc5d74137d..7c3bf947e743d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.trees._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 81c60e00505c5..4cd8e7d657250 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -22,14 +22,11 @@ import org.apache.spark.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.{ScalaReflection, trees} -import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical._ - object SparkPlan { protected[sql] val currentContext = new ThreadLocal[SQLContext]() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala index 4160a80621c77..d4ef517981699 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala @@ -23,7 +23,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql._ import org.apache.spark.sql.api.java.{DataType => JDataType, StructField => JStructField, - MetadataBuilder => JMetaDataBuilder, UDTWrappers, JavaToScalaUDTWrapper} + MetadataBuilder => JMetaDataBuilder, UDTWrappers} import org.apache.spark.sql.api.java.{DecimalType => JDecimalType} import org.apache.spark.sql.catalyst.types.decimal.Decimal import org.apache.spark.sql.catalyst.ScalaReflection From 51b1fe1426ffecac6c4644523633ea1562ff9a4e Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Mon, 8 Dec 2014 17:39:12 -0800 Subject: [PATCH 013/155] [SPARK-4769] [SQL] CTAS does not work when reading from temporary tables This is the code refactor and follow ups for #2570 Author: Cheng Hao Closes #3336 from chenghao-intel/createtbl and squashes the following commits: 3563142 [Cheng Hao] remove the unused variable e215187 [Cheng Hao] eliminate the compiling warning 4f97f14 [Cheng Hao] fix bug in unittest 5d58812 [Cheng Hao] revert the API changes b85b620 [Cheng Hao] fix the regression of temp tabl not found in CTAS --- .../spark/sql/hive/HiveMetastoreCatalog.scala | 26 +++++++++++++++++-- .../spark/sql/hive/HiveStrategies.scala | 14 +++++++--- .../hive/execution/CreateTableAsSelect.scala | 16 ++++-------- .../sql/hive/execution/SQLQuerySuite.scala | 9 +++++++ 4 files changed, 49 insertions(+), 16 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 91a157785d5bb..60865638e1073 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -254,15 +254,37 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with * For example, because of a CREATE TABLE X AS statement. */ object CreateTables extends Rule[LogicalPlan] { + import org.apache.hadoop.hive.ql.Context + import org.apache.hadoop.hive.ql.parse.{QB, ASTNode, SemanticAnalyzer} + def apply(plan: LogicalPlan): LogicalPlan = plan transform { // Wait until children are resolved. case p: LogicalPlan if !p.childrenResolved => p - case CreateTableAsSelect(db, tableName, child, allowExisting, extra) => + case CreateTableAsSelect(db, tableName, child, allowExisting, Some(extra: ASTNode)) => val (dbName, tblName) = processDatabaseAndTableName(db, tableName) val databaseName = dbName.getOrElse(hive.sessionState.getCurrentDatabase) - CreateTableAsSelect(Some(databaseName), tableName, child, allowExisting, extra) + // Get the CreateTableDesc from Hive SemanticAnalyzer + val desc: Option[CreateTableDesc] = if (tableExists(Some(databaseName), tblName)) { + None + } else { + val sa = new SemanticAnalyzer(hive.hiveconf) { + override def analyzeInternal(ast: ASTNode) { + // A hack to intercept the SemanticAnalyzer.analyzeInternal, + // to ignore the SELECT clause of the CTAS + val method = classOf[SemanticAnalyzer].getDeclaredMethod( + "analyzeCreateTable", classOf[ASTNode], classOf[QB]) + method.setAccessible(true) + method.invoke(this, ast, this.getQB) + } + } + + sa.analyze(extra, new Context(hive.hiveconf)) + Some(sa.getQB().getTableDesc) + } + + CreateTableAsSelect(Some(databaseName), tblName, child, allowExisting, desc) } } 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 edf291f917f07..5f02e95ac3c34 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,6 +18,7 @@ package org.apache.spark.sql.hive import org.apache.hadoop.hive.ql.parse.ASTNode +import org.apache.hadoop.hive.ql.plan.CreateTableDesc import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute @@ -181,13 +182,20 @@ private[hive] trait HiveStrategies { execution.InsertIntoHiveTable( table, partition, planLater(child), overwrite)(hiveContext) :: Nil case logical.CreateTableAsSelect( - Some(database), tableName, child, allowExisting, Some(extra: ASTNode)) => - CreateTableAsSelect( + Some(database), tableName, child, allowExisting, Some(desc: CreateTableDesc)) => + execution.CreateTableAsSelect( database, tableName, child, allowExisting, - extra) :: Nil + Some(desc)) :: Nil + case logical.CreateTableAsSelect(Some(database), tableName, child, allowExisting, None) => + execution.CreateTableAsSelect( + database, + tableName, + child, + allowExisting, + None) :: Nil case _ => Nil } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala index 3d24d87bc3d38..b83689ceabb84 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.hive.execution -import org.apache.hadoop.hive.ql.Context -import org.apache.hadoop.hive.ql.parse.{SemanticAnalyzer, ASTNode} +import org.apache.hadoop.hive.ql.plan.CreateTableDesc + import org.apache.spark.annotation.Experimental import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.Row @@ -35,8 +35,7 @@ import org.apache.spark.sql.hive.MetastoreRelation * @param query the query whose result will be insert into the new relation * @param allowExisting allow continue working if it's already exists, otherwise * raise exception - * @param extra the extra information for this Operator, it should be the - * ASTNode object for extracting the CreateTableDesc. + * @param desc the CreateTableDesc, which may contains serde, storage handler etc. */ @Experimental @@ -45,7 +44,7 @@ case class CreateTableAsSelect( tableName: String, query: LogicalPlan, allowExisting: Boolean, - extra: ASTNode) extends LeafNode with Command { + desc: Option[CreateTableDesc]) extends LeafNode with Command { def output = Seq.empty @@ -53,13 +52,8 @@ case class CreateTableAsSelect( // A lazy computing of the metastoreRelation private[this] lazy val metastoreRelation: MetastoreRelation = { - // Get the CreateTableDesc from Hive SemanticAnalyzer - val sa = new SemanticAnalyzer(sc.hiveconf) - - sa.analyze(extra, new Context(sc.hiveconf)) - val desc = sa.getQB().getTableDesc // Create Hive Table - sc.catalog.createTable(database, tableName, query.output, allowExisting, Some(desc)) + sc.catalog.createTable(database, tableName, query.output, allowExisting, desc) // Get the Metastore Relation sc.catalog.lookupRelation(Some(database), tableName, None) match { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index e9b1943ff8db7..b341eae512417 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -119,6 +119,15 @@ class SQLQuerySuite extends QueryTest { checkAnswer( sql("SELECT f1.f2.f3 FROM nested"), 1) + checkAnswer(sql("CREATE TABLE test_ctas_1234 AS SELECT * from nested"), + Seq.empty[Row]) + checkAnswer( + sql("SELECT * FROM test_ctas_1234"), + sql("SELECT * FROM nested").collect().toSeq) + + intercept[org.apache.hadoop.hive.ql.metadata.InvalidTableException] { + sql("CREATE TABLE test_ctas_12345 AS SELECT * from notexists").collect() + } } test("test CTAS") { From bcb5cdad614d4fce43725dfec3ce88172d2f8c11 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Mon, 8 Dec 2014 23:54:15 -0800 Subject: [PATCH 014/155] [SPARK-3154][STREAMING] Replace ConcurrentHashMap with mutable.HashMap and remove @volatile from 'stopped' Since `sequenceNumberToProcessor` and `stopped` are both protected by the lock `sequenceNumberToProcessor`, `ConcurrentHashMap` and `volatile` is unnecessary. So this PR updated them accordingly. Author: zsxwing Closes #3634 from zsxwing/SPARK-3154 and squashes the following commits: 0d087ac [zsxwing] Replace ConcurrentHashMap with mutable.HashMap and remove @volatile from 'stopped' --- .../flume/sink/SparkAvroCallbackHandler.scala | 23 ++++++++++--------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala index 3c656a381bd9b..4373be443e67d 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala @@ -16,10 +16,10 @@ */ package org.apache.spark.streaming.flume.sink -import java.util.concurrent.{CountDownLatch, ConcurrentHashMap, Executors} +import java.util.concurrent.{CountDownLatch, Executors} import java.util.concurrent.atomic.AtomicLong -import scala.collection.JavaConversions._ +import scala.collection.mutable import org.apache.flume.Channel import org.apache.commons.lang.RandomStringUtils @@ -47,8 +47,8 @@ private[flume] class SparkAvroCallbackHandler(val threads: Int, val channel: Cha val transactionExecutorOpt = Option(Executors.newFixedThreadPool(threads, new ThreadFactoryBuilder().setDaemon(true) .setNameFormat("Spark Sink Processor Thread - %d").build())) - private val sequenceNumberToProcessor = - new ConcurrentHashMap[CharSequence, TransactionProcessor]() + // Protected by `sequenceNumberToProcessor` + private val sequenceNumberToProcessor = mutable.HashMap[CharSequence, TransactionProcessor]() // This sink will not persist sequence numbers and reuses them if it gets restarted. // So it is possible to commit a transaction which may have been meant for the sink before the // restart. @@ -58,8 +58,8 @@ private[flume] class SparkAvroCallbackHandler(val threads: Int, val channel: Cha private val seqBase = RandomStringUtils.randomAlphanumeric(8) private val seqCounter = new AtomicLong(0) - - @volatile private var stopped = false + // Protected by `sequenceNumberToProcessor` + private var stopped = false @volatile private var isTest = false private var testLatch: CountDownLatch = null @@ -131,7 +131,7 @@ private[flume] class SparkAvroCallbackHandler(val threads: Int, val channel: Cha * @param success Whether the batch was successful or not. */ private def completeTransaction(sequenceNumber: CharSequence, success: Boolean) { - Option(removeAndGetProcessor(sequenceNumber)).foreach(processor => { + removeAndGetProcessor(sequenceNumber).foreach(processor => { processor.batchProcessed(success) }) } @@ -139,10 +139,11 @@ private[flume] class SparkAvroCallbackHandler(val threads: Int, val channel: Cha /** * Helper method to remove the TxnProcessor for a Sequence Number. Can be used to avoid a leak. * @param sequenceNumber - * @return The transaction processor for the corresponding batch. Note that this instance is no - * longer tracked and the caller is responsible for that txn processor. + * @return An `Option` of the transaction processor for the corresponding batch. Note that this + * instance is no longer tracked and the caller is responsible for that txn processor. */ - private[sink] def removeAndGetProcessor(sequenceNumber: CharSequence): TransactionProcessor = { + private[sink] def removeAndGetProcessor(sequenceNumber: CharSequence): + Option[TransactionProcessor] = { sequenceNumberToProcessor.synchronized { sequenceNumberToProcessor.remove(sequenceNumber.toString) } @@ -160,7 +161,7 @@ private[flume] class SparkAvroCallbackHandler(val threads: Int, val channel: Cha logInfo("Shutting down Spark Avro Callback Handler") sequenceNumberToProcessor.synchronized { stopped = true - sequenceNumberToProcessor.values().foreach(_.shutdown()) + sequenceNumberToProcessor.values.foreach(_.shutdown()) } transactionExecutorOpt.foreach(_.shutdownNow()) } From 383c5555c9f26c080bc9e3a463aab21dd5b3797f Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 9 Dec 2014 10:28:15 -0800 Subject: [PATCH 015/155] [SPARK-4785][SQL] Initilize Hive UDFs on the driver and serialize them with a wrapper Different from Hive 0.12.0, in Hive 0.13.1 UDF/UDAF/UDTF (aka Hive function) objects should only be initialized once on the driver side and then serialized to executors. However, not all function objects are serializable (e.g. GenericUDF doesn't implement Serializable). Hive 0.13.1 solves this issue with Kryo or XML serializer. Several utility ser/de methods are provided in class o.a.h.h.q.e.Utilities for this purpose. In this PR we chose Kryo for efficiency. The Kryo serializer used here is created in Hive. Spark Kryo serializer wasn't used because there's no available SparkConf instance. Author: Cheng Hao Author: Cheng Lian Closes #3640 from chenghao-intel/udf_serde and squashes the following commits: 8e13756 [Cheng Hao] Update the comment 74466a3 [Cheng Hao] refactor as feedbacks 396c0e1 [Cheng Hao] avoid Simple UDF to be serialized e9c3212 [Cheng Hao] update the comment 19cbd46 [Cheng Hao] support udf instance ser/de after initialization --- .../org/apache/spark/sql/hive/HiveQl.scala | 5 +- .../org/apache/spark/sql/hive/hiveUdfs.scala | 93 +++++++-------- .../sql/hive/execution/HiveUdfSuite.scala | 7 ++ .../org/apache/spark/sql/hive/Shim12.scala | 11 ++ .../org/apache/spark/sql/hive/Shim13.scala | 107 ++++++++++++++++++ 5 files changed, 173 insertions(+), 50 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 f4c42bbc5b03d..cd4e5a239ec66 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 @@ -1128,7 +1128,10 @@ private[hive] object HiveQl { Explode(attributes, nodeToExpr(child)) case Token("TOK_FUNCTION", Token(functionName, Nil) :: children) => - HiveGenericUdtf(functionName, attributes, children.map(nodeToExpr)) + HiveGenericUdtf( + new HiveFunctionWrapper(functionName), + attributes, + children.map(nodeToExpr)) case a: ASTNode => throw new NotImplementedError( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index fecf8faaf4cda..ed2e96df8ad77 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -54,46 +54,31 @@ private[hive] abstract class HiveFunctionRegistry val functionClassName = functionInfo.getFunctionClass.getName if (classOf[UDF].isAssignableFrom(functionInfo.getFunctionClass)) { - HiveSimpleUdf(functionClassName, children) + HiveSimpleUdf(new HiveFunctionWrapper(functionClassName), children) } else if (classOf[GenericUDF].isAssignableFrom(functionInfo.getFunctionClass)) { - HiveGenericUdf(functionClassName, children) + HiveGenericUdf(new HiveFunctionWrapper(functionClassName), children) } else if ( classOf[AbstractGenericUDAFResolver].isAssignableFrom(functionInfo.getFunctionClass)) { - HiveGenericUdaf(functionClassName, children) + HiveGenericUdaf(new HiveFunctionWrapper(functionClassName), children) } else if (classOf[UDAF].isAssignableFrom(functionInfo.getFunctionClass)) { - HiveUdaf(functionClassName, children) + HiveUdaf(new HiveFunctionWrapper(functionClassName), children) } else if (classOf[GenericUDTF].isAssignableFrom(functionInfo.getFunctionClass)) { - HiveGenericUdtf(functionClassName, Nil, children) + HiveGenericUdtf(new HiveFunctionWrapper(functionClassName), Nil, children) } else { sys.error(s"No handler for udf ${functionInfo.getFunctionClass}") } } } -private[hive] trait HiveFunctionFactory { - val functionClassName: String - - def createFunction[UDFType]() = - getContextOrSparkClassLoader.loadClass(functionClassName).newInstance.asInstanceOf[UDFType] -} - -private[hive] abstract class HiveUdf extends Expression with Logging with HiveFunctionFactory { - self: Product => - - type UDFType +private[hive] case class HiveSimpleUdf(funcWrapper: HiveFunctionWrapper, children: Seq[Expression]) + extends Expression with HiveInspectors with Logging { type EvaluatedType = Any + type UDFType = UDF def nullable = true - lazy val function = createFunction[UDFType]() - - override def toString = s"$nodeName#$functionClassName(${children.mkString(",")})" -} - -private[hive] case class HiveSimpleUdf(functionClassName: String, children: Seq[Expression]) - extends HiveUdf with HiveInspectors { - - type UDFType = UDF + @transient + lazy val function = funcWrapper.createFunction[UDFType]() @transient protected lazy val method = @@ -131,6 +116,8 @@ private[hive] case class HiveSimpleUdf(functionClassName: String, children: Seq[ .convertIfNecessary(wrap(children.map(c => c.eval(input)), arguments, cached): _*): _*), returnInspector) } + + override def toString = s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" } // Adapter from Catalyst ExpressionResult to Hive DeferredObject @@ -144,16 +131,23 @@ private[hive] class DeferredObjectAdapter(oi: ObjectInspector) override def get(): AnyRef = wrap(func(), oi) } -private[hive] case class HiveGenericUdf(functionClassName: String, children: Seq[Expression]) - extends HiveUdf with HiveInspectors { +private[hive] case class HiveGenericUdf(funcWrapper: HiveFunctionWrapper, children: Seq[Expression]) + extends Expression with HiveInspectors with Logging { type UDFType = GenericUDF + type EvaluatedType = Any + + def nullable = true + + @transient + lazy val function = funcWrapper.createFunction[UDFType]() @transient protected lazy val argumentInspectors = children.map(toInspector) @transient - protected lazy val returnInspector = + protected lazy val returnInspector = { function.initializeAndFoldConstants(argumentInspectors.toArray) + } @transient protected lazy val isUDFDeterministic = { @@ -183,18 +177,19 @@ private[hive] case class HiveGenericUdf(functionClassName: String, children: Seq } unwrap(function.evaluate(deferedObjects), returnInspector) } + + override def toString = s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" } private[hive] case class HiveGenericUdaf( - functionClassName: String, + funcWrapper: HiveFunctionWrapper, children: Seq[Expression]) extends AggregateExpression - with HiveInspectors - with HiveFunctionFactory { + with HiveInspectors { type UDFType = AbstractGenericUDAFResolver @transient - protected lazy val resolver: AbstractGenericUDAFResolver = createFunction() + protected lazy val resolver: AbstractGenericUDAFResolver = funcWrapper.createFunction() @transient protected lazy val objectInspector = { @@ -209,22 +204,22 @@ private[hive] case class HiveGenericUdaf( def nullable: Boolean = true - override def toString = s"$nodeName#$functionClassName(${children.mkString(",")})" + override def toString = s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" - def newInstance() = new HiveUdafFunction(functionClassName, children, this) + def newInstance() = new HiveUdafFunction(funcWrapper, children, this) } /** It is used as a wrapper for the hive functions which uses UDAF interface */ private[hive] case class HiveUdaf( - functionClassName: String, + funcWrapper: HiveFunctionWrapper, children: Seq[Expression]) extends AggregateExpression - with HiveInspectors - with HiveFunctionFactory { + with HiveInspectors { type UDFType = UDAF @transient - protected lazy val resolver: AbstractGenericUDAFResolver = new GenericUDAFBridge(createFunction()) + protected lazy val resolver: AbstractGenericUDAFResolver = + new GenericUDAFBridge(funcWrapper.createFunction()) @transient protected lazy val objectInspector = { @@ -239,10 +234,10 @@ private[hive] case class HiveUdaf( def nullable: Boolean = true - override def toString = s"$nodeName#$functionClassName(${children.mkString(",")})" + override def toString = s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" def newInstance() = - new HiveUdafFunction(functionClassName, children, this, true) + new HiveUdafFunction(funcWrapper, children, this, true) } /** @@ -257,13 +252,13 @@ private[hive] case class HiveUdaf( * user defined aggregations, which have clean semantics even in a partitioned execution. */ private[hive] case class HiveGenericUdtf( - functionClassName: String, + funcWrapper: HiveFunctionWrapper, aliasNames: Seq[String], children: Seq[Expression]) - extends Generator with HiveInspectors with HiveFunctionFactory { + extends Generator with HiveInspectors { @transient - protected lazy val function: GenericUDTF = createFunction() + protected lazy val function: GenericUDTF = funcWrapper.createFunction() @transient protected lazy val inputInspectors = children.map(_.dataType).map(toInspector) @@ -320,25 +315,24 @@ private[hive] case class HiveGenericUdtf( } } - override def toString = s"$nodeName#$functionClassName(${children.mkString(",")})" + override def toString = s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" } private[hive] case class HiveUdafFunction( - functionClassName: String, + funcWrapper: HiveFunctionWrapper, exprs: Seq[Expression], base: AggregateExpression, isUDAFBridgeRequired: Boolean = false) extends AggregateFunction - with HiveInspectors - with HiveFunctionFactory { + with HiveInspectors { def this() = this(null, null, null) private val resolver = if (isUDAFBridgeRequired) { - new GenericUDAFBridge(createFunction[UDAF]()) + new GenericUDAFBridge(funcWrapper.createFunction[UDAF]()) } else { - createFunction[AbstractGenericUDAFResolver]() + funcWrapper.createFunction[AbstractGenericUDAFResolver]() } private val inspectors = exprs.map(_.dataType).map(toInspector).toArray @@ -361,3 +355,4 @@ private[hive] case class HiveUdafFunction( function.iterate(buffer, inputs) } } + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala index 872f28d514efe..5fcaf671a80de 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala @@ -60,6 +60,13 @@ class HiveUdfSuite extends QueryTest { | getStruct(1).f5 FROM src LIMIT 1 """.stripMargin).first() === Row(1, 2, 3, 4, 5)) } + + test("SPARK-4785 When called with arguments referring column fields, PMOD throws NPE") { + checkAnswer( + sql("SELECT PMOD(CAST(key as INT), 10) FROM src LIMIT 1"), + 8 + ) + } test("hive struct udf") { sql( diff --git a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala index 76f09cbcdec99..754ffc422072d 100644 --- a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala +++ b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala @@ -43,6 +43,17 @@ import scala.language.implicitConversions import org.apache.spark.sql.catalyst.types.DecimalType +class HiveFunctionWrapper(var functionClassName: String) extends java.io.Serializable { + // for Serialization + def this() = this(null) + + import org.apache.spark.util.Utils._ + def createFunction[UDFType <: AnyRef](): UDFType = { + getContextOrSparkClassLoader + .loadClass(functionClassName).newInstance.asInstanceOf[UDFType] + } +} + /** * A compatibility layer for interacting with Hive version 0.12.0. */ diff --git a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala index 91f7ceac21177..7c8cbf10c1c30 100644 --- a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala +++ b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive import java.util.{ArrayList => JArrayList} import java.util.Properties + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.mapred.InputFormat @@ -42,6 +43,112 @@ import org.apache.spark.sql.catalyst.types.decimal.Decimal import scala.collection.JavaConversions._ import scala.language.implicitConversions + +/** + * This class provides the UDF creation and also the UDF instance serialization and + * de-serialization cross process boundary. + * + * Detail discussion can be found at https://github.com/apache/spark/pull/3640 + * + * @param functionClassName UDF class name + */ +class HiveFunctionWrapper(var functionClassName: String) extends java.io.Externalizable { + // for Serialization + def this() = this(null) + + import java.io.{OutputStream, InputStream} + import com.esotericsoftware.kryo.Kryo + import org.apache.spark.util.Utils._ + import org.apache.hadoop.hive.ql.exec.Utilities + import org.apache.hadoop.hive.ql.exec.UDF + + @transient + private val methodDeSerialize = { + val method = classOf[Utilities].getDeclaredMethod( + "deserializeObjectByKryo", + classOf[Kryo], + classOf[InputStream], + classOf[Class[_]]) + method.setAccessible(true) + + method + } + + @transient + private val methodSerialize = { + val method = classOf[Utilities].getDeclaredMethod( + "serializeObjectByKryo", + classOf[Kryo], + classOf[Object], + classOf[OutputStream]) + method.setAccessible(true) + + method + } + + def deserializePlan[UDFType](is: java.io.InputStream, clazz: Class[_]): UDFType = { + methodDeSerialize.invoke(null, Utilities.runtimeSerializationKryo.get(), is, clazz) + .asInstanceOf[UDFType] + } + + def serializePlan(function: AnyRef, out: java.io.OutputStream): Unit = { + methodSerialize.invoke(null, Utilities.runtimeSerializationKryo.get(), function, out) + } + + private var instance: AnyRef = null + + def writeExternal(out: java.io.ObjectOutput) { + // output the function name + out.writeUTF(functionClassName) + + // Write a flag if instance is null or not + out.writeBoolean(instance != null) + if (instance != null) { + // Some of the UDF are serializable, but some others are not + // Hive Utilities can handle both cases + val baos = new java.io.ByteArrayOutputStream() + serializePlan(instance, baos) + val functionInBytes = baos.toByteArray + + // output the function bytes + out.writeInt(functionInBytes.length) + out.write(functionInBytes, 0, functionInBytes.length) + } + } + + def readExternal(in: java.io.ObjectInput) { + // read the function name + functionClassName = in.readUTF() + + if (in.readBoolean()) { + // if the instance is not null + // read the function in bytes + val functionInBytesLength = in.readInt() + val functionInBytes = new Array[Byte](functionInBytesLength) + in.read(functionInBytes, 0, functionInBytesLength) + + // deserialize the function object via Hive Utilities + instance = deserializePlan[AnyRef](new java.io.ByteArrayInputStream(functionInBytes), + getContextOrSparkClassLoader.loadClass(functionClassName)) + } + } + + def createFunction[UDFType <: AnyRef](): UDFType = { + if (instance != null) { + instance.asInstanceOf[UDFType] + } else { + val func = getContextOrSparkClassLoader + .loadClass(functionClassName).newInstance.asInstanceOf[UDFType] + if (!func.isInstanceOf[UDF]) { + // We cache the function if it's no the Simple UDF, + // as we always have to create new instance for Simple UDF + instance = func + } + func + } + } +} + /** * A compatibility layer for interacting with Hive version 0.13.1. */ From 912563aa3553afc0871d5b5858f533aa39cb99e5 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Tue, 9 Dec 2014 11:02:43 -0800 Subject: [PATCH 016/155] SPARK-4338. [YARN] Ditch yarn-alpha. Sorry if this is a little premature with 1.2 still not out the door, but it will make other work like SPARK-4136 and SPARK-2089 a lot easier. Author: Sandy Ryza Closes #3215 from sryza/sandy-spark-4338 and squashes the following commits: 1c5ac08 [Sandy Ryza] Update building Spark docs and remove unnecessary newline 9c1421c [Sandy Ryza] SPARK-4338. Ditch yarn-alpha. --- assembly/pom.xml | 10 - dev/scalastyle | 5 +- docs/building-spark.md | 25 +- docs/running-on-yarn.md | 2 +- pom.xml | 7 - project/SparkBuild.scala | 20 +- yarn/README.md | 12 - yarn/alpha/pom.xml | 35 --- .../org/apache/spark/deploy/yarn/Client.scala | 145 ----------- .../spark/deploy/yarn/ExecutorRunnable.scala | 139 ----------- .../deploy/yarn/YarnAllocationHandler.scala | 229 ------------------ .../spark/deploy/yarn/YarnRMClientImpl.scala | 118 --------- yarn/pom.xml | 129 +++++----- .../spark/deploy/yarn/ApplicationMaster.scala | 0 .../yarn/ApplicationMasterArguments.scala | 0 .../org/apache/spark/deploy/yarn/Client.scala | 0 .../spark/deploy/yarn/ClientArguments.scala | 36 +-- .../apache/spark/deploy/yarn/ClientBase.scala | 0 .../yarn/ClientDistributedCacheManager.scala | 0 .../spark/deploy/yarn/ExecutorRunnable.scala | 0 .../deploy/yarn/ExecutorRunnableUtil.scala | 7 +- .../deploy/yarn/YarnAllocationHandler.scala | 0 .../spark/deploy/yarn/YarnAllocator.scala | 0 .../spark/deploy/yarn/YarnRMClient.scala | 0 .../spark/deploy/yarn/YarnRMClientImpl.scala | 0 .../deploy/yarn/YarnSparkHadoopUtil.scala | 10 +- .../cluster/YarnClientClusterScheduler.scala | 0 .../cluster/YarnClientSchedulerBackend.scala | 0 .../cluster/YarnClusterScheduler.scala | 0 .../cluster/YarnClusterSchedulerBackend.scala | 0 .../src/test/resources/log4j.properties | 0 .../spark/deploy/yarn/ClientBaseSuite.scala | 0 .../ClientDistributedCacheManagerSuite.scala | 0 .../deploy/yarn/YarnAllocatorSuite.scala | 0 .../spark/deploy/yarn/YarnClusterSuite.scala | 0 .../yarn/YarnSparkHadoopUtilSuite.scala | 0 yarn/stable/pom.xml | 95 -------- 37 files changed, 96 insertions(+), 928 deletions(-) delete mode 100644 yarn/README.md delete mode 100644 yarn/alpha/pom.xml delete mode 100644 yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala delete mode 100644 yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala delete mode 100644 yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala delete mode 100644 yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala rename yarn/{common => }/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala (100%) rename yarn/{common => }/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala (100%) rename yarn/{stable => }/src/main/scala/org/apache/spark/deploy/yarn/Client.scala (100%) rename yarn/{common => }/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala (82%) rename yarn/{common => }/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala (100%) rename yarn/{common => }/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala (100%) rename yarn/{stable => }/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala (100%) rename yarn/{common => }/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala (97%) rename yarn/{stable => }/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala (100%) rename yarn/{common => }/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala (100%) rename yarn/{common => }/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala (100%) rename yarn/{stable => }/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala (100%) rename yarn/{common => }/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala (96%) rename yarn/{common => }/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala (100%) rename yarn/{common => }/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala (100%) rename yarn/{common => }/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala (100%) rename yarn/{common => }/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala (100%) rename yarn/{stable => }/src/test/resources/log4j.properties (100%) rename yarn/{common => }/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala (100%) rename yarn/{common => }/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala (100%) rename yarn/{common => }/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala (100%) rename yarn/{stable => }/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala (100%) rename yarn/{common => }/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala (100%) delete mode 100644 yarn/stable/pom.xml diff --git a/assembly/pom.xml b/assembly/pom.xml index 4e2b773e7d2f3..78fb908f9a9ef 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -169,16 +169,6 @@ - - yarn-alpha - - - org.apache.spark - spark-yarn-alpha_${scala.binary.version} - ${project.version} - - - yarn diff --git a/dev/scalastyle b/dev/scalastyle index c3c6012e74ffa..3a4df6e4bf1bc 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -18,11 +18,8 @@ # echo -e "q\n" | sbt/sbt -Phive -Phive-thriftserver scalastyle > scalastyle.txt -# Check style with YARN alpha built too -echo -e "q\n" | sbt/sbt -Pyarn-alpha -Phadoop-0.23 -Dhadoop.version=0.23.9 yarn-alpha/scalastyle \ - >> scalastyle.txt # Check style with YARN built too -echo -e "q\n" | sbt/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 yarn/scalastyle \ +echo -e "q\n" | sbt/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 scalastyle \ >> scalastyle.txt ERRORS=$(cat scalastyle.txt | awk '{if($1~/error/)print}') diff --git a/docs/building-spark.md b/docs/building-spark.md index 6cca2da8e86d2..4922e877e9b78 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -60,32 +60,11 @@ mvn -Dhadoop.version=2.0.0-mr1-cdh4.2.0 -DskipTests clean package mvn -Phadoop-0.23 -Dhadoop.version=0.23.7 -DskipTests clean package {% endhighlight %} -For Apache Hadoop 2.x, 0.23.x, Cloudera CDH, and other Hadoop versions with YARN, you can enable the "yarn-alpha" or "yarn" profile and optionally set the "yarn.version" property if it is different from "hadoop.version". The additional build profile required depends on the YARN version: - - - - - - - - - -
YARN versionProfile required
0.23.x to 2.1.xyarn-alpha (Deprecated.)
2.2.x and lateryarn
- -Note: Support for YARN-alpha API's will be removed in Spark 1.3 (see SPARK-3445). +For Apache Hadoop 2.x, 0.23.x, Cloudera CDH, and other Hadoop versions with YARN, you can enable the "yarn" profile and optionally set the "yarn.version" property if it is different from "hadoop.version". As of Spark 1.3, Spark only supports YARN versions 2.2.0 and later. Examples: {% highlight bash %} -# Apache Hadoop 2.0.5-alpha -mvn -Pyarn-alpha -Dhadoop.version=2.0.5-alpha -DskipTests clean package - -# Cloudera CDH 4.2.0 -mvn -Pyarn-alpha -Dhadoop.version=2.0.0-cdh4.2.0 -DskipTests clean package - -# Apache Hadoop 0.23.x -mvn -Pyarn-alpha -Phadoop-0.23 -Dhadoop.version=0.23.7 -DskipTests clean package - # Apache Hadoop 2.2.X mvn -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -DskipTests clean package @@ -99,7 +78,7 @@ Versions of Hadoop after 2.5.X may or may not work with the -Phadoop-2.4 profile released after this version of Spark). # Different versions of HDFS and YARN. -mvn -Pyarn-alpha -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=0.23.7 -DskipTests clean package +mvn -Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=2.2.0 -DskipTests clean package {% endhighlight %} # Building With Hive and JDBC Support diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 16897dbb65311..62b317129b72a 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -216,7 +216,7 @@ If you need a reference to the proper location to put log files in the YARN so t # Important notes -- Before Hadoop 2.2, YARN does not support cores in container resource requests. Thus, when running against an earlier version, the numbers of cores given via command line arguments cannot be passed to YARN. Whether core requests are honored in scheduling decisions depends on which scheduler is in use and how it is configured. +- Whether core requests are honored in scheduling decisions depends on which scheduler is in use and how it is configured. - The local directories used by Spark executors will be the local directories configured for YARN (Hadoop YARN config `yarn.nodemanager.local-dirs`). If the user specifies `spark.local.dir`, it will be ignored. - The `--files` and `--archives` options support specifying file names with the # similar to Hadoop. For example you can specify: `--files localtest.txt#appSees.txt` and this will upload the file you have locally named localtest.txt into HDFS but this will be linked to by the name `appSees.txt`, and your application should use the name as `appSees.txt` to reference it when running on YARN. - The `--jars` option allows the `SparkContext.addJar` function to work if you are using it with local files and running in `yarn-cluster` mode. It does not need to be used if you are using it with HDFS, HTTP, HTTPS, or FTP files. diff --git a/pom.xml b/pom.xml index b7df53d3e5eb1..f42257265ede3 100644 --- a/pom.xml +++ b/pom.xml @@ -1293,13 +1293,6 @@
- - yarn-alpha - - yarn - - - yarn diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 6ff08723772aa..39ac27f820d89 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -38,9 +38,9 @@ object BuildCommons { "streaming-flume", "streaming-kafka", "streaming-mqtt", "streaming-twitter", "streaming-zeromq").map(ProjectRef(buildLocation, _)) - val optionallyEnabledProjects@Seq(yarn, yarnStable, yarnAlpha, java8Tests, - sparkGangliaLgpl, sparkKinesisAsl) = Seq("yarn", "yarn-stable", "yarn-alpha", - "java8-tests", "ganglia-lgpl", "kinesis-asl").map(ProjectRef(buildLocation, _)) + val optionallyEnabledProjects@Seq(yarn, yarnStable, java8Tests, sparkGangliaLgpl, + sparkKinesisAsl) = Seq("yarn", "yarn-stable", "java8-tests", "ganglia-lgpl", + "kinesis-asl").map(ProjectRef(buildLocation, _)) val assemblyProjects@Seq(assembly, examples, networkYarn) = Seq("assembly", "examples", "network-yarn").map(ProjectRef(buildLocation, _)) @@ -79,14 +79,8 @@ object SparkBuild extends PomBuild { case None => } if (Properties.envOrNone("SPARK_YARN").isDefined) { - if(isAlphaYarn) { - println("NOTE: SPARK_YARN is deprecated, please use -Pyarn-alpha flag.") - profiles ++= Seq("yarn-alpha") - } - else { - println("NOTE: SPARK_YARN is deprecated, please use -Pyarn flag.") - profiles ++= Seq("yarn") - } + println("NOTE: SPARK_YARN is deprecated, please use -Pyarn flag.") + profiles ++= Seq("yarn") } profiles } @@ -335,9 +329,9 @@ object Unidoc { publish := {}, unidocProjectFilter in(ScalaUnidoc, unidoc) := - inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, catalyst, streamingFlumeSink, yarn, yarnAlpha), + inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, catalyst, streamingFlumeSink, yarn), unidocProjectFilter in(JavaUnidoc, unidoc) := - inAnyProject -- inProjects(OldDeps.project, repl, bagel, examples, tools, catalyst, streamingFlumeSink, yarn, yarnAlpha), + inAnyProject -- inProjects(OldDeps.project, repl, bagel, examples, tools, catalyst, streamingFlumeSink, yarn), // Skip class names containing $ and some internal packages in Javadocs unidocAllSources in (JavaUnidoc, unidoc) := { diff --git a/yarn/README.md b/yarn/README.md deleted file mode 100644 index 65ee85447e04a..0000000000000 --- a/yarn/README.md +++ /dev/null @@ -1,12 +0,0 @@ -# YARN DIRECTORY LAYOUT - -Hadoop Yarn related codes are organized in separate directories to minimize duplicated code. - - * common : Common codes that do not depending on specific version of Hadoop. - - * alpha / stable : Codes that involve specific version of Hadoop YARN API. - - alpha represents 0.23 and 2.0.x - stable represents 2.2 and later, until the API changes again. - -alpha / stable will build together with common dir into a single jar diff --git a/yarn/alpha/pom.xml b/yarn/alpha/pom.xml deleted file mode 100644 index 40e9e99c6f855..0000000000000 --- a/yarn/alpha/pom.xml +++ /dev/null @@ -1,35 +0,0 @@ - - - - 4.0.0 - - org.apache.spark - yarn-parent_2.10 - 1.3.0-SNAPSHOT - ../pom.xml - - - yarn-alpha - - - org.apache.spark - spark-yarn-alpha_2.10 - jar - Spark Project YARN Alpha API - - diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala deleted file mode 100644 index 73b705ba50051..0000000000000 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ /dev/null @@ -1,145 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn - -import java.nio.ByteBuffer - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.io.DataOutputBuffer -import org.apache.hadoop.security.UserGroupInformation -import org.apache.hadoop.yarn.api._ -import org.apache.hadoop.yarn.api.protocolrecords._ -import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.client.YarnClientImpl -import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.util.Records - -import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.deploy.SparkHadoopUtil - -/** - * Version of [[org.apache.spark.deploy.yarn.ClientBase]] tailored to YARN's alpha API. - */ -@deprecated("use yarn/stable", "1.2.0") -private[spark] class Client( - val args: ClientArguments, - val hadoopConf: Configuration, - val sparkConf: SparkConf) - extends YarnClientImpl with ClientBase with Logging { - - def this(clientArgs: ClientArguments, spConf: SparkConf) = - this(clientArgs, SparkHadoopUtil.get.newConfiguration(spConf), spConf) - - def this(clientArgs: ClientArguments) = this(clientArgs, new SparkConf()) - - val yarnConf: YarnConfiguration = new YarnConfiguration(hadoopConf) - - /* ------------------------------------------------------------------------------------- * - | The following methods have much in common in the stable and alpha versions of Client, | - | but cannot be implemented in the parent trait due to subtle API differences across | - | hadoop versions. | - * ------------------------------------------------------------------------------------- */ - - /** Submit an application running our ApplicationMaster to the ResourceManager. */ - override def submitApplication(): ApplicationId = { - init(yarnConf) - start() - - logInfo("Requesting a new application from cluster with %d NodeManagers" - .format(getYarnClusterMetrics.getNumNodeManagers)) - - // Get a new application from our RM - val newAppResponse = getNewApplication() - val appId = newAppResponse.getApplicationId() - - // Verify whether the cluster has enough resources for our AM - verifyClusterResources(newAppResponse) - - // Set up the appropriate contexts to launch our AM - val containerContext = createContainerLaunchContext(newAppResponse) - val appContext = createApplicationSubmissionContext(appId, containerContext) - - // Finally, submit and monitor the application - logInfo(s"Submitting application ${appId.getId} to ResourceManager") - submitApplication(appContext) - appId - } - - /** - * Set up a context for launching our ApplicationMaster container. - * In the Yarn alpha API, the memory requirements of this container must be set in - * the ContainerLaunchContext instead of the ApplicationSubmissionContext. - */ - override def createContainerLaunchContext(newAppResponse: GetNewApplicationResponse) - : ContainerLaunchContext = { - val containerContext = super.createContainerLaunchContext(newAppResponse) - val capability = Records.newRecord(classOf[Resource]) - capability.setMemory(args.amMemory + amMemoryOverhead) - containerContext.setResource(capability) - containerContext - } - - /** Set up the context for submitting our ApplicationMaster. */ - def createApplicationSubmissionContext( - appId: ApplicationId, - containerContext: ContainerLaunchContext): ApplicationSubmissionContext = { - val appContext = Records.newRecord(classOf[ApplicationSubmissionContext]) - appContext.setApplicationId(appId) - appContext.setApplicationName(args.appName) - appContext.setQueue(args.amQueue) - appContext.setAMContainerSpec(containerContext) - appContext.setUser(UserGroupInformation.getCurrentUser.getShortUserName) - appContext - } - - /** - * Set up security tokens for launching our ApplicationMaster container. - * ContainerLaunchContext#setContainerTokens is renamed `setTokens` in the stable API. - */ - override def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = { - val dob = new DataOutputBuffer() - credentials.writeTokenStorageToStream(dob) - amContainer.setContainerTokens(ByteBuffer.wrap(dob.getData())) - } - - /** - * Return the security token used by this client to communicate with the ApplicationMaster. - * If no security is enabled, the token returned by the report is null. - * ApplicationReport#getClientToken is renamed `getClientToAMToken` in the stable API. - */ - override def getClientToken(report: ApplicationReport): String = - Option(report.getClientToken).map(_.toString).getOrElse("") -} - -object Client { - def main(argStrings: Array[String]) { - if (!sys.props.contains("SPARK_SUBMIT")) { - println("WARNING: This client is deprecated and will be removed in a " + - "future version of Spark. Use ./bin/spark-submit with \"--master yarn\"") - } - println("WARNING: Support for YARN-alpha API's will be removed in Spark 1.3 (see SPARK-3445)") - - // Set an env variable indicating we are running in YARN mode. - // Note that any env variable with the SPARK_ prefix gets propagated to all (remote) processes - System.setProperty("SPARK_YARN_MODE", "true") - val sparkConf = new SparkConf - - val args = new ClientArguments(argStrings, sparkConf) - new Client(args, sparkConf).run() - } -} 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 deleted file mode 100644 index 7023a1170654f..0000000000000 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ /dev/null @@ -1,139 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn - -import java.net.URI -import java.nio.ByteBuffer -import java.security.PrivilegedExceptionAction - -import scala.collection.JavaConversions._ -import scala.collection.mutable.HashMap - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.io.DataOutputBuffer -import org.apache.hadoop.net.NetUtils -import org.apache.hadoop.security.UserGroupInformation -import org.apache.hadoop.yarn.api._ -import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.api.protocolrecords._ -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.{SecurityManager, SparkConf, Logging} -import org.apache.spark.network.util.JavaUtils - -@deprecated("use yarn/stable", "1.2.0") -class ExecutorRunnable( - container: Container, - conf: Configuration, - spConf: SparkConf, - masterAddress: String, - slaveId: String, - hostname: String, - executorMemory: Int, - executorCores: Int, - appAttemptId: String, - securityMgr: SecurityManager) - extends Runnable with ExecutorRunnableUtil with Logging { - - var rpc: YarnRPC = YarnRPC.create(conf) - var cm: ContainerManager = _ - val sparkConf = spConf - val yarnConf: YarnConfiguration = new YarnConfiguration(conf) - - def run = { - logInfo("Starting Executor Container") - cm = connectToCM - startContainer - } - - def startContainer = { - logInfo("Setting up ContainerLaunchContext") - - val ctx = Records.newRecord(classOf[ContainerLaunchContext]) - .asInstanceOf[ContainerLaunchContext] - - ctx.setContainerId(container.getId()) - ctx.setResource(container.getResource()) - val localResources = prepareLocalResources - ctx.setLocalResources(localResources) - - val env = prepareEnvironment - ctx.setEnvironment(env) - - ctx.setUser(UserGroupInformation.getCurrentUser().getShortUserName()) - - val credentials = UserGroupInformation.getCurrentUser().getCredentials() - val dob = new DataOutputBuffer() - credentials.writeTokenStorageToStream(dob) - ctx.setContainerTokens(ByteBuffer.wrap(dob.getData())) - - val commands = prepareCommand(masterAddress, slaveId, hostname, executorMemory, executorCores, - appAttemptId, localResources) - logInfo("Setting up executor with commands: " + commands) - ctx.setCommands(commands) - - ctx.setApplicationACLs(YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr)) - - // If external shuffle service is enabled, register with the Yarn shuffle service already - // started on the NodeManager and, if authentication is enabled, provide it with our secret - // key for fetching shuffle files later - if (sparkConf.getBoolean("spark.shuffle.service.enabled", false)) { - val secretString = securityMgr.getSecretKey() - val secretBytes = - if (secretString != null) { - // This conversion must match how the YarnShuffleService decodes our secret - JavaUtils.stringToBytes(secretString) - } else { - // Authentication is not enabled, so just provide dummy metadata - ByteBuffer.allocate(0) - } - ctx.setServiceData(Map[String, ByteBuffer]("spark_shuffle" -> secretBytes)) - } - - // Send the start request to the ContainerManager - val startReq = Records.newRecord(classOf[StartContainerRequest]) - .asInstanceOf[StartContainerRequest] - startReq.setContainerLaunchContext(ctx) - cm.startContainer(startReq) - } - - def connectToCM: ContainerManager = { - val cmHostPortStr = container.getNodeId().getHost() + ":" + container.getNodeId().getPort() - val cmAddress = NetUtils.createSocketAddr(cmHostPortStr) - logInfo("Connecting to ContainerManager at " + cmHostPortStr) - - // Use doAs and remoteUser here so we can add the container token and not pollute the current - // users credentials with all of the individual container tokens - val user = UserGroupInformation.createRemoteUser(container.getId().toString()) - val containerToken = container.getContainerToken() - if (containerToken != null) { - user.addToken(ProtoUtils.convertFromProtoFormat(containerToken, cmAddress)) - } - - val proxy = user - .doAs(new PrivilegedExceptionAction[ContainerManager] { - def run: ContainerManager = { - rpc.getProxy(classOf[ContainerManager], cmAddress, conf).asInstanceOf[ContainerManager] - } - }) - proxy - } - -} 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 deleted file mode 100644 index abd37834ed3cc..0000000000000 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ /dev/null @@ -1,229 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn - -import java.util.concurrent.CopyOnWriteArrayList -import java.util.concurrent.atomic.AtomicInteger - -import scala.collection.JavaConversions._ -import scala.collection.mutable.{ArrayBuffer, HashMap} - -import org.apache.spark.{SecurityManager, 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._ -import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest -import org.apache.hadoop.yarn.util.Records - -/** - * Acquires resources for executors from a ResourceManager and launches executors in new containers. - */ -private[yarn] class YarnAllocationHandler( - conf: Configuration, - sparkConf: SparkConf, - resourceManager: AMRMProtocol, - appAttemptId: ApplicationAttemptId, - args: ApplicationMasterArguments, - preferredNodes: collection.Map[String, collection.Set[SplitInfo]], - securityMgr: SecurityManager) - extends YarnAllocator(conf, sparkConf, appAttemptId, args, preferredNodes, securityMgr) { - - private val lastResponseId = new AtomicInteger() - private val releaseList: CopyOnWriteArrayList[ContainerId] = new CopyOnWriteArrayList() - - override protected def allocateContainers(count: Int, pending: Int): YarnAllocateResponse = { - var resourceRequests: List[ResourceRequest] = null - - logDebug("asking for additional executors: " + count + " with already pending: " + pending) - val totalNumAsk = count + pending - if (count <= 0) { - resourceRequests = List() - } else if (preferredHostToCount.isEmpty) { - logDebug("host preferences is empty") - resourceRequests = List(createResourceRequest( - AllocationType.ANY, null, totalNumAsk, YarnSparkHadoopUtil.RM_REQUEST_PRIORITY)) - } else { - // request for all hosts in preferred nodes and for numExecutors - - // candidates.size, request by default allocation policy. - val hostContainerRequests: ArrayBuffer[ResourceRequest] = - new ArrayBuffer[ResourceRequest](preferredHostToCount.size) - for ((candidateHost, candidateCount) <- preferredHostToCount) { - val requiredCount = candidateCount - allocatedContainersOnHost(candidateHost) - - if (requiredCount > 0) { - hostContainerRequests += createResourceRequest( - AllocationType.HOST, - candidateHost, - requiredCount, - YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) - } - } - val rackContainerRequests: List[ResourceRequest] = createRackResourceRequests( - hostContainerRequests.toList) - - val anyContainerRequests: ResourceRequest = createResourceRequest( - AllocationType.ANY, - resource = null, - totalNumAsk, - YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) - - val containerRequests: ArrayBuffer[ResourceRequest] = new ArrayBuffer[ResourceRequest]( - hostContainerRequests.size + rackContainerRequests.size + 1) - - containerRequests ++= hostContainerRequests - containerRequests ++= rackContainerRequests - containerRequests += anyContainerRequests - - resourceRequests = containerRequests.toList - } - - val req = Records.newRecord(classOf[AllocateRequest]) - req.setResponseId(lastResponseId.incrementAndGet) - req.setApplicationAttemptId(appAttemptId) - - req.addAllAsks(resourceRequests) - - val releasedContainerList = createReleasedContainerList() - req.addAllReleases(releasedContainerList) - - if (count > 0) { - logInfo("Allocating %d executor containers with %d of memory each.".format(totalNumAsk, - executorMemory + memoryOverhead)) - } else { - logDebug("Empty allocation req .. release : " + releasedContainerList) - } - - for (request <- resourceRequests) { - logInfo("ResourceRequest (host : %s, num containers: %d, priority = %s , capability : %s)". - format( - request.getHostName, - request.getNumContainers, - request.getPriority, - request.getCapability)) - } - 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, - resource:String, - numExecutors: Int, - priority: Int): ResourceRequest = { - - // If hostname specified, we need atleast two requests - node local and rack local. - // There must be a third request - which is ANY : that will be specially handled. - requestType match { - case AllocationType.HOST => { - assert(YarnSparkHadoopUtil.ANY_HOST != resource) - val hostname = resource - val nodeLocal = createResourceRequestImpl(hostname, numExecutors, priority) - - // Add to host->rack mapping - YarnSparkHadoopUtil.populateRackInfo(conf, hostname) - - nodeLocal - } - case AllocationType.RACK => { - val rack = resource - createResourceRequestImpl(rack, numExecutors, priority) - } - case AllocationType.ANY => createResourceRequestImpl( - YarnSparkHadoopUtil.ANY_HOST, numExecutors, priority) - case _ => throw new IllegalArgumentException( - "Unexpected/unsupported request type: " + requestType) - } - } - - private def createResourceRequestImpl( - hostname:String, - numExecutors: Int, - priority: Int): ResourceRequest = { - - val rsrcRequest = Records.newRecord(classOf[ResourceRequest]) - val memCapability = Records.newRecord(classOf[Resource]) - // There probably is some overhead here, let's reserve a bit more memory. - memCapability.setMemory(executorMemory + memoryOverhead) - rsrcRequest.setCapability(memCapability) - - val pri = Records.newRecord(classOf[Priority]) - pri.setPriority(priority) - rsrcRequest.setPriority(pri) - - rsrcRequest.setHostName(hostname) - - rsrcRequest.setNumContainers(java.lang.Math.max(numExecutors, 0)) - rsrcRequest - } - - private def createReleasedContainerList(): ArrayBuffer[ContainerId] = { - val retval = new ArrayBuffer[ContainerId](1) - // Iterator on COW list ... - for (container <- releaseList.iterator()){ - retval += container - } - // Remove from the original list. - if (!retval.isEmpty) { - releaseList.removeAll(retval) - logInfo("Releasing " + retval.size + " containers.") - } - retval - } - - 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/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala deleted file mode 100644 index e342cc82f454e..0000000000000 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala +++ /dev/null @@ -1,118 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn - -import scala.collection.{Map, Set} -import java.net.URI - -import org.apache.hadoop.net.NetUtils -import org.apache.hadoop.yarn.api._ -import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.api.protocolrecords._ -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, SecurityManager, SparkConf} -import org.apache.spark.scheduler.SplitInfo -import org.apache.spark.util.Utils - -/** - * YarnRMClient implementation for the Yarn alpha API. - */ -private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMClient with Logging { - - private var rpc: YarnRPC = null - private var resourceManager: AMRMProtocol = _ - private var uiHistoryAddress: String = _ - private var registered: Boolean = false - - override def register( - conf: YarnConfiguration, - sparkConf: SparkConf, - preferredNodeLocations: Map[String, Set[SplitInfo]], - uiAddress: String, - uiHistoryAddress: String, - securityMgr: SecurityManager) = { - this.rpc = YarnRPC.create(conf) - this.uiHistoryAddress = uiHistoryAddress - - synchronized { - resourceManager = registerWithResourceManager(conf) - registerApplicationMaster(uiAddress) - registered = true - } - - new YarnAllocationHandler(conf, sparkConf, resourceManager, getAttemptId(), args, - preferredNodeLocations, securityMgr) - } - - override def getAttemptId() = { - val envs = System.getenv() - val containerIdString = envs.get(ApplicationConstants.AM_CONTAINER_ID_ENV) - val containerId = ConverterUtils.toContainerId(containerIdString) - val appAttemptId = containerId.getApplicationAttemptId() - appAttemptId - } - - override def unregister(status: FinalApplicationStatus, diagnostics: String = "") = synchronized { - if (registered) { - val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) - .asInstanceOf[FinishApplicationMasterRequest] - finishReq.setAppAttemptId(getAttemptId()) - finishReq.setFinishApplicationStatus(status) - finishReq.setDiagnostics(diagnostics) - finishReq.setTrackingUrl(uiHistoryAddress) - resourceManager.finishApplicationMaster(finishReq) - } - } - - override def getAmIpFilterParams(conf: YarnConfiguration, proxyBase: String) = { - val proxy = YarnConfiguration.getProxyHostAndPort(conf) - val parts = proxy.split(":") - val uriBase = "http://" + proxy + proxyBase - Map("PROXY_HOST" -> parts(0), "PROXY_URI_BASE" -> uriBase) - } - - override def getMaxRegAttempts(conf: YarnConfiguration) = - conf.getInt(YarnConfiguration.RM_AM_MAX_RETRIES, YarnConfiguration.DEFAULT_RM_AM_MAX_RETRIES) - - private def registerWithResourceManager(conf: YarnConfiguration): AMRMProtocol = { - val rmAddress = NetUtils.createSocketAddr(conf.get(YarnConfiguration.RM_SCHEDULER_ADDRESS, - YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS)) - logInfo("Connecting to ResourceManager at " + rmAddress) - rpc.getProxy(classOf[AMRMProtocol], rmAddress, conf).asInstanceOf[AMRMProtocol] - } - - private def registerApplicationMaster(uiAddress: String): RegisterApplicationMasterResponse = { - val appMasterRequest = Records.newRecord(classOf[RegisterApplicationMasterRequest]) - .asInstanceOf[RegisterApplicationMasterRequest] - appMasterRequest.setApplicationAttemptId(getAttemptId()) - // Setting this to master host,port - so that the ApplicationReport at client has some - // sensible info. - // Users can then monitor stderr/stdout on that node if required. - appMasterRequest.setHost(Utils.localHostName()) - appMasterRequest.setRpcPort(0) - // remove the scheme from the url if it exists since Hadoop does not expect scheme - val uri = new URI(uiAddress) - val authority = if (uri.getScheme == null) uiAddress else uri.getAuthority - appMasterRequest.setTrackingUrl(authority) - resourceManager.registerApplicationMaster(appMasterRequest) - } - -} diff --git a/yarn/pom.xml b/yarn/pom.xml index bba73648c7abe..d7579bf9622d6 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -25,9 +25,9 @@ org.apache.spark - yarn-parent_2.10 - pom - Spark Project YARN Parent POM + spark-yarn_2.10 + jar + Spark Project YARN yarn @@ -58,6 +58,12 @@ org.apache.hadoop hadoop-client + + org.apache.hadoop + hadoop-yarn-server-tests + tests + test + org.scalatest scalatest_${scala.binary.version} @@ -70,41 +76,54 @@ + - yarn-alpha - - - - maven-antrun-plugin - - - validate - - run - - - - ******************************************************************************************* - ***WARNING***: Support for YARN-alpha API's will be removed in Spark 1.3 (see SPARK-3445).* - ******************************************************************************************* - - - - - - - - - alpha - - - - - yarn - - stable - + hadoop-2.2 + + 1.9 + + + + org.mortbay.jetty + jetty + 6.1.26 + + + org.mortbay.jetty + servlet-api + + + test + + + com.sun.jersey + jersey-core + ${jersey.version} + test + + + com.sun.jersey + jersey-json + ${jersey.version} + test + + + stax + stax-api + + + + + com.sun.jersey + jersey-server + ${jersey.version} + test + + @@ -124,38 +143,6 @@ true - - org.codehaus.mojo - build-helper-maven-plugin - - - add-scala-sources - generate-sources - - add-source - - - - src/main/scala - ../common/src/main/scala - - - - - add-scala-test-sources - generate-test-sources - - add-test-source - - - - src/test/scala - ../common/src/test/scala - - - - - org.scalatest scalatest-maven-plugin @@ -169,12 +156,6 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - - - - ../common/src/main/resources - - diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala similarity index 100% rename from yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala similarity index 100% rename from yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala similarity index 100% rename from yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala similarity index 82% rename from yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 4d859450efc63..c439969510fc9 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -178,21 +178,25 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) private def getUsageMessage(unknownParam: List[String] = null): String = { val message = if (unknownParam != null) s"Unknown/unsupported param $unknownParam\n" else "" - message + - "Usage: org.apache.spark.deploy.yarn.Client [options] \n" + - "Options:\n" + - " --jar JAR_PATH Path to your application's JAR file (required in yarn-cluster mode)\n" + - " --class CLASS_NAME Name of your application's main class (required)\n" + - " --arg ARG Argument to be passed to your application's main class.\n" + - " Multiple invocations are possible, each will be passed in order.\n" + - " --num-executors NUM Number of executors to start (Default: 2)\n" + - " --executor-cores NUM Number of cores for the executors (Default: 1).\n" + - " --driver-memory MEM Memory for driver (e.g. 1000M, 2G) (Default: 512 Mb)\n" + - " --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G)\n" + - " --name NAME The name of your application (Default: Spark)\n" + - " --queue QUEUE The hadoop queue to use for allocation requests (Default: 'default')\n" + - " --addJars jars Comma separated list of local jars that want SparkContext.addJar to work with.\n" + - " --files files Comma separated list of files to be distributed with the job.\n" + - " --archives archives Comma separated list of archives to be distributed with the job." + message + """ + |Usage: org.apache.spark.deploy.yarn.Client [options] + |Options: + | --jar JAR_PATH Path to your application's JAR file (required in yarn-cluster + | mode) + | --class CLASS_NAME Name of your application's main class (required) + | --arg ARG Argument to be passed to your application's main class. + | Multiple invocations are possible, each will be passed in order. + | --num-executors NUM Number of executors to start (Default: 2) + | --executor-cores NUM Number of cores for the executors (Default: 1). + | --driver-memory MEM Memory for driver (e.g. 1000M, 2G) (Default: 512 Mb) + | --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G) + | --name NAME The name of your application (Default: Spark) + | --queue QUEUE The hadoop queue to use for allocation requests (Default: + | 'default') + | --addJars jars Comma separated list of local jars that want SparkContext.addJar + | to work with. + | --files files Comma separated list of files to be distributed with the job. + | --archives archives Comma separated list of archives to be distributed with the job. + """ } } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala similarity index 100% rename from yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala similarity index 100% rename from yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala similarity index 100% rename from yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala similarity index 97% rename from yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index 88dad0febd03f..22d73ecf6d010 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -93,12 +93,13 @@ trait ExecutorRunnableUtil extends Logging { /* else { // If no java_opts specified, default to using -XX:+CMSIncrementalMode - // It might be possible that other modes/config is being done in spark.executor.extraJavaOptions, - // so we dont want to mess with it. + // It might be possible that other modes/config is being done in + // spark.executor.extraJavaOptions, so we dont want to mess with it. // In our expts, using (default) throughput collector has severe perf ramnifications in // multi-tennent machines // The options are based on - // http://www.oracle.com/technetwork/java/gc-tuning-5-138395.html#0.0.0.%20When%20to%20Use%20the%20Concurrent%20Low%20Pause%20Collector|outline + // http://www.oracle.com/technetwork/java/gc-tuning-5-138395.html#0.0.0.%20When%20to%20Use + // %20the%20Concurrent%20Low%20Pause%20Collector|outline javaOpts += " -XX:+UseConcMarkSweepGC " javaOpts += " -XX:+CMSIncrementalMode " javaOpts += " -XX:+CMSIncrementalPacing " diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala similarity index 100% rename from yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala similarity index 100% rename from yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala similarity index 100% rename from yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala similarity index 100% rename from yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala similarity index 96% rename from yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 7d453ecb7983c..d7cf904db1c9e 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -48,15 +48,17 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { dest.addCredentials(source.getCredentials()) } - // Note that all params which start with SPARK are propagated all the way through, so if in yarn mode, this MUST be set to true. + // Note that all params which start with SPARK are propagated all the way through, so if in yarn + // mode, this MUST be set to true. override def isYarnMode(): Boolean = { true } - // Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop subsystems - // Always create a new config, dont reuse yarnConf. + // Return an appropriate (subclass) of Configuration. Creating a config initializes some Hadoop + // subsystems. Always create a new config, dont reuse yarnConf. override def newConfiguration(conf: SparkConf): Configuration = new YarnConfiguration(super.newConfiguration(conf)) - // add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster + // Add any user credentials to the job conf which are necessary for running on a secure Hadoop + // cluster override def addCredentials(conf: JobConf) { val jobCreds = conf.getCredentials() jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala similarity index 100% rename from yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala rename to yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala similarity index 100% rename from yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala rename to yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala similarity index 100% rename from yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala rename to yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala similarity index 100% rename from yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala rename to yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala diff --git a/yarn/stable/src/test/resources/log4j.properties b/yarn/src/test/resources/log4j.properties similarity index 100% rename from yarn/stable/src/test/resources/log4j.properties rename to yarn/src/test/resources/log4j.properties diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala similarity index 100% rename from yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala rename to yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala similarity index 100% rename from yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala rename to yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala similarity index 100% rename from yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala rename to yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala diff --git a/yarn/stable/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala similarity index 100% rename from yarn/stable/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala rename to yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala similarity index 100% rename from yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala rename to yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml deleted file mode 100644 index 8b6521ad7f859..0000000000000 --- a/yarn/stable/pom.xml +++ /dev/null @@ -1,95 +0,0 @@ - - - - 4.0.0 - - org.apache.spark - yarn-parent_2.10 - 1.3.0-SNAPSHOT - ../pom.xml - - - yarn-stable - - - org.apache.spark - spark-yarn_2.10 - jar - Spark Project YARN Stable API - - - - org.apache.hadoop - hadoop-yarn-server-tests - tests - test - - - - - - - hadoop-2.2 - - 1.9 - - - - org.mortbay.jetty - jetty - 6.1.26 - - - org.mortbay.jetty - servlet-api - - - test - - - com.sun.jersey - jersey-core - ${jersey.version} - test - - - com.sun.jersey - jersey-json - ${jersey.version} - test - - - stax - stax-api - - - - - com.sun.jersey - jersey-server - ${jersey.version} - test - - - - - - From 61f1a7022767e64ab092aa91c0c5aa1b2fdbef7c Mon Sep 17 00:00:00 2001 From: jbencook Date: Tue, 9 Dec 2014 12:16:19 -0800 Subject: [PATCH 017/155] [SPARK-874] adding a --wait flag This PR adds a --wait flag to the `./sbin/stop-all.sh` script. Author: jbencook Closes #3567 from jbencook/master and squashes the following commits: d05c5bb [jbencook] [SPARK-874] adding a --wait flag --- sbin/stop-all.sh | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/sbin/stop-all.sh b/sbin/stop-all.sh index 298c6a9859795..971d5d49da664 100755 --- a/sbin/stop-all.sh +++ b/sbin/stop-all.sh @@ -30,3 +30,20 @@ sbin="`cd "$sbin"; pwd`" # Stop the slaves, then the master "$sbin"/stop-slaves.sh "$sbin"/stop-master.sh + +if [ "$1" == "--wait" ] +then + printf "Waiting for workers to shut down..." + while true + do + running=`$sbin/slaves.sh ps -ef | grep -v grep | grep deploy.worker.Worker` + if [ -z "$running" ] + then + printf "\nAll workers successfully shut down.\n" + break + else + printf "." + sleep 10 + fi + done +fi From b31074466a83d3d1387fc1e4337dfab9e164fc04 Mon Sep 17 00:00:00 2001 From: maji2014 Date: Tue, 9 Dec 2014 13:13:12 -0800 Subject: [PATCH 018/155] [SPARK-4691][shuffle] Restructure a few lines in shuffle code In HashShuffleReader.scala and HashShuffleWriter.scala, no need to judge "dep.aggregator.isEmpty" again as this is judged by "dep.aggregator.isDefined" In SortShuffleWriter.scala, "dep.aggregator.isEmpty" is better than "!dep.aggregator.isDefined" ? Author: maji2014 Closes #3553 from maji2014/spark-4691 and squashes the following commits: bf7b14d [maji2014] change a elegant way for SortShuffleWriter.scala 10d0cf0 [maji2014] change a elegant way d8f52dc [maji2014] code optimization for judgement --- .../org/apache/spark/shuffle/hash/HashShuffleReader.scala | 4 ++-- .../org/apache/spark/shuffle/hash/HashShuffleWriter.scala | 3 +-- .../org/apache/spark/shuffle/sort/SortShuffleWriter.scala | 4 +--- 3 files changed, 4 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala index 5baf45db45c17..de72148ccc7ac 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala @@ -45,9 +45,9 @@ private[spark] class HashShuffleReader[K, C]( } else { new InterruptibleIterator(context, dep.aggregator.get.combineValuesByKey(iter, context)) } - } else if (dep.aggregator.isEmpty && dep.mapSideCombine) { - throw new IllegalStateException("Aggregator is empty for map-side combine") } else { + require(!dep.mapSideCombine, "Map-side combine without Aggregator specified!") + // Convert the Product2s to pairs since this is what downstream RDDs currently expect iter.asInstanceOf[Iterator[Product2[K, C]]].map(pair => (pair._1, pair._2)) } diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala index 183a30373b28c..755f17d6aa15a 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala @@ -56,9 +56,8 @@ private[spark] class HashShuffleWriter[K, V]( } else { records } - } else if (dep.aggregator.isEmpty && dep.mapSideCombine) { - throw new IllegalStateException("Aggregator is empty for map-side combine") } else { + require(!dep.mapSideCombine, "Map-side combine without Aggregator specified!") records } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index d75f9d7311fad..27496c5a289cb 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -50,9 +50,7 @@ private[spark] class SortShuffleWriter[K, V, C]( /** Write a bunch of records to this task's output */ override def write(records: Iterator[_ <: Product2[K, V]]): Unit = { if (dep.mapSideCombine) { - if (!dep.aggregator.isDefined) { - throw new IllegalStateException("Aggregator is empty for map-side combine") - } + require(dep.aggregator.isDefined, "Map-side combine without Aggregator specified!") sorter = new ExternalSorter[K, V, C]( dep.aggregator, Some(dep.partitioner), dep.keyOrdering, dep.serializer) sorter.insertAll(records) From 1f5110630c1abb13a357b463c805a39772923b82 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Tue, 9 Dec 2014 15:10:36 -0800 Subject: [PATCH 019/155] [SPARK-4765] Make GC time always shown in UI. This commit removes the GC time for each task from the set of optional, additional metrics, and instead always shows it for each task. cc pwendell Author: Kay Ousterhout Closes #3622 from kayousterhout/gc_time and squashes the following commits: 15ac242 [Kay Ousterhout] Make TaskDetailsClassNames private[spark] e71d893 [Kay Ousterhout] [SPARK-4765] Make GC time always shown in UI. --- .../resources/org/apache/spark/ui/static/webui.css | 2 +- .../scala/org/apache/spark/ui/jobs/StagePage.scala | 13 +++---------- .../spark/ui/jobs/TaskDetailsClassNames.scala | 3 +-- 3 files changed, 5 insertions(+), 13 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index cdf85bfbf326f..68c52ac09b55a 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -171,6 +171,6 @@ span.additional-metric-title { /* Hide all additional metrics by default. This is done here rather than using JavaScript to * avoid slow page loads for stage pages with large numbers (e.g., thousands) of tasks. */ -.scheduler_delay, .gc_time, .deserialization_time, .serialization_time, .getting_result_time { +.scheduler_delay, .deserialization_time, .serialization_time, .getting_result_time { display: none; } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index bfa54f8492068..09a936c2234c0 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -132,13 +132,6 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { Task Deserialization Time -
  • - - - GC Time - -
  • @@ -168,7 +161,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { ("Executor ID / Host", ""), ("Launch Time", ""), ("Duration", ""), ("Scheduler Delay", TaskDetailsClassNames.SCHEDULER_DELAY), ("Task Deserialization Time", TaskDetailsClassNames.TASK_DESERIALIZATION_TIME), - ("GC Time", TaskDetailsClassNames.GC_TIME), + ("GC Time", ""), ("Result Serialization Time", TaskDetailsClassNames.RESULT_SERIALIZATION_TIME), ("Getting Result Time", TaskDetailsClassNames.GETTING_RESULT_TIME)) ++ {if (hasAccumulators) Seq(("Accumulators", "")) else Nil} ++ @@ -308,7 +301,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { {deserializationQuantiles} - {gcQuantiles}, + {gcQuantiles}, {serializationQuantiles} , @@ -429,7 +422,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { class={TaskDetailsClassNames.TASK_DESERIALIZATION_TIME}> {UIUtils.formatDuration(taskDeserializationTime.toLong)} - + {if (gcTime > 0) UIUtils.formatDuration(gcTime) else ""} Date: Tue, 9 Dec 2014 15:11:20 -0800 Subject: [PATCH 020/155] [SPARK-4714] BlockManager.dropFromMemory() should check whether block has been removed after synchronizing on BlockInfo instance. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit After synchronizing on the `info` lock in the `removeBlock`/`dropOldBlocks`/`dropFromMemory` methods in BlockManager, the block that `info` represented may have already removed. The three methods have the same logic to get the `info` lock: ``` info = blockInfo.get(id) if (info != null) { info.synchronized { // do something } } ``` So, there is chance that when a thread enters the `info.synchronized` block, `info` has already been removed from the `blockInfo` map by some other thread who entered `info.synchronized` first. The `removeBlock` and `dropOldBlocks` methods are idempotent, so it's safe for them to run on blocks that have already been removed. But in `dropFromMemory` it may be problematic since it may drop block data which already removed into the diskstore, and this calls data store operations that are not designed to handle missing blocks. This patch fixes this issue by adding a check to `dropFromMemory` to test whether blocks have been removed by a racing thread. Author: hushan[胡珊] Closes #3574 from suyanNone/refine-block-concurrency and squashes the following commits: edb989d [hushan[胡珊]] Refine code style and comments position 55fa4ba [hushan[胡珊]] refine code e57e270 [hushan[胡珊]] add check info is already remove or not while having gotten info.syn --- .../main/scala/org/apache/spark/storage/BlockManager.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 308c59eda594d..d7b184f8a10e9 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -1014,8 +1014,10 @@ private[spark] class BlockManager( // If we get here, the block write failed. logWarning(s"Block $blockId was marked as failure. Nothing to drop") return None + } else if (blockInfo.get(blockId).isEmpty) { + logWarning(s"Block $blockId was already dropped.") + return None } - var blockIsUpdated = false val level = info.level From 5e4c06f8e54265a4024857f5978ec54c936aeea2 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Tue, 9 Dec 2014 16:26:07 -0800 Subject: [PATCH 021/155] SPARK-4567. Make SparkJobInfo and SparkStageInfo serializable Author: Sandy Ryza Closes #3426 from sryza/sandy-spark-4567 and squashes the following commits: cb4b8d2 [Sandy Ryza] SPARK-4567. Make SparkJobInfo and SparkStageInfo serializable --- core/src/main/java/org/apache/spark/SparkJobInfo.java | 4 +++- core/src/main/java/org/apache/spark/SparkStageInfo.java | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/spark/SparkJobInfo.java b/core/src/main/java/org/apache/spark/SparkJobInfo.java index 4e3c983b1170a..e31c4401632a6 100644 --- a/core/src/main/java/org/apache/spark/SparkJobInfo.java +++ b/core/src/main/java/org/apache/spark/SparkJobInfo.java @@ -17,13 +17,15 @@ package org.apache.spark; +import java.io.Serializable; + /** * Exposes information about Spark Jobs. * * This interface is not designed to be implemented outside of Spark. We may add additional methods * which may break binary compatibility with outside implementations. */ -public interface SparkJobInfo { +public interface SparkJobInfo extends Serializable { int jobId(); int[] stageIds(); JobExecutionStatus status(); diff --git a/core/src/main/java/org/apache/spark/SparkStageInfo.java b/core/src/main/java/org/apache/spark/SparkStageInfo.java index fd74321093658..b7d462abd72d6 100644 --- a/core/src/main/java/org/apache/spark/SparkStageInfo.java +++ b/core/src/main/java/org/apache/spark/SparkStageInfo.java @@ -17,13 +17,15 @@ package org.apache.spark; +import java.io.Serializable; + /** * Exposes information about Spark Stages. * * This interface is not designed to be implemented outside of Spark. We may add additional methods * which may break binary compatibility with outside implementations. */ -public interface SparkStageInfo { +public interface SparkStageInfo extends Serializable { int stageId(); int currentAttemptId(); long submissionTime(); From d8f84f26e388055ca7459810e001d05ab60af15b Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 9 Dec 2014 16:38:27 -0800 Subject: [PATCH 022/155] SPARK-4805 [CORE] BlockTransferMessage.toByteArray() trips assertion Allocate enough room for type byte as well as message, to avoid tripping assertion about capacity of the buffer Author: Sean Owen Closes #3650 from srowen/SPARK-4805 and squashes the following commits: 9e1d502 [Sean Owen] Allocate enough room for type byte as well as message, to avoid tripping assertion about capacity of the buffer --- .../spark/network/shuffle/protocol/BlockTransferMessage.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java index b4b13b8a6ef5d..6c1210b33268a 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java @@ -67,7 +67,8 @@ public static BlockTransferMessage fromByteArray(byte[] msg) { /** Serializes the 'type' byte followed by the message itself. */ public byte[] toByteArray() { - ByteBuf buf = Unpooled.buffer(encodedLength()); + // Allow room for encoded message, plus the type byte + ByteBuf buf = Unpooled.buffer(encodedLength() + 1); buf.writeByte(type().id); encode(buf); assert buf.writableBytes() == 0 : "Writable bytes remain: " + buf.writableBytes(); From 2b9b72682e587909a84d3ace214c22cec830eeaf Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 9 Dec 2014 17:49:59 -0800 Subject: [PATCH 023/155] [SPARK-4740] Create multiple concurrent connections between two peer nodes in Netty. It's been reported that when the number of disks is large and the number of nodes is small, Netty network throughput is low compared with NIO. We suspect the problem is that only a small number of disks are utilized to serve shuffle files at any given point, due to connection reuse. This patch adds a new config parameter to specify the number of concurrent connections between two peer nodes, default to 2. Author: Reynold Xin Closes #3625 from rxin/SPARK-4740 and squashes the following commits: ad4241a [Reynold Xin] Updated javadoc. f33c72b [Reynold Xin] Code review feedback. 0fefabb [Reynold Xin] Use double check in synchronization. 41dfcb2 [Reynold Xin] Added test case. 9076b4a [Reynold Xin] Fixed two NPEs. 3e1306c [Reynold Xin] Minor style fix. 4f21673 [Reynold Xin] [SPARK-4740] Create multiple concurrent connections between two peer nodes in Netty. --- .../client/TransportClientFactory.java | 124 ++++++++++++------ .../spark/network/util/TransportConf.java | 5 + .../network/TransportClientFactorySuite.java | 97 +++++++++++++- 3 files changed, 180 insertions(+), 46 deletions(-) diff --git a/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java b/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java index 9afd5decd5e6b..d26b9b4d6055f 100644 --- a/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java +++ b/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java @@ -22,6 +22,7 @@ import java.net.InetSocketAddress; import java.net.SocketAddress; import java.util.List; +import java.util.Random; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicReference; @@ -42,6 +43,7 @@ import org.apache.spark.network.TransportContext; import org.apache.spark.network.server.TransportChannelHandler; import org.apache.spark.network.util.IOMode; +import org.apache.spark.network.util.JavaUtils; import org.apache.spark.network.util.NettyUtils; import org.apache.spark.network.util.TransportConf; @@ -56,12 +58,31 @@ * TransportClient, all given {@link TransportClientBootstrap}s will be run. */ public class TransportClientFactory implements Closeable { + + /** A simple data structure to track the pool of clients between two peer nodes. */ + private static class ClientPool { + TransportClient[] clients; + Object[] locks; + + public ClientPool(int size) { + clients = new TransportClient[size]; + locks = new Object[size]; + for (int i = 0; i < size; i++) { + locks[i] = new Object(); + } + } + } + private final Logger logger = LoggerFactory.getLogger(TransportClientFactory.class); private final TransportContext context; private final TransportConf conf; private final List clientBootstraps; - private final ConcurrentHashMap connectionPool; + private final ConcurrentHashMap connectionPool; + + /** Random number generator for picking connections between peers. */ + private final Random rand; + private final int numConnectionsPerPeer; private final Class socketChannelClass; private EventLoopGroup workerGroup; @@ -73,7 +94,9 @@ public TransportClientFactory( this.context = Preconditions.checkNotNull(context); this.conf = context.getConf(); this.clientBootstraps = Lists.newArrayList(Preconditions.checkNotNull(clientBootstraps)); - this.connectionPool = new ConcurrentHashMap(); + this.connectionPool = new ConcurrentHashMap(); + this.numConnectionsPerPeer = conf.numConnectionsPerPeer(); + this.rand = new Random(); IOMode ioMode = IOMode.valueOf(conf.ioMode()); this.socketChannelClass = NettyUtils.getClientChannelClass(ioMode); @@ -84,10 +107,14 @@ public TransportClientFactory( } /** - * Create a new {@link TransportClient} connecting to the given remote host / port. This will - * reuse TransportClients if they are still active and are for the same remote address. Prior - * to the creation of a new TransportClient, we will execute all {@link TransportClientBootstrap}s - * that are registered with this factory. + * Create a {@link TransportClient} connecting to the given remote host / port. + * + * We maintains an array of clients (size determined by spark.shuffle.io.numConnectionsPerPeer) + * and randomly picks one to use. If no client was previously created in the randomly selected + * spot, this function creates a new client and places it there. + * + * Prior to the creation of a new TransportClient, we will execute all + * {@link TransportClientBootstrap}s that are registered with this factory. * * This blocks until a connection is successfully established and fully bootstrapped. * @@ -97,23 +124,48 @@ public TransportClient createClient(String remoteHost, int remotePort) throws IO // Get connection from the connection pool first. // If it is not found or not active, create a new one. final InetSocketAddress address = new InetSocketAddress(remoteHost, remotePort); - TransportClient cachedClient = connectionPool.get(address); - if (cachedClient != null) { - if (cachedClient.isActive()) { - logger.trace("Returning cached connection to {}: {}", address, cachedClient); - return cachedClient; - } else { - logger.info("Found inactive connection to {}, closing it.", address); - connectionPool.remove(address, cachedClient); // Remove inactive clients. + + // Create the ClientPool if we don't have it yet. + ClientPool clientPool = connectionPool.get(address); + if (clientPool == null) { + connectionPool.putIfAbsent(address, new ClientPool(numConnectionsPerPeer)); + clientPool = connectionPool.get(address); + } + + int clientIndex = rand.nextInt(numConnectionsPerPeer); + TransportClient cachedClient = clientPool.clients[clientIndex]; + + if (cachedClient != null && cachedClient.isActive()) { + logger.trace("Returning cached connection to {}: {}", address, cachedClient); + return cachedClient; + } + + // If we reach here, we don't have an existing connection open. Let's create a new one. + // Multiple threads might race here to create new connections. Keep only one of them active. + synchronized (clientPool.locks[clientIndex]) { + cachedClient = clientPool.clients[clientIndex]; + + if (cachedClient != null) { + if (cachedClient.isActive()) { + logger.trace("Returning cached connection to {}: {}", address, cachedClient); + return cachedClient; + } else { + logger.info("Found inactive connection to {}, creating a new one.", address); + } } + clientPool.clients[clientIndex] = createClient(address); + return clientPool.clients[clientIndex]; } + } + /** Create a completely new {@link TransportClient} to the remote address. */ + private TransportClient createClient(InetSocketAddress address) throws IOException { logger.debug("Creating new connection to " + address); Bootstrap bootstrap = new Bootstrap(); bootstrap.group(workerGroup) .channel(socketChannelClass) - // Disable Nagle's Algorithm since we don't want packets to wait + // Disable Nagle's Algorithm since we don't want packets to wait .option(ChannelOption.TCP_NODELAY, true) .option(ChannelOption.SO_KEEPALIVE, true) .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, conf.connectionTimeoutMs()) @@ -130,7 +182,7 @@ public void initChannel(SocketChannel ch) { }); // Connect to the remote server - long preConnect = System.currentTimeMillis(); + long preConnect = System.nanoTime(); ChannelFuture cf = bootstrap.connect(address); if (!cf.awaitUninterruptibly(conf.connectionTimeoutMs())) { throw new IOException( @@ -143,43 +195,37 @@ public void initChannel(SocketChannel ch) { assert client != null : "Channel future completed successfully with null client"; // Execute any client bootstraps synchronously before marking the Client as successful. - long preBootstrap = System.currentTimeMillis(); + long preBootstrap = System.nanoTime(); logger.debug("Connection to {} successful, running bootstraps...", address); try { for (TransportClientBootstrap clientBootstrap : clientBootstraps) { clientBootstrap.doBootstrap(client); } } catch (Exception e) { // catch non-RuntimeExceptions too as bootstrap may be written in Scala - long bootstrapTime = System.currentTimeMillis() - preBootstrap; - logger.error("Exception while bootstrapping client after " + bootstrapTime + " ms", e); + long bootstrapTimeMs = (System.nanoTime() - preBootstrap) / 1000000; + logger.error("Exception while bootstrapping client after " + bootstrapTimeMs + " ms", e); client.close(); throw Throwables.propagate(e); } - long postBootstrap = System.currentTimeMillis(); - - // Successful connection & bootstrap -- in the event that two threads raced to create a client, - // use the first one that was put into the connectionPool and close the one we made here. - TransportClient oldClient = connectionPool.putIfAbsent(address, client); - if (oldClient == null) { - logger.debug("Successfully created connection to {} after {} ms ({} ms spent in bootstraps)", - address, postBootstrap - preConnect, postBootstrap - preBootstrap); - return client; - } else { - logger.debug("Two clients were created concurrently after {} ms, second will be disposed.", - postBootstrap - preConnect); - client.close(); - return oldClient; - } + long postBootstrap = System.nanoTime(); + + logger.debug("Successfully created connection to {} after {} ms ({} ms spent in bootstraps)", + address, (postBootstrap - preConnect) / 1000000, (postBootstrap - preBootstrap) / 1000000); + + return client; } /** Close all connections in the connection pool, and shutdown the worker thread pool. */ @Override public void close() { - for (TransportClient client : connectionPool.values()) { - try { - client.close(); - } catch (RuntimeException e) { - logger.warn("Ignoring exception during close", e); + // Go through all clients and close them if they are active. + for (ClientPool clientPool : connectionPool.values()) { + for (int i = 0; i < clientPool.clients.length; i++) { + TransportClient client = clientPool.clients[i]; + if (client != null) { + clientPool.clients[i] = null; + JavaUtils.closeQuietly(client); + } } } connectionPool.clear(); diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java index 1af40acf8b4af..f60573998f7ae 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -40,6 +40,11 @@ public int connectionTimeoutMs() { return conf.getInt("spark.shuffle.io.connectionTimeout", 120) * 1000; } + /** Number of concurrent connections between two nodes for fetching data. **/ + public int numConnectionsPerPeer() { + return conf.getInt("spark.shuffle.io.numConnectionsPerPeer", 2); + } + /** Requested maximum length of the queue of incoming connections. Default -1 for no backlog. */ public int backLog() { return conf.getInt("spark.shuffle.io.backLog", -1); } diff --git a/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java b/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java index 822bef1d81b2a..416dc1b969fa4 100644 --- a/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java +++ b/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java @@ -18,7 +18,11 @@ package org.apache.spark.network; import java.io.IOException; -import java.util.concurrent.TimeoutException; +import java.util.Collections; +import java.util.HashSet; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; import org.junit.After; import org.junit.Before; @@ -32,6 +36,7 @@ import org.apache.spark.network.server.NoOpRpcHandler; import org.apache.spark.network.server.RpcHandler; import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.util.ConfigProvider; import org.apache.spark.network.util.JavaUtils; import org.apache.spark.network.util.SystemPropertyConfigProvider; import org.apache.spark.network.util.TransportConf; @@ -57,16 +62,94 @@ public void tearDown() { JavaUtils.closeQuietly(server2); } + /** + * Request a bunch of clients to a single server to test + * we create up to maxConnections of clients. + * + * If concurrent is true, create multiple threads to create clients in parallel. + */ + private void testClientReuse(final int maxConnections, boolean concurrent) + throws IOException, InterruptedException { + TransportConf conf = new TransportConf(new ConfigProvider() { + @Override + public String get(String name) { + if (name.equals("spark.shuffle.io.numConnectionsPerPeer")) { + return Integer.toString(maxConnections); + } else { + throw new NoSuchElementException(); + } + } + }); + + RpcHandler rpcHandler = new NoOpRpcHandler(); + TransportContext context = new TransportContext(conf, rpcHandler); + final TransportClientFactory factory = context.createClientFactory(); + final Set clients = Collections.synchronizedSet( + new HashSet()); + + final AtomicInteger failed = new AtomicInteger(); + Thread[] attempts = new Thread[maxConnections * 10]; + + // Launch a bunch of threads to create new clients. + for (int i = 0; i < attempts.length; i++) { + attempts[i] = new Thread() { + @Override + public void run() { + try { + TransportClient client = + factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + assert (client.isActive()); + clients.add(client); + } catch (IOException e) { + failed.incrementAndGet(); + } + } + }; + + if (concurrent) { + attempts[i].start(); + } else { + attempts[i].run(); + } + } + + // Wait until all the threads complete. + for (int i = 0; i < attempts.length; i++) { + attempts[i].join(); + } + + assert(failed.get() == 0); + assert(clients.size() == maxConnections); + + for (TransportClient client : clients) { + client.close(); + } + } + + @Test + public void reuseClientsUpToConfigVariable() throws Exception { + testClientReuse(1, false); + testClientReuse(2, false); + testClientReuse(3, false); + testClientReuse(4, false); + } + @Test - public void createAndReuseBlockClients() throws IOException { + public void reuseClientsUpToConfigVariableConcurrent() throws Exception { + testClientReuse(1, true); + testClientReuse(2, true); + testClientReuse(3, true); + testClientReuse(4, true); + } + + @Test + public void returnDifferentClientsForDifferentServers() throws IOException { TransportClientFactory factory = context.createClientFactory(); TransportClient c1 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); - TransportClient c2 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); - TransportClient c3 = factory.createClient(TestUtils.getLocalHost(), server2.getPort()); + TransportClient c2 = factory.createClient(TestUtils.getLocalHost(), server2.getPort()); assertTrue(c1.isActive()); - assertTrue(c3.isActive()); - assertTrue(c1 == c2); - assertTrue(c1 != c3); + assertTrue(c2.isActive()); + assertTrue(c1 != c2); factory.close(); } From 9bd9334f588dbb44d01554f9f4ca68a153a48993 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 9 Dec 2014 19:29:09 -0800 Subject: [PATCH 024/155] Config updates for the new shuffle transport. Author: Reynold Xin Closes #3657 from rxin/conf-update and squashes the following commits: 7370eab [Reynold Xin] Config updates for the new shuffle transport. --- .../java/org/apache/spark/network/util/TransportConf.java | 8 ++++---- .../apache/spark/network/sasl/SaslClientBootstrap.java | 2 +- .../spark/network/shuffle/RetryingBlockFetcher.java | 2 +- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java index f60573998f7ae..13b37f96f8ce2 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -35,14 +35,14 @@ public boolean preferDirectBufs() { return conf.getBoolean("spark.shuffle.io.preferDirectBufs", true); } - /** Connect timeout in secs. Default 120 secs. */ + /** Connect timeout in milliseconds. Default 120 secs. */ public int connectionTimeoutMs() { return conf.getInt("spark.shuffle.io.connectionTimeout", 120) * 1000; } /** Number of concurrent connections between two nodes for fetching data. **/ public int numConnectionsPerPeer() { - return conf.getInt("spark.shuffle.io.numConnectionsPerPeer", 2); + return conf.getInt("spark.shuffle.io.numConnectionsPerPeer", 1); } /** Requested maximum length of the queue of incoming connections. Default -1 for no backlog. */ @@ -67,7 +67,7 @@ public int numConnectionsPerPeer() { public int sendBuf() { return conf.getInt("spark.shuffle.io.sendBuffer", -1); } /** Timeout for a single round trip of SASL token exchange, in milliseconds. */ - public int saslRTTimeout() { return conf.getInt("spark.shuffle.sasl.timeout", 30000); } + public int saslRTTimeoutMs() { return conf.getInt("spark.shuffle.sasl.timeout", 30) * 1000; } /** * Max number of times we will try IO exceptions (such as connection timeouts) per request. @@ -79,7 +79,7 @@ public int numConnectionsPerPeer() { * Time (in milliseconds) that we will wait in order to perform a retry after an IOException. * Only relevant if maxIORetries > 0. */ - public int ioRetryWaitTime() { return conf.getInt("spark.shuffle.io.retryWaitMs", 5000); } + public int ioRetryWaitTimeMs() { return conf.getInt("spark.shuffle.io.retryWait", 5) * 1000; } /** * Minimum size of a block that we should start using memory map rather than reading in through diff --git a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java index 7bc91e375371f..33aa1344345ff 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java @@ -59,7 +59,7 @@ public void doBootstrap(TransportClient client) { ByteBuf buf = Unpooled.buffer(msg.encodedLength()); msg.encode(buf); - byte[] response = client.sendRpcSync(buf.array(), conf.saslRTTimeout()); + byte[] response = client.sendRpcSync(buf.array(), conf.saslRTTimeoutMs()); payload = saslClient.response(response); } } finally { diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java index f8a1a266863bb..4bb0498e5d5aa 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java @@ -106,7 +106,7 @@ public RetryingBlockFetcher( this.fetchStarter = fetchStarter; this.listener = listener; this.maxRetries = conf.maxIORetries(); - this.retryWaitTime = conf.ioRetryWaitTime(); + this.retryWaitTime = conf.ioRetryWaitTimeMs(); this.outstandingBlocksIds = Sets.newLinkedHashSet(); Collections.addAll(outstandingBlocksIds, blockIds); this.currentListener = new RetryingBlockFetchListener(); From f79c1cfc997c1a7ddee480ca3d46f5341b69d3b7 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 9 Dec 2014 23:47:05 -0800 Subject: [PATCH 025/155] [Minor] Use tag for help icon in web UI page header This small commit makes the `(?)` web UI help link into a superscript, which should address feedback that the current design makes it look like an error occurred or like information is missing. Before: ![image](https://cloud.githubusercontent.com/assets/50748/5370611/a3ed0034-7fd9-11e4-870f-05bd9faad5b9.png) After: ![image](https://cloud.githubusercontent.com/assets/50748/5370602/6c5ca8d6-7fd9-11e4-8d1a-568d71290aa7.png) Author: Josh Rosen Closes #3659 from JoshRosen/webui-help-sup and squashes the following commits: bd72899 [Josh Rosen] Use tag for help icon in web UI page header. --- core/src/main/scala/org/apache/spark/ui/UIUtils.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 315327c3c6b7c..d970fa30c1c35 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -181,7 +181,9 @@ private[spark] object UIUtils extends Logging {
  • } val helpButton: Seq[Node] = helpText.map { helpText => - (?) + + (?) + }.getOrElse(Seq.empty) From 94b377f94487109a1cc3e07dd230b1df7a96e28d Mon Sep 17 00:00:00 2001 From: Nathan Kronenfeld Date: Tue, 9 Dec 2014 23:53:17 -0800 Subject: [PATCH 026/155] [SPARK-4772] Clear local copies of accumulators as soon as we're done with them Accumulators keep thread-local copies of themselves. These copies were only cleared at the beginning of a task. This meant that (a) the memory they used was tied up until the next task ran on that thread, and (b) if a thread died, the memory it had used for accumulators was locked up forever on that worker. This PR clears the thread-local copies of accumulators at the end of each task, in the tasks finally block, to make sure they are cleaned up between tasks. It also stores them in a ThreadLocal object, so that if, for some reason, the thread dies, any memory they are using at the time should be freed up. Author: Nathan Kronenfeld Closes #3570 from nkronenfeld/Accumulator-Improvements and squashes the following commits: a581f3f [Nathan Kronenfeld] Change Accumulators to private[spark] instead of adding mima exclude to get around false positive in mima tests b6c2180 [Nathan Kronenfeld] Include MiMa exclude as per build error instructions - this version incompatibility should be irrelevent, as it will only surface if a master is talking to a worker running a different version of spark. 537baad [Nathan Kronenfeld] Fuller refactoring as intended, incorporating JR's suggestions for ThreadLocal localAccums, and keeping clear(), but also calling it in tasks' finally block, rather than just at the beginning of the task. 39a82f2 [Nathan Kronenfeld] Clear local copies of accumulators as soon as we're done with them --- .../main/scala/org/apache/spark/Accumulators.scala | 14 ++++++++------ .../scala/org/apache/spark/executor/Executor.scala | 3 ++- 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 000bbd6b532ad..5f31bfba3f8d6 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -19,6 +19,7 @@ package org.apache.spark import java.io.{ObjectInputStream, Serializable} import java.util.concurrent.atomic.AtomicLong +import java.lang.ThreadLocal import scala.collection.generic.Growable import scala.collection.mutable.Map @@ -278,10 +279,12 @@ object AccumulatorParam { // TODO: The multi-thread support in accumulators is kind of lame; check // if there's a more intuitive way of doing it right -private object Accumulators { +private[spark] object Accumulators { // TODO: Use soft references? => need to make readObject work properly then val originals = Map[Long, Accumulable[_, _]]() - val localAccums = Map[Thread, Map[Long, Accumulable[_, _]]]() + val localAccums = new ThreadLocal[Map[Long, Accumulable[_, _]]]() { + override protected def initialValue() = Map[Long, Accumulable[_, _]]() + } var lastId: Long = 0 def newId(): Long = synchronized { @@ -293,22 +296,21 @@ private object Accumulators { if (original) { originals(a.id) = a } else { - val accums = localAccums.getOrElseUpdate(Thread.currentThread, Map()) - accums(a.id) = a + localAccums.get()(a.id) = a } } // Clear the local (non-original) accumulators for the current thread def clear() { synchronized { - localAccums.remove(Thread.currentThread) + localAccums.get.clear } } // Get the values of the local accumulators for the current thread (by ID) def values: Map[Long, Any] = synchronized { val ret = Map[Long, Any]() - for ((id, accum) <- localAccums.getOrElse(Thread.currentThread, Map())) { + for ((id, accum) <- localAccums.get) { ret(id) = accum.localValue } return ret diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 835157fc520aa..52de6980ecbf8 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -172,7 +172,6 @@ private[spark] class Executor( val startGCTime = gcTime try { - Accumulators.clear() val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(serializedTask) updateDependencies(taskFiles, taskJars) task = ser.deserialize[Task[Any]](taskBytes, Thread.currentThread.getContextClassLoader) @@ -278,6 +277,8 @@ private[spark] class Executor( env.shuffleMemoryManager.releaseMemoryForThisThread() // Release memory used by this thread for unrolling blocks env.blockManager.memoryStore.releaseUnrollMemoryForThisThread() + // Release memory used by this thread for accumulators + Accumulators.clear() runningTasks.remove(taskId) } } From 742e7093eca8865225c29bacf4344f2e89bfea41 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Wed, 10 Dec 2014 12:24:04 -0800 Subject: [PATCH 027/155] [SPARK-4161]Spark shell class path is not correctly set if "spark.driver.extraClassPath" is set in defaults.conf Author: GuoQiang Li Closes #3050 from witgo/SPARK-4161 and squashes the following commits: abb6fa4 [GuoQiang Li] move usejavacp opt to spark-shell 89e39e7 [GuoQiang Li] review commit c2a6f04 [GuoQiang Li] Spark shell class path is not correctly set if "spark.driver.extraClassPath" is set in defaults.conf --- bin/spark-shell | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/bin/spark-shell b/bin/spark-shell index 4a0670fc6c8aa..cca5aa0676123 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -45,6 +45,13 @@ source "$FWDIR"/bin/utils.sh SUBMIT_USAGE_FUNCTION=usage gatherSparkSubmitOpts "$@" +# SPARK-4161: scala does not assume use of the java classpath, +# so we need to add the "-Dscala.usejavacp=true" flag mnually. We +# do this specifically for the Spark shell because the scala REPL +# has its own class loader, and any additional classpath specified +# through spark.driver.extraClassPath is not automatically propagated. +SPARK_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Dscala.usejavacp=true" + function main() { if $cygwin; then # Workaround for issue involving JLine and Cygwin From 0fc637b4c27f9afdf5c829d26c7a86efd8681490 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 10 Dec 2014 12:29:00 -0800 Subject: [PATCH 028/155] [SPARK-4329][WebUI] HistoryPage pagenation Current HistoryPage have links only to previous page or next page. I suggest to add index to access history pages easily. I implemented like following pics. If there are many pages, current page +/- N pages, head page and last page are indexed. ![2014-11-10 16 13 25](https://cloud.githubusercontent.com/assets/4736016/4986246/9c7bbac4-6937-11e4-8695-8634d039d5b6.png) ![2014-11-10 16 03 21](https://cloud.githubusercontent.com/assets/4736016/4986210/3951bb74-6937-11e4-8b4e-9f90d266d736.png) ![2014-11-10 16 03 39](https://cloud.githubusercontent.com/assets/4736016/4986211/3b196ad8-6937-11e4-9f81-74bc0a6dad5b.png) ![2014-11-10 16 03 49](https://cloud.githubusercontent.com/assets/4736016/4986213/40686138-6937-11e4-86c0-41100f0404f6.png) ![2014-11-10 16 04 04](https://cloud.githubusercontent.com/assets/4736016/4986215/4326c9b4-6937-11e4-87ac-0f30c86ec6e3.png) Author: Kousuke Saruta Closes #3194 from sarutak/history-page-indexing and squashes the following commits: 15d3d2d [Kousuke Saruta] Simplified code c93932e [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into history-page-indexing 1c2f605 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into history-page-indexing 76b05e3 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into history-page-indexing b2240f8 [Kousuke Saruta] Fixed style ec7922e [Kousuke Saruta] Simplified code 755a004 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into history-page-indexing cfa242b [Kousuke Saruta] Added index to HistoryPage --- .../spark/deploy/history/HistoryPage.scala | 41 +++++++++++++++++-- 1 file changed, 37 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index 5fdc350cd8512..0d5dcfb1ddffe 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -26,6 +26,7 @@ import org.apache.spark.ui.{WebUIPage, UIUtils} private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { private val pageSize = 20 + private val plusOrMinus = 2 def render(request: HttpServletRequest): Seq[Node] = { val requestedPage = Option(request.getParameter("page")).getOrElse("1").toInt @@ -39,6 +40,9 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { val last = Math.min(actualFirst + pageSize, allApps.size) - 1 val pageCount = allApps.size / pageSize + (if (allApps.size % pageSize > 0) 1 else 0) + val secondPageFromLeft = 2 + val secondPageFromRight = pageCount - 1 + val appTable = UIUtils.listingTable(appHeader, appRow, apps) val providerConfig = parent.getProviderConfig() val content = @@ -48,13 +52,38 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { {providerConfig.map { case (k, v) =>
  • {k}: {v}
  • }} { + // This displays the indices of pages that are within `plusOrMinus` pages of + // the current page. Regardless of where the current page is, this also links + // to the first and last page. If the current page +/- `plusOrMinus` is greater + // than the 2nd page from the first page or less than the 2nd page from the last + // page, `...` will be displayed. if (allApps.size > 0) { + val leftSideIndices = + rangeIndices(actualPage - plusOrMinus until actualPage, 1 < _) + val rightSideIndices = + rangeIndices(actualPage + 1 to actualPage + plusOrMinus, _ < pageCount) +

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

    ++ appTable } else { @@ -81,6 +110,10 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { "Spark User", "Last Updated") + private def rangeIndices(range: Seq[Int], condition: Int => Boolean): Seq[Node] = { + range.filter(condition).map(nextPage => {nextPage} ) + } + private def appRow(info: ApplicationHistoryInfo): Seq[Node] = { val uiAddress = HistoryServer.UI_PATH_PREFIX + s"/${info.id}" val startTime = UIUtils.formatDate(info.startTime) From 56212831c6436e287a19908e82c26117cbcb16b0 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 10 Dec 2014 12:41:36 -0800 Subject: [PATCH 029/155] [SPARK-4771][Docs] Document standalone cluster supervise mode tdas looks like streaming already refers to the supervise mode. The link from there is broken though. Author: Andrew Or Closes #3627 from andrewor14/document-supervise and squashes the following commits: 9ca0908 [Andrew Or] Wording changes 2b55ed2 [Andrew Or] Document standalone cluster supervise mode --- docs/spark-standalone.md | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index ae7b81d5bb71f..5c6084fb46255 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -257,7 +257,7 @@ To run an interactive Spark shell against the cluster, run the following command You can also pass an option `--total-executor-cores ` to control the number of cores that spark-shell uses on the cluster. -# Launching Compiled Spark Applications +# Launching Spark Applications The [`spark-submit` script](submitting-applications.html) provides the most straightforward way to submit a compiled Spark application to the cluster. For standalone clusters, Spark currently @@ -272,6 +272,15 @@ should specify them through the `--jars` flag using comma as a delimiter (e.g. ` To control the application's configuration or execution environment, see [Spark Configuration](configuration.html). +Additionally, standalone `cluster` mode supports restarting your application automatically if it +exited with non-zero exit code. To use this feature, you may pass in the `--supervise` flag to +`spark-submit` when launching your application. Then, if you wish to kill an application that is +failing repeatedly, you may do so through: + + ./bin/spark-class org.apache.spark.deploy.Client kill + +You can find the driver ID through the standalone Master web UI at `http://:8080`. + # Resource Scheduling The standalone cluster mode currently only supports a simple FIFO scheduler across applications. From faa8fd8178642ef8fce14186abc45a189042efd4 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 10 Dec 2014 12:48:24 -0800 Subject: [PATCH 030/155] [SPARK-4215] Allow requesting / killing executors only in YARN mode Currently this doesn't do anything in other modes, so we might as well just disable it rather than having the user mistakenly rely on it. Author: Andrew Or Closes #3615 from andrewor14/dynamic-allocation-yarn-only and squashes the following commits: ce6487a [Andrew Or] Allow requesting / killing executors only in YARN mode --- .../src/main/scala/org/apache/spark/SparkContext.scala | 10 +++++++++- .../apache/spark/ExecutorAllocationManagerSuite.scala | 1 + 2 files changed, 10 insertions(+), 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 aded7c12e274e..8e5378ecc08de 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -357,8 +357,12 @@ class SparkContext(config: SparkConf) extends Logging { } // Optionally scale number of executors dynamically based on workload. Exposed for testing. + private val dynamicAllocationEnabled = conf.getBoolean("spark.dynamicAllocation.enabled", false) + private val dynamicAllocationTesting = conf.getBoolean("spark.dynamicAllocation.testing", false) private[spark] val executorAllocationManager: Option[ExecutorAllocationManager] = - if (conf.getBoolean("spark.dynamicAllocation.enabled", false)) { + if (dynamicAllocationEnabled) { + assert(master.contains("yarn") || dynamicAllocationTesting, + "Dynamic allocation of executors is currently only supported in YARN mode") Some(new ExecutorAllocationManager(this)) } else { None @@ -989,6 +993,8 @@ class SparkContext(config: SparkConf) extends Logging { */ @DeveloperApi def requestExecutors(numAdditionalExecutors: Int): Boolean = { + assert(master.contains("yarn") || dynamicAllocationTesting, + "Requesting executors is currently only supported in YARN mode") schedulerBackend match { case b: CoarseGrainedSchedulerBackend => b.requestExecutors(numAdditionalExecutors) @@ -1005,6 +1011,8 @@ class SparkContext(config: SparkConf) extends Logging { */ @DeveloperApi def killExecutors(executorIds: Seq[String]): Boolean = { + assert(master.contains("yarn") || dynamicAllocationTesting, + "Killing executors is currently only supported in YARN mode") schedulerBackend match { case b: CoarseGrainedSchedulerBackend => b.killExecutors(executorIds) diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index ce804f94f3267..c817f6dcede75 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -35,6 +35,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { .setMaster("local") .setAppName("test-executor-allocation-manager") .set("spark.dynamicAllocation.enabled", "true") + .set("spark.dynamicAllocation.testing", "true") intercept[SparkException] { new SparkContext(conf) } SparkEnv.get.stop() // cleanup the created environment SparkContext.clearActiveContext() From e230da18f8c354b4b80416aa5277420397acf4f2 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Wed, 10 Dec 2014 13:29:27 -0800 Subject: [PATCH 031/155] [SPARK-4793] [Deploy] ensure .jar at end of line sometimes I switch between different version and do not want to rebuild spark, so I rename assembly.jar into .jar.bak, but still caught by `compute-classpath.sh` Author: Daoyuan Wang Closes #3641 from adrian-wang/jar and squashes the following commits: 45cbfd0 [Daoyuan Wang] ensure .jar at end of line --- bin/compute-classpath.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 298641f2684de..685051eeed9f1 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -68,14 +68,14 @@ else assembly_folder="$ASSEMBLY_DIR" fi -num_jars="$(ls "$assembly_folder" | grep "spark-assembly.*hadoop.*\.jar" | wc -l)" +num_jars="$(ls "$assembly_folder" | grep "spark-assembly.*hadoop.*\.jar$" | wc -l)" if [ "$num_jars" -eq "0" ]; then echo "Failed to find Spark assembly in $assembly_folder" echo "You need to build Spark before running this program." exit 1 fi if [ "$num_jars" -gt "1" ]; then - jars_list=$(ls "$assembly_folder" | grep "spark-assembly.*hadoop.*.jar") + jars_list=$(ls "$assembly_folder" | grep "spark-assembly.*hadoop.*.jar$") echo "Found multiple Spark assembly jars in $assembly_folder:" echo "$jars_list" echo "Please remove all but one jar." @@ -108,7 +108,7 @@ else datanucleus_dir="$FWDIR"/lib_managed/jars fi -datanucleus_jars="$(find "$datanucleus_dir" 2>/dev/null | grep "datanucleus-.*\\.jar")" +datanucleus_jars="$(find "$datanucleus_dir" 2>/dev/null | grep "datanucleus-.*\\.jar$")" datanucleus_jars="$(echo "$datanucleus_jars" | tr "\n" : | sed s/:$//g)" if [ -n "$datanucleus_jars" ]; then From 447ae2de5d4c2af865fdb63f8b876b865de60f74 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 10 Dec 2014 14:19:37 -0800 Subject: [PATCH 032/155] [SPARK-4569] Rename 'externalSorting' in Aggregator Hi all - I've renamed the unhelpfully named variable and added a comment clarifying what's actually happening. Author: Ilya Ganelin Closes #3666 from ilganeli/SPARK-4569B and squashes the following commits: 1810394 [Ilya Ganelin] [SPARK-4569] Rename 'externalSorting' in Aggregator e2d2092 [Ilya Ganelin] [SPARK-4569] Rename 'externalSorting' in Aggregator d7cefec [Ilya Ganelin] [SPARK-4569] Rename 'externalSorting' in Aggregator 5b3f39c [Ilya Ganelin] [SPARK-4569] Rename in Aggregator --- core/src/main/scala/org/apache/spark/Aggregator.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index 79c9c451d273d..09eb9605fb799 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -34,7 +34,9 @@ case class Aggregator[K, V, C] ( mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) { - private val externalSorting = SparkEnv.get.conf.getBoolean("spark.shuffle.spill", true) + // When spilling is enabled sorting will happen externally, but not necessarily with an + // ExternalSorter. + private val isSpillEnabled = SparkEnv.get.conf.getBoolean("spark.shuffle.spill", true) @deprecated("use combineValuesByKey with TaskContext argument", "0.9.0") def combineValuesByKey(iter: Iterator[_ <: Product2[K, V]]): Iterator[(K, C)] = @@ -42,7 +44,7 @@ case class Aggregator[K, V, C] ( def combineValuesByKey(iter: Iterator[_ <: Product2[K, V]], context: TaskContext): Iterator[(K, C)] = { - if (!externalSorting) { + if (!isSpillEnabled) { val combiners = new AppendOnlyMap[K,C] var kv: Product2[K, V] = null val update = (hadValue: Boolean, oldValue: C) => { @@ -71,9 +73,9 @@ case class Aggregator[K, V, C] ( combineCombinersByKey(iter, null) def combineCombinersByKey(iter: Iterator[_ <: Product2[K, C]], context: TaskContext) - : Iterator[(K, C)] = + : Iterator[(K, C)] = { - if (!externalSorting) { + if (!isSpillEnabled) { val combiners = new AppendOnlyMap[K,C] var kc: Product2[K, C] = null val update = (hadValue: Boolean, oldValue: C) => { From 4f93d0cabe5d1fc7c0fd0a33d992fd85df1fecb4 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 10 Dec 2014 14:27:53 -0800 Subject: [PATCH 033/155] [SPARK-4759] Fix driver hanging from coalescing partitions The driver hangs sometimes when we coalesce RDD partitions. See JIRA for more details and reproduction. This is because our use of empty string as default preferred location in `CoalescedRDDPartition` causes the `TaskSetManager` to schedule the corresponding task on host `""` (empty string). The intended semantics here, however, is that the partition does not have a preferred location, and the TSM should schedule the corresponding task accordingly. Author: Andrew Or Closes #3633 from andrewor14/coalesce-preferred-loc and squashes the following commits: e520d6b [Andrew Or] Oops 3ebf8bd [Andrew Or] A few comments f370a4e [Andrew Or] Fix tests 2f7dfb6 [Andrew Or] Avoid using empty string as default preferred location --- .../org/apache/spark/rdd/CoalescedRDD.scala | 36 +++++++++++-------- .../scala/org/apache/spark/rdd/RDDSuite.scala | 2 +- 2 files changed, 22 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala index 9fab1d78abb04..b073eba8a1574 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -35,11 +35,10 @@ import org.apache.spark.util.Utils * @param preferredLocation the preferred location for this partition */ private[spark] case class CoalescedRDDPartition( - index: Int, - @transient rdd: RDD[_], - parentsIndices: Array[Int], - @transient preferredLocation: String = "" - ) extends Partition { + index: Int, + @transient rdd: RDD[_], + parentsIndices: Array[Int], + @transient preferredLocation: Option[String] = None) extends Partition { var parents: Seq[Partition] = parentsIndices.map(rdd.partitions(_)) @throws(classOf[IOException]) @@ -55,9 +54,10 @@ private[spark] case class CoalescedRDDPartition( * @return locality of this coalesced partition between 0 and 1 */ def localFraction: Double = { - val loc = parents.count(p => - rdd.context.getPreferredLocs(rdd, p.index).map(tl => tl.host).contains(preferredLocation)) - + val loc = parents.count { p => + val parentPreferredLocations = rdd.context.getPreferredLocs(rdd, p.index).map(_.host) + preferredLocation.exists(parentPreferredLocations.contains) + } if (parents.size == 0) 0.0 else (loc.toDouble / parents.size.toDouble) } } @@ -73,9 +73,9 @@ private[spark] case class CoalescedRDDPartition( * @param balanceSlack used to trade-off balance and locality. 1.0 is all locality, 0 is all balance */ private[spark] class CoalescedRDD[T: ClassTag]( - @transient var prev: RDD[T], - maxPartitions: Int, - balanceSlack: Double = 0.10) + @transient var prev: RDD[T], + maxPartitions: Int, + balanceSlack: Double = 0.10) extends RDD[T](prev.context, Nil) { // Nil since we implement getDependencies override def getPartitions: Array[Partition] = { @@ -113,7 +113,7 @@ private[spark] class CoalescedRDD[T: ClassTag]( * @return the machine most preferred by split */ override def getPreferredLocations(partition: Partition): Seq[String] = { - List(partition.asInstanceOf[CoalescedRDDPartition].preferredLocation) + partition.asInstanceOf[CoalescedRDDPartition].preferredLocation.toSeq } } @@ -147,7 +147,7 @@ private[spark] class CoalescedRDD[T: ClassTag]( * */ -private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) { +private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) { def compare(o1: PartitionGroup, o2: PartitionGroup): Boolean = o1.size < o2.size def compare(o1: Option[PartitionGroup], o2: Option[PartitionGroup]): Boolean = @@ -341,8 +341,14 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanc } } -private[spark] case class PartitionGroup(prefLoc: String = "") { +private case class PartitionGroup(prefLoc: Option[String] = None) { var arr = mutable.ArrayBuffer[Partition]() - def size = arr.size } + +private object PartitionGroup { + def apply(prefLoc: String): PartitionGroup = { + require(prefLoc != "", "Preferred location must not be empty") + PartitionGroup(Some(prefLoc)) + } +} diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 46fcb80fa1845..6836e9ab0fd6b 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -294,7 +294,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("coalesced RDDs with locality") { val data3 = sc.makeRDD(List((1,List("a","c")), (2,List("a","b","c")), (3,List("b")))) val coal3 = data3.coalesce(3) - val list3 = coal3.partitions.map(p => p.asInstanceOf[CoalescedRDDPartition].preferredLocation) + val list3 = coal3.partitions.flatMap(_.asInstanceOf[CoalescedRDDPartition].preferredLocation) assert(list3.sorted === Array("a","b","c"), "Locality preferences are dropped") // RDD with locality preferences spread (non-randomly) over 6 machines, m0 through m5 From 36bdb5b748ff670a9bafd787e40c9e142c9a85b9 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 10 Dec 2014 14:41:16 -0800 Subject: [PATCH 034/155] MAINTENANCE: Automated closing of pull requests. This commit exists to close the following pull requests on Github: Closes #2883 (close requested by 'pwendell') Closes #3364 (close requested by 'pwendell') Closes #4458 (close requested by 'pwendell') Closes #1574 (close requested by 'andrewor14') Closes #2546 (close requested by 'andrewor14') Closes #2516 (close requested by 'andrewor14') Closes #154 (close requested by 'andrewor14') From 652b781a9b543cb17d7da91f5c3bebe5a02e0478 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Wed, 10 Dec 2014 15:01:15 -0800 Subject: [PATCH 035/155] SPARK-3526 Add section about data locality to the tuning guide cc kayousterhout I have a few outstanding questions from compiling this documentation: - What's the difference between NO_PREF and ANY? I understand the implications of the ordering but don't know what an example of each would be - Why is NO_PREF ahead of RACK_LOCAL? I would think it'd be better to schedule rack-local tasks ahead of no preference if you could only do one or the other. Is the idea to wait longer and hope for the rack-local tasks to turn into node-local or better? - Will there be a datacenter-local locality level in the future? Apache Cassandra for example has this level Author: Andrew Ash Closes #2519 from ash211/SPARK-3526 and squashes the following commits: 44cff28 [Andrew Ash] Link to spark.locality parameters rather than copying the list 6d5d966 [Andrew Ash] Stay focused on Spark, no astronaut architecture mumbo-jumbo 20e0e31 [Andrew Ash] SPARK-3526 Add section about data locality to the tuning guide --- docs/tuning.md | 33 +++++++++++++++++++++++++++++++++ 1 file changed, 33 insertions(+) diff --git a/docs/tuning.md b/docs/tuning.md index 0e2447dd46394..c4ca766328c1e 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -233,6 +233,39 @@ Spark prints the serialized size of each task on the master, so you can look at decide whether your tasks are too large; in general tasks larger than about 20 KB are probably worth optimizing. +## Data Locality + +Data locality can have a major impact on the performance of Spark jobs. If data and the code that +operates on it are together than computation tends to be fast. But if code and data are separated, +one must move to the other. Typically it is faster to ship serialized code from place to place than +a chunk of data because code size is much smaller than data. Spark builds its scheduling around +this general principle of data locality. + +Data locality is how close data is to the code processing it. There are several levels of +locality based on the data's current location. In order from closest to farthest: + +- `PROCESS_LOCAL` data is in the same JVM as the running code. This is the best locality + possible +- `NODE_LOCAL` data is on the same node. Examples might be in HDFS on the same node, or in + another executor on the same node. This is a little slower than `PROCESS_LOCAL` because the data + has to travel between processes +- `NO_PREF` data is accessed equally quickly from anywhere and has no locality preference +- `RACK_LOCAL` data is on the same rack of servers. Data is on a different server on the same rack + so needs to be sent over the network, typically through a single switch +- `ANY` data is elsewhere on the network and not in the same rack + +Spark prefers to schedule all tasks at the best locality level, but this is not always possible. In +situations where there is no unprocessed data on any idle executor, Spark switches to lower locality +levels. There are two options: a) wait until a busy CPU frees up to start a task on data on the same +server, or b) immediately start a new task in a farther away place that requires moving data there. + +What Spark typically does is wait a bit in the hopes that a busy CPU frees up. Once that timeout +expires, it starts moving the data from far away to the free CPU. The wait timeout for fallback +between each level can be configured individually or all together in one parameter; see the +`spark.locality` parameters on the [configuration page](configuration.html#scheduling) for details. +You should increase these settings if your tasks are long and see poor locality, but the default +usually works well. + # Summary This has been a short guide to point out the main concerns you should know about when tuning a From 57d37f9c7193f1cb04a22d7f512178696b655b71 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Wed, 10 Dec 2014 20:44:59 -0800 Subject: [PATCH 036/155] [CORE]codeStyle: uniform ConcurrentHashMap define in StorageLevel.scala with other places Author: Zhang, Liye Closes #2793 from liyezhang556520/uniformHashMap and squashes the following commits: 5884735 [Zhang, Liye] [CORE]codeStyle: uniform ConcurrentHashMap define in StorageLevel.scala --- .../main/scala/org/apache/spark/storage/StorageLevel.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index 56edc4fe2e4ad..e5e1cf5a69a19 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -18,6 +18,7 @@ package org.apache.spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} +import java.util.concurrent.ConcurrentHashMap import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.Utils @@ -220,8 +221,7 @@ object StorageLevel { getCachedStorageLevel(obj) } - private[spark] val storageLevelCache = - new java.util.concurrent.ConcurrentHashMap[StorageLevel, StorageLevel]() + private[spark] val storageLevelCache = new ConcurrentHashMap[StorageLevel, StorageLevel]() private[spark] def getCachedStorageLevel(level: StorageLevel): StorageLevel = { storageLevelCache.putIfAbsent(level, level) From 2a5b5fd4ccf28fab5b7e32a54170be92d5d23ba6 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 10 Dec 2014 23:41:15 -0800 Subject: [PATCH 037/155] [SPARK-4791] [sql] Infer schema from case class with multiple constructors Modified ScalaReflection.schemaFor to take primary constructor of Product when there are multiple constructors. Added test to suite which failed before but works now. Needed for [https://github.com/apache/spark/pull/3637] CC: marmbrus Author: Joseph K. Bradley Closes #3646 from jkbradley/sql-reflection and squashes the following commits: 796b2e4 [Joseph K. Bradley] Modified ScalaReflection.schemaFor to take primary constructor of Product when there are multiple constructors. Added test to suite which failed before but works now. --- .../spark/sql/catalyst/ScalaReflection.scala | 14 +++++++++++++- .../spark/sql/catalyst/ScalaReflectionSuite.scala | 14 ++++++++++++++ 2 files changed, 27 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 71034c2c43c77..2cf241de61f7a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -118,7 +118,19 @@ trait ScalaReflection { case t if t <:< typeOf[Product] => val formalTypeArgs = t.typeSymbol.asClass.typeParams val TypeRef(_, _, actualTypeArgs) = t - val params = t.member(nme.CONSTRUCTOR).asMethod.paramss + val constructorSymbol = t.member(nme.CONSTRUCTOR) + val params = if (constructorSymbol.isMethod) { + constructorSymbol.asMethod.paramss + } else { + // Find the primary constructor, and use its parameter ordering. + val primaryConstructorSymbol: Option[Symbol] = constructorSymbol.asTerm.alternatives.find( + s => s.isMethod && s.asMethod.isPrimaryConstructor) + if (primaryConstructorSymbol.isEmpty) { + sys.error("Internal SQL error: Product object did not have a primary constructor.") + } else { + primaryConstructorSymbol.get.asMethod.paramss + } + } Schema(StructType( params.head.map { p => val Schema(dataType, nullable) = diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala index ddc3d44869c98..7be24bea7d5a6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala @@ -68,6 +68,10 @@ case class ComplexData( case class GenericData[A]( genericField: A) +case class MultipleConstructorsData(a: Int, b: String, c: Double) { + def this(b: String, a: Int) = this(a, b, c = 1.0) +} + class ScalaReflectionSuite extends FunSuite { import ScalaReflection._ @@ -253,4 +257,14 @@ class ScalaReflectionSuite extends FunSuite { Row(1, 1, 1, 1, 1, 1, true)) assert(convertToCatalyst(data, dataType) === convertedData) } + + test("infer schema from case class with multiple constructors") { + val dataType = schemaFor[MultipleConstructorsData].dataType + dataType match { + case s: StructType => + // Schema should have order: a: Int, b: String, c: Double + assert(s.fieldNames === Seq("a", "b", "c")) + assert(s.fields.map(_.dataType) === Seq(IntegerType, StringType, DoubleType)) + } + } } From b004150adb503ddbb54d5cd544e39ad974497c41 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 11 Dec 2014 06:21:23 -0800 Subject: [PATCH 038/155] [SPARK-4806] Streaming doc update for 1.2 Important updates to the streaming programming guide - Make the fault-tolerance properties easier to understand, with information about write ahead logs - Update the information about deploying the spark streaming app with information about Driver HA - Update Receiver guide to discuss reliable vs unreliable receivers. Author: Tathagata Das Author: Josh Rosen Author: Josh Rosen Closes #3653 from tdas/streaming-doc-update-1.2 and squashes the following commits: f53154a [Tathagata Das] Addressed Josh's comments. ce299e4 [Tathagata Das] Minor update. ca19078 [Tathagata Das] Minor change f746951 [Tathagata Das] Mentioned performance problem with WAL 7787209 [Tathagata Das] Merge branch 'streaming-doc-update-1.2' of github.com:tdas/spark into streaming-doc-update-1.2 2184729 [Tathagata Das] Updated Kafka and Flume guides with reliability information. 2f3178c [Tathagata Das] Added more information about writing reliable receivers in the custom receiver guide. 91aa5aa [Tathagata Das] Improved API Docs menu 5707581 [Tathagata Das] Added Pythn API badge b9c8c24 [Tathagata Das] Merge pull request #26 from JoshRosen/streaming-programming-guide b8c8382 [Josh Rosen] minor fixes a4ef126 [Josh Rosen] Restructure parts of the fault-tolerance section to read a bit nicer when skipping over the headings 65f66cd [Josh Rosen] Fix broken link to fault-tolerance semantics section. f015397 [Josh Rosen] Minor grammar / pluralization fixes. 3019f3a [Josh Rosen] Fix minor Markdown formatting issues aa8bb87 [Tathagata Das] Small update. 195852c [Tathagata Das] Updated based on Josh's comments, updated receiver reliability and deploying section, and also updated configuration. 17b99fb [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-doc-update-1.2 a0217c0 [Tathagata Das] Changed Deploying menu layout 67fcffc [Tathagata Das] Added cluster mode + supervise example to submitting application guide. e45453b [Tathagata Das] Update streaming guide, added deploying section. 192c7a7 [Tathagata Das] Added more info about Python API, and rewrote the checkpointing section. --- docs/_layouts/global.html | 13 +- docs/configuration.md | 133 ++-- docs/streaming-custom-receivers.md | 90 ++- docs/streaming-flume-integration.md | 13 +- docs/streaming-kafka-integration.md | 17 + docs/streaming-programming-guide.md | 1068 +++++++++++++++------------ docs/submitting-applications.md | 36 +- 7 files changed, 819 insertions(+), 551 deletions(-) diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 627ed37de4a9c..8841f7675d35e 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -33,7 +33,7 @@