From 5168c6ca9f0008027d688661bae57c28cf386b54 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Sun, 16 Nov 2014 21:31:51 -0800 Subject: [PATCH 001/109] [SPARK-4422][MLLIB]In some cases, Vectors.fromBreeze get wrong results. cc mengxr Author: GuoQiang Li Closes #3281 from witgo/SPARK-4422 and squashes the following commits: 5f1fa5e [GuoQiang Li] import order 50783bd [GuoQiang Li] review commits 7a10123 [GuoQiang Li] In some cases, Vectors.fromBreeze get wrong results. --- .../main/scala/org/apache/spark/mllib/linalg/Vectors.scala | 2 +- .../scala/org/apache/spark/mllib/linalg/VectorsSuite.scala | 7 +++++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 9fccd6341ba7d..60ab2aaa8f27a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -237,7 +237,7 @@ object Vectors { private[mllib] def fromBreeze(breezeVector: BV[Double]): Vector = { breezeVector match { case v: BDV[Double] => - if (v.offset == 0 && v.stride == 1) { + if (v.offset == 0 && v.stride == 1 && v.length == v.data.length) { new DenseVector(v.data) } else { new DenseVector(v.toArray) // Can't use underlying array directly, so make a new one diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala index 93a84fe07b32a..59cd85eab27d0 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.mllib.linalg +import breeze.linalg.{DenseMatrix => BDM} import org.scalatest.FunSuite import org.apache.spark.SparkException @@ -166,4 +167,10 @@ class VectorsSuite extends FunSuite { assert(v === udt.deserialize(udt.serialize(v))) } } + + test("fromBreeze") { + val x = BDM.zeros[Double](10, 10) + val v = Vectors.fromBreeze(x(::, 0)) + assert(v.size === x.rows) + } } From 64c6b9bad559c21f25cd9fbe37c8813cdab939f2 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 16 Nov 2014 21:55:57 -0800 Subject: [PATCH 002/109] [SPARK-4410][SQL] Add support for external sort Adds a new operator that uses Spark's `ExternalSort` class. It is off by default now, but we might consider making it the default if benchmarks show that it does not regress performance. Author: Michael Armbrust Closes #3268 from marmbrus/externalSort and squashes the following commits: 48b9726 [Michael Armbrust] comments b98799d [Michael Armbrust] Add test afd7562 [Michael Armbrust] Add support for external sort. --- .../scala/org/apache/spark/sql/SQLConf.scala | 7 ++++ .../spark/sql/execution/SparkStrategies.scala | 5 ++- .../spark/sql/execution/basicOperators.scala | 37 +++++++++++++++++-- .../org/apache/spark/sql/SQLQuerySuite.scala | 16 +++++++- 4 files changed, 59 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index cd7d78e684791..9697beb132fbb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -39,6 +39,10 @@ private[spark] object SQLConf { val COLUMN_NAME_OF_CORRUPT_RECORD = "spark.sql.columnNameOfCorruptRecord" + // Options that control which operators can be chosen by the query planner. These should be + // considered hints and may be ignored by future versions of Spark SQL. + val EXTERNAL_SORT = "spark.sql.planner.externalSort" + // This is only used for the thriftserver val THRIFTSERVER_POOL = "spark.sql.thriftserver.scheduler.pool" @@ -96,6 +100,9 @@ private[sql] trait SQLConf { private[spark] def parquetFilterPushDown = getConf(PARQUET_FILTER_PUSHDOWN_ENABLED, "false").toBoolean + /** When true the planner will use the external sort, which may spill to disk. */ + private[spark] def externalSortEnabled: Boolean = getConf(EXTERNAL_SORT, "false").toBoolean + /** * When set to true, Spark SQL will use the Scala compiler at runtime to generate custom bytecode * that evaluates expressions found in queries. In general this custom code runs much faster diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 03cd5bd6272bb..7ef1f9f2c5c02 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -263,9 +263,12 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.Distinct(child) => execution.Distinct(partial = false, execution.Distinct(partial = true, planLater(child))) :: Nil + + case logical.Sort(sortExprs, child) if sqlContext.externalSortEnabled => + execution.ExternalSort(sortExprs, global = true, planLater(child)):: Nil case logical.Sort(sortExprs, child) => - // This sort is a global sort. Its requiredDistribution will be an OrderedDistribution. execution.Sort(sortExprs, global = true, planLater(child)):: Nil + case logical.SortPartitions(sortExprs, child) => // This sort only sorts tuples within a partition. Its requiredDistribution will be // an UnspecifiedDistribution. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 1b8ba3ace2a82..e53723c176569 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, OrderedDistribution, SinglePartition, UnspecifiedDistribution} import org.apache.spark.util.MutablePair +import org.apache.spark.util.collection.ExternalSorter /** * :: DeveloperApi :: @@ -189,6 +190,9 @@ case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) /** * :: DeveloperApi :: + * Performs a sort on-heap. + * @param global when true performs a global sort of all partitions by shuffling the data first + * if necessary. */ @DeveloperApi case class Sort( @@ -199,12 +203,37 @@ case class Sort( override def requiredChildDistribution = if (global) OrderedDistribution(sortOrder) :: Nil else UnspecifiedDistribution :: Nil + override def execute() = attachTree(this, "sort") { + child.execute().mapPartitions( { iterator => + val ordering = newOrdering(sortOrder, child.output) + iterator.map(_.copy()).toArray.sorted(ordering).iterator + }, preservesPartitioning = true) + } + + override def output = child.output +} + +/** + * :: DeveloperApi :: + * Performs a sort, spilling to disk as needed. + * @param global when true performs a global sort of all partitions by shuffling the data first + * if necessary. + */ +@DeveloperApi +case class ExternalSort( + sortOrder: Seq[SortOrder], + global: Boolean, + child: SparkPlan) + extends UnaryNode { + override def requiredChildDistribution = + if (global) OrderedDistribution(sortOrder) :: Nil else UnspecifiedDistribution :: Nil override def execute() = attachTree(this, "sort") { - child.execute() - .mapPartitions( { iterator => - val ordering = newOrdering(sortOrder, child.output) - iterator.map(_.copy()).toArray.sorted(ordering).iterator + child.execute().mapPartitions( { iterator => + val ordering = newOrdering(sortOrder, child.output) + val sorter = new ExternalSorter[Row, Null, Row](ordering = Some(ordering)) + sorter.insertAll(iterator.map(r => (r, null))) + sorter.iterator.map(_._1) }, preservesPartitioning = true) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index ce5672c08653a..a63515464c688 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -196,7 +196,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { Seq(Seq("1"))) } - test("sorting") { + def sortTest() = { checkAnswer( sql("SELECT * FROM testData2 ORDER BY a ASC, b ASC"), Seq((1,1), (1,2), (2,1), (2,2), (3,1), (3,2))) @@ -238,6 +238,20 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { mapData.collect().sortBy(_.data(1)).reverse.toSeq) } + test("sorting") { + val before = externalSortEnabled + setConf(SQLConf.EXTERNAL_SORT, "false") + sortTest() + setConf(SQLConf.EXTERNAL_SORT, before.toString) + } + + test("external sorting") { + val before = externalSortEnabled + setConf(SQLConf.EXTERNAL_SORT, "true") + sortTest() + setConf(SQLConf.EXTERNAL_SORT, before.toString) + } + test("limit") { checkAnswer( sql("SELECT * FROM testData LIMIT 10"), From 5c92d47ad2e3414f2ae089cb47f3c6daccba8d90 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 17 Nov 2014 10:40:33 -0800 Subject: [PATCH 003/109] SPARK-4445, Don't display storage level in toDebugString unless RDD is persisted. Author: Prashant Sharma Closes #3310 from ScrapCodes/SPARK-4445/rddDebugStringFix and squashes the following commits: 4e57c52 [Prashant Sharma] SPARK-4445, Don't display storage level in toDebugString unless RDD is persisted --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index cb64d43c6c54a..e4025bcf48db6 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1309,7 +1309,7 @@ abstract class RDD[T: ClassTag]( def debugSelf (rdd: RDD[_]): Seq[String] = { import Utils.bytesToString - val persistence = storageLevel.description + val persistence = if (storageLevel != StorageLevel.NONE) storageLevel.description else "" val storageInfo = rdd.context.getRDDStorageInfo.filter(_.id == rdd.id).map(info => " CachedPartitions: %d; MemorySize: %s; TachyonSize: %s; DiskSize: %s".format( info.numCachedPartitions, bytesToString(info.memSize), From e7690ed20a2734b7ca88e78a60a8e75ba19e9d8b Mon Sep 17 00:00:00 2001 From: Adam Pingel Date: Mon, 17 Nov 2014 10:47:29 -0800 Subject: [PATCH 004/109] SPARK-2811 upgrade algebird to 0.8.1 Author: Adam Pingel Closes #3282 from adampingel/master and squashes the following commits: 70c8d3c [Adam Pingel] relocate the algebird example back to example/src 7a9d8be [Adam Pingel] SPARK-2811 upgrade algebird to 0.8.1 --- examples/pom.xml | 14 +++++++------- .../examples/streaming/TwitterAlgebirdCMS.scala | 0 .../examples/streaming/TwitterAlgebirdHLL.scala | 0 3 files changed, 7 insertions(+), 7 deletions(-) rename examples/{scala-2.10 => }/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala (100%) rename examples/{scala-2.10 => }/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala (100%) diff --git a/examples/pom.xml b/examples/pom.xml index 2752ce3ca9821..85e133779e465 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -217,6 +217,11 @@ org.apache.commons commons-math3 + + com.twitter + algebird-core_${scala.binary.version} + 0.8.1 + org.scalatest scalatest_${scala.binary.version} @@ -389,8 +394,8 @@ - + scala-2.10 !scala-2.11 @@ -401,11 +406,6 @@ spark-streaming-kafka_${scala.binary.version} ${project.version} - - com.twitter - algebird-core_${scala.binary.version} - 0.1.11 - diff --git a/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala similarity index 100% rename from examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala diff --git a/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala similarity index 100% rename from examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala From 9ac2bb18ede2e9f73c255fa33445af89aaf8a000 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Mon, 17 Nov 2014 11:06:31 -0800 Subject: [PATCH 005/109] [SPARK-4444] Drop VD type parameter from EdgeRDD Due to vertex attribute caching, EdgeRDD previously took two type parameters: ED and VD. However, this is an implementation detail that should not be exposed in the interface, so this PR drops the VD type parameter. This requires removing the `filter` method from the EdgeRDD interface, because it depends on vertex attribute caching. Author: Ankur Dave Closes #3303 from ankurdave/edgerdd-drop-tparam and squashes the following commits: 38dca9b [Ankur Dave] Leave EdgeRDD.fromEdges public fafeb51 [Ankur Dave] Drop VD type parameter from EdgeRDD --- .../org/apache/spark/graphx/EdgeRDD.scala | 35 +++++++------------ .../scala/org/apache/spark/graphx/Graph.scala | 2 +- .../org/apache/spark/graphx/VertexRDD.scala | 10 +++--- .../spark/graphx/impl/EdgeRDDImpl.scala | 24 ++++++------- .../apache/spark/graphx/impl/GraphImpl.scala | 13 +++---- .../graphx/impl/ReplicatedVertexView.scala | 4 +-- .../spark/graphx/impl/VertexRDDImpl.scala | 2 +- 7 files changed, 40 insertions(+), 50 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala index 869ef15893eb9..cc70b396a8dd4 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala @@ -17,6 +17,7 @@ package org.apache.spark.graphx +import scala.language.existentials import scala.reflect.ClassTag import org.apache.spark.Dependency @@ -36,16 +37,16 @@ import org.apache.spark.graphx.impl.EdgeRDDImpl * edge to provide the triplet view. Shipping of the vertex attributes is managed by * `impl.ReplicatedVertexView`. */ -abstract class EdgeRDD[ED, VD]( +abstract class EdgeRDD[ED]( @transient sc: SparkContext, @transient deps: Seq[Dependency[_]]) extends RDD[Edge[ED]](sc, deps) { - private[graphx] def partitionsRDD: RDD[(PartitionID, EdgePartition[ED, VD])] + private[graphx] def partitionsRDD: RDD[(PartitionID, EdgePartition[ED, VD])] forSome { type VD } override protected def getPartitions: Array[Partition] = partitionsRDD.partitions override def compute(part: Partition, context: TaskContext): Iterator[Edge[ED]] = { - val p = firstParent[(PartitionID, EdgePartition[ED, VD])].iterator(part, context) + val p = firstParent[(PartitionID, EdgePartition[ED, _])].iterator(part, context) if (p.hasNext) { p.next._2.iterator.map(_.copy()) } else { @@ -60,19 +61,14 @@ abstract class EdgeRDD[ED, VD]( * @param f the function from an edge to a new edge value * @return a new EdgeRDD containing the new edge values */ - def mapValues[ED2: ClassTag](f: Edge[ED] => ED2): EdgeRDD[ED2, VD] + def mapValues[ED2: ClassTag](f: Edge[ED] => ED2): EdgeRDD[ED2] /** * Reverse all the edges in this RDD. * * @return a new EdgeRDD containing all the edges reversed */ - def reverse: EdgeRDD[ED, VD] - - /** Removes all edges but those matching `epred` and where both vertices match `vpred`. */ - def filter( - epred: EdgeTriplet[VD, ED] => Boolean, - vpred: (VertexId, VD) => Boolean): EdgeRDD[ED, VD] + def reverse: EdgeRDD[ED] /** * Inner joins this EdgeRDD with another EdgeRDD, assuming both are partitioned using the same @@ -84,15 +80,8 @@ abstract class EdgeRDD[ED, VD]( * with values supplied by `f` */ def innerJoin[ED2: ClassTag, ED3: ClassTag] - (other: EdgeRDD[ED2, _]) - (f: (VertexId, VertexId, ED, ED2) => ED3): EdgeRDD[ED3, VD] - - private[graphx] def mapEdgePartitions[ED2: ClassTag, VD2: ClassTag]( - f: (PartitionID, EdgePartition[ED, VD]) => EdgePartition[ED2, VD2]): EdgeRDD[ED2, VD2] - - /** Replaces the edge partitions while preserving all other properties of the EdgeRDD. */ - private[graphx] def withPartitionsRDD[ED2: ClassTag, VD2: ClassTag]( - partitionsRDD: RDD[(PartitionID, EdgePartition[ED2, VD2])]): EdgeRDD[ED2, VD2] + (other: EdgeRDD[ED2]) + (f: (VertexId, VertexId, ED, ED2) => ED3): EdgeRDD[ED3] /** * Changes the target storage level while preserving all other properties of the @@ -101,7 +90,7 @@ abstract class EdgeRDD[ED, VD]( * This does not actually trigger a cache; to do this, call * [[org.apache.spark.graphx.EdgeRDD#cache]] on the returned EdgeRDD. */ - private[graphx] def withTargetStorageLevel(targetStorageLevel: StorageLevel): EdgeRDD[ED, VD] + private[graphx] def withTargetStorageLevel(targetStorageLevel: StorageLevel): EdgeRDD[ED] } object EdgeRDD { @@ -111,7 +100,7 @@ object EdgeRDD { * @tparam ED the edge attribute type * @tparam VD the type of the vertex attributes that may be joined with the returned EdgeRDD */ - def fromEdges[ED: ClassTag, VD: ClassTag](edges: RDD[Edge[ED]]): EdgeRDD[ED, VD] = { + def fromEdges[ED: ClassTag, VD: ClassTag](edges: RDD[Edge[ED]]): EdgeRDDImpl[ED, VD] = { val edgePartitions = edges.mapPartitionsWithIndex { (pid, iter) => val builder = new EdgePartitionBuilder[ED, VD] iter.foreach { e => @@ -128,8 +117,8 @@ object EdgeRDD { * @tparam ED the edge attribute type * @tparam VD the type of the vertex attributes that may be joined with the returned EdgeRDD */ - def fromEdgePartitions[ED: ClassTag, VD: ClassTag]( - edgePartitions: RDD[(Int, EdgePartition[ED, VD])]): EdgeRDD[ED, VD] = { + private[graphx] def fromEdgePartitions[ED: ClassTag, VD: ClassTag]( + edgePartitions: RDD[(Int, EdgePartition[ED, VD])]): EdgeRDDImpl[ED, VD] = { new EdgeRDDImpl(edgePartitions) } } 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 2c1b9518a3d16..637791543514c 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -59,7 +59,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab * along with their vertex data. * */ - @transient val edges: EdgeRDD[ED, VD] + @transient val edges: EdgeRDD[ED] /** * An RDD containing the edge triplets, which are edges along with the vertex data associated with diff --git a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala index f8be17669d892..1db3df03c8052 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -207,7 +207,7 @@ abstract class VertexRDD[VD]( def reverseRoutingTables(): VertexRDD[VD] /** Prepares this VertexRDD for efficient joins with the given EdgeRDD. */ - def withEdges(edges: EdgeRDD[_, _]): VertexRDD[VD] + def withEdges(edges: EdgeRDD[_]): VertexRDD[VD] /** Replaces the vertex partitions while preserving all other properties of the VertexRDD. */ private[graphx] def withPartitionsRDD[VD2: ClassTag]( @@ -269,7 +269,7 @@ object VertexRDD { * @param defaultVal the vertex attribute to use when creating missing vertices */ def apply[VD: ClassTag]( - vertices: RDD[(VertexId, VD)], edges: EdgeRDD[_, _], defaultVal: VD): VertexRDD[VD] = { + vertices: RDD[(VertexId, VD)], edges: EdgeRDD[_], defaultVal: VD): VertexRDD[VD] = { VertexRDD(vertices, edges, defaultVal, (a, b) => a) } @@ -286,7 +286,7 @@ object VertexRDD { * @param mergeFunc the commutative, associative duplicate vertex attribute merge function */ def apply[VD: ClassTag]( - vertices: RDD[(VertexId, VD)], edges: EdgeRDD[_, _], defaultVal: VD, mergeFunc: (VD, VD) => VD + vertices: RDD[(VertexId, VD)], edges: EdgeRDD[_], defaultVal: VD, mergeFunc: (VD, VD) => VD ): VertexRDD[VD] = { val vPartitioned: RDD[(VertexId, VD)] = vertices.partitioner match { case Some(p) => vertices @@ -314,7 +314,7 @@ object VertexRDD { * @param defaultVal the vertex attribute to use when creating missing vertices */ def fromEdges[VD: ClassTag]( - edges: EdgeRDD[_, _], numPartitions: Int, defaultVal: VD): VertexRDD[VD] = { + edges: EdgeRDD[_], numPartitions: Int, defaultVal: VD): VertexRDD[VD] = { val routingTables = createRoutingTables(edges, new HashPartitioner(numPartitions)) val vertexPartitions = routingTables.mapPartitions({ routingTableIter => val routingTable = @@ -325,7 +325,7 @@ object VertexRDD { } private[graphx] def createRoutingTables( - edges: EdgeRDD[_, _], vertexPartitioner: Partitioner): RDD[RoutingTablePartition] = { + edges: EdgeRDD[_], vertexPartitioner: Partitioner): RDD[RoutingTablePartition] = { // Determine which vertices each edge partition needs by creating a mapping from vid to pid. val vid2pid = edges.partitionsRDD.mapPartitions(_.flatMap( Function.tupled(RoutingTablePartition.edgePartitionToMsgs))) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala index 4100a85d17ee3..a8169613b4fd2 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala @@ -28,7 +28,7 @@ import org.apache.spark.graphx._ class EdgeRDDImpl[ED: ClassTag, VD: ClassTag] private[graphx] ( override val partitionsRDD: RDD[(PartitionID, EdgePartition[ED, VD])], val targetStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY) - extends EdgeRDD[ED, VD](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { + extends EdgeRDD[ED](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { override def setName(_name: String): this.type = { if (partitionsRDD.name != null) { @@ -75,20 +75,20 @@ class EdgeRDDImpl[ED: ClassTag, VD: ClassTag] private[graphx] ( partitionsRDD.map(_._2.size.toLong).reduce(_ + _) } - override def mapValues[ED2: ClassTag](f: Edge[ED] => ED2): EdgeRDD[ED2, VD] = + override def mapValues[ED2: ClassTag](f: Edge[ED] => ED2): EdgeRDDImpl[ED2, VD] = mapEdgePartitions((pid, part) => part.map(f)) - override def reverse: EdgeRDD[ED, VD] = mapEdgePartitions((pid, part) => part.reverse) + override def reverse: EdgeRDDImpl[ED, VD] = mapEdgePartitions((pid, part) => part.reverse) - override def filter( + def filter( epred: EdgeTriplet[VD, ED] => Boolean, - vpred: (VertexId, VD) => Boolean): EdgeRDD[ED, VD] = { + vpred: (VertexId, VD) => Boolean): EdgeRDDImpl[ED, VD] = { mapEdgePartitions((pid, part) => part.filter(epred, vpred)) } override def innerJoin[ED2: ClassTag, ED3: ClassTag] - (other: EdgeRDD[ED2, _]) - (f: (VertexId, VertexId, ED, ED2) => ED3): EdgeRDD[ED3, VD] = { + (other: EdgeRDD[ED2]) + (f: (VertexId, VertexId, ED, ED2) => ED3): EdgeRDDImpl[ED3, VD] = { val ed2Tag = classTag[ED2] val ed3Tag = classTag[ED3] this.withPartitionsRDD[ED3, VD](partitionsRDD.zipPartitions(other.partitionsRDD, true) { @@ -99,8 +99,8 @@ class EdgeRDDImpl[ED: ClassTag, VD: ClassTag] private[graphx] ( }) } - override private[graphx] def mapEdgePartitions[ED2: ClassTag, VD2: ClassTag]( - f: (PartitionID, EdgePartition[ED, VD]) => EdgePartition[ED2, VD2]): EdgeRDD[ED2, VD2] = { + def mapEdgePartitions[ED2: ClassTag, VD2: ClassTag]( + f: (PartitionID, EdgePartition[ED, VD]) => EdgePartition[ED2, VD2]): EdgeRDDImpl[ED2, VD2] = { this.withPartitionsRDD[ED2, VD2](partitionsRDD.mapPartitions({ iter => if (iter.hasNext) { val (pid, ep) = iter.next() @@ -111,13 +111,13 @@ class EdgeRDDImpl[ED: ClassTag, VD: ClassTag] private[graphx] ( }, preservesPartitioning = true)) } - override private[graphx] def withPartitionsRDD[ED2: ClassTag, VD2: ClassTag]( - partitionsRDD: RDD[(PartitionID, EdgePartition[ED2, VD2])]): EdgeRDD[ED2, VD2] = { + private[graphx] def withPartitionsRDD[ED2: ClassTag, VD2: ClassTag]( + partitionsRDD: RDD[(PartitionID, EdgePartition[ED2, VD2])]): EdgeRDDImpl[ED2, VD2] = { new EdgeRDDImpl(partitionsRDD, this.targetStorageLevel) } override private[graphx] def withTargetStorageLevel( - targetStorageLevel: StorageLevel): EdgeRDD[ED, VD] = { + targetStorageLevel: StorageLevel): EdgeRDDImpl[ED, VD] = { new EdgeRDDImpl(this.partitionsRDD, targetStorageLevel) } 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 2b4636a6c6ddf..0eae2a673874a 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 @@ -43,7 +43,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( /** Default constructor is provided to support serialization */ protected def this() = this(null, null) - @transient override val edges: EdgeRDD[ED, VD] = replicatedVertexView.edges + @transient override val edges: EdgeRDDImpl[ED, VD] = replicatedVertexView.edges /** Return a RDD that brings edges together with their source and destination vertices. */ @transient override lazy val triplets: RDD[EdgeTriplet[VD, ED]] = { @@ -323,9 +323,10 @@ object GraphImpl { */ def apply[VD: ClassTag, ED: ClassTag]( vertices: VertexRDD[VD], - edges: EdgeRDD[ED, _]): GraphImpl[VD, ED] = { + edges: EdgeRDD[ED]): GraphImpl[VD, ED] = { // Convert the vertex partitions in edges to the correct type - val newEdges = edges.mapEdgePartitions((pid, part) => part.withoutVertexAttributes[VD]) + val newEdges = edges.asInstanceOf[EdgeRDDImpl[ED, _]] + .mapEdgePartitions((pid, part) => part.withoutVertexAttributes[VD]) GraphImpl.fromExistingRDDs(vertices, newEdges) } @@ -336,8 +337,8 @@ object GraphImpl { */ def fromExistingRDDs[VD: ClassTag, ED: ClassTag]( vertices: VertexRDD[VD], - edges: EdgeRDD[ED, VD]): GraphImpl[VD, ED] = { - new GraphImpl(vertices, new ReplicatedVertexView(edges)) + edges: EdgeRDD[ED]): GraphImpl[VD, ED] = { + new GraphImpl(vertices, new ReplicatedVertexView(edges.asInstanceOf[EdgeRDDImpl[ED, VD]])) } /** @@ -345,7 +346,7 @@ object GraphImpl { * `defaultVertexAttr`. The vertices will have the same number of partitions as the EdgeRDD. */ private def fromEdgeRDD[VD: ClassTag, ED: ClassTag]( - edges: EdgeRDD[ED, VD], + edges: EdgeRDDImpl[ED, VD], defaultVertexAttr: VD, edgeStorageLevel: StorageLevel, vertexStorageLevel: StorageLevel): GraphImpl[VD, ED] = { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala index 86b366eb9202b..8ab255bd4038c 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala @@ -33,7 +33,7 @@ import org.apache.spark.graphx._ */ private[impl] class ReplicatedVertexView[VD: ClassTag, ED: ClassTag]( - var edges: EdgeRDD[ED, VD], + var edges: EdgeRDDImpl[ED, VD], var hasSrcId: Boolean = false, var hasDstId: Boolean = false) { @@ -42,7 +42,7 @@ class ReplicatedVertexView[VD: ClassTag, ED: ClassTag]( * shipping level. */ def withEdges[VD2: ClassTag, ED2: ClassTag]( - edges_ : EdgeRDD[ED2, VD2]): ReplicatedVertexView[VD2, ED2] = { + edges_ : EdgeRDDImpl[ED2, VD2]): ReplicatedVertexView[VD2, ED2] = { new ReplicatedVertexView(edges_, hasSrcId, hasDstId) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala index 08405629bc052..d92a55a189298 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala @@ -172,7 +172,7 @@ class VertexRDDImpl[VD] private[graphx] ( override def reverseRoutingTables(): VertexRDD[VD] = this.mapVertexPartitions(vPart => vPart.withRoutingTable(vPart.routingTable.reverse)) - override def withEdges(edges: EdgeRDD[_, _]): VertexRDD[VD] = { + override def withEdges(edges: EdgeRDD[_]): VertexRDD[VD] = { val routingTables = VertexRDD.createRoutingTables(edges, this.partitioner.get) val vertexPartitions = partitionsRDD.zipPartitions(routingTables, true) { (partIter, routingTableIter) => From dbb9da5c3d86301eb1b2f06857d6501552503a63 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 17 Nov 2014 11:24:28 -0800 Subject: [PATCH 006/109] Revert "[SPARK-4075] [Deploy] Jar url validation is not enough for Jar file" This reverts commit 098f83c7ccd7dad9f9228596da69fe5f55711a52. --- .../org/apache/spark/deploy/ClientArguments.scala | 11 +---------- .../scala/org/apache/spark/deploy/ClientSuite.scala | 6 ------ 2 files changed, 1 insertion(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala index 4e802e02c4149..39150deab863c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala @@ -17,8 +17,6 @@ package org.apache.spark.deploy -import java.net.{URI, URISyntaxException} - import scala.collection.mutable.ListBuffer import org.apache.log4j.Level @@ -116,12 +114,5 @@ private[spark] class ClientArguments(args: Array[String]) { } object ClientArguments { - def isValidJarUrl(s: String): Boolean = { - try { - val uri = new URI(s) - uri.getScheme != null && uri.getAuthority != null && s.endsWith("jar") - } catch { - case _: URISyntaxException => false - } - } + def isValidJarUrl(s: String): Boolean = s.matches("(.+):(.+)jar") } diff --git a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala index 94a2bdd74e744..4161aede1d1d0 100644 --- a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala @@ -29,12 +29,6 @@ class ClientSuite extends FunSuite with Matchers { ClientArguments.isValidJarUrl("hdfs://someHost:1234/foo") should be (false) ClientArguments.isValidJarUrl("/missing/a/protocol/jarfile.jar") should be (false) ClientArguments.isValidJarUrl("not-even-a-path.jar") should be (false) - - // No authority - ClientArguments.isValidJarUrl("hdfs:someHost:1234/jarfile.jar") should be (false) - - // Invalid syntax - ClientArguments.isValidJarUrl("hdfs:") should be (false) } } From cec1116b4b80c36b36a8a13338b948e4d6ade377 Mon Sep 17 00:00:00 2001 From: Andy Konwinski Date: Mon, 17 Nov 2014 11:52:23 -0800 Subject: [PATCH 007/109] [DOCS][SQL] Fix broken link to Row class scaladoc Author: Andy Konwinski Closes #3323 from andyk/patch-2 and squashes the following commits: 4699fdc [Andy Konwinski] Fix broken link to Row class scaladoc --- docs/sql-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 48e8267ac072c..5500da83b2b66 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -14,7 +14,7 @@ title: Spark SQL Programming Guide Spark SQL allows relational queries expressed in SQL, HiveQL, or Scala to be executed using Spark. At the core of this component is a new type of RDD, [SchemaRDD](api/scala/index.html#org.apache.spark.sql.SchemaRDD). SchemaRDDs are composed of -[Row](api/scala/index.html#org.apache.spark.sql.catalyst.expressions.Row) objects, along with +[Row](api/scala/index.html#org.apache.spark.sql.package@Row:org.apache.spark.sql.catalyst.expressions.Row.type) objects, along with a schema that describes the data types of each column in the row. A SchemaRDD is similar to a table in a traditional relational database. A SchemaRDD can be created from an existing RDD, a [Parquet](http://parquet.io) file, a JSON dataset, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). From 0f3ceb56c78e7260725a09fba0e10aa193cbda4b Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 17 Nov 2014 12:48:18 -0800 Subject: [PATCH 008/109] [SPARK-4180] [Core] Prevent creation of multiple active SparkContexts This patch adds error-detection logic to throw an exception when attempting to create multiple active SparkContexts in the same JVM, since this is currently unsupported and has been known to cause confusing behavior (see SPARK-2243 for more details). **The solution implemented here is only a partial fix.** A complete fix would have the following properties: 1. Only one SparkContext may ever be under construction at any given time. 2. Once a SparkContext has been successfully constructed, any subsequent construction attempts should fail until the active SparkContext is stopped. 3. If the SparkContext constructor throws an exception, then all resources created in the constructor should be cleaned up (SPARK-4194). 4. If a user attempts to create a SparkContext but the creation fails, then the user should be able to create new SparkContexts. This PR only provides 2) and 4); we should be able to provide all of these properties, but the correct fix will involve larger changes to SparkContext's construction / initialization, so we'll target it for a different Spark release. ### The correct solution: I think that the correct way to do this would be to move the construction of SparkContext's dependencies into a static method in the SparkContext companion object. Specifically, we could make the default SparkContext constructor `private` and change it to accept a `SparkContextDependencies` object that contains all of SparkContext's dependencies (e.g. DAGScheduler, ContextCleaner, etc.). Secondary constructors could call a method on the SparkContext companion object to create the `SparkContextDependencies` and pass the result to the primary SparkContext constructor. For example: ```scala class SparkContext private (deps: SparkContextDependencies) { def this(conf: SparkConf) { this(SparkContext.getDeps(conf)) } } object SparkContext( private[spark] def getDeps(conf: SparkConf): SparkContextDependencies = synchronized { if (anotherSparkContextIsActive) { throw Exception(...) } var dagScheduler: DAGScheduler = null try { dagScheduler = new DAGScheduler(...) [...] } catch { case e: Exception => Option(dagScheduler).foreach(_.stop()) [...] } SparkContextDependencies(dagScheduler, ....) } } ``` This gives us mutual exclusion and ensures that any resources created during the failed SparkContext initialization are properly cleaned up. This indirection is necessary to maintain binary compatibility. In retrospect, it would have been nice if SparkContext had no private constructors and could only be created through builder / factory methods on its companion object, since this buys us lots of flexibility and makes dependency injection easier. ### Alternative solutions: As an alternative solution, we could refactor SparkContext's primary constructor to perform all object creation in a giant `try-finally` block. Unfortunately, this will require us to turn a bunch of `vals` into `vars` so that they can be assigned from the `try` block. If we still want `vals`, we could wrap each `val` in its own `try` block (since the try block can return a value), but this will lead to extremely messy code and won't guard against the introduction of future code which doesn't properly handle failures. The more complex approach outlined above gives us some nice dependency injection benefits, so I think that might be preferable to a `var`-ification. ### This PR's solution: - At the start of the constructor, check whether some other SparkContext is active; if so, throw an exception. - If another SparkContext might be under construction (or has thrown an exception during construction), allow the new SparkContext to begin construction but log a warning (since resources might have been leaked from a failed creation attempt). - At the end of the SparkContext constructor, check whether some other SparkContext constructor has raced and successfully created an active context. If so, throw an exception. This guarantees that no two SparkContexts will ever be active and exposed to users (since we check at the very end of the constructor). If two threads race to construct SparkContexts, then one of them will win and another will throw an exception. This exception can be turned into a warning by setting `spark.driver.allowMultipleContexts = true`. The exception is disabled in unit tests, since there are some suites (such as Hive) that may require more significant refactoring to clean up their SparkContexts. I've made a few changes to other suites' test fixtures to properly clean up SparkContexts so that the unit test logs contain fewer warnings. Author: Josh Rosen Closes #3121 from JoshRosen/SPARK-4180 and squashes the following commits: 23c7123 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4180 d38251b [Josh Rosen] Address latest round of feedback. c0987d3 [Josh Rosen] Accept boolean instead of SparkConf in methods. 85a424a [Josh Rosen] Incorporate more review feedback. 372d0d3 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4180 f5bb78c [Josh Rosen] Update mvn build, too. d809cb4 [Josh Rosen] Improve handling of failed SparkContext creation attempts. 79a7e6f [Josh Rosen] Fix commented out test a1cba65 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4180 7ba6db8 [Josh Rosen] Add utility to set system properties in tests. 4629d5c [Josh Rosen] Set spark.driver.allowMultipleContexts=true in tests. ed17e14 [Josh Rosen] Address review feedback; expose hack workaround for existing unit tests. 1c66070 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4180 06c5c54 [Josh Rosen] Add / improve SparkContext cleanup in streaming BasicOperationsSuite d0437eb [Josh Rosen] StreamingContext.stop() should stop SparkContext even if StreamingContext has not been started yet. c4d35a2 [Josh Rosen] Log long form of creation site to aid debugging. 918e878 [Josh Rosen] Document "one SparkContext per JVM" limitation. afaa7e3 [Josh Rosen] [SPARK-4180] Prevent creations of multiple active SparkContexts. --- .../scala/org/apache/spark/SparkContext.scala | 167 +++++++++++++--- .../spark/api/java/JavaSparkContext.scala | 3 + .../ExecutorAllocationManagerSuite.scala | 4 + .../org/apache/spark/SparkContextSuite.scala | 57 +++++- docs/programming-guide.md | 2 + pom.xml | 1 + project/SparkBuild.scala | 1 + .../streaming/BasicOperationsSuite.scala | 186 +++++++++--------- .../spark/streaming/TestSuiteBase.scala | 52 ++++- 9 files changed, 347 insertions(+), 126 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 65edeeffb837a..7cccf74003431 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -58,12 +58,26 @@ import org.apache.spark.util._ * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark * cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster. * + * Only one SparkContext may be active per JVM. You must `stop()` the active SparkContext before + * creating a new one. This limitation may eventually be removed; see SPARK-2243 for more details. + * * @param config a Spark Config object describing the application configuration. Any settings in * this config overrides the default configs as well as system properties. */ - class SparkContext(config: SparkConf) extends Logging { + // The call site where this SparkContext was constructed. + private val creationSite: CallSite = Utils.getCallSite() + + // If true, log warnings instead of throwing exceptions when multiple SparkContexts are active + private val allowMultipleContexts: Boolean = + config.getBoolean("spark.driver.allowMultipleContexts", false) + + // In order to prevent multiple SparkContexts from being active at the same time, mark this + // context as having started construction. + // NOTE: this must be placed at the beginning of the SparkContext constructor. + SparkContext.markPartiallyConstructed(this, allowMultipleContexts) + // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, // etc) too. This is typically generated from InputFormatInfo.computePreferredLocations. It // contains a map from hostname to a list of input format splits on the host. @@ -1166,27 +1180,30 @@ class SparkContext(config: SparkConf) extends Logging { /** Shut down the SparkContext. */ def stop() { - postApplicationEnd() - ui.foreach(_.stop()) - // Do this only if not stopped already - best case effort. - // prevent NPE if stopped more than once. - val dagSchedulerCopy = dagScheduler - dagScheduler = null - if (dagSchedulerCopy != null) { - env.metricsSystem.report() - metadataCleaner.cancel() - env.actorSystem.stop(heartbeatReceiver) - cleaner.foreach(_.stop()) - dagSchedulerCopy.stop() - taskScheduler = null - // TODO: Cache.stop()? - env.stop() - SparkEnv.set(null) - listenerBus.stop() - eventLogger.foreach(_.stop()) - logInfo("Successfully stopped SparkContext") - } else { - logInfo("SparkContext already stopped") + SparkContext.SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { + postApplicationEnd() + ui.foreach(_.stop()) + // Do this only if not stopped already - best case effort. + // prevent NPE if stopped more than once. + val dagSchedulerCopy = dagScheduler + dagScheduler = null + if (dagSchedulerCopy != null) { + env.metricsSystem.report() + metadataCleaner.cancel() + env.actorSystem.stop(heartbeatReceiver) + cleaner.foreach(_.stop()) + dagSchedulerCopy.stop() + taskScheduler = null + // TODO: Cache.stop()? + env.stop() + SparkEnv.set(null) + listenerBus.stop() + eventLogger.foreach(_.stop()) + logInfo("Successfully stopped SparkContext") + SparkContext.clearActiveContext() + } else { + logInfo("SparkContext already stopped") + } } } @@ -1475,6 +1492,11 @@ class SparkContext(config: SparkConf) extends Logging { private[spark] def cleanup(cleanupTime: Long) { persistentRdds.clearOldValues(cleanupTime) } + + // In order to prevent multiple SparkContexts from being active at the same time, mark this + // context as having finished construction. + // NOTE: this must be placed at the end of the SparkContext constructor. + SparkContext.setActiveContext(this, allowMultipleContexts) } /** @@ -1483,6 +1505,107 @@ class SparkContext(config: SparkConf) extends Logging { */ object SparkContext extends Logging { + /** + * Lock that guards access to global variables that track SparkContext construction. + */ + private val SPARK_CONTEXT_CONSTRUCTOR_LOCK = new Object() + + /** + * The active, fully-constructed SparkContext. If no SparkContext is active, then this is `None`. + * + * Access to this field is guarded by SPARK_CONTEXT_CONSTRUCTOR_LOCK + */ + private var activeContext: Option[SparkContext] = None + + /** + * Points to a partially-constructed SparkContext if some thread is in the SparkContext + * constructor, or `None` if no SparkContext is being constructed. + * + * Access to this field is guarded by SPARK_CONTEXT_CONSTRUCTOR_LOCK + */ + private var contextBeingConstructed: Option[SparkContext] = None + + /** + * Called to ensure that no other SparkContext is running in this JVM. + * + * Throws an exception if a running context is detected and logs a warning if another thread is + * constructing a SparkContext. This warning is necessary because the current locking scheme + * prevents us from reliably distinguishing between cases where another context is being + * constructed and cases where another constructor threw an exception. + */ + private def assertNoOtherContextIsRunning( + sc: SparkContext, + allowMultipleContexts: Boolean): Unit = { + SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { + contextBeingConstructed.foreach { otherContext => + if (otherContext ne sc) { // checks for reference equality + // Since otherContext might point to a partially-constructed context, guard against + // its creationSite field being null: + val otherContextCreationSite = + Option(otherContext.creationSite).map(_.longForm).getOrElse("unknown location") + val warnMsg = "Another SparkContext is being constructed (or threw an exception in its" + + " constructor). This may indicate an error, since only one SparkContext may be" + + " running in this JVM (see SPARK-2243)." + + s" The other SparkContext was created at:\n$otherContextCreationSite" + logWarning(warnMsg) + } + + activeContext.foreach { ctx => + val errMsg = "Only one SparkContext may be running in this JVM (see SPARK-2243)." + + " To ignore this error, set spark.driver.allowMultipleContexts = true. " + + s"The currently running SparkContext was created at:\n${ctx.creationSite.longForm}" + val exception = new SparkException(errMsg) + if (allowMultipleContexts) { + logWarning("Multiple running SparkContexts detected in the same JVM!", exception) + } else { + throw exception + } + } + } + } + } + + /** + * Called at the beginning of the SparkContext constructor to ensure that no SparkContext is + * running. Throws an exception if a running context is detected and logs a warning if another + * thread is constructing a SparkContext. This warning is necessary because the current locking + * scheme prevents us from reliably distinguishing between cases where another context is being + * constructed and cases where another constructor threw an exception. + */ + private[spark] def markPartiallyConstructed( + sc: SparkContext, + allowMultipleContexts: Boolean): Unit = { + SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { + assertNoOtherContextIsRunning(sc, allowMultipleContexts) + contextBeingConstructed = Some(sc) + } + } + + /** + * Called at the end of the SparkContext constructor to ensure that no other SparkContext has + * raced with this constructor and started. + */ + private[spark] def setActiveContext( + sc: SparkContext, + allowMultipleContexts: Boolean): Unit = { + SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { + assertNoOtherContextIsRunning(sc, allowMultipleContexts) + contextBeingConstructed = None + activeContext = Some(sc) + } + } + + /** + * Clears the active SparkContext metadata. This is called by `SparkContext#stop()`. It's + * also called in unit tests to prevent a flood of warnings from test suites that don't / can't + * properly clean up their SparkContexts. + */ + private[spark] def clearActiveContext(): Unit = { + SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { + activeContext = None + } + } + private[spark] val SPARK_JOB_DESCRIPTION = "spark.job.description" private[spark] val SPARK_JOB_GROUP_ID = "spark.jobGroup.id" diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index d50ed32ca085c..6a6d9bf6857d3 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -42,6 +42,9 @@ import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, NewHadoopRDD, RDD} /** * A Java-friendly version of [[org.apache.spark.SparkContext]] that returns * [[org.apache.spark.api.java.JavaRDD]]s and works with Java collections instead of Scala ones. + * + * Only one SparkContext may be active per JVM. You must `stop()` the active SparkContext before + * creating a new one. This limitation may eventually be removed; see SPARK-2243 for more details. */ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWorkaround with Closeable { diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 4b27477790212..ce804f94f3267 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -37,20 +37,24 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { .set("spark.dynamicAllocation.enabled", "true") intercept[SparkException] { new SparkContext(conf) } SparkEnv.get.stop() // cleanup the created environment + SparkContext.clearActiveContext() // Only min val conf1 = conf.clone().set("spark.dynamicAllocation.minExecutors", "1") intercept[SparkException] { new SparkContext(conf1) } SparkEnv.get.stop() + SparkContext.clearActiveContext() // Only max val conf2 = conf.clone().set("spark.dynamicAllocation.maxExecutors", "2") intercept[SparkException] { new SparkContext(conf2) } SparkEnv.get.stop() + SparkContext.clearActiveContext() // Both min and max, but min > max intercept[SparkException] { createSparkContext(2, 1) } SparkEnv.get.stop() + SparkContext.clearActiveContext() // Both min and max, and min == max val sc1 = createSparkContext(1, 1) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 31edad1c56c73..9e454ddcc52a6 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -21,9 +21,62 @@ import org.scalatest.FunSuite import org.apache.hadoop.io.BytesWritable -class SparkContextSuite extends FunSuite { - //Regression test for SPARK-3121 +class SparkContextSuite extends FunSuite with LocalSparkContext { + + /** Allows system properties to be changed in tests */ + private def withSystemProperty[T](property: String, value: String)(block: => T): T = { + val originalValue = System.getProperty(property) + try { + System.setProperty(property, value) + block + } finally { + if (originalValue == null) { + System.clearProperty(property) + } else { + System.setProperty(property, originalValue) + } + } + } + + test("Only one SparkContext may be active at a time") { + // Regression test for SPARK-4180 + withSystemProperty("spark.driver.allowMultipleContexts", "false") { + val conf = new SparkConf().setAppName("test").setMaster("local") + sc = new SparkContext(conf) + // A SparkContext is already running, so we shouldn't be able to create a second one + intercept[SparkException] { new SparkContext(conf) } + // After stopping the running context, we should be able to create a new one + resetSparkContext() + sc = new SparkContext(conf) + } + } + + test("Can still construct a new SparkContext after failing to construct a previous one") { + withSystemProperty("spark.driver.allowMultipleContexts", "false") { + // This is an invalid configuration (no app name or master URL) + intercept[SparkException] { + new SparkContext(new SparkConf()) + } + // Even though those earlier calls failed, we should still be able to create a new context + sc = new SparkContext(new SparkConf().setMaster("local").setAppName("test")) + } + } + + test("Check for multiple SparkContexts can be disabled via undocumented debug option") { + withSystemProperty("spark.driver.allowMultipleContexts", "true") { + var secondSparkContext: SparkContext = null + try { + val conf = new SparkConf().setAppName("test").setMaster("local") + sc = new SparkContext(conf) + secondSparkContext = new SparkContext(conf) + } finally { + Option(secondSparkContext).foreach(_.stop()) + } + } + } + test("BytesWritable implicit conversion is correct") { + // Regression test for SPARK-3121 val bytesWritable = new BytesWritable() val inputArray = (1 to 10).map(_.toByte).toArray bytesWritable.set(inputArray, 0, 10) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 9de2f914b8b4c..49f319ba775e5 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -117,6 +117,8 @@ The first thing a Spark program must do is to create a [SparkContext](api/scala/ how to access a cluster. To create a `SparkContext` you first need to build a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) object that contains information about your application. +Only one SparkContext may be active per JVM. You must `stop()` the active SparkContext before creating a new one. + {% highlight scala %} val conf = new SparkConf().setAppName(appName).setMaster(master) new SparkContext(conf) diff --git a/pom.xml b/pom.xml index 639ea22a1fda3..cc7bce175778f 100644 --- a/pom.xml +++ b/pom.xml @@ -978,6 +978,7 @@ 1 false ${test_classpath} + true diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index c96a6c49545c1..1697b6d4f2d43 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -377,6 +377,7 @@ object TestSettings { javaOptions in Test += "-Dspark.testing=1", javaOptions in Test += "-Dspark.port.maxRetries=100", javaOptions in Test += "-Dspark.ui.enabled=false", + javaOptions in Test += "-Dspark.driver.allowMultipleContexts=true", javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true", javaOptions in Test ++= System.getProperties.filter(_._1 startsWith "spark") .map { case (k,v) => s"-D$k=$v" }.toSeq, diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala index 30a359677cc74..86b96785d7b87 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -470,32 +470,31 @@ class BasicOperationsSuite extends TestSuiteBase { } test("slice") { - val ssc = new StreamingContext(conf, Seconds(1)) - val input = Seq(Seq(1), Seq(2), Seq(3), Seq(4)) - val stream = new TestInputStream[Int](ssc, input, 2) - stream.foreachRDD(_ => {}) // Dummy output stream - ssc.start() - Thread.sleep(2000) - def getInputFromSlice(fromMillis: Long, toMillis: Long) = { - stream.slice(new Time(fromMillis), new Time(toMillis)).flatMap(_.collect()).toSet - } + withStreamingContext(new StreamingContext(conf, Seconds(1))) { ssc => + val input = Seq(Seq(1), Seq(2), Seq(3), Seq(4)) + val stream = new TestInputStream[Int](ssc, input, 2) + stream.foreachRDD(_ => {}) // Dummy output stream + ssc.start() + Thread.sleep(2000) + def getInputFromSlice(fromMillis: Long, toMillis: Long) = { + stream.slice(new Time(fromMillis), new Time(toMillis)).flatMap(_.collect()).toSet + } - assert(getInputFromSlice(0, 1000) == Set(1)) - assert(getInputFromSlice(0, 2000) == Set(1, 2)) - assert(getInputFromSlice(1000, 2000) == Set(1, 2)) - assert(getInputFromSlice(2000, 4000) == Set(2, 3, 4)) - ssc.stop() - Thread.sleep(1000) + assert(getInputFromSlice(0, 1000) == Set(1)) + assert(getInputFromSlice(0, 2000) == Set(1, 2)) + assert(getInputFromSlice(1000, 2000) == Set(1, 2)) + assert(getInputFromSlice(2000, 4000) == Set(2, 3, 4)) + } } - test("slice - has not been initialized") { - val ssc = new StreamingContext(conf, Seconds(1)) - val input = Seq(Seq(1), Seq(2), Seq(3), Seq(4)) - val stream = new TestInputStream[Int](ssc, input, 2) - val thrown = intercept[SparkException] { - stream.slice(new Time(0), new Time(1000)) + withStreamingContext(new StreamingContext(conf, Seconds(1))) { ssc => + val input = Seq(Seq(1), Seq(2), Seq(3), Seq(4)) + val stream = new TestInputStream[Int](ssc, input, 2) + val thrown = intercept[SparkException] { + stream.slice(new Time(0), new Time(1000)) + } + assert(thrown.getMessage.contains("has not been initialized")) } - assert(thrown.getMessage.contains("has not been initialized")) } val cleanupTestInput = (0 until 10).map(x => Seq(x, x + 1)).toSeq @@ -555,73 +554,72 @@ class BasicOperationsSuite extends TestSuiteBase { test("rdd cleanup - input blocks and persisted RDDs") { // Actually receive data over through receiver to create BlockRDDs - // Start the server - val testServer = new TestServer() - testServer.start() - - // Set up the streaming context and input streams - val ssc = new StreamingContext(conf, batchDuration) - val networkStream = ssc.socketTextStream("localhost", testServer.port, StorageLevel.MEMORY_AND_DISK) - val mappedStream = networkStream.map(_ + ".").persist() - val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] - val outputStream = new TestOutputStream(mappedStream, outputBuffer) - - outputStream.register() - ssc.start() - - // Feed data to the server to send to the network receiver - val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] - val input = Seq(1, 2, 3, 4, 5, 6) + withTestServer(new TestServer()) { testServer => + withStreamingContext(new StreamingContext(conf, batchDuration)) { ssc => + testServer.start() + // Set up the streaming context and input streams + val networkStream = + ssc.socketTextStream("localhost", testServer.port, StorageLevel.MEMORY_AND_DISK) + val mappedStream = networkStream.map(_ + ".").persist() + val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] + val outputStream = new TestOutputStream(mappedStream, outputBuffer) + + outputStream.register() + ssc.start() + + // Feed data to the server to send to the network receiver + val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + val input = Seq(1, 2, 3, 4, 5, 6) + + val blockRdds = new mutable.HashMap[Time, BlockRDD[_]] + val persistentRddIds = new mutable.HashMap[Time, Int] + + def collectRddInfo() { // get all RDD info required for verification + networkStream.generatedRDDs.foreach { case (time, rdd) => + blockRdds(time) = rdd.asInstanceOf[BlockRDD[_]] + } + mappedStream.generatedRDDs.foreach { case (time, rdd) => + persistentRddIds(time) = rdd.id + } + } - val blockRdds = new mutable.HashMap[Time, BlockRDD[_]] - val persistentRddIds = new mutable.HashMap[Time, Int] + Thread.sleep(200) + for (i <- 0 until input.size) { + testServer.send(input(i).toString + "\n") + Thread.sleep(200) + clock.addToTime(batchDuration.milliseconds) + collectRddInfo() + } - def collectRddInfo() { // get all RDD info required for verification - networkStream.generatedRDDs.foreach { case (time, rdd) => - blockRdds(time) = rdd.asInstanceOf[BlockRDD[_]] - } - mappedStream.generatedRDDs.foreach { case (time, rdd) => - persistentRddIds(time) = rdd.id + Thread.sleep(200) + collectRddInfo() + logInfo("Stopping server") + testServer.stop() + + // verify data has been received + assert(outputBuffer.size > 0) + assert(blockRdds.size > 0) + assert(persistentRddIds.size > 0) + + import Time._ + + val latestPersistedRddId = persistentRddIds(persistentRddIds.keySet.max) + val earliestPersistedRddId = persistentRddIds(persistentRddIds.keySet.min) + val latestBlockRdd = blockRdds(blockRdds.keySet.max) + val earliestBlockRdd = blockRdds(blockRdds.keySet.min) + // verify that the latest mapped RDD is persisted but the earliest one has been unpersisted + assert(ssc.sparkContext.persistentRdds.contains(latestPersistedRddId)) + assert(!ssc.sparkContext.persistentRdds.contains(earliestPersistedRddId)) + + // verify that the latest input blocks are present but the earliest blocks have been removed + assert(latestBlockRdd.isValid) + assert(latestBlockRdd.collect != null) + assert(!earliestBlockRdd.isValid) + earliestBlockRdd.blockIds.foreach { blockId => + assert(!ssc.sparkContext.env.blockManager.master.contains(blockId)) + } } } - - Thread.sleep(200) - for (i <- 0 until input.size) { - testServer.send(input(i).toString + "\n") - Thread.sleep(200) - clock.addToTime(batchDuration.milliseconds) - collectRddInfo() - } - - Thread.sleep(200) - collectRddInfo() - logInfo("Stopping server") - testServer.stop() - logInfo("Stopping context") - - // verify data has been received - assert(outputBuffer.size > 0) - assert(blockRdds.size > 0) - assert(persistentRddIds.size > 0) - - import Time._ - - val latestPersistedRddId = persistentRddIds(persistentRddIds.keySet.max) - val earliestPersistedRddId = persistentRddIds(persistentRddIds.keySet.min) - val latestBlockRdd = blockRdds(blockRdds.keySet.max) - val earliestBlockRdd = blockRdds(blockRdds.keySet.min) - // verify that the latest mapped RDD is persisted but the earliest one has been unpersisted - assert(ssc.sparkContext.persistentRdds.contains(latestPersistedRddId)) - assert(!ssc.sparkContext.persistentRdds.contains(earliestPersistedRddId)) - - // verify that the latest input blocks are present but the earliest blocks have been removed - assert(latestBlockRdd.isValid) - assert(latestBlockRdd.collect != null) - assert(!earliestBlockRdd.isValid) - earliestBlockRdd.blockIds.foreach { blockId => - assert(!ssc.sparkContext.env.blockManager.master.contains(blockId)) - } - ssc.stop() } /** Test cleanup of RDDs in DStream metadata */ @@ -635,13 +633,15 @@ class BasicOperationsSuite extends TestSuiteBase { // Setup the stream computation assert(batchDuration === Seconds(1), "Batch duration has changed from 1 second, check cleanup tests") - val ssc = setupStreams(cleanupTestInput, operation) - val operatedStream = ssc.graph.getOutputStreams().head.dependencies.head.asInstanceOf[DStream[T]] - if (rememberDuration != null) ssc.remember(rememberDuration) - val output = runStreams[(Int, Int)](ssc, cleanupTestInput.size, numExpectedOutput) - val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] - assert(clock.time === Seconds(10).milliseconds) - assert(output.size === numExpectedOutput) - operatedStream + withStreamingContext(setupStreams(cleanupTestInput, operation)) { ssc => + val operatedStream = + ssc.graph.getOutputStreams().head.dependencies.head.asInstanceOf[DStream[T]] + if (rememberDuration != null) ssc.remember(rememberDuration) + val output = runStreams[(Int, Int)](ssc, cleanupTestInput.size, numExpectedOutput) + val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + assert(clock.time === Seconds(10).milliseconds) + assert(output.size === numExpectedOutput) + operatedStream + } } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index 2154c24abda3a..52972f63c6c5c 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -163,6 +163,40 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { before(beforeFunction) after(afterFunction) + /** + * Run a block of code with the given StreamingContext and automatically + * stop the context when the block completes or when an exception is thrown. + */ + def withStreamingContext[R](ssc: StreamingContext)(block: StreamingContext => R): R = { + try { + block(ssc) + } finally { + try { + ssc.stop(stopSparkContext = true) + } catch { + case e: Exception => + logError("Error stopping StreamingContext", e) + } + } + } + + /** + * Run a block of code with the given TestServer and automatically + * stop the server when the block completes or when an exception is thrown. + */ + def withTestServer[R](testServer: TestServer)(block: TestServer => R): R = { + try { + block(testServer) + } finally { + try { + testServer.stop() + } catch { + case e: Exception => + logError("Error stopping TestServer", e) + } + } + } + /** * Set up required DStreams to test the DStream operation using the two sequences * of input collections. @@ -282,10 +316,8 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { assert(output.size === numExpectedOutput, "Unexpected number of outputs generated") Thread.sleep(100) // Give some time for the forgetting old RDDs to complete - } catch { - case e: Exception => {e.printStackTrace(); throw e} } finally { - ssc.stop() + ssc.stop(stopSparkContext = true) } output } @@ -351,9 +383,10 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { useSet: Boolean ) { val numBatches_ = if (numBatches > 0) numBatches else expectedOutput.size - val ssc = setupStreams[U, V](input, operation) - val output = runStreams[V](ssc, numBatches_, expectedOutput.size) - verifyOutput[V](output, expectedOutput, useSet) + withStreamingContext(setupStreams[U, V](input, operation)) { ssc => + val output = runStreams[V](ssc, numBatches_, expectedOutput.size) + verifyOutput[V](output, expectedOutput, useSet) + } } /** @@ -389,8 +422,9 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { useSet: Boolean ) { val numBatches_ = if (numBatches > 0) numBatches else expectedOutput.size - val ssc = setupStreams[U, V, W](input1, input2, operation) - val output = runStreams[W](ssc, numBatches_, expectedOutput.size) - verifyOutput[W](output, expectedOutput, useSet) + withStreamingContext(setupStreams[U, V, W](input1, input2, operation)) { ssc => + val output = runStreams[W](ssc, numBatches_, expectedOutput.size) + verifyOutput[W](output, expectedOutput, useSet) + } } } From 5ce7dae859dc273b0fc532c9456b5960b1eca399 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 17 Nov 2014 15:33:13 -0800 Subject: [PATCH 009/109] [SQL] Makes conjunction pushdown more aggressive for in-memory table This is inspired by the [Parquet record filter generation code](https://github.com/apache/spark/blob/64c6b9bad559c21f25cd9fbe37c8813cdab939f2/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala#L387-L400). [Review on Reviewable](https://reviewable.io/reviews/apache/spark/3318) Author: Cheng Lian Closes #3318 from liancheng/aggresive-conj-pushdown and squashes the following commits: 78b69d2 [Cheng Lian] Makes conjunction pushdown more aggressive --- .../sql/columnar/InMemoryColumnarTableScan.scala | 4 ++-- .../sql/columnar/PartitionBatchPruningSuite.scala | 12 +++++++++--- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala index 455b415d9d959..881d32b105c5f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala @@ -182,8 +182,8 @@ private[sql] case class InMemoryColumnarTableScan( // to evaluate to `true' based on statistics collected about this partition batch. val buildFilter: PartialFunction[Expression, Expression] = { case And(lhs: Expression, rhs: Expression) - if buildFilter.isDefinedAt(lhs) && buildFilter.isDefinedAt(rhs) => - buildFilter(lhs) && buildFilter(rhs) + if buildFilter.isDefinedAt(lhs) || buildFilter.isDefinedAt(rhs) => + (buildFilter.lift(lhs) ++ buildFilter.lift(rhs)).reduce(_ && _) case Or(lhs: Expression, rhs: Expression) if buildFilter.isDefinedAt(lhs) && buildFilter.isDefinedAt(rhs) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala index 9ba3c210171bd..82afa31a99a7e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala @@ -78,17 +78,23 @@ class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with Be // Conjunction and disjunction checkBatchPruning("SELECT key FROM pruningData WHERE key > 8 AND key <= 21", 2, 3)(9 to 21) checkBatchPruning("SELECT key FROM pruningData WHERE key < 2 OR key > 99", 2, 2)(Seq(1, 100)) + checkBatchPruning("SELECT key FROM pruningData WHERE key < 12 AND key IS NOT NULL", 1, 2)(1 to 11) checkBatchPruning("SELECT key FROM pruningData WHERE key < 2 OR (key > 78 AND key < 92)", 3, 4) { Seq(1) ++ (79 to 91) } + checkBatchPruning("SELECT key FROM pruningData WHERE NOT (key < 88)", 1, 2) { + // Although the `NOT` operator isn't supported directly, the optimizer can transform + // `NOT (a < b)` to `b >= a` + 88 to 100 + } // With unsupported predicate - checkBatchPruning("SELECT key FROM pruningData WHERE NOT (key < 88)", 1, 2)(88 to 100) - checkBatchPruning("SELECT key FROM pruningData WHERE key < 12 AND key IS NOT NULL", 1, 2)(1 to 11) - { val seq = (1 to 30).mkString(", ") checkBatchPruning(s"SELECT key FROM pruningData WHERE NOT (key IN ($seq))", 5, 10)(31 to 100) + checkBatchPruning(s"SELECT key FROM pruningData WHERE NOT (key IN ($seq)) AND key > 88", 1, 2) { + 89 to 100 + } } def checkBatchPruning( From 3a81a1c9e0963173534d96850f3c0b7a16350838 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Mon, 17 Nov 2014 16:26:48 -0800 Subject: [PATCH 010/109] [SPARK-4420][SQL] Change nullability of Cast from DoubleType/FloatType to DecimalType. This is follow-up of [SPARK-4390](https://issues.apache.org/jira/browse/SPARK-4390) (#3256). Author: Takuya UESHIN Closes #3278 from ueshin/issues/SPARK-4420 and squashes the following commits: 7fea558 [Takuya UESHIN] Add some tests. cb2301a [Takuya UESHIN] Fix tests. 133bad5 [Takuya UESHIN] Change nullability of Cast from DoubleType/FloatType to DecimalType. --- .../spark/sql/catalyst/expressions/Cast.scala | 2 ++ .../expressions/ExpressionEvaluationSuite.scala | 14 ++++++++++++-- 2 files changed, 14 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 34697a1249644..b401096ce1d87 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -37,6 +37,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case (BooleanType, DateType) => true case (DateType, _: NumericType) => true case (DateType, BooleanType) => true + case (DoubleType, _: DecimalType) => true + case (FloatType, _: DecimalType) => true case (_, DecimalType.Fixed(_, _)) => true // TODO: not all upcasts here can really give null case _ => child.nullable } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 2f57be94a80fb..3a6a0203afef6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -347,8 +347,8 @@ class ExpressionEvaluationSuite extends FunSuite { // - Because of this, casts to fixed-precision decimals should be nullable assert(Cast(Literal(123), DecimalType.Unlimited).nullable === false) - assert(Cast(Literal(10.03f), DecimalType.Unlimited).nullable === false) - assert(Cast(Literal(10.03), DecimalType.Unlimited).nullable === false) + assert(Cast(Literal(10.03f), DecimalType.Unlimited).nullable === true) + assert(Cast(Literal(10.03), DecimalType.Unlimited).nullable === true) assert(Cast(Literal(Decimal(10.03)), DecimalType.Unlimited).nullable === false) assert(Cast(Literal(123), DecimalType(2, 1)).nullable === true) @@ -396,6 +396,16 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Cast(Literal(-9.95), DecimalType(1, 0)), null) checkEvaluation(Cast(Literal(Decimal(-9.95)), DecimalType(3, 1)), Decimal(-10.0)) checkEvaluation(Cast(Literal(Decimal(-9.95)), DecimalType(1, 0)), null) + + checkEvaluation(Cast(Literal(Double.NaN), DecimalType.Unlimited), null) + checkEvaluation(Cast(Literal(1.0 / 0.0), DecimalType.Unlimited), null) + checkEvaluation(Cast(Literal(Float.NaN), DecimalType.Unlimited), null) + checkEvaluation(Cast(Literal(1.0f / 0.0f), DecimalType.Unlimited), null) + + checkEvaluation(Cast(Literal(Double.NaN), DecimalType(2, 1)), null) + checkEvaluation(Cast(Literal(1.0 / 0.0), DecimalType(2, 1)), null) + checkEvaluation(Cast(Literal(Float.NaN), DecimalType(2, 1)), null) + checkEvaluation(Cast(Literal(1.0f / 0.0f), DecimalType(2, 1)), null) } test("timestamp") { From 566c791931645bfaaaf57ee5a15b9ffad534f81e Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Mon, 17 Nov 2014 16:28:07 -0800 Subject: [PATCH 011/109] [SPARK-4425][SQL] Handle NaN or Infinity cast to Timestamp correctly. `Cast` from `NaN` or `Infinity` of `Double` or `Float` to `TimestampType` throws `NumberFormatException`. Author: Takuya UESHIN Closes #3283 from ueshin/issues/SPARK-4425 and squashes the following commits: 14def0c [Takuya UESHIN] Fix Cast to be able to handle NaN or Infinity to TimestampType. --- .../spark/sql/catalyst/expressions/Cast.scala | 14 ++++++++++++-- .../expressions/ExpressionEvaluationSuite.scala | 5 +++++ 2 files changed, 17 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index b401096ce1d87..b47865f87a3aa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -32,6 +32,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w override def nullable = (child.dataType, dataType) match { case (StringType, _: NumericType) => true case (StringType, TimestampType) => true + case (DoubleType, TimestampType) => true + case (FloatType, TimestampType) => true case (StringType, DateType) => true case (_: NumericType, DateType) => true case (BooleanType, DateType) => true @@ -117,10 +119,18 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w buildCast[Decimal](_, d => decimalToTimestamp(d)) // TimestampWritable.doubleToTimestamp case DoubleType => - buildCast[Double](_, d => decimalToTimestamp(Decimal(d))) + buildCast[Double](_, d => try { + decimalToTimestamp(Decimal(d)) + } catch { + case _: NumberFormatException => null + }) // TimestampWritable.floatToTimestamp case FloatType => - buildCast[Float](_, f => decimalToTimestamp(Decimal(f))) + buildCast[Float](_, f => try { + decimalToTimestamp(Decimal(f)) + } catch { + case _: NumberFormatException => null + }) } private[this] def decimalToTimestamp(d: Decimal) = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 3a6a0203afef6..3f5b9f698f827 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -450,6 +450,11 @@ class ExpressionEvaluationSuite extends FunSuite { // A test for higher precision than millis checkEvaluation(Cast(Cast(0.00000001, TimestampType), DoubleType), 0.00000001) + + checkEvaluation(Cast(Literal(Double.NaN), TimestampType), null) + checkEvaluation(Cast(Literal(1.0 / 0.0), TimestampType), null) + checkEvaluation(Cast(Literal(Float.NaN), TimestampType), null) + checkEvaluation(Cast(Literal(1.0f / 0.0f), TimestampType), null) } test("null checking") { From 69e858cc7748b6babadd0cbe20e65f3982161cbf Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Mon, 17 Nov 2014 16:29:52 -0800 Subject: [PATCH 012/109] [SQL] Construct the MutableRow from an Array Author: Cheng Hao Closes #3217 from chenghao-intel/mutablerow and squashes the following commits: e8a10bd [Cheng Hao] revert the change of Row object 4681aea [Cheng Hao] Add toMutableRow method in object Row a751838 [Cheng Hao] Construct the MutableRow from an existed row --- .../org/apache/spark/sql/catalyst/expressions/Row.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala index d00ec39774c35..463f3667fc445 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala @@ -215,9 +215,11 @@ class GenericRow(protected[sql] val values: Array[Any]) extends Row { def copy() = this } -class GenericMutableRow(size: Int) extends GenericRow(size) with MutableRow { +class GenericMutableRow(v: Array[Any]) extends GenericRow(v) with MutableRow { /** No-arg constructor for serialization. */ - def this() = this(0) + def this() = this(null) + + def this(size: Int) = this(new Array[Any](size)) override def setBoolean(ordinal: Int, value: Boolean): Unit = { values(ordinal) = value } override def setByte(ordinal: Int, value: Byte): Unit = { values(ordinal) = value } From 6b7f2f753d16ff038881772f1958e3f4fd5597a7 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 17 Nov 2014 16:31:05 -0800 Subject: [PATCH 013/109] [SPARK-4309][SPARK-4407][SQL] Date type support for Thrift server, and fixes for complex types This PR is exactly the same as #3178 except it reverts the `FileStatus.isDir` to `FileStatus.isDirectory` change, since it doesn't compile with Hadoop 1. [Review on Reviewable](https://reviewable.io/reviews/apache/spark/3298) Author: Cheng Lian Closes #3298 from liancheng/date-for-thriftserver and squashes the following commits: 866037e [Cheng Lian] Revers isDirectory to isDir (it breaks Hadoop 1 profile) 6f71d0b [Cheng Lian] Makes toHiveString static 26fa955 [Cheng Lian] Fixes complex type support in Hive 0.13.1 shim a92882a [Cheng Lian] Updates HiveShim for 0.13.1 73f442b [Cheng Lian] Adds Date support for HiveThriftServer2 (Hive 0.12.0) --- .../thriftserver/HiveThriftServer2Suite.scala | 90 +++++++++---- .../spark/sql/hive/thriftserver/Shim12.scala | 11 +- .../spark/sql/hive/thriftserver/Shim13.scala | 29 ++-- .../apache/spark/sql/hive/HiveContext.scala | 125 ++++++++---------- 4 files changed, 141 insertions(+), 114 deletions(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala index bba29b2bdca4d..23d12cbff3495 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala @@ -19,9 +19,10 @@ package org.apache.spark.sql.hive.thriftserver import java.io.File import java.net.ServerSocket -import java.sql.{DriverManager, Statement} +import java.sql.{Date, DriverManager, Statement} import java.util.concurrent.TimeoutException +import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ import scala.concurrent.{Await, Promise} @@ -51,6 +52,15 @@ import org.apache.spark.sql.hive.HiveShim class HiveThriftServer2Suite extends FunSuite with Logging { Class.forName(classOf[HiveDriver].getCanonicalName) + object TestData { + def getTestDataFilePath(name: String) = { + Thread.currentThread().getContextClassLoader.getResource(s"data/files/$name") + } + + val smallKv = getTestDataFilePath("small_kv.txt") + val smallKvWithNull = getTestDataFilePath("small_kv_with_null.txt") + } + def randomListeningPort = { // Let the system to choose a random available port to avoid collision with other parallel // builds. @@ -145,12 +155,8 @@ class HiveThriftServer2Suite extends FunSuite with Logging { } } - val env = Seq( - // Resets SPARK_TESTING to avoid loading Log4J configurations in testing class paths - "SPARK_TESTING" -> "0", - // Prevents loading classes out of the assembly jar. Otherwise Utils.sparkVersion can't read - // proper version information from the jar manifest. - "SPARK_PREPEND_CLASSES" -> "") + // Resets SPARK_TESTING to avoid loading Log4J configurations in testing class paths + val env = Seq("SPARK_TESTING" -> "0") Process(command, None, env: _*).run(ProcessLogger( captureThriftServerOutput("stdout"), @@ -194,15 +200,12 @@ class HiveThriftServer2Suite extends FunSuite with Logging { test("Test JDBC query execution") { withJdbcStatement() { statement => - val dataFilePath = - Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt") - - val queries = - s"""SET spark.sql.shuffle.partitions=3; - |CREATE TABLE test(key INT, val STRING); - |LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test; - |CACHE TABLE test; - """.stripMargin.split(";").map(_.trim).filter(_.nonEmpty) + val queries = Seq( + "SET spark.sql.shuffle.partitions=3", + "DROP TABLE IF EXISTS test", + "CREATE TABLE test(key INT, val STRING)", + s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test", + "CACHE TABLE test") queries.foreach(statement.execute) @@ -216,14 +219,10 @@ class HiveThriftServer2Suite extends FunSuite with Logging { test("SPARK-3004 regression: result set containing NULL") { withJdbcStatement() { statement => - val dataFilePath = - Thread.currentThread().getContextClassLoader.getResource( - "data/files/small_kv_with_null.txt") - val queries = Seq( "DROP TABLE IF EXISTS test_null", "CREATE TABLE test_null(key INT, val STRING)", - s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test_null") + s"LOAD DATA LOCAL INPATH '${TestData.smallKvWithNull}' OVERWRITE INTO TABLE test_null") queries.foreach(statement.execute) @@ -270,13 +269,10 @@ class HiveThriftServer2Suite extends FunSuite with Logging { test("SPARK-4292 regression: result set iterator issue") { withJdbcStatement() { statement => - val dataFilePath = - Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt") - val queries = Seq( "DROP TABLE IF EXISTS test_4292", "CREATE TABLE test_4292(key INT, val STRING)", - s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test_4292") + s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_4292") queries.foreach(statement.execute) @@ -284,10 +280,52 @@ class HiveThriftServer2Suite extends FunSuite with Logging { Seq(238, 86, 311, 27, 165).foreach { key => resultSet.next() - assert(resultSet.getInt(1) == key) + assert(resultSet.getInt(1) === key) } statement.executeQuery("DROP TABLE IF EXISTS test_4292") } } + + test("SPARK-4309 regression: Date type support") { + withJdbcStatement() { statement => + val queries = Seq( + "DROP TABLE IF EXISTS test_date", + "CREATE TABLE test_date(key INT, value STRING)", + s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_date") + + queries.foreach(statement.execute) + + assertResult(Date.valueOf("2011-01-01")) { + val resultSet = statement.executeQuery( + "SELECT CAST('2011-01-01' as date) FROM test_date LIMIT 1") + resultSet.next() + resultSet.getDate(1) + } + } + } + + test("SPARK-4407 regression: Complex type support") { + withJdbcStatement() { statement => + val queries = Seq( + "DROP TABLE IF EXISTS test_map", + "CREATE TABLE test_map(key INT, value STRING)", + s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_map") + + queries.foreach(statement.execute) + + assertResult("""{238:"val_238"}""") { + val resultSet = statement.executeQuery("SELECT MAP(key, value) FROM test_map LIMIT 1") + resultSet.next() + resultSet.getString(1) + } + + assertResult("""["238","val_238"]""") { + val resultSet = statement.executeQuery( + "SELECT ARRAY(CAST(key AS STRING), value) FROM test_map LIMIT 1") + resultSet.next() + resultSet.getString(1) + } + } + } } diff --git a/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala b/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala index aa2e3cab72bb9..9258ad0cdf1d0 100644 --- a/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala +++ b/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive.thriftserver -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import java.util.{ArrayList => JArrayList, Map => JMap} import scala.collection.JavaConversions._ @@ -131,14 +131,13 @@ private[hive] class SparkExecuteStatementOperation( to.addColumnValue(ColumnValue.byteValue(from.getByte(ordinal))) case ShortType => to.addColumnValue(ColumnValue.shortValue(from.getShort(ordinal))) + case DateType => + to.addColumnValue(ColumnValue.dateValue(from(ordinal).asInstanceOf[Date])) case TimestampType => to.addColumnValue( ColumnValue.timestampValue(from.get(ordinal).asInstanceOf[Timestamp])) case BinaryType | _: ArrayType | _: StructType | _: MapType => - val hiveString = result - .queryExecution - .asInstanceOf[HiveContext#QueryExecution] - .toHiveString((from.get(ordinal), dataTypes(ordinal))) + val hiveString = HiveContext.toHiveString((from.get(ordinal), dataTypes(ordinal))) to.addColumnValue(ColumnValue.stringValue(hiveString)) } } @@ -163,6 +162,8 @@ private[hive] class SparkExecuteStatementOperation( to.addColumnValue(ColumnValue.byteValue(null)) case ShortType => to.addColumnValue(ColumnValue.shortValue(null)) + case DateType => + to.addColumnValue(ColumnValue.dateValue(null)) case TimestampType => to.addColumnValue(ColumnValue.timestampValue(null)) case BinaryType | _: ArrayType | _: StructType | _: MapType => diff --git a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala index a642478d08857..3c7f62af450d9 100644 --- a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala +++ b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.hive.thriftserver import java.security.PrivilegedExceptionAction -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import java.util.concurrent.Future import java.util.{ArrayList => JArrayList, List => JList, Map => JMap} @@ -113,7 +113,7 @@ private[hive] class SparkExecuteStatementOperation( def addNonNullColumnValue(from: SparkRow, to: ArrayBuffer[Any], ordinal: Int) { dataTypes(ordinal) match { case StringType => - to += from.get(ordinal).asInstanceOf[String] + to += from.getString(ordinal) case IntegerType => to += from.getInt(ordinal) case BooleanType => @@ -123,23 +123,20 @@ private[hive] class SparkExecuteStatementOperation( case FloatType => to += from.getFloat(ordinal) case DecimalType() => - to += from.get(ordinal).asInstanceOf[BigDecimal].bigDecimal + to += from.getAs[BigDecimal](ordinal).bigDecimal case LongType => to += from.getLong(ordinal) case ByteType => to += from.getByte(ordinal) case ShortType => to += from.getShort(ordinal) + case DateType => + to += from.getAs[Date](ordinal) case TimestampType => - to += from.get(ordinal).asInstanceOf[Timestamp] - case BinaryType => - to += from.get(ordinal).asInstanceOf[String] - case _: ArrayType => - to += from.get(ordinal).asInstanceOf[String] - case _: StructType => - to += from.get(ordinal).asInstanceOf[String] - case _: MapType => - to += from.get(ordinal).asInstanceOf[String] + to += from.getAs[Timestamp](ordinal) + case BinaryType | _: ArrayType | _: StructType | _: MapType => + val hiveString = HiveContext.toHiveString((from.get(ordinal), dataTypes(ordinal))) + to += hiveString } } @@ -147,9 +144,9 @@ private[hive] class SparkExecuteStatementOperation( validateDefaultFetchOrientation(order) assertState(OperationState.FINISHED) setHasResultSet(true) - val reultRowSet: RowSet = RowSetFactory.create(getResultSetSchema, getProtocolVersion) + val resultRowSet: RowSet = RowSetFactory.create(getResultSetSchema, getProtocolVersion) if (!iter.hasNext) { - reultRowSet + resultRowSet } else { // maxRowsL here typically maps to java.sql.Statement.getFetchSize, which is an int val maxRows = maxRowsL.toInt @@ -166,10 +163,10 @@ private[hive] class SparkExecuteStatementOperation( } curCol += 1 } - reultRowSet.addRow(row.toArray.asInstanceOf[Array[Object]]) + resultRowSet.addRow(row.toArray.asInstanceOf[Array[Object]]) curRow += 1 } - reultRowSet + resultRowSet } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index e88afaaf001c0..304b9a73ee91d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -19,36 +19,27 @@ package org.apache.spark.sql.hive import java.io.{BufferedReader, File, InputStreamReader, PrintStream} import java.sql.{Date, Timestamp} -import java.util.{ArrayList => JArrayList} - -import org.apache.hadoop.hive.common.`type`.HiveDecimal -import org.apache.spark.sql.catalyst.types.DecimalType -import org.apache.spark.sql.catalyst.types.decimal.Decimal import scala.collection.JavaConversions._ import scala.language.implicitConversions -import scala.reflect.runtime.universe.{TypeTag, typeTag} +import scala.reflect.runtime.universe.TypeTag -import org.apache.hadoop.fs.FileSystem -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.metadata.Table import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.session.SessionState -import org.apache.hadoop.hive.serde2.io.TimestampWritable -import org.apache.hadoop.hive.serde2.io.DateWritable +import org.apache.hadoop.hive.serde2.io.{DateWritable, TimestampWritable} import org.apache.spark.SparkContext -import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.catalyst.analysis.{Analyzer, EliminateAnalysisOperators} -import org.apache.spark.sql.catalyst.analysis.{OverrideCatalog, OverrideFunctionRegistry} +import org.apache.spark.sql.catalyst.analysis.{Analyzer, EliminateAnalysisOperators, OverrideCatalog, OverrideFunctionRegistry} import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.execution.ExtractPythonUdfs -import org.apache.spark.sql.execution.QueryExecutionException -import org.apache.spark.sql.execution.{Command => PhysicalCommand} +import org.apache.spark.sql.catalyst.types.DecimalType +import org.apache.spark.sql.catalyst.types.decimal.Decimal +import org.apache.spark.sql.execution.{ExtractPythonUdfs, QueryExecutionException, Command => PhysicalCommand} import org.apache.spark.sql.hive.execution.DescribeHiveTableCommand import org.apache.spark.sql.sources.DataSourceStrategy @@ -136,7 +127,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { val relation = EliminateAnalysisOperators(catalog.lookupRelation(None, tableName)) relation match { - case relation: MetastoreRelation => { + case relation: MetastoreRelation => // This method is mainly based on // org.apache.hadoop.hive.ql.stats.StatsUtils.getFileSizeForTable(HiveConf, Table) // in Hive 0.13 (except that we do not use fs.getContentSummary). @@ -157,7 +148,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } def getFileSizeForTable(conf: HiveConf, table: Table): Long = { - val path = table.getPath() + val path = table.getPath var size: Long = 0L try { val fs = path.getFileSystem(conf) @@ -187,15 +178,14 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { val hiveTTable = relation.hiveQlTable.getTTable hiveTTable.setParameters(tableParameters) val tableFullName = - relation.hiveQlTable.getDbName() + "." + relation.hiveQlTable.getTableName() + relation.hiveQlTable.getDbName + "." + relation.hiveQlTable.getTableName catalog.client.alterTable(tableFullName, new Table(hiveTTable)) } - } case otherRelation => throw new NotImplementedError( s"Analyze has only implemented for Hive tables, " + - s"but ${tableName} is a ${otherRelation.nodeName}") + s"but $tableName is a ${otherRelation.nodeName}") } } @@ -374,50 +364,6 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { /** Extends QueryExecution with hive specific features. */ protected[sql] abstract class QueryExecution extends super.QueryExecution { - protected val primitiveTypes = - Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType, ByteType, - ShortType, DateType, TimestampType, BinaryType) - - protected[sql] def toHiveString(a: (Any, DataType)): String = a match { - case (struct: Row, StructType(fields)) => - struct.zip(fields).map { - case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}""" - }.mkString("{", ",", "}") - case (seq: Seq[_], ArrayType(typ, _)) => - seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") - case (map: Map[_,_], MapType(kType, vType, _)) => - map.map { - case (key, value) => - toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) - }.toSeq.sorted.mkString("{", ",", "}") - case (null, _) => "NULL" - case (d: Date, DateType) => new DateWritable(d).toString - case (t: Timestamp, TimestampType) => new TimestampWritable(t).toString - case (bin: Array[Byte], BinaryType) => new String(bin, "UTF-8") - case (decimal: Decimal, DecimalType()) => // Hive strips trailing zeros so use its toString - HiveShim.createDecimal(decimal.toBigDecimal.underlying()).toString - case (other, tpe) if primitiveTypes contains tpe => other.toString - } - - /** Hive outputs fields of structs slightly differently than top level attributes. */ - protected def toHiveStructString(a: (Any, DataType)): String = a match { - case (struct: Row, StructType(fields)) => - struct.zip(fields).map { - case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}""" - }.mkString("{", ",", "}") - case (seq: Seq[_], ArrayType(typ, _)) => - seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") - case (map: Map[_, _], MapType(kType, vType, _)) => - map.map { - case (key, value) => - toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) - }.toSeq.sorted.mkString("{", ",", "}") - case (null, _) => "null" - case (s: String, StringType) => "\"" + s + "\"" - case (decimal, DecimalType()) => decimal.toString - case (other, tpe) if primitiveTypes contains tpe => other.toString - } - /** * Returns the result as a hive compatible sequence of strings. For native commands, the * execution is simply passed back to Hive. @@ -435,8 +381,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { // We need the types so we can output struct field names val types = analyzed.output.map(_.dataType) // Reformat to match hive tab delimited output. - val asString = result.map(_.zip(types).map(toHiveString)).map(_.mkString("\t")).toSeq - asString + result.map(_.zip(types).map(HiveContext.toHiveString)).map(_.mkString("\t")).toSeq } override def simpleString: String = @@ -447,3 +392,49 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } } } + +object HiveContext { + protected val primitiveTypes = + Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType, ByteType, + ShortType, DateType, TimestampType, BinaryType) + + protected[sql] def toHiveString(a: (Any, DataType)): String = a match { + case (struct: Row, StructType(fields)) => + struct.zip(fields).map { + case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}""" + }.mkString("{", ",", "}") + case (seq: Seq[_], ArrayType(typ, _)) => + seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") + case (map: Map[_,_], MapType(kType, vType, _)) => + map.map { + case (key, value) => + toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) + }.toSeq.sorted.mkString("{", ",", "}") + case (null, _) => "NULL" + case (d: Date, DateType) => new DateWritable(d).toString + case (t: Timestamp, TimestampType) => new TimestampWritable(t).toString + case (bin: Array[Byte], BinaryType) => new String(bin, "UTF-8") + case (decimal: Decimal, DecimalType()) => // Hive strips trailing zeros so use its toString + HiveShim.createDecimal(decimal.toBigDecimal.underlying()).toString + case (other, tpe) if primitiveTypes contains tpe => other.toString + } + + /** Hive outputs fields of structs slightly differently than top level attributes. */ + protected def toHiveStructString(a: (Any, DataType)): String = a match { + case (struct: Row, StructType(fields)) => + struct.zip(fields).map { + case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}""" + }.mkString("{", ",", "}") + case (seq: Seq[_], ArrayType(typ, _)) => + seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") + case (map: Map[_, _], MapType(kType, vType, _)) => + map.map { + case (key, value) => + toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) + }.toSeq.sorted.mkString("{", ",", "}") + case (null, _) => "null" + case (s: String, StringType) => "\"" + s + "\"" + case (decimal, DecimalType()) => decimal.toString + case (other, tpe) if primitiveTypes contains tpe => other.toString + } +} From 42389b1780311d90499b4ce2315ceabf5b6ab384 Mon Sep 17 00:00:00 2001 From: w00228970 Date: Mon, 17 Nov 2014 16:33:50 -0800 Subject: [PATCH 014/109] [SPARK-4443][SQL] Fix statistics for external table in spark sql hive The `totalSize` of external table is always zero, which will influence join strategy(always use broadcast join for external table). Author: w00228970 Closes #3304 from scwf/statistics and squashes the following commits: 568f321 [w00228970] fix statistics for external table --- .../apache/spark/sql/hive/HiveMetastoreCatalog.scala | 11 ++++++++--- .../main/scala/org/apache/spark/sql/hive/Shim12.scala | 2 ++ .../main/scala/org/apache/spark/sql/hive/Shim13.scala | 2 ++ 3 files changed, 12 insertions(+), 3 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 9045fc8558276..91a157785d5bb 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 @@ -447,6 +447,8 @@ private[hive] case class MetastoreRelation @transient override lazy val statistics = Statistics( sizeInBytes = { + val totalSize = hiveQlTable.getParameters.get(HiveShim.getStatsSetupConstTotalSize) + val rawDataSize = hiveQlTable.getParameters.get(HiveShim.getStatsSetupConstRawDataSize) // TODO: check if this estimate is valid for tables after partition pruning. // NOTE: getting `totalSize` directly from params is kind of hacky, but this should be // relatively cheap if parameters for the table are populated into the metastore. An @@ -454,9 +456,12 @@ private[hive] case class MetastoreRelation // of RPCs are involved. Besides `totalSize`, there are also `numFiles`, `numRows`, // `rawDataSize` keys (see StatsSetupConst in Hive) that we can look at in the future. BigInt( - Option(hiveQlTable.getParameters.get(HiveShim.getStatsSetupConstTotalSize)) - .map(_.toLong) - .getOrElse(sqlContext.defaultSizeInBytes)) + // When table is external,`totalSize` is always zero, which will influence join strategy + // so when `totalSize` is zero, use `rawDataSize` instead + // if the size is still less than zero, we use default size + Option(totalSize).map(_.toLong).filter(_ > 0) + .getOrElse(Option(rawDataSize).map(_.toLong).filter(_ > 0) + .getOrElse(sqlContext.defaultSizeInBytes))) } ) 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 8ba25f889d176..76f09cbcdec99 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 @@ -136,6 +136,8 @@ private[hive] object HiveShim { def getStatsSetupConstTotalSize = StatsSetupConst.TOTAL_SIZE + def getStatsSetupConstRawDataSize = StatsSetupConst.RAW_DATA_SIZE + def createDefaultDBIfNeeded(context: HiveContext) = { } def getCommandProcessor(cmd: Array[String], conf: HiveConf) = { 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 e4aee57f0ad9f..91f7ceac21177 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 @@ -154,6 +154,8 @@ private[hive] object HiveShim { def getStatsSetupConstTotalSize = StatsSetupConst.TOTAL_SIZE + def getStatsSetupConstRawDataSize = StatsSetupConst.RAW_DATA_SIZE + def createDefaultDBIfNeeded(context: HiveContext) = { context.runSqlHive("CREATE DATABASE default") context.runSqlHive("USE default") From ef7c464effa1510b24bd8e665e4df6c4839b0c87 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Mon, 17 Nov 2014 16:35:49 -0800 Subject: [PATCH 015/109] [SPARK-4448] [SQL] unwrap for the ConstantObjectInspector Author: Cheng Hao Closes #3308 from chenghao-intel/unwrap_constant_oi and squashes the following commits: 156b500 [Cheng Hao] rebase the master c5b20ab [Cheng Hao] unwrap for the ConstantObjectInspector --- .../spark/sql/hive/HiveInspectors.scala | 36 ++++++++++++++++--- 1 file changed, 32 insertions(+), 4 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index 7e76aff642bb5..ada980acb1f77 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -86,13 +86,41 @@ private[hive] trait HiveInspectors { * @param data the data in Hive type * @param oi the ObjectInspector associated with the Hive Type * @return convert the data into catalyst type + * TODO return the function of (data => Any) instead for performance consideration */ def unwrap(data: Any, oi: ObjectInspector): Any = oi match { case _ if data == null => null - case hvoi: HiveVarcharObjectInspector => - if (data == null) null else hvoi.getPrimitiveJavaObject(data).getValue - case hdoi: HiveDecimalObjectInspector => - if (data == null) null else HiveShim.toCatalystDecimal(hdoi, data) + case poi: VoidObjectInspector => null + case poi: WritableConstantHiveVarcharObjectInspector => + poi.getWritableConstantValue.getHiveVarchar.getValue + case poi: WritableConstantHiveDecimalObjectInspector => + HiveShim.toCatalystDecimal( + PrimitiveObjectInspectorFactory.javaHiveDecimalObjectInspector, + poi.getWritableConstantValue.getHiveDecimal) + case poi: WritableConstantTimestampObjectInspector => + poi.getWritableConstantValue.getTimestamp.clone() + case poi: WritableConstantIntObjectInspector => + poi.getWritableConstantValue.get() + case poi: WritableConstantDoubleObjectInspector => + poi.getWritableConstantValue.get() + case poi: WritableConstantBooleanObjectInspector => + poi.getWritableConstantValue.get() + case poi: WritableConstantLongObjectInspector => + poi.getWritableConstantValue.get() + case poi: WritableConstantFloatObjectInspector => + poi.getWritableConstantValue.get() + case poi: WritableConstantShortObjectInspector => + poi.getWritableConstantValue.get() + case poi: WritableConstantByteObjectInspector => + poi.getWritableConstantValue.get() + case poi: WritableConstantBinaryObjectInspector => + val writable = poi.getWritableConstantValue + val temp = new Array[Byte](writable.getLength) + System.arraycopy(writable.getBytes, 0, temp, 0, temp.length) + temp + case poi: WritableConstantDateObjectInspector => poi.getWritableConstantValue.get() + case hvoi: HiveVarcharObjectInspector => hvoi.getPrimitiveJavaObject(data).getValue + case hdoi: HiveDecimalObjectInspector => HiveShim.toCatalystDecimal(hdoi, data) // org.apache.hadoop.hive.serde2.io.TimestampWritable.set will reset current time object // if next timestamp is null, so Timestamp object is cloned case ti: TimestampObjectInspector => ti.getPrimitiveJavaObject(data).clone() From 36b0956a3eadc7343ed0d25c79a6ce0496eaaccd Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 17 Nov 2014 16:55:12 -0800 Subject: [PATCH 016/109] [SPARK-4453][SPARK-4213][SQL] Simplifies Parquet filter generation code While reviewing PR #3083 and #3161, I noticed that Parquet record filter generation code can be simplified significantly according to the clue stated in [SPARK-4453](https://issues.apache.org/jira/browse/SPARK-4213). This PR addresses both SPARK-4453 and SPARK-4213 with this simplification. While generating `ParquetTableScan` operator, we need to remove all Catalyst predicates that have already been pushed down to Parquet. Originally, we first generate the record filter, and then call `findExpression` to traverse the generated filter to find out all pushed down predicates [[1](https://github.com/apache/spark/blob/64c6b9bad559c21f25cd9fbe37c8813cdab939f2/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala#L213-L228)]. In this way, we have to introduce the `CatalystFilter` class hierarchy to bind the Catalyst predicates together with their generated Parquet filter, and complicate the code base a lot. The basic idea of this PR is that, we don't need `findExpression` after filter generation, because we already know a predicate can be pushed down if we can successfully generate its corresponding Parquet filter. SPARK-4213 is fixed by returning `None` for any unsupported predicate type. [Review on Reviewable](https://reviewable.io/reviews/apache/spark/3317) Author: Cheng Lian Closes #3317 from liancheng/simplify-parquet-filters and squashes the following commits: d6a9499 [Cheng Lian] Fixes import styling issue 43760e8 [Cheng Lian] Simplifies Parquet filter generation logic --- .../expressions/namedExpressions.scala | 1 + .../spark/sql/execution/SparkStrategies.scala | 25 +- .../spark/sql/parquet/ParquetFilters.scala | 693 +++--------------- .../sql/parquet/ParquetTableOperations.scala | 77 +- .../spark/sql/parquet/ParquetQuerySuite.scala | 58 +- 5 files changed, 161 insertions(+), 693 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index fc90a54a58259..7634d392d4111 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.util.Metadata object NamedExpression { private val curId = new java.util.concurrent.atomic.AtomicLong() def newExprId = ExprId(curId.getAndIncrement()) + def unapply(expr: NamedExpression): Option[(String, DataType)] = Some(expr.name, expr.dataType) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 7ef1f9f2c5c02..1225d18857af2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -209,22 +209,15 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case PhysicalOperation(projectList, filters: Seq[Expression], relation: ParquetRelation) => val prunePushedDownFilters = if (sqlContext.parquetFilterPushDown) { - (filters: Seq[Expression]) => { - filters.filter { filter => - // Note: filters cannot be pushed down to Parquet if they contain more complex - // expressions than simple "Attribute cmp Literal" comparisons. Here we remove - // all filters that have been pushed down. Note that a predicate such as - // "(A AND B) OR C" can result in "A OR C" being pushed down. - val recordFilter = ParquetFilters.createFilter(filter) - if (!recordFilter.isDefined) { - // First case: the pushdown did not result in any record filter. - true - } else { - // Second case: a record filter was created; here we are conservative in - // the sense that even if "A" was pushed and we check for "A AND B" we - // still want to keep "A AND B" in the higher-level filter, not just "B". - !ParquetFilters.findExpression(recordFilter.get, filter).isDefined - } + (predicates: Seq[Expression]) => { + // Note: filters cannot be pushed down to Parquet if they contain more complex + // expressions than simple "Attribute cmp Literal" comparisons. Here we remove all + // filters that have been pushed down. Note that a predicate such as "(A AND B) OR C" + // can result in "A OR C" being pushed down. Here we are conservative in the sense + // that even if "A" was pushed and we check for "A AND B" we still want to keep + // "A AND B" in the higher-level filter, not just "B". + predicates.map(p => p -> ParquetFilters.createFilter(p)).collect { + case (predicate, None) => predicate } } } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala index 9a3f6d388d621..3a9e1499e2dc4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala @@ -18,406 +18,152 @@ package org.apache.spark.sql.parquet import java.nio.ByteBuffer -import java.sql.{Date, Timestamp} +import com.google.common.io.BaseEncoding import org.apache.hadoop.conf.Configuration - -import parquet.common.schema.ColumnPath import parquet.filter2.compat.FilterCompat import parquet.filter2.compat.FilterCompat._ -import parquet.filter2.predicate.Operators.{Column, SupportsLtGt} -import parquet.filter2.predicate.{FilterApi, FilterPredicate} import parquet.filter2.predicate.FilterApi._ +import parquet.filter2.predicate.{FilterApi, FilterPredicate} import parquet.io.api.Binary -import parquet.column.ColumnReader - -import com.google.common.io.BaseEncoding import org.apache.spark.SparkEnv -import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.catalyst.types.decimal.Decimal -import org.apache.spark.sql.catalyst.expressions.{Predicate => CatalystPredicate} import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.execution.SparkSqlSerializer -import org.apache.spark.sql.parquet.ParquetColumns._ +import org.apache.spark.sql.catalyst.types._ private[sql] object ParquetFilters { val PARQUET_FILTER_DATA = "org.apache.spark.sql.parquet.row.filter" - def createRecordFilter(filterExpressions: Seq[Expression]): Filter = { - val filters: Seq[CatalystFilter] = filterExpressions.collect { - case (expression: Expression) if createFilter(expression).isDefined => - createFilter(expression).get - } - if (filters.length > 0) FilterCompat.get(filters.reduce(FilterApi.and)) else null + def createRecordFilter(filterExpressions: Seq[Expression]): Option[Filter] = { + filterExpressions.flatMap(createFilter).reduceOption(FilterApi.and).map(FilterCompat.get) } - def createFilter(expression: Expression): Option[CatalystFilter] = { - def createEqualityFilter( - name: String, - literal: Literal, - predicate: CatalystPredicate) = literal.dataType match { + def createFilter(predicate: Expression): Option[FilterPredicate] = { + val makeEq: PartialFunction[DataType, (String, Any) => FilterPredicate] = { case BooleanType => - ComparisonFilter.createBooleanEqualityFilter( - name, - literal.value.asInstanceOf[Boolean], - predicate) - case ByteType => - new ComparisonFilter( - name, - FilterApi.eq(byteColumn(name), literal.value.asInstanceOf[java.lang.Byte]), - predicate) - case ShortType => - new ComparisonFilter( - name, - FilterApi.eq(shortColumn(name), literal.value.asInstanceOf[java.lang.Short]), - predicate) + (n: String, v: Any) => FilterApi.eq(booleanColumn(n), v.asInstanceOf[java.lang.Boolean]) case IntegerType => - new ComparisonFilter( - name, - FilterApi.eq(intColumn(name), literal.value.asInstanceOf[Integer]), - predicate) + (n: String, v: Any) => FilterApi.eq(intColumn(n), v.asInstanceOf[Integer]) case LongType => - new ComparisonFilter( - name, - FilterApi.eq(longColumn(name), literal.value.asInstanceOf[java.lang.Long]), - predicate) - case DoubleType => - new ComparisonFilter( - name, - FilterApi.eq(doubleColumn(name), literal.value.asInstanceOf[java.lang.Double]), - predicate) + (n: String, v: Any) => FilterApi.eq(longColumn(n), v.asInstanceOf[java.lang.Long]) case FloatType => - new ComparisonFilter( - name, - FilterApi.eq(floatColumn(name), literal.value.asInstanceOf[java.lang.Float]), - predicate) + (n: String, v: Any) => FilterApi.eq(floatColumn(n), v.asInstanceOf[java.lang.Float]) + case DoubleType => + (n: String, v: Any) => FilterApi.eq(doubleColumn(n), v.asInstanceOf[java.lang.Double]) case StringType => - ComparisonFilter.createStringEqualityFilter( - name, - literal.value.asInstanceOf[String], - predicate) + (n: String, v: Any) => + FilterApi.eq(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) case BinaryType => - ComparisonFilter.createBinaryEqualityFilter( - name, - literal.value.asInstanceOf[Array[Byte]], - predicate) - case DateType => - new ComparisonFilter( - name, - FilterApi.eq(dateColumn(name), new WrappedDate(literal.value.asInstanceOf[Date])), - predicate) - case TimestampType => - new ComparisonFilter( - name, - FilterApi.eq(timestampColumn(name), - new WrappedTimestamp(literal.value.asInstanceOf[Timestamp])), - predicate) - case DecimalType.Unlimited => - new ComparisonFilter( - name, - FilterApi.eq(decimalColumn(name), literal.value.asInstanceOf[Decimal]), - predicate) + (n: String, v: Any) => + FilterApi.eq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) } - def createLessThanFilter( - name: String, - literal: Literal, - predicate: CatalystPredicate) = literal.dataType match { - case ByteType => - new ComparisonFilter( - name, - FilterApi.lt(byteColumn(name), literal.value.asInstanceOf[java.lang.Byte]), - predicate) - case ShortType => - new ComparisonFilter( - name, - FilterApi.lt(shortColumn(name), literal.value.asInstanceOf[java.lang.Short]), - predicate) + val makeLt: PartialFunction[DataType, (String, Any) => FilterPredicate] = { case IntegerType => - new ComparisonFilter( - name, - FilterApi.lt(intColumn(name), literal.value.asInstanceOf[Integer]), - predicate) + (n: String, v: Any) => FilterApi.lt(intColumn(n), v.asInstanceOf[Integer]) case LongType => - new ComparisonFilter( - name, - FilterApi.lt(longColumn(name), literal.value.asInstanceOf[java.lang.Long]), - predicate) - case DoubleType => - new ComparisonFilter( - name, - FilterApi.lt(doubleColumn(name), literal.value.asInstanceOf[java.lang.Double]), - predicate) + (n: String, v: Any) => FilterApi.lt(longColumn(n), v.asInstanceOf[java.lang.Long]) case FloatType => - new ComparisonFilter( - name, - FilterApi.lt(floatColumn(name), literal.value.asInstanceOf[java.lang.Float]), - predicate) + (n: String, v: Any) => FilterApi.lt(floatColumn(n), v.asInstanceOf[java.lang.Float]) + case DoubleType => + (n: String, v: Any) => FilterApi.lt(doubleColumn(n), v.asInstanceOf[java.lang.Double]) case StringType => - ComparisonFilter.createStringLessThanFilter( - name, - literal.value.asInstanceOf[String], - predicate) + (n: String, v: Any) => + FilterApi.lt(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) case BinaryType => - ComparisonFilter.createBinaryLessThanFilter( - name, - literal.value.asInstanceOf[Array[Byte]], - predicate) - case DateType => - new ComparisonFilter( - name, - FilterApi.lt(dateColumn(name), new WrappedDate(literal.value.asInstanceOf[Date])), - predicate) - case TimestampType => - new ComparisonFilter( - name, - FilterApi.lt(timestampColumn(name), - new WrappedTimestamp(literal.value.asInstanceOf[Timestamp])), - predicate) - case DecimalType.Unlimited => - new ComparisonFilter( - name, - FilterApi.lt(decimalColumn(name), literal.value.asInstanceOf[Decimal]), - predicate) + (n: String, v: Any) => + FilterApi.lt(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) } - def createLessThanOrEqualFilter( - name: String, - literal: Literal, - predicate: CatalystPredicate) = literal.dataType match { - case ByteType => - new ComparisonFilter( - name, - FilterApi.ltEq(byteColumn(name), literal.value.asInstanceOf[java.lang.Byte]), - predicate) - case ShortType => - new ComparisonFilter( - name, - FilterApi.ltEq(shortColumn(name), literal.value.asInstanceOf[java.lang.Short]), - predicate) + + val makeLtEq: PartialFunction[DataType, (String, Any) => FilterPredicate] = { case IntegerType => - new ComparisonFilter( - name, - FilterApi.ltEq(intColumn(name), literal.value.asInstanceOf[Integer]), - predicate) + (n: String, v: Any) => FilterApi.ltEq(intColumn(n), v.asInstanceOf[java.lang.Integer]) case LongType => - new ComparisonFilter( - name, - FilterApi.ltEq(longColumn(name), literal.value.asInstanceOf[java.lang.Long]), - predicate) - case DoubleType => - new ComparisonFilter( - name, - FilterApi.ltEq(doubleColumn(name), literal.value.asInstanceOf[java.lang.Double]), - predicate) + (n: String, v: Any) => FilterApi.ltEq(longColumn(n), v.asInstanceOf[java.lang.Long]) case FloatType => - new ComparisonFilter( - name, - FilterApi.ltEq(floatColumn(name), literal.value.asInstanceOf[java.lang.Float]), - predicate) + (n: String, v: Any) => FilterApi.ltEq(floatColumn(n), v.asInstanceOf[java.lang.Float]) + case DoubleType => + (n: String, v: Any) => FilterApi.ltEq(doubleColumn(n), v.asInstanceOf[java.lang.Double]) case StringType => - ComparisonFilter.createStringLessThanOrEqualFilter( - name, - literal.value.asInstanceOf[String], - predicate) + (n: String, v: Any) => + FilterApi.ltEq(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) case BinaryType => - ComparisonFilter.createBinaryLessThanOrEqualFilter( - name, - literal.value.asInstanceOf[Array[Byte]], - predicate) - case DateType => - new ComparisonFilter( - name, - FilterApi.ltEq(dateColumn(name), new WrappedDate(literal.value.asInstanceOf[Date])), - predicate) - case TimestampType => - new ComparisonFilter( - name, - FilterApi.ltEq(timestampColumn(name), - new WrappedTimestamp(literal.value.asInstanceOf[Timestamp])), - predicate) - case DecimalType.Unlimited => - new ComparisonFilter( - name, - FilterApi.ltEq(decimalColumn(name), literal.value.asInstanceOf[Decimal]), - predicate) + (n: String, v: Any) => + FilterApi.ltEq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) } - // TODO: combine these two types somehow? - def createGreaterThanFilter( - name: String, - literal: Literal, - predicate: CatalystPredicate) = literal.dataType match { - case ByteType => - new ComparisonFilter( - name, - FilterApi.gt(byteColumn(name), literal.value.asInstanceOf[java.lang.Byte]), - predicate) - case ShortType => - new ComparisonFilter( - name, - FilterApi.gt(shortColumn(name), literal.value.asInstanceOf[java.lang.Short]), - predicate) + + val makeGt: PartialFunction[DataType, (String, Any) => FilterPredicate] = { case IntegerType => - new ComparisonFilter( - name, - FilterApi.gt(intColumn(name), literal.value.asInstanceOf[Integer]), - predicate) + (n: String, v: Any) => FilterApi.gt(intColumn(n), v.asInstanceOf[java.lang.Integer]) case LongType => - new ComparisonFilter( - name, - FilterApi.gt(longColumn(name), literal.value.asInstanceOf[java.lang.Long]), - predicate) - case DoubleType => - new ComparisonFilter( - name, - FilterApi.gt(doubleColumn(name), literal.value.asInstanceOf[java.lang.Double]), - predicate) + (n: String, v: Any) => FilterApi.gt(longColumn(n), v.asInstanceOf[java.lang.Long]) case FloatType => - new ComparisonFilter( - name, - FilterApi.gt(floatColumn(name), literal.value.asInstanceOf[java.lang.Float]), - predicate) + (n: String, v: Any) => FilterApi.gt(floatColumn(n), v.asInstanceOf[java.lang.Float]) + case DoubleType => + (n: String, v: Any) => FilterApi.gt(doubleColumn(n), v.asInstanceOf[java.lang.Double]) case StringType => - ComparisonFilter.createStringGreaterThanFilter( - name, - literal.value.asInstanceOf[String], - predicate) + (n: String, v: Any) => + FilterApi.gt(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) case BinaryType => - ComparisonFilter.createBinaryGreaterThanFilter( - name, - literal.value.asInstanceOf[Array[Byte]], - predicate) - case DateType => - new ComparisonFilter( - name, - FilterApi.gt(dateColumn(name), new WrappedDate(literal.value.asInstanceOf[Date])), - predicate) - case TimestampType => - new ComparisonFilter( - name, - FilterApi.gt(timestampColumn(name), - new WrappedTimestamp(literal.value.asInstanceOf[Timestamp])), - predicate) - case DecimalType.Unlimited => - new ComparisonFilter( - name, - FilterApi.gt(decimalColumn(name), literal.value.asInstanceOf[Decimal]), - predicate) + (n: String, v: Any) => + FilterApi.gt(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) } - def createGreaterThanOrEqualFilter( - name: String, - literal: Literal, - predicate: CatalystPredicate) = literal.dataType match { - case ByteType => - new ComparisonFilter( - name, - FilterApi.gtEq(byteColumn(name), literal.value.asInstanceOf[java.lang.Byte]), - predicate) - case ShortType => - new ComparisonFilter( - name, - FilterApi.gtEq(shortColumn(name), literal.value.asInstanceOf[java.lang.Short]), - predicate) + + val makeGtEq: PartialFunction[DataType, (String, Any) => FilterPredicate] = { case IntegerType => - new ComparisonFilter( - name, - FilterApi.gtEq(intColumn(name), literal.value.asInstanceOf[Integer]), - predicate) + (n: String, v: Any) => FilterApi.gtEq(intColumn(n), v.asInstanceOf[java.lang.Integer]) case LongType => - new ComparisonFilter( - name, - FilterApi.gtEq(longColumn(name), literal.value.asInstanceOf[java.lang.Long]), - predicate) - case DoubleType => - new ComparisonFilter( - name, - FilterApi.gtEq(doubleColumn(name), literal.value.asInstanceOf[java.lang.Double]), - predicate) + (n: String, v: Any) => FilterApi.gtEq(longColumn(n), v.asInstanceOf[java.lang.Long]) case FloatType => - new ComparisonFilter( - name, - FilterApi.gtEq(floatColumn(name), literal.value.asInstanceOf[java.lang.Float]), - predicate) + (n: String, v: Any) => FilterApi.gtEq(floatColumn(n), v.asInstanceOf[java.lang.Float]) + case DoubleType => + (n: String, v: Any) => FilterApi.gtEq(doubleColumn(n), v.asInstanceOf[java.lang.Double]) case StringType => - ComparisonFilter.createStringGreaterThanOrEqualFilter( - name, - literal.value.asInstanceOf[String], - predicate) + (n: String, v: Any) => + FilterApi.gtEq(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) case BinaryType => - ComparisonFilter.createBinaryGreaterThanOrEqualFilter( - name, - literal.value.asInstanceOf[Array[Byte]], - predicate) - case DateType => - new ComparisonFilter( - name, - FilterApi.gtEq(dateColumn(name), new WrappedDate(literal.value.asInstanceOf[Date])), - predicate) - case TimestampType => - new ComparisonFilter( - name, - FilterApi.gtEq(timestampColumn(name), - new WrappedTimestamp(literal.value.asInstanceOf[Timestamp])), - predicate) - case DecimalType.Unlimited => - new ComparisonFilter( - name, - FilterApi.gtEq(decimalColumn(name), literal.value.asInstanceOf[Decimal]), - predicate) + (n: String, v: Any) => + FilterApi.gtEq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) } - /** - * TODO: we currently only filter on non-nullable (Parquet REQUIRED) attributes until - * https://github.com/Parquet/parquet-mr/issues/371 - * has been resolved. - */ - expression match { - case p @ Or(left: Expression, right: Expression) - if createFilter(left).isDefined && createFilter(right).isDefined => { - // If either side of this Or-predicate is empty then this means - // it contains a more complex comparison than between attribute and literal - // (e.g., it contained a CAST). The only safe thing to do is then to disregard - // this disjunction, which could be contained in a conjunction. If it stands - // alone then it is also safe to drop it, since a Null return value of this - // function is interpreted as having no filters at all. - val leftFilter = createFilter(left).get - val rightFilter = createFilter(right).get - Some(new OrFilter(leftFilter, rightFilter)) - } - case p @ And(left: Expression, right: Expression) => { - // This treats nested conjunctions; since either side of the conjunction - // may contain more complex filter expressions we may actually generate - // strictly weaker filter predicates in the process. - val leftFilter = createFilter(left) - val rightFilter = createFilter(right) - (leftFilter, rightFilter) match { - case (None, Some(filter)) => Some(filter) - case (Some(filter), None) => Some(filter) - case (Some(leftF), Some(rightF)) => - Some(new AndFilter(leftF, rightF)) - case _ => None - } - } - case p @ EqualTo(left: Literal, right: NamedExpression) if left.dataType != NullType => - Some(createEqualityFilter(right.name, left, p)) - case p @ EqualTo(left: NamedExpression, right: Literal) if right.dataType != NullType => - Some(createEqualityFilter(left.name, right, p)) - case p @ LessThan(left: Literal, right: NamedExpression) => - Some(createLessThanFilter(right.name, left, p)) - case p @ LessThan(left: NamedExpression, right: Literal) => - Some(createLessThanFilter(left.name, right, p)) - case p @ LessThanOrEqual(left: Literal, right: NamedExpression) => - Some(createLessThanOrEqualFilter(right.name, left, p)) - case p @ LessThanOrEqual(left: NamedExpression, right: Literal) => - Some(createLessThanOrEqualFilter(left.name, right, p)) - case p @ GreaterThan(left: Literal, right: NamedExpression) => - Some(createGreaterThanFilter(right.name, left, p)) - case p @ GreaterThan(left: NamedExpression, right: Literal) => - Some(createGreaterThanFilter(left.name, right, p)) - case p @ GreaterThanOrEqual(left: Literal, right: NamedExpression) => - Some(createGreaterThanOrEqualFilter(right.name, left, p)) - case p @ GreaterThanOrEqual(left: NamedExpression, right: Literal) => - Some(createGreaterThanOrEqualFilter(left.name, right, p)) + predicate match { + case EqualTo(NamedExpression(name, _), Literal(value, dataType)) if dataType != NullType => + makeEq.lift(dataType).map(_(name, value)) + case EqualTo(Literal(value, dataType), NamedExpression(name, _)) if dataType != NullType => + makeEq.lift(dataType).map(_(name, value)) + + case LessThan(NamedExpression(name, _), Literal(value, dataType)) => + makeLt.lift(dataType).map(_(name, value)) + case LessThan(Literal(value, dataType), NamedExpression(name, _)) => + makeLt.lift(dataType).map(_(name, value)) + + case LessThanOrEqual(NamedExpression(name, _), Literal(value, dataType)) => + makeLtEq.lift(dataType).map(_(name, value)) + case LessThanOrEqual(Literal(value, dataType), NamedExpression(name, _)) => + makeLtEq.lift(dataType).map(_(name, value)) + + case GreaterThan(NamedExpression(name, _), Literal(value, dataType)) => + makeGt.lift(dataType).map(_(name, value)) + case GreaterThan(Literal(value, dataType), NamedExpression(name, _)) => + makeGt.lift(dataType).map(_(name, value)) + + case GreaterThanOrEqual(NamedExpression(name, _), Literal(value, dataType)) => + makeGtEq.lift(dataType).map(_(name, value)) + case GreaterThanOrEqual(Literal(value, dataType), NamedExpression(name, _)) => + makeGtEq.lift(dataType).map(_(name, value)) + + case And(lhs, rhs) => + (createFilter(lhs) ++ createFilter(rhs)).reduceOption(FilterApi.and) + + case Or(lhs, rhs) => + for { + lhsFilter <- createFilter(lhs) + rhsFilter <- createFilter(rhs) + } yield FilterApi.or(lhsFilter, rhsFilter) + + case Not(pred) => + createFilter(pred).map(FilterApi.not) + case _ => None } } @@ -428,7 +174,7 @@ private[sql] object ParquetFilters { * the actual filter predicate. */ def serializeFilterExpressions(filters: Seq[Expression], conf: Configuration): Unit = { - if (filters.length > 0) { + if (filters.nonEmpty) { val serialized: Array[Byte] = SparkEnv.get.closureSerializer.newInstance().serialize(filters).array() val encoded: String = BaseEncoding.base64().encode(serialized) @@ -450,245 +196,4 @@ private[sql] object ParquetFilters { Seq() } } - - /** - * Try to find the given expression in the tree of filters in order to - * determine whether it is safe to remove it from the higher level filters. Note - * that strictly speaking we could stop the search whenever an expression is found - * that contains this expression as subexpression (e.g., when searching for "a" - * and "(a or c)" is found) but we don't care about optimizations here since the - * filter tree is assumed to be small. - * - * @param filter The [[org.apache.spark.sql.parquet.CatalystFilter]] to expand - * and search - * @param expression The expression to look for - * @return An optional [[org.apache.spark.sql.parquet.CatalystFilter]] that - * contains the expression. - */ - def findExpression( - filter: CatalystFilter, - expression: Expression): Option[CatalystFilter] = filter match { - case f @ OrFilter(_, leftFilter, rightFilter, _) => - if (f.predicate == expression) { - Some(f) - } else { - val left = findExpression(leftFilter, expression) - if (left.isDefined) left else findExpression(rightFilter, expression) - } - case f @ AndFilter(_, leftFilter, rightFilter, _) => - if (f.predicate == expression) { - Some(f) - } else { - val left = findExpression(leftFilter, expression) - if (left.isDefined) left else findExpression(rightFilter, expression) - } - case f @ ComparisonFilter(_, _, predicate) => - if (predicate == expression) Some(f) else None - case _ => None - } -} - -abstract private[parquet] class CatalystFilter( - @transient val predicate: CatalystPredicate) extends FilterPredicate - -private[parquet] case class ComparisonFilter( - val columnName: String, - private var filter: FilterPredicate, - @transient override val predicate: CatalystPredicate) - extends CatalystFilter(predicate) { - override def accept[R](visitor: FilterPredicate.Visitor[R]): R = { - filter.accept(visitor) - } -} - -private[parquet] case class OrFilter( - private var filter: FilterPredicate, - @transient val left: CatalystFilter, - @transient val right: CatalystFilter, - @transient override val predicate: Or) - extends CatalystFilter(predicate) { - def this(l: CatalystFilter, r: CatalystFilter) = - this( - FilterApi.or(l, r), - l, - r, - Or(l.predicate, r.predicate)) - - override def accept[R](visitor: FilterPredicate.Visitor[R]): R = { - filter.accept(visitor); - } - -} - -private[parquet] case class AndFilter( - private var filter: FilterPredicate, - @transient val left: CatalystFilter, - @transient val right: CatalystFilter, - @transient override val predicate: And) - extends CatalystFilter(predicate) { - def this(l: CatalystFilter, r: CatalystFilter) = - this( - FilterApi.and(l, r), - l, - r, - And(l.predicate, r.predicate)) - - override def accept[R](visitor: FilterPredicate.Visitor[R]): R = { - filter.accept(visitor); - } - -} - -private[parquet] object ComparisonFilter { - def createBooleanEqualityFilter( - columnName: String, - value: Boolean, - predicate: CatalystPredicate): CatalystFilter = - new ComparisonFilter( - columnName, - FilterApi.eq(booleanColumn(columnName), value.asInstanceOf[java.lang.Boolean]), - predicate) - - def createStringEqualityFilter( - columnName: String, - value: String, - predicate: CatalystPredicate): CatalystFilter = - new ComparisonFilter( - columnName, - FilterApi.eq(binaryColumn(columnName), Binary.fromString(value)), - predicate) - - def createStringLessThanFilter( - columnName: String, - value: String, - predicate: CatalystPredicate): CatalystFilter = - new ComparisonFilter( - columnName, - FilterApi.lt(binaryColumn(columnName), Binary.fromString(value)), - predicate) - - def createStringLessThanOrEqualFilter( - columnName: String, - value: String, - predicate: CatalystPredicate): CatalystFilter = - new ComparisonFilter( - columnName, - FilterApi.ltEq(binaryColumn(columnName), Binary.fromString(value)), - predicate) - - def createStringGreaterThanFilter( - columnName: String, - value: String, - predicate: CatalystPredicate): CatalystFilter = - new ComparisonFilter( - columnName, - FilterApi.gt(binaryColumn(columnName), Binary.fromString(value)), - predicate) - - def createStringGreaterThanOrEqualFilter( - columnName: String, - value: String, - predicate: CatalystPredicate): CatalystFilter = - new ComparisonFilter( - columnName, - FilterApi.gtEq(binaryColumn(columnName), Binary.fromString(value)), - predicate) - - def createBinaryEqualityFilter( - columnName: String, - value: Array[Byte], - predicate: CatalystPredicate): CatalystFilter = - new ComparisonFilter( - columnName, - FilterApi.eq(binaryColumn(columnName), Binary.fromByteArray(value)), - predicate) - - def createBinaryLessThanFilter( - columnName: String, - value: Array[Byte], - predicate: CatalystPredicate): CatalystFilter = - new ComparisonFilter( - columnName, - FilterApi.lt(binaryColumn(columnName), Binary.fromByteArray(value)), - predicate) - - def createBinaryLessThanOrEqualFilter( - columnName: String, - value: Array[Byte], - predicate: CatalystPredicate): CatalystFilter = - new ComparisonFilter( - columnName, - FilterApi.ltEq(binaryColumn(columnName), Binary.fromByteArray(value)), - predicate) - - def createBinaryGreaterThanFilter( - columnName: String, - value: Array[Byte], - predicate: CatalystPredicate): CatalystFilter = - new ComparisonFilter( - columnName, - FilterApi.gt(binaryColumn(columnName), Binary.fromByteArray(value)), - predicate) - - def createBinaryGreaterThanOrEqualFilter( - columnName: String, - value: Array[Byte], - predicate: CatalystPredicate): CatalystFilter = - new ComparisonFilter( - columnName, - FilterApi.gtEq(binaryColumn(columnName), Binary.fromByteArray(value)), - predicate) -} - -private[spark] object ParquetColumns { - - def byteColumn(columnPath: String): ByteColumn = { - new ByteColumn(ColumnPath.fromDotString(columnPath)) - } - - final class ByteColumn(columnPath: ColumnPath) - extends Column[java.lang.Byte](columnPath, classOf[java.lang.Byte]) with SupportsLtGt - - def shortColumn(columnPath: String): ShortColumn = { - new ShortColumn(ColumnPath.fromDotString(columnPath)) - } - - final class ShortColumn(columnPath: ColumnPath) - extends Column[java.lang.Short](columnPath, classOf[java.lang.Short]) with SupportsLtGt - - - def dateColumn(columnPath: String): DateColumn = { - new DateColumn(ColumnPath.fromDotString(columnPath)) - } - - final class DateColumn(columnPath: ColumnPath) - extends Column[WrappedDate](columnPath, classOf[WrappedDate]) with SupportsLtGt - - def timestampColumn(columnPath: String): TimestampColumn = { - new TimestampColumn(ColumnPath.fromDotString(columnPath)) - } - - final class TimestampColumn(columnPath: ColumnPath) - extends Column[WrappedTimestamp](columnPath, classOf[WrappedTimestamp]) with SupportsLtGt - - def decimalColumn(columnPath: String): DecimalColumn = { - new DecimalColumn(ColumnPath.fromDotString(columnPath)) - } - - final class DecimalColumn(columnPath: ColumnPath) - extends Column[Decimal](columnPath, classOf[Decimal]) with SupportsLtGt - - final class WrappedDate(val date: Date) extends Comparable[WrappedDate] { - - override def compareTo(other: WrappedDate): Int = { - date.compareTo(other.date) - } - } - - final class WrappedTimestamp(val timestamp: Timestamp) extends Comparable[WrappedTimestamp] { - - override def compareTo(other: WrappedTimestamp): Int = { - timestamp.compareTo(other.timestamp) - } - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index f6bed5016fbfb..5d0643a64a044 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -23,8 +23,6 @@ import java.text.SimpleDateFormat import java.util.concurrent.{Callable, TimeUnit} import java.util.{ArrayList, Collections, Date, List => JList} -import org.apache.spark.annotation.DeveloperApi - import scala.collection.JavaConversions._ import scala.collection.mutable import scala.util.Try @@ -34,22 +32,20 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{BlockLocation, FileStatus, Path} import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} -import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} -import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter - +import org.apache.hadoop.mapreduce.lib.output.{FileOutputCommitter, FileOutputFormat => NewFileOutputFormat} import parquet.hadoop._ +import parquet.hadoop.api.ReadSupport.ReadContext import parquet.hadoop.api.{InitContext, ReadSupport} import parquet.hadoop.metadata.GlobalMetaData -import parquet.hadoop.api.ReadSupport.ReadContext import parquet.hadoop.util.ContextUtil import parquet.io.ParquetDecodingException import parquet.schema.MessageType +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.SQLConf -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Row} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Row, _} import org.apache.spark.sql.execution.{LeafNode, SparkPlan, UnaryNode} import org.apache.spark.{Logging, SerializableWritable, TaskContext} @@ -82,8 +78,6 @@ case class ParquetTableScan( override def execute(): RDD[Row] = { import parquet.filter2.compat.FilterCompat.FilterPredicateCompat - import parquet.filter2.compat.FilterCompat.Filter - import parquet.filter2.predicate.FilterPredicate val sc = sqlContext.sparkContext val job = new Job(sc.hadoopConfiguration) @@ -111,14 +105,11 @@ case class ParquetTableScan( // Note 1: the input format ignores all predicates that cannot be expressed // as simple column predicate filters in Parquet. Here we just record // the whole pruning predicate. - if (columnPruningPred.length > 0) { + ParquetFilters + .createRecordFilter(columnPruningPred) + .map(_.asInstanceOf[FilterPredicateCompat].getFilterPredicate) // Set this in configuration of ParquetInputFormat, needed for RowGroupFiltering - val filter: Filter = ParquetFilters.createRecordFilter(columnPruningPred) - if (filter != null){ - val filterPredicate = filter.asInstanceOf[FilterPredicateCompat].getFilterPredicate - ParquetInputFormat.setFilterPredicate(conf, filterPredicate) - } - } + .foreach(ParquetInputFormat.setFilterPredicate(conf, _)) // Tell FilteringParquetRowInputFormat whether it's okay to cache Parquet and FS metadata conf.set( @@ -317,7 +308,7 @@ case class InsertIntoParquetTable( } writer.close(hadoopContext) committer.commitTask(hadoopContext) - return 1 + 1 } val jobFormat = new AppendingParquetOutputFormat(taskIdOffset) /* apparently we need a TaskAttemptID to construct an OutputCommitter; @@ -375,9 +366,8 @@ private[parquet] class FilteringParquetRowInputFormat override def createRecordReader( inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): RecordReader[Void, Row] = { - + import parquet.filter2.compat.FilterCompat.NoOpFilter - import parquet.filter2.compat.FilterCompat.Filter val readSupport: ReadSupport[Row] = new RowReadSupport() @@ -392,7 +382,7 @@ private[parquet] class FilteringParquetRowInputFormat } override def getFooters(jobContext: JobContext): JList[Footer] = { - import FilteringParquetRowInputFormat.footerCache + import org.apache.spark.sql.parquet.FilteringParquetRowInputFormat.footerCache if (footers eq null) { val conf = ContextUtil.getConfiguration(jobContext) @@ -442,13 +432,13 @@ private[parquet] class FilteringParquetRowInputFormat val taskSideMetaData = configuration.getBoolean(ParquetInputFormat.TASK_SIDE_METADATA, true) val maxSplitSize: JLong = configuration.getLong("mapred.max.split.size", Long.MaxValue) val minSplitSize: JLong = - Math.max(getFormatMinSplitSize(), configuration.getLong("mapred.min.split.size", 0L)) + Math.max(getFormatMinSplitSize, configuration.getLong("mapred.min.split.size", 0L)) if (maxSplitSize < 0 || minSplitSize < 0) { throw new ParquetDecodingException( s"maxSplitSize or minSplitSie should not be negative: maxSplitSize = $maxSplitSize;" + s" minSplitSize = $minSplitSize") } - + // Uses strict type checking by default val getGlobalMetaData = classOf[ParquetFileWriter].getDeclaredMethod("getGlobalMetaData", classOf[JList[Footer]]) @@ -458,29 +448,29 @@ private[parquet] class FilteringParquetRowInputFormat if (globalMetaData == null) { val splits = mutable.ArrayBuffer.empty[ParquetInputSplit] return splits - } - + } + val readContext = getReadSupport(configuration).init( new InitContext(configuration, - globalMetaData.getKeyValueMetaData(), - globalMetaData.getSchema())) - + globalMetaData.getKeyValueMetaData, + globalMetaData.getSchema)) + if (taskSideMetaData){ logInfo("Using Task Side Metadata Split Strategy") - return getTaskSideSplits(configuration, + getTaskSideSplits(configuration, footers, maxSplitSize, minSplitSize, readContext) } else { logInfo("Using Client Side Metadata Split Strategy") - return getClientSideSplits(configuration, + getClientSideSplits(configuration, footers, maxSplitSize, minSplitSize, readContext) } - + } def getClientSideSplits( @@ -489,12 +479,11 @@ private[parquet] class FilteringParquetRowInputFormat maxSplitSize: JLong, minSplitSize: JLong, readContext: ReadContext): JList[ParquetInputSplit] = { - - import FilteringParquetRowInputFormat.blockLocationCache - import parquet.filter2.compat.FilterCompat; - import parquet.filter2.compat.FilterCompat.Filter; - import parquet.filter2.compat.RowGroupFilter; - + + import parquet.filter2.compat.FilterCompat.Filter + import parquet.filter2.compat.RowGroupFilter + import org.apache.spark.sql.parquet.FilteringParquetRowInputFormat.blockLocationCache + val cacheMetadata = configuration.getBoolean(SQLConf.PARQUET_CACHE_METADATA, true) val splits = mutable.ArrayBuffer.empty[ParquetInputSplit] @@ -503,7 +492,7 @@ private[parquet] class FilteringParquetRowInputFormat var totalRowGroups: Long = 0 // Ugly hack, stuck with it until PR: - // https://github.com/apache/incubator-parquet-mr/pull/17 + // https://github.com/apache/incubator-parquet-mr/pull/17 // is resolved val generateSplits = Class.forName("parquet.hadoop.ClientSideMetadataSplitStrategy") @@ -523,7 +512,7 @@ private[parquet] class FilteringParquetRowInputFormat blocks, parquetMetaData.getFileMetaData.getSchema) rowGroupsDropped = rowGroupsDropped + (blocks.size - filteredBlocks.size) - + if (!filteredBlocks.isEmpty){ var blockLocations: Array[BlockLocation] = null if (!cacheMetadata) { @@ -566,7 +555,7 @@ private[parquet] class FilteringParquetRowInputFormat readContext: ReadContext): JList[ParquetInputSplit] = { val splits = mutable.ArrayBuffer.empty[ParquetInputSplit] - + // Ugly hack, stuck with it until PR: // https://github.com/apache/incubator-parquet-mr/pull/17 // is resolved @@ -576,7 +565,7 @@ private[parquet] class FilteringParquetRowInputFormat sys.error( s"Failed to reflectively invoke TaskSideMetadataSplitStrategy.generateTaskSideMDSplits")) generateSplits.setAccessible(true) - + for (footer <- footers) { val file = footer.getFile val fs = file.getFileSystem(configuration) @@ -594,7 +583,7 @@ private[parquet] class FilteringParquetRowInputFormat } splits - } + } } @@ -636,11 +625,9 @@ private[parquet] object FileSystemHelper { files.map(_.getName).map { case nameP(taskid) => taskid.toInt case hiddenFileP() => 0 - case other: String => { + case other: String => sys.error("ERROR: attempting to append to set of Parquet files and found file" + s"that does not match name pattern: $other") - 0 - } case _ => 0 }.reduceLeft((a, b) => if (a < b) b else a) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 80a3e0b4c91ae..d31a9d8418dee 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -17,11 +17,13 @@ package org.apache.spark.sql.parquet +import _root_.parquet.filter2.predicate.{FilterPredicate, Operators} import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.mapreduce.Job import org.scalatest.{BeforeAndAfterAll, FunSuiteLike} import parquet.hadoop.ParquetFileWriter import parquet.hadoop.util.ContextUtil + import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types.IntegerType @@ -447,44 +449,24 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(true) } - test("create RecordFilter for simple predicates") { - val attribute1 = new AttributeReference("first", IntegerType, false)() - val predicate1 = new EqualTo(attribute1, new Literal(1, IntegerType)) - val filter1 = ParquetFilters.createFilter(predicate1) - assert(filter1.isDefined) - assert(filter1.get.predicate == predicate1, "predicates do not match") - assert(filter1.get.isInstanceOf[ComparisonFilter]) - val cmpFilter1 = filter1.get.asInstanceOf[ComparisonFilter] - assert(cmpFilter1.columnName == "first", "column name incorrect") - - val predicate2 = new LessThan(attribute1, new Literal(4, IntegerType)) - val filter2 = ParquetFilters.createFilter(predicate2) - assert(filter2.isDefined) - assert(filter2.get.predicate == predicate2, "predicates do not match") - assert(filter2.get.isInstanceOf[ComparisonFilter]) - val cmpFilter2 = filter2.get.asInstanceOf[ComparisonFilter] - assert(cmpFilter2.columnName == "first", "column name incorrect") - - val predicate3 = new And(predicate1, predicate2) - val filter3 = ParquetFilters.createFilter(predicate3) - assert(filter3.isDefined) - assert(filter3.get.predicate == predicate3, "predicates do not match") - assert(filter3.get.isInstanceOf[AndFilter]) - - val predicate4 = new Or(predicate1, predicate2) - val filter4 = ParquetFilters.createFilter(predicate4) - assert(filter4.isDefined) - assert(filter4.get.predicate == predicate4, "predicates do not match") - assert(filter4.get.isInstanceOf[OrFilter]) - - val attribute2 = new AttributeReference("second", IntegerType, false)() - val predicate5 = new GreaterThan(attribute1, attribute2) - val badfilter = ParquetFilters.createFilter(predicate5) - assert(badfilter.isDefined === false) - - val predicate6 = And(GreaterThan(attribute1, attribute2), GreaterThan(attribute1, attribute2)) - val badfilter2 = ParquetFilters.createFilter(predicate6) - assert(badfilter2.isDefined === false) + test("make RecordFilter for simple predicates") { + def checkFilter[T <: FilterPredicate](predicate: Expression, defined: Boolean = true): Unit = { + val filter = ParquetFilters.createFilter(predicate) + if (defined) { + assert(filter.isDefined) + assert(filter.get.isInstanceOf[T]) + } else { + assert(filter.isEmpty) + } + } + + checkFilter[Operators.Eq[Integer]]('a.int === 1) + checkFilter[Operators.Lt[Integer]]('a.int < 4) + checkFilter[Operators.And]('a.int === 1 && 'a.int < 4) + checkFilter[Operators.Or]('a.int === 1 || 'a.int < 4) + + checkFilter('a.int > 'b.int, defined = false) + checkFilter(('a.int > 'b.int) && ('a.int > 'b.int), defined = false) } test("test filter by predicate pushdown") { From c6e0c2ab1c29c184a9302d23ad75e4ccd8060242 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 17 Nov 2014 21:07:50 -0800 Subject: [PATCH 017/109] SPARK-4466: Provide support for publishing Scala 2.11 artifacts to Maven The maven release plug-in does not have support for publishing two separate sets of artifacts for a single release. Because of the way that Scala 2.11 support in Spark works, we have to write some customized code to do this. The good news is that the Maven release API is just a thin wrapper on doing git commits and pushing artifacts to the HTTP API of Apache's Sonatype server and this might overall make our deployment easier to understand. This was already used for the 1.2 snapshot, so I think it is working well. One other nice thing is this could be pretty easily extended to publish nightly snapshots. Author: Patrick Wendell Closes #3332 from pwendell/releases and squashes the following commits: 2fedaed [Patrick Wendell] Automate the opening and closing of Sonatype repos e2a24bb [Patrick Wendell] Fixing issue where we overrode non-spark version numbers 9df3a50 [Patrick Wendell] Adding TODO 1cc1749 [Patrick Wendell] Don't build the thriftserver for 2.11 933201a [Patrick Wendell] Make tagging of release commit eager d0388a6 [Patrick Wendell] Support Scala 2.11 build 4f4dc62 [Patrick Wendell] Change to 2.11 should not be included when committing new patch bf742e1 [Patrick Wendell] Minor fixes ffa1df2 [Patrick Wendell] Adding a Scala 2.11 package to test it 9ac4381 [Patrick Wendell] Addressing TODO b3105ff [Patrick Wendell] Removing commented out code d906803 [Patrick Wendell] Small fix 3f4d985 [Patrick Wendell] More work fcd54c2 [Patrick Wendell] Consolidating use of keys df2af30 [Patrick Wendell] Changes to release stuff --- .../spark/api/java/function/package.scala | 2 +- dev/create-release/create-release.sh | 138 +++++++++++++----- 2 files changed, 106 insertions(+), 34 deletions(-) diff --git a/core/src/main/java/org/apache/spark/api/java/function/package.scala b/core/src/main/java/org/apache/spark/api/java/function/package.scala index 7f91de653a64a..0f9bac7164162 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/package.scala +++ b/core/src/main/java/org/apache/spark/api/java/function/package.scala @@ -22,4 +22,4 @@ package org.apache.spark.api.java * these interfaces to pass functions to various Java API methods for Spark. Please visit Spark's * Java programming guide for more details. */ -package object function \ No newline at end of file +package object function diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index a6e90a15ee84b..8a0b0348db8c0 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -28,13 +28,19 @@ # - Send output to stderr and have useful logging in stdout # Note: The following variables must be set before use! -GIT_USERNAME=${GIT_USERNAME:-pwendell} -GIT_PASSWORD=${GIT_PASSWORD:-XXX} +ASF_USERNAME=${ASF_USERNAME:-pwendell} +ASF_PASSWORD=${ASF_PASSWORD:-XXX} GPG_PASSPHRASE=${GPG_PASSPHRASE:-XXX} GIT_BRANCH=${GIT_BRANCH:-branch-1.0} -RELEASE_VERSION=${RELEASE_VERSION:-1.0.0} +RELEASE_VERSION=${RELEASE_VERSION:-1.2.0} +NEXT_VERSION=${NEXT_VERSION:-1.2.1} RC_NAME=${RC_NAME:-rc2} -USER_NAME=${USER_NAME:-pwendell} + +M2_REPO=~/.m2/repository +SPARK_REPO=$M2_REPO/org/apache/spark +NEXUS_ROOT=https://repository.apache.org/service/local/staging +NEXUS_UPLOAD=$NEXUS_ROOT/deploy/maven2 +NEXUS_PROFILE=d63f592e7eac0 # Profile for Spark staging uploads if [ -z "$JAVA_HOME" ]; then echo "Error: JAVA_HOME is not set, cannot proceed." @@ -47,31 +53,90 @@ set -e GIT_TAG=v$RELEASE_VERSION-$RC_NAME if [[ ! "$@" =~ --package-only ]]; then - echo "Creating and publishing release" + echo "Creating release commit and publishing to Apache repository" # Artifact publishing - git clone https://git-wip-us.apache.org/repos/asf/spark.git -b $GIT_BRANCH - cd spark + git clone https://$ASF_USERNAME:$ASF_PASSWORD@git-wip-us.apache.org/repos/asf/spark.git \ + -b $GIT_BRANCH + pushd spark export MAVEN_OPTS="-Xmx3g -XX:MaxPermSize=1g -XX:ReservedCodeCacheSize=1g" - mvn -Pyarn release:clean - - mvn -DskipTests \ - -Darguments="-DskipTests=true -Dmaven.javadoc.skip=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ - -Dusername=$GIT_USERNAME -Dpassword=$GIT_PASSWORD \ - -Dmaven.javadoc.skip=true \ - -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Dtag=$GIT_TAG -DautoVersionSubmodules=true \ - -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \ - --batch-mode release:prepare - - mvn -DskipTests \ - -Darguments="-DskipTests=true -Dmaven.javadoc.skip=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ - -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Dmaven.javadoc.skip=true \ + # Create release commits and push them to github + # NOTE: This is done "eagerly" i.e. we don't check if we can succesfully build + # or before we coin the release commit. This helps avoid races where + # other people add commits to this branch while we are in the middle of building. + old=" ${RELEASE_VERSION}-SNAPSHOT<\/version>" + new=" ${RELEASE_VERSION}<\/version>" + find . -name pom.xml -o -name package.scala | grep -v dev | xargs -I {} sed -i \ + -e "s/$old/$new/" {} + git commit -a -m "Preparing Spark release $GIT_TAG" + echo "Creating tag $GIT_TAG at the head of $GIT_BRANCH" + git tag $GIT_TAG + + old=" ${RELEASE_VERSION}<\/version>" + new=" ${NEXT_VERSION}-SNAPSHOT<\/version>" + find . -name pom.xml -o -name package.scala | grep -v dev | xargs -I {} sed -i \ + -e "s/$old/$new/" {} + git commit -a -m "Preparing development version ${NEXT_VERSION}-SNAPSHOT" + git push origin $GIT_TAG + git push origin HEAD:$GIT_BRANCH + git checkout -f $GIT_TAG + + # Using Nexus API documented here: + # https://support.sonatype.com/entries/39720203-Uploading-to-a-Staging-Repository-via-REST-API + echo "Creating Nexus staging repository" + repo_request="Apache Spark $GIT_TAG" + out=$(curl -X POST -d "$repo_request" -u $ASF_USERNAME:$ASF_PASSWORD \ + -H "Content-Type:application/xml" -v \ + $NEXUS_ROOT/profiles/$NEXUS_PROFILE/start) + staged_repo_id=$(echo $out | sed -e "s/.*\(orgapachespark-[0-9]\{4\}\).*/\1/") + echo "Created Nexus staging repository: $staged_repo_id" + + rm -rf $SPARK_REPO + + mvn -DskipTests -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \ - release:perform + clean install - cd .. + ./dev/change-version-to-2.11.sh + + mvn -DskipTests -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ + -Dscala-2.11 -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \ + clean install + + ./dev/change-version-to-2.10.sh + + pushd $SPARK_REPO + + # Remove any extra files generated during install + find . -type f |grep -v \.jar |grep -v \.pom | xargs rm + + echo "Creating hash and signature files" + for file in $(find . -type f) + do + echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --output $file.asc --detach-sig --armour $file; + gpg --print-md MD5 $file > $file.md5; + gpg --print-md SHA1 $file > $file.sha1 + done + + echo "Uplading files to $NEXUS_UPLOAD" + for file in $(find . -type f) + do + # strip leading ./ + file_short=$(echo $file | sed -e "s/\.\///") + dest_url="$NEXUS_UPLOAD/org/apache/spark/$file_short" + echo " Uploading $file_short" + curl -u $ASF_USERNAME:$ASF_PASSWORD --upload-file $file_short $dest_url + done + + echo "Closing nexus staging repository" + repo_request="$staged_repo_idApache Spark $GIT_TAG" + out=$(curl -X POST -d "$repo_request" -u $ASF_USERNAME:$ASF_PASSWORD \ + -H "Content-Type:application/xml" -v \ + $NEXUS_ROOT/profiles/$NEXUS_PROFILE/finish) + echo "Closed Nexus staging repository: $staged_repo_id" + + popd + popd rm -rf spark fi @@ -102,6 +167,12 @@ make_binary_release() { cp -r spark spark-$RELEASE_VERSION-bin-$NAME cd spark-$RELEASE_VERSION-bin-$NAME + + # TODO There should probably be a flag to make-distribution to allow 2.11 support + if [[ $FLAGS == *scala-2.11* ]]; then + ./dev/change-version-to-2.11.sh + fi + ./make-distribution.sh --name $NAME --tgz $FLAGS 2>&1 | tee ../binary-release-$NAME.log cd .. cp spark-$RELEASE_VERSION-bin-$NAME/spark-$RELEASE_VERSION-bin-$NAME.tgz . @@ -118,11 +189,12 @@ make_binary_release() { spark-$RELEASE_VERSION-bin-$NAME.tgz.sha } + make_binary_release "hadoop1" "-Phive -Phive-thriftserver -Dhadoop.version=1.0.4" & +make_binary_release "hadoop1-scala2.11" "-Phive -Dscala-2.11" & make_binary_release "cdh4" "-Phive -Phive-thriftserver -Dhadoop.version=2.0.0-mr1-cdh4.2.0" & make_binary_release "hadoop2.3" "-Phadoop-2.3 -Phive -Phive-thriftserver -Pyarn" & make_binary_release "hadoop2.4" "-Phadoop-2.4 -Phive -Phive-thriftserver -Pyarn" & -make_binary_release "hadoop2.4-without-hive" "-Phadoop-2.4 -Pyarn" & make_binary_release "mapr3" "-Pmapr3 -Phive -Phive-thriftserver" & make_binary_release "mapr4" "-Pmapr4 -Pyarn -Phive -Phive-thriftserver" & wait @@ -130,10 +202,10 @@ wait # Copy data echo "Copying release tarballs" rc_folder=spark-$RELEASE_VERSION-$RC_NAME -ssh $USER_NAME@people.apache.org \ - mkdir /home/$USER_NAME/public_html/$rc_folder +ssh $ASF_USERNAME@people.apache.org \ + mkdir /home/$ASF_USERNAME/public_html/$rc_folder scp spark-* \ - $USER_NAME@people.apache.org:/home/$USER_NAME/public_html/$rc_folder/ + $ASF_USERNAME@people.apache.org:/home/$ASF_USERNAME/public_html/$rc_folder/ # Docs cd spark @@ -143,12 +215,12 @@ cd docs JAVA_HOME=$JAVA_7_HOME PRODUCTION=1 jekyll build echo "Copying release documentation" rc_docs_folder=${rc_folder}-docs -ssh $USER_NAME@people.apache.org \ - mkdir /home/$USER_NAME/public_html/$rc_docs_folder -rsync -r _site/* $USER_NAME@people.apache.org:/home/$USER_NAME/public_html/$rc_docs_folder +ssh $ASF_USERNAME@people.apache.org \ + mkdir /home/$ASF_USERNAME/public_html/$rc_docs_folder +rsync -r _site/* $ASF_USERNAME@people.apache.org:/home/$ASF_USERNAME/public_html/$rc_docs_folder echo "Release $RELEASE_VERSION completed:" echo "Git tag:\t $GIT_TAG" echo "Release commit:\t $release_hash" -echo "Binary location:\t http://people.apache.org/~$USER_NAME/$rc_folder" -echo "Doc location:\t http://people.apache.org/~$USER_NAME/$rc_docs_folder" +echo "Binary location:\t http://people.apache.org/~$ASF_USERNAME/$rc_folder" +echo "Doc location:\t http://people.apache.org/~$ASF_USERNAME/$rc_docs_folder" From cedc3b5aa43a16e2da62f12a36317f00aa1002cc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Felix=20Maximilian=20M=C3=B6ller?= Date: Tue, 18 Nov 2014 10:08:24 -0800 Subject: [PATCH 018/109] ALS implicit: added missing parameter alpha in doc string MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Author: Felix Maximilian Möller Closes #3343 from felixmaximilian/fix-documentation and squashes the following commits: 43dcdfb [Felix Maximilian Möller] Removed the information about the switch implicitPrefs. The parameter implicitPrefs cannot be set in this context because it is inherent true when calling the trainImplicit method. 7d172ba [Felix Maximilian Möller] added missing parameter alpha in doc string. --- .../scala/org/apache/spark/mllib/recommendation/ALS.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 038edc3521f14..90ac252226006 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -746,7 +746,7 @@ object ALS { * @param iterations number of iterations of ALS (recommended: 10-20) * @param lambda regularization factor (recommended: 0.01) * @param blocks level of parallelism to split computation into - * @param alpha confidence parameter (only applies when immplicitPrefs = true) + * @param alpha confidence parameter * @param seed random seed */ def trainImplicit( @@ -773,7 +773,7 @@ object ALS { * @param iterations number of iterations of ALS (recommended: 10-20) * @param lambda regularization factor (recommended: 0.01) * @param blocks level of parallelism to split computation into - * @param alpha confidence parameter (only applies when immplicitPrefs = true) + * @param alpha confidence parameter */ def trainImplicit( ratings: RDD[Rating], @@ -797,6 +797,7 @@ object ALS { * @param rank number of features to use * @param iterations number of iterations of ALS (recommended: 10-20) * @param lambda regularization factor (recommended: 0.01) + * @param alpha confidence parameter */ def trainImplicit(ratings: RDD[Rating], rank: Int, iterations: Int, lambda: Double, alpha: Double) : MatrixFactorizationModel = { From 8fbf72b7903b5bbec8d949151aa4693b4af26ff5 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 18 Nov 2014 10:11:13 -0800 Subject: [PATCH 019/109] [SPARK-4435] [MLlib] [PySpark] improve classification This PR add setThrehold() and clearThreshold() for LogisticRegressionModel and SVMModel, also support RDD of vector in LogisticRegressionModel.predict(), SVNModel.predict() and NaiveBayes.predict() Author: Davies Liu Closes #3305 from davies/setThreshold and squashes the following commits: d0b835f [Davies Liu] Merge branch 'master' of github.com:apache/spark into setThreshold e4acd76 [Davies Liu] address comments 2231a5f [Davies Liu] bugfix 7bd9009 [Davies Liu] address comments 0b0a8a7 [Davies Liu] address comments c1e5573 [Davies Liu] improve classification --- .../classification/LogisticRegression.scala | 2 +- .../spark/mllib/classification/SVM.scala | 2 +- python/pyspark/mllib/classification.py | 135 ++++++++++++++---- 3 files changed, 108 insertions(+), 31 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 18b95f1edc0b0..94d757bc317ab 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -64,7 +64,7 @@ class LogisticRegressionModel ( val margin = weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept val score = 1.0 / (1.0 + math.exp(-margin)) threshold match { - case Some(t) => if (score < t) 0.0 else 1.0 + case Some(t) => if (score > t) 1.0 else 0.0 case None => score } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index ab9515b2a6db8..dd514ff8a37f2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -65,7 +65,7 @@ class SVMModel ( intercept: Double) = { val margin = weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept threshold match { - case Some(t) => if (margin < t) 0.0 else 1.0 + case Some(t) => if (margin > t) 1.0 else 0.0 case None => margin } } diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index b654813fb4cf6..ee0729b1eb656 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -20,6 +20,7 @@ import numpy from numpy import array +from pyspark import RDD from pyspark.mllib.common import callMLlibFunc from pyspark.mllib.linalg import SparseVector, _convert_to_vector from pyspark.mllib.regression import LabeledPoint, LinearModel, _regression_train_wrapper @@ -29,39 +30,88 @@ 'SVMWithSGD', 'NaiveBayesModel', 'NaiveBayes'] -class LogisticRegressionModel(LinearModel): +class LinearBinaryClassificationModel(LinearModel): + """ + Represents a linear binary classification model that predicts to whether an + example is positive (1.0) or negative (0.0). + """ + def __init__(self, weights, intercept): + super(LinearBinaryClassificationModel, self).__init__(weights, intercept) + self._threshold = None + + def setThreshold(self, value): + """ + :: Experimental :: + + Sets the threshold that separates positive predictions from negative + predictions. An example with prediction score greater than or equal + to this threshold is identified as an positive, and negative otherwise. + """ + self._threshold = value + + def clearThreshold(self): + """ + :: Experimental :: + + Clears the threshold so that `predict` will output raw prediction scores. + """ + self._threshold = None + + def predict(self, test): + """ + Predict values for a single data point or an RDD of points using + the model trained. + """ + raise NotImplementedError + + +class LogisticRegressionModel(LinearBinaryClassificationModel): """A linear binary classification model derived from logistic regression. >>> data = [ - ... LabeledPoint(0.0, [0.0]), - ... LabeledPoint(1.0, [1.0]), - ... LabeledPoint(1.0, [2.0]), - ... LabeledPoint(1.0, [3.0]) + ... LabeledPoint(0.0, [0.0, 1.0]), + ... LabeledPoint(1.0, [1.0, 0.0]), ... ] >>> lrm = LogisticRegressionWithSGD.train(sc.parallelize(data)) - >>> lrm.predict(array([1.0])) > 0 - True - >>> lrm.predict(array([0.0])) <= 0 - True + >>> lrm.predict([1.0, 0.0]) + 1 + >>> lrm.predict([0.0, 1.0]) + 0 + >>> lrm.predict(sc.parallelize([[1.0, 0.0], [0.0, 1.0]])).collect() + [1, 0] + >>> lrm.clearThreshold() + >>> lrm.predict([0.0, 1.0]) + 0.123... + >>> sparse_data = [ ... LabeledPoint(0.0, SparseVector(2, {0: 0.0})), ... LabeledPoint(1.0, SparseVector(2, {1: 1.0})), - ... LabeledPoint(0.0, SparseVector(2, {0: 0.0})), + ... LabeledPoint(0.0, SparseVector(2, {0: 1.0})), ... LabeledPoint(1.0, SparseVector(2, {1: 2.0})) ... ] >>> lrm = LogisticRegressionWithSGD.train(sc.parallelize(sparse_data)) - >>> lrm.predict(array([0.0, 1.0])) > 0 - True - >>> lrm.predict(array([0.0, 0.0])) <= 0 - True - >>> lrm.predict(SparseVector(2, {1: 1.0})) > 0 - True - >>> lrm.predict(SparseVector(2, {1: 0.0})) <= 0 - True + >>> lrm.predict(array([0.0, 1.0])) + 1 + >>> lrm.predict(array([1.0, 0.0])) + 0 + >>> lrm.predict(SparseVector(2, {1: 1.0})) + 1 + >>> lrm.predict(SparseVector(2, {0: 1.0})) + 0 """ + def __init__(self, weights, intercept): + super(LogisticRegressionModel, self).__init__(weights, intercept) + self._threshold = 0.5 def predict(self, x): + """ + Predict values for a single data point or an RDD of points using + the model trained. + """ + if isinstance(x, RDD): + return x.map(lambda v: self.predict(v)) + x = _convert_to_vector(x) margin = self.weights.dot(x) + self._intercept if margin > 0: @@ -69,7 +119,10 @@ def predict(self, x): else: exp_margin = exp(margin) prob = exp_margin / (1 + exp_margin) - return 1 if prob > 0.5 else 0 + if self._threshold is None: + return prob + else: + return 1 if prob > self._threshold else 0 class LogisticRegressionWithSGD(object): @@ -111,7 +164,7 @@ def train(rdd, i): return _regression_train_wrapper(train, LogisticRegressionModel, data, initialWeights) -class SVMModel(LinearModel): +class SVMModel(LinearBinaryClassificationModel): """A support vector machine. @@ -122,8 +175,14 @@ class SVMModel(LinearModel): ... LabeledPoint(1.0, [3.0]) ... ] >>> svm = SVMWithSGD.train(sc.parallelize(data)) - >>> svm.predict(array([1.0])) > 0 - True + >>> svm.predict([1.0]) + 1 + >>> svm.predict(sc.parallelize([[1.0]])).collect() + [1] + >>> svm.clearThreshold() + >>> svm.predict(array([1.0])) + 1.25... + >>> sparse_data = [ ... LabeledPoint(0.0, SparseVector(2, {0: -1.0})), ... LabeledPoint(1.0, SparseVector(2, {1: 1.0})), @@ -131,16 +190,29 @@ class SVMModel(LinearModel): ... LabeledPoint(1.0, SparseVector(2, {1: 2.0})) ... ] >>> svm = SVMWithSGD.train(sc.parallelize(sparse_data)) - >>> svm.predict(SparseVector(2, {1: 1.0})) > 0 - True - >>> svm.predict(SparseVector(2, {0: -1.0})) <= 0 - True + >>> svm.predict(SparseVector(2, {1: 1.0})) + 1 + >>> svm.predict(SparseVector(2, {0: -1.0})) + 0 """ + def __init__(self, weights, intercept): + super(SVMModel, self).__init__(weights, intercept) + self._threshold = 0.0 def predict(self, x): + """ + Predict values for a single data point or an RDD of points using + the model trained. + """ + if isinstance(x, RDD): + return x.map(lambda v: self.predict(v)) + x = _convert_to_vector(x) margin = self.weights.dot(x) + self.intercept - return 1 if margin >= 0 else 0 + if self._threshold is None: + return margin + else: + return 1 if margin > self._threshold else 0 class SVMWithSGD(object): @@ -201,6 +273,8 @@ class NaiveBayesModel(object): 0.0 >>> model.predict(array([1.0, 0.0])) 1.0 + >>> model.predict(sc.parallelize([[1.0, 0.0]])).collect() + [1.0] >>> sparse_data = [ ... LabeledPoint(0.0, SparseVector(2, {1: 0.0})), ... LabeledPoint(0.0, SparseVector(2, {1: 1.0})), @@ -219,7 +293,9 @@ def __init__(self, labels, pi, theta): self.theta = theta def predict(self, x): - """Return the most likely class for a data vector x""" + """Return the most likely class for a data vector or an RDD of vectors""" + if isinstance(x, RDD): + return x.map(lambda v: self.predict(v)) x = _convert_to_vector(x) return self.labels[numpy.argmax(self.pi + x.dot(self.theta.transpose()))] @@ -250,7 +326,8 @@ def train(cls, data, lambda_=1.0): def _test(): import doctest from pyspark import SparkContext - globs = globals().copy() + import pyspark.mllib.classification + globs = pyspark.mllib.classification.__dict__.copy() globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) globs['sc'].stop() From b54c6ab3c54e65238d6766832ea1f3fcd694f2fd Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 18 Nov 2014 10:35:29 -0800 Subject: [PATCH 020/109] [SPARK-4396] allow lookup by index in Python's Rating In PySpark, ALS can take an RDD of (user, product, rating) tuples as input. However, model.predict outputs an RDD of Rating. So on the input side, users can use r[0], r[1], r[2], while on the output side, users have to use r.user, r.product, r.rating. We should allow lookup by index in Rating by making Rating a namedtuple. davies [Review on Reviewable](https://reviewable.io/reviews/apache/spark/3261) Author: Xiangrui Meng Closes #3261 from mengxr/SPARK-4396 and squashes the following commits: 543aef0 [Xiangrui Meng] use named tuple to implement ALS 0b61bae [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into SPARK-4396 d3bd7d4 [Xiangrui Meng] allow lookup by index in Python's Rating --- python/pyspark/mllib/recommendation.py | 26 +++++++++++++++----------- 1 file changed, 15 insertions(+), 11 deletions(-) diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index 41bbd9a779c70..2bcbf2aaf8e3e 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -15,24 +15,28 @@ # limitations under the License. # +from collections import namedtuple + from pyspark import SparkContext from pyspark.rdd import RDD from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc, _to_java_object_rdd -__all__ = ['MatrixFactorizationModel', 'ALS'] +__all__ = ['MatrixFactorizationModel', 'ALS', 'Rating'] -class Rating(object): - def __init__(self, user, product, rating): - self.user = int(user) - self.product = int(product) - self.rating = float(rating) +class Rating(namedtuple("Rating", ["user", "product", "rating"])): + """ + Represents a (user, product, rating) tuple. - def __reduce__(self): - return Rating, (self.user, self.product, self.rating) + >>> r = Rating(1, 2, 5.0) + >>> (r.user, r.product, r.rating) + (1, 2, 5.0) + >>> (r[0], r[1], r[2]) + (1, 2, 5.0) + """ - def __repr__(self): - return "Rating(%d, %d, %s)" % (self.user, self.product, self.rating) + def __reduce__(self): + return Rating, (int(self.user), int(self.product), float(self.rating)) class MatrixFactorizationModel(JavaModelWrapper): @@ -51,7 +55,7 @@ class MatrixFactorizationModel(JavaModelWrapper): >>> testset = sc.parallelize([(1, 2), (1, 1)]) >>> model = ALS.train(ratings, 1, seed=10) >>> model.predictAll(testset).collect() - [Rating(1, 1, 1.0471...), Rating(1, 2, 1.9679...)] + [Rating(user=1, product=1, rating=1.0471...), Rating(user=1, product=2, rating=1.9679...)] >>> model = ALS.train(ratings, 4, seed=10) >>> model.userFeatures().collect() From 90d72ec8502f7ec11d2fe42f08c884ad2159266f Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 18 Nov 2014 12:13:23 -0800 Subject: [PATCH 021/109] [SQL] Support partitioned parquet tables that have the key in both the directory and the file Author: Michael Armbrust Closes #3272 from marmbrus/keyInPartitionedTable and squashes the following commits: 447f08c [Michael Armbrust] Support partitioned parquet tables that have the key in both the directory and the file --- .../spark/sql/hive/HiveStrategies.scala | 18 +- .../sql/parquet/ParquetMetastoreSuite.scala | 158 +++++++++++------- 2 files changed, 108 insertions(+), 68 deletions(-) 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 3a49dddd858d9..56fc85239e1c0 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 @@ -58,12 +58,18 @@ private[hive] trait HiveStrategies { def lowerCase = new SchemaRDD(s.sqlContext, s.logicalPlan) - def addPartitioningAttributes(attrs: Seq[Attribute]) = - new SchemaRDD( - s.sqlContext, - s.logicalPlan transform { - case p: ParquetRelation => p.copy(partitioningAttributes = attrs) - }) + def addPartitioningAttributes(attrs: Seq[Attribute]) = { + // Don't add the partitioning key if its already present in the data. + if (attrs.map(_.name).toSet.subsetOf(s.logicalPlan.output.map(_.name).toSet)) { + s + } else { + new SchemaRDD( + s.sqlContext, + s.logicalPlan transform { + case p: ParquetRelation => p.copy(partitioningAttributes = attrs) + }) + } + } } implicit class PhysicalPlanHacks(originalPlan: SparkPlan) { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala index 86adbbf3ad2d8..cc65242c0da9b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala @@ -27,7 +27,11 @@ import org.apache.spark.sql.QueryTest import org.apache.spark.sql.hive.execution.HiveTableScan import org.apache.spark.sql.hive.test.TestHive._ +// The data where the partitioning key exists only in the directory structure. case class ParquetData(intField: Int, stringField: String) +// The data that also includes the partitioning key +case class ParquetDataWithKey(p: Int, intField: Int, stringField: String) + /** * Tests for our SerDe -> Native parquet scan conversion. @@ -45,6 +49,17 @@ class ParquetMetastoreSuite extends QueryTest with BeforeAndAfterAll { .saveAsParquetFile(partDir.getCanonicalPath) } + val partitionedTableDirWithKey = File.createTempFile("parquettests", "sparksql") + partitionedTableDirWithKey.delete() + partitionedTableDirWithKey.mkdir() + + (1 to 10).foreach { p => + val partDir = new File(partitionedTableDirWithKey, s"p=$p") + sparkContext.makeRDD(1 to 10) + .map(i => ParquetDataWithKey(p, i, s"part-$p")) + .saveAsParquetFile(partDir.getCanonicalPath) + } + sql(s""" create external table partitioned_parquet ( @@ -59,6 +74,20 @@ class ParquetMetastoreSuite extends QueryTest with BeforeAndAfterAll { location '${partitionedTableDir.getCanonicalPath}' """) + sql(s""" + create external table partitioned_parquet_with_key + ( + intField INT, + stringField STRING + ) + PARTITIONED BY (p int) + ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + STORED AS + INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + location '${partitionedTableDirWithKey.getCanonicalPath}' + """) + sql(s""" create external table normal_parquet ( @@ -76,6 +105,10 @@ class ParquetMetastoreSuite extends QueryTest with BeforeAndAfterAll { sql(s"ALTER TABLE partitioned_parquet ADD PARTITION (p=$p)") } + (1 to 10).foreach { p => + sql(s"ALTER TABLE partitioned_parquet_with_key ADD PARTITION (p=$p)") + } + setConf("spark.sql.hive.convertMetastoreParquet", "true") } @@ -83,75 +116,76 @@ class ParquetMetastoreSuite extends QueryTest with BeforeAndAfterAll { setConf("spark.sql.hive.convertMetastoreParquet", "false") } - test("project the partitioning column") { - checkAnswer( - sql("SELECT p, count(*) FROM partitioned_parquet group by p"), - (1, 10) :: - (2, 10) :: - (3, 10) :: - (4, 10) :: - (5, 10) :: - (6, 10) :: - (7, 10) :: - (8, 10) :: - (9, 10) :: - (10, 10) :: Nil - ) - } + Seq("partitioned_parquet", "partitioned_parquet_with_key").foreach { table => + test(s"project the partitioning column $table") { + checkAnswer( + sql(s"SELECT p, count(*) FROM $table group by p"), + (1, 10) :: + (2, 10) :: + (3, 10) :: + (4, 10) :: + (5, 10) :: + (6, 10) :: + (7, 10) :: + (8, 10) :: + (9, 10) :: + (10, 10) :: Nil + ) + } - test("project partitioning and non-partitioning columns") { - checkAnswer( - sql("SELECT stringField, p, count(intField) " + - "FROM partitioned_parquet GROUP BY p, stringField"), - ("part-1", 1, 10) :: - ("part-2", 2, 10) :: - ("part-3", 3, 10) :: - ("part-4", 4, 10) :: - ("part-5", 5, 10) :: - ("part-6", 6, 10) :: - ("part-7", 7, 10) :: - ("part-8", 8, 10) :: - ("part-9", 9, 10) :: - ("part-10", 10, 10) :: Nil - ) - } + test(s"project partitioning and non-partitioning columns $table") { + checkAnswer( + sql(s"SELECT stringField, p, count(intField) FROM $table GROUP BY p, stringField"), + ("part-1", 1, 10) :: + ("part-2", 2, 10) :: + ("part-3", 3, 10) :: + ("part-4", 4, 10) :: + ("part-5", 5, 10) :: + ("part-6", 6, 10) :: + ("part-7", 7, 10) :: + ("part-8", 8, 10) :: + ("part-9", 9, 10) :: + ("part-10", 10, 10) :: Nil + ) + } - test("simple count") { - checkAnswer( - sql("SELECT COUNT(*) FROM partitioned_parquet"), - 100) - } + test(s"simple count $table") { + checkAnswer( + sql(s"SELECT COUNT(*) FROM $table"), + 100) + } - test("pruned count") { - checkAnswer( - sql("SELECT COUNT(*) FROM partitioned_parquet WHERE p = 1"), - 10) - } + test(s"pruned count $table") { + checkAnswer( + sql(s"SELECT COUNT(*) FROM $table WHERE p = 1"), + 10) + } - test("multi-partition pruned count") { - checkAnswer( - sql("SELECT COUNT(*) FROM partitioned_parquet WHERE p IN (1,2,3)"), - 30) - } + test(s"multi-partition pruned count $table") { + checkAnswer( + sql(s"SELECT COUNT(*) FROM $table WHERE p IN (1,2,3)"), + 30) + } - test("non-partition predicates") { - checkAnswer( - sql("SELECT COUNT(*) FROM partitioned_parquet WHERE intField IN (1,2,3)"), - 30) - } + test(s"non-partition predicates $table") { + checkAnswer( + sql(s"SELECT COUNT(*) FROM $table WHERE intField IN (1,2,3)"), + 30) + } - test("sum") { - checkAnswer( - sql("SELECT SUM(intField) FROM partitioned_parquet WHERE intField IN (1,2,3) AND p = 1"), - 1 + 2 + 3) - } + test(s"sum $table") { + checkAnswer( + sql(s"SELECT SUM(intField) FROM $table WHERE intField IN (1,2,3) AND p = 1"), + 1 + 2 + 3) + } - test("hive udfs") { - checkAnswer( - sql("SELECT concat(stringField, stringField) FROM partitioned_parquet"), - sql("SELECT stringField FROM partitioned_parquet").map { - case Row(s: String) => Row(s + s) - }.collect().toSeq) + test(s"hive udfs $table") { + checkAnswer( + sql(s"SELECT concat(stringField, stringField) FROM $table"), + sql(s"SELECT stringField FROM $table").map { + case Row(s: String) => Row(s + s) + }.collect().toSeq) + } } test("non-part select(*)") { From bfebfd8b28eeb7e75292333f7885aa0830fcb5fe Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 18 Nov 2014 12:17:33 -0800 Subject: [PATCH 022/109] [SPARK-4075][SPARK-4434] Fix the URI validation logic for Application Jar name. This PR adds a regression test for SPARK-4434. Author: Kousuke Saruta Closes #3326 from sarutak/add-triple-slash-testcase and squashes the following commits: 82bc9cc [Kousuke Saruta] Fixed wrong grammar in comment 9149027 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into add-triple-slash-testcase c1c80ca [Kousuke Saruta] Fixed style 4f30210 [Kousuke Saruta] Modified comments 9e09da2 [Kousuke Saruta] Fixed URI validation for jar file d4b99ef [Kousuke Saruta] [SPARK-4075] [Deploy] Jar url validation is not enough for Jar file ac79906 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into add-triple-slash-testcase 6d4f47e [Kousuke Saruta] Added a test case as a regression check for SPARK-4434 --- .../apache/spark/deploy/ClientArguments.scala | 14 ++++++++++++-- .../org/apache/spark/deploy/ClientSuite.scala | 17 ++++++++++++++++- 2 files changed, 28 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala index 39150deab863c..2e1e52906ceeb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala @@ -17,6 +17,8 @@ package org.apache.spark.deploy +import java.net.{URI, URISyntaxException} + import scala.collection.mutable.ListBuffer import org.apache.log4j.Level @@ -73,7 +75,8 @@ private[spark] class ClientArguments(args: Array[String]) { if (!ClientArguments.isValidJarUrl(_jarUrl)) { println(s"Jar url '${_jarUrl}' is not in valid format.") - println(s"Must be a jar file path in URL format (e.g. hdfs://XX.jar, file://XX.jar)") + println(s"Must be a jar file path in URL format " + + "(e.g. hdfs://host:port/XX.jar, file:///XX.jar)") printUsageAndExit(-1) } @@ -114,5 +117,12 @@ private[spark] class ClientArguments(args: Array[String]) { } object ClientArguments { - def isValidJarUrl(s: String): Boolean = s.matches("(.+):(.+)jar") + def isValidJarUrl(s: String): Boolean = { + try { + val uri = new URI(s) + uri.getScheme != null && uri.getPath != null && uri.getPath.endsWith(".jar") + } catch { + case _: URISyntaxException => false + } + } } diff --git a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala index 4161aede1d1d0..d2dae34be7bfb 100644 --- a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala @@ -23,12 +23,27 @@ import org.scalatest.Matchers class ClientSuite extends FunSuite with Matchers { test("correctly validates driver jar URL's") { ClientArguments.isValidJarUrl("http://someHost:8080/foo.jar") should be (true) - ClientArguments.isValidJarUrl("file://some/path/to/a/jarFile.jar") should be (true) + + // file scheme with authority and path is valid. + ClientArguments.isValidJarUrl("file://somehost/path/to/a/jarFile.jar") should be (true) + + // file scheme without path is not valid. + // In this case, jarFile.jar is recognized as authority. + ClientArguments.isValidJarUrl("file://jarFile.jar") should be (false) + + // file scheme without authority but with triple slash is valid. + ClientArguments.isValidJarUrl("file:///some/path/to/a/jarFile.jar") should be (true) ClientArguments.isValidJarUrl("hdfs://someHost:1234/foo.jar") should be (true) ClientArguments.isValidJarUrl("hdfs://someHost:1234/foo") should be (false) ClientArguments.isValidJarUrl("/missing/a/protocol/jarfile.jar") should be (false) ClientArguments.isValidJarUrl("not-even-a-path.jar") should be (false) + + // This URI doesn't have authority and path. + ClientArguments.isValidJarUrl("hdfs:someHost:1234/jarfile.jar") should be (false) + + // Invalid syntax. + ClientArguments.isValidJarUrl("hdfs:") should be (false) } } From 80f31778820586a93d73fa15279a204611cc3c60 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 18 Nov 2014 13:11:38 -0800 Subject: [PATCH 023/109] [SPARK-4404] remove sys.exit() in shutdown hook If SparkSubmit die first, then bootstrapper will be blocked by shutdown hook. sys.exit() in a shutdown hook will cause some kind of dead lock. cc andrewor14 Author: Davies Liu Closes #3289 from davies/fix_bootstraper and squashes the following commits: ea5cdd1 [Davies Liu] Merge branch 'master' of github.com:apache/spark into fix_bootstraper e04b690 [Davies Liu] remove sys.exit in hook 4d11366 [Davies Liu] remove shutdown hook if subprocess die fist --- .../org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala index aa3743ca7df63..d2687faad62b1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala @@ -134,7 +134,7 @@ private[spark] object SparkSubmitDriverBootstrapper { override def run() = { if (process != null) { process.destroy() - sys.exit(process.waitFor()) + process.waitFor() } } }) From e34f38ff1a0dfbb0ffa4bd11071e03b1a58de998 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 18 Nov 2014 13:37:21 -0800 Subject: [PATCH 024/109] [SPARK-4017] show progress bar in console The progress bar will look like this: ![1___spark_job__85_250_finished__4_are_running___java_](https://cloud.githubusercontent.com/assets/40902/4854813/a02f44ac-6099-11e4-9060-7c73a73151d6.png) In the right corner, the numbers are: finished tasks, running tasks, total tasks. After the stage has finished, it will disappear. The progress bar is only showed if logging level is WARN or higher (but progress in title is still showed), it can be turned off by spark.driver.showConsoleProgress. Author: Davies Liu Closes #3029 from davies/progress and squashes the following commits: 95336d5 [Davies Liu] Merge branch 'master' of github.com:apache/spark into progress fc49ac8 [Davies Liu] address commentse 2e90f75 [Davies Liu] show multiple stages in same time 0081bcc [Davies Liu] address comments 38c42f1 [Davies Liu] fix tests ab87958 [Davies Liu] disable progress bar during tests 30ac852 [Davies Liu] re-implement progress bar b3f34e5 [Davies Liu] Merge branch 'master' of github.com:apache/spark into progress 6fd30ff [Davies Liu] show progress bar if no task finished in 500ms e4e7344 [Davies Liu] refactor e1f524d [Davies Liu] revert unnecessary change a60477c [Davies Liu] Merge branch 'master' of github.com:apache/spark into progress 5cae3f2 [Davies Liu] fix style ea49fe0 [Davies Liu] address comments bc53d99 [Davies Liu] refactor e6bb189 [Davies Liu] fix logging in sparkshell 7e7d4e7 [Davies Liu] address commments 5df26bb [Davies Liu] fix style 9e42208 [Davies Liu] show progress bar in console and title --- bin/spark-submit | 3 + .../java/org/apache/spark/SparkStageInfo.java | 1 + .../scala/org/apache/spark/SparkContext.scala | 10 +- .../org/apache/spark/SparkStatusTracker.scala | 1 + .../org/apache/spark/StatusAPIImpl.scala | 1 + .../apache/spark/ui/ConsoleProgressBar.scala | 124 ++++++++++++++++++ pom.xml | 1 + project/SparkBuild.scala | 1 + 8 files changed, 141 insertions(+), 1 deletion(-) create mode 100644 core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala diff --git a/bin/spark-submit b/bin/spark-submit index c557311b4b20e..f92d90c3a66b0 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -22,6 +22,9 @@ export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" ORIG_ARGS=("$@") +# Set COLUMNS for progress bar +export COLUMNS=`tput cols` + while (($#)); do if [ "$1" = "--deploy-mode" ]; then SPARK_SUBMIT_DEPLOY_MODE=$2 diff --git a/core/src/main/java/org/apache/spark/SparkStageInfo.java b/core/src/main/java/org/apache/spark/SparkStageInfo.java index 04e2247210ecc..fd74321093658 100644 --- a/core/src/main/java/org/apache/spark/SparkStageInfo.java +++ b/core/src/main/java/org/apache/spark/SparkStageInfo.java @@ -26,6 +26,7 @@ public interface SparkStageInfo { int stageId(); int currentAttemptId(); + long submissionTime(); String name(); int numTasks(); int numActiveTasks(); diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 7cccf74003431..37013121c572a 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -50,7 +50,7 @@ import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SparkD import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import org.apache.spark.scheduler.local.LocalBackend import org.apache.spark.storage._ -import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.{SparkUI, ConsoleProgressBar} import org.apache.spark.ui.jobs.JobProgressListener import org.apache.spark.util._ @@ -245,6 +245,13 @@ class SparkContext(config: SparkConf) extends Logging { val statusTracker = new SparkStatusTracker(this) + private[spark] val progressBar: Option[ConsoleProgressBar] = + if (conf.getBoolean("spark.ui.showConsoleProgress", true) && !log.isInfoEnabled) { + Some(new ConsoleProgressBar(this)) + } else { + None + } + // Initialize the Spark UI private[spark] val ui: Option[SparkUI] = if (conf.getBoolean("spark.ui.enabled", true)) { @@ -1274,6 +1281,7 @@ class SparkContext(config: SparkConf) extends Logging { logInfo("Starting job: " + callSite.shortForm) dagScheduler.runJob(rdd, cleanedFunc, partitions, callSite, allowLocal, resultHandler, localProperties.get) + progressBar.foreach(_.finishAll()) rdd.doCheckpoint() } diff --git a/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala b/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala index c18d763d7ff4d..edbdda8a0bcb6 100644 --- a/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala +++ b/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala @@ -96,6 +96,7 @@ class SparkStatusTracker private[spark] (sc: SparkContext) { new SparkStageInfoImpl( stageId, info.attemptId, + info.submissionTime.getOrElse(0), info.name, info.numTasks, data.numActiveTasks, diff --git a/core/src/main/scala/org/apache/spark/StatusAPIImpl.scala b/core/src/main/scala/org/apache/spark/StatusAPIImpl.scala index 90b47c847fbca..e5c7c8d0db578 100644 --- a/core/src/main/scala/org/apache/spark/StatusAPIImpl.scala +++ b/core/src/main/scala/org/apache/spark/StatusAPIImpl.scala @@ -26,6 +26,7 @@ private class SparkJobInfoImpl ( private class SparkStageInfoImpl( val stageId: Int, val currentAttemptId: Int, + val submissionTime: Long, val name: String, val numTasks: Int, val numActiveTasks: Int, diff --git a/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala b/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala new file mode 100644 index 0000000000000..27ba9e18237b5 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala @@ -0,0 +1,124 @@ +/* + * 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.ui + +import java.util.{Timer, TimerTask} + +import org.apache.spark._ + +/** + * ConsoleProgressBar shows the progress of stages in the next line of the console. It poll the + * status of active stages from `sc.statusTracker` periodically, the progress bar will be showed + * up after the stage has ran at least 500ms. If multiple stages run in the same time, the status + * of them will be combined together, showed in one line. + */ +private[spark] class ConsoleProgressBar(sc: SparkContext) extends Logging { + + // Carrige return + val CR = '\r' + // Update period of progress bar, in milliseconds + val UPDATE_PERIOD = 200L + // Delay to show up a progress bar, in milliseconds + val FIRST_DELAY = 500L + + // The width of terminal + val TerminalWidth = if (!sys.env.getOrElse("COLUMNS", "").isEmpty) { + sys.env.get("COLUMNS").get.toInt + } else { + 80 + } + + var lastFinishTime = 0L + var lastUpdateTime = 0L + var lastProgressBar = "" + + // Schedule a refresh thread to run periodically + private val timer = new Timer("refresh progress", true) + timer.schedule(new TimerTask{ + override def run() { + refresh() + } + }, FIRST_DELAY, UPDATE_PERIOD) + + /** + * Try to refresh the progress bar in every cycle + */ + private def refresh(): Unit = synchronized { + val now = System.currentTimeMillis() + if (now - lastFinishTime < FIRST_DELAY) { + return + } + val stageIds = sc.statusTracker.getActiveStageIds() + val stages = stageIds.map(sc.statusTracker.getStageInfo).flatten.filter(_.numTasks() > 1) + .filter(now - _.submissionTime() > FIRST_DELAY).sortBy(_.stageId()) + if (stages.size > 0) { + show(now, stages.take(3)) // display at most 3 stages in same time + } + } + + /** + * Show progress bar in console. The progress bar is displayed in the next line + * after your last output, keeps overwriting itself to hold in one line. The logging will follow + * the progress bar, then progress bar will be showed in next line without overwrite logs. + */ + private def show(now: Long, stages: Seq[SparkStageInfo]) { + val width = TerminalWidth / stages.size + val bar = stages.map { s => + val total = s.numTasks() + val header = s"[Stage ${s.stageId()}:" + val tailer = s"(${s.numCompletedTasks()} + ${s.numActiveTasks()}) / $total]" + val w = width - header.size - tailer.size + val bar = if (w > 0) { + val percent = w * s.numCompletedTasks() / total + (0 until w).map { i => + if (i < percent) "=" else if (i == percent) ">" else " " + }.mkString("") + } else { + "" + } + header + bar + tailer + }.mkString("") + + // only refresh if it's changed of after 1 minute (or the ssh connection will be closed + // after idle some time) + if (bar != lastProgressBar || now - lastUpdateTime > 60 * 1000L) { + System.err.print(CR + bar) + lastUpdateTime = now + } + lastProgressBar = bar + } + + /** + * Clear the progress bar if showed. + */ + private def clear() { + if (!lastProgressBar.isEmpty) { + System.err.printf(CR + " " * TerminalWidth + CR) + lastProgressBar = "" + } + } + + /** + * Mark all the stages as finished, clear the progress bar if showed, then the progress will not + * interweave with output of jobs. + */ + def finishAll(): Unit = synchronized { + clear() + lastFinishTime = System.currentTimeMillis() + } +} diff --git a/pom.xml b/pom.xml index cc7bce175778f..94c4422743df3 100644 --- a/pom.xml +++ b/pom.xml @@ -977,6 +977,7 @@ ${session.executionRootDirectory} 1 false + false ${test_classpath} true diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 1697b6d4f2d43..c1879ce4ba0eb 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -377,6 +377,7 @@ object TestSettings { javaOptions in Test += "-Dspark.testing=1", javaOptions in Test += "-Dspark.port.maxRetries=100", javaOptions in Test += "-Dspark.ui.enabled=false", + javaOptions in Test += "-Dspark.ui.showConsoleProgress=false", javaOptions in Test += "-Dspark.driver.allowMultipleContexts=true", javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true", javaOptions in Test ++= System.getProperties.filter(_._1 startsWith "spark") From 010bc86e40a0e54b6850b75abd6105e70eb1af10 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Tue, 18 Nov 2014 15:01:06 -0800 Subject: [PATCH 025/109] [SPARK-4463] Add (de)select all button for add'l metrics. This commit removes the behavior where when a user clicks "Show additional metrics" on the stage page, all of the additional metrics are automatically selected; now, collapsing and expanding the additional metrics has no effect on which options are selected. Instead, there's a "(De)select All" box at the top; checking this box checks all additional metrics (and similarly, unchecking it unchecks all additional metrics). This commit is intended to be backported to 1.2, so that the additional metrics behavior is not confusing to users. Now when a user clicks the "Show additional metrics" menu, this is what it looks like: ![image](https://cloud.githubusercontent.com/assets/1108612/5094347/1541ead6-6f15-11e4-8e8c-25a65ddbdfb2.png) Author: Kay Ousterhout Closes #3331 from kayousterhout/SPARK-4463 and squashes the following commits: 9e17cea [Kay Ousterhout] Added italics b731230 [Kay Ousterhout] [SPARK-4463] Add (de)select all button for add'l metrics. --- .../spark/ui/static/additional-metrics.js | 17 ++++++++++------- .../org/apache/spark/ui/jobs/StagePage.scala | 4 ++++ 2 files changed, 14 insertions(+), 7 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js b/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js index badd85ed48c82..d33c5c769d683 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js +++ b/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js @@ -26,13 +26,6 @@ $(function() { // Switch the class of the arrow from open to closed. $(this).find('.expand-additional-metrics-arrow').toggleClass('arrow-open'); $(this).find('.expand-additional-metrics-arrow').toggleClass('arrow-closed'); - - // If clicking caused the metrics to expand, automatically check all options for additional - // metrics (don't trigger a click when collapsing metrics, because it leads to weird - // toggling behavior). - if (!$(additionalMetricsDiv).hasClass('collapsed')) { - $(this).parent().find('input:checkbox:not(:checked)').trigger('click'); - } }); $("input:checkbox:not(:checked)").each(function() { @@ -48,6 +41,16 @@ $(function() { stripeTables(); }); + $("#select-all-metrics").click(function() { + if (this.checked) { + // Toggle all un-checked options. + $('input:checkbox:not(:checked)').trigger('click'); + } else { + // Toggle all checked options. + $('input:checkbox:checked').trigger('click'); + } + }); + // Trigger a click on the checkbox if a user clicks the label next to it. $("span.additional-metric-title").click(function() { $(this).parent().find('input:checkbox').trigger('click'); 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 16bc3f6c18d09..36afc4942e085 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 @@ -114,6 +114,10 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { @@ -283,4 +288,24 @@ private[spark] object UIUtils extends Logging { } + + def makeProgressBar( + started: Int, + completed: Int, + failed: Int, + skipped:Int, + total: Int): Seq[Node] = { + val completeWidth = "width: %s%%".format((completed.toDouble/total)*100) + val startWidth = "width: %s%%".format((started.toDouble/total)*100) + +
+ + {completed}/{total} + { if (failed > 0) s"($failed failed)" } + { if (skipped > 0) s"($skipped skipped)" } + +
+
+
+ } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala new file mode 100644 index 0000000000000..ea2d187a0e8e4 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -0,0 +1,151 @@ +/* + * 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.ui.jobs + +import scala.xml.{Node, NodeSeq} + +import javax.servlet.http.HttpServletRequest + +import org.apache.spark.JobExecutionStatus +import org.apache.spark.ui.{WebUIPage, UIUtils} +import org.apache.spark.ui.jobs.UIData.JobUIData + +/** Page showing list of all ongoing and recently finished jobs */ +private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { + private val startTime: Option[Long] = parent.sc.map(_.startTime) + private val listener = parent.listener + + private def jobsTable(jobs: Seq[JobUIData]): Seq[Node] = { + val someJobHasJobGroup = jobs.exists(_.jobGroup.isDefined) + + val columns: Seq[Node] = { + {if (someJobHasJobGroup) "Job Id (Job Group)" else "Job Id"} + Description + Submitted + Duration + Stages: Succeeded/Total + Tasks (for all stages): Succeeded/Total + } + + def makeRow(job: JobUIData): Seq[Node] = { + val lastStageInfo = listener.stageIdToInfo.get(job.stageIds.max) + val lastStageData = lastStageInfo.flatMap { s => + listener.stageIdToData.get((s.stageId, s.attemptId)) + } + val isComplete = job.status == JobExecutionStatus.SUCCEEDED + val lastStageName = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)") + val lastStageDescription = lastStageData.flatMap(_.description).getOrElse("") + val duration: Option[Long] = { + job.startTime.map { start => + val end = job.endTime.getOrElse(System.currentTimeMillis()) + end - start + } + } + val formattedDuration = duration.map(d => UIUtils.formatDuration(d)).getOrElse("Unknown") + val formattedSubmissionTime = job.startTime.map(UIUtils.formatDate).getOrElse("Unknown") + val detailUrl = + "%s/jobs/job?id=%s".format(UIUtils.prependBaseUri(parent.basePath), job.jobId) + + + {job.jobId} {job.jobGroup.map(id => s"($id)").getOrElse("")} + + +
{lastStageDescription}
+ {lastStageName} + + + {formattedSubmissionTime} + + {formattedDuration} + + {job.completedStageIndices.size}/{job.stageIds.size - job.numSkippedStages} + {if (job.numFailedStages > 0) s"(${job.numFailedStages} failed)"} + {if (job.numSkippedStages > 0) s"(${job.numSkippedStages} skipped)"} + + + {UIUtils.makeProgressBar(started = job.numActiveTasks, completed = job.numCompletedTasks, + failed = job.numFailedTasks, skipped = job.numSkippedTasks, + total = job.numTasks - job.numSkippedTasks)} + + + } + + + {columns} + + {jobs.map(makeRow)} + +
+ } + + def render(request: HttpServletRequest): Seq[Node] = { + listener.synchronized { + val activeJobs = listener.activeJobs.values.toSeq + val completedJobs = listener.completedJobs.reverse.toSeq + val failedJobs = listener.failedJobs.reverse.toSeq + val now = System.currentTimeMillis + + val activeJobsTable = + jobsTable(activeJobs.sortBy(_.startTime.getOrElse(-1L)).reverse) + val completedJobsTable = + jobsTable(completedJobs.sortBy(_.endTime.getOrElse(-1L)).reverse) + val failedJobsTable = + jobsTable(failedJobs.sortBy(_.endTime.getOrElse(-1L)).reverse) + + val summary: NodeSeq = +
+
    + {if (startTime.isDefined) { + // Total duration is not meaningful unless the UI is live +
  • + Total Duration: + {UIUtils.formatDuration(now - startTime.get)} +
  • + }} +
  • + Scheduling Mode: + {listener.schedulingMode.map(_.toString).getOrElse("Unknown")} +
  • +
  • + Active Jobs: + {activeJobs.size} +
  • +
  • + Completed Jobs: + {completedJobs.size} +
  • +
  • + Failed Jobs: + {failedJobs.size} +
  • +
+
+ + val content = summary ++ +

Active Jobs ({activeJobs.size})

++ activeJobsTable ++ +

Completed Jobs ({completedJobs.size})

++ completedJobsTable ++ +

Failed Jobs ({failedJobs.size})

++ failedJobsTable + + val helpText = """A job is triggered by a action, like "count()" or "saveAsTextFile()".""" + + " Click on a job's title to see information about the stages of tasks associated with" + + " the job." + + UIUtils.headerSparkPage("Spark Jobs", content, parent, helpText = Some(helpText)) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala similarity index 87% rename from core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala rename to core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala index 83a7898071c9b..b0f8ca2ab0d3f 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala @@ -25,7 +25,7 @@ import org.apache.spark.scheduler.Schedulable import org.apache.spark.ui.{WebUIPage, UIUtils} /** Page showing list of all ongoing and recently finished stages and pools */ -private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("") { +private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { private val sc = parent.sc private val listener = parent.listener private def isFairScheduler = parent.isFairScheduler @@ -41,11 +41,14 @@ private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("") val activeStagesTable = new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, - parent, parent.killEnabled) + parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler, + killEnabled = parent.killEnabled) val completedStagesTable = - new StageTableBase(completedStages.sortBy(_.submissionTime).reverse, parent) + new StageTableBase(completedStages.sortBy(_.submissionTime).reverse, parent.basePath, + parent.listener, isFairScheduler = parent.isFairScheduler, killEnabled = false) val failedStagesTable = - new FailedStageTable(failedStages.sortBy(_.submissionTime).reverse, parent) + new FailedStageTable(failedStages.sortBy(_.submissionTime).reverse, parent.basePath, + parent.listener, isFairScheduler = parent.isFairScheduler) // For now, pool information is only accessible in live UIs val pools = sc.map(_.getAllPools).getOrElse(Seq.empty[Schedulable]) @@ -93,7 +96,7 @@ private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("")

Failed Stages ({numFailedStages})

++ failedStagesTable.toNodeSeq - UIUtils.headerSparkPage("Spark Stages", content, parent) + UIUtils.headerSparkPage("Spark Stages (for all jobs)", content, parent) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index fa0f96bff34ff..35bbe8b4f9ac8 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -25,7 +25,7 @@ import org.apache.spark.ui.jobs.UIData.StageUIData import org.apache.spark.util.Utils /** Stage summary grouped by executors. */ -private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: JobProgressTab) { +private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: StagesTab) { private val listener = parent.listener def toNodeSeq: Seq[Node] = { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala new file mode 100644 index 0000000000000..77d36209c6048 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala @@ -0,0 +1,177 @@ +/* + * 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.ui.jobs + +import scala.collection.mutable +import scala.xml.{NodeSeq, Node} + +import javax.servlet.http.HttpServletRequest + +import org.apache.spark.JobExecutionStatus +import org.apache.spark.scheduler.StageInfo +import org.apache.spark.ui.{UIUtils, WebUIPage} + +/** Page showing statistics and stage list for a given job */ +private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { + private val listener = parent.listener + + def render(request: HttpServletRequest): Seq[Node] = { + listener.synchronized { + val jobId = request.getParameter("id").toInt + val jobDataOption = listener.jobIdToData.get(jobId) + if (jobDataOption.isEmpty) { + val content = +
+

No information to display for job {jobId}

+
+ return UIUtils.headerSparkPage( + s"Details for Job $jobId", content, parent) + } + val jobData = jobDataOption.get + val isComplete = jobData.status != JobExecutionStatus.RUNNING + val stages = jobData.stageIds.map { stageId => + // This could be empty if the JobProgressListener hasn't received information about the + // stage or if the stage information has been garbage collected + listener.stageIdToInfo.getOrElse(stageId, + new StageInfo(stageId, 0, "Unknown", 0, Seq.empty, "Unknown")) + } + + val activeStages = mutable.Buffer[StageInfo]() + val completedStages = mutable.Buffer[StageInfo]() + // If the job is completed, then any pending stages are displayed as "skipped": + val pendingOrSkippedStages = mutable.Buffer[StageInfo]() + val failedStages = mutable.Buffer[StageInfo]() + for (stage <- stages) { + if (stage.submissionTime.isEmpty) { + pendingOrSkippedStages += stage + } else if (stage.completionTime.isDefined) { + if (stage.failureReason.isDefined) { + failedStages += stage + } else { + completedStages += stage + } + } else { + activeStages += stage + } + } + + val activeStagesTable = + new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, + parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler, + killEnabled = parent.killEnabled) + val pendingOrSkippedStagesTable = + new StageTableBase(pendingOrSkippedStages.sortBy(_.stageId).reverse, + parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler, + killEnabled = false) + val completedStagesTable = + new StageTableBase(completedStages.sortBy(_.submissionTime).reverse, parent.basePath, + parent.listener, isFairScheduler = parent.isFairScheduler, killEnabled = false) + val failedStagesTable = + new FailedStageTable(failedStages.sortBy(_.submissionTime).reverse, parent.basePath, + parent.listener, isFairScheduler = parent.isFairScheduler) + + val shouldShowActiveStages = activeStages.nonEmpty + val shouldShowPendingStages = !isComplete && pendingOrSkippedStages.nonEmpty + val shouldShowCompletedStages = completedStages.nonEmpty + val shouldShowSkippedStages = isComplete && pendingOrSkippedStages.nonEmpty + val shouldShowFailedStages = failedStages.nonEmpty + + val summary: NodeSeq = +
+
    +
  • + Status: + {jobData.status} +
  • + { + if (jobData.jobGroup.isDefined) { +
  • + Job Group: + {jobData.jobGroup.get} +
  • + } + } + { + if (shouldShowActiveStages) { +
  • + Active Stages: + {activeStages.size} +
  • + } + } + { + if (shouldShowPendingStages) { +
  • + + Pending Stages: + {pendingOrSkippedStages.size} +
  • + } + } + { + if (shouldShowCompletedStages) { +
  • + Completed Stages: + {completedStages.size} +
  • + } + } + { + if (shouldShowSkippedStages) { +
  • + Skipped Stages: + {pendingOrSkippedStages.size} +
  • + } + } + { + if (shouldShowFailedStages) { +
  • + Failed Stages: + {failedStages.size} +
  • + } + } +
+
+ + var content = summary + if (shouldShowActiveStages) { + content ++=

Active Stages ({activeStages.size})

++ + activeStagesTable.toNodeSeq + } + if (shouldShowPendingStages) { + content ++=

Pending Stages ({pendingOrSkippedStages.size})

++ + pendingOrSkippedStagesTable.toNodeSeq + } + if (shouldShowCompletedStages) { + content ++=

Completed Stages ({completedStages.size})

++ + completedStagesTable.toNodeSeq + } + if (shouldShowSkippedStages) { + content ++=

Skipped Stages ({pendingOrSkippedStages.size})

++ + pendingOrSkippedStagesTable.toNodeSeq + } + if (shouldShowFailedStages) { + content ++=

Failed Stages ({failedStages.size})

++ + failedStagesTable.toNodeSeq + } + UIUtils.headerSparkPage(s"Details for Job $jobId", content, parent) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index ccdcf0e047f48..72935beb3a34a 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -17,7 +17,7 @@ package org.apache.spark.ui.jobs -import scala.collection.mutable.{HashMap, ListBuffer} +import scala.collection.mutable.{HashMap, HashSet, ListBuffer} import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi @@ -49,8 +49,6 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { type PoolName = String type ExecutorId = String - // Define all of our state: - // Jobs: val activeJobs = new HashMap[JobId, JobUIData] val completedJobs = ListBuffer[JobUIData]() @@ -60,9 +58,11 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { // Stages: val activeStages = new HashMap[StageId, StageInfo] val completedStages = ListBuffer[StageInfo]() + val skippedStages = ListBuffer[StageInfo]() val failedStages = ListBuffer[StageInfo]() val stageIdToData = new HashMap[(StageId, StageAttemptId), StageUIData] val stageIdToInfo = new HashMap[StageId, StageInfo] + val stageIdToActiveJobIds = new HashMap[StageId, HashSet[JobId]] val poolToActiveStages = HashMap[PoolName, HashMap[StageId, StageInfo]]() // Total of completed and failed stages that have ever been run. These may be greater than // `completedStages.size` and `failedStages.size` if we have run more stages or jobs than @@ -95,7 +95,8 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { Map( "activeStages" -> activeStages.size, "activeJobs" -> activeJobs.size, - "poolToActiveStages" -> poolToActiveStages.values.map(_.size).sum + "poolToActiveStages" -> poolToActiveStages.values.map(_.size).sum, + "stageIdToActiveJobIds" -> stageIdToActiveJobIds.values.map(_.size).sum ) } @@ -106,6 +107,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { "completedJobs" -> completedJobs.size, "failedJobs" -> failedJobs.size, "completedStages" -> completedStages.size, + "skippedStages" -> skippedStages.size, "failedStages" -> failedStages.size ) } @@ -144,11 +146,39 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { } override def onJobStart(jobStart: SparkListenerJobStart) = synchronized { - val jobGroup = Option(jobStart.properties).map(_.getProperty(SparkContext.SPARK_JOB_GROUP_ID)) + val jobGroup = for ( + props <- Option(jobStart.properties); + group <- Option(props.getProperty(SparkContext.SPARK_JOB_GROUP_ID)) + ) yield group val jobData: JobUIData = - new JobUIData(jobStart.jobId, jobStart.stageIds, jobGroup, JobExecutionStatus.RUNNING) + new JobUIData( + jobId = jobStart.jobId, + startTime = Some(System.currentTimeMillis), + endTime = None, + stageIds = jobStart.stageIds, + jobGroup = jobGroup, + status = JobExecutionStatus.RUNNING) + // Compute (a potential underestimate of) the number of tasks that will be run by this job. + // This may be an underestimate because the job start event references all of the result + // stages's transitive stage dependencies, but some of these stages might be skipped if their + // output is available from earlier runs. + // See https://github.com/apache/spark/pull/3009 for a more extensive discussion. + jobData.numTasks = { + val allStages = jobStart.stageInfos + val missingStages = allStages.filter(_.completionTime.isEmpty) + missingStages.map(_.numTasks).sum + } jobIdToData(jobStart.jobId) = jobData activeJobs(jobStart.jobId) = jobData + for (stageId <- jobStart.stageIds) { + stageIdToActiveJobIds.getOrElseUpdate(stageId, new HashSet[StageId]).add(jobStart.jobId) + } + // If there's no information for a stage, store the StageInfo received from the scheduler + // so that we can display stage descriptions for pending stages: + for (stageInfo <- jobStart.stageInfos) { + stageIdToInfo.getOrElseUpdate(stageInfo.stageId, stageInfo) + stageIdToData.getOrElseUpdate((stageInfo.stageId, stageInfo.attemptId), new StageUIData) + } } override def onJobEnd(jobEnd: SparkListenerJobEnd) = synchronized { @@ -156,6 +186,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { logWarning(s"Job completed for unknown job ${jobEnd.jobId}") new JobUIData(jobId = jobEnd.jobId) } + jobData.endTime = Some(System.currentTimeMillis()) jobEnd.jobResult match { case JobSucceeded => completedJobs += jobData @@ -166,6 +197,20 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { trimJobsIfNecessary(failedJobs) jobData.status = JobExecutionStatus.FAILED } + for (stageId <- jobData.stageIds) { + stageIdToActiveJobIds.get(stageId).foreach { jobsUsingStage => + jobsUsingStage.remove(jobEnd.jobId) + stageIdToInfo.get(stageId).foreach { stageInfo => + if (stageInfo.submissionTime.isEmpty) { + // if this stage is pending, it won't complete, so mark it as "skipped": + skippedStages += stageInfo + trimStagesIfNecessary(skippedStages) + jobData.numSkippedStages += 1 + jobData.numSkippedTasks += stageInfo.numTasks + } + } + } + } } override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { @@ -193,6 +238,19 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { numFailedStages += 1 trimStagesIfNecessary(failedStages) } + + for ( + activeJobsDependentOnStage <- stageIdToActiveJobIds.get(stage.stageId); + jobId <- activeJobsDependentOnStage; + jobData <- jobIdToData.get(jobId) + ) { + jobData.numActiveStages -= 1 + if (stage.failureReason.isEmpty) { + jobData.completedStageIndices.add(stage.stageId) + } else { + jobData.numFailedStages += 1 + } + } } /** For FIFO, all stages are contained by "default" pool but "default" pool here is meaningless */ @@ -214,6 +272,14 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashMap[Int, StageInfo]) stages(stage.stageId) = stage + + for ( + activeJobsDependentOnStage <- stageIdToActiveJobIds.get(stage.stageId); + jobId <- activeJobsDependentOnStage; + jobData <- jobIdToData.get(jobId) + ) { + jobData.numActiveStages += 1 + } } override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { @@ -226,6 +292,13 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { stageData.numActiveTasks += 1 stageData.taskData.put(taskInfo.taskId, new TaskUIData(taskInfo)) } + for ( + activeJobsDependentOnStage <- stageIdToActiveJobIds.get(taskStart.stageId); + jobId <- activeJobsDependentOnStage; + jobData <- jobIdToData.get(jobId) + ) { + jobData.numActiveTasks += 1 + } } override def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult) { @@ -283,6 +356,20 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { taskData.taskInfo = info taskData.taskMetrics = metrics taskData.errorMessage = errorMessage + + for ( + activeJobsDependentOnStage <- stageIdToActiveJobIds.get(taskEnd.stageId); + jobId <- activeJobsDependentOnStage; + jobData <- jobIdToData.get(jobId) + ) { + jobData.numActiveTasks -= 1 + taskEnd.reason match { + case Success => + jobData.numCompletedTasks += 1 + case _ => + jobData.numFailedTasks += 1 + } + } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala new file mode 100644 index 0000000000000..b2bbfdee56946 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala @@ -0,0 +1,32 @@ +/* + * 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.ui.jobs + +import org.apache.spark.scheduler.SchedulingMode +import org.apache.spark.ui.{SparkUI, SparkUITab} + +/** Web UI showing progress status of all jobs in the given SparkContext. */ +private[ui] class JobsTab(parent: SparkUI) extends SparkUITab(parent, "jobs") { + val sc = parent.sc + val killEnabled = parent.killEnabled + def isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) + val listener = parent.jobProgressListener + + attachPage(new AllJobsPage(this)) + attachPage(new JobPage(this)) +} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index 770d99eea1c9d..5fc6cc7533150 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -25,7 +25,7 @@ import org.apache.spark.scheduler.{Schedulable, StageInfo} import org.apache.spark.ui.{WebUIPage, UIUtils} /** Page showing specific pool details */ -private[ui] class PoolPage(parent: JobProgressTab) extends WebUIPage("pool") { +private[ui] class PoolPage(parent: StagesTab) extends WebUIPage("pool") { private val sc = parent.sc private val listener = parent.listener @@ -37,8 +37,9 @@ private[ui] class PoolPage(parent: JobProgressTab) extends WebUIPage("pool") { case Some(s) => s.values.toSeq case None => Seq[StageInfo]() } - val activeStagesTable = - new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, parent) + val activeStagesTable = new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, + parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler, + killEnabled = parent.killEnabled) // For now, pool information is only accessible in live UIs val pools = sc.map(_.getPoolForName(poolName).get).toSeq diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala index 64178e1e33d41..df1899e7a9b84 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala @@ -24,7 +24,7 @@ import org.apache.spark.scheduler.{Schedulable, StageInfo} import org.apache.spark.ui.UIUtils /** Table showing list of pools */ -private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressTab) { +private[ui] class PoolTable(pools: Seq[Schedulable], parent: StagesTab) { private val listener = parent.listener def toNodeSeq: Seq[Node] = { 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 36afc4942e085..40e05f86b661d 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 @@ -31,7 +31,7 @@ import org.apache.spark.util.{Utils, Distribution} import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo} /** Page showing statistics and task list for a given stage */ -private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { +private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 2ff561ccc7da0..e7d6244dcd679 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -31,11 +31,10 @@ import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished stages */ private[ui] class StageTableBase( stages: Seq[StageInfo], - parent: JobProgressTab, - killEnabled: Boolean = false) { - - private val listener = parent.listener - protected def isFairScheduler = parent.isFairScheduler + basePath: String, + listener: JobProgressListener, + isFairScheduler: Boolean, + killEnabled: Boolean) { protected def columns: Seq[Node] = { Stage Id ++ @@ -73,25 +72,11 @@ private[ui] class StageTableBase( } - private def makeProgressBar(started: Int, completed: Int, failed: Int, total: Int): Seq[Node] = - { - val completeWidth = "width: %s%%".format((completed.toDouble/total)*100) - val startWidth = "width: %s%%".format((started.toDouble/total)*100) - -
- - {completed}/{total} { if (failed > 0) s"($failed failed)" else "" } - -
-
-
- } - private def makeDescription(s: StageInfo): Seq[Node] = { // scalastyle:off val killLink = if (killEnabled) { val killLinkUri = "%s/stages/stage/kill?id=%s&terminate=true" - .format(UIUtils.prependBaseUri(parent.basePath), s.stageId) + .format(UIUtils.prependBaseUri(basePath), s.stageId) val confirm = "return window.confirm('Are you sure you want to kill stage %s ?');" .format(s.stageId) @@ -101,7 +86,7 @@ private[ui] class StageTableBase( // scalastyle:on val nameLinkUri ="%s/stages/stage?id=%s&attempt=%s" - .format(UIUtils.prependBaseUri(parent.basePath), s.stageId, s.attemptId) + .format(UIUtils.prependBaseUri(basePath), s.stageId, s.attemptId) val nameLink = {s.name} val cachedRddInfos = s.rddInfos.filter(_.numCachedPartitions > 0) @@ -115,7 +100,7 @@ private[ui] class StageTableBase( Text("RDD: ") ++ // scalastyle:off cachedRddInfos.map { i => - {i.name} + {i.name} } // scalastyle:on }} @@ -167,7 +152,7 @@ private[ui] class StageTableBase( {if (isFairScheduler) { + .format(UIUtils.prependBaseUri(basePath), stageData.schedulingPool)}> {stageData.schedulingPool} @@ -180,8 +165,9 @@ private[ui] class StageTableBase( {formattedDuration} - {makeProgressBar(stageData.numActiveTasks, stageData.completedIndices.size, - stageData.numFailedTasks, s.numTasks)} + {UIUtils.makeProgressBar(started = stageData.numActiveTasks, + completed = stageData.completedIndices.size, failed = stageData.numFailedTasks, + skipped = 0, total = s.numTasks)} {inputReadWithUnit} {outputWriteWithUnit} @@ -195,9 +181,10 @@ private[ui] class StageTableBase( private[ui] class FailedStageTable( stages: Seq[StageInfo], - parent: JobProgressTab, - killEnabled: Boolean = false) - extends StageTableBase(stages, parent, killEnabled) { + basePath: String, + listener: JobProgressListener, + isFairScheduler: Boolean) + extends StageTableBase(stages, basePath, listener, isFairScheduler, killEnabled = false) { override protected def columns: Seq[Node] = super.columns ++ Failure Reason diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala similarity index 83% rename from core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala rename to core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala index 03ca918e2e8b3..937261de00e3a 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala @@ -19,18 +19,16 @@ package org.apache.spark.ui.jobs import javax.servlet.http.HttpServletRequest -import org.apache.spark.SparkConf import org.apache.spark.scheduler.SchedulingMode import org.apache.spark.ui.{SparkUI, SparkUITab} -/** Web UI showing progress status of all jobs in the given SparkContext. */ -private[ui] class JobProgressTab(parent: SparkUI) extends SparkUITab(parent, "stages") { +/** Web UI showing progress status of all stages in the given SparkContext. */ +private[ui] class StagesTab(parent: SparkUI) extends SparkUITab(parent, "stages") { val sc = parent.sc - val conf = sc.map(_.conf).getOrElse(new SparkConf) - val killEnabled = sc.map(_.conf.getBoolean("spark.ui.killEnabled", true)).getOrElse(false) + val killEnabled = parent.killEnabled val listener = parent.jobProgressListener - attachPage(new JobProgressPage(this)) + attachPage(new AllStagesPage(this)) attachPage(new StagePage(this)) attachPage(new PoolPage(this)) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index 2f7d618df5f6f..48fd7caa1a1ed 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -40,9 +40,28 @@ private[jobs] object UIData { class JobUIData( var jobId: Int = -1, + var startTime: Option[Long] = None, + var endTime: Option[Long] = None, var stageIds: Seq[Int] = Seq.empty, var jobGroup: Option[String] = None, - var status: JobExecutionStatus = JobExecutionStatus.UNKNOWN + var status: JobExecutionStatus = JobExecutionStatus.UNKNOWN, + /* Tasks */ + // `numTasks` is a potential underestimate of the true number of tasks that this job will run. + // This may be an underestimate because the job start event references all of the result + // stages's transitive stage dependencies, but some of these stages might be skipped if their + // output is available from earlier runs. + // See https://github.com/apache/spark/pull/3009 for a more extensive discussion. + var numTasks: Int = 0, + var numActiveTasks: Int = 0, + var numCompletedTasks: Int = 0, + var numSkippedTasks: Int = 0, + var numFailedTasks: Int = 0, + /* Stages */ + var numActiveStages: Int = 0, + // This needs to be a set instead of a simple count to prevent double-counting of rerun stages: + var completedStageIndices: OpenHashSet[Int] = new OpenHashSet[Int](), + var numSkippedStages: Int = 0, + var numFailedStages: Int = 0 ) class StageUIData { diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 7e536edfe807b..7b5db1ed76265 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -31,6 +31,21 @@ import org.apache.spark.scheduler._ import org.apache.spark.storage._ import org.apache.spark._ +/** + * Serializes SparkListener events to/from JSON. This protocol provides strong backwards- + * and forwards-compatibility guarantees: any version of Spark should be able to read JSON output + * written by any other version, including newer versions. + * + * JsonProtocolSuite contains backwards-compatibility tests which check that the current version of + * JsonProtocol is able to read output written by earlier versions. We do not currently have tests + * for reading newer JSON output with older Spark versions. + * + * To ensure that we provide these guarantees, follow these rules when modifying these methods: + * + * - Never delete any JSON fields. + * - Any new JSON fields should be optional; use `Utils.jsonOption` when reading these fields + * in `*FromJson` methods. + */ private[spark] object JsonProtocol { // TODO: Remove this file and put JSON serialization into each individual class. @@ -121,6 +136,7 @@ private[spark] object JsonProtocol { val properties = propertiesToJson(jobStart.properties) ("Event" -> Utils.getFormattedClassName(jobStart)) ~ ("Job ID" -> jobStart.jobId) ~ + ("Stage Infos" -> jobStart.stageInfos.map(stageInfoToJson)) ~ // Added in Spark 1.2.0 ("Stage IDs" -> jobStart.stageIds) ~ ("Properties" -> properties) } @@ -455,7 +471,12 @@ private[spark] object JsonProtocol { val jobId = (json \ "Job ID").extract[Int] val stageIds = (json \ "Stage IDs").extract[List[JValue]].map(_.extract[Int]) val properties = propertiesFromJson(json \ "Properties") - SparkListenerJobStart(jobId, stageIds, properties) + // The "Stage Infos" field was added in Spark 1.2.0 + val stageInfos = Utils.jsonOption(json \ "Stage Infos") + .map(_.extract[Seq[JValue]].map(stageInfoFromJson)).getOrElse { + stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, "unknown")) + } + SparkListenerJobStart(jobId, stageInfos, properties) } def jobEndFromJson(json: JValue): SparkListenerJobEnd = { diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index bacf6a16fc233..d2857b8b55664 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -17,16 +17,20 @@ package org.apache.spark.ui -import org.apache.spark.api.java.StorageLevels -import org.apache.spark.{SparkException, SparkConf, SparkContext} -import org.openqa.selenium.WebDriver +import scala.collection.JavaConversions._ + +import org.openqa.selenium.{By, WebDriver} import org.openqa.selenium.htmlunit.HtmlUnitDriver import org.scalatest._ import org.scalatest.concurrent.Eventually._ import org.scalatest.selenium.WebBrowser import org.scalatest.time.SpanSugar._ +import org.apache.spark._ +import org.apache.spark.SparkContext._ import org.apache.spark.LocalSparkContext._ +import org.apache.spark.api.java.StorageLevels +import org.apache.spark.shuffle.FetchFailedException /** * Selenium tests for the Spark Web UI. These tests are not run by default @@ -89,7 +93,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { sc.parallelize(1 to 10).map { x => throw new Exception()}.collect() } eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to sc.ui.get.appUIAddress + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") find(id("active")).get.text should be("Active Stages (0)") find(id("failed")).get.text should be("Failed Stages (1)") } @@ -101,7 +105,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { sc.parallelize(1 to 10).map { x => unserializableObject}.collect() } eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to sc.ui.get.appUIAddress + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") find(id("active")).get.text should be("Active Stages (0)") // The failure occurs before the stage becomes active, hence we should still show only one // failed stage, not two: @@ -109,4 +113,191 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { } } } + + test("spark.ui.killEnabled should properly control kill button display") { + def getSparkContext(killEnabled: Boolean): SparkContext = { + val conf = new SparkConf() + .setMaster("local") + .setAppName("test") + .set("spark.ui.enabled", "true") + .set("spark.ui.killEnabled", killEnabled.toString) + new SparkContext(conf) + } + + def hasKillLink = find(className("kill-link")).isDefined + def runSlowJob(sc: SparkContext) { + sc.parallelize(1 to 10).map{x => Thread.sleep(10000); x}.countAsync() + } + + withSpark(getSparkContext(killEnabled = true)) { sc => + runSlowJob(sc) + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") + assert(hasKillLink) + } + } + + withSpark(getSparkContext(killEnabled = false)) { sc => + runSlowJob(sc) + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") + assert(!hasKillLink) + } + } + } + + test("jobs page should not display job group name unless some job was submitted in a job group") { + withSpark(newSparkContext()) { sc => + // If no job has been run in a job group, then "(Job Group)" should not appear in the header + sc.parallelize(Seq(1, 2, 3)).count() + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + val tableHeaders = findAll(cssSelector("th")).map(_.text).toSeq + tableHeaders should not contain "Job Id (Job Group)" + } + // Once at least one job has been run in a job group, then we should display the group name: + sc.setJobGroup("my-job-group", "my-job-group-description") + sc.parallelize(Seq(1, 2, 3)).count() + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + val tableHeaders = findAll(cssSelector("th")).map(_.text).toSeq + tableHeaders should contain ("Job Id (Job Group)") + } + } + } + + test("job progress bars should handle stage / task failures") { + withSpark(newSparkContext()) { sc => + val data = sc.parallelize(Seq(1, 2, 3)).map(identity).groupBy(identity) + val shuffleHandle = + data.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleHandle + // Simulate fetch failures: + val mappedData = data.map { x => + val taskContext = TaskContext.get + if (taskContext.attemptId() == 1) { // Cause this stage to fail on its first attempt. + val env = SparkEnv.get + val bmAddress = env.blockManager.blockManagerId + val shuffleId = shuffleHandle.shuffleId + val mapId = 0 + val reduceId = taskContext.partitionId() + val message = "Simulated fetch failure" + throw new FetchFailedException(bmAddress, shuffleId, mapId, reduceId, message) + } else { + x + } + } + mappedData.count() + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + find(cssSelector(".stage-progress-cell")).get.text should be ("2/2 (1 failed)") + // Ideally, the following test would pass, but currently we overcount completed tasks + // if task recomputations occur: + // find(cssSelector(".progress-cell .progress")).get.text should be ("2/2 (1 failed)") + // Instead, we guarantee that the total number of tasks is always correct, while the number + // of completed tasks may be higher: + find(cssSelector(".progress-cell .progress")).get.text should be ("3/2 (1 failed)") + } + } + } + + test("job details page should display useful information for stages that haven't started") { + withSpark(newSparkContext()) { sc => + // Create a multi-stage job with a long delay in the first stage: + val rdd = sc.parallelize(Seq(1, 2, 3)).map { x => + // This long sleep call won't slow down the tests because we don't actually need to wait + // for the job to finish. + Thread.sleep(20000) + }.groupBy(identity).map(identity).groupBy(identity).map(identity) + // Start the job: + rdd.countAsync() + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs/job/?id=0") + find(id("active")).get.text should be ("Active Stages (1)") + find(id("pending")).get.text should be ("Pending Stages (2)") + // Essentially, we want to check that none of the stage rows show + // "No data available for this stage". Checking for the absence of that string is brittle + // because someone could change the error message and cause this test to pass by accident. + // Instead, it's safer to check that each row contains a link to a stage details page. + findAll(cssSelector("tbody tr")).foreach { row => + val link = row.underlying.findElement(By.xpath(".//a")) + link.getAttribute("href") should include ("stage") + } + } + } + } + + test("job progress bars / cells reflect skipped stages / tasks") { + withSpark(newSparkContext()) { sc => + // Create an RDD that involves multiple stages: + val rdd = sc.parallelize(1 to 8, 8) + .map(x => x).groupBy((x: Int) => x, numPartitions = 8) + .flatMap(x => x._2).groupBy((x: Int) => x, numPartitions = 8) + // Run it twice; this will cause the second job to have two "phantom" stages that were + // mentioned in its job start event but which were never actually executed: + rdd.count() + rdd.count() + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + // The completed jobs table should have two rows. The first row will be the most recent job: + val firstRow = find(cssSelector("tbody tr")).get.underlying + val firstRowColumns = firstRow.findElements(By.tagName("td")) + firstRowColumns(0).getText should be ("1") + firstRowColumns(4).getText should be ("1/1 (2 skipped)") + firstRowColumns(5).getText should be ("8/8 (16 skipped)") + // The second row is the first run of the job, where nothing was skipped: + val secondRow = findAll(cssSelector("tbody tr")).toSeq(1).underlying + val secondRowColumns = secondRow.findElements(By.tagName("td")) + secondRowColumns(0).getText should be ("0") + secondRowColumns(4).getText should be ("3/3") + secondRowColumns(5).getText should be ("24/24") + } + } + } + + test("stages that aren't run appear as 'skipped stages' after a job finishes") { + withSpark(newSparkContext()) { sc => + // Create an RDD that involves multiple stages: + val rdd = + sc.parallelize(Seq(1, 2, 3)).map(identity).groupBy(identity).map(identity).groupBy(identity) + // Run it twice; this will cause the second job to have two "phantom" stages that were + // mentioned in its job start event but which were never actually executed: + rdd.count() + rdd.count() + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs/job/?id=1") + find(id("pending")) should be (None) + find(id("active")) should be (None) + find(id("failed")) should be (None) + find(id("completed")).get.text should be ("Completed Stages (1)") + find(id("skipped")).get.text should be ("Skipped Stages (2)") + // Essentially, we want to check that none of the stage rows show + // "No data available for this stage". Checking for the absence of that string is brittle + // because someone could change the error message and cause this test to pass by accident. + // Instead, it's safer to check that each row contains a link to a stage details page. + findAll(cssSelector("tbody tr")).foreach { row => + val link = row.underlying.findElement(By.xpath(".//a")) + link.getAttribute("href") should include ("stage") + } + } + } + } + + test("jobs with stages that are skipped should show correct link descriptions on all jobs page") { + withSpark(newSparkContext()) { sc => + // Create an RDD that involves multiple stages: + val rdd = + sc.parallelize(Seq(1, 2, 3)).map(identity).groupBy(identity).map(identity).groupBy(identity) + // Run it twice; this will cause the second job to have two "phantom" stages that were + // mentioned in its job start event but which were never actually executed: + rdd.count() + rdd.count() + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + findAll(cssSelector("tbody tr a")).foreach { link => + link.text.toLowerCase should include ("count") + link.text.toLowerCase should not include "unknown" + } + } + } + } } diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index 15c5b4e702efa..12af60caf7d54 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -43,7 +43,10 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc } private def createJobStartEvent(jobId: Int, stageIds: Seq[Int]) = { - SparkListenerJobStart(jobId, stageIds) + val stageInfos = stageIds.map { stageId => + new StageInfo(stageId, 0, stageId.toString, 0, null, "") + } + SparkListenerJobStart(jobId, stageInfos) } private def createJobEndEvent(jobId: Int, failed: Boolean = false) = { @@ -52,8 +55,9 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc } private def runJob(listener: SparkListener, jobId: Int, shouldFail: Boolean = false) { + val stagesThatWontBeRun = jobId * 200 to jobId * 200 + 10 val stageIds = jobId * 100 to jobId * 100 + 50 - listener.onJobStart(createJobStartEvent(jobId, stageIds)) + listener.onJobStart(createJobStartEvent(jobId, stageIds ++ stagesThatWontBeRun)) for (stageId <- stageIds) { listener.onStageSubmitted(createStageStartEvent(stageId)) listener.onStageCompleted(createStageEndEvent(stageId, failed = stageId % 2 == 0)) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 50f42054b9296..0bc9492675863 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -47,7 +47,12 @@ class JsonProtocolSuite extends FunSuite { val taskEndWithOutput = SparkListenerTaskEnd(1, 0, "ResultTask", Success, makeTaskInfo(123L, 234, 67, 345L, false), makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true, hasOutput = true)) - val jobStart = SparkListenerJobStart(10, Seq[Int](1, 2, 3, 4), properties) + val jobStart = { + val stageIds = Seq[Int](1, 2, 3, 4) + val stageInfos = stageIds.map(x => + makeStageInfo(x, x * 200, x * 300, x * 400L, x * 500L)) + SparkListenerJobStart(10, stageInfos, properties) + } val jobEnd = SparkListenerJobEnd(20, JobSucceeded) val environmentUpdate = SparkListenerEnvironmentUpdate(Map[String, Seq[(String, String)]]( "JVM Information" -> Seq(("GC speed", "9999 objects/s"), ("Java home", "Land of coffee")), @@ -224,6 +229,19 @@ class JsonProtocolSuite extends FunSuite { assert(expectedExecutorLostFailure === JsonProtocol.taskEndReasonFromJson(oldEvent)) } + test("SparkListenerJobStart backward compatibility") { + // Prior to Spark 1.2.0, SparkListenerJobStart did not have a "Stage Infos" property. + val stageIds = Seq[Int](1, 2, 3, 4) + val stageInfos = stageIds.map(x => makeStageInfo(x, x * 200, x * 300, x * 400, x * 500)) + val dummyStageInfos = + stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, "unknown")) + val jobStart = SparkListenerJobStart(10, stageInfos, properties) + val oldEvent = JsonProtocol.jobStartToJson(jobStart).removeField({_._1 == "Stage Infos"}) + val expectedJobStart = + SparkListenerJobStart(10, dummyStageInfos, properties) + assertEquals(expectedJobStart, JsonProtocol.jobStartFromJson(oldEvent)) + } + /** -------------------------- * | Helper test running methods | * --------------------------- */ @@ -306,7 +324,7 @@ class JsonProtocolSuite extends FunSuite { case (e1: SparkListenerJobStart, e2: SparkListenerJobStart) => assert(e1.jobId === e2.jobId) assert(e1.properties === e2.properties) - assertSeqEquals(e1.stageIds, e2.stageIds, (i1: Int, i2: Int) => assert(i1 === i2)) + assert(e1.stageIds === e2.stageIds) case (e1: SparkListenerJobEnd, e2: SparkListenerJobEnd) => assert(e1.jobId === e2.jobId) assertEquals(e1.jobResult, e2.jobResult) @@ -1051,6 +1069,260 @@ class JsonProtocolSuite extends FunSuite { |{ | "Event": "SparkListenerJobStart", | "Job ID": 10, + | "Stage Infos": [ + | { + | "Stage ID": 1, + | "Stage Attempt ID": 0, + | "Stage Name": "greetings", + | "Number of Tasks": 200, + | "RDD Info": [ + | { + | "RDD ID": 1, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 200, + | "Number of Cached Partitions": 300, + | "Memory Size": 400, + | "Tachyon Size": 0, + | "Disk Size": 500 + | } + | ], + | "Details": "details", + | "Accumulables": [ + | { + | "ID": 2, + | "Name": " Accumulable 2", + | "Update": "delta2", + | "Value": "val2" + | }, + | { + | "ID": 1, + | "Name": " Accumulable 1", + | "Update": "delta1", + | "Value": "val1" + | } + | ] + | }, + | { + | "Stage ID": 2, + | "Stage Attempt ID": 0, + | "Stage Name": "greetings", + | "Number of Tasks": 400, + | "RDD Info": [ + | { + | "RDD ID": 2, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 400, + | "Number of Cached Partitions": 600, + | "Memory Size": 800, + | "Tachyon Size": 0, + | "Disk Size": 1000 + | }, + | { + | "RDD ID": 3, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 401, + | "Number of Cached Partitions": 601, + | "Memory Size": 801, + | "Tachyon Size": 0, + | "Disk Size": 1001 + | } + | ], + | "Details": "details", + | "Accumulables": [ + | { + | "ID": 2, + | "Name": " Accumulable 2", + | "Update": "delta2", + | "Value": "val2" + | }, + | { + | "ID": 1, + | "Name": " Accumulable 1", + | "Update": "delta1", + | "Value": "val1" + | } + | ] + | }, + | { + | "Stage ID": 3, + | "Stage Attempt ID": 0, + | "Stage Name": "greetings", + | "Number of Tasks": 600, + | "RDD Info": [ + | { + | "RDD ID": 3, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 600, + | "Number of Cached Partitions": 900, + | "Memory Size": 1200, + | "Tachyon Size": 0, + | "Disk Size": 1500 + | }, + | { + | "RDD ID": 4, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 601, + | "Number of Cached Partitions": 901, + | "Memory Size": 1201, + | "Tachyon Size": 0, + | "Disk Size": 1501 + | }, + | { + | "RDD ID": 5, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 602, + | "Number of Cached Partitions": 902, + | "Memory Size": 1202, + | "Tachyon Size": 0, + | "Disk Size": 1502 + | } + | ], + | "Details": "details", + | "Accumulables": [ + | { + | "ID": 2, + | "Name": " Accumulable 2", + | "Update": "delta2", + | "Value": "val2" + | }, + | { + | "ID": 1, + | "Name": " Accumulable 1", + | "Update": "delta1", + | "Value": "val1" + | } + | ] + | }, + | { + | "Stage ID": 4, + | "Stage Attempt ID": 0, + | "Stage Name": "greetings", + | "Number of Tasks": 800, + | "RDD Info": [ + | { + | "RDD ID": 4, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 800, + | "Number of Cached Partitions": 1200, + | "Memory Size": 1600, + | "Tachyon Size": 0, + | "Disk Size": 2000 + | }, + | { + | "RDD ID": 5, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 801, + | "Number of Cached Partitions": 1201, + | "Memory Size": 1601, + | "Tachyon Size": 0, + | "Disk Size": 2001 + | }, + | { + | "RDD ID": 6, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 802, + | "Number of Cached Partitions": 1202, + | "Memory Size": 1602, + | "Tachyon Size": 0, + | "Disk Size": 2002 + | }, + | { + | "RDD ID": 7, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 803, + | "Number of Cached Partitions": 1203, + | "Memory Size": 1603, + | "Tachyon Size": 0, + | "Disk Size": 2003 + | } + | ], + | "Details": "details", + | "Accumulables": [ + | { + | "ID": 2, + | "Name": " Accumulable 2", + | "Update": "delta2", + | "Value": "val2" + | }, + | { + | "ID": 1, + | "Name": " Accumulable 1", + | "Update": "delta1", + | "Value": "val1" + | } + | ] + | } + | ], | "Stage IDs": [ | 1, | 2, From cb0e9b0980f38befe88bf52aa037fe33262730f7 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 24 Nov 2014 13:50:20 -0800 Subject: [PATCH 079/109] [SPARK-4518][SPARK-4519][Streaming] Refactored file stream to prevent files from being processed multiple times Because of a corner case, a file already selected for batch t can get considered again for batch t+2. This refactoring fixes it by remembering all the files selected in the last 1 minute, so that this corner case does not arise. Also uses spark context's hadoop configuration to access the file system API for listing directories. pwendell Please take look. I still have not run long-running integration tests, so I cannot say for sure whether this has indeed solved the issue. You could do a first pass on this in the meantime. Author: Tathagata Das Closes #3419 from tdas/filestream-fix2 and squashes the following commits: c19dd8a [Tathagata Das] Addressed PR comments. 513b608 [Tathagata Das] Updated docs. d364faf [Tathagata Das] Added the current time condition back 5526222 [Tathagata Das] Removed unnecessary imports. 38bb736 [Tathagata Das] Fix long line. 203bbc7 [Tathagata Das] Un-ignore tests. eaef4e1 [Tathagata Das] Fixed SPARK-4519 9dbd40a [Tathagata Das] Refactored FileInputDStream to remember last few batches. --- .../spark/streaming/dstream/DStream.scala | 2 +- .../streaming/dstream/FileInputDStream.scala | 291 +++++++++++------- .../spark/streaming/CheckpointSuite.scala | 2 +- .../spark/streaming/InputStreamsSuite.scala | 106 ++++--- 4 files changed, 245 insertions(+), 156 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index eabd61d713e0c..dbf1ebbaf653a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -254,7 +254,7 @@ abstract class DStream[T: ClassTag] ( } private[streaming] def remember(duration: Duration) { - if (duration != null && duration > rememberDuration) { + if (duration != null && (rememberDuration == null || duration > rememberDuration)) { rememberDuration = duration logInfo("Duration for remembering RDDs set to " + rememberDuration + " for " + this) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala index 55d6cf6a783ea..5f13fdc5579ed 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala @@ -17,18 +17,55 @@ package org.apache.spark.streaming.dstream -import java.io.{ObjectInputStream, IOException} -import scala.collection.mutable.{HashSet, HashMap} +import java.io.{IOException, ObjectInputStream} + +import scala.collection.mutable import scala.reflect.ClassTag + import org.apache.hadoop.fs.{FileSystem, Path, PathFilter} -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} -import org.apache.spark.rdd.RDD -import org.apache.spark.rdd.UnionRDD -import org.apache.spark.streaming.{StreamingContext, Time} -import org.apache.spark.util.{TimeStampedHashMap, Utils} +import org.apache.spark.rdd.{RDD, UnionRDD} +import org.apache.spark.streaming._ +import org.apache.spark.util.{TimeStampedHashMap, Utils} +/** + * This class represents an input stream that monitors a Hadoop-compatible filesystem for new + * files and creates a stream out of them. The way it works as follows. + * + * At each batch interval, the file system is queried for files in the given directory and + * detected new files are selected for that batch. In this case "new" means files that + * became visible to readers during that time period. Some extra care is needed to deal + * with the fact that files may become visible after they are created. For this purpose, this + * class remembers the information about the files selected in past batches for + * a certain duration (say, "remember window") as shown in the figure below. + * + * |<----- remember window ----->| + * ignore threshold --->| |<--- current batch time + * |____.____.____.____.____.____| + * | | | | | | | + * ---------------------|----|----|----|----|----|----|-----------------------> Time + * |____|____|____|____|____|____| + * remembered batches + * + * The trailing end of the window is the "ignore threshold" and all files whose mod times + * are less than this threshold are assumed to have already been selected and are therefore + * ignored. Files whose mod times are within the "remember window" are checked against files + * that have already been selected. At a high level, this is how new files are identified in + * each batch - files whose mod times are greater than the ignore threshold and + * have not been considered within the remember window. See the documentation on the method + * `isNewFile` for more details. + * + * This makes some assumptions from the underlying file system that the system is monitoring. + * - The clock of the file system is assumed to synchronized with the clock of the machine running + * the streaming app. + * - If a file is to be visible in the directory listings, it must be visible within a certain + * duration of the mod time of the file. This duration is the "remember window", which is set to + * 1 minute (see `FileInputDStream.MIN_REMEMBER_DURATION`). Otherwise, the file will never be + * selected as the mod time will be less than the ignore threshold when it becomes visible. + * - Once a file is visible, the mod time cannot change. If it does due to appends, then the + * processing semantics are undefined. + */ private[streaming] class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : ClassTag]( @transient ssc_ : StreamingContext, @@ -37,22 +74,37 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas newFilesOnly: Boolean = true) extends InputDStream[(K, V)](ssc_) { + // Data to be saved as part of the streaming checkpoints protected[streaming] override val checkpointData = new FileInputDStreamCheckpointData - // files found in the last interval - private val lastFoundFiles = new HashSet[String] + // Initial ignore threshold based on which old, existing files in the directory (at the time of + // starting the streaming application) will be ignored or considered + private val initialModTimeIgnoreThreshold = if (newFilesOnly) System.currentTimeMillis() else 0L + + /* + * Make sure that the information of files selected in the last few batches are remembered. + * This would allow us to filter away not-too-old files which have already been recently + * selected and processed. + */ + private val numBatchesToRemember = FileInputDStream.calculateNumBatchesToRemember(slideDuration) + private val durationToRemember = slideDuration * numBatchesToRemember + remember(durationToRemember) - // Files with mod time earlier than this is ignored. This is updated every interval - // such that in the current interval, files older than any file found in the - // previous interval will be ignored. Obviously this time keeps moving forward. - private var ignoreTime = if (newFilesOnly) System.currentTimeMillis() else 0L + // Map of batch-time to selected file info for the remembered batches + @transient private[streaming] var batchTimeToSelectedFiles = + new mutable.HashMap[Time, Array[String]] + + // Set of files that were selected in the remembered batches + @transient private var recentlySelectedFiles = new mutable.HashSet[String]() + + // Read-through cache of file mod times, used to speed up mod time lookups + @transient private var fileToModTime = new TimeStampedHashMap[String, Long](true) + + // Timestamp of the last round of finding files + @transient private var lastNewFileFindingTime = 0L - // Latest file mod time seen till any point of time @transient private var path_ : Path = null @transient private var fs_ : FileSystem = null - @transient private[streaming] var files = new HashMap[Time, Array[String]] - @transient private var fileModTimes = new TimeStampedHashMap[String, Long](true) - @transient private var lastNewFileFindingTime = 0L override def start() { } @@ -68,54 +120,113 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas * the previous call. */ override def compute(validTime: Time): Option[RDD[(K, V)]] = { - assert(validTime.milliseconds >= ignoreTime, - "Trying to get new files for a really old time [" + validTime + " < " + ignoreTime + "]") - // Find new files - val (newFiles, minNewFileModTime) = findNewFiles(validTime.milliseconds) + val newFiles = findNewFiles(validTime.milliseconds) logInfo("New files at time " + validTime + ":\n" + newFiles.mkString("\n")) - if (!newFiles.isEmpty) { - lastFoundFiles.clear() - lastFoundFiles ++= newFiles - ignoreTime = minNewFileModTime - } - files += ((validTime, newFiles.toArray)) + batchTimeToSelectedFiles += ((validTime, newFiles)) + recentlySelectedFiles ++= newFiles Some(filesToRDD(newFiles)) } /** Clear the old time-to-files mappings along with old RDDs */ protected[streaming] override def clearMetadata(time: Time) { super.clearMetadata(time) - val oldFiles = files.filter(_._1 < (time - rememberDuration)) - files --= oldFiles.keys + val oldFiles = batchTimeToSelectedFiles.filter(_._1 < (time - rememberDuration)) + batchTimeToSelectedFiles --= oldFiles.keys + recentlySelectedFiles --= oldFiles.values.flatten logInfo("Cleared " + oldFiles.size + " old files that were older than " + (time - rememberDuration) + ": " + oldFiles.keys.mkString(", ")) logDebug("Cleared files are:\n" + oldFiles.map(p => (p._1, p._2.mkString(", "))).mkString("\n")) // Delete file mod times that weren't accessed in the last round of getting new files - fileModTimes.clearOldValues(lastNewFileFindingTime - 1) + fileToModTime.clearOldValues(lastNewFileFindingTime - 1) } /** - * Find files which have modification timestamp <= current time and return a 3-tuple of - * (new files found, latest modification time among them, files with latest modification time) + * Find new files for the batch of `currentTime`. This is done by first calculating the + * ignore threshold for file mod times, and then getting a list of files filtered based on + * the current batch time and the ignore threshold. The ignore threshold is the max of + * initial ignore threshold and the trailing end of the remember window (that is, which ever + * is later in time). */ - private def findNewFiles(currentTime: Long): (Seq[String], Long) = { - logDebug("Trying to get new files for time " + currentTime) - lastNewFileFindingTime = System.currentTimeMillis - val filter = new CustomPathFilter(currentTime) - val newFiles = fs.listStatus(directoryPath, filter).map(_.getPath.toString) - val timeTaken = System.currentTimeMillis - lastNewFileFindingTime - logInfo("Finding new files took " + timeTaken + " ms") - logDebug("# cached file times = " + fileModTimes.size) - if (timeTaken > slideDuration.milliseconds) { - logWarning( - "Time taken to find new files exceeds the batch size. " + - "Consider increasing the batch size or reduceing the number of " + - "files in the monitored directory." + private def findNewFiles(currentTime: Long): Array[String] = { + try { + lastNewFileFindingTime = System.currentTimeMillis + + // Calculate ignore threshold + val modTimeIgnoreThreshold = math.max( + initialModTimeIgnoreThreshold, // initial threshold based on newFilesOnly setting + currentTime - durationToRemember.milliseconds // trailing end of the remember window ) + logDebug(s"Getting new files for time $currentTime, " + + s"ignoring files older than $modTimeIgnoreThreshold") + val filter = new PathFilter { + def accept(path: Path): Boolean = isNewFile(path, currentTime, modTimeIgnoreThreshold) + } + val newFiles = fs.listStatus(directoryPath, filter).map(_.getPath.toString) + val timeTaken = System.currentTimeMillis - lastNewFileFindingTime + logInfo("Finding new files took " + timeTaken + " ms") + logDebug("# cached file times = " + fileToModTime.size) + if (timeTaken > slideDuration.milliseconds) { + logWarning( + "Time taken to find new files exceeds the batch size. " + + "Consider increasing the batch size or reducing the number of " + + "files in the monitored directory." + ) + } + newFiles + } catch { + case e: Exception => + logWarning("Error finding new files", e) + reset() + Array.empty + } + } + + /** + * Identify whether the given `path` is a new file for the batch of `currentTime`. For it to be + * accepted, it has to pass the following criteria. + * - It must pass the user-provided file filter. + * - It must be newer than the ignore threshold. It is assumed that files older than the ignore + * threshold have already been considered or are existing files before start + * (when newFileOnly = true). + * - It must not be present in the recently selected files that this class remembers. + * - It must not be newer than the time of the batch (i.e. `currentTime` for which this + * file is being tested. This can occur if the driver was recovered, and the missing batches + * (during downtime) are being generated. In that case, a batch of time T may be generated + * at time T+x. Say x = 5. If that batch T contains file of mod time T+5, then bad things can + * happen. Let's say the selected files are remembered for 60 seconds. At time t+61, + * the batch of time t is forgotten, and the ignore threshold is still T+1. + * The files with mod time T+5 are not remembered and cannot be ignored (since, t+5 > t+1). + * Hence they can get selected as new files again. To prevent this, files whose mod time is more + * than current batch time are not considered. + */ + private def isNewFile(path: Path, currentTime: Long, modTimeIgnoreThreshold: Long): Boolean = { + val pathStr = path.toString + // Reject file if it does not satisfy filter + if (!filter(path)) { + logDebug(s"$pathStr rejected by filter") + return false + } + // Reject file if it was created before the ignore time + val modTime = getFileModTime(path) + if (modTime <= modTimeIgnoreThreshold) { + // Use <= instead of < to avoid SPARK-4518 + logDebug(s"$pathStr ignored as mod time $modTime <= ignore time $modTimeIgnoreThreshold") + return false } - (newFiles, filter.minNewFileModTime) + // Reject file if mod time > current batch time + if (modTime > currentTime) { + logDebug(s"$pathStr not selected as mod time $modTime > current time $currentTime") + return false + } + // Reject file if it was considered earlier + if (recentlySelectedFiles.contains(pathStr)) { + logDebug(s"$pathStr already considered") + return false + } + logDebug(s"$pathStr accepted with mod time $modTime") + return true } /** Generate one RDD from an array of files */ @@ -132,21 +243,21 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas new UnionRDD(context.sparkContext, fileRDDs) } + /** Get file mod time from cache or fetch it from the file system */ + private def getFileModTime(path: Path) = { + fileToModTime.getOrElseUpdate(path.toString, fs.getFileStatus(path).getModificationTime()) + } + private def directoryPath: Path = { if (path_ == null) path_ = new Path(directory) path_ } private def fs: FileSystem = { - if (fs_ == null) fs_ = directoryPath.getFileSystem(new Configuration()) + if (fs_ == null) fs_ = directoryPath.getFileSystem(ssc.sparkContext.hadoopConfiguration) fs_ } - private def getFileModTime(path: Path) = { - // Get file mod time from cache or fetch it from the file system - fileModTimes.getOrElseUpdate(path.toString, fs.getFileStatus(path).getModificationTime()) - } - private def reset() { fs_ = null } @@ -155,9 +266,10 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas private def readObject(ois: ObjectInputStream): Unit = Utils.tryOrIOException { logDebug(this.getClass().getSimpleName + ".readObject used") ois.defaultReadObject() - generatedRDDs = new HashMap[Time, RDD[(K,V)]] () - files = new HashMap[Time, Array[String]] - fileModTimes = new TimeStampedHashMap[String, Long](true) + generatedRDDs = new mutable.HashMap[Time, RDD[(K,V)]] () + batchTimeToSelectedFiles = new mutable.HashMap[Time, Array[String]]() + recentlySelectedFiles = new mutable.HashSet[String]() + fileToModTime = new TimeStampedHashMap[String, Long](true) } /** @@ -167,11 +279,11 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas private[streaming] class FileInputDStreamCheckpointData extends DStreamCheckpointData(this) { - def hadoopFiles = data.asInstanceOf[HashMap[Time, Array[String]]] + def hadoopFiles = data.asInstanceOf[mutable.HashMap[Time, Array[String]]] override def update(time: Time) { hadoopFiles.clear() - hadoopFiles ++= files + hadoopFiles ++= batchTimeToSelectedFiles } override def cleanup(time: Time) { } @@ -182,7 +294,8 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas // Restore the metadata in both files and generatedRDDs logInfo("Restoring files for time " + t + " - " + f.mkString("[", ", ", "]") ) - files += ((t, f)) + batchTimeToSelectedFiles += ((t, f)) + recentlySelectedFiles ++= f generatedRDDs += ((t, filesToRDD(f))) } } @@ -193,57 +306,25 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas hadoopFiles.map(p => (p._1, p._2.mkString(", "))).mkString("\n") + "\n]" } } +} + +private[streaming] +object FileInputDStream { /** - * Custom PathFilter class to find new files that - * ... have modification time more than ignore time - * ... have not been seen in the last interval - * ... have modification time less than maxModTime + * Minimum duration of remembering the information of selected files. Files with mod times + * older than this "window" of remembering will be ignored. So if new files are visible + * within this window, then the file will get selected in the next batch. */ - private[streaming] - class CustomPathFilter(maxModTime: Long) extends PathFilter { + private val MIN_REMEMBER_DURATION = Minutes(1) - // Minimum of the mod times of new files found in the current interval - var minNewFileModTime = -1L + def defaultFilter(path: Path): Boolean = !path.getName().startsWith(".") - def accept(path: Path): Boolean = { - try { - if (!filter(path)) { // Reject file if it does not satisfy filter - logDebug("Rejected by filter " + path) - return false - } - // Reject file if it was found in the last interval - if (lastFoundFiles.contains(path.toString)) { - logDebug("Mod time equal to last mod time, but file considered already") - return false - } - val modTime = getFileModTime(path) - logDebug("Mod time for " + path + " is " + modTime) - if (modTime < ignoreTime) { - // Reject file if it was created before the ignore time (or, before last interval) - logDebug("Mod time " + modTime + " less than ignore time " + ignoreTime) - return false - } else if (modTime > maxModTime) { - // Reject file if it is too new that considering it may give errors - logDebug("Mod time more than ") - return false - } - if (minNewFileModTime < 0 || modTime < minNewFileModTime) { - minNewFileModTime = modTime - } - logDebug("Accepted " + path) - } catch { - case fnfe: java.io.FileNotFoundException => - logWarning("Error finding new files", fnfe) - reset() - return false - } - true - } + /** + * Calculate the number of last batches to remember, such that all the files selected in + * at least last MIN_REMEMBER_DURATION duration can be remembered. + */ + def calculateNumBatchesToRemember(batchDuration: Duration): Int = { + math.ceil(MIN_REMEMBER_DURATION.milliseconds.toDouble / batchDuration.milliseconds).toInt } } - -private[streaming] -object FileInputDStream { - def defaultFilter(path: Path): Boolean = !path.getName().startsWith(".") -} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index e5592e52b0d2d..77ff1ca780a58 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -265,7 +265,7 @@ class CheckpointSuite extends TestSuiteBase { // Verify whether files created have been recorded correctly or not var fileInputDStream = ssc.graph.getInputStreams().head.asInstanceOf[FileInputDStream[_, _, _]] - def recordedFiles = fileInputDStream.files.values.flatMap(x => x) + def recordedFiles = fileInputDStream.batchTimeToSelectedFiles.values.flatten assert(!recordedFiles.filter(_.endsWith("1")).isEmpty) assert(!recordedFiles.filter(_.endsWith("2")).isEmpty) assert(!recordedFiles.filter(_.endsWith("3")).isEmpty) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index fa04fa326e370..307052a4a9cbb 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -28,9 +28,12 @@ import java.util.concurrent.{Executors, TimeUnit, ArrayBlockingQueue} import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer, SynchronizedQueue} +import scala.concurrent.duration._ +import scala.language.postfixOps import com.google.common.io.Files import org.scalatest.BeforeAndAfter +import org.scalatest.concurrent.Eventually._ import org.apache.spark.Logging import org.apache.spark.storage.StorageLevel @@ -38,6 +41,9 @@ import org.apache.spark.streaming.util.ManualClock import org.apache.spark.util.Utils import org.apache.spark.streaming.receiver.{ActorHelper, Receiver} import org.apache.spark.rdd.RDD +import org.apache.hadoop.io.{Text, LongWritable} +import org.apache.hadoop.mapreduce.lib.input.TextInputFormat +import org.apache.hadoop.fs.Path class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { @@ -91,54 +97,12 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { } - test("file input stream") { - // Disable manual clock as FileInputDStream does not work with manual clock - conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock") - - // Set up the streaming context and input streams - val testDir = Utils.createTempDir() - val ssc = new StreamingContext(conf, batchDuration) - val fileStream = ssc.textFileStream(testDir.toString) - val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] - def output = outputBuffer.flatMap(x => x) - val outputStream = new TestOutputStream(fileStream, outputBuffer) - outputStream.register() - ssc.start() - - // Create files in the temporary directory so that Spark Streaming can read data from it - val input = Seq(1, 2, 3, 4, 5) - val expectedOutput = input.map(_.toString) - Thread.sleep(1000) - for (i <- 0 until input.size) { - val file = new File(testDir, i.toString) - Files.write(input(i) + "\n", file, Charset.forName("UTF-8")) - logInfo("Created file " + file) - Thread.sleep(batchDuration.milliseconds) - Thread.sleep(1000) - } - val startTime = System.currentTimeMillis() - Thread.sleep(1000) - val timeTaken = System.currentTimeMillis() - startTime - assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms") - logInfo("Stopping context") - ssc.stop() - - // Verify whether data received by Spark Streaming was as expected - logInfo("--------------------------------") - logInfo("output, size = " + outputBuffer.size) - outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]")) - logInfo("expected output, size = " + expectedOutput.size) - expectedOutput.foreach(x => logInfo("[" + x.mkString(",") + "]")) - logInfo("--------------------------------") - - // Verify whether all the elements received are as expected - // (whether the elements were received one in each interval is not verified) - assert(output.toList === expectedOutput.toList) - - Utils.deleteRecursively(testDir) + test("file input stream - newFilesOnly = true") { + testFileStream(newFilesOnly = true) + } - // Enable manual clock back again for other tests - conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") + test("file input stream - newFilesOnly = false") { + testFileStream(newFilesOnly = false) } test("multi-thread receiver") { @@ -180,7 +144,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { assert(output.sum === numTotalRecords) } - test("queue input stream - oneAtATime=true") { + test("queue input stream - oneAtATime = true") { // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) val queue = new SynchronizedQueue[RDD[String]]() @@ -223,7 +187,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { } } - test("queue input stream - oneAtATime=false") { + test("queue input stream - oneAtATime = false") { // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) val queue = new SynchronizedQueue[RDD[String]]() @@ -268,6 +232,50 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { assert(output(i) === expectedOutput(i)) } } + + def testFileStream(newFilesOnly: Boolean) { + var ssc: StreamingContext = null + val testDir: File = null + try { + val testDir = Utils.createTempDir() + val existingFile = new File(testDir, "0") + Files.write("0\n", existingFile, Charset.forName("UTF-8")) + + Thread.sleep(1000) + // Set up the streaming context and input streams + val newConf = conf.clone.set( + "spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock") + ssc = new StreamingContext(newConf, batchDuration) + val fileStream = ssc.fileStream[LongWritable, Text, TextInputFormat]( + testDir.toString, (x: Path) => true, newFilesOnly = newFilesOnly).map(_._2.toString) + val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] + val outputStream = new TestOutputStream(fileStream, outputBuffer) + outputStream.register() + ssc.start() + + // Create files in the directory + val input = Seq(1, 2, 3, 4, 5) + input.foreach { i => + Thread.sleep(batchDuration.milliseconds) + val file = new File(testDir, i.toString) + Files.write(i + "\n", file, Charset.forName("UTF-8")) + logInfo("Created file " + file) + } + + // Verify that all the files have been read + val expectedOutput = if (newFilesOnly) { + input.map(_.toString).toSet + } else { + (Seq(0) ++ input).map(_.toString).toSet + } + eventually(timeout(maxWaitTimeMillis milliseconds), interval(100 milliseconds)) { + assert(outputBuffer.flatten.toSet === expectedOutput) + } + } finally { + if (ssc != null) ssc.stop() + if (testDir != null) Utils.deleteRecursively(testDir) + } + } } From b660de7a9cbdea3df4a37fbcf60c1c33c71782b8 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 24 Nov 2014 16:37:14 -0800 Subject: [PATCH 080/109] [SPARK-4562] [MLlib] speedup vector This PR change the underline array of DenseVector to numpy.ndarray to avoid the conversion, because most of the users will using numpy.array. It also improve the serialization of DenseVector. Before this change: trial | trainingTime | testTime -------|--------|-------- 0 | 5.126 | 1.786 1 |2.698 |1.693 After the change: trial | trainingTime | testTime -------|--------|-------- 0 |4.692 |0.554 1 |2.307 |0.525 This could partially fix the performance regression during test. Author: Davies Liu Closes #3420 from davies/ser2 and squashes the following commits: 0e1e6f3 [Davies Liu] fix tests 426f5db [Davies Liu] impove toArray() 44707ec [Davies Liu] add name for ISO-8859-1 fa7d791 [Davies Liu] address comments 1cfb137 [Davies Liu] handle zero sparse vector 2548ee2 [Davies Liu] fix tests 9e6389d [Davies Liu] bugfix 470f702 [Davies Liu] speed up DenseMatrix f0d3c40 [Davies Liu] speedup SparseVector ef6ce70 [Davies Liu] speed up dense vector --- .../mllib/api/python/PythonMLLibAPI.scala | 73 +++++++++++++++++-- python/pyspark/mllib/linalg.py | 73 ++++++++++++------- python/pyspark/mllib/tests.py | 6 +- 3 files changed, 118 insertions(+), 34 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index f04df1c156898..9f20cd5d00dcd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -18,6 +18,7 @@ package org.apache.spark.mllib.api.python import java.io.OutputStream +import java.nio.{ByteBuffer, ByteOrder} import java.util.{ArrayList => JArrayList, List => JList, Map => JMap} import scala.collection.JavaConverters._ @@ -684,6 +685,7 @@ class PythonMLLibAPI extends Serializable { private[spark] object SerDe extends Serializable { val PYSPARK_PACKAGE = "pyspark.mllib" + val LATIN1 = "ISO-8859-1" /** * Base class used for pickle @@ -735,7 +737,16 @@ private[spark] object SerDe extends Serializable { def saveState(obj: Object, out: OutputStream, pickler: Pickler) = { val vector: DenseVector = obj.asInstanceOf[DenseVector] - saveObjects(out, pickler, vector.toArray) + val bytes = new Array[Byte](8 * vector.size) + val bb = ByteBuffer.wrap(bytes) + bb.order(ByteOrder.nativeOrder()) + val db = bb.asDoubleBuffer() + db.put(vector.values) + + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(bytes.length)) + out.write(bytes) + out.write(Opcodes.TUPLE1) } def construct(args: Array[Object]): Object = { @@ -743,7 +754,13 @@ private[spark] object SerDe extends Serializable { if (args.length != 1) { throw new PickleException("should be 1") } - new DenseVector(args(0).asInstanceOf[Array[Double]]) + val bytes = args(0).asInstanceOf[String].getBytes(LATIN1) + val bb = ByteBuffer.wrap(bytes, 0, bytes.length) + bb.order(ByteOrder.nativeOrder()) + val db = bb.asDoubleBuffer() + val ans = new Array[Double](bytes.length / 8) + db.get(ans) + Vectors.dense(ans) } } @@ -752,15 +769,30 @@ private[spark] object SerDe extends Serializable { def saveState(obj: Object, out: OutputStream, pickler: Pickler) = { val m: DenseMatrix = obj.asInstanceOf[DenseMatrix] - saveObjects(out, pickler, m.numRows, m.numCols, m.values) + val bytes = new Array[Byte](8 * m.values.size) + val order = ByteOrder.nativeOrder() + ByteBuffer.wrap(bytes).order(order).asDoubleBuffer().put(m.values) + + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(m.numRows)) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(m.numCols)) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(bytes.length)) + out.write(bytes) + out.write(Opcodes.TUPLE3) } def construct(args: Array[Object]): Object = { if (args.length != 3) { throw new PickleException("should be 3") } - new DenseMatrix(args(0).asInstanceOf[Int], args(1).asInstanceOf[Int], - args(2).asInstanceOf[Array[Double]]) + val bytes = args(2).asInstanceOf[String].getBytes(LATIN1) + val n = bytes.length / 8 + val values = new Array[Double](n) + val order = ByteOrder.nativeOrder() + ByteBuffer.wrap(bytes).order(order).asDoubleBuffer().get(values) + new DenseMatrix(args(0).asInstanceOf[Int], args(1).asInstanceOf[Int], values) } } @@ -769,15 +801,40 @@ private[spark] object SerDe extends Serializable { def saveState(obj: Object, out: OutputStream, pickler: Pickler) = { val v: SparseVector = obj.asInstanceOf[SparseVector] - saveObjects(out, pickler, v.size, v.indices, v.values) + val n = v.indices.size + val indiceBytes = new Array[Byte](4 * n) + val order = ByteOrder.nativeOrder() + ByteBuffer.wrap(indiceBytes).order(order).asIntBuffer().put(v.indices) + val valueBytes = new Array[Byte](8 * n) + ByteBuffer.wrap(valueBytes).order(order).asDoubleBuffer().put(v.values) + + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(v.size)) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(indiceBytes.length)) + out.write(indiceBytes) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(valueBytes.length)) + out.write(valueBytes) + out.write(Opcodes.TUPLE3) } def construct(args: Array[Object]): Object = { if (args.length != 3) { throw new PickleException("should be 3") } - new SparseVector(args(0).asInstanceOf[Int], args(1).asInstanceOf[Array[Int]], - args(2).asInstanceOf[Array[Double]]) + val size = args(0).asInstanceOf[Int] + val indiceBytes = args(1).asInstanceOf[String].getBytes(LATIN1) + val valueBytes = args(2).asInstanceOf[String].getBytes(LATIN1) + val n = indiceBytes.length / 4 + val indices = new Array[Int](n) + val values = new Array[Double](n) + if (n > 0) { + val order = ByteOrder.nativeOrder() + ByteBuffer.wrap(indiceBytes).order(order).asIntBuffer().get(indices) + ByteBuffer.wrap(valueBytes).order(order).asDoubleBuffer().get(values) + } + new SparseVector(size, indices, values) } } diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index 537b17657809c..f7aa2b0cb04b3 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -30,7 +30,7 @@ import numpy as np from pyspark.sql import UserDefinedType, StructField, StructType, ArrayType, DoubleType, \ - IntegerType, ByteType, Row + IntegerType, ByteType __all__ = ['Vector', 'DenseVector', 'SparseVector', 'Vectors', 'DenseMatrix', 'Matrices'] @@ -173,12 +173,16 @@ class DenseVector(Vector): A dense vector represented by a value array. """ def __init__(self, ar): - if not isinstance(ar, array.array): - ar = array.array('d', ar) + if isinstance(ar, basestring): + ar = np.frombuffer(ar, dtype=np.float64) + elif not isinstance(ar, np.ndarray): + ar = np.array(ar, dtype=np.float64) + if ar.dtype != np.float64: + ar.astype(np.float64) self.array = ar def __reduce__(self): - return DenseVector, (self.array,) + return DenseVector, (self.array.tostring(),) def dot(self, other): """ @@ -207,9 +211,10 @@ def dot(self, other): ... AssertionError: dimension mismatch """ - if type(other) == np.ndarray and other.ndim > 1: - assert len(self) == other.shape[0], "dimension mismatch" - return np.dot(self.toArray(), other) + if type(other) == np.ndarray: + if other.ndim > 1: + assert len(self) == other.shape[0], "dimension mismatch" + return np.dot(self.array, other) elif _have_scipy and scipy.sparse.issparse(other): assert len(self) == other.shape[0], "dimension mismatch" return other.transpose().dot(self.toArray()) @@ -261,7 +266,7 @@ def squared_distance(self, other): return np.dot(diff, diff) def toArray(self): - return np.array(self.array) + return self.array def __getitem__(self, item): return self.array[item] @@ -276,7 +281,7 @@ def __repr__(self): return "DenseVector([%s])" % (', '.join(_format_float(i) for i in self.array)) def __eq__(self, other): - return isinstance(other, DenseVector) and self.array == other.array + return isinstance(other, DenseVector) and np.array_equal(self.array, other.array) def __ne__(self, other): return not self == other @@ -314,18 +319,28 @@ def __init__(self, size, *args): if type(pairs) == dict: pairs = pairs.items() pairs = sorted(pairs) - self.indices = array.array('i', [p[0] for p in pairs]) - self.values = array.array('d', [p[1] for p in pairs]) + self.indices = np.array([p[0] for p in pairs], dtype=np.int32) + self.values = np.array([p[1] for p in pairs], dtype=np.float64) else: - assert len(args[0]) == len(args[1]), "index and value arrays not same length" - self.indices = array.array('i', args[0]) - self.values = array.array('d', args[1]) + if isinstance(args[0], basestring): + assert isinstance(args[1], str), "values should be string too" + if args[0]: + self.indices = np.frombuffer(args[0], np.int32) + self.values = np.frombuffer(args[1], np.float64) + else: + # np.frombuffer() doesn't work well with empty string in older version + self.indices = np.array([], dtype=np.int32) + self.values = np.array([], dtype=np.float64) + else: + self.indices = np.array(args[0], dtype=np.int32) + self.values = np.array(args[1], dtype=np.float64) + assert len(self.indices) == len(self.values), "index and value arrays not same length" for i in xrange(len(self.indices) - 1): if self.indices[i] >= self.indices[i + 1]: raise TypeError("indices array must be sorted") def __reduce__(self): - return (SparseVector, (self.size, self.indices, self.values)) + return (SparseVector, (self.size, self.indices.tostring(), self.values.tostring())) def dot(self, other): """ @@ -461,8 +476,7 @@ def toArray(self): Returns a copy of this SparseVector as a 1-dimensional NumPy array. """ arr = np.zeros((self.size,), dtype=np.float64) - for i in xrange(len(self.indices)): - arr[self.indices[i]] = self.values[i] + arr[self.indices] = self.values return arr def __len__(self): @@ -493,8 +507,8 @@ def __eq__(self, other): """ return (isinstance(other, self.__class__) and other.size == self.size - and other.indices == self.indices - and other.values == self.values) + and np.array_equal(other.indices, self.indices) + and np.array_equal(other.values, self.values)) def __ne__(self, other): return not self.__eq__(other) @@ -577,25 +591,34 @@ class DenseMatrix(Matrix): """ def __init__(self, numRows, numCols, values): Matrix.__init__(self, numRows, numCols) + if isinstance(values, basestring): + values = np.frombuffer(values, dtype=np.float64) + elif not isinstance(values, np.ndarray): + values = np.array(values, dtype=np.float64) assert len(values) == numRows * numCols - if not isinstance(values, array.array): - values = array.array('d', values) + if values.dtype != np.float64: + values.astype(np.float64) self.values = values def __reduce__(self): - return DenseMatrix, (self.numRows, self.numCols, self.values) + return DenseMatrix, (self.numRows, self.numCols, self.values.tostring()) def toArray(self): """ Return an numpy.ndarray - >>> arr = array.array('d', [float(i) for i in range(4)]) - >>> m = DenseMatrix(2, 2, arr) + >>> m = DenseMatrix(2, 2, range(4)) >>> m.toArray() array([[ 0., 2.], [ 1., 3.]]) """ - return np.reshape(self.values, (self.numRows, self.numCols), order='F') + return self.values.reshape((self.numRows, self.numCols), order='F') + + def __eq__(self, other): + return (isinstance(other, DenseMatrix) and + self.numRows == other.numRows and + self.numCols == other.numCols and + all(self.values == other.values)) class Matrices(object): diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 9fa4d6f6a2f5f..8332f8e061f48 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -33,7 +33,8 @@ else: import unittest -from pyspark.mllib.linalg import Vector, SparseVector, DenseVector, VectorUDT, _convert_to_vector +from pyspark.mllib.linalg import Vector, SparseVector, DenseVector, VectorUDT, _convert_to_vector,\ + DenseMatrix from pyspark.mllib.regression import LabeledPoint from pyspark.mllib.random import RandomRDDs from pyspark.mllib.stat import Statistics @@ -62,6 +63,7 @@ def _squared_distance(a, b): class VectorTests(PySparkTestCase): def _test_serialize(self, v): + self.assertEqual(v, ser.loads(ser.dumps(v))) jvec = self.sc._jvm.SerDe.loads(bytearray(ser.dumps(v))) nv = ser.loads(str(self.sc._jvm.SerDe.dumps(jvec))) self.assertEqual(v, nv) @@ -75,6 +77,8 @@ def test_serialize(self): self._test_serialize(DenseVector(array([1., 2., 3., 4.]))) self._test_serialize(DenseVector(pyarray.array('d', range(10)))) self._test_serialize(SparseVector(4, {1: 1, 3: 2})) + self._test_serialize(SparseVector(3, {})) + self._test_serialize(DenseMatrix(2, 3, range(6))) def test_dot(self): sv = SparseVector(4, {1: 1, 3: 2}) From 050616b408c60eae02256913ceb645912dbff62e Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 24 Nov 2014 16:41:23 -0800 Subject: [PATCH 081/109] [SPARK-4578] fix asDict() with nested Row() The Row object is created on the fly once the field is accessed, so we should access them by getattr() in asDict(0 Author: Davies Liu Closes #3434 from davies/fix_asDict and squashes the following commits: b20f1e7 [Davies Liu] fix asDict() with nested Row() --- python/pyspark/sql.py | 2 +- python/pyspark/tests.py | 7 ++++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index abb284d1e3dd9..ae288471b0e51 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -1178,7 +1178,7 @@ class Row(tuple): def asDict(self): """ Return as a dict """ - return dict(zip(self.__FIELDS__, self)) + return dict((n, getattr(self, n)) for n in self.__FIELDS__) def __repr__(self): # call collect __repr__ for nested objects diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index a01bd8d415787..29bcd38908d10 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -803,7 +803,7 @@ def setUpClass(cls): @classmethod def tearDownClass(cls): ReusedPySparkTestCase.tearDownClass() - shutil.rmtree(cls.tempdir.name) + shutil.rmtree(cls.tempdir.name, ignore_errors=True) def setUp(self): self.sqlCtx = SQLContext(self.sc) @@ -930,8 +930,9 @@ def test_convert_row_to_dict(self): rdd = self.sc.parallelize([row]) srdd = self.sqlCtx.inferSchema(rdd) srdd.registerTempTable("test") - row = self.sqlCtx.sql("select l[0].a AS la from test").first() - self.assertEqual(1, row.asDict()["la"]) + row = self.sqlCtx.sql("select l, d from test").first() + self.assertEqual(1, row.asDict()["l"][0].a) + self.assertEqual(1.0, row.asDict()['d']['key'].c) def test_infer_schema_with_udt(self): from pyspark.tests import ExamplePoint, ExamplePointUDT From 6cf507685efd01df77d663145ae08e48c7f92948 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 24 Nov 2014 17:17:03 -0800 Subject: [PATCH 082/109] [SPARK-4548] []SPARK-4517] improve performance of python broadcast Re-implement the Python broadcast using file: 1) serialize the python object using cPickle, write into disks. 2) Create a wrapper in JVM (for the dumped file), it read data from during serialization 3) Using TorrentBroadcast or HttpBroadcast to transfer the data (compressed) into executors 4) During deserialization, writing the data into disk. 5) Passing the path into Python worker, read data from disk and unpickle it into python object, until the first access. It fixes the performance regression introduced in #2659, has similar performance as 1.1, but support object larger than 2G, also improve the memory efficiency (only one compressed copy in driver and executor). Testing with a 500M broadcast and 4 tasks (excluding the benefit from reused worker in 1.2): name | 1.1 | 1.2 with this patch | improvement ---------|--------|---------|-------- python-broadcast-w-bytes | 25.20 | 9.33 | 170.13% | python-broadcast-w-set | 4.13 | 4.50 | -8.35% | Testing with 100 tasks (16 CPUs): name | 1.1 | 1.2 with this patch | improvement ---------|--------|---------|-------- python-broadcast-w-bytes | 38.16 | 8.40 | 353.98% python-broadcast-w-set | 23.29 | 9.59 | 142.80% Author: Davies Liu Closes #3417 from davies/pybroadcast and squashes the following commits: 50a58e0 [Davies Liu] address comments b98de1d [Davies Liu] disable gc while unpickle e5ee6b9 [Davies Liu] support large string 09303b8 [Davies Liu] read all data into memory dde02dd [Davies Liu] improve performance of python broadcast --- .../apache/spark/api/python/PythonRDD.scala | 73 ++++--- python/pyspark/broadcast.py | 95 +++++++--- python/pyspark/context.py | 12 +- python/pyspark/serializers.py | 178 +----------------- python/pyspark/tests.py | 18 +- python/pyspark/worker.py | 10 +- .../apache/spark/sql/UdfRegistration.scala | 3 +- .../spark/sql/execution/pythonUdfs.scala | 4 +- 8 files changed, 135 insertions(+), 258 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index b80c771d58a8f..e0bc00e1eb249 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -19,7 +19,7 @@ package org.apache.spark.api.python import java.io._ import java.net._ -import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collections} +import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, UUID, Collections} import org.apache.spark.input.PortableDataStream @@ -47,7 +47,7 @@ private[spark] class PythonRDD( pythonIncludes: JList[String], preservePartitoning: Boolean, pythonExec: String, - broadcastVars: JList[Broadcast[Array[Array[Byte]]]], + broadcastVars: JList[Broadcast[PythonBroadcast]], accumulator: Accumulator[JList[Array[Byte]]]) extends RDD[Array[Byte]](parent) { @@ -230,8 +230,7 @@ private[spark] class PythonRDD( if (!oldBids.contains(broadcast.id)) { // send new broadcast dataOut.writeLong(broadcast.id) - dataOut.writeLong(broadcast.value.map(_.length.toLong).sum) - broadcast.value.foreach(dataOut.write) + PythonRDD.writeUTF(broadcast.value.path, dataOut) oldBids.add(broadcast.id) } } @@ -368,24 +367,8 @@ private[spark] object PythonRDD extends Logging { } } - def readBroadcastFromFile( - sc: JavaSparkContext, - filename: String): Broadcast[Array[Array[Byte]]] = { - val size = new File(filename).length() - val file = new DataInputStream(new FileInputStream(filename)) - val blockSize = 1 << 20 - val n = ((size + blockSize - 1) / blockSize).toInt - val obj = new Array[Array[Byte]](n) - try { - for (i <- 0 until n) { - val length = if (i < (n - 1)) blockSize else (size % blockSize).toInt - obj(i) = new Array[Byte](length) - file.readFully(obj(i)) - } - } finally { - file.close() - } - sc.broadcast(obj) + def readBroadcastFromFile(sc: JavaSparkContext, path: String): Broadcast[PythonBroadcast] = { + sc.broadcast(new PythonBroadcast(path)) } def writeIteratorToStream[T](iter: Iterator[T], dataOut: DataOutputStream) { @@ -824,3 +807,49 @@ private class PythonAccumulatorParam(@transient serverHost: String, serverPort: } } } + +/** + * An Wrapper for Python Broadcast, which is written into disk by Python. It also will + * write the data into disk after deserialization, then Python can read it from disks. + */ +private[spark] class PythonBroadcast(@transient var path: String) extends Serializable { + + /** + * Read data from disks, then copy it to `out` + */ + private def writeObject(out: ObjectOutputStream): Unit = Utils.tryOrIOException { + val in = new FileInputStream(new File(path)) + try { + Utils.copyStream(in, out) + } finally { + in.close() + } + } + + /** + * Write data into disk, using randomly generated name. + */ + private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { + val dir = new File(Utils.getLocalDir(SparkEnv.get.conf)) + val file = File.createTempFile("broadcast", "", dir) + path = file.getAbsolutePath + val out = new FileOutputStream(file) + try { + Utils.copyStream(in, out) + } finally { + out.close() + } + } + + /** + * Delete the file once the object is GCed. + */ + override def finalize() { + if (!path.isEmpty) { + val file = new File(path) + if (file.exists()) { + file.delete() + } + } + } +} diff --git a/python/pyspark/broadcast.py b/python/pyspark/broadcast.py index 01cac3c72c690..6b8a8b256a891 100644 --- a/python/pyspark/broadcast.py +++ b/python/pyspark/broadcast.py @@ -15,21 +15,10 @@ # limitations under the License. # -""" ->>> from pyspark.context import SparkContext ->>> sc = SparkContext('local', 'test') ->>> b = sc.broadcast([1, 2, 3, 4, 5]) ->>> b.value -[1, 2, 3, 4, 5] ->>> sc.parallelize([0, 0]).flatMap(lambda x: b.value).collect() -[1, 2, 3, 4, 5, 1, 2, 3, 4, 5] ->>> b.unpersist() - ->>> large_broadcast = sc.broadcast(list(range(10000))) -""" import os - -from pyspark.serializers import LargeObjectSerializer +import cPickle +import gc +from tempfile import NamedTemporaryFile __all__ = ['Broadcast'] @@ -49,44 +38,88 @@ def _from_id(bid): class Broadcast(object): """ - A broadcast variable created with - L{SparkContext.broadcast()}. + A broadcast variable created with L{SparkContext.broadcast()}. Access its value through C{.value}. + + Examples: + + >>> from pyspark.context import SparkContext + >>> sc = SparkContext('local', 'test') + >>> b = sc.broadcast([1, 2, 3, 4, 5]) + >>> b.value + [1, 2, 3, 4, 5] + >>> sc.parallelize([0, 0]).flatMap(lambda x: b.value).collect() + [1, 2, 3, 4, 5, 1, 2, 3, 4, 5] + >>> b.unpersist() + + >>> large_broadcast = sc.broadcast(range(10000)) """ - def __init__(self, bid, value, java_broadcast=None, - pickle_registry=None, path=None): + def __init__(self, sc=None, value=None, pickle_registry=None, path=None): """ - Should not be called directly by users -- use - L{SparkContext.broadcast()} + Should not be called directly by users -- use L{SparkContext.broadcast()} instead. """ - self.bid = bid - if path is None: - self._value = value - self._jbroadcast = java_broadcast - self._pickle_registry = pickle_registry - self.path = path + if sc is not None: + f = NamedTemporaryFile(delete=False, dir=sc._temp_dir) + self._path = self.dump(value, f) + self._jbroadcast = sc._jvm.PythonRDD.readBroadcastFromFile(sc._jsc, self._path) + self._pickle_registry = pickle_registry + else: + self._jbroadcast = None + self._path = path + + def dump(self, value, f): + if isinstance(value, basestring): + if isinstance(value, unicode): + f.write('U') + value = value.encode('utf8') + else: + f.write('S') + f.write(value) + else: + f.write('P') + cPickle.dump(value, f, 2) + f.close() + return f.name + + def load(self, path): + with open(path, 'rb', 1 << 20) as f: + flag = f.read(1) + data = f.read() + if flag == 'P': + # cPickle.loads() may create lots of objects, disable GC + # temporary for better performance + gc.disable() + try: + return cPickle.loads(data) + finally: + gc.enable() + else: + return data.decode('utf8') if flag == 'U' else data @property def value(self): """ Return the broadcasted value """ - if not hasattr(self, "_value") and self.path is not None: - ser = LargeObjectSerializer() - self._value = ser.load_stream(open(self.path)).next() + if not hasattr(self, "_value") and self._path is not None: + self._value = self.load(self._path) return self._value def unpersist(self, blocking=False): """ Delete cached copies of this broadcast on the executors. """ + if self._jbroadcast is None: + raise Exception("Broadcast can only be unpersisted in driver") self._jbroadcast.unpersist(blocking) - os.unlink(self.path) + os.unlink(self._path) def __reduce__(self): + if self._jbroadcast is None: + raise Exception("Broadcast can only be serialized in driver") self._pickle_registry.add(self) - return (_from_id, (self.bid, )) + return _from_id, (self._jbroadcast.id(),) if __name__ == "__main__": diff --git a/python/pyspark/context.py b/python/pyspark/context.py index ec67ec8d0f824..ed7351d60cff2 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -29,7 +29,7 @@ from pyspark.files import SparkFiles from pyspark.java_gateway import launch_gateway from pyspark.serializers import PickleSerializer, BatchedSerializer, UTF8Deserializer, \ - PairDeserializer, AutoBatchedSerializer, NoOpSerializer, LargeObjectSerializer + PairDeserializer, AutoBatchedSerializer, NoOpSerializer from pyspark.storagelevel import StorageLevel from pyspark.rdd import RDD from pyspark.traceback_utils import CallSite, first_spark_call @@ -624,15 +624,7 @@ def broadcast(self, value): object for reading it in distributed functions. The variable will be sent to each cluster only once. """ - ser = LargeObjectSerializer() - - # pass large object by py4j is very slow and need much memory - tempFile = NamedTemporaryFile(delete=False, dir=self._temp_dir) - ser.dump_stream([value], tempFile) - tempFile.close() - jbroadcast = self._jvm.PythonRDD.readBroadcastFromFile(self._jsc, tempFile.name) - return Broadcast(jbroadcast.id(), None, jbroadcast, - self._pickled_broadcast_vars, tempFile.name) + return Broadcast(self, value, self._pickled_broadcast_vars) def accumulator(self, value, accum_param=None): """ diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 760a509f0ef6d..33aa55f7f1429 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -448,184 +448,20 @@ def loads(self, obj): raise ValueError("invalid sevialization type: %s" % _type) -class SizeLimitedStream(object): - """ - Read at most `limit` bytes from underlying stream - - >>> from StringIO import StringIO - >>> io = StringIO() - >>> io.write("Hello world") - >>> io.seek(0) - >>> lio = SizeLimitedStream(io, 5) - >>> lio.read() - 'Hello' - """ - def __init__(self, stream, limit): - self.stream = stream - self.limit = limit - - def read(self, n=0): - if n > self.limit or n == 0: - n = self.limit - buf = self.stream.read(n) - self.limit -= len(buf) - return buf - - -class CompressedStream(object): - """ - Compress the data using zlib - - >>> from StringIO import StringIO - >>> io = StringIO() - >>> wio = CompressedStream(io, 'w') - >>> wio.write("Hello world") - >>> wio.flush() - >>> io.seek(0) - >>> rio = CompressedStream(io, 'r') - >>> rio.read() - 'Hello world' - >>> rio.read() - '' - """ - MAX_BATCH = 1 << 20 # 1MB - - def __init__(self, stream, mode='w', level=1): - self.stream = stream - self.mode = mode - if mode == 'w': - self.compresser = zlib.compressobj(level) - elif mode == 'r': - self.decompresser = zlib.decompressobj() - self.buf = '' - else: - raise ValueError("can only support mode 'w' or 'r' ") - - def write(self, buf): - assert self.mode == 'w', "It's not opened for write" - if len(buf) > self.MAX_BATCH: - # zlib can not compress string larger than 2G - batches = len(buf) / self.MAX_BATCH + 1 # last one may be empty - for i in xrange(batches): - self.write(buf[i * self.MAX_BATCH:(i + 1) * self.MAX_BATCH]) - else: - compressed = self.compresser.compress(buf) - self.stream.write(compressed) - - def flush(self, mode=zlib.Z_FULL_FLUSH): - if self.mode == 'w': - d = self.compresser.flush(mode) - self.stream.write(d) - self.stream.flush() - - def close(self): - if self.mode == 'w': - self.flush(zlib.Z_FINISH) - self.stream.close() - - def read(self, size=0): - assert self.mode == 'r', "It's not opened for read" - if not size: - data = self.stream.read() - result = self.decompresser.decompress(data) - last = self.decompresser.flush() - return self.buf + result + last - - # fast path for small read() - if size <= len(self.buf): - result = self.buf[:size] - self.buf = self.buf[size:] - return result - - result = [self.buf] - size -= len(self.buf) - self.buf = '' - while size: - need = min(size, self.MAX_BATCH) - input = self.stream.read(need) - if input: - buf = self.decompresser.decompress(input) - else: - buf = self.decompresser.flush() - - if len(buf) >= size: - self.buf = buf[size:] - result.append(buf[:size]) - return ''.join(result) - - size -= len(buf) - result.append(buf) - if not input: - return ''.join(result) - - def readline(self): - """ - This is needed for pickle, but not used in protocol 2 - """ - line = [] - b = self.read(1) - while b and b != '\n': - line.append(b) - b = self.read(1) - line.append(b) - return ''.join(line) - - -class LargeObjectSerializer(Serializer): - """ - Serialize large object which could be larger than 2G - - It uses cPickle to serialize the objects - """ - def dump_stream(self, iterator, stream): - stream = CompressedStream(stream, 'w') - for value in iterator: - if isinstance(value, basestring): - if isinstance(value, unicode): - stream.write('U') - value = value.encode("utf-8") - else: - stream.write('S') - write_long(len(value), stream) - stream.write(value) - else: - stream.write('P') - cPickle.dump(value, stream, 2) - stream.flush() - - def load_stream(self, stream): - stream = CompressedStream(stream, 'r') - while True: - type = stream.read(1) - if not type: - return - if type in ('S', 'U'): - length = read_long(stream) - value = stream.read(length) - if type == 'U': - value = value.decode('utf-8') - yield value - elif type == 'P': - yield cPickle.load(stream) - else: - raise ValueError("unknown type: %s" % type) - - -class CompressedSerializer(Serializer): +class CompressedSerializer(FramedSerializer): """ Compress the serialized data """ def __init__(self, serializer): + FramedSerializer.__init__(self) + assert isinstance(serializer, FramedSerializer), "serializer must be a FramedSerializer" self.serializer = serializer - def load_stream(self, stream): - stream = CompressedStream(stream, "r") - return self.serializer.load_stream(stream) + def dumps(self, obj): + return zlib.compress(self.serializer.dumps(obj), 1) - def dump_stream(self, iterator, stream): - stream = CompressedStream(stream, "w") - self.serializer.dump_stream(iterator, stream) - stream.flush() + def loads(self, obj): + return self.serializer.loads(zlib.decompress(obj)) class UTF8Deserializer(Serializer): diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 29bcd38908d10..32645778c2b8f 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -48,7 +48,7 @@ from pyspark.context import SparkContext from pyspark.files import SparkFiles from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer, \ - CloudPickleSerializer, SizeLimitedStream, CompressedSerializer, LargeObjectSerializer + CloudPickleSerializer, CompressedSerializer from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, ExternalSorter from pyspark.sql import SQLContext, IntegerType, Row, ArrayType, StructType, StructField, \ UserDefinedType, DoubleType @@ -237,26 +237,16 @@ def foo(): self.assertTrue("exit" in foo.func_code.co_names) ser.dumps(foo) - def _test_serializer(self, ser): + def test_compressed_serializer(self): + ser = CompressedSerializer(PickleSerializer()) from StringIO import StringIO io = StringIO() ser.dump_stream(["abc", u"123", range(5)], io) io.seek(0) self.assertEqual(["abc", u"123", range(5)], list(ser.load_stream(io))) - size = io.tell() ser.dump_stream(range(1000), io) io.seek(0) - first = SizeLimitedStream(io, size) - self.assertEqual(["abc", u"123", range(5)], list(ser.load_stream(first))) - self.assertEqual(range(1000), list(ser.load_stream(io))) - - def test_compressed_serializer(self): - ser = CompressedSerializer(PickleSerializer()) - self._test_serializer(ser) - - def test_large_object_serializer(self): - ser = LargeObjectSerializer() - self._test_serializer(ser) + self.assertEqual(["abc", u"123", range(5)] + range(1000), list(ser.load_stream(io))) class PySparkTestCase(unittest.TestCase): diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index e1552a0b0b4ff..7e5343c973dc5 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -30,8 +30,7 @@ from pyspark.broadcast import Broadcast, _broadcastRegistry from pyspark.files import SparkFiles from pyspark.serializers import write_with_length, write_int, read_long, \ - write_long, read_int, SpecialLengths, UTF8Deserializer, PickleSerializer, \ - SizeLimitedStream, LargeObjectSerializer + write_long, read_int, SpecialLengths, UTF8Deserializer, PickleSerializer from pyspark import shuffle pickleSer = PickleSerializer() @@ -78,14 +77,11 @@ def main(infile, outfile): # fetch names and values of broadcast variables num_broadcast_variables = read_int(infile) - bser = LargeObjectSerializer() for _ in range(num_broadcast_variables): bid = read_long(infile) if bid >= 0: - size = read_long(infile) - s = SizeLimitedStream(infile, size) - value = list((bser.load_stream(s)))[0] # read out all the bytes - _broadcastRegistry[bid] = Broadcast(bid, value) + path = utf8_deserializer.loads(infile) + _broadcastRegistry[bid] = Broadcast(path=path) else: bid = - bid - 1 _broadcastRegistry.pop(bid) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala index ddcb5db6c3a21..00d6b43a57812 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql import java.util.{List => JList, Map => JMap} import org.apache.spark.Accumulator +import org.apache.spark.api.python.PythonBroadcast import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.expressions.{Expression, ScalaUdf} @@ -39,7 +40,7 @@ private[sql] trait UDFRegistration { envVars: JMap[String, String], pythonIncludes: JList[String], pythonExec: String, - broadcastVars: JList[Broadcast[Array[Array[Byte]]]], + broadcastVars: JList[Broadcast[PythonBroadcast]], accumulator: Accumulator[JList[Array[Byte]]], stringDataType: String): Unit = { log.debug( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala index f98cae3f17e4a..2b4a88d5e864e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala @@ -26,7 +26,7 @@ import scala.collection.JavaConverters._ import net.razorvine.pickle.{Pickler, Unpickler} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.api.python.PythonRDD +import org.apache.spark.api.python.{PythonBroadcast, PythonRDD} import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.catalyst.expressions._ @@ -45,7 +45,7 @@ private[spark] case class PythonUDF( envVars: JMap[String, String], pythonIncludes: JList[String], pythonExec: String, - broadcastVars: JList[Broadcast[Array[Array[Byte]]]], + broadcastVars: JList[Broadcast[PythonBroadcast]], accumulator: Accumulator[JList[Array[Byte]]], dataType: DataType, children: Seq[Expression]) extends Expression with SparkLogging { From d24d5bf064572a2319627736b1fbf112b4a78edf Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Mon, 24 Nov 2014 18:03:10 -0800 Subject: [PATCH 083/109] [SPARK-4266] [Web-UI] Reduce stage page load time. The commit changes the java script used to show/hide additional metrics in order to reduce page load time. SPARK-4016 significantly increased page load time for the stage page when stages had a lot (thousands or tens of thousands) of tasks, due to the additional Javascript to hide some metrics by default and stripe the tables. This commit reduces page load time in two ways: (1) Now, all of the metrics that are hidden by default are hidden by setting "display: none;" using CSS for the page, rather than hiding them using javascript after the page loads. Without this change, for stages with thousands of tasks, there was a few second delay after page load, where first the additional metrics were shown, and then after a delay were hidden once the relevant JS finished running. (2) CSS is used to stripe all of the tables except for the summary table. The summary table needs javascript to do the striping because some rows are hidden, but the javascript striping is slower, which again resulted in a delay when it was used for the task table (where for a few seconds after page load, all of the rows in the task table would be white, while the browser finished running the JS to stripe the table). cc pwendell This change is intended to be backported to 1.2 to avoid a regression in UI performance when users run large jobs. Author: Kay Ousterhout Closes #3328 from kayousterhout/SPARK-4266 and squashes the following commits: f964091 [Kay Ousterhout] [SPARK-4266] [Web-UI] Reduce stage page load time. --- .../spark/ui/static/additional-metrics.js | 9 ++----- .../org/apache/spark/ui/static/table.js | 24 ++++++++++--------- .../org/apache/spark/ui/static/webui.css | 6 +++++ .../scala/org/apache/spark/ui/UIUtils.scala | 11 ++++----- .../apache/spark/ui/exec/ExecutorsPage.scala | 2 +- .../apache/spark/ui/jobs/ExecutorTable.scala | 2 +- .../org/apache/spark/ui/jobs/StagePage.scala | 9 ++++++- .../spark/ui/jobs/TaskDetailsClassNames.scala | 3 +++ 8 files changed, 39 insertions(+), 27 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js b/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js index d33c5c769d683..14ba37d7c9bd9 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js +++ b/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js @@ -28,17 +28,12 @@ $(function() { $(this).find('.expand-additional-metrics-arrow').toggleClass('arrow-closed'); }); - $("input:checkbox:not(:checked)").each(function() { - var column = "table ." + $(this).attr("name"); - $(column).hide(); - }); - // Stripe table rows after rows have been hidden to ensure correct striping. - stripeTables(); + stripeSummaryTable(); $("input:checkbox").click(function() { var column = "table ." + $(this).attr("name"); $(column).toggle(); - stripeTables(); + stripeSummaryTable(); }); $("#select-all-metrics").click(function() { diff --git a/core/src/main/resources/org/apache/spark/ui/static/table.js b/core/src/main/resources/org/apache/spark/ui/static/table.js index 6bb03015abb51..656147e40d13e 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/table.js +++ b/core/src/main/resources/org/apache/spark/ui/static/table.js @@ -15,16 +15,18 @@ * limitations under the License. */ -/* Adds background colors to stripe table rows. This is necessary (instead of using css or the - * table striping provided by bootstrap) to appropriately stripe tables with hidden rows. */ -function stripeTables() { - $("table.table-striped-custom").each(function() { - $(this).find("tr:not(:hidden)").each(function (index) { - if (index % 2 == 1) { - $(this).css("background-color", "#f9f9f9"); - } else { - $(this).css("background-color", "#ffffff"); - } - }); +/* Adds background colors to stripe table rows in the summary table (on the stage page). This is + * necessary (instead of using css or the table striping provided by bootstrap) because the summary + * table has hidden rows. + * + * An ID selector (rather than a class selector) is used to ensure this runs quickly even on pages + * with thousands of task rows (ID selectors are much faster than class selectors). */ +function stripeSummaryTable() { + $("#task-summary-table").find("tr:not(:hidden)").each(function (index) { + if (index % 2 == 1) { + $(this).css("background-color", "#f9f9f9"); + } else { + $(this).css("background-color", "#ffffff"); + } }); } 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 db57712c83503..cdf85bfbf326f 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 @@ -168,3 +168,9 @@ span.additional-metric-title { border-left: 5px solid black; display: inline-block; } + +/* 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 { + display: none; +} 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 0c418beaf7581..09079bbd43f6f 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -26,7 +26,8 @@ import org.apache.spark.Logging /** Utility functions for generating XML pages with spark content. */ private[spark] object UIUtils extends Logging { - val TABLE_CLASS = "table table-bordered table-striped-custom table-condensed sortable" + val TABLE_CLASS_NOT_STRIPED = "table table-bordered table-condensed sortable" + val TABLE_CLASS_STRIPED = TABLE_CLASS_NOT_STRIPED + " table-striped" // SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use. private val dateFormat = new ThreadLocal[SimpleDateFormat]() { @@ -248,12 +249,10 @@ private[spark] object UIUtils extends Logging { data: Iterable[T], fixedWidth: Boolean = false, id: Option[String] = None, - headerClasses: Seq[String] = Seq.empty): Seq[Node] = { + headerClasses: Seq[String] = Seq.empty, + stripeRowsWithCss: Boolean = true): Seq[Node] = { - var listingTableClass = TABLE_CLASS - if (fixedWidth) { - listingTableClass += " table-fixed" - } + val listingTableClass = if (stripeRowsWithCss) TABLE_CLASS_STRIPED else TABLE_CLASS_NOT_STRIPED val colWidth = 100.toDouble / headers.size val colWidthAttr = if (fixedWidth) colWidth + "%" else "" diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index 71b59b1d078ca..363cb96de7998 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -57,7 +57,7 @@ private[ui] class ExecutorsPage( val execInfoSorted = execInfo.sortBy(_.id) val execTable = - +
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index 35bbe8b4f9ac8..9836d11a6d85f 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -36,7 +36,7 @@ private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: Stage /** Special table which merges two header cells. */ private def executorTable[T](): Seq[Node] = { -
Executor ID Address
+
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 40e05f86b661d..bfa54f8492068 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 @@ -322,8 +322,15 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { val quantileHeaders = Seq("Metric", "Min", "25th percentile", "Median", "75th percentile", "Max") + // The summary table does not use CSS to stripe rows, which doesn't work with hidden + // rows (instead, JavaScript in table.js is used to stripe the non-hidden rows). Some(UIUtils.listingTable( - quantileHeaders, identity[Seq[Node]], listings, fixedWidth = true)) + quantileHeaders, + identity[Seq[Node]], + listings, + fixedWidth = true, + id = Some("task-summary-table"), + stripeRowsWithCss = false)) } val executorTable = new ExecutorTable(stageId, stageAttemptId, parent) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala b/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala index eb371bd0ea7ed..ca942c4051c84 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala @@ -20,6 +20,9 @@ package org.apache.spark.ui.jobs /** * Names of the CSS classes corresponding to each type of task detail. Used to allow users * to optionally show/hide columns. + * + * If new optional metrics are added here, they should also be added to the end of webui.css + * to have the style set to "display: none;" by default. */ private object TaskDetailsClassNames { val SCHEDULER_DELAY = "scheduler_delay" From b043c27424d05e3200e7ba99a1a65656b57fa2f0 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 24 Nov 2014 19:14:14 -0800 Subject: [PATCH 084/109] [SPARK-4525] Mesos should decline unused offers Functionally, this is just a small change on top of #3393 (by jongyoul). The issue being addressed is discussed in the comments there. I have not yet added a test for the bug there. I will add one shortly. I've also done some minor renaming/clean-up of variables in this class and tests. Author: Patrick Wendell Author: Jongyoul Lee Closes #3436 from pwendell/mesos-issue and squashes the following commits: 58c35b5 [Patrick Wendell] Adding unit test for this situation c4f0697 [Patrick Wendell] Additional clean-up and fixes on top of existing fix f20f1b3 [Jongyoul Lee] [SPARK-4525] MesosSchedulerBackend.resourceOffers cannot decline unused offers from acceptedOffers - Added code for declining unused offers among acceptedOffers - Edited testCase for checking declining unused offers --- .../cluster/mesos/MesosSchedulerBackend.scala | 25 ++++++-- .../mesos/MesosSchedulerBackendSuite.scala | 61 ++++++++++++++----- 2 files changed, 65 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index d13795186c48e..10e6886c16a4f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -208,10 +208,12 @@ private[spark] class MesosSchedulerBackend( */ override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) { inClassLoader() { - val (acceptedOffers, declinedOffers) = offers.partition { o => + // Fail-fast on offers we know will be rejected + val (usableOffers, unUsableOffers) = offers.partition { o => val mem = getResource(o.getResourcesList, "mem") val cpus = getResource(o.getResourcesList, "cpus") val slaveId = o.getSlaveId.getValue + // TODO(pwendell): Should below be 1 + scheduler.CPUS_PER_TASK? (mem >= MemoryUtils.calculateTotalMemory(sc) && // need at least 1 for executor, 1 for task cpus >= 2 * scheduler.CPUS_PER_TASK) || @@ -219,11 +221,12 @@ private[spark] class MesosSchedulerBackend( cpus >= scheduler.CPUS_PER_TASK) } - val offerableWorkers = acceptedOffers.map { o => + val workerOffers = usableOffers.map { o => val cpus = if (slaveIdsWithExecutors.contains(o.getSlaveId.getValue)) { getResource(o.getResourcesList, "cpus").toInt } else { // If the executor doesn't exist yet, subtract CPU for executor + // TODO(pwendell): Should below just subtract "1"? getResource(o.getResourcesList, "cpus").toInt - scheduler.CPUS_PER_TASK } @@ -233,17 +236,20 @@ private[spark] class MesosSchedulerBackend( cpus) } - val slaveIdToOffer = acceptedOffers.map(o => o.getSlaveId.getValue -> o).toMap + val slaveIdToOffer = usableOffers.map(o => o.getSlaveId.getValue -> o).toMap val mesosTasks = new HashMap[String, JArrayList[MesosTaskInfo]] + val slavesIdsOfAcceptedOffers = HashSet[String]() + // Call into the TaskSchedulerImpl - scheduler.resourceOffers(offerableWorkers) - .filter(!_.isEmpty) + val acceptedOffers = scheduler.resourceOffers(workerOffers).filter(!_.isEmpty) + acceptedOffers .foreach { offer => offer.foreach { taskDesc => val slaveId = taskDesc.executorId slaveIdsWithExecutors += slaveId + slavesIdsOfAcceptedOffers += slaveId taskIdToSlaveId(taskDesc.taskId) = slaveId mesosTasks.getOrElseUpdate(slaveId, new JArrayList[MesosTaskInfo]) .add(createMesosTask(taskDesc, slaveId)) @@ -257,7 +263,14 @@ private[spark] class MesosSchedulerBackend( d.launchTasks(Collections.singleton(slaveIdToOffer(slaveId).getId), tasks, filters) } - declinedOffers.foreach(o => d.declineOffer(o.getId)) + // Decline offers that weren't used + // NOTE: This logic assumes that we only get a single offer for each host in a given batch + for (o <- usableOffers if !slavesIdsOfAcceptedOffers.contains(o.getSlaveId.getValue)) { + d.declineOffer(o.getId) + } + + // Decline offers we ruled out immediately + unUsableOffers.foreach(o => d.declineOffer(o.getId)) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala index bef8d3a58ba63..e60e70afd3218 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala @@ -30,9 +30,11 @@ import java.nio.ByteBuffer import java.util.Collections import java.util import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with EasyMockSugar { - test("mesos resource offer is launching tasks") { + + test("mesos resource offers result in launching tasks") { def createOffer(id: Int, mem: Int, cpu: Int) = { val builder = Offer.newBuilder() builder.addResourcesBuilder() @@ -43,46 +45,61 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Ea .setName("cpus") .setType(Value.Type.SCALAR) .setScalar(Scalar.newBuilder().setValue(cpu)) - builder.setId(OfferID.newBuilder().setValue(id.toString).build()).setFrameworkId(FrameworkID.newBuilder().setValue("f1")) - .setSlaveId(SlaveID.newBuilder().setValue("s1")).setHostname("localhost").build() + builder.setId(OfferID.newBuilder().setValue(s"o${id.toString}").build()).setFrameworkId(FrameworkID.newBuilder().setValue("f1")) + .setSlaveId(SlaveID.newBuilder().setValue(s"s${id.toString}")).setHostname(s"host${id.toString}").build() } val driver = EasyMock.createMock(classOf[SchedulerDriver]) val taskScheduler = EasyMock.createMock(classOf[TaskSchedulerImpl]) val sc = EasyMock.createMock(classOf[SparkContext]) - EasyMock.expect(sc.executorMemory).andReturn(100).anyTimes() EasyMock.expect(sc.getSparkHome()).andReturn(Option("/path")).anyTimes() EasyMock.expect(sc.executorEnvs).andReturn(new mutable.HashMap).anyTimes() EasyMock.expect(sc.conf).andReturn(new SparkConf).anyTimes() EasyMock.replay(sc) + val minMem = MemoryUtils.calculateTotalMemory(sc).toInt val minCpu = 4 - val offers = new java.util.ArrayList[Offer] - offers.add(createOffer(1, minMem, minCpu)) - offers.add(createOffer(1, minMem - 1, minCpu)) + + val mesosOffers = new java.util.ArrayList[Offer] + mesosOffers.add(createOffer(1, minMem, minCpu)) + mesosOffers.add(createOffer(2, minMem - 1, minCpu)) + mesosOffers.add(createOffer(3, minMem, minCpu)) + val backend = new MesosSchedulerBackend(taskScheduler, sc, "master") - val workerOffers = Seq(offers.get(0)).map(o => new WorkerOffer( - o.getSlaveId.getValue, - o.getHostname, + + val expectedWorkerOffers = new ArrayBuffer[WorkerOffer](2) + expectedWorkerOffers.append(new WorkerOffer( + mesosOffers.get(0).getSlaveId.getValue, + mesosOffers.get(0).getHostname, + 2 + )) + expectedWorkerOffers.append(new WorkerOffer( + mesosOffers.get(2).getSlaveId.getValue, + mesosOffers.get(2).getHostname, 2 )) val taskDesc = new TaskDescription(1L, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0))) - EasyMock.expect(taskScheduler.resourceOffers(EasyMock.eq(workerOffers))).andReturn(Seq(Seq(taskDesc))) + EasyMock.expect(taskScheduler.resourceOffers(EasyMock.eq(expectedWorkerOffers))).andReturn(Seq(Seq(taskDesc))) EasyMock.expect(taskScheduler.CPUS_PER_TASK).andReturn(2).anyTimes() EasyMock.replay(taskScheduler) + val capture = new Capture[util.Collection[TaskInfo]] EasyMock.expect( driver.launchTasks( - EasyMock.eq(Collections.singleton(offers.get(0).getId)), + EasyMock.eq(Collections.singleton(mesosOffers.get(0).getId)), EasyMock.capture(capture), EasyMock.anyObject(classOf[Filters]) ) - ).andReturn(Status.valueOf(1)) - EasyMock.expect(driver.declineOffer(offers.get(1).getId)).andReturn(Status.valueOf(1)) + ).andReturn(Status.valueOf(1)).once + EasyMock.expect(driver.declineOffer(mesosOffers.get(1).getId)).andReturn(Status.valueOf(1)).times(1) + EasyMock.expect(driver.declineOffer(mesosOffers.get(2).getId)).andReturn(Status.valueOf(1)).times(1) EasyMock.replay(driver) - backend.resourceOffers(driver, offers) + + backend.resourceOffers(driver, mesosOffers) + + EasyMock.verify(driver) assert(capture.getValue.size() == 1) val taskInfo = capture.getValue.iterator().next() assert(taskInfo.getName.equals("n1")) @@ -90,5 +107,19 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Ea assert(cpus.getName.equals("cpus")) assert(cpus.getScalar.getValue.equals(2.0)) assert(taskInfo.getSlaveId.getValue.equals("s1")) + + // Unwanted resources offered on an existing node. Make sure they are declined + val mesosOffers2 = new java.util.ArrayList[Offer] + mesosOffers2.add(createOffer(1, minMem, minCpu)) + EasyMock.reset(taskScheduler) + EasyMock.reset(driver) + EasyMock.expect(taskScheduler.resourceOffers(EasyMock.anyObject(classOf[Seq[WorkerOffer]])).andReturn(Seq(Seq()))) + EasyMock.expect(taskScheduler.CPUS_PER_TASK).andReturn(2).anyTimes() + EasyMock.replay(taskScheduler) + EasyMock.expect(driver.declineOffer(mesosOffers2.get(0).getId)).andReturn(Status.valueOf(1)).times(1) + EasyMock.replay(driver) + + backend.resourceOffers(driver, mesosOffers2) + EasyMock.verify(driver) } } From a68d442270fa85b5831d201016f3e911af128679 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 24 Nov 2014 19:16:53 -0800 Subject: [PATCH 085/109] Revert "[SPARK-4525] Mesos should decline unused offers" This reverts commit b043c27424d05e3200e7ba99a1a65656b57fa2f0. I accidentally committed this using my own authorship credential. However, I should have given authoriship to the original author: Jongyoul Lee. --- .../cluster/mesos/MesosSchedulerBackend.scala | 25 ++------ .../mesos/MesosSchedulerBackendSuite.scala | 61 +++++-------------- 2 files changed, 21 insertions(+), 65 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 10e6886c16a4f..d13795186c48e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -208,12 +208,10 @@ private[spark] class MesosSchedulerBackend( */ override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) { inClassLoader() { - // Fail-fast on offers we know will be rejected - val (usableOffers, unUsableOffers) = offers.partition { o => + val (acceptedOffers, declinedOffers) = offers.partition { o => val mem = getResource(o.getResourcesList, "mem") val cpus = getResource(o.getResourcesList, "cpus") val slaveId = o.getSlaveId.getValue - // TODO(pwendell): Should below be 1 + scheduler.CPUS_PER_TASK? (mem >= MemoryUtils.calculateTotalMemory(sc) && // need at least 1 for executor, 1 for task cpus >= 2 * scheduler.CPUS_PER_TASK) || @@ -221,12 +219,11 @@ private[spark] class MesosSchedulerBackend( cpus >= scheduler.CPUS_PER_TASK) } - val workerOffers = usableOffers.map { o => + val offerableWorkers = acceptedOffers.map { o => val cpus = if (slaveIdsWithExecutors.contains(o.getSlaveId.getValue)) { getResource(o.getResourcesList, "cpus").toInt } else { // If the executor doesn't exist yet, subtract CPU for executor - // TODO(pwendell): Should below just subtract "1"? getResource(o.getResourcesList, "cpus").toInt - scheduler.CPUS_PER_TASK } @@ -236,20 +233,17 @@ private[spark] class MesosSchedulerBackend( cpus) } - val slaveIdToOffer = usableOffers.map(o => o.getSlaveId.getValue -> o).toMap + val slaveIdToOffer = acceptedOffers.map(o => o.getSlaveId.getValue -> o).toMap val mesosTasks = new HashMap[String, JArrayList[MesosTaskInfo]] - val slavesIdsOfAcceptedOffers = HashSet[String]() - // Call into the TaskSchedulerImpl - val acceptedOffers = scheduler.resourceOffers(workerOffers).filter(!_.isEmpty) - acceptedOffers + scheduler.resourceOffers(offerableWorkers) + .filter(!_.isEmpty) .foreach { offer => offer.foreach { taskDesc => val slaveId = taskDesc.executorId slaveIdsWithExecutors += slaveId - slavesIdsOfAcceptedOffers += slaveId taskIdToSlaveId(taskDesc.taskId) = slaveId mesosTasks.getOrElseUpdate(slaveId, new JArrayList[MesosTaskInfo]) .add(createMesosTask(taskDesc, slaveId)) @@ -263,14 +257,7 @@ private[spark] class MesosSchedulerBackend( d.launchTasks(Collections.singleton(slaveIdToOffer(slaveId).getId), tasks, filters) } - // Decline offers that weren't used - // NOTE: This logic assumes that we only get a single offer for each host in a given batch - for (o <- usableOffers if !slavesIdsOfAcceptedOffers.contains(o.getSlaveId.getValue)) { - d.declineOffer(o.getId) - } - - // Decline offers we ruled out immediately - unUsableOffers.foreach(o => d.declineOffer(o.getId)) + declinedOffers.foreach(o => d.declineOffer(o.getId)) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala index e60e70afd3218..bef8d3a58ba63 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala @@ -30,11 +30,9 @@ import java.nio.ByteBuffer import java.util.Collections import java.util import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with EasyMockSugar { - - test("mesos resource offers result in launching tasks") { + test("mesos resource offer is launching tasks") { def createOffer(id: Int, mem: Int, cpu: Int) = { val builder = Offer.newBuilder() builder.addResourcesBuilder() @@ -45,61 +43,46 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Ea .setName("cpus") .setType(Value.Type.SCALAR) .setScalar(Scalar.newBuilder().setValue(cpu)) - builder.setId(OfferID.newBuilder().setValue(s"o${id.toString}").build()).setFrameworkId(FrameworkID.newBuilder().setValue("f1")) - .setSlaveId(SlaveID.newBuilder().setValue(s"s${id.toString}")).setHostname(s"host${id.toString}").build() + builder.setId(OfferID.newBuilder().setValue(id.toString).build()).setFrameworkId(FrameworkID.newBuilder().setValue("f1")) + .setSlaveId(SlaveID.newBuilder().setValue("s1")).setHostname("localhost").build() } val driver = EasyMock.createMock(classOf[SchedulerDriver]) val taskScheduler = EasyMock.createMock(classOf[TaskSchedulerImpl]) val sc = EasyMock.createMock(classOf[SparkContext]) + EasyMock.expect(sc.executorMemory).andReturn(100).anyTimes() EasyMock.expect(sc.getSparkHome()).andReturn(Option("/path")).anyTimes() EasyMock.expect(sc.executorEnvs).andReturn(new mutable.HashMap).anyTimes() EasyMock.expect(sc.conf).andReturn(new SparkConf).anyTimes() EasyMock.replay(sc) - val minMem = MemoryUtils.calculateTotalMemory(sc).toInt val minCpu = 4 - - val mesosOffers = new java.util.ArrayList[Offer] - mesosOffers.add(createOffer(1, minMem, minCpu)) - mesosOffers.add(createOffer(2, minMem - 1, minCpu)) - mesosOffers.add(createOffer(3, minMem, minCpu)) - + val offers = new java.util.ArrayList[Offer] + offers.add(createOffer(1, minMem, minCpu)) + offers.add(createOffer(1, minMem - 1, minCpu)) val backend = new MesosSchedulerBackend(taskScheduler, sc, "master") - - val expectedWorkerOffers = new ArrayBuffer[WorkerOffer](2) - expectedWorkerOffers.append(new WorkerOffer( - mesosOffers.get(0).getSlaveId.getValue, - mesosOffers.get(0).getHostname, - 2 - )) - expectedWorkerOffers.append(new WorkerOffer( - mesosOffers.get(2).getSlaveId.getValue, - mesosOffers.get(2).getHostname, + val workerOffers = Seq(offers.get(0)).map(o => new WorkerOffer( + o.getSlaveId.getValue, + o.getHostname, 2 )) val taskDesc = new TaskDescription(1L, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0))) - EasyMock.expect(taskScheduler.resourceOffers(EasyMock.eq(expectedWorkerOffers))).andReturn(Seq(Seq(taskDesc))) + EasyMock.expect(taskScheduler.resourceOffers(EasyMock.eq(workerOffers))).andReturn(Seq(Seq(taskDesc))) EasyMock.expect(taskScheduler.CPUS_PER_TASK).andReturn(2).anyTimes() EasyMock.replay(taskScheduler) - val capture = new Capture[util.Collection[TaskInfo]] EasyMock.expect( driver.launchTasks( - EasyMock.eq(Collections.singleton(mesosOffers.get(0).getId)), + EasyMock.eq(Collections.singleton(offers.get(0).getId)), EasyMock.capture(capture), EasyMock.anyObject(classOf[Filters]) ) - ).andReturn(Status.valueOf(1)).once - EasyMock.expect(driver.declineOffer(mesosOffers.get(1).getId)).andReturn(Status.valueOf(1)).times(1) - EasyMock.expect(driver.declineOffer(mesosOffers.get(2).getId)).andReturn(Status.valueOf(1)).times(1) + ).andReturn(Status.valueOf(1)) + EasyMock.expect(driver.declineOffer(offers.get(1).getId)).andReturn(Status.valueOf(1)) EasyMock.replay(driver) - - backend.resourceOffers(driver, mesosOffers) - - EasyMock.verify(driver) + backend.resourceOffers(driver, offers) assert(capture.getValue.size() == 1) val taskInfo = capture.getValue.iterator().next() assert(taskInfo.getName.equals("n1")) @@ -107,19 +90,5 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Ea assert(cpus.getName.equals("cpus")) assert(cpus.getScalar.getValue.equals(2.0)) assert(taskInfo.getSlaveId.getValue.equals("s1")) - - // Unwanted resources offered on an existing node. Make sure they are declined - val mesosOffers2 = new java.util.ArrayList[Offer] - mesosOffers2.add(createOffer(1, minMem, minCpu)) - EasyMock.reset(taskScheduler) - EasyMock.reset(driver) - EasyMock.expect(taskScheduler.resourceOffers(EasyMock.anyObject(classOf[Seq[WorkerOffer]])).andReturn(Seq(Seq()))) - EasyMock.expect(taskScheduler.CPUS_PER_TASK).andReturn(2).anyTimes() - EasyMock.replay(taskScheduler) - EasyMock.expect(driver.declineOffer(mesosOffers2.get(0).getId)).andReturn(Status.valueOf(1)).times(1) - EasyMock.replay(driver) - - backend.resourceOffers(driver, mesosOffers2) - EasyMock.verify(driver) } } From f0afb623dc51fd3008bd80496b8d1eaa991323d6 Mon Sep 17 00:00:00 2001 From: Jongyoul Lee Date: Mon, 24 Nov 2014 19:14:14 -0800 Subject: [PATCH 086/109] [SPARK-4525] Mesos should decline unused offers Functionally, this is just a small change on top of #3393 (by jongyoul). The issue being addressed is discussed in the comments there. I have not yet added a test for the bug there. I will add one shortly. I've also done some minor renaming/clean-up of variables in this class and tests. Author: Patrick Wendell Author: Jongyoul Lee Closes #3436 from pwendell/mesos-issue and squashes the following commits: 58c35b5 [Patrick Wendell] Adding unit test for this situation c4f0697 [Patrick Wendell] Additional clean-up and fixes on top of existing fix f20f1b3 [Jongyoul Lee] [SPARK-4525] MesosSchedulerBackend.resourceOffers cannot decline unused offers from acceptedOffers - Added code for declining unused offers among acceptedOffers - Edited testCase for checking declining unused offers --- .../cluster/mesos/MesosSchedulerBackend.scala | 25 ++++++-- .../mesos/MesosSchedulerBackendSuite.scala | 61 ++++++++++++++----- 2 files changed, 65 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index d13795186c48e..10e6886c16a4f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -208,10 +208,12 @@ private[spark] class MesosSchedulerBackend( */ override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) { inClassLoader() { - val (acceptedOffers, declinedOffers) = offers.partition { o => + // Fail-fast on offers we know will be rejected + val (usableOffers, unUsableOffers) = offers.partition { o => val mem = getResource(o.getResourcesList, "mem") val cpus = getResource(o.getResourcesList, "cpus") val slaveId = o.getSlaveId.getValue + // TODO(pwendell): Should below be 1 + scheduler.CPUS_PER_TASK? (mem >= MemoryUtils.calculateTotalMemory(sc) && // need at least 1 for executor, 1 for task cpus >= 2 * scheduler.CPUS_PER_TASK) || @@ -219,11 +221,12 @@ private[spark] class MesosSchedulerBackend( cpus >= scheduler.CPUS_PER_TASK) } - val offerableWorkers = acceptedOffers.map { o => + val workerOffers = usableOffers.map { o => val cpus = if (slaveIdsWithExecutors.contains(o.getSlaveId.getValue)) { getResource(o.getResourcesList, "cpus").toInt } else { // If the executor doesn't exist yet, subtract CPU for executor + // TODO(pwendell): Should below just subtract "1"? getResource(o.getResourcesList, "cpus").toInt - scheduler.CPUS_PER_TASK } @@ -233,17 +236,20 @@ private[spark] class MesosSchedulerBackend( cpus) } - val slaveIdToOffer = acceptedOffers.map(o => o.getSlaveId.getValue -> o).toMap + val slaveIdToOffer = usableOffers.map(o => o.getSlaveId.getValue -> o).toMap val mesosTasks = new HashMap[String, JArrayList[MesosTaskInfo]] + val slavesIdsOfAcceptedOffers = HashSet[String]() + // Call into the TaskSchedulerImpl - scheduler.resourceOffers(offerableWorkers) - .filter(!_.isEmpty) + val acceptedOffers = scheduler.resourceOffers(workerOffers).filter(!_.isEmpty) + acceptedOffers .foreach { offer => offer.foreach { taskDesc => val slaveId = taskDesc.executorId slaveIdsWithExecutors += slaveId + slavesIdsOfAcceptedOffers += slaveId taskIdToSlaveId(taskDesc.taskId) = slaveId mesosTasks.getOrElseUpdate(slaveId, new JArrayList[MesosTaskInfo]) .add(createMesosTask(taskDesc, slaveId)) @@ -257,7 +263,14 @@ private[spark] class MesosSchedulerBackend( d.launchTasks(Collections.singleton(slaveIdToOffer(slaveId).getId), tasks, filters) } - declinedOffers.foreach(o => d.declineOffer(o.getId)) + // Decline offers that weren't used + // NOTE: This logic assumes that we only get a single offer for each host in a given batch + for (o <- usableOffers if !slavesIdsOfAcceptedOffers.contains(o.getSlaveId.getValue)) { + d.declineOffer(o.getId) + } + + // Decline offers we ruled out immediately + unUsableOffers.foreach(o => d.declineOffer(o.getId)) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala index bef8d3a58ba63..e60e70afd3218 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala @@ -30,9 +30,11 @@ import java.nio.ByteBuffer import java.util.Collections import java.util import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with EasyMockSugar { - test("mesos resource offer is launching tasks") { + + test("mesos resource offers result in launching tasks") { def createOffer(id: Int, mem: Int, cpu: Int) = { val builder = Offer.newBuilder() builder.addResourcesBuilder() @@ -43,46 +45,61 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Ea .setName("cpus") .setType(Value.Type.SCALAR) .setScalar(Scalar.newBuilder().setValue(cpu)) - builder.setId(OfferID.newBuilder().setValue(id.toString).build()).setFrameworkId(FrameworkID.newBuilder().setValue("f1")) - .setSlaveId(SlaveID.newBuilder().setValue("s1")).setHostname("localhost").build() + builder.setId(OfferID.newBuilder().setValue(s"o${id.toString}").build()).setFrameworkId(FrameworkID.newBuilder().setValue("f1")) + .setSlaveId(SlaveID.newBuilder().setValue(s"s${id.toString}")).setHostname(s"host${id.toString}").build() } val driver = EasyMock.createMock(classOf[SchedulerDriver]) val taskScheduler = EasyMock.createMock(classOf[TaskSchedulerImpl]) val sc = EasyMock.createMock(classOf[SparkContext]) - EasyMock.expect(sc.executorMemory).andReturn(100).anyTimes() EasyMock.expect(sc.getSparkHome()).andReturn(Option("/path")).anyTimes() EasyMock.expect(sc.executorEnvs).andReturn(new mutable.HashMap).anyTimes() EasyMock.expect(sc.conf).andReturn(new SparkConf).anyTimes() EasyMock.replay(sc) + val minMem = MemoryUtils.calculateTotalMemory(sc).toInt val minCpu = 4 - val offers = new java.util.ArrayList[Offer] - offers.add(createOffer(1, minMem, minCpu)) - offers.add(createOffer(1, minMem - 1, minCpu)) + + val mesosOffers = new java.util.ArrayList[Offer] + mesosOffers.add(createOffer(1, minMem, minCpu)) + mesosOffers.add(createOffer(2, minMem - 1, minCpu)) + mesosOffers.add(createOffer(3, minMem, minCpu)) + val backend = new MesosSchedulerBackend(taskScheduler, sc, "master") - val workerOffers = Seq(offers.get(0)).map(o => new WorkerOffer( - o.getSlaveId.getValue, - o.getHostname, + + val expectedWorkerOffers = new ArrayBuffer[WorkerOffer](2) + expectedWorkerOffers.append(new WorkerOffer( + mesosOffers.get(0).getSlaveId.getValue, + mesosOffers.get(0).getHostname, + 2 + )) + expectedWorkerOffers.append(new WorkerOffer( + mesosOffers.get(2).getSlaveId.getValue, + mesosOffers.get(2).getHostname, 2 )) val taskDesc = new TaskDescription(1L, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0))) - EasyMock.expect(taskScheduler.resourceOffers(EasyMock.eq(workerOffers))).andReturn(Seq(Seq(taskDesc))) + EasyMock.expect(taskScheduler.resourceOffers(EasyMock.eq(expectedWorkerOffers))).andReturn(Seq(Seq(taskDesc))) EasyMock.expect(taskScheduler.CPUS_PER_TASK).andReturn(2).anyTimes() EasyMock.replay(taskScheduler) + val capture = new Capture[util.Collection[TaskInfo]] EasyMock.expect( driver.launchTasks( - EasyMock.eq(Collections.singleton(offers.get(0).getId)), + EasyMock.eq(Collections.singleton(mesosOffers.get(0).getId)), EasyMock.capture(capture), EasyMock.anyObject(classOf[Filters]) ) - ).andReturn(Status.valueOf(1)) - EasyMock.expect(driver.declineOffer(offers.get(1).getId)).andReturn(Status.valueOf(1)) + ).andReturn(Status.valueOf(1)).once + EasyMock.expect(driver.declineOffer(mesosOffers.get(1).getId)).andReturn(Status.valueOf(1)).times(1) + EasyMock.expect(driver.declineOffer(mesosOffers.get(2).getId)).andReturn(Status.valueOf(1)).times(1) EasyMock.replay(driver) - backend.resourceOffers(driver, offers) + + backend.resourceOffers(driver, mesosOffers) + + EasyMock.verify(driver) assert(capture.getValue.size() == 1) val taskInfo = capture.getValue.iterator().next() assert(taskInfo.getName.equals("n1")) @@ -90,5 +107,19 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Ea assert(cpus.getName.equals("cpus")) assert(cpus.getScalar.getValue.equals(2.0)) assert(taskInfo.getSlaveId.getValue.equals("s1")) + + // Unwanted resources offered on an existing node. Make sure they are declined + val mesosOffers2 = new java.util.ArrayList[Offer] + mesosOffers2.add(createOffer(1, minMem, minCpu)) + EasyMock.reset(taskScheduler) + EasyMock.reset(driver) + EasyMock.expect(taskScheduler.resourceOffers(EasyMock.anyObject(classOf[Seq[WorkerOffer]])).andReturn(Seq(Seq()))) + EasyMock.expect(taskScheduler.CPUS_PER_TASK).andReturn(2).anyTimes() + EasyMock.replay(taskScheduler) + EasyMock.expect(driver.declineOffer(mesosOffers2.get(0).getId)).andReturn(Status.valueOf(1)).times(1) + EasyMock.replay(driver) + + backend.resourceOffers(driver, mesosOffers2) + EasyMock.verify(driver) } } From 9ce2bf3821784cded5c2f1e1a75d038f7677be74 Mon Sep 17 00:00:00 2001 From: tkaessmann Date: Mon, 24 Nov 2014 19:58:01 -0800 Subject: [PATCH 087/109] [SPARK-4582][MLLIB] get raw vectors for further processing in Word2Vec This is #3309 for the master branch. e.g. clustering Author: tkaessmann Closes #3309 from tkaessmann/branch-1.2 and squashes the following commits: e3a3142 [tkaessmann] changes the comment for getVectors 58d3d83 [tkaessmann] removes sign from comment a5be213 [tkaessmann] fixes getVectors to fit code guidelines 3782fa9 [tkaessmann] get raw vectors for further processing Author: tkaessmann Closes #3437 from mengxr/SPARK-4582 and squashes the following commits: 6c666b4 [tkaessmann] get raw vectors for further processing in Word2Vec --- .../scala/org/apache/spark/mllib/feature/Word2Vec.scala | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index f5f7ad613d4c4..7960f3cab576f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -461,4 +461,11 @@ class Word2VecModel private[mllib] ( .tail .toArray } + + /** + * Returns a map of words to their vector representations. + */ + def getVectors: Map[String, Array[Float]] = { + model + } } From 723be60e233d0f85944d948efd06845ef546c9f5 Mon Sep 17 00:00:00 2001 From: w00228970 Date: Mon, 24 Nov 2014 21:17:24 -0800 Subject: [PATCH 088/109] [SQL] Compute timeTaken correctly ```timeTaken``` should not count the time of printing result. Author: w00228970 Closes #3423 from scwf/time-taken-bug and squashes the following commits: da7e102 [w00228970] compute time taken correctly --- .../sql/hive/thriftserver/SparkSQLCLIDriver.scala | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index 2cd02ae9269f5..7385952861ee5 100755 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -272,8 +272,10 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { if (sessionState.getIsVerbose) { out.println(cmd) } - val rc = driver.run(cmd) + val end = System.currentTimeMillis() + val timeTaken:Double = (end - start) / 1000.0 + ret = rc.getResponseCode if (ret != 0) { console.printError(rc.getErrorMessage()) @@ -309,12 +311,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { ret = cret } - val end = System.currentTimeMillis() - if (end > start) { - val timeTaken:Double = (end - start) / 1000.0 - console.printInfo(s"Time taken: $timeTaken seconds", null) - } - + console.printInfo(s"Time taken: $timeTaken seconds", null) // Destroy the driver to release all the locks. driver.destroy() } else { From 0fe54cff19759dad2dc2a0950bd6c1d31c95e858 Mon Sep 17 00:00:00 2001 From: wangfei Date: Mon, 24 Nov 2014 22:32:39 -0800 Subject: [PATCH 089/109] [DOC][Build] Wrong cmd for build spark with apache hadoop 2.4.X and hive 12 Author: wangfei Closes #3335 from scwf/patch-10 and squashes the following commits: d343113 [wangfei] add '-Phive' 60d595e [wangfei] [DOC] Wrong cmd for build spark with apache hadoop 2.4.X and Hive 12 support --- docs/building-spark.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/building-spark.md b/docs/building-spark.md index fee6a8440634c..40a47410e683a 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -112,7 +112,7 @@ Hive 0.12.0 using the `-Phive-0.12.0` profile. mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -Phive -Phive-thriftserver -DskipTests clean package # Apache Hadoop 2.4.X with Hive 12 support -mvn -Pyarn -Phive -Phive-thriftserver-0.12.0 -Phadoop-2.4 -Dhadoop.version=2.4.0 -Phive -Phive-thriftserver -DskipTests clean package +mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -Phive -Phive-0.12.0 -Phive-thriftserver -DskipTests clean package {% endhighlight %} # Building for Scala 2.11 From 89f912264603741c7d980135c26102d63e11791f Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Tue, 25 Nov 2014 01:57:34 -0800 Subject: [PATCH 090/109] [SPARK-4596][MLLib] Refactorize Normalizer to make code cleaner In this refactoring, the performance will be slightly increased due to removing the overhead from breeze vector. The bottleneck is still in breeze norm which is implemented by activeIterator. This inefficiency of breeze norm will be addressed in next PR. At least, this PR makes the code more consistent in the codebase. Author: DB Tsai Closes #3446 from dbtsai/normalizer and squashes the following commits: e20a2b9 [DB Tsai] first commit --- .../spark/mllib/feature/Normalizer.scala | 29 ++++++++++++------- 1 file changed, 19 insertions(+), 10 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala index dfad25d57c947..a9c2e23717896 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala @@ -17,10 +17,10 @@ package org.apache.spark.mllib.feature -import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, norm => brzNorm} +import breeze.linalg.{norm => brzNorm} import org.apache.spark.annotation.Experimental -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} /** * :: Experimental :: @@ -47,22 +47,31 @@ class Normalizer(p: Double) extends VectorTransformer { * @return normalized vector. If the norm of the input is zero, it will return the input vector. */ override def transform(vector: Vector): Vector = { - var norm = brzNorm(vector.toBreeze, p) + val norm = brzNorm(vector.toBreeze, p) if (norm != 0.0) { // For dense vector, we've to allocate new memory for new output vector. // However, for sparse vector, the `index` array will not be changed, // so we can re-use it to save memory. - vector.toBreeze match { - case dv: BDV[Double] => Vectors.fromBreeze(dv :/ norm) - case sv: BSV[Double] => - val output = new BSV[Double](sv.index, sv.data.clone(), sv.length) + vector match { + case dv: DenseVector => + val values = dv.values.clone() + val size = values.size var i = 0 - while (i < output.data.length) { - output.data(i) /= norm + while (i < size) { + values(i) /= norm i += 1 } - Vectors.fromBreeze(output) + Vectors.dense(values) + case sv: SparseVector => + val values = sv.values.clone() + val nnz = values.size + var i = 0 + while (i < nnz) { + values(i) /= norm + i += 1 + } + Vectors.sparse(sv.size, sv.indices, values) case v => throw new IllegalArgumentException("Do not support vector type " + v.getClass) } } else { From f515f9432b05f7e090b651c5536aa706d1cde487 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Tue, 25 Nov 2014 02:01:19 -0800 Subject: [PATCH 091/109] [SPARK-4526][MLLIB]GradientDescent get a wrong gradient value according to the gradient formula. This is caused by the miniBatchSize parameter.The number of `RDD.sample` returns is not fixed. cc mengxr Author: GuoQiang Li Closes #3399 from witgo/GradientDescent and squashes the following commits: 13cb228 [GuoQiang Li] review commit 668ab66 [GuoQiang Li] Double to Long b6aa11a [GuoQiang Li] Check miniBatchSize is greater than 0 0b5c3e3 [GuoQiang Li] Minor fix 12e7424 [GuoQiang Li] GradientDescent get a wrong gradient value according to the gradient formula, which is caused by the miniBatchSize parameter. --- .../mllib/optimization/GradientDescent.scala | 45 +++++++++++-------- 1 file changed, 26 insertions(+), 19 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index a6912056395d7..0857877951c82 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -160,14 +160,15 @@ object GradientDescent extends Logging { val stochasticLossHistory = new ArrayBuffer[Double](numIterations) val numExamples = data.count() - val miniBatchSize = numExamples * miniBatchFraction // if no data, return initial weights to avoid NaNs if (numExamples == 0) { - - logInfo("GradientDescent.runMiniBatchSGD returning initial weights, no data found") + logWarning("GradientDescent.runMiniBatchSGD returning initial weights, no data found") return (initialWeights, stochasticLossHistory.toArray) + } + if (numExamples * miniBatchFraction < 1) { + logWarning("The miniBatchFraction is too small") } // Initialize weights as a column vector @@ -185,25 +186,31 @@ object GradientDescent extends Logging { val bcWeights = data.context.broadcast(weights) // Sample a subset (fraction miniBatchFraction) of the total data // compute and sum up the subgradients on this subset (this is one map-reduce) - val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42 + i) - .treeAggregate((BDV.zeros[Double](n), 0.0))( - seqOp = (c, v) => (c, v) match { case ((grad, loss), (label, features)) => - val l = gradient.compute(features, label, bcWeights.value, Vectors.fromBreeze(grad)) - (grad, loss + l) + val (gradientSum, lossSum, miniBatchSize) = data.sample(false, miniBatchFraction, 42 + i) + .treeAggregate((BDV.zeros[Double](n), 0.0, 0L))( + seqOp = (c, v) => { + // c: (grad, loss, count), v: (label, features) + val l = gradient.compute(v._2, v._1, bcWeights.value, Vectors.fromBreeze(c._1)) + (c._1, c._2 + l, c._3 + 1) }, - combOp = (c1, c2) => (c1, c2) match { case ((grad1, loss1), (grad2, loss2)) => - (grad1 += grad2, loss1 + loss2) + combOp = (c1, c2) => { + // c: (grad, loss, count) + (c1._1 += c2._1, c1._2 + c2._2, c1._3 + c2._3) }) - /** - * NOTE(Xinghao): lossSum is computed using the weights from the previous iteration - * and regVal is the regularization value computed in the previous iteration as well. - */ - stochasticLossHistory.append(lossSum / miniBatchSize + regVal) - val update = updater.compute( - weights, Vectors.fromBreeze(gradientSum / miniBatchSize), stepSize, i, regParam) - weights = update._1 - regVal = update._2 + if (miniBatchSize > 0) { + /** + * NOTE(Xinghao): lossSum is computed using the weights from the previous iteration + * and regVal is the regularization value computed in the previous iteration as well. + */ + stochasticLossHistory.append(lossSum / miniBatchSize + regVal) + val update = updater.compute( + weights, Vectors.fromBreeze(gradientSum / miniBatchSize.toDouble), stepSize, i, regParam) + weights = update._1 + regVal = update._2 + } else { + logWarning(s"Iteration ($i/$numIterations). The size of sampled batch is zero") + } } logInfo("GradientDescent.runMiniBatchSGD finished. Last 10 stochastic losses %s".format( From a51118a34a4617c07373480c4b021e53124c3c00 Mon Sep 17 00:00:00 2001 From: q00251598 Date: Tue, 25 Nov 2014 04:01:56 -0800 Subject: [PATCH 092/109] [SPARK-4535][Streaming] Fix the error in comments change `NetworkInputDStream` to `ReceiverInputDStream` change `ReceiverInputTracker` to `ReceiverTracker` Author: q00251598 Closes #3400 from watermen/fix-comments and squashes the following commits: 75d795c [q00251598] change 'NetworkInputDStream' to 'ReceiverInputDStream' && change 'ReceiverInputTracker' to 'ReceiverTracker' --- .../examples/streaming/StatefulNetworkWordCount.scala | 2 +- .../examples/streaming/clickstream/PageViewStream.scala | 2 +- .../org/apache/spark/streaming/StreamingContext.scala | 2 +- .../spark/streaming/api/java/JavaStreamingContext.scala | 2 +- .../spark/streaming/dstream/ReceiverInputDStream.scala | 8 ++++---- .../spark/streaming/scheduler/ReceiverTracker.scala | 2 +- 6 files changed, 9 insertions(+), 9 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala index 514252b89e74e..ed186ea5650c4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala @@ -64,7 +64,7 @@ object StatefulNetworkWordCount { // Initial RDD input to updateStateByKey val initialRDD = ssc.sparkContext.parallelize(List(("hello", 1), ("world", 1))) - // Create a NetworkInputDStream on target ip:port and count the + // Create a ReceiverInputDStream on target ip:port and count the // words in input stream of \n delimited test (eg. generated by 'nc') val lines = ssc.socketTextStream(args(0), args(1).toInt) val words = lines.flatMap(_.split(" ")) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala index d9b886eff77cc..55226c0a6df60 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala @@ -50,7 +50,7 @@ object PageViewStream { val ssc = new StreamingContext("local[2]", "PageViewStream", Seconds(1), System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) - // Create a NetworkInputDStream on target host:port and convert each line to a PageView + // Create a ReceiverInputDStream on target host:port and convert each line to a PageView val pageViews = ssc.socketTextStream(host, port) .flatMap(_.split("\n")) .map(PageView.fromString(_)) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 54b219711efb9..ec59221459c88 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -187,7 +187,7 @@ class StreamingContext private[streaming] ( /** * Set each DStreams in this context to remember RDDs it generated in the last given duration. * DStreams remember RDDs only for a limited duration of time and releases them for garbage - * collection. This method allows the developer to specify how to long to remember the RDDs ( + * collection. This method allows the developer to specify how long to remember the RDDs ( * if the developer wishes to query old data outside the DStream computation). * @param duration Minimum duration that each DStream should remember its RDDs */ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index 7db66c69a6d73..d8695b8e05962 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -479,7 +479,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { /** * Sets each DStreams in this context to remember RDDs it generated in the last given duration. * DStreams remember RDDs only for a limited duration of duration and releases them for garbage - * collection. This method allows the developer to specify how to long to remember the RDDs ( + * collection. This method allows the developer to specify how long to remember the RDDs ( * if the developer wishes to query old data outside the DStream computation). * @param duration Minimum duration that each DStream should remember its RDDs */ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala index 3e67161363e50..c834744631e02 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala @@ -29,7 +29,7 @@ import org.apache.spark.streaming.scheduler.ReceivedBlockInfo /** * Abstract class for defining any [[org.apache.spark.streaming.dstream.InputDStream]] * that has to start a receiver on worker nodes to receive external data. - * Specific implementations of NetworkInputDStream must + * Specific implementations of ReceiverInputDStream must * define `the getReceiver()` function that gets the receiver object of type * [[org.apache.spark.streaming.receiver.Receiver]] that will be sent * to the workers to receive data. @@ -39,17 +39,17 @@ import org.apache.spark.streaming.scheduler.ReceivedBlockInfo abstract class ReceiverInputDStream[T: ClassTag](@transient ssc_ : StreamingContext) extends InputDStream[T](ssc_) { - /** This is an unique identifier for the network input stream. */ + /** This is an unique identifier for the receiver input stream. */ val id = ssc.getNewReceiverStreamId() /** * Gets the receiver object that will be sent to the worker nodes * to receive data. This method needs to defined by any specific implementation - * of a NetworkInputDStream. + * of a ReceiverInputDStream. */ def getReceiver(): Receiver[T] - // Nothing to start or stop as both taken care of by the ReceiverInputTracker. + // Nothing to start or stop as both taken care of by the ReceiverTracker. def start() {} def stop() {} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 1c3984d968d20..32e481dabc8ca 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -46,7 +46,7 @@ private[streaming] case class DeregisterReceiver(streamId: Int, msg: String, err extends ReceiverTrackerMessage /** - * This class manages the execution of the receivers of NetworkInputDStreams. Instance of + * This class manages the execution of the receivers of ReceiverInputDStreams. Instance of * this class must be created after all input streams have been added and StreamingContext.start() * has been called because it needs the final set of input streams at the time of instantiation. * From fef27b29431c2adadc17580f26c23afa6a3bd1d2 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 25 Nov 2014 05:36:29 -0800 Subject: [PATCH 093/109] [SPARK-4381][Streaming]Add warning log when user set spark.master to local in Spark Streaming and there's no job executed Author: jerryshao Closes #3244 from jerryshao/SPARK-4381 and squashes the following commits: d2486c7 [jerryshao] Improve the warning log d726e85 [jerryshao] Add local[1] to the filter condition eca428b [jerryshao] Add warning log --- .../scala/org/apache/spark/streaming/StreamingContext.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index ec59221459c88..ecab5510a8e7b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -121,6 +121,11 @@ class StreamingContext private[streaming] ( } } + if (sc.conf.get("spark.master") == "local" || sc.conf.get("spark.master") == "local[1]") { + logWarning("spark.master should be set as local[n], n > 1 in local mode if you have receivers" + + " to get data, otherwise Spark jobs will not get resources to process the received data.") + } + private[streaming] val conf = sc.conf private[streaming] val env = SparkEnv.get From d240760191f692ee7b88dfc82f06a31a340a88a2 Mon Sep 17 00:00:00 2001 From: arahuja Date: Tue, 25 Nov 2014 08:23:41 -0600 Subject: [PATCH 094/109] [SPARK-4344][DOCS] adding documentation on spark.yarn.user.classpath.first The documentation for the two parameters is the same with a pointer from the standalone parameter to the yarn parameter Author: arahuja Closes #3209 from arahuja/yarn-classpath-first-param and squashes the following commits: 51cb9b2 [arahuja] [SPARK-4344][DOCS] adding documentation for YARN on userClassPathFirst --- docs/configuration.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/configuration.md b/docs/configuration.md index 8839162c3a13e..0b77f5ab645c9 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -224,6 +224,7 @@ Apart from these, the following properties are also available, and may be useful (Experimental) Whether to give user-added jars precedence over Spark's own jars when loading classes in Executors. This feature can be used to mitigate conflicts between Spark's dependencies and user dependencies. It is currently an experimental feature. + (Currently, this setting does not work for YARN, see SPARK-2996 for more details). From 69cd53eae205eb10d52eaf38466db58a23b6ae81 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 25 Nov 2014 06:50:36 -0800 Subject: [PATCH 095/109] [SPARK-4601][Streaming] Set correct call site for streaming jobs so that it is displayed correctly on the Spark UI When running the NetworkWordCount, the description of the word count jobs are set as "getCallsite at DStream:xxx" . This should be set to the line number of the streaming application that has the output operation that led to the job being created. This is because the callsite is incorrectly set in the thread launching the jobs. This PR fixes that. Author: Tathagata Das Closes #3455 from tdas/streaming-callsite-fix and squashes the following commits: 69fc26f [Tathagata Das] Set correct call site for streaming jobs so that it is displayed correctly on the Spark UI --- .../org/apache/spark/streaming/dstream/ForEachDStream.scala | 1 + .../org/apache/spark/streaming/StreamingContextSuite.scala | 6 +++++- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala index 905bc723f69a9..1361c30395b57 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala @@ -38,6 +38,7 @@ class ForEachDStream[T: ClassTag] ( parent.getOrCompute(time) match { case Some(rdd) => val jobFunc = () => { + ssc.sparkContext.setCallSite(creationSite) foreachFunc(rdd, time) } Some(new Job(time, jobFunc)) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 4b49c4d251645..9f352bdcb0893 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -336,16 +336,20 @@ package object testPackage extends Assertions { // Verify creation site of generated RDDs var rddGenerated = false - var rddCreationSiteCorrect = true + var rddCreationSiteCorrect = false + var foreachCallSiteCorrect = false inputStream.foreachRDD { rdd => rddCreationSiteCorrect = rdd.creationSite == creationSite + foreachCallSiteCorrect = + rdd.sparkContext.getCallSite().shortForm.contains("StreamingContextSuite") rddGenerated = true } ssc.start() eventually(timeout(10000 millis), interval(10 millis)) { assert(rddGenerated && rddCreationSiteCorrect, "RDD creation site was not correct") + assert(rddGenerated && foreachCallSiteCorrect, "Call site in foreachRDD was not correct") } } finally { ssc.stop() From bf1a6aaac577757a293a573fe8eae9669697310a Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Tue, 25 Nov 2014 11:07:11 -0800 Subject: [PATCH 096/109] [SPARK-4581][MLlib] Refactorize StandardScaler to improve the transformation performance The following optimizations are done to improve the StandardScaler model transformation performance. 1) Covert Breeze dense vector to primitive vector to reduce the overhead. 2) Since mean can be potentially a sparse vector, we explicitly convert it to dense primitive vector. 3) Have a local reference to `shift` and `factor` array so JVM can locate the value with one operation call. 4) In pattern matching part, we use the mllib SparseVector/DenseVector instead of breeze's vector to make the codebase cleaner. Benchmark with mnist8m dataset: Before, DenseVector withMean and withStd: 50.97secs DenseVector withMean and withoutStd: 42.11secs DenseVector withoutMean and withStd: 8.75secs SparseVector withoutMean and withStd: 5.437secs With this PR, DenseVector withMean and withStd: 5.76secs DenseVector withMean and withoutStd: 5.28secs DenseVector withoutMean and withStd: 5.30secs SparseVector withoutMean and withStd: 1.27secs Note that without the local reference copy of `factor` and `shift` arrays, the runtime is almost three time slower. DenseVector withMean and withStd: 18.15secs DenseVector withMean and withoutStd: 18.05secs DenseVector withoutMean and withStd: 18.54secs SparseVector withoutMean and withStd: 2.01secs The following code, ```scala while (i < size) { values(i) = (values(i) - shift(i)) * factor(i) i += 1 } ``` will generate the bytecode ``` L13 LINENUMBER 106 L13 FRAME FULL [org/apache/spark/mllib/feature/StandardScalerModel org/apache/spark/mllib/linalg/Vector org/apache/spark/mllib/linalg/Vector org/apache/spark/mllib/linalg/DenseVector T [D I I] [] ILOAD 7 ILOAD 6 IF_ICMPGE L14 L15 LINENUMBER 107 L15 ALOAD 5 ILOAD 7 ALOAD 5 ILOAD 7 DALOAD ALOAD 0 INVOKESPECIAL org/apache/spark/mllib/feature/StandardScalerModel.shift ()[D ILOAD 7 DALOAD DSUB ALOAD 0 INVOKESPECIAL org/apache/spark/mllib/feature/StandardScalerModel.factor ()[D ILOAD 7 DALOAD DMUL DASTORE L16 LINENUMBER 108 L16 ILOAD 7 ICONST_1 IADD ISTORE 7 GOTO L13 ``` , while with the local reference of the `shift` and `factor` arrays, the bytecode will be ``` L14 LINENUMBER 107 L14 ALOAD 0 INVOKESPECIAL org/apache/spark/mllib/feature/StandardScalerModel.factor ()[D ASTORE 9 L15 LINENUMBER 108 L15 FRAME FULL [org/apache/spark/mllib/feature/StandardScalerModel org/apache/spark/mllib/linalg/Vector [D org/apache/spark/mllib/linalg/Vector org/apache/spark/mllib/linalg/DenseVector T [D I I [D] [] ILOAD 8 ILOAD 7 IF_ICMPGE L16 L17 LINENUMBER 109 L17 ALOAD 6 ILOAD 8 ALOAD 6 ILOAD 8 DALOAD ALOAD 2 ILOAD 8 DALOAD DSUB ALOAD 9 ILOAD 8 DALOAD DMUL DASTORE L18 LINENUMBER 110 L18 ILOAD 8 ICONST_1 IADD ISTORE 8 GOTO L15 ``` You can see that with local reference, the both of the arrays will be in the stack, so JVM can access the value without calling `INVOKESPECIAL`. Author: DB Tsai Closes #3435 from dbtsai/standardscaler and squashes the following commits: 85885a9 [DB Tsai] revert to have lazy in shift array. daf2b06 [DB Tsai] Address the feedback cdb5cef [DB Tsai] small change 9c51eef [DB Tsai] style fc795e4 [DB Tsai] update 5bffd3d [DB Tsai] first commit --- .../spark/mllib/feature/StandardScaler.scala | 70 +++++++++++++------ 1 file changed, 50 insertions(+), 20 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala index 4dfd1f0ab8134..8c4c5db5258d5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala @@ -17,11 +17,9 @@ package org.apache.spark.mllib.feature -import breeze.linalg.{DenseVector => BDV, SparseVector => BSV} - import org.apache.spark.Logging import org.apache.spark.annotation.Experimental -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.mllib.rdd.RDDFunctions._ import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer import org.apache.spark.rdd.RDD @@ -77,8 +75,8 @@ class StandardScalerModel private[mllib] ( require(mean.size == variance.size) - private lazy val factor: BDV[Double] = { - val f = BDV.zeros[Double](variance.size) + private lazy val factor: Array[Double] = { + val f = Array.ofDim[Double](variance.size) var i = 0 while (i < f.size) { f(i) = if (variance(i) != 0.0) 1.0 / math.sqrt(variance(i)) else 0.0 @@ -87,6 +85,11 @@ class StandardScalerModel private[mllib] ( f } + // Since `shift` will be only used in `withMean` branch, we have it as + // `lazy val` so it will be evaluated in that branch. Note that we don't + // want to create this array multiple times in `transform` function. + private lazy val shift: Array[Double] = mean.toArray + /** * Applies standardization transformation on a vector. * @@ -97,30 +100,57 @@ class StandardScalerModel private[mllib] ( override def transform(vector: Vector): Vector = { require(mean.size == vector.size) if (withMean) { - vector.toBreeze match { - case dv: BDV[Double] => - val output = vector.toBreeze.copy - var i = 0 - while (i < output.length) { - output(i) = (output(i) - mean(i)) * (if (withStd) factor(i) else 1.0) - i += 1 + // By default, Scala generates Java methods for member variables. So every time when + // the member variables are accessed, `invokespecial` will be called which is expensive. + // This can be avoid by having a local reference of `shift`. + val localShift = shift + vector match { + case dv: DenseVector => + val values = dv.values.clone() + val size = values.size + if (withStd) { + // Having a local reference of `factor` to avoid overhead as the comment before. + val localFactor = factor + var i = 0 + while (i < size) { + values(i) = (values(i) - localShift(i)) * localFactor(i) + i += 1 + } + } else { + var i = 0 + while (i < size) { + values(i) -= localShift(i) + i += 1 + } } - Vectors.fromBreeze(output) + Vectors.dense(values) case v => throw new IllegalArgumentException("Do not support vector type " + v.getClass) } } else if (withStd) { - vector.toBreeze match { - case dv: BDV[Double] => Vectors.fromBreeze(dv :* factor) - case sv: BSV[Double] => + // Having a local reference of `factor` to avoid overhead as the comment before. + val localFactor = factor + vector match { + case dv: DenseVector => + val values = dv.values.clone() + val size = values.size + var i = 0 + while(i < size) { + values(i) *= localFactor(i) + i += 1 + } + Vectors.dense(values) + case sv: SparseVector => // For sparse vector, the `index` array inside sparse vector object will not be changed, // so we can re-use it to save memory. - val output = new BSV[Double](sv.index, sv.data.clone(), sv.length) + val indices = sv.indices + val values = sv.values.clone() + val nnz = values.size var i = 0 - while (i < output.data.length) { - output.data(i) *= factor(output.index(i)) + while (i < nnz) { + values(i) *= localFactor(indices(i)) i += 1 } - Vectors.fromBreeze(output) + Vectors.sparse(sv.size, indices, values) case v => throw new IllegalArgumentException("Do not support vector type " + v.getClass) } } else { From 8838ad7c135a585cde015dc38b5cb23314502dd9 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 25 Nov 2014 14:16:27 -0800 Subject: [PATCH 097/109] [SPARK-4196][SPARK-4602][Streaming] Fix serialization issue in PairDStreamFunctions.saveAsNewAPIHadoopFiles Solves two JIRAs in one shot - Makes the ForechDStream created by saveAsNewAPIHadoopFiles serializable for checkpoints - Makes the default configuration object used saveAsNewAPIHadoopFiles be the Spark's hadoop configuration Author: Tathagata Das Closes #3457 from tdas/savefiles-fix and squashes the following commits: bb4729a [Tathagata Das] Same treatment for saveAsHadoopFiles b382ea9 [Tathagata Das] Fix serialization issue in PairDStreamFunctions.saveAsNewAPIHadoopFiles. --- .../dstream/PairDStreamFunctions.scala | 30 +++++----- .../spark/streaming/CheckpointSuite.scala | 56 ++++++++++++++++++- 2 files changed, 70 insertions(+), 16 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala index 3f03f42270252..98539e06b4e29 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala @@ -17,20 +17,17 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.streaming.StreamingContext._ - -import org.apache.spark.{Partitioner, HashPartitioner} -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.RDD - import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag -import org.apache.hadoop.mapred.JobConf -import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} -import org.apache.hadoop.mapred.OutputFormat import org.apache.hadoop.conf.Configuration -import org.apache.spark.streaming.{Time, Duration} +import org.apache.hadoop.mapred.{JobConf, OutputFormat} +import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} + +import org.apache.spark.{HashPartitioner, Partitioner, SerializableWritable} +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.{Duration, Time} +import org.apache.spark.streaming.StreamingContext._ /** * Extra functions available on DStream of (key, value) pairs through an implicit conversion. @@ -671,11 +668,13 @@ class PairDStreamFunctions[K, V](self: DStream[(K,V)]) keyClass: Class[_], valueClass: Class[_], outputFormatClass: Class[_ <: OutputFormat[_, _]], - conf: JobConf = new JobConf + conf: JobConf = new JobConf(ssc.sparkContext.hadoopConfiguration) ) { + // Wrap conf in SerializableWritable so that ForeachDStream can be serialized for checkpoints + val serializableConf = new SerializableWritable(conf) val saveFunc = (rdd: RDD[(K, V)], time: Time) => { val file = rddToFileName(prefix, suffix, time) - rdd.saveAsHadoopFile(file, keyClass, valueClass, outputFormatClass, conf) + rdd.saveAsHadoopFile(file, keyClass, valueClass, outputFormatClass, serializableConf.value) } self.foreachRDD(saveFunc) } @@ -702,11 +701,14 @@ class PairDStreamFunctions[K, V](self: DStream[(K,V)]) keyClass: Class[_], valueClass: Class[_], outputFormatClass: Class[_ <: NewOutputFormat[_, _]], - conf: Configuration = new Configuration + conf: Configuration = ssc.sparkContext.hadoopConfiguration ) { + // Wrap conf in SerializableWritable so that ForeachDStream can be serialized for checkpoints + val serializableConf = new SerializableWritable(conf) val saveFunc = (rdd: RDD[(K, V)], time: Time) => { val file = rddToFileName(prefix, suffix, time) - rdd.saveAsNewAPIHadoopFile(file, keyClass, valueClass, outputFormatClass, conf) + rdd.saveAsNewAPIHadoopFile( + file, keyClass, valueClass, outputFormatClass, serializableConf.value) } self.foreachRDD(saveFunc) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index 77ff1ca780a58..c97998add8ffa 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -22,9 +22,14 @@ import java.nio.charset.Charset import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag + import com.google.common.io.Files -import org.apache.hadoop.fs.{Path, FileSystem} import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.io.{IntWritable, Text} +import org.apache.hadoop.mapred.TextOutputFormat +import org.apache.hadoop.mapreduce.lib.output.{TextOutputFormat => NewTextOutputFormat} + import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.dstream.{DStream, FileInputDStream} import org.apache.spark.streaming.util.ManualClock @@ -205,6 +210,51 @@ class CheckpointSuite extends TestSuiteBase { testCheckpointedOperation(input, operation, output, 7) } + test("recovery with saveAsHadoopFiles operation") { + val tempDir = Files.createTempDir() + try { + testCheckpointedOperation( + Seq(Seq("a", "a", "b"), Seq("", ""), Seq(), Seq("a", "a", "b"), Seq("", ""), Seq()), + (s: DStream[String]) => { + val output = s.map(x => (x, 1)).reduceByKey(_ + _) + output.saveAsHadoopFiles( + tempDir.toURI.toString, + "result", + classOf[Text], + classOf[IntWritable], + classOf[TextOutputFormat[Text, IntWritable]]) + output + }, + Seq(Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq(), Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq()), + 3 + ) + } finally { + Utils.deleteRecursively(tempDir) + } + } + + test("recovery with saveAsNewAPIHadoopFiles operation") { + val tempDir = Files.createTempDir() + try { + testCheckpointedOperation( + Seq(Seq("a", "a", "b"), Seq("", ""), Seq(), Seq("a", "a", "b"), Seq("", ""), Seq()), + (s: DStream[String]) => { + val output = s.map(x => (x, 1)).reduceByKey(_ + _) + output.saveAsNewAPIHadoopFiles( + tempDir.toURI.toString, + "result", + classOf[Text], + classOf[IntWritable], + classOf[NewTextOutputFormat[Text, IntWritable]]) + output + }, + Seq(Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq(), Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq()), + 3 + ) + } finally { + Utils.deleteRecursively(tempDir) + } + } // This tests whether the StateDStream's RDD checkpoints works correctly such // that the system can recover from a master failure. This assumes as reliable, @@ -391,7 +441,9 @@ class CheckpointSuite extends TestSuiteBase { logInfo("Manual clock after advancing = " + clock.time) Thread.sleep(batchDuration.milliseconds) - val outputStream = ssc.graph.getOutputStreams.head.asInstanceOf[TestOutputStreamWithPartitions[V]] + val outputStream = ssc.graph.getOutputStreams.filter { dstream => + dstream.isInstanceOf[TestOutputStreamWithPartitions[V]] + }.head.asInstanceOf[TestOutputStreamWithPartitions[V]] outputStream.output.map(_.flatten) } } From 1b2ab1cd1b7cab9076f3c511188a610eda935701 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 25 Nov 2014 15:46:26 -0800 Subject: [PATCH 098/109] [SPARK-4592] Avoid duplicate worker registrations in standalone mode **Summary.** On failover, the Master may receive duplicate registrations from the same worker, causing the worker to exit. This is caused by this commit https://github.com/apache/spark/commit/4afe9a4852ebeb4cc77322a14225cd3dec165f3f, which adds logic for the worker to re-register with the master in case of failures. However, the following race condition may occur: (1) Master A fails and Worker attempts to reconnect to all masters (2) Master B takes over and notifies Worker (3) Worker responds by registering with Master B (4) Meanwhile, Worker's previous reconnection attempt reaches Master B, causing the same Worker to register with Master B twice **Fix.** Instead of attempting to register with all known masters, the worker should re-register with only the one that it has been communicating with. This is safe because the fact that a failover has occurred means the old master must have died. Then, when the worker is finally notified of a new master, it gives up on the old one in favor of the new one. **Caveat.** Even this fix is subject to more obscure race conditions. For instance, if Master B fails and Master A recovers immediately, then Master A may still observe duplicate worker registrations. However, this and other potential race conditions summarized in [SPARK-4592](https://issues.apache.org/jira/browse/SPARK-4592), are much, much less likely than the one described above, which is deterministically reproducible. Author: Andrew Or Closes #3447 from andrewor14/standalone-failover and squashes the following commits: 0d9716c [Andrew Or] Move re-registration logic to actor for thread-safety 79286dc [Andrew Or] Preserve old behavior for initial retries 83b321c [Andrew Or] Tweak wording 1fce6a9 [Andrew Or] Active master actor could be null in the beginning b6f269e [Andrew Or] Avoid duplicate worker registrations --- .../apache/spark/deploy/DeployMessage.scala | 2 + .../apache/spark/deploy/worker/Worker.scala | 52 ++++++++++++++++--- 2 files changed, 47 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index b9dd8557ee904..c46f84de8444a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -92,6 +92,8 @@ private[deploy] object DeployMessages { case object WorkDirCleanup // Sent to Worker actor periodically for cleaning up app folders + case object ReregisterWithMaster // used when a worker attempts to reconnect to a master + // AppClient to Master case class RegisterApplication(appDescription: ApplicationDescription) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index ca262de832e25..eb11163538b20 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -21,7 +21,6 @@ import java.io.File import java.io.IOException import java.text.SimpleDateFormat import java.util.{UUID, Date} -import java.util.concurrent.TimeUnit import scala.collection.JavaConversions._ import scala.collection.mutable.HashMap @@ -177,6 +176,9 @@ private[spark] class Worker( throw new SparkException("Invalid spark URL: " + x) } connected = true + // Cancel any outstanding re-registration attempts because we found a new master + registrationRetryTimer.foreach(_.cancel()) + registrationRetryTimer = None } private def tryRegisterAllMasters() { @@ -187,7 +189,12 @@ private[spark] class Worker( } } - private def retryConnectToMaster() { + /** + * Re-register with the master because a network failure or a master failure has occurred. + * If the re-registration attempt threshold is exceeded, the worker exits with error. + * Note that for thread-safety this should only be called from the actor. + */ + private def reregisterWithMaster(): Unit = { Utils.tryOrExit { connectionAttemptCount += 1 if (registered) { @@ -195,12 +202,40 @@ private[spark] class Worker( registrationRetryTimer = None } else if (connectionAttemptCount <= TOTAL_REGISTRATION_RETRIES) { logInfo(s"Retrying connection to master (attempt # $connectionAttemptCount)") - tryRegisterAllMasters() + /** + * Re-register with the active master this worker has been communicating with. If there + * is none, then it means this worker is still bootstrapping and hasn't established a + * connection with a master yet, in which case we should re-register with all masters. + * + * It is important to re-register only with the active master during failures. Otherwise, + * if the worker unconditionally attempts to re-register with all masters, the following + * race condition may arise and cause a "duplicate worker" error detailed in SPARK-4592: + * + * (1) Master A fails and Worker attempts to reconnect to all masters + * (2) Master B takes over and notifies Worker + * (3) Worker responds by registering with Master B + * (4) Meanwhile, Worker's previous reconnection attempt reaches Master B, + * causing the same Worker to register with Master B twice + * + * Instead, if we only register with the known active master, we can assume that the + * old master must have died because another master has taken over. Note that this is + * still not safe if the old master recovers within this interval, but this is a much + * less likely scenario. + */ + if (master != null) { + master ! RegisterWorker( + workerId, host, port, cores, memory, webUi.boundPort, publicAddress) + } else { + // We are retrying the initial registration + tryRegisterAllMasters() + } + // We have exceeded the initial registration retry threshold + // All retries from now on should use a higher interval if (connectionAttemptCount == INITIAL_REGISTRATION_RETRIES) { registrationRetryTimer.foreach(_.cancel()) registrationRetryTimer = Some { context.system.scheduler.schedule(PROLONGED_REGISTRATION_RETRY_INTERVAL, - PROLONGED_REGISTRATION_RETRY_INTERVAL)(retryConnectToMaster) + PROLONGED_REGISTRATION_RETRY_INTERVAL, self, ReregisterWithMaster) } } } else { @@ -220,7 +255,7 @@ private[spark] class Worker( connectionAttemptCount = 0 registrationRetryTimer = Some { context.system.scheduler.schedule(INITIAL_REGISTRATION_RETRY_INTERVAL, - INITIAL_REGISTRATION_RETRY_INTERVAL)(retryConnectToMaster) + INITIAL_REGISTRATION_RETRY_INTERVAL, self, ReregisterWithMaster) } case Some(_) => logInfo("Not spawning another attempt to register with the master, since there is an" + @@ -400,12 +435,15 @@ private[spark] class Worker( logInfo(s"$x Disassociated !") masterDisconnected() - case RequestWorkerState => { + case RequestWorkerState => sender ! WorkerStateResponse(host, port, workerId, executors.values.toList, finishedExecutors.values.toList, drivers.values.toList, finishedDrivers.values.toList, activeMasterUrl, cores, memory, coresUsed, memoryUsed, activeMasterWebUiUrl) - } + + case ReregisterWithMaster => + reregisterWithMaster() + } private def masterDisconnected() { From 9afcbe494a3535a9bf7958429b72e989972f82d9 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 25 Nov 2014 15:48:02 -0800 Subject: [PATCH 099/109] [SPARK-4546] Improve HistoryServer first time user experience The documentation points the user to run the following ``` sbin/start-history-server.sh ``` The first thing this does is throw an exception that complains a log directory is not specified. The exception message itself does not say anything about what to set. Instead we should have a default and a landing page with a better message. The new default log directory is `file:/tmp/spark-events`. This is what it looks like as of this PR: ![after](https://issues.apache.org/jira/secure/attachment/12682985/after.png) Author: Andrew Or Closes #3411 from andrewor14/minor-history-improvements and squashes the following commits: f33d6b3 [Andrew Or] Point user to set config if default log dir does not exist fc4c17a [Andrew Or] Improve HistoryServer UX --- .../deploy/history/FsHistoryProvider.scala | 37 ++++++++++++------- .../spark/deploy/history/HistoryPage.scala | 8 +++- .../history/HistoryServerArguments.scala | 15 ++++---- docs/monitoring.md | 2 +- 4 files changed, 40 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 2d1609b973607..82a54dbfb5330 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -29,22 +29,27 @@ import org.apache.spark.scheduler._ import org.apache.spark.ui.SparkUI import org.apache.spark.util.Utils +/** + * A class that provides application history from event logs stored in the file system. + * This provider checks for new finished applications in the background periodically and + * renders the history application UI by parsing the associated event logs. + */ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider with Logging { + import FsHistoryProvider._ + private val NOT_STARTED = "" // Interval between each check for event log updates private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", conf.getInt("spark.history.updateInterval", 10)) * 1000 - private val logDir = conf.get("spark.history.fs.logDirectory", null) - private val resolvedLogDir = Option(logDir) - .map { d => Utils.resolveURI(d) } - .getOrElse { throw new IllegalArgumentException("Logging directory must be specified.") } + private val logDir = conf.getOption("spark.history.fs.logDirectory") + .map { d => Utils.resolveURI(d).toString } + .getOrElse(DEFAULT_LOG_DIR) - private val fs = Utils.getHadoopFileSystem(resolvedLogDir, - SparkHadoopUtil.get.newConfiguration(conf)) + private val fs = Utils.getHadoopFileSystem(logDir, SparkHadoopUtil.get.newConfiguration(conf)) // A timestamp of when the disk was last accessed to check for log updates private var lastLogCheckTimeMs = -1L @@ -87,14 +92,17 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis private def initialize() { // Validate the log directory. - val path = new Path(resolvedLogDir) + val path = new Path(logDir) if (!fs.exists(path)) { - throw new IllegalArgumentException( - "Logging directory specified does not exist: %s".format(resolvedLogDir)) + var msg = s"Log directory specified does not exist: $logDir." + if (logDir == DEFAULT_LOG_DIR) { + msg += " Did you configure the correct one through spark.fs.history.logDirectory?" + } + throw new IllegalArgumentException(msg) } if (!fs.getFileStatus(path).isDir) { throw new IllegalArgumentException( - "Logging directory specified is not a directory: %s".format(resolvedLogDir)) + "Logging directory specified is not a directory: %s".format(logDir)) } checkForLogs() @@ -134,8 +142,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis } } - override def getConfig(): Map[String, String] = - Map("Event Log Location" -> resolvedLogDir.toString) + override def getConfig(): Map[String, String] = Map("Event log directory" -> logDir.toString) /** * Builds the application list based on the current contents of the log directory. @@ -146,7 +153,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis lastLogCheckTimeMs = getMonotonicTimeMs() logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs)) try { - val logStatus = fs.listStatus(new Path(resolvedLogDir)) + val logStatus = fs.listStatus(new Path(logDir)) val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]() // Load all new logs from the log directory. Only directories that have a modification time @@ -244,6 +251,10 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis } +private object FsHistoryProvider { + val DEFAULT_LOG_DIR = "file:/tmp/spark-events" +} + private class FsApplicationHistoryInfo( val logDir: String, id: String, 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 0e249e51a77d8..5fdc350cd8512 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 @@ -58,7 +58,13 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { ++ appTable } else { -

No Completed Applications Found

+

No completed applications found!

++ +

Did you specify the correct logging directory? + Please verify your setting of + spark.history.fs.logDirectory and whether you have the permissions to + access it.
It is also possible that your application did not run to + completion or did not stop the SparkContext. +

} } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala index 5bce32a04d16d..b1270ade9f750 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala @@ -17,14 +17,13 @@ package org.apache.spark.deploy.history -import org.apache.spark.SparkConf +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.util.Utils /** * Command-line parser for the master. */ -private[spark] class HistoryServerArguments(conf: SparkConf, args: Array[String]) { - private var logDir: String = null +private[spark] class HistoryServerArguments(conf: SparkConf, args: Array[String]) extends Logging { private var propertiesFile: String = null parse(args.toList) @@ -32,7 +31,8 @@ private[spark] class HistoryServerArguments(conf: SparkConf, args: Array[String] private def parse(args: List[String]): Unit = { args match { case ("--dir" | "-d") :: value :: tail => - logDir = value + logWarning("Setting log directory through the command line is deprecated as of " + + "Spark 1.1.0. Please set this through spark.history.fs.logDirectory instead.") conf.set("spark.history.fs.logDirectory", value) System.setProperty("spark.history.fs.logDirectory", value) parse(tail) @@ -78,9 +78,10 @@ private[spark] class HistoryServerArguments(conf: SparkConf, args: Array[String] | (default 50) |FsHistoryProvider options: | - | spark.history.fs.logDirectory Directory where app logs are stored (required) - | spark.history.fs.updateInterval How often to reload log data from storage (in seconds, - | default 10) + | spark.history.fs.logDirectory Directory where app logs are stored + | (default: file:/tmp/spark-events) + | spark.history.fs.updateInterval How often to reload log data from storage + | (in seconds, default: 10) |""".stripMargin) System.exit(exitCode) } diff --git a/docs/monitoring.md b/docs/monitoring.md index e3f81a76acdbb..f32cdef240d31 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -79,7 +79,7 @@ follows:
- + From 9bdf5da59036c0b052df756fc4a28d64677072e7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?hushan=5B=E8=83=A1=E7=8F=8A=5D?= Date: Tue, 25 Nov 2014 15:51:08 -0800 Subject: [PATCH 100/109] Fix SPARK-4471: blockManagerIdFromJson function throws exception while B... MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Fix [SPARK-4471](https://issues.apache.org/jira/browse/SPARK-4471): blockManagerIdFromJson function throws exception while BlockManagerId be null in MetadataFetchFailedException Author: hushan[胡珊] Closes #3340 from suyanNone/fix-blockmanagerId-jnothing-2 and squashes the following commits: 159f9a3 [hushan[胡珊]] Refine test code for blockmanager is null 4380d73 [hushan[胡珊]] remove useless blank line 3ccf651 [hushan[胡珊]] Fix SPARK-4471: blockManagerIdFromJson function throws exception while metadata fetch failed --- .../org/apache/spark/util/JsonProtocol.scala | 4 ++++ .../org/apache/spark/util/JsonProtocolSuite.scala | 15 ++++++++++++--- 2 files changed, 16 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 7b5db1ed76265..e7b80e8774b9c 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -688,6 +688,10 @@ private[spark] object JsonProtocol { } def blockManagerIdFromJson(json: JValue): BlockManagerId = { + // On metadata fetch fail, block manager ID can be null (SPARK-4471) + if (json == JNothing) { + return null + } val executorId = (json \ "Executor ID").extract[String] val host = (json \ "Host").extract[String] val port = (json \ "Port").extract[Int] diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 0bc9492675863..593d6dd8c3794 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.util import java.util.Properties +import org.apache.spark.shuffle.MetadataFetchFailedException + import scala.collection.Map import org.json4s.jackson.JsonMethods._ @@ -116,10 +118,13 @@ class JsonProtocolSuite extends FunSuite { // TaskEndReason val fetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 18, 19, "Some exception") + val fetchMetadataFailed = new MetadataFetchFailedException(17, + 19, "metadata Fetch failed exception").toTaskEndReason val exceptionFailure = new ExceptionFailure(exception, None) testTaskEndReason(Success) testTaskEndReason(Resubmitted) testTaskEndReason(fetchFailed) + testTaskEndReason(fetchMetadataFailed) testTaskEndReason(exceptionFailure) testTaskEndReason(TaskResultLost) testTaskEndReason(TaskKilled) @@ -431,9 +436,13 @@ class JsonProtocolSuite extends FunSuite { } private def assertEquals(bm1: BlockManagerId, bm2: BlockManagerId) { - assert(bm1.executorId === bm2.executorId) - assert(bm1.host === bm2.host) - assert(bm1.port === bm2.port) + if (bm1 == null || bm2 == null) { + assert(bm1 === bm2) + } else { + assert(bm1.executorId === bm2.executorId) + assert(bm1.host === bm2.host) + assert(bm1.port === bm2.port) + } } private def assertEquals(result1: JobResult, result2: JobResult) { From 7eba0fbe456c451122d7a2353ff0beca00f15223 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 25 Nov 2014 16:07:09 -0800 Subject: [PATCH 101/109] [Spark-4509] Revert EC2 tag-based cluster membership patch This PR reverts changes related to tag-based cluster membership. As discussed in SPARK-3332, we didn't figure out a safe strategy to use tags to determine cluster membership, because tagging is not atomic. The following changes are reverted: SPARK-2333: 94053a7b766788bb62e2dbbf352ccbcc75f71fc0 SPARK-3213: 7faf755ae4f0cf510048e432340260a6e609066d SPARK-3608: 78d4220fa0bf2f9ee663e34bbf3544a5313b02f0. I tested launch, login, and destroy. It is easy to check the diff by comparing it to Josh's patch for branch-1.1: https://github.com/apache/spark/pull/2225/files JoshRosen I sent the PR to master. It might be easier for us to keep master and branch-1.2 the same at this time. We can always re-apply the patch once we figure out a stable solution. Author: Xiangrui Meng Closes #3453 from mengxr/SPARK-4509 and squashes the following commits: f0b708b [Xiangrui Meng] revert 94053a7b766788bb62e2dbbf352ccbcc75f71fc0 4298ea5 [Xiangrui Meng] revert 7faf755ae4f0cf510048e432340260a6e609066d 35963a1 [Xiangrui Meng] Revert "SPARK-3608 Break if the instance tag naming succeeds" --- docs/ec2-scripts.md | 14 ++++---- ec2/spark_ec2.py | 83 ++++++++++++--------------------------------- 2 files changed, 28 insertions(+), 69 deletions(-) diff --git a/docs/ec2-scripts.md b/docs/ec2-scripts.md index 530798f2b8022..66bf5f1a855ed 100644 --- a/docs/ec2-scripts.md +++ b/docs/ec2-scripts.md @@ -12,16 +12,14 @@ on the [Amazon Web Services site](http://aws.amazon.com/). `spark-ec2` is designed to manage multiple named clusters. You can launch a new cluster (telling the script its size and giving it a name), -shutdown an existing cluster, or log into a cluster. Each cluster -launches a set of instances, which are tagged with the cluster name, -and placed into EC2 security groups. If you don't specify a security -group, the `spark-ec2` script will create security groups based on the -cluster name you request. For example, a cluster named +shutdown an existing cluster, or log into a cluster. Each cluster is +identified by placing its machines into EC2 security groups whose names +are derived from the name of the cluster. For example, a cluster named `test` will contain a master node in a security group called `test-master`, and a number of slave nodes in a security group called -`test-slaves`. You can also specify a security group prefix to be used -in place of the cluster name. Machines in a cluster can be identified -by looking for the "Name" tag of the instance in the Amazon EC2 Console. +`test-slaves`. The `spark-ec2` script will create these security groups +for you based on the cluster name you request. You can also use them to +identify machines belonging to each cluster in the Amazon EC2 Console. # Before You Start diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index a5396c2375915..a4ab844a56adf 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -138,7 +138,7 @@ def parse_args(): help="The SSH user you want to connect as (default: %default)") parser.add_option( "--delete-groups", action="store_true", default=False, - help="When destroying a cluster, delete the security groups that were created.") + help="When destroying a cluster, delete the security groups that were created") parser.add_option( "--use-existing-master", action="store_true", default=False, help="Launch fresh slaves, but use an existing stopped master if possible") @@ -152,9 +152,6 @@ def parse_args(): parser.add_option( "--user-data", type="string", default="", help="Path to a user-data file (most AMI's interpret this as an initialization script)") - parser.add_option( - "--security-group-prefix", type="string", default=None, - help="Use this prefix for the security group rather than the cluster name.") parser.add_option( "--authorized-address", type="string", default="0.0.0.0/0", help="Address to authorize on created security groups (default: %default)") @@ -305,12 +302,8 @@ def launch_cluster(conn, opts, cluster_name): user_data_content = user_data_file.read() print "Setting up security groups..." - if opts.security_group_prefix is None: - master_group = get_or_make_group(conn, cluster_name + "-master") - slave_group = get_or_make_group(conn, cluster_name + "-slaves") - else: - master_group = get_or_make_group(conn, opts.security_group_prefix + "-master") - slave_group = get_or_make_group(conn, opts.security_group_prefix + "-slaves") + master_group = get_or_make_group(conn, cluster_name + "-master") + slave_group = get_or_make_group(conn, cluster_name + "-slaves") authorized_address = opts.authorized_address if master_group.rules == []: # Group was just now created master_group.authorize(src_group=master_group) @@ -335,11 +328,12 @@ def launch_cluster(conn, opts, cluster_name): slave_group.authorize('tcp', 60060, 60060, authorized_address) slave_group.authorize('tcp', 60075, 60075, authorized_address) - # Check if instances are already running with the cluster name + # Check if instances are already running in our groups existing_masters, existing_slaves = get_existing_cluster(conn, opts, cluster_name, die_on_error=False) if existing_slaves or (existing_masters and not opts.use_existing_master): - print >> stderr, ("ERROR: There are already instances for name: %s " % cluster_name) + print >> stderr, ("ERROR: There are already instances running in " + + "group %s or %s" % (master_group.name, slave_group.name)) sys.exit(1) # Figure out Spark AMI @@ -413,13 +407,9 @@ def launch_cluster(conn, opts, cluster_name): for r in reqs: id_to_req[r.id] = r active_instance_ids = [] - outstanding_request_ids = [] for i in my_req_ids: - if i in id_to_req: - if id_to_req[i].state == "active": - active_instance_ids.append(id_to_req[i].instance_id) - else: - outstanding_request_ids.append(i) + if i in id_to_req and id_to_req[i].state == "active": + active_instance_ids.append(id_to_req[i].instance_id) if len(active_instance_ids) == opts.slaves: print "All %d slaves granted" % opts.slaves reservations = conn.get_all_instances(active_instance_ids) @@ -428,8 +418,8 @@ def launch_cluster(conn, opts, cluster_name): slave_nodes += r.instances break else: - print "%d of %d slaves granted, waiting longer for request ids including %s" % ( - len(active_instance_ids), opts.slaves, outstanding_request_ids[0:10]) + print "%d of %d slaves granted, waiting longer" % ( + len(active_instance_ids), opts.slaves) except: print "Canceling spot instance requests" conn.cancel_spot_instance_requests(my_req_ids) @@ -488,59 +478,34 @@ def launch_cluster(conn, opts, cluster_name): # Give the instances descriptive names for master in master_nodes: - name = '{cn}-master-{iid}'.format(cn=cluster_name, iid=master.id) - tag_instance(master, name) - + master.add_tag( + key='Name', + value='{cn}-master-{iid}'.format(cn=cluster_name, iid=master.id)) for slave in slave_nodes: - name = '{cn}-slave-{iid}'.format(cn=cluster_name, iid=slave.id) - tag_instance(slave, name) + slave.add_tag( + key='Name', + value='{cn}-slave-{iid}'.format(cn=cluster_name, iid=slave.id)) # Return all the instances return (master_nodes, slave_nodes) -def tag_instance(instance, name): - for i in range(0, 5): - try: - instance.add_tag(key='Name', value=name) - break - except: - print "Failed attempt %i of 5 to tag %s" % ((i + 1), name) - if i == 5: - raise "Error - failed max attempts to add name tag" - time.sleep(5) - # Get the EC2 instances in an existing cluster if available. # Returns a tuple of lists of EC2 instance objects for the masters and slaves def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): print "Searching for existing cluster " + cluster_name + "..." - # Search all the spot instance requests, and copy any tags from the spot - # instance request to the cluster. - spot_instance_requests = conn.get_all_spot_instance_requests() - for req in spot_instance_requests: - if req.state != u'active': - continue - name = req.tags.get(u'Name', "") - if name.startswith(cluster_name): - reservations = conn.get_all_instances(instance_ids=[req.instance_id]) - for res in reservations: - active = [i for i in res.instances if is_active(i)] - for instance in active: - if instance.tags.get(u'Name') is None: - tag_instance(instance, name) - # Now proceed to detect master and slaves instances. reservations = conn.get_all_instances() master_nodes = [] slave_nodes = [] for res in reservations: active = [i for i in res.instances if is_active(i)] for inst in active: - name = inst.tags.get(u'Name', "") - if name.startswith(cluster_name + "-master"): + group_names = [g.name for g in inst.groups] + if group_names == [cluster_name + "-master"]: master_nodes.append(inst) - elif name.startswith(cluster_name + "-slave"): + elif group_names == [cluster_name + "-slaves"]: slave_nodes.append(inst) if any((master_nodes, slave_nodes)): print "Found %d master(s), %d slaves" % (len(master_nodes), len(slave_nodes)) @@ -548,12 +513,12 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): return (master_nodes, slave_nodes) else: if master_nodes == [] and slave_nodes != []: - print >> sys.stderr, "ERROR: Could not find master in with name " + \ - cluster_name + "-master" + print >> sys.stderr, "ERROR: Could not find master in group " + cluster_name + "-master" else: print >> sys.stderr, "ERROR: Could not find any existing cluster" sys.exit(1) + # Deploy configuration files and run setup scripts on a newly launched # or started EC2 cluster. @@ -984,11 +949,7 @@ def real_main(): # Delete security groups as well if opts.delete_groups: print "Deleting security groups (this will take some time)..." - if opts.security_group_prefix is None: - group_names = [cluster_name + "-master", cluster_name + "-slaves"] - else: - group_names = [opts.security_group_prefix + "-master", - opts.security_group_prefix + "-slaves"] + group_names = [cluster_name + "-master", cluster_name + "-slaves"] wait_for_cluster_state( cluster_instances=(master_nodes + slave_nodes), cluster_state='terminated', From c251fd7405db57d3ab2686c38712601fd8f13ccd Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Tue, 25 Nov 2014 20:10:15 -0800 Subject: [PATCH 102/109] [SPARK-4583] [mllib] LogLoss for GradientBoostedTrees fix + doc updates Currently, the LogLoss used by GradientBoostedTrees has 2 issues: * the gradient (and therefore loss) does not match that used by Friedman (1999) * the error computation uses 0/1 accuracy, not log loss This PR updates LogLoss. It also adds some doc for boosting and forests. I tested it on sample data and made sure the log loss is monotonically decreasing with each boosting iteration. CC: mengxr manishamde codedeft Author: Joseph K. Bradley Closes #3439 from jkbradley/gbt-loss-fix and squashes the following commits: cfec17e [Joseph K. Bradley] removed forgotten temp comments a27eb6d [Joseph K. Bradley] corrections to last log loss commit ed5da2c [Joseph K. Bradley] updated LogLoss (boosting) for numerical stability 5e52bff [Joseph K. Bradley] * Removed the 1/2 from SquaredError. This also required updating the test suite since it effectively doubles the gradient and loss. * Added doc for developers within RandomForest. * Small cleanup in test suite (generating data only once) e57897a [Joseph K. Bradley] Fixed LogLoss for GradientBoostedTrees, and updated doc for losses, forests, and boosting --- .../mllib/tree/GradientBoostedTrees.scala | 18 +++-- .../spark/mllib/tree/RandomForest.scala | 44 ++++++++++- .../spark/mllib/tree/loss/AbsoluteError.scala | 26 +++---- .../spark/mllib/tree/loss/LogLoss.scala | 34 ++++++--- .../spark/mllib/tree/loss/SquaredError.scala | 22 +++--- .../tree/GradientBoostedTreesSuite.scala | 74 ++++++++++++------- 6 files changed, 146 insertions(+), 72 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala index cb4ddfc814f91..61f6b1313f82e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala @@ -31,18 +31,20 @@ import org.apache.spark.storage.StorageLevel /** * :: Experimental :: - * A class that implements Stochastic Gradient Boosting for regression and binary classification. + * A class that implements + * [[http://en.wikipedia.org/wiki/Gradient_boosting Stochastic Gradient Boosting]] + * for regression and binary classification. * * The implementation is based upon: * J.H. Friedman. "Stochastic Gradient Boosting." 1999. * - * Notes: - * - This currently can be run with several loss functions. However, only SquaredError is - * fully supported. Specifically, the loss function should be used to compute the gradient - * (to re-label training instances on each iteration) and to weight weak hypotheses. - * Currently, gradients are computed correctly for the available loss functions, - * but weak hypothesis weights are not computed correctly for LogLoss or AbsoluteError. - * Running with those losses will likely behave reasonably, but lacks the same guarantees. + * Notes on Gradient Boosting vs. TreeBoost: + * - This implementation is for Stochastic Gradient Boosting, not for TreeBoost. + * - Both algorithms learn tree ensembles by minimizing loss functions. + * - TreeBoost (Friedman, 1999) additionally modifies the outputs at tree leaf nodes + * based on the loss function, whereas the original gradient boosting method does not. + * - When the loss is SquaredError, these methods give the same result, but they could differ + * for other loss functions. * * @param boostingStrategy Parameters for the gradient boosting algorithm. */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala index 3ae6fa2a0ec2f..482d3395516e7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala @@ -37,7 +37,8 @@ import org.apache.spark.util.Utils /** * :: Experimental :: - * A class which implements a random forest learning algorithm for classification and regression. + * A class that implements a [[http://en.wikipedia.org/wiki/Random_forest Random Forest]] + * learning algorithm for classification and regression. * It supports both continuous and categorical features. * * The settings for featureSubsetStrategy are based on the following references: @@ -70,6 +71,47 @@ private class RandomForest ( private val seed: Int) extends Serializable with Logging { + /* + ALGORITHM + This is a sketch of the algorithm to help new developers. + + The algorithm partitions data by instances (rows). + On each iteration, the algorithm splits a set of nodes. In order to choose the best split + for a given node, sufficient statistics are collected from the distributed data. + For each node, the statistics are collected to some worker node, and that worker selects + the best split. + + This setup requires discretization of continuous features. This binning is done in the + findSplitsBins() method during initialization, after which each continuous feature becomes + an ordered discretized feature with at most maxBins possible values. + + The main loop in the algorithm operates on a queue of nodes (nodeQueue). These nodes + lie at the periphery of the tree being trained. If multiple trees are being trained at once, + then this queue contains nodes from all of them. Each iteration works roughly as follows: + On the master node: + - Some number of nodes are pulled off of the queue (based on the amount of memory + required for their sufficient statistics). + - For random forests, if featureSubsetStrategy is not "all," then a subset of candidate + features are chosen for each node. See method selectNodesToSplit(). + On worker nodes, via method findBestSplits(): + - The worker makes one pass over its subset of instances. + - For each (tree, node, feature, split) tuple, the worker collects statistics about + splitting. Note that the set of (tree, node) pairs is limited to the nodes selected + from the queue for this iteration. The set of features considered can also be limited + based on featureSubsetStrategy. + - For each node, the statistics for that node are aggregated to a particular worker + via reduceByKey(). The designated worker chooses the best (feature, split) pair, + or chooses to stop splitting if the stopping criteria are met. + On the master node: + - The master collects all decisions about splitting nodes and updates the model. + - The updated model is passed to the workers on the next iteration. + This process continues until the node queue is empty. + + Most of the methods in this implementation support the statistics aggregation, which is + the heaviest part of the computation. In general, this implementation is bound by either + the cost of statistics computation on workers or by communicating the sufficient statistics. + */ + strategy.assertValid() require(numTrees > 0, s"RandomForest requires numTrees > 0, but was given numTrees = $numTrees.") require(RandomForest.supportedFeatureSubsetStrategies.contains(featureSubsetStrategy), diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala index e8288668094df..d1bde15e6b150 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala @@ -17,7 +17,6 @@ package org.apache.spark.mllib.tree.loss -import org.apache.spark.SparkContext._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.model.TreeEnsembleModel @@ -25,11 +24,11 @@ import org.apache.spark.rdd.RDD /** * :: DeveloperApi :: - * Class for least absolute error loss calculation. - * The features x and the corresponding label y is predicted using the function F. - * For each instance: - * Loss: |y - F| - * Negative gradient: sign(y - F) + * Class for absolute error loss calculation (for regression). + * + * The absolute (L1) error is defined as: + * |y - F(x)| + * where y is the label and F(x) is the model prediction for features x. */ @DeveloperApi object AbsoluteError extends Loss { @@ -37,7 +36,8 @@ object AbsoluteError extends Loss { /** * Method to calculate the gradients for the gradient boosting calculation for least * absolute error calculation. - * @param model Model of the weak learner + * The gradient with respect to F(x) is: sign(F(x) - y) + * @param model Ensemble model * @param point Instance of the training dataset * @return Loss gradient */ @@ -48,19 +48,17 @@ object AbsoluteError extends Loss { } /** - * Method to calculate error of the base learner for the gradient boosting calculation. + * Method to calculate loss of the base learner for the gradient boosting calculation. * Note: This method is not used by the gradient boosting algorithm but is useful for debugging * purposes. - * @param model Model of the weak learner. + * @param model Ensemble model * @param data Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. - * @return + * @return Mean absolute error of model on data */ override def computeError(model: TreeEnsembleModel, data: RDD[LabeledPoint]): Double = { - val sumOfAbsolutes = data.map { y => + data.map { y => val err = model.predict(y.features) - y.label math.abs(err) - }.sum() - sumOfAbsolutes / data.count() + }.mean() } - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala index 8b8adb44aea92..7ce9fa6f86c42 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala @@ -24,12 +24,12 @@ import org.apache.spark.rdd.RDD /** * :: DeveloperApi :: - * Class for least squares error loss calculation. + * Class for log loss calculation (for classification). + * This uses twice the binomial negative log likelihood, called "deviance" in Friedman (1999). * - * The features x and the corresponding label y is predicted using the function F. - * For each instance: - * Loss: log(1 + exp(-2yF)), y in {-1, 1} - * Negative gradient: 2y / ( 1 + exp(2yF)) + * The log loss is defined as: + * 2 log(1 + exp(-2 y F(x))) + * where y is a label in {-1, 1} and F(x) is the model prediction for features x. */ @DeveloperApi object LogLoss extends Loss { @@ -37,7 +37,8 @@ object LogLoss extends Loss { /** * Method to calculate the loss gradients for the gradient boosting calculation for binary * classification - * @param model Model of the weak learner + * The gradient with respect to F(x) is: - 4 y / (1 + exp(2 y F(x))) + * @param model Ensemble model * @param point Instance of the training dataset * @return Loss gradient */ @@ -45,19 +46,28 @@ object LogLoss extends Loss { model: TreeEnsembleModel, point: LabeledPoint): Double = { val prediction = model.predict(point.features) - 1.0 / (1.0 + math.exp(-prediction)) - point.label + - 4.0 * point.label / (1.0 + math.exp(2.0 * point.label * prediction)) } /** - * Method to calculate error of the base learner for the gradient boosting calculation. + * Method to calculate loss of the base learner for the gradient boosting calculation. * Note: This method is not used by the gradient boosting algorithm but is useful for debugging * purposes. - * @param model Model of the weak learner. + * @param model Ensemble model * @param data Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. - * @return + * @return Mean log loss of model on data */ override def computeError(model: TreeEnsembleModel, data: RDD[LabeledPoint]): Double = { - val wrongPredictions = data.filter(lp => model.predict(lp.features) != lp.label).count() - wrongPredictions / data.count + data.map { case point => + val prediction = model.predict(point.features) + val margin = 2.0 * point.label * prediction + // The following are equivalent to 2.0 * log(1 + exp(-margin)) but are more numerically + // stable. + if (margin >= 0) { + 2.0 * math.log1p(math.exp(-margin)) + } else { + 2.0 * (-margin + math.log1p(math.exp(margin))) + } + }.mean() } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala index cfe395b1d0491..50ecaa2f86f35 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala @@ -17,7 +17,6 @@ package org.apache.spark.mllib.tree.loss -import org.apache.spark.SparkContext._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.model.TreeEnsembleModel @@ -25,12 +24,11 @@ import org.apache.spark.rdd.RDD /** * :: DeveloperApi :: - * Class for least squares error loss calculation. + * Class for squared error loss calculation. * - * The features x and the corresponding label y is predicted using the function F. - * For each instance: - * Loss: (y - F)**2/2 - * Negative gradient: y - F + * The squared (L2) error is defined as: + * (y - F(x))**2 + * where y is the label and F(x) is the model prediction for features x. */ @DeveloperApi object SquaredError extends Loss { @@ -38,23 +36,24 @@ object SquaredError extends Loss { /** * Method to calculate the gradients for the gradient boosting calculation for least * squares error calculation. - * @param model Model of the weak learner + * The gradient with respect to F(x) is: - 2 (y - F(x)) + * @param model Ensemble model * @param point Instance of the training dataset * @return Loss gradient */ override def gradient( model: TreeEnsembleModel, point: LabeledPoint): Double = { - model.predict(point.features) - point.label + 2.0 * (model.predict(point.features) - point.label) } /** - * Method to calculate error of the base learner for the gradient boosting calculation. + * Method to calculate loss of the base learner for the gradient boosting calculation. * Note: This method is not used by the gradient boosting algorithm but is useful for debugging * purposes. - * @param model Model of the weak learner. + * @param model Ensemble model * @param data Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. - * @return + * @return Mean squared error of model on data */ override def computeError(model: TreeEnsembleModel, data: RDD[LabeledPoint]): Double = { data.map { y => @@ -62,5 +61,4 @@ object SquaredError extends Loss { err * err }.mean() } - } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala index f3f8eff2db300..d4d54cf4c9e2a 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala @@ -35,32 +35,39 @@ class GradientBoostedTreesSuite extends FunSuite with MLlibTestSparkContext { test("Regression with continuous features: SquaredError") { GradientBoostedTreesSuite.testCombinations.foreach { case (numIterations, learningRate, subsamplingRate) => - val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 10, 100) - val rdd = sc.parallelize(arr, 2) - - val treeStrategy = new Strategy(algo = Regression, impurity = Variance, maxDepth = 2, - categoricalFeaturesInfo = Map.empty, subsamplingRate = subsamplingRate) - val boostingStrategy = - new BoostingStrategy(treeStrategy, SquaredError, numIterations, learningRate) - - val gbt = GradientBoostedTrees.train(rdd, boostingStrategy) - - assert(gbt.trees.size === numIterations) - EnsembleTestHelper.validateRegressor(gbt, arr, 0.03) - - val remappedInput = rdd.map(x => new LabeledPoint((x.label * 2) - 1, x.features)) - val dt = DecisionTree.train(remappedInput, treeStrategy) - - // Make sure trees are the same. - assert(gbt.trees.head.toString == dt.toString) + GradientBoostedTreesSuite.randomSeeds.foreach { randomSeed => + val rdd = sc.parallelize(GradientBoostedTreesSuite.data, 2) + + val treeStrategy = new Strategy(algo = Regression, impurity = Variance, maxDepth = 2, + categoricalFeaturesInfo = Map.empty, subsamplingRate = subsamplingRate) + val boostingStrategy = + new BoostingStrategy(treeStrategy, SquaredError, numIterations, learningRate) + + val gbt = GradientBoostedTrees.train(rdd, boostingStrategy) + + assert(gbt.trees.size === numIterations) + try { + EnsembleTestHelper.validateRegressor(gbt, GradientBoostedTreesSuite.data, 0.06) + } catch { + case e: java.lang.AssertionError => + println(s"FAILED for numIterations=$numIterations, learningRate=$learningRate," + + s" subsamplingRate=$subsamplingRate") + throw e + } + + val remappedInput = rdd.map(x => new LabeledPoint((x.label * 2) - 1, x.features)) + val dt = DecisionTree.train(remappedInput, treeStrategy) + + // Make sure trees are the same. + assert(gbt.trees.head.toString == dt.toString) + } } } test("Regression with continuous features: Absolute Error") { GradientBoostedTreesSuite.testCombinations.foreach { case (numIterations, learningRate, subsamplingRate) => - val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 10, 100) - val rdd = sc.parallelize(arr, 2) + val rdd = sc.parallelize(GradientBoostedTreesSuite.data, 2) val treeStrategy = new Strategy(algo = Regression, impurity = Variance, maxDepth = 2, categoricalFeaturesInfo = Map.empty, subsamplingRate = subsamplingRate) @@ -70,7 +77,14 @@ class GradientBoostedTreesSuite extends FunSuite with MLlibTestSparkContext { val gbt = GradientBoostedTrees.train(rdd, boostingStrategy) assert(gbt.trees.size === numIterations) - EnsembleTestHelper.validateRegressor(gbt, arr, 0.85, "mae") + try { + EnsembleTestHelper.validateRegressor(gbt, GradientBoostedTreesSuite.data, 0.85, "mae") + } catch { + case e: java.lang.AssertionError => + println(s"FAILED for numIterations=$numIterations, learningRate=$learningRate," + + s" subsamplingRate=$subsamplingRate") + throw e + } val remappedInput = rdd.map(x => new LabeledPoint((x.label * 2) - 1, x.features)) val dt = DecisionTree.train(remappedInput, treeStrategy) @@ -83,8 +97,7 @@ class GradientBoostedTreesSuite extends FunSuite with MLlibTestSparkContext { test("Binary classification with continuous features: Log Loss") { GradientBoostedTreesSuite.testCombinations.foreach { case (numIterations, learningRate, subsamplingRate) => - val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 10, 100) - val rdd = sc.parallelize(arr, 2) + val rdd = sc.parallelize(GradientBoostedTreesSuite.data, 2) val treeStrategy = new Strategy(algo = Classification, impurity = Variance, maxDepth = 2, numClassesForClassification = 2, categoricalFeaturesInfo = Map.empty, @@ -95,7 +108,14 @@ class GradientBoostedTreesSuite extends FunSuite with MLlibTestSparkContext { val gbt = GradientBoostedTrees.train(rdd, boostingStrategy) assert(gbt.trees.size === numIterations) - EnsembleTestHelper.validateClassifier(gbt, arr, 0.9) + try { + EnsembleTestHelper.validateClassifier(gbt, GradientBoostedTreesSuite.data, 0.9) + } catch { + case e: java.lang.AssertionError => + println(s"FAILED for numIterations=$numIterations, learningRate=$learningRate," + + s" subsamplingRate=$subsamplingRate") + throw e + } val remappedInput = rdd.map(x => new LabeledPoint((x.label * 2) - 1, x.features)) val ensembleStrategy = treeStrategy.copy @@ -113,5 +133,9 @@ class GradientBoostedTreesSuite extends FunSuite with MLlibTestSparkContext { object GradientBoostedTreesSuite { // Combinations for estimators, learning rates and subsamplingRate - val testCombinations = Array((10, 1.0, 1.0), (10, 0.1, 1.0), (10, 1.0, 0.75), (10, 0.1, 0.75)) + val testCombinations = Array((10, 1.0, 1.0), (10, 0.1, 1.0), (10, 0.5, 0.75), (10, 0.1, 0.75)) + + val randomSeeds = Array(681283, 4398) + + val data = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 10, 100) } From 4d95526a75ad1630554683fe7a7e583da44ba6e4 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 25 Nov 2014 23:10:19 -0500 Subject: [PATCH 103/109] [HOTFIX]: Adding back without-hive dist --- dev/create-release/create-release.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 8a0b0348db8c0..e0aca467ac949 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -197,6 +197,7 @@ make_binary_release "hadoop2.3" "-Phadoop-2.3 -Phive -Phive-thriftserver -Pyarn" make_binary_release "hadoop2.4" "-Phadoop-2.4 -Phive -Phive-thriftserver -Pyarn" & make_binary_release "mapr3" "-Pmapr3 -Phive -Phive-thriftserver" & make_binary_release "mapr4" "-Pmapr4 -Pyarn -Phive -Phive-thriftserver" & +make_binary_release "hadoop2.4-without-hive" "-Phadoop-2.4 -Pyarn" & wait # Copy data From b5fb1410c5eed1156decb4f9fcc22436a658ce4d Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 25 Nov 2014 20:11:40 -0800 Subject: [PATCH 104/109] [SPARK-4604][MLLIB] make MatrixFactorizationModel public User could construct an MF model directly. I added a note about the performance. Author: Xiangrui Meng Closes #3459 from mengxr/SPARK-4604 and squashes the following commits: f64bcd3 [Xiangrui Meng] organize imports ed08214 [Xiangrui Meng] check preconditions and unit tests a624c12 [Xiangrui Meng] make MatrixFactorizationModel public --- .../MatrixFactorizationModel.scala | 28 +++++++++- .../MatrixFactorizationModelSuite.scala | 56 +++++++++++++++++++ 2 files changed, 81 insertions(+), 3 deletions(-) create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModelSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index 969e23be21623..ed2f8b41bcae5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -21,23 +21,45 @@ import java.lang.{Integer => JavaInteger} import org.jblas.DoubleMatrix -import org.apache.spark.SparkContext._ +import org.apache.spark.Logging import org.apache.spark.api.java.{JavaPairRDD, JavaRDD} import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel /** * Model representing the result of matrix factorization. * + * Note: If you create the model directly using constructor, please be aware that fast prediction + * requires cached user/product features and their associated partitioners. + * * @param rank Rank for the features in this model. * @param userFeatures RDD of tuples where each tuple represents the userId and * the features computed for this user. * @param productFeatures RDD of tuples where each tuple represents the productId * and the features computed for this product. */ -class MatrixFactorizationModel private[mllib] ( +class MatrixFactorizationModel( val rank: Int, val userFeatures: RDD[(Int, Array[Double])], - val productFeatures: RDD[(Int, Array[Double])]) extends Serializable { + val productFeatures: RDD[(Int, Array[Double])]) extends Serializable with Logging { + + require(rank > 0) + validateFeatures("User", userFeatures) + validateFeatures("Product", productFeatures) + + /** Validates factors and warns users if there are performance concerns. */ + private def validateFeatures(name: String, features: RDD[(Int, Array[Double])]): Unit = { + require(features.first()._2.size == rank, + s"$name feature dimension does not match the rank $rank.") + if (features.partitioner.isEmpty) { + logWarning(s"$name factor does not have a partitioner. " + + "Prediction on individual records could be slow.") + } + if (features.getStorageLevel == StorageLevel.NONE) { + logWarning(s"$name factor is not cached. Prediction could be slow.") + } + } + /** Predict the rating of one user for one product. */ def predict(user: Int, product: Int): Double = { val userVector = new DoubleMatrix(userFeatures.lookup(user).head) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModelSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModelSuite.scala new file mode 100644 index 0000000000000..b9caecc904a23 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModelSuite.scala @@ -0,0 +1,56 @@ +/* + * 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.mllib.recommendation + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.rdd.RDD + +class MatrixFactorizationModelSuite extends FunSuite with MLlibTestSparkContext { + + val rank = 2 + var userFeatures: RDD[(Int, Array[Double])] = _ + var prodFeatures: RDD[(Int, Array[Double])] = _ + + override def beforeAll(): Unit = { + super.beforeAll() + userFeatures = sc.parallelize(Seq((0, Array(1.0, 2.0)), (1, Array(3.0, 4.0)))) + prodFeatures = sc.parallelize(Seq((2, Array(5.0, 6.0)))) + } + + test("constructor") { + val model = new MatrixFactorizationModel(rank, userFeatures, prodFeatures) + assert(model.predict(0, 2) ~== 17.0 relTol 1e-14) + + intercept[IllegalArgumentException] { + new MatrixFactorizationModel(1, userFeatures, prodFeatures) + } + + val userFeatures1 = sc.parallelize(Seq((0, Array(1.0)), (1, Array(3.0)))) + intercept[IllegalArgumentException] { + new MatrixFactorizationModel(rank, userFeatures1, prodFeatures) + } + + val prodFeatures1 = sc.parallelize(Seq((2, Array(5.0)))) + intercept[IllegalArgumentException] { + new MatrixFactorizationModel(rank, userFeatures, prodFeatures1) + } + } +} From f5f2d27385c243959f03a9d78a149d5f405b2f50 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Tue, 25 Nov 2014 23:57:04 -0500 Subject: [PATCH 105/109] [SPARK-4516] Cap default number of Netty threads at 8 In practice, only 2-4 cores should be required to transfer roughly 10 Gb/s, and each core that we use will have an initial overhead of roughly 32 MB of off-heap memory, which comes at a premium. Thus, this value should still retain maximum throughput and reduce wasted off-heap memory allocation. It can be overridden by setting the number of serverThreads and clientThreads manually in Spark's configuration. Author: Aaron Davidson Closes #3469 from aarondav/fewer-pools2 and squashes the following commits: 087c59f [Aaron Davidson] [SPARK-4516] Cap default number of Netty threads at 8 --- .../network/netty/SparkTransportConf.scala | 44 ++++++++++++++++--- 1 file changed, 37 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/netty/SparkTransportConf.scala b/core/src/main/scala/org/apache/spark/network/netty/SparkTransportConf.scala index ce4225cae6d88..cef203006d685 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/SparkTransportConf.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/SparkTransportConf.scala @@ -20,7 +20,24 @@ package org.apache.spark.network.netty import org.apache.spark.SparkConf import org.apache.spark.network.util.{TransportConf, ConfigProvider} +/** + * Provides a utility for transforming from a SparkConf inside a Spark JVM (e.g., Executor, + * Driver, or a standalone shuffle service) into a TransportConf with details on our environment + * like the number of cores that are allocated to this JVM. + */ object SparkTransportConf { + /** + * Specifies an upper bound on the number of Netty threads that Spark requires by default. + * In practice, only 2-4 cores should be required to transfer roughly 10 Gb/s, and each core + * that we use will have an initial overhead of roughly 32 MB of off-heap memory, which comes + * at a premium. + * + * Thus, this value should still retain maximum throughput and reduce wasted off-heap memory + * allocation. It can be overridden by setting the number of serverThreads and clientThreads + * manually in Spark's configuration. + */ + private val MAX_DEFAULT_NETTY_THREADS = 8 + /** * Utility for creating a [[TransportConf]] from a [[SparkConf]]. * @param numUsableCores if nonzero, this will restrict the server and client threads to only @@ -29,15 +46,28 @@ object SparkTransportConf { */ def fromSparkConf(_conf: SparkConf, numUsableCores: Int = 0): TransportConf = { val conf = _conf.clone - if (numUsableCores > 0) { - // Only set if serverThreads/clientThreads not already set. - conf.set("spark.shuffle.io.serverThreads", - conf.get("spark.shuffle.io.serverThreads", numUsableCores.toString)) - conf.set("spark.shuffle.io.clientThreads", - conf.get("spark.shuffle.io.clientThreads", numUsableCores.toString)) - } + + // Specify thread configuration based on our JVM's allocation of cores (rather than necessarily + // assuming we have all the machine's cores). + // NB: Only set if serverThreads/clientThreads not already set. + val numThreads = defaultNumThreads(numUsableCores) + conf.set("spark.shuffle.io.serverThreads", + conf.get("spark.shuffle.io.serverThreads", numThreads.toString)) + conf.set("spark.shuffle.io.clientThreads", + conf.get("spark.shuffle.io.clientThreads", numThreads.toString)) + new TransportConf(new ConfigProvider { override def get(name: String): String = conf.get(name) }) } + + /** + * Returns the default number of threads for both the Netty client and server thread pools. + * If numUsableCores is 0, we will use Runtime get an approximate number of available cores. + */ + private def defaultNumThreads(numUsableCores: Int): Int = { + val availableCores = + if (numUsableCores > 0) numUsableCores else Runtime.getRuntime.availableProcessors() + math.min(availableCores, MAX_DEFAULT_NETTY_THREADS) + } } From 346bc17a2ec8fc9e6eaff90733aa1e8b6b46883e Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Wed, 26 Nov 2014 00:32:45 -0500 Subject: [PATCH 106/109] [SPARK-4516] Avoid allocating Netty PooledByteBufAllocators unnecessarily Turns out we are allocating an allocator pool for every TransportClient (which means that the number increases with the number of nodes in the cluster), when really we should just reuse one for all clients. This patch, as expected, greatly decreases off-heap memory allocation, and appears to make allocation only proportional to the number of cores. Author: Aaron Davidson Closes #3465 from aarondav/fewer-pools and squashes the following commits: 36c49da [Aaron Davidson] [SPARK-4516] Avoid allocating unnecessarily Netty PooledByteBufAllocators --- .../spark/network/client/TransportClientFactory.java | 12 +++++------- .../org/apache/spark/network/util/NettyUtils.java | 6 +++--- 2 files changed, 8 insertions(+), 10 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 76bce8592816a..9afd5decd5e6b 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 @@ -19,7 +19,6 @@ import java.io.Closeable; import java.io.IOException; -import java.lang.reflect.Field; import java.net.InetSocketAddress; import java.net.SocketAddress; import java.util.List; @@ -37,7 +36,6 @@ import io.netty.channel.ChannelOption; import io.netty.channel.EventLoopGroup; import io.netty.channel.socket.SocketChannel; -import io.netty.util.internal.PlatformDependent; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -67,6 +65,7 @@ public class TransportClientFactory implements Closeable { private final Class socketChannelClass; private EventLoopGroup workerGroup; + private PooledByteBufAllocator pooledAllocator; public TransportClientFactory( TransportContext context, @@ -80,6 +79,8 @@ public TransportClientFactory( this.socketChannelClass = NettyUtils.getClientChannelClass(ioMode); // TODO: Make thread pool name configurable. this.workerGroup = NettyUtils.createEventLoop(ioMode, conf.clientThreads(), "shuffle-client"); + this.pooledAllocator = NettyUtils.createPooledByteBufAllocator( + conf.preferDirectBufs(), false /* allowCache */, conf.clientThreads()); } /** @@ -115,11 +116,8 @@ public TransportClient createClient(String remoteHost, int remotePort) throws IO // 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()); - - // Use pooled buffers to reduce temporary buffer allocation - bootstrap.option(ChannelOption.ALLOCATOR, NettyUtils.createPooledByteBufAllocator( - conf.preferDirectBufs(), false /* allowCache */, conf.clientThreads())); + .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, conf.connectionTimeoutMs()) + .option(ChannelOption.ALLOCATOR, pooledAllocator); final AtomicReference clientRef = new AtomicReference(); diff --git a/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java b/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java index 5c654a6fd6ebe..b3991a6577cfe 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java +++ b/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java @@ -109,9 +109,9 @@ public static String getRemoteAddress(Channel channel) { /** * Create a pooled ByteBuf allocator but disables the thread-local cache. Thread-local caches - * are disabled because the ByteBufs are allocated by the event loop thread, but released by the - * executor thread rather than the event loop thread. Those thread-local caches actually delay - * the recycling of buffers, leading to larger memory usage. + * are disabled for TransportClients because the ByteBufs are allocated by the event loop thread, + * but released by the executor thread rather than the event loop thread. Those thread-local + * caches actually delay the recycling of buffers, leading to larger memory usage. */ public static PooledByteBufAllocator createPooledByteBufAllocator( boolean allowDirectBufs, From e7f4d2534bb3361ec4b7af0d42bc798a7a425226 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 25 Nov 2014 23:15:58 -0800 Subject: [PATCH 107/109] [SPARK-4612] Reduce task latency and increase scheduling throughput by making configuration initialization lazy https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/executor/Executor.scala#L337 creates a configuration object for every task that is launched, even if there is no new dependent file/JAR to update. This is a heavy-weight creation that should be avoided if there is no new file/JAR to update. This PR makes that creation lazy. Quick local test in spark-perf scheduling throughput tests gives the following numbers in a local standalone scheduler mode. 1 job with 10000 tasks: before 7.8395 seconds, after 2.6415 seconds = 3x increase in task scheduling throughput pwendell JoshRosen Author: Tathagata Das Closes #3463 from tdas/lazy-config and squashes the following commits: c791c1e [Tathagata Das] Reduce task latency by making configuration initialization lazy --- core/src/main/scala/org/apache/spark/executor/Executor.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 5fa584591d935..835157fc520aa 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -334,7 +334,7 @@ private[spark] class Executor( * SparkContext. Also adds any new JARs we fetched to the class loader. */ private def updateDependencies(newFiles: HashMap[String, Long], newJars: HashMap[String, Long]) { - val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + lazy val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) synchronized { // Fetch missing dependencies for ((name, timestamp) <- newFiles if currentFiles.getOrElse(name, -1L) < timestamp) { From 288ce583b05004a8c71dcd836fab23caff5d4ba7 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Wed, 26 Nov 2014 00:55:28 -0800 Subject: [PATCH 108/109] Removing confusing TripletFields After additional discussion with rxin, I think having all the possible `TripletField` options is confusing. This pull request reduces the triplet fields to: ```java /** * None of the triplet fields are exposed. */ public static final TripletFields None = new TripletFields(false, false, false); /** * Expose only the edge field and not the source or destination field. */ public static final TripletFields EdgeOnly = new TripletFields(false, false, true); /** * Expose the source and edge fields but not the destination field. (Same as Src) */ public static final TripletFields Src = new TripletFields(true, false, true); /** * Expose the destination and edge fields but not the source field. (Same as Dst) */ public static final TripletFields Dst = new TripletFields(false, true, true); /** * Expose all the fields (source, edge, and destination). */ public static final TripletFields All = new TripletFields(true, true, true); ``` Author: Joseph E. Gonzalez Closes #3472 from jegonzal/SimplifyTripletFields and squashes the following commits: 91796b5 [Joseph E. Gonzalez] removing confusing triplet fields --- .../org/apache/spark/graphx/GraphOps.scala | 6 ++-- .../apache/spark/graphx/TripletFields.java | 29 ++----------------- .../apache/spark/graphx/lib/PageRank.scala | 4 +-- .../org/apache/spark/graphx/GraphSuite.scala | 2 +- 4 files changed, 8 insertions(+), 33 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala index d5150382d599b..116d1ea700175 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -129,15 +129,15 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali ctx.sendToSrc(Array((ctx.dstId, ctx.dstAttr))) ctx.sendToDst(Array((ctx.srcId, ctx.srcAttr))) }, - (a, b) => a ++ b, TripletFields.SrcDstOnly) + (a, b) => a ++ b, TripletFields.All) case EdgeDirection.In => graph.aggregateMessages[Array[(VertexId,VD)]]( ctx => ctx.sendToDst(Array((ctx.srcId, ctx.srcAttr))), - (a, b) => a ++ b, TripletFields.SrcOnly) + (a, b) => a ++ b, TripletFields.Src) case EdgeDirection.Out => graph.aggregateMessages[Array[(VertexId,VD)]]( ctx => ctx.sendToSrc(Array((ctx.dstId, ctx.dstAttr))), - (a, b) => a ++ b, TripletFields.DstOnly) + (a, b) => a ++ b, TripletFields.Dst) case EdgeDirection.Both => throw new SparkException("collectEdges does not support EdgeDirection.Both. Use" + "EdgeDirection.Either instead.") diff --git a/graphx/src/main/scala/org/apache/spark/graphx/TripletFields.java b/graphx/src/main/scala/org/apache/spark/graphx/TripletFields.java index 8dfccfe2e23bd..7eb4ae0f44602 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/TripletFields.java +++ b/graphx/src/main/scala/org/apache/spark/graphx/TripletFields.java @@ -55,40 +55,15 @@ public TripletFields(boolean useSrc, boolean useDst, boolean useEdge) { */ public static final TripletFields EdgeOnly = new TripletFields(false, false, true); - /** - * Expose only the source field and not the edge or destination field. - */ - public static final TripletFields SrcOnly = new TripletFields(true, false, false); - - /** - * Expose only the destination field and not the edge or source field. - */ - public static final TripletFields DstOnly = new TripletFields(false, true, false); - - /** - * Expose the source and destination fields but not the edge field. - */ - public static final TripletFields SrcDstOnly = new TripletFields(true, true, false); - /** * Expose the source and edge fields but not the destination field. (Same as Src) */ - public static final TripletFields SrcAndEdge = new TripletFields(true, false, true); - - /** - * Expose the source and edge fields but not the destination field. (Same as SrcAndEdge) - */ - public static final TripletFields Src = SrcAndEdge; + public static final TripletFields Src = new TripletFields(true, false, true); /** * Expose the destination and edge fields but not the source field. (Same as Dst) */ - public static final TripletFields DstAndEdge = new TripletFields(false, true, true); - - /** - * Expose the destination and edge fields but not the source field. (Same as DstAndEdge) - */ - public static final TripletFields Dst = DstAndEdge; + public static final TripletFields Dst = new TripletFields(false, true, true); /** * Expose all the fields (source, edge, and destination). diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala index e40ae0d615466..e139959c3f5c1 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala @@ -85,7 +85,7 @@ object PageRank extends Logging { // Associate the degree with each vertex .outerJoinVertices(graph.outDegrees) { (vid, vdata, deg) => deg.getOrElse(0) } // Set the weight on the edges based on the degree - .mapTriplets( e => 1.0 / e.srcAttr, TripletFields.SrcOnly ) + .mapTriplets( e => 1.0 / e.srcAttr, TripletFields.Src ) // Set the vertex attributes to the initial pagerank values .mapVertices( (id, attr) => resetProb ) @@ -97,7 +97,7 @@ object PageRank extends Logging { // Compute the outgoing rank contributions of each vertex, perform local preaggregation, and // do the final aggregation at the receiving vertices. Requires a shuffle for aggregation. val rankUpdates = rankGraph.aggregateMessages[Double]( - ctx => ctx.sendToDst(ctx.srcAttr * ctx.attr), _ + _, TripletFields.SrcAndEdge) + ctx => ctx.sendToDst(ctx.srcAttr * ctx.attr), _ + _, TripletFields.Src) // Apply the final rank updates to get the new ranks, using join to preserve ranks of vertices // that didn't receive a message. Requires a shuffle for broadcasting updated ranks to the 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 df773db6e4326..a05d1ddb21295 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -328,7 +328,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { "expected ctx.dstAttr to be null due to TripletFields, but it was " + ctx.dstAttr) } ctx.sendToDst(ctx.srcAttr) - }, _ + _, TripletFields.SrcOnly) + }, _ + _, TripletFields.Src) assert(agg.collect().toSet === (1 to n).map(x => (x: VertexId, "v")).toSet) } } From 561d31d2f13cc7b1112ba9f9aa8f08bcd032aebb Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 26 Nov 2014 08:22:50 -0800 Subject: [PATCH 109/109] [SPARK-4614][MLLIB] Slight API changes in Matrix and Matrices Before we have a full picture of the operators we want to add, it might be safer to hide `Matrix.transposeMultiply` in 1.2.0. Another update we want to change is `Matrix.randn` and `Matrix.rand`, both of which should take a `Random` implementation. Otherwise, it is very likely to produce inconsistent RDDs. I also added some unit tests for matrix factory methods. All APIs are new in 1.2, so there is no incompatible changes. brkyvz Author: Xiangrui Meng Closes #3468 from mengxr/SPARK-4614 and squashes the following commits: 3b0e4e2 [Xiangrui Meng] add mima excludes 6bfd8a4 [Xiangrui Meng] hide transposeMultiply; add rng to rand and randn; add unit tests --- .../apache/spark/mllib/linalg/Matrices.scala | 20 ++++---- .../spark/mllib/linalg/MatricesSuite.scala | 50 +++++++++++++++++++ project/MimaExcludes.scala | 6 +++ 3 files changed, 65 insertions(+), 11 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala index 2cc52e94282ba..327366a1a3a82 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -17,12 +17,10 @@ package org.apache.spark.mllib.linalg -import java.util.Arrays +import java.util.{Random, Arrays} import breeze.linalg.{Matrix => BM, DenseMatrix => BDM, CSCMatrix => BSM} -import org.apache.spark.util.random.XORShiftRandom - /** * Trait for a local matrix. */ @@ -67,14 +65,14 @@ sealed trait Matrix extends Serializable { } /** Convenience method for `Matrix`^T^-`DenseMatrix` multiplication. */ - def transposeMultiply(y: DenseMatrix): DenseMatrix = { + private[mllib] def transposeMultiply(y: DenseMatrix): DenseMatrix = { val C: DenseMatrix = Matrices.zeros(numCols, y.numCols).asInstanceOf[DenseMatrix] BLAS.gemm(true, false, 1.0, this, y, 0.0, C) C } /** Convenience method for `Matrix`^T^-`DenseVector` multiplication. */ - def transposeMultiply(y: DenseVector): DenseVector = { + private[mllib] def transposeMultiply(y: DenseVector): DenseVector = { val output = new DenseVector(new Array[Double](numCols)) BLAS.gemv(true, 1.0, this, y, 0.0, output) output @@ -291,22 +289,22 @@ object Matrices { * Generate a `DenseMatrix` consisting of i.i.d. uniform random numbers. * @param numRows number of rows of the matrix * @param numCols number of columns of the matrix + * @param rng a random number generator * @return `DenseMatrix` with size `numRows` x `numCols` and values in U(0, 1) */ - def rand(numRows: Int, numCols: Int): Matrix = { - val rand = new XORShiftRandom - new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rand.nextDouble())) + def rand(numRows: Int, numCols: Int, rng: Random): Matrix = { + new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rng.nextDouble())) } /** * Generate a `DenseMatrix` consisting of i.i.d. gaussian random numbers. * @param numRows number of rows of the matrix * @param numCols number of columns of the matrix + * @param rng a random number generator * @return `DenseMatrix` with size `numRows` x `numCols` and values in N(0, 1) */ - def randn(numRows: Int, numCols: Int): Matrix = { - val rand = new XORShiftRandom - new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rand.nextGaussian())) + def randn(numRows: Int, numCols: Int, rng: Random): Matrix = { + new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rng.nextGaussian())) } /** diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala index 5f8b8c4b72697..322a0e9242918 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala @@ -17,7 +17,11 @@ package org.apache.spark.mllib.linalg +import java.util.Random + +import org.mockito.Mockito.when import org.scalatest.FunSuite +import org.scalatest.mock.MockitoSugar._ class MatricesSuite extends FunSuite { test("dense matrix construction") { @@ -112,4 +116,50 @@ class MatricesSuite extends FunSuite { assert(sparseMat(0, 1) === 10.0) assert(sparseMat.values(2) === 10.0) } + + test("zeros") { + val mat = Matrices.zeros(2, 3).asInstanceOf[DenseMatrix] + assert(mat.numRows === 2) + assert(mat.numCols === 3) + assert(mat.values.forall(_ == 0.0)) + } + + test("ones") { + val mat = Matrices.ones(2, 3).asInstanceOf[DenseMatrix] + assert(mat.numRows === 2) + assert(mat.numCols === 3) + assert(mat.values.forall(_ == 1.0)) + } + + test("eye") { + val mat = Matrices.eye(2).asInstanceOf[DenseMatrix] + assert(mat.numCols === 2) + assert(mat.numCols === 2) + assert(mat.values.toSeq === Seq(1.0, 0.0, 0.0, 1.0)) + } + + test("rand") { + val rng = mock[Random] + when(rng.nextDouble()).thenReturn(1.0, 2.0, 3.0, 4.0) + val mat = Matrices.rand(2, 2, rng).asInstanceOf[DenseMatrix] + assert(mat.numRows === 2) + assert(mat.numCols === 2) + assert(mat.values.toSeq === Seq(1.0, 2.0, 3.0, 4.0)) + } + + test("randn") { + val rng = mock[Random] + when(rng.nextGaussian()).thenReturn(1.0, 2.0, 3.0, 4.0) + val mat = Matrices.randn(2, 2, rng).asInstanceOf[DenseMatrix] + assert(mat.numRows === 2) + assert(mat.numCols === 2) + assert(mat.values.toSeq === Seq(1.0, 2.0, 3.0, 4.0)) + } + + test("diag") { + val mat = Matrices.diag(Vectors.dense(1.0, 2.0)).asInstanceOf[DenseMatrix] + assert(mat.numRows === 2) + assert(mat.numCols === 2) + assert(mat.values.toSeq === Seq(1.0, 0.0, 0.0, 2.0)) + } } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 94de14ddbd2bb..230239aa40500 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -47,6 +47,12 @@ object MimaExcludes { "org.apache.spark.SparkStageInfoImpl.this"), ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.SparkStageInfo.submissionTime") + ) ++ Seq( + // SPARK-4614 + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.mllib.linalg.Matrices.randn"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.mllib.linalg.Matrices.rand") ) case v if v.startsWith("1.2") =>
Executor ID Address
spark.history.fs.logDirectory(none)file:/tmp/spark-events Directory that contains application event logs to be loaded by the history server