From aa7120e0cd8b40a9d0b3edf7c33f18a530d597bc Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Thu, 4 Feb 2016 20:13:42 +0100 Subject: [PATCH 01/19] Initial Broadcast design --- .../plans/physical/partitioning.scala | 7 ++ .../spark/sql/execution/Broadcast.scala | 94 +++++++++++++++++++ .../apache/spark/sql/execution/Exchange.scala | 11 ++- .../execution/joins/BroadcastHashJoin.scala | 70 +++----------- .../joins/BroadcastHashOuterJoin.scala | 68 +++----------- .../apache/spark/sql/execution/python.scala | 4 +- 6 files changed, 135 insertions(+), 119 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index d6e10c412ca1c..547d13707914b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.plans.physical +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.{DataType, IntegerType} @@ -75,6 +76,12 @@ case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { def clustering: Set[Expression] = ordering.map(_.child).toSet } +/** + * Represents data where tuples are broadcasted to every node. It is quite common that the + * entire set of tuples is transformed into different data structure. + */ +case class BroadcastDistribution(f: Iterable[InternalRow] => Any = identity) extends Distribution + /** * Describes how an operator's output is split across partitions. The `compatibleWith`, * `guarantees`, and `satisfies` methods describe relationships between child partitionings, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala new file mode 100644 index 0000000000000..3abb54de9f096 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution + +import scala.concurrent._ +import scala.concurrent.duration._ + +import org.apache.spark.broadcast +import org.apache.spark.rdd.{EmptyRDD, RDD} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.util.ThreadUtils + +/** + * A broadcaster collects transforms and broadcasts the result of an underlying spark plan. + * + * TODO whole stage codegen. + */ +case class Broadcast(f: Iterable[InternalRow] => Any, child: SparkPlan) extends UnaryNode { + override def output: Seq[Attribute] = child.output + + val timeout: Duration = { + val timeoutValue = sqlContext.conf.broadcastTimeout + if (timeoutValue < 0) { + Duration.Inf + } else { + timeoutValue.seconds + } + } + + @transient + private lazy val relation: broadcast.Broadcast[Any] = { + val numBuildRows = longMetric("numRows") + + // broadcastFuture is used in "doExecute". Therefore we can get the execution id correctly here. + val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + val future = Future { + // This will run in another thread. Set the execution id so that we can connect these jobs + // with the correct execution. + SQLExecution.withExecutionId(sparkContext, executionId) { + // Note that we use .execute().collect() because we don't want to convert data to Scala + // types + val input: Array[InternalRow] = child.execute().map { row => + numBuildRows += 1 + row.copy() + }.collect() + + // Construct and broadcast the relation. + sparkContext.broadcast(f(input)) + } + }(Broadcast.executionContext) + Await.result(future, timeout) + } + + + + override protected def doPrepare(): Unit = { + // Materialize the relation. + relation + } + + override protected def doExecute(): RDD[InternalRow] = { + // Return an empty RDD. + // TODO this might violate the principle of least surprise. + new EmptyRDD[InternalRow](sparkContext) + } + + /** Get the constructed relation. */ + def broadcastRelation[T]: broadcast.Broadcast[T] = relation.asInstanceOf[broadcast.Broadcast[T]] +} + +object Broadcast { + def broadcastRelation[T](plan: SparkPlan): broadcast.Broadcast[T] = plan match { + case builder: Broadcast => builder.broadcastRelation + case _ => sys.error("The given plan is not a Broadcaster") + } + + private[execution] val executionContext = ExecutionContext.fromExecutorService( + ThreadUtils.newDaemonCachedThreadPool("build-broadcast", 128)) +} \ No newline at end of file diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 3770883af1e2f..f429495ef6140 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -384,10 +384,13 @@ private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[ // Ensure that the operator's children satisfy their output distribution requirements: children = children.zip(requiredChildDistributions).map { case (child, distribution) => - if (child.outputPartitioning.satisfies(distribution)) { - child - } else { - Exchange(createPartitioning(distribution, defaultNumPreShufflePartitions), child) + distribution match { + case BroadcastDistribution(f) => + Broadcast(f, child) + case _ if child.outputPartitioning.satisfies(distribution) => + child + case _ => + Exchange(createPartitioning(distribution, defaultNumPreShufflePartitions), child) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala index 8b275e886c46c..d148c382c000d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala @@ -17,19 +17,14 @@ package org.apache.spark.sql.execution.joins -import scala.concurrent._ -import scala.concurrent.duration._ - import org.apache.spark.TaskContext -import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{BindReferences, BoundReference, Expression, UnsafeRow} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode, GenerateUnsafeProjection} -import org.apache.spark.sql.catalyst.plans.physical.{Distribution, Partitioning, UnspecifiedDistribution} -import org.apache.spark.sql.execution.{BinaryNode, CodegenSupport, SparkPlan, SQLExecution} +import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Distribution, Partitioning, UnspecifiedDistribution} +import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.util.ThreadUtils import org.apache.spark.util.collection.CompactBuffer /** @@ -52,53 +47,17 @@ case class BroadcastHashJoin( "numRightRows" -> SQLMetrics.createLongMetric(sparkContext, "number of right rows"), "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows")) - val timeout: Duration = { - val timeoutValue = sqlContext.conf.broadcastTimeout - if (timeoutValue < 0) { - Duration.Inf - } else { - timeoutValue.seconds - } - } - override def outputPartitioning: Partitioning = streamedPlan.outputPartitioning - override def requiredChildDistribution: Seq[Distribution] = - UnspecifiedDistribution :: UnspecifiedDistribution :: Nil - - // Use lazy so that we won't do broadcast when calling explain but still cache the broadcast value - // for the same query. - @transient - private lazy val broadcastFuture = { - val numBuildRows = buildSide match { - case BuildLeft => longMetric("numLeftRows") - case BuildRight => longMetric("numRightRows") - } - - // broadcastFuture is used in "doExecute". Therefore we can get the execution id correctly here. - val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - future { - // This will run in another thread. Set the execution id so that we can connect these jobs - // with the correct execution. - SQLExecution.withExecutionId(sparkContext, executionId) { - // Note that we use .execute().collect() because we don't want to convert data to Scala - // types - val input: Array[InternalRow] = buildPlan.execute().map { row => - numBuildRows += 1 - row.copy() - }.collect() - // The following line doesn't run in a job so we cannot track the metric value. However, we - // have already tracked it in the above lines. So here we can use - // `SQLMetrics.nullLongMetric` to ignore it. - val hashed = HashedRelation( - input.iterator, SQLMetrics.nullLongMetric, buildSideKeyGenerator, input.size) - sparkContext.broadcast(hashed) - } - }(BroadcastHashJoin.broadcastHashJoinExecutionContext) + override def requiredChildDistribution: Seq[Distribution] = buildSide match { + case BuildLeft => longMetric("numLeftRows") + BroadcastDistribution(buildRelation) :: UnspecifiedDistribution :: Nil + case BuildRight => + UnspecifiedDistribution :: BroadcastDistribution(buildRelation) :: Nil } - protected override def doPrepare(): Unit = { - broadcastFuture + private val buildRelation: Iterable[InternalRow] => HashedRelation = { input => + HashedRelation(input.iterator, SQLMetrics.nullLongMetric, buildSideKeyGenerator, input.size) } protected override def doExecute(): RDD[InternalRow] = { @@ -108,8 +67,7 @@ case class BroadcastHashJoin( } val numOutputRows = longMetric("numOutputRows") - val broadcastRelation = Await.result(broadcastFuture, timeout) - + val broadcastRelation = Broadcast.broadcastRelation[HashedRelation](buildPlan) streamedPlan.execute().mapPartitions { streamedIter => val hashedRelation = broadcastRelation.value hashedRelation match { @@ -130,7 +88,7 @@ case class BroadcastHashJoin( override def doProduce(ctx: CodegenContext): String = { // create a name for HashRelation - val broadcastRelation = Await.result(broadcastFuture, timeout) + val broadcastRelation = Broadcast.broadcastRelation[HashedRelation](buildPlan) val broadcast = ctx.addReferenceObj("broadcast", broadcastRelation) relationTerm = ctx.freshName("relation") // TODO: create specialized HashRelation for single join key @@ -202,9 +160,3 @@ case class BroadcastHashJoin( """.stripMargin } } - -object BroadcastHashJoin { - - private[joins] val broadcastHashJoinExecutionContext = ExecutionContext.fromExecutorService( - ThreadUtils.newDaemonCachedThreadPool("broadcast-hash-join", 128)) -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala index db8edd169dcfa..bf91e5b5b9795 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala @@ -17,16 +17,13 @@ package org.apache.spark.sql.execution.joins -import scala.concurrent._ -import scala.concurrent.duration._ - -import org.apache.spark.{InternalAccumulator, TaskContext} +import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.{JoinType, LeftOuter, RightOuter} -import org.apache.spark.sql.catalyst.plans.physical.{Distribution, Partitioning, UnspecifiedDistribution} -import org.apache.spark.sql.execution.{BinaryNode, SparkPlan, SQLExecution} +import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Distribution, Partitioning, UnspecifiedDistribution} +import org.apache.spark.sql.execution.{BinaryNode, Broadcast, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics /** @@ -48,57 +45,20 @@ case class BroadcastHashOuterJoin( "numRightRows" -> SQLMetrics.createLongMetric(sparkContext, "number of right rows"), "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows")) - val timeout = { - val timeoutValue = sqlContext.conf.broadcastTimeout - if (timeoutValue < 0) { - Duration.Inf - } else { - timeoutValue.seconds - } + override def requiredChildDistribution: Seq[Distribution] = joinType match { + case RightOuter => longMetric("numLeftRows") + BroadcastDistribution(buildRelation) :: UnspecifiedDistribution :: Nil + case LeftOuter => + UnspecifiedDistribution :: BroadcastDistribution(buildRelation) :: Nil + case x => + throw new IllegalArgumentException(s"HashOuterJoin should not take $x as the JoinType") } - override def requiredChildDistribution: Seq[Distribution] = - UnspecifiedDistribution :: UnspecifiedDistribution :: Nil - - override def outputPartitioning: Partitioning = streamedPlan.outputPartitioning - - // Use lazy so that we won't do broadcast when calling explain but still cache the broadcast value - // for the same query. - @transient - private lazy val broadcastFuture = { - val numBuildRows = joinType match { - case RightOuter => longMetric("numLeftRows") - case LeftOuter => longMetric("numRightRows") - case x => - throw new IllegalArgumentException( - s"HashOuterJoin should not take $x as the JoinType") - } - - // broadcastFuture is used in "doExecute". Therefore we can get the execution id correctly here. - val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - future { - // This will run in another thread. Set the execution id so that we can connect these jobs - // with the correct execution. - SQLExecution.withExecutionId(sparkContext, executionId) { - // Note that we use .execute().collect() because we don't want to convert data to Scala - // types - val input: Array[InternalRow] = buildPlan.execute().map { row => - numBuildRows += 1 - row.copy() - }.collect() - // The following line doesn't run in a job so we cannot track the metric value. However, we - // have already tracked it in the above lines. So here we can use - // `SQLMetrics.nullLongMetric` to ignore it. - val hashed = HashedRelation( - input.iterator, SQLMetrics.nullLongMetric, buildKeyGenerator, input.size) - sparkContext.broadcast(hashed) - } - }(BroadcastHashJoin.broadcastHashJoinExecutionContext) + private val buildRelation: Iterable[InternalRow] => HashedRelation = { input => + HashedRelation(input.iterator, SQLMetrics.nullLongMetric, buildKeyGenerator, input.size) } - protected override def doPrepare(): Unit = { - broadcastFuture - } + override def outputPartitioning: Partitioning = streamedPlan.outputPartitioning override def doExecute(): RDD[InternalRow] = { val numStreamedRows = joinType match { @@ -110,7 +70,7 @@ case class BroadcastHashOuterJoin( } val numOutputRows = longMetric("numOutputRows") - val broadcastRelation = Await.result(broadcastFuture, timeout) + val broadcastRelation = Broadcast.broadcastRelation[UnsafeHashedRelation](buildPlan) streamedPlan.execute().mapPartitions { streamedIter => val joinedRow = new JoinedRow() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python.scala index bf62bb05c3d93..9c2e88a49ae7f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python.scala @@ -26,7 +26,7 @@ import net.razorvine.pickle._ import org.apache.spark.{Accumulator, Logging => SparkLogging, TaskContext} import org.apache.spark.api.python.{PythonBroadcast, PythonRDD, PythonRunner, SerDeUtil} -import org.apache.spark.broadcast.Broadcast +import org.apache.spark.broadcast import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame import org.apache.spark.sql.catalyst.InternalRow @@ -48,7 +48,7 @@ private[spark] case class PythonUDF( pythonIncludes: JList[String], pythonExec: String, pythonVer: String, - broadcastVars: JList[Broadcast[PythonBroadcast]], + broadcastVars: JList[broadcast.Broadcast[PythonBroadcast]], accumulator: Accumulator[JList[Array[Byte]]], dataType: DataType, children: Seq[Expression]) extends Expression with Unevaluable with SparkLogging { From c2b7533f1fb30e9d93856adf4cef4107945670cc Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Thu, 4 Feb 2016 22:34:34 +0100 Subject: [PATCH 02/19] Fix Exchange and initial code gen attempt. --- .../spark/sql/execution/Broadcast.scala | 20 +++++++++++--- .../apache/spark/sql/execution/Exchange.scala | 14 ++++++---- .../execution/joins/BroadcastHashJoin.scala | 15 ++++++----- .../joins/BroadcastHashOuterJoin.scala | 26 ++++++++++--------- .../execution/joins/BroadcastJoinSuite.scala | 3 ++- 5 files changed, 50 insertions(+), 28 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala index 3abb54de9f096..4212d85850bed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala @@ -23,6 +23,8 @@ import org.apache.spark.broadcast import org.apache.spark.rdd.{EmptyRDD, RDD} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext +import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.util.ThreadUtils /** @@ -30,9 +32,17 @@ import org.apache.spark.util.ThreadUtils * * TODO whole stage codegen. */ -case class Broadcast(f: Iterable[InternalRow] => Any, child: SparkPlan) extends UnaryNode { +case class Broadcast( + f: Iterable[InternalRow] => Any, + child: SparkPlan) + extends UnaryNode with CodegenSupport { + override def output: Seq[Attribute] = child.output + override private[sql] lazy val metrics = Map( + "numRows" -> SQLMetrics.createLongMetric(sparkContext, "number of rows") + ) + val timeout: Duration = { val timeoutValue = sqlContext.conf.broadcastTimeout if (timeoutValue < 0) { @@ -66,7 +76,11 @@ case class Broadcast(f: Iterable[InternalRow] => Any, child: SparkPlan) extends Await.result(future, timeout) } + override def upstream(): RDD[InternalRow] = { + child.asInstanceOf[CodegenSupport].upstream() + } + override def doProduce(ctx: CodegenContext): String = "" override protected def doPrepare(): Unit = { // Materialize the relation. @@ -86,9 +100,9 @@ case class Broadcast(f: Iterable[InternalRow] => Any, child: SparkPlan) extends object Broadcast { def broadcastRelation[T](plan: SparkPlan): broadcast.Broadcast[T] = plan match { case builder: Broadcast => builder.broadcastRelation - case _ => sys.error("The given plan is not a Broadcaster") + case _ => sys.error("The given plan is not a Broadcast") } private[execution] val executionContext = ExecutionContext.fromExecutorService( ThreadUtils.newDaemonCachedThreadPool("build-broadcast", 128)) -} \ No newline at end of file +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index f429495ef6140..12d07bcc4b8cd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -385,10 +385,10 @@ private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[ // Ensure that the operator's children satisfy their output distribution requirements: children = children.zip(requiredChildDistributions).map { case (child, distribution) => distribution match { - case BroadcastDistribution(f) => - Broadcast(f, child) case _ if child.outputPartitioning.satisfies(distribution) => child + case BroadcastDistribution(f) => + Broadcast(f, child) case _ => Exchange(createPartitioning(distribution, defaultNumPreShufflePartitions), child) } @@ -396,8 +396,13 @@ private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[ // If the operator has multiple children and specifies child output distributions (e.g. join), // then the children's output partitionings must be compatible: + def requireCompatiblePartitioning(distribution: Distribution): Boolean = distribution match { + case UnspecifiedDistribution => false + case BroadcastDistribution(_) => false + case _ => true + } if (children.length > 1 - && requiredChildDistributions.toSet != Set(UnspecifiedDistribution) + && requiredChildDistributions.exists(requireCompatiblePartitioning) && !Partitioning.allCompatible(children.map(_.outputPartitioning))) { // First check if the existing partitions of the children all match. This means they are @@ -434,8 +439,7 @@ private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[ children.zip(requiredChildDistributions).map { case (child, distribution) => { - val targetPartitioning = - createPartitioning(distribution, numPartitions) + val targetPartitioning = createPartitioning(distribution, numPartitions) if (child.outputPartitioning.guarantees(targetPartitioning)) { child } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala index d148c382c000d..1122ec1d4df7d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala @@ -42,15 +42,19 @@ case class BroadcastHashJoin( right: SparkPlan) extends BinaryNode with HashJoin with CodegenSupport { + val streamSideName = buildSide match { + case BuildLeft => "right" + case BuildRight => "left" + } + override private[sql] lazy val metrics = Map( - "numLeftRows" -> SQLMetrics.createLongMetric(sparkContext, "number of left rows"), - "numRightRows" -> SQLMetrics.createLongMetric(sparkContext, "number of right rows"), + "numStreamRows" -> SQLMetrics.createLongMetric(sparkContext, s"number of $streamSideName rows"), "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows")) override def outputPartitioning: Partitioning = streamedPlan.outputPartitioning override def requiredChildDistribution: Seq[Distribution] = buildSide match { - case BuildLeft => longMetric("numLeftRows") + case BuildLeft => BroadcastDistribution(buildRelation) :: UnspecifiedDistribution :: Nil case BuildRight => UnspecifiedDistribution :: BroadcastDistribution(buildRelation) :: Nil @@ -61,10 +65,7 @@ case class BroadcastHashJoin( } protected override def doExecute(): RDD[InternalRow] = { - val numStreamedRows = buildSide match { - case BuildLeft => longMetric("numRightRows") - case BuildRight => longMetric("numLeftRows") - } + val numStreamedRows = longMetric("numStreamRows") val numOutputRows = longMetric("numOutputRows") val broadcastRelation = Broadcast.broadcastRelation[HashedRelation](buildPlan) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala index bf91e5b5b9795..ae6024a61234f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala @@ -40,9 +40,18 @@ case class BroadcastHashOuterJoin( left: SparkPlan, right: SparkPlan) extends BinaryNode with HashOuterJoin { + private[this] def failOnWrongJoinType(jt: JoinType): Nothing = { + throw new IllegalArgumentException(s"HashOuterJoin should not take $jt as the JoinType") + } + + val streamSideName = joinType match { + case RightOuter => "right" + case LeftOuter => "left" + case jt => failOnWrongJoinType(jt) + } + override private[sql] lazy val metrics = Map( - "numLeftRows" -> SQLMetrics.createLongMetric(sparkContext, "number of left rows"), - "numRightRows" -> SQLMetrics.createLongMetric(sparkContext, "number of right rows"), + "numStreamRows" -> SQLMetrics.createLongMetric(sparkContext, s"number of $streamSideName rows"), "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows")) override def requiredChildDistribution: Seq[Distribution] = joinType match { @@ -51,7 +60,7 @@ case class BroadcastHashOuterJoin( case LeftOuter => UnspecifiedDistribution :: BroadcastDistribution(buildRelation) :: Nil case x => - throw new IllegalArgumentException(s"HashOuterJoin should not take $x as the JoinType") + failOnWrongJoinType(x) } private val buildRelation: Iterable[InternalRow] => HashedRelation = { input => @@ -61,13 +70,7 @@ case class BroadcastHashOuterJoin( override def outputPartitioning: Partitioning = streamedPlan.outputPartitioning override def doExecute(): RDD[InternalRow] = { - val numStreamedRows = joinType match { - case RightOuter => longMetric("numRightRows") - case LeftOuter => longMetric("numLeftRows") - case x => - throw new IllegalArgumentException( - s"HashOuterJoin should not take $x as the JoinType") - } + val numStreamedRows = longMetric("numStreamRows") val numOutputRows = longMetric("numOutputRows") val broadcastRelation = Broadcast.broadcastRelation[UnsafeHashedRelation](buildPlan) @@ -102,8 +105,7 @@ case class BroadcastHashOuterJoin( }) case x => - throw new IllegalArgumentException( - s"BroadcastHashOuterJoin should not take $x as the JoinType") + failOnWrongJoinType(x) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala index aee8e84db56e2..82520347816b8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala @@ -43,6 +43,7 @@ class BroadcastJoinSuite extends QueryTest with BeforeAndAfterAll { val conf = new SparkConf() .setMaster("local-cluster[2,1,1024]") .setAppName("testing") + .set("spark.sql.codegen.wholeStage", "false") val sc = new SparkContext(conf) sqlContext = new SQLContext(sc) } @@ -62,7 +63,7 @@ class BroadcastJoinSuite extends QueryTest with BeforeAndAfterAll { // Comparison at the end is for broadcast left semi join val joinExpression = df1("key") === df2("key") && df1("value") > df2("value") val df3 = df1.join(broadcast(df2), joinExpression, joinType) - val plan = df3.queryExecution.sparkPlan + val plan = df3.queryExecution.executedPlan assert(plan.collect { case p: T => p }.size === 1) plan.executeCollect() } From 6a5568a72461dc9ba5199464c9f0ba950bb530e5 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Sat, 6 Feb 2016 16:09:18 +0100 Subject: [PATCH 03/19] Move broadcast retreval to SparkPlan --- .../spark/sql/execution/Broadcast.scala | 20 +++++------ .../spark/sql/execution/SparkPlan.scala | 34 ++++++++++++++++--- .../sql/execution/WholeStageCodegen.scala | 5 +++ .../execution/joins/BroadcastHashJoin.scala | 5 +-- .../joins/BroadcastHashOuterJoin.scala | 4 +-- .../sql/execution/joins/InnerJoinSuite.scala | 9 ++++- 6 files changed, 55 insertions(+), 22 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala index 4212d85850bed..c132fa8cac864 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala @@ -53,12 +53,12 @@ case class Broadcast( } @transient - private lazy val relation: broadcast.Broadcast[Any] = { + private lazy val relationFuture: Future[broadcast.Broadcast[Any]] = { val numBuildRows = longMetric("numRows") // broadcastFuture is used in "doExecute". Therefore we can get the execution id correctly here. val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - val future = Future { + Future { // This will run in another thread. Set the execution id so that we can connect these jobs // with the correct execution. SQLExecution.withExecutionId(sparkContext, executionId) { @@ -73,7 +73,6 @@ case class Broadcast( sparkContext.broadcast(f(input)) } }(Broadcast.executionContext) - Await.result(future, timeout) } override def upstream(): RDD[InternalRow] = { @@ -83,8 +82,8 @@ case class Broadcast( override def doProduce(ctx: CodegenContext): String = "" override protected def doPrepare(): Unit = { - // Materialize the relation. - relation + // Materialize the future. + relationFuture } override protected def doExecute(): RDD[InternalRow] = { @@ -93,16 +92,13 @@ case class Broadcast( new EmptyRDD[InternalRow](sparkContext) } - /** Get the constructed relation. */ - def broadcastRelation[T]: broadcast.Broadcast[T] = relation.asInstanceOf[broadcast.Broadcast[T]] + override protected[sql] def doExecuteBroadcast[T](): broadcast.Broadcast[T] = { + val result = Await.result(relationFuture, timeout) + result.asInstanceOf[broadcast.Broadcast[T]] + } } object Broadcast { - def broadcastRelation[T](plan: SparkPlan): broadcast.Broadcast[T] = plan match { - case builder: Broadcast => builder.broadcastRelation - case _ => sys.error("The given plan is not a Broadcast") - } - private[execution] val executionContext = ExecutionContext.fromExecutorService( ThreadUtils.newDaemonCachedThreadPool("build-broadcast", 128)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index b19b772409d83..3f7a96b6ecd46 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -22,6 +22,7 @@ import java.util.concurrent.atomic.AtomicBoolean import scala.collection.mutable.ArrayBuffer import org.apache.spark.Logging +import org.apache.spark.broadcast import org.apache.spark.rdd.{RDD, RDDOperationScope} import org.apache.spark.sql.{Row, SQLContext} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} @@ -98,14 +99,29 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ def requiredChildOrdering: Seq[Seq[SortOrder]] = Seq.fill(children.size)(Nil) /** - * Returns the result of this query as an RDD[InternalRow] by delegating to doExecute - * after adding query plan information to created RDDs for visualization. - * Concrete implementations of SparkPlan should override doExecute instead. + * Returns the result of this query as an RDD[InternalRow] by delegating to doExecute after + * preparations. Concrete implementations of SparkPlan should override doExecute. */ - final def execute(): RDD[InternalRow] = { + final def execute(): RDD[InternalRow] = executeQuery { + doExecute() + } + + /** + * Returns the result of this query as a broadcast variable by delegating to doBroadcast after + * preparations. Concrete implementations of SparkPlan should override doBroadcast. + */ + final def executeBroadcast[T](): broadcast.Broadcast[T] = executeQuery { + doExecuteBroadcast() + } + + /** + * Execute a query after preparing the query and adding query plan information to created RDDs + * for visualization. + */ + private final def executeQuery[T](query: => T): T = { RDDOperationScope.withScope(sparkContext, nodeName, false, true) { prepare() - doExecute() + query } } @@ -135,6 +151,14 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ */ protected def doExecute(): RDD[InternalRow] + /** + * Overridden by concrete implementations of SparkPlan. + * Produces the result of the query as a broadcast variable. + */ + protected[sql] def doExecuteBroadcast[T](): broadcast.Broadcast[T] = { + throw new NotImplementedError(s"$nodeName does not implement createBroadcast") + } + /** * Runs this query returning the result as an array. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala index 131efea20f31e..2ecf64d1e33e7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution import scala.collection.mutable.ArrayBuffer +import org.apache.spark.broadcast import org.apache.spark.rdd.RDD import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.InternalRow @@ -156,6 +157,10 @@ case class InputAdapter(child: SparkPlan) extends LeafNode with CodegenSupport { child.execute() } + override protected[sql] def doExecuteBroadcast[T](): broadcast.Broadcast[T] = { + child.doExecuteBroadcast[T]() + } + override def supportCodegen: Boolean = false override def upstream(): RDD[InternalRow] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala index 1122ec1d4df7d..f9ffa7b5cf0e4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala @@ -68,9 +68,10 @@ case class BroadcastHashJoin( val numStreamedRows = longMetric("numStreamRows") val numOutputRows = longMetric("numOutputRows") - val broadcastRelation = Broadcast.broadcastRelation[HashedRelation](buildPlan) + val broadcastRelation = buildPlan.executeBroadcast[UnsafeHashedRelation]() streamedPlan.execute().mapPartitions { streamedIter => val hashedRelation = broadcastRelation.value + logWarning(s"Using Hashed Relation size=${hashedRelation.estimatedSize}") hashedRelation match { case unsafe: UnsafeHashedRelation => TaskContext.get().taskMetrics().incPeakExecutionMemory(unsafe.getUnsafeSize) @@ -89,7 +90,7 @@ case class BroadcastHashJoin( override def doProduce(ctx: CodegenContext): String = { // create a name for HashRelation - val broadcastRelation = Broadcast.broadcastRelation[HashedRelation](buildPlan) + val broadcastRelation = buildPlan.executeBroadcast[UnsafeHashedRelation]() val broadcast = ctx.addReferenceObj("broadcast", broadcastRelation) relationTerm = ctx.freshName("relation") // TODO: create specialized HashRelation for single join key diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala index ae6024a61234f..20ad4b9501005 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.{JoinType, LeftOuter, RightOuter} import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Distribution, Partitioning, UnspecifiedDistribution} -import org.apache.spark.sql.execution.{BinaryNode, Broadcast, SparkPlan} +import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics /** @@ -73,7 +73,7 @@ case class BroadcastHashOuterJoin( val numStreamedRows = longMetric("numStreamRows") val numOutputRows = longMetric("numOutputRows") - val broadcastRelation = Broadcast.broadcastRelation[UnsafeHashedRelation](buildPlan) + val broadcastRelation = buildPlan.executeBroadcast[UnsafeHashedRelation]() streamedPlan.execute().mapPartitions { streamedIter => val joinedRow = new JoinedRow() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala index 149f34dbd748f..4081047cc227e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala @@ -29,6 +29,11 @@ import org.apache.spark.sql.types.{IntegerType, StringType, StructType} class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { import testImplicits.localSeqToDataFrameHolder + override def beforeAll(): Unit = { + super.beforeAll() + sqlContext.setConf("spark.sql.codegen.wholeStage", "false") + } + private lazy val myUpperCaseData = sqlContext.createDataFrame( sparkContext.parallelize(Seq( Row(1, "A"), @@ -88,7 +93,9 @@ class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { leftPlan: SparkPlan, rightPlan: SparkPlan, side: BuildSide) = { - joins.BroadcastHashJoin(leftKeys, rightKeys, side, boundCondition, leftPlan, rightPlan) + val broadcastJoin = + joins.BroadcastHashJoin(leftKeys, rightKeys, side, boundCondition, leftPlan, rightPlan) + EnsureRequirements(sqlContext).apply(broadcastJoin) } def makeSortMergeJoin( From d0194fb891f589436cf56f7360fb4d2324dd6837 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Sat, 6 Feb 2016 22:00:28 +0100 Subject: [PATCH 04/19] Fix Codegen & Add other broadcast joins. --- .../plans/physical/partitioning.scala | 5 ++- .../spark/sql/execution/Broadcast.scala | 8 ++--- .../apache/spark/sql/execution/Exchange.scala | 23 ++++++++----- .../spark/sql/execution/SparkPlan.scala | 2 +- .../sql/execution/WholeStageCodegen.scala | 14 ++++---- .../execution/joins/BroadcastHashJoin.scala | 3 +- .../joins/BroadcastHashOuterJoin.scala | 4 +-- .../joins/BroadcastLeftSemiJoinHash.scala | 33 ++++++++++--------- .../joins/BroadcastNestedLoopJoin.scala | 31 +++++++++-------- .../sql/execution/joins/LeftSemiJoinBNL.scala | 26 +++++---------- .../execution/metric/SQLMetricsSuite.scala | 30 ++++++++++------- 11 files changed, 93 insertions(+), 86 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index 547d13707914b..c54b09cf259eb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -220,7 +220,10 @@ case class RoundRobinPartitioning(numPartitions: Int) extends Partitioning { case object SinglePartition extends Partitioning { val numPartitions = 1 - override def satisfies(required: Distribution): Boolean = true + override def satisfies(required: Distribution): Boolean = required match { + case _: BroadcastDistribution => false + case _ => true + } override def compatibleWith(other: Partitioning): Boolean = other.numPartitions == 1 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala index c132fa8cac864..3db16c4071410 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala @@ -20,7 +20,7 @@ import scala.concurrent._ import scala.concurrent.duration._ import org.apache.spark.broadcast -import org.apache.spark.rdd.{EmptyRDD, RDD} +import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext @@ -28,7 +28,7 @@ import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.util.ThreadUtils /** - * A broadcaster collects transforms and broadcasts the result of an underlying spark plan. + * A broadcast collects, transforms and finally broadcasts the result of a transformed SparkPlan. * * TODO whole stage codegen. */ @@ -87,9 +87,7 @@ case class Broadcast( } override protected def doExecute(): RDD[InternalRow] = { - // Return an empty RDD. - // TODO this might violate the principle of least surprise. - new EmptyRDD[InternalRow](sparkContext) + child.execute() // TODO throw an Exception here? } override protected[sql] def doExecuteBroadcast[T](): broadcast.Broadcast[T] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 12d07bcc4b8cd..a8d5f538abfcc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -383,15 +383,20 @@ private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[ assert(requiredChildOrderings.length == children.length) // Ensure that the operator's children satisfy their output distribution requirements: - children = children.zip(requiredChildDistributions).map { case (child, distribution) => - distribution match { - case _ if child.outputPartitioning.satisfies(distribution) => - child - case BroadcastDistribution(f) => - Broadcast(f, child) - case _ => - Exchange(createPartitioning(distribution, defaultNumPreShufflePartitions), child) - } + children = children.zip(requiredChildDistributions).map { + case (child, distribution) if child.outputPartitioning.satisfies(distribution) => + child + case (child, BroadcastDistribution(f1)) => + child match { + // The child is broadcasting the same variable: keep the child. + case Broadcast(f2, _) if f1 == f2 => child + // The child is broadcasting a different variable: replace the child. + case Broadcast(f2, src) => Broadcast(f1, src) + // Create a broadcast on top of the child. + case _ => Broadcast(f1, child) + } + case (child, distribution) => + Exchange(createPartitioning(distribution, defaultNumPreShufflePartitions), child) } // If the operator has multiple children and specifies child output distributions (e.g. join), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index e7adb037c97fa..cd595d861d803 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -156,7 +156,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ * Produces the result of the query as a broadcast variable. */ protected[sql] def doExecuteBroadcast[T](): broadcast.Broadcast[T] = { - throw new NotImplementedError(s"$nodeName does not implement createBroadcast") + throw new NotImplementedError(s"$nodeName does not implement doExecuteBroadcast:\n$this") } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala index 2ecf64d1e33e7..4a257d37fc9c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala @@ -157,10 +157,6 @@ case class InputAdapter(child: SparkPlan) extends LeafNode with CodegenSupport { child.execute() } - override protected[sql] def doExecuteBroadcast[T](): broadcast.Broadcast[T] = { - child.doExecuteBroadcast[T]() - } - override def supportCodegen: Boolean = false override def upstream(): RDD[InternalRow] = { @@ -371,6 +367,8 @@ private[sql] case class CollapseCodegenStages(sqlContext: SQLContext) extends Ru if (sqlContext.conf.wholeStageEnabled) { plan.transform { case plan: CodegenSupport if supportCodegen(plan) && + // A broadcast operator cannot be at the top level. + !plan.isInstanceOf[Broadcast] && // Whole stage codegen is only useful when there are at least two levels of operators that // support it (save at least one projection/iterator). (Utils.isTesting || plan.children.exists(supportCodegen)) => @@ -378,10 +376,10 @@ private[sql] case class CollapseCodegenStages(sqlContext: SQLContext) extends Ru var inputs = ArrayBuffer[SparkPlan]() val combined = plan.transform { // The build side can't be compiled together - case b @ BroadcastHashJoin(_, _, BuildLeft, _, left, right) => - b.copy(left = apply(left)) - case b @ BroadcastHashJoin(_, _, BuildRight, _, left, right) => - b.copy(right = apply(right)) + case b @ BroadcastHashJoin(_, _, BuildLeft, _, Broadcast(f, left), _) => + b.copy(left = Broadcast(f, apply(left))) + case b @ BroadcastHashJoin(_, _, BuildRight, _, _, Broadcast(f, right)) => + b.copy(right = Broadcast(f, apply(right))) case p if !supportCodegen(p) => val input = apply(p) // collapse them recursively inputs += input diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala index f9ffa7b5cf0e4..c28297ceb700b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala @@ -60,7 +60,7 @@ case class BroadcastHashJoin( UnspecifiedDistribution :: BroadcastDistribution(buildRelation) :: Nil } - private val buildRelation: Iterable[InternalRow] => HashedRelation = { input => + private[this] val buildRelation: Iterable[InternalRow] => HashedRelation = { input => HashedRelation(input.iterator, SQLMetrics.nullLongMetric, buildSideKeyGenerator, input.size) } @@ -71,7 +71,6 @@ case class BroadcastHashJoin( val broadcastRelation = buildPlan.executeBroadcast[UnsafeHashedRelation]() streamedPlan.execute().mapPartitions { streamedIter => val hashedRelation = broadcastRelation.value - logWarning(s"Using Hashed Relation size=${hashedRelation.estimatedSize}") hashedRelation match { case unsafe: UnsafeHashedRelation => TaskContext.get().taskMetrics().incPeakExecutionMemory(unsafe.getUnsafeSize) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala index 20ad4b9501005..dbfab0c6b9c64 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala @@ -55,7 +55,7 @@ case class BroadcastHashOuterJoin( "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows")) override def requiredChildDistribution: Seq[Distribution] = joinType match { - case RightOuter => longMetric("numLeftRows") + case RightOuter => BroadcastDistribution(buildRelation) :: UnspecifiedDistribution :: Nil case LeftOuter => UnspecifiedDistribution :: BroadcastDistribution(buildRelation) :: Nil @@ -63,7 +63,7 @@ case class BroadcastHashOuterJoin( failOnWrongJoinType(x) } - private val buildRelation: Iterable[InternalRow] => HashedRelation = { input => + private[this] val buildRelation: Iterable[InternalRow] => HashedRelation = { input => HashedRelation(input.iterator, SQLMetrics.nullLongMetric, buildKeyGenerator, input.size) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala index 8929dc3af1912..8defcdb727961 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala @@ -17,10 +17,11 @@ package org.apache.spark.sql.execution.joins -import org.apache.spark.{InternalAccumulator, TaskContext} +import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Distribution, UnspecifiedDistribution} import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics @@ -37,31 +38,33 @@ case class BroadcastLeftSemiJoinHash( override private[sql] lazy val metrics = Map( "numLeftRows" -> SQLMetrics.createLongMetric(sparkContext, "number of left rows"), - "numRightRows" -> SQLMetrics.createLongMetric(sparkContext, "number of right rows"), "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows")) + override def requiredChildDistribution: Seq[Distribution] = { + UnspecifiedDistribution :: BroadcastDistribution(buildRelation) :: Nil + } + + private[this] val buildRelation: Iterable[InternalRow] => Any = { + if (condition.isEmpty) { + (input: Iterable[InternalRow]) => + buildKeyHashSet(input.toIterator, SQLMetrics.nullLongMetric) + } else { + (input: Iterable[InternalRow]) => + HashedRelation(input.toIterator, SQLMetrics.nullLongMetric, rightKeyGenerator, input.size) + } + } + protected override def doExecute(): RDD[InternalRow] = { val numLeftRows = longMetric("numLeftRows") - val numRightRows = longMetric("numRightRows") val numOutputRows = longMetric("numOutputRows") - val input = right.execute().map { row => - numRightRows += 1 - row.copy() - }.collect() - if (condition.isEmpty) { - val hashSet = buildKeyHashSet(input.toIterator, SQLMetrics.nullLongMetric) - val broadcastedRelation = sparkContext.broadcast(hashSet) - + val broadcastedRelation = right.executeBroadcast[java.util.Set[InternalRow]]() left.execute().mapPartitionsInternal { streamIter => hashSemiJoin(streamIter, numLeftRows, broadcastedRelation.value, numOutputRows) } } else { - val hashRelation = - HashedRelation(input.toIterator, SQLMetrics.nullLongMetric, rightKeyGenerator, input.size) - val broadcastedRelation = sparkContext.broadcast(hashRelation) - + val broadcastedRelation = right.executeBroadcast[HashedRelation]() left.execute().mapPartitionsInternal { streamIter => val hashedRelation = broadcastedRelation.value hashedRelation match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala index e55f8694781a3..545aeeabe4327 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala @@ -21,7 +21,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Distribution, Partitioning, UnspecifiedDistribution} import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.util.collection.{BitSet, CompactBuffer} @@ -33,11 +33,14 @@ case class BroadcastNestedLoopJoin( buildSide: BuildSide, joinType: JoinType, condition: Option[Expression]) extends BinaryNode { - // TODO: Override requiredChildDistribution. + + val streamSideName = buildSide match { + case BuildLeft => "right" + case BuildRight => "left" + } override private[sql] lazy val metrics = Map( - "numLeftRows" -> SQLMetrics.createLongMetric(sparkContext, "number of left rows"), - "numRightRows" -> SQLMetrics.createLongMetric(sparkContext, "number of right rows"), + "numStreamRows" -> SQLMetrics.createLongMetric(sparkContext, s"number of $streamSideName rows"), "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows")) /** BuildRight means the right relation <=> the broadcast relation. */ @@ -46,8 +49,15 @@ case class BroadcastNestedLoopJoin( case BuildLeft => (right, left) } + override def requiredChildDistribution: Seq[Distribution] = buildSide match { + case BuildLeft => + BroadcastDistribution(_.toIndexedSeq) :: UnspecifiedDistribution :: Nil + case BuildRight => + UnspecifiedDistribution :: BroadcastDistribution(_.toIndexedSeq) :: Nil + } + private[this] def genResultProjection: InternalRow => InternalRow = { - UnsafeProjection.create(schema) + UnsafeProjection.create(schema) } override def outputPartitioning: Partitioning = streamed.outputPartitioning @@ -73,17 +83,10 @@ case class BroadcastNestedLoopJoin( newPredicate(condition.getOrElse(Literal(true)), left.output ++ right.output) protected override def doExecute(): RDD[InternalRow] = { - val (numStreamedRows, numBuildRows) = buildSide match { - case BuildRight => (longMetric("numLeftRows"), longMetric("numRightRows")) - case BuildLeft => (longMetric("numRightRows"), longMetric("numLeftRows")) - } + val numStreamedRows = longMetric("numStreamRows") val numOutputRows = longMetric("numOutputRows") - val broadcastedRelation = - sparkContext.broadcast(broadcast.execute().map { row => - numBuildRows += 1 - row.copy() - }.collect().toIndexedSeq) + val broadcastedRelation = broadcast.executeBroadcast[IndexedSeq[InternalRow]]() /** All rows that either match both-way, or rows from streamed joined with nulls. */ val matchesOrStreamedRowsWithNulls = streamed.execute().mapPartitions { streamedIter => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala index 82498ee395649..5624ea72fb9e4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Distribution, Partitioning, UnspecifiedDistribution} import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics @@ -29,40 +29,32 @@ import org.apache.spark.sql.execution.metric.SQLMetrics * for hash join. */ case class LeftSemiJoinBNL( - streamed: SparkPlan, broadcast: SparkPlan, condition: Option[Expression]) + left: SparkPlan, right: SparkPlan, condition: Option[Expression]) extends BinaryNode { // TODO: Override requiredChildDistribution. override private[sql] lazy val metrics = Map( "numLeftRows" -> SQLMetrics.createLongMetric(sparkContext, "number of left rows"), - "numRightRows" -> SQLMetrics.createLongMetric(sparkContext, "number of right rows"), "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows")) - override def outputPartitioning: Partitioning = streamed.outputPartitioning - - override def output: Seq[Attribute] = left.output + override def requiredChildDistribution: Seq[Distribution] = { + UnspecifiedDistribution :: BroadcastDistribution(_.toIndexedSeq) :: Nil + } - /** The Streamed Relation */ - override def left: SparkPlan = streamed + override def outputPartitioning: Partitioning = left.outputPartitioning - /** The Broadcast relation */ - override def right: SparkPlan = broadcast + override def output: Seq[Attribute] = left.output @transient private lazy val boundCondition = newPredicate(condition.getOrElse(Literal(true)), left.output ++ right.output) protected override def doExecute(): RDD[InternalRow] = { val numLeftRows = longMetric("numLeftRows") - val numRightRows = longMetric("numRightRows") val numOutputRows = longMetric("numOutputRows") - val broadcastedRelation = - sparkContext.broadcast(broadcast.execute().map { row => - numRightRows += 1 - row.copy() - }.collect().toIndexedSeq) + val broadcastedRelation = right.executeBroadcast[IndexedSeq[InternalRow]]() - streamed.execute().mapPartitions { streamedIter => + left.execute().mapPartitions { streamedIter => val joinedRow = new JoinedRow streamedIter.filter(streamedRow => { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 2260e4870299a..58833239b6ecf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -227,8 +227,9 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { testSparkPlanMetrics(df, 2, Map( 1L -> ("BroadcastHashJoin", Map( "number of left rows" -> 2L, - "number of right rows" -> 4L, - "number of output rows" -> 2L))) + "number of output rows" -> 2L)), + 3L -> ("Broadcast", Map( + "number of rows" -> 4L))) ) } @@ -241,16 +242,18 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { testSparkPlanMetrics(df, 2, Map( 0L -> ("BroadcastHashOuterJoin", Map( "number of left rows" -> 3L, - "number of right rows" -> 4L, - "number of output rows" -> 5L))) + "number of output rows" -> 5L)), + 2L -> ("Broadcast", Map( + "number of rows" -> 4L))) ) val df3 = df1.join(broadcast(df2), $"key" === $"key2", "right_outer") testSparkPlanMetrics(df3, 2, Map( 0L -> ("BroadcastHashOuterJoin", Map( - "number of left rows" -> 3L, "number of right rows" -> 4L, - "number of output rows" -> 6L))) + "number of output rows" -> 6L)), + 1L -> ("Broadcast", Map( + "number of rows" -> 3L))) ) } @@ -266,8 +269,9 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { testSparkPlanMetrics(df, 3, Map( 1L -> ("BroadcastNestedLoopJoin", Map( "number of left rows" -> 12L, // left needs to be scanned twice - "number of right rows" -> 2L, - "number of output rows" -> 12L))) + "number of output rows" -> 12L)), + 3L -> ("Broadcast", Map( + "number of rows" -> 2L))) ) } } @@ -281,8 +285,9 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { testSparkPlanMetrics(df, 2, Map( 0L -> ("BroadcastLeftSemiJoinHash", Map( "number of left rows" -> 2L, - "number of right rows" -> 4L, - "number of output rows" -> 2L))) + "number of output rows" -> 2L)), + 2L -> ("Broadcast", Map( + "number of rows" -> 4L))) ) } @@ -311,8 +316,9 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { testSparkPlanMetrics(df, 2, Map( 0L -> ("LeftSemiJoinBNL", Map( "number of left rows" -> 2L, - "number of right rows" -> 4L, - "number of output rows" -> 2L))) + "number of output rows" -> 2L)), + 2L -> ("Broadcast", Map( + "number of rows" -> 4L))) ) } From 02a61b86c8584d007826c4e2d5e542f22185a0f9 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Sat, 6 Feb 2016 22:02:58 +0100 Subject: [PATCH 05/19] Minor touchup --- .../main/scala/org/apache/spark/sql/execution/SparkPlan.scala | 2 +- .../org/apache/spark/sql/execution/joins/InnerJoinSuite.scala | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index cd595d861d803..0b8352bfc0f1d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -156,7 +156,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ * Produces the result of the query as a broadcast variable. */ protected[sql] def doExecuteBroadcast[T](): broadcast.Broadcast[T] = { - throw new NotImplementedError(s"$nodeName does not implement doExecuteBroadcast:\n$this") + throw new NotImplementedError(s"$nodeName does not implement doExecuteBroadcast") } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala index 4081047cc227e..d34859ec401cf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala @@ -31,7 +31,6 @@ class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { override def beforeAll(): Unit = { super.beforeAll() - sqlContext.setConf("spark.sql.codegen.wholeStage", "false") } private lazy val myUpperCaseData = sqlContext.createDataFrame( From c12c8e6294e229944dc3b33f378c3ce8cff66591 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Sun, 7 Feb 2016 10:42:51 +0100 Subject: [PATCH 06/19] Move broadcast relation retrieval. --- .../spark/sql/execution/joins/BroadcastNestedLoopJoin.scala | 6 ++++-- .../apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala | 6 +++--- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala index 545aeeabe4327..bb86fa5c73d8b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala @@ -98,13 +98,15 @@ case class BroadcastNestedLoopJoin( val rightNulls = new GenericMutableRow(right.output.size) val resultProj = genResultProjection + val relation = broadcastedRelation.value + streamedIter.foreach { streamedRow => var i = 0 var streamRowMatched = false numStreamedRows += 1 - while (i < broadcastedRelation.value.size) { - val broadcastedRow = broadcastedRelation.value(i) + while (i < relation.size) { + val broadcastedRow = relation(i) buildSide match { case BuildRight if boundCondition(joinedRow(streamedRow, broadcastedRow)) => matchedRows += resultProj(joinedRow(streamedRow, broadcastedRow)).copy() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala index 5624ea72fb9e4..b18c7b1e7295b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala @@ -56,15 +56,15 @@ case class LeftSemiJoinBNL( left.execute().mapPartitions { streamedIter => val joinedRow = new JoinedRow + val relation = broadcastedRelation.value streamedIter.filter(streamedRow => { numLeftRows += 1 var i = 0 var matched = false - while (i < broadcastedRelation.value.size && !matched) { - val broadcastedRow = broadcastedRelation.value(i) - if (boundCondition(joinedRow(streamedRow, broadcastedRow))) { + while (i < relation.size && !matched) { + if (boundCondition(joinedRow(streamedRow, relation(i)))) { matched = true } i += 1 From c7dd7ae56962b5b8eb96add85ec06b4b60b6f351 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Mon, 8 Feb 2016 22:51:18 +0100 Subject: [PATCH 07/19] Remove codegen from broadcast. --- .../org/apache/spark/sql/execution/Broadcast.scala | 13 +------------ .../spark/sql/execution/WholeStageCodegen.scala | 5 +++-- 2 files changed, 4 insertions(+), 14 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala index 3db16c4071410..194b6f4de8ac7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala @@ -29,13 +29,8 @@ import org.apache.spark.util.ThreadUtils /** * A broadcast collects, transforms and finally broadcasts the result of a transformed SparkPlan. - * - * TODO whole stage codegen. */ -case class Broadcast( - f: Iterable[InternalRow] => Any, - child: SparkPlan) - extends UnaryNode with CodegenSupport { +case class Broadcast(f: Iterable[InternalRow] => Any, child: SparkPlan) extends UnaryNode { override def output: Seq[Attribute] = child.output @@ -75,12 +70,6 @@ case class Broadcast( }(Broadcast.executionContext) } - override def upstream(): RDD[InternalRow] = { - child.asInstanceOf[CodegenSupport].upstream() - } - - override def doProduce(ctx: CodegenContext): String = "" - override protected def doPrepare(): Unit = { // Materialize the future. relationFuture diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala index 4a257d37fc9c2..b0c5e7dbe94cb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala @@ -360,6 +360,9 @@ private[sql] case class CollapseCodegenStages(sqlContext: SQLContext) extends Ru // the generated code will be huge if there are too many columns val haveManyColumns = plan.output.length > 200 !willFallback && !haveManyColumns + // Collapse a broadcast into the stage - it should not contain any code that can be + // codegenerated. + case _: Broadcast => true case _ => false } @@ -367,8 +370,6 @@ private[sql] case class CollapseCodegenStages(sqlContext: SQLContext) extends Ru if (sqlContext.conf.wholeStageEnabled) { plan.transform { case plan: CodegenSupport if supportCodegen(plan) && - // A broadcast operator cannot be at the top level. - !plan.isInstanceOf[Broadcast] && // Whole stage codegen is only useful when there are at least two levels of operators that // support it (save at least one projection/iterator). (Utils.isTesting || plan.children.exists(supportCodegen)) => From 9c0f4bf4912a4f253f99fc091f7f95ef0b080538 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Sun, 14 Feb 2016 23:57:06 +0100 Subject: [PATCH 08/19] Remove closure passing. --- .../plans/physical/partitioning.scala | 13 ++++- .../spark/sql/execution/Broadcast.scala | 19 +++---- .../apache/spark/sql/execution/Exchange.scala | 22 ++++++-- .../sql/execution/WholeStageCodegen.scala | 11 ++-- .../execution/joins/BroadcastHashJoin.scala | 20 +++---- .../joins/BroadcastHashOuterJoin.scala | 21 ++++---- .../joins/BroadcastLeftSemiJoinHash.scala | 13 ++--- .../joins/BroadcastNestedLoopJoin.scala | 16 +++--- .../sql/execution/joins/HashSemiJoin.scala | 53 +++++++++++++------ .../sql/execution/joins/HashedRelation.scala | 24 +++++++-- .../sql/execution/joins/LeftSemiJoinBNL.scala | 10 ++-- 11 files changed, 133 insertions(+), 89 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index c54b09cf259eb..a632385c238f2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -76,11 +76,22 @@ case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { def clustering: Set[Expression] = ordering.map(_.child).toSet } +/** + * Marker trait to identify the shape in which tuples are broadcasted. Typical examples of this are + * identity (tuples remain unchanged) or hashed (tuples are converted into some hash index). + */ +trait BroadcastMode + +/** + * IdentityBroadcastMode requires that rows are broadcasted in their original form. + */ +case object IdentityBroadcastMode extends BroadcastMode + /** * Represents data where tuples are broadcasted to every node. It is quite common that the * entire set of tuples is transformed into different data structure. */ -case class BroadcastDistribution(f: Iterable[InternalRow] => Any = identity) extends Distribution +case class BroadcastDistribution(mode: BroadcastMode) extends Distribution /** * Describes how an operator's output is split across partitions. The `compatibleWith`, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala index 194b6f4de8ac7..a875e9fd06b84 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala @@ -23,21 +23,19 @@ import org.apache.spark.broadcast import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext -import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.catalyst.plans.physical.BroadcastMode import org.apache.spark.util.ThreadUtils /** * A broadcast collects, transforms and finally broadcasts the result of a transformed SparkPlan. */ -case class Broadcast(f: Iterable[InternalRow] => Any, child: SparkPlan) extends UnaryNode { +case class Broadcast( + mode: BroadcastMode, + transform: Array[InternalRow] => Any, + child: SparkPlan) extends UnaryNode { override def output: Seq[Attribute] = child.output - override private[sql] lazy val metrics = Map( - "numRows" -> SQLMetrics.createLongMetric(sparkContext, "number of rows") - ) - val timeout: Duration = { val timeoutValue = sqlContext.conf.broadcastTimeout if (timeoutValue < 0) { @@ -49,8 +47,6 @@ case class Broadcast(f: Iterable[InternalRow] => Any, child: SparkPlan) extends @transient private lazy val relationFuture: Future[broadcast.Broadcast[Any]] = { - val numBuildRows = longMetric("numRows") - // broadcastFuture is used in "doExecute". Therefore we can get the execution id correctly here. val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) Future { @@ -60,12 +56,11 @@ case class Broadcast(f: Iterable[InternalRow] => Any, child: SparkPlan) extends // Note that we use .execute().collect() because we don't want to convert data to Scala // types val input: Array[InternalRow] = child.execute().map { row => - numBuildRows += 1 row.copy() }.collect() // Construct and broadcast the relation. - sparkContext.broadcast(f(input)) + sparkContext.broadcast(transform(input)) } }(Broadcast.executionContext) } @@ -87,5 +82,5 @@ case class Broadcast(f: Iterable[InternalRow] => Any, child: SparkPlan) extends object Broadcast { private[execution] val executionContext = ExecutionContext.fromExecutorService( - ThreadUtils.newDaemonCachedThreadPool("build-broadcast", 128)) + ThreadUtils.newDaemonCachedThreadPool("broadcast", 128)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index dced52b6a93b8..ccc798e03d99b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.errors.attachTree import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.joins.{HashedRelation, HashedRelationBroadcastMode, HashSemiJoin, HashSetBroadcastMode} import org.apache.spark.util.MutablePair /** @@ -387,6 +388,19 @@ private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[ withCoordinator } + /** + * Create a [[Broadcast]] operator for a given [[BroadcastMode]] and [[SparkPlan]]. + */ + private def createBroadcast(mode: BroadcastMode, plan: SparkPlan): Broadcast = mode match { + case IdentityBroadcastMode => + Broadcast(mode, identity, plan) + case HashSetBroadcastMode(keys) => + Broadcast(mode, HashSemiJoin.buildKeyHashSet(keys, plan, _), plan) + case HashedRelationBroadcastMode(canJoinKeyFitWithinLong, keys) => + Broadcast(mode, HashedRelation(canJoinKeyFitWithinLong, keys, plan, _), plan) + case _ => sys.error(s"Unknown BroadcastMode: $mode") + } + private def ensureDistributionAndOrdering(operator: SparkPlan): SparkPlan = { val requiredChildDistributions: Seq[Distribution] = operator.requiredChildDistribution val requiredChildOrderings: Seq[Seq[SortOrder]] = operator.requiredChildOrdering @@ -398,14 +412,14 @@ private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[ children = children.zip(requiredChildDistributions).map { case (child, distribution) if child.outputPartitioning.satisfies(distribution) => child - case (child, BroadcastDistribution(f1)) => + case (child, BroadcastDistribution(m1)) => child match { // The child is broadcasting the same variable: keep the child. - case Broadcast(f2, _) if f1 == f2 => child + case Broadcast(m2, _, _) if m1 == m2 => child // The child is broadcasting a different variable: replace the child. - case Broadcast(f2, src) => Broadcast(f1, src) + case Broadcast(m2, _, src) => createBroadcast(m1, src) // Create a broadcast on top of the child. - case _ => Broadcast(f1, child) + case _ => createBroadcast(m1, child) } case (child, distribution) => Exchange(createPartitioning(distribution, defaultNumPreShufflePartitions), child) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala index b2e789753d158..bf58092e5667e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.execution import scala.collection.mutable.ArrayBuffer -import org.apache.spark.broadcast import org.apache.spark.rdd.RDD import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.InternalRow @@ -29,7 +28,7 @@ import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.aggregate.TungstenAggregate import org.apache.spark.sql.execution.joins.{BroadcastHashJoin, BuildLeft, BuildRight} -import org.apache.spark.sql.execution.metric.{LongSQLMetric, LongSQLMetricValue, SQLMetric} +import org.apache.spark.sql.execution.metric.LongSQLMetricValue /** * An interface for those physical operators that support codegen. @@ -395,10 +394,10 @@ private[sql] case class CollapseCodegenStages(sqlContext: SQLContext) extends Ru var inputs = ArrayBuffer[SparkPlan]() val combined = plan.transform { // The build side can't be compiled together - case b @ BroadcastHashJoin(_, _, BuildLeft, _, Broadcast(f, left), _) => - b.copy(left = Broadcast(f, apply(left))) - case b @ BroadcastHashJoin(_, _, BuildRight, _, _, Broadcast(f, right)) => - b.copy(right = Broadcast(f, apply(right))) + case b @ BroadcastHashJoin(_, _, BuildLeft, _, bc: Broadcast, _) => + b.copy(left = bc.copy(child = apply(bc.child))) + case b @ BroadcastHashJoin(_, _, BuildRight, _, _, bc: Broadcast) => + b.copy(right = bc.copy(child = apply(bc.child))) case p if !supportCodegen(p) => val input = apply(p) // collapse them recursively inputs += input diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala index bc83b0ca116c0..94225d10b1cac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala @@ -52,19 +52,13 @@ case class BroadcastHashJoin( override def outputPartitioning: Partitioning = streamedPlan.outputPartitioning - override def requiredChildDistribution: Seq[Distribution] = buildSide match { - case BuildLeft => - BroadcastDistribution(buildRelation) :: UnspecifiedDistribution :: Nil - case BuildRight => - UnspecifiedDistribution :: BroadcastDistribution(buildRelation) :: Nil - } - - private[this] val buildRelation: Iterable[InternalRow] => HashedRelation = { input => - // TODO: move this check into HashedRelation - if (canJoinKeyFitWithinLong) { - LongHashedRelation(input.iterator, buildSideKeyGenerator, input.size) - } else { - HashedRelation(input.iterator, buildSideKeyGenerator, input.size) + override def requiredChildDistribution: Seq[Distribution] = { + val mode = HashedRelationBroadcastMode(canJoinKeyFitWithinLong, rewriteKeyExpr(buildKeys)) + buildSide match { + case BuildLeft => + BroadcastDistribution(mode) :: UnspecifiedDistribution :: Nil + case BuildRight => + UnspecifiedDistribution :: BroadcastDistribution(mode) :: Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala index c0fb898960b41..d790dde0cb03b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala @@ -47,17 +47,16 @@ case class BroadcastHashOuterJoin( override private[sql] lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows")) - override def requiredChildDistribution: Seq[Distribution] = joinType match { - case RightOuter => - BroadcastDistribution(buildRelation) :: UnspecifiedDistribution :: Nil - case LeftOuter => - UnspecifiedDistribution :: BroadcastDistribution(buildRelation) :: Nil - case x => - failOnWrongJoinType(x) - } - - private[this] val buildRelation: Iterable[InternalRow] => HashedRelation = { input => - HashedRelation(input.iterator, buildKeyGenerator, input.size) + override def requiredChildDistribution: Seq[Distribution] = { + val mode = HashedRelationBroadcastMode(canJoinKeyFitWithinLong = false, buildKeys) + joinType match { + case RightOuter => + BroadcastDistribution(mode) :: UnspecifiedDistribution :: Nil + case LeftOuter => + UnspecifiedDistribution :: BroadcastDistribution(mode) :: Nil + case x => + failOnWrongJoinType(x) + } } override def outputPartitioning: Partitioning = streamedPlan.outputPartitioning diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala index 2f042483f6f29..9350658f3b089 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala @@ -40,17 +40,12 @@ case class BroadcastLeftSemiJoinHash( "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows")) override def requiredChildDistribution: Seq[Distribution] = { - UnspecifiedDistribution :: BroadcastDistribution(buildRelation) :: Nil - } - - private[this] val buildRelation: Iterable[InternalRow] => Any = { - if (condition.isEmpty) { - (input: Iterable[InternalRow]) => - buildKeyHashSet(input.toIterator) + val mode = if (condition.isEmpty) { + HashSetBroadcastMode(rightKeys) } else { - (input: Iterable[InternalRow]) => - HashedRelation(input.toIterator, rightKeyGenerator, input.size) + HashedRelationBroadcastMode(canJoinKeyFitWithinLong = false, rightKeys) } + UnspecifiedDistribution :: BroadcastDistribution(mode) :: Nil } protected override def doExecute(): RDD[InternalRow] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala index e82608494ea7f..e8bd7f69dbab9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala @@ -21,7 +21,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Distribution, Partitioning, UnspecifiedDistribution} +import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.util.collection.{BitSet, CompactBuffer} @@ -45,9 +45,9 @@ case class BroadcastNestedLoopJoin( override def requiredChildDistribution: Seq[Distribution] = buildSide match { case BuildLeft => - BroadcastDistribution(_.toIndexedSeq) :: UnspecifiedDistribution :: Nil + BroadcastDistribution(IdentityBroadcastMode) :: UnspecifiedDistribution :: Nil case BuildRight => - UnspecifiedDistribution :: BroadcastDistribution(_.toIndexedSeq) :: Nil + UnspecifiedDistribution :: BroadcastDistribution(IdentityBroadcastMode) :: Nil } private[this] def genResultProjection: InternalRow => InternalRow = { @@ -79,25 +79,25 @@ case class BroadcastNestedLoopJoin( protected override def doExecute(): RDD[InternalRow] = { val numOutputRows = longMetric("numOutputRows") - val broadcastedRelation = broadcast.executeBroadcast[IndexedSeq[InternalRow]]() + val broadcastedRelation = broadcast.executeBroadcast[Array[InternalRow]]() /** All rows that either match both-way, or rows from streamed joined with nulls. */ val matchesOrStreamedRowsWithNulls = streamed.execute().mapPartitions { streamedIter => + val relation = broadcastedRelation.value + val matchedRows = new CompactBuffer[InternalRow] - val includedBroadcastTuples = new BitSet(broadcastedRelation.value.size) + val includedBroadcastTuples = new BitSet(relation.length) val joinedRow = new JoinedRow val leftNulls = new GenericMutableRow(left.output.size) val rightNulls = new GenericMutableRow(right.output.size) val resultProj = genResultProjection - val relation = broadcastedRelation.value - streamedIter.foreach { streamedRow => var i = 0 var streamRowMatched = false - while (i < relation.size) { + while (i < relation.length) { val broadcastedRow = relation(i) buildSide match { case BuildRight if boundCondition(joinedRow(streamedRow, broadcastedRow)) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashSemiJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashSemiJoin.scala index 0220e0b8a7c2b..7169c88122450 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashSemiJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashSemiJoin.scala @@ -19,7 +19,8 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.catalyst.plans.physical.BroadcastMode +import org.apache.spark.sql.execution.{Broadcast, SparkPlan} import org.apache.spark.sql.execution.metric.LongSQLMetric @@ -44,22 +45,7 @@ trait HashSemiJoin { protected def buildKeyHashSet( buildIter: Iterator[InternalRow]): java.util.Set[InternalRow] = { - val hashSet = new java.util.HashSet[InternalRow]() - - // Create a Hash set of buildKeys - val rightKey = rightKeyGenerator - while (buildIter.hasNext) { - val currentRow = buildIter.next() - val rowKey = rightKey(currentRow) - if (!rowKey.anyNull) { - val keyExists = hashSet.contains(rowKey) - if (!keyExists) { - hashSet.add(rowKey.copy()) - } - } - } - - hashSet + HashSemiJoin.buildKeyHashSet(rightKeys, right.output, buildIter) } protected def hashSemiJoin( @@ -92,3 +78,36 @@ trait HashSemiJoin { } } } + +private[execution] object HashSemiJoin { + def buildKeyHashSet( + keys: Seq[Expression], + plan: SparkPlan, + rows: Array[InternalRow]): java.util.HashSet[InternalRow] = { + buildKeyHashSet(keys, plan.output, rows.iterator) + } + + def buildKeyHashSet( + keys: Seq[Expression], + attributes: Seq[Attribute], + rows: Iterator[InternalRow]): java.util.HashSet[InternalRow] = { + val hashSet = new java.util.HashSet[InternalRow]() + + // Create a Hash set of buildKeys + val key = UnsafeProjection.create(keys, attributes) + while (rows.hasNext) { + val currentRow = rows.next() + val rowKey = key(currentRow) + if (!rowKey.anyNull) { + val keyExists = hashSet.contains(rowKey) + if (!keyExists) { + hashSet.add(rowKey.copy()) + } + } + } + hashSet + } +} + +/** HashSetBroadcastMode requires that the input rows are broadcasted as a set. */ +private[execution] case class HashSetBroadcastMode(keys: Seq[Expression]) extends BroadcastMode diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index 0978570d429f9..c862053216e03 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -25,12 +25,11 @@ import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.memory.{StaticMemoryManager, TaskMemoryManager} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.execution.SparkSqlSerializer +import org.apache.spark.sql.catalyst.plans.physical.BroadcastMode +import org.apache.spark.sql.execution.{SparkPlan, SparkSqlSerializer} import org.apache.spark.sql.execution.local.LocalNode -import org.apache.spark.sql.execution.metric.{LongSQLMetric, SQLMetrics} import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.map.BytesToBytesMap -import org.apache.spark.unsafe.memory.MemoryLocation import org.apache.spark.util.{KnownSizeEstimation, SizeEstimator, Utils} import org.apache.spark.util.collection.CompactBuffer @@ -210,6 +209,19 @@ private[execution] object HashedRelation { new GeneralHashedRelation(hashTable) } } + + def apply( + canJoinKeyFitWithinLong: Boolean, + keys: Seq[Expression], + child: SparkPlan, + rows: Array[InternalRow]): HashedRelation = { + val generator = UnsafeProjection.create(keys, child.output) + if (canJoinKeyFitWithinLong) { + LongHashedRelation(rows.iterator, generator, rows.length) + } else { + HashedRelation(rows.iterator, generator, rows.length) + } + } } /** @@ -675,3 +687,9 @@ private[joins] object LongHashedRelation { } } } + +/** The HashedRelationBroadcastMode requires that rows are broadcasted as a HashedRelation. */ +private[execution] case class HashedRelationBroadcastMode( + canJoinKeyFitWithinLong: Boolean, + keys: Seq[Expression]) extends BroadcastMode + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala index 6f8b247dec3a5..ea20c086abad0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala @@ -20,8 +20,8 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Distribution, Partitioning, UnspecifiedDistribution} -import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} +import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.SQLMetrics /** @@ -37,7 +37,7 @@ case class LeftSemiJoinBNL( "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows")) override def requiredChildDistribution: Seq[Distribution] = { - UnspecifiedDistribution :: BroadcastDistribution(_.toIndexedSeq) :: Nil + UnspecifiedDistribution :: BroadcastDistribution(IdentityBroadcastMode) :: Nil } override def outputPartitioning: Partitioning = left.outputPartitioning @@ -50,7 +50,7 @@ case class LeftSemiJoinBNL( protected override def doExecute(): RDD[InternalRow] = { val numOutputRows = longMetric("numOutputRows") - val broadcastedRelation = right.executeBroadcast[IndexedSeq[InternalRow]]() + val broadcastedRelation = right.executeBroadcast[Array[InternalRow]]() left.execute().mapPartitions { streamedIter => val joinedRow = new JoinedRow @@ -60,7 +60,7 @@ case class LeftSemiJoinBNL( var i = 0 var matched = false - while (i < relation.size && !matched) { + while (i < relation.length && !matched) { if (boundCondition(joinedRow(streamedRow, relation(i)))) { matched = true } From 681f34718f830e69a7a370a0b0bf465283a738e2 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Mon, 15 Feb 2016 18:34:37 +0100 Subject: [PATCH 09/19] Move transform into BroadcastMode --- .../plans/physical/partitioning.scala | 6 +++-- .../spark/sql/execution/Broadcast.scala | 3 +-- .../apache/spark/sql/execution/Exchange.scala | 19 +++----------- .../execution/joins/BroadcastHashJoin.scala | 5 +++- .../joins/BroadcastHashOuterJoin.scala | 5 +++- .../joins/BroadcastLeftSemiJoinHash.scala | 4 +-- .../sql/execution/joins/HashSemiJoin.scala | 15 ++++++----- .../sql/execution/joins/HashedRelation.scala | 25 ++++++++----------- 8 files changed, 36 insertions(+), 46 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index a632385c238f2..54cb1be7c3701 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -80,12 +80,14 @@ case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { * Marker trait to identify the shape in which tuples are broadcasted. Typical examples of this are * identity (tuples remain unchanged) or hashed (tuples are converted into some hash index). */ -trait BroadcastMode +trait BroadcastMode extends (Array[InternalRow] => Any) /** * IdentityBroadcastMode requires that rows are broadcasted in their original form. */ -case object IdentityBroadcastMode extends BroadcastMode +case object IdentityBroadcastMode extends BroadcastMode { + def apply(rows: Array[InternalRow]): Array[InternalRow] = rows +} /** * Represents data where tuples are broadcasted to every node. It is quite common that the diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala index a875e9fd06b84..bb232f2a627b3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala @@ -31,7 +31,6 @@ import org.apache.spark.util.ThreadUtils */ case class Broadcast( mode: BroadcastMode, - transform: Array[InternalRow] => Any, child: SparkPlan) extends UnaryNode { override def output: Seq[Attribute] = child.output @@ -60,7 +59,7 @@ case class Broadcast( }.collect() // Construct and broadcast the relation. - sparkContext.broadcast(transform(input)) + sparkContext.broadcast(mode(input)) } }(Broadcast.executionContext) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index ccc798e03d99b..69e8493e3b1a7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -388,19 +388,6 @@ private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[ withCoordinator } - /** - * Create a [[Broadcast]] operator for a given [[BroadcastMode]] and [[SparkPlan]]. - */ - private def createBroadcast(mode: BroadcastMode, plan: SparkPlan): Broadcast = mode match { - case IdentityBroadcastMode => - Broadcast(mode, identity, plan) - case HashSetBroadcastMode(keys) => - Broadcast(mode, HashSemiJoin.buildKeyHashSet(keys, plan, _), plan) - case HashedRelationBroadcastMode(canJoinKeyFitWithinLong, keys) => - Broadcast(mode, HashedRelation(canJoinKeyFitWithinLong, keys, plan, _), plan) - case _ => sys.error(s"Unknown BroadcastMode: $mode") - } - private def ensureDistributionAndOrdering(operator: SparkPlan): SparkPlan = { val requiredChildDistributions: Seq[Distribution] = operator.requiredChildDistribution val requiredChildOrderings: Seq[Seq[SortOrder]] = operator.requiredChildOrdering @@ -415,11 +402,11 @@ private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[ case (child, BroadcastDistribution(m1)) => child match { // The child is broadcasting the same variable: keep the child. - case Broadcast(m2, _, _) if m1 == m2 => child + case Broadcast(m2, _) if m1 == m2 => child // The child is broadcasting a different variable: replace the child. - case Broadcast(m2, _, src) => createBroadcast(m1, src) + case Broadcast(m2, src) => Broadcast(m1, src) // Create a broadcast on top of the child. - case _ => createBroadcast(m1, child) + case _ => Broadcast(m1, child) } case (child, distribution) => Exchange(createPartitioning(distribution, defaultNumPreShufflePartitions), child) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala index 94225d10b1cac..2e72593426358 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala @@ -53,7 +53,10 @@ case class BroadcastHashJoin( override def outputPartitioning: Partitioning = streamedPlan.outputPartitioning override def requiredChildDistribution: Seq[Distribution] = { - val mode = HashedRelationBroadcastMode(canJoinKeyFitWithinLong, rewriteKeyExpr(buildKeys)) + val mode = HashedRelationBroadcastMode( + canJoinKeyFitWithinLong, + rewriteKeyExpr(buildKeys), + buildPlan.output) buildSide match { case BuildLeft => BroadcastDistribution(mode) :: UnspecifiedDistribution :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala index d790dde0cb03b..bd052683b863f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala @@ -48,7 +48,10 @@ case class BroadcastHashOuterJoin( "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows")) override def requiredChildDistribution: Seq[Distribution] = { - val mode = HashedRelationBroadcastMode(canJoinKeyFitWithinLong = false, buildKeys) + val mode = HashedRelationBroadcastMode( + canJoinKeyFitWithinLong = false, + buildKeys, + buildPlan.output) joinType match { case RightOuter => BroadcastDistribution(mode) :: UnspecifiedDistribution :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala index 9350658f3b089..1f99fbedde411 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala @@ -41,9 +41,9 @@ case class BroadcastLeftSemiJoinHash( override def requiredChildDistribution: Seq[Distribution] = { val mode = if (condition.isEmpty) { - HashSetBroadcastMode(rightKeys) + HashSetBroadcastMode(rightKeys, right.output) } else { - HashedRelationBroadcastMode(canJoinKeyFitWithinLong = false, rightKeys) + HashedRelationBroadcastMode(canJoinKeyFitWithinLong = false, rightKeys, right.output) } UnspecifiedDistribution :: BroadcastDistribution(mode) :: Nil } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashSemiJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashSemiJoin.scala index 7169c88122450..49cce6a08fc17 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashSemiJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashSemiJoin.scala @@ -80,13 +80,6 @@ trait HashSemiJoin { } private[execution] object HashSemiJoin { - def buildKeyHashSet( - keys: Seq[Expression], - plan: SparkPlan, - rows: Array[InternalRow]): java.util.HashSet[InternalRow] = { - buildKeyHashSet(keys, plan.output, rows.iterator) - } - def buildKeyHashSet( keys: Seq[Expression], attributes: Seq[Attribute], @@ -110,4 +103,10 @@ private[execution] object HashSemiJoin { } /** HashSetBroadcastMode requires that the input rows are broadcasted as a set. */ -private[execution] case class HashSetBroadcastMode(keys: Seq[Expression]) extends BroadcastMode +private[execution] case class HashSetBroadcastMode( + keys: Seq[Expression], + attributes: Seq[Attribute]) extends BroadcastMode { + def apply(rows: Array[InternalRow]): java.util.HashSet[InternalRow] = { + HashSemiJoin.buildKeyHashSet(keys, attributes, rows.iterator) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index c862053216e03..357891286b881 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -209,19 +209,6 @@ private[execution] object HashedRelation { new GeneralHashedRelation(hashTable) } } - - def apply( - canJoinKeyFitWithinLong: Boolean, - keys: Seq[Expression], - child: SparkPlan, - rows: Array[InternalRow]): HashedRelation = { - val generator = UnsafeProjection.create(keys, child.output) - if (canJoinKeyFitWithinLong) { - LongHashedRelation(rows.iterator, generator, rows.length) - } else { - HashedRelation(rows.iterator, generator, rows.length) - } - } } /** @@ -691,5 +678,15 @@ private[joins] object LongHashedRelation { /** The HashedRelationBroadcastMode requires that rows are broadcasted as a HashedRelation. */ private[execution] case class HashedRelationBroadcastMode( canJoinKeyFitWithinLong: Boolean, - keys: Seq[Expression]) extends BroadcastMode + keys: Seq[Expression], + attributes: Seq[Attribute]) extends BroadcastMode { + def apply(rows: Array[InternalRow]): HashedRelation = { + val generator = UnsafeProjection.create(keys, attributes) + if (canJoinKeyFitWithinLong) { + LongHashedRelation(rows.iterator, generator, rows.length) + } else { + HashedRelation(rows.iterator, generator, rows.length) + } + } +} From 7db240a93ea4fce858c36c80c7d5c82784190dbc Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Mon, 15 Feb 2016 18:58:31 +0100 Subject: [PATCH 10/19] Clean-up --- .../scala/org/apache/spark/sql/execution/Exchange.scala | 1 - .../apache/spark/sql/execution/python/EvaluatePython.scala | 1 + .../spark/sql/execution/joins/BroadcastJoinSuite.scala | 6 +++--- .../apache/spark/sql/execution/joins/InnerJoinSuite.scala | 4 ---- 4 files changed, 4 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 69e8493e3b1a7..37cf12aea9429 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -30,7 +30,6 @@ import org.apache.spark.sql.catalyst.errors.attachTree import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.joins.{HashedRelation, HashedRelationBroadcastMode, HashSemiJoin, HashSetBroadcastMode} import org.apache.spark.util.MutablePair /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala index 70e9f862a34ad..8c46516594a2d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala @@ -49,6 +49,7 @@ case class EvaluatePython( override def references: AttributeSet = udf.references } + object EvaluatePython { def apply(udf: PythonUDF, child: LogicalPlan): EvaluatePython = new EvaluatePython(udf, child, AttributeReference("pythonUDF", udf.dataType)()) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala index 82520347816b8..e18f2a188d71d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala @@ -22,7 +22,8 @@ import scala.reflect.ClassTag import org.scalatest.BeforeAndAfterAll import org.apache.spark.{AccumulatorSuite, SparkConf, SparkContext} -import org.apache.spark.sql.{QueryTest, SQLConf, SQLContext} +import org.apache.spark.sql.{QueryTest, SQLContext} +import org.apache.spark.sql.execution.EnsureRequirements import org.apache.spark.sql.functions._ /** @@ -43,7 +44,6 @@ class BroadcastJoinSuite extends QueryTest with BeforeAndAfterAll { val conf = new SparkConf() .setMaster("local-cluster[2,1,1024]") .setAppName("testing") - .set("spark.sql.codegen.wholeStage", "false") val sc = new SparkContext(conf) sqlContext = new SQLContext(sc) } @@ -63,7 +63,7 @@ class BroadcastJoinSuite extends QueryTest with BeforeAndAfterAll { // Comparison at the end is for broadcast left semi join val joinExpression = df1("key") === df2("key") && df1("value") > df2("value") val df3 = df1.join(broadcast(df2), joinExpression, joinType) - val plan = df3.queryExecution.executedPlan + val plan = EnsureRequirements(sqlContext).apply(df3.queryExecution.sparkPlan) assert(plan.collect { case p: T => p }.size === 1) plan.executeCollect() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala index d34859ec401cf..f1f936fdd2b07 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala @@ -29,10 +29,6 @@ import org.apache.spark.sql.types.{IntegerType, StringType, StructType} class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { import testImplicits.localSeqToDataFrameHolder - override def beforeAll(): Unit = { - super.beforeAll() - } - private lazy val myUpperCaseData = sqlContext.createDataFrame( sparkContext.parallelize(Seq( Row(1, "A"), From 3ad839d1c7df75299c766a31b6bdcdeb6897cff8 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Tue, 16 Feb 2016 22:52:37 +0100 Subject: [PATCH 11/19] Code Review. --- .../plans/physical/partitioning.scala | 24 +++++++++++++++++-- .../spark/sql/execution/Broadcast.scala | 8 ++++--- .../apache/spark/sql/execution/Exchange.scala | 11 ++------- .../spark/sql/execution/SparkPlan.scala | 2 +- .../sql/execution/WholeStageCodegen.scala | 16 +++++++------ .../sql/execution/joins/HashSemiJoin.scala | 3 ++- .../sql/execution/joins/HashedRelation.scala | 3 ++- 7 files changed, 43 insertions(+), 24 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index 54cb1be7c3701..d021422aeaf78 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -80,13 +80,15 @@ case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { * Marker trait to identify the shape in which tuples are broadcasted. Typical examples of this are * identity (tuples remain unchanged) or hashed (tuples are converted into some hash index). */ -trait BroadcastMode extends (Array[InternalRow] => Any) +trait BroadcastMode { + def transform(rows: Array[InternalRow]): Any +} /** * IdentityBroadcastMode requires that rows are broadcasted in their original form. */ case object IdentityBroadcastMode extends BroadcastMode { - def apply(rows: Array[InternalRow]): Array[InternalRow] = rows + override def transform(rows: Array[InternalRow]): Array[InternalRow] = rows } /** @@ -374,3 +376,21 @@ case class PartitioningCollection(partitionings: Seq[Partitioning]) partitionings.map(_.toString).mkString("(", " or ", ")") } } + +/** + * Represents a partitioning where rows are collected, transformed and broadcasted to each + * node in the cluster. + */ +case class BroadcastPartitioning(mode: BroadcastMode) extends Partitioning { + override val numPartitions: Int = 1 + + override def satisfies(required: Distribution): Boolean = required match { + case BroadcastDistribution(m) if m == mode => true + case _ => false + } + + override def compatibleWith(other: Partitioning): Boolean = other match { + case BroadcastPartitioning(m) if m == mode => true + case _ => false + } +} \ No newline at end of file diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala index bb232f2a627b3..ba03531f6b72b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala @@ -23,7 +23,7 @@ import org.apache.spark.broadcast import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.physical.BroadcastMode +import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, BroadcastPartitioning, Partitioning} import org.apache.spark.util.ThreadUtils /** @@ -35,6 +35,8 @@ case class Broadcast( override def output: Seq[Attribute] = child.output + override def outputPartitioning: Partitioning = BroadcastPartitioning(mode) + val timeout: Duration = { val timeoutValue = sqlContext.conf.broadcastTimeout if (timeoutValue < 0) { @@ -59,7 +61,7 @@ case class Broadcast( }.collect() // Construct and broadcast the relation. - sparkContext.broadcast(mode(input)) + sparkContext.broadcast(mode.transform(input)) } }(Broadcast.executionContext) } @@ -70,7 +72,7 @@ case class Broadcast( } override protected def doExecute(): RDD[InternalRow] = { - child.execute() // TODO throw an Exception here? + throw new UnsupportedOperationException("Broadcast does not support the execute() code path.") } override protected[sql] def doExecuteBroadcast[T](): broadcast.Broadcast[T] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 37cf12aea9429..e91a82e0ddbb6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -398,15 +398,8 @@ private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[ children = children.zip(requiredChildDistributions).map { case (child, distribution) if child.outputPartitioning.satisfies(distribution) => child - case (child, BroadcastDistribution(m1)) => - child match { - // The child is broadcasting the same variable: keep the child. - case Broadcast(m2, _) if m1 == m2 => child - // The child is broadcasting a different variable: replace the child. - case Broadcast(m2, src) => Broadcast(m1, src) - // Create a broadcast on top of the child. - case _ => Broadcast(m1, child) - } + case (child, BroadcastDistribution(mode)) => + Broadcast(mode, child) case (child, distribution) => Exchange(createPartitioning(distribution, defaultNumPreShufflePartitions), child) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index acdc9dd8ceee9..0d73be7fa6a00 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -163,7 +163,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ * Produces the result of the query as a broadcast variable. */ protected[sql] def doExecuteBroadcast[T](): broadcast.Broadcast[T] = { - throw new NotImplementedError(s"$nodeName does not implement doExecuteBroadcast") + throw new UnsupportedOperationException(s"$nodeName does not implement doExecuteBroadcast") } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala index bf58092e5667e..9adc2b228a98a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution import scala.collection.mutable.ArrayBuffer +import org.apache.spark.broadcast import org.apache.spark.rdd.RDD import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.InternalRow @@ -170,6 +171,10 @@ case class InputAdapter(child: SparkPlan) extends LeafNode with CodegenSupport { child.execute() } + override def doExecuteBroadcast[T](): broadcast.Broadcast[T] = { + child.doExecuteBroadcast() + } + override def supportCodegen: Boolean = false override def upstream(): RDD[InternalRow] = { @@ -381,9 +386,6 @@ private[sql] case class CollapseCodegenStages(sqlContext: SQLContext) extends Ru // the generated code will be huge if there are too many columns val haveManyColumns = plan.output.length > 200 !willFallback && !haveManyColumns - // Collapse a broadcast into the stage - it should not contain any code that can be - // codegenerated. - case _: Broadcast => true case _ => false } @@ -394,10 +396,10 @@ private[sql] case class CollapseCodegenStages(sqlContext: SQLContext) extends Ru var inputs = ArrayBuffer[SparkPlan]() val combined = plan.transform { // The build side can't be compiled together - case b @ BroadcastHashJoin(_, _, BuildLeft, _, bc: Broadcast, _) => - b.copy(left = bc.copy(child = apply(bc.child))) - case b @ BroadcastHashJoin(_, _, BuildRight, _, _, bc: Broadcast) => - b.copy(right = bc.copy(child = apply(bc.child))) + case b @ BroadcastHashJoin(_, _, BuildLeft, _, left, right) => + b.copy(left = apply(left)) + case b @ BroadcastHashJoin(_, _, BuildRight, _, left, right) => + b.copy(right = apply(right)) case p if !supportCodegen(p) => val input = apply(p) // collapse them recursively inputs += input diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashSemiJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashSemiJoin.scala index 49cce6a08fc17..5ae5d98541ab5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashSemiJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashSemiJoin.scala @@ -106,7 +106,8 @@ private[execution] object HashSemiJoin { private[execution] case class HashSetBroadcastMode( keys: Seq[Expression], attributes: Seq[Attribute]) extends BroadcastMode { - def apply(rows: Array[InternalRow]): java.util.HashSet[InternalRow] = { + + override def transform(rows: Array[InternalRow]): java.util.HashSet[InternalRow] = { HashSemiJoin.buildKeyHashSet(keys, attributes, rows.iterator) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index 357891286b881..606269bf25ab3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -680,7 +680,8 @@ private[execution] case class HashedRelationBroadcastMode( canJoinKeyFitWithinLong: Boolean, keys: Seq[Expression], attributes: Seq[Attribute]) extends BroadcastMode { - def apply(rows: Array[InternalRow]): HashedRelation = { + + def transform(rows: Array[InternalRow]): HashedRelation = { val generator = UnsafeProjection.create(keys, attributes) if (canJoinKeyFitWithinLong) { LongHashedRelation(rows.iterator, generator, rows.length) From 11167680cb5efb06dae5504f35eb0b7a5c20419d Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Tue, 16 Feb 2016 23:00:30 +0100 Subject: [PATCH 12/19] No newline at EOF :( --- .../apache/spark/sql/catalyst/plans/physical/partitioning.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index d021422aeaf78..61dc0fef4c4fc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -393,4 +393,4 @@ case class BroadcastPartitioning(mode: BroadcastMode) extends Partitioning { case BroadcastPartitioning(m) if m == mode => true case _ => false } -} \ No newline at end of file +} From a5501cf73c0620d07e2bda9144723314e7b856ab Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Wed, 17 Feb 2016 07:45:06 +0100 Subject: [PATCH 13/19] Rename exchanges and merge Broadcast.scala into exchange.scala. --- .../spark/sql/execution/Broadcast.scala | 87 --------------- .../sql/execution/ExchangeCoordinator.scala | 43 ++++---- .../spark/sql/execution/SparkStrategies.scala | 4 +- .../{Exchange.scala => exchange.scala} | 104 ++++++++++++++---- .../sql/execution/joins/HashSemiJoin.scala | 2 +- .../apache/spark/sql/execution/limit.scala | 6 +- .../apache/spark/sql/CachedTableSuite.scala | 4 +- .../org/apache/spark/sql/DataFrameSuite.scala | 4 +- .../execution/ExchangeCoordinatorSuite.scala | 20 ++-- .../spark/sql/execution/ExchangeSuite.scala | 2 +- .../spark/sql/execution/PlannerSuite.scala | 20 ++-- .../spark/sql/sources/BucketedReadSuite.scala | 12 +- 12 files changed, 145 insertions(+), 163 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala rename sql/core/src/main/scala/org/apache/spark/sql/execution/{Exchange.scala => exchange.scala} (86%) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala deleted file mode 100644 index ba03531f6b72b..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Broadcast.scala +++ /dev/null @@ -1,87 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.sql.execution - -import scala.concurrent._ -import scala.concurrent.duration._ - -import org.apache.spark.broadcast -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, BroadcastPartitioning, Partitioning} -import org.apache.spark.util.ThreadUtils - -/** - * A broadcast collects, transforms and finally broadcasts the result of a transformed SparkPlan. - */ -case class Broadcast( - mode: BroadcastMode, - child: SparkPlan) extends UnaryNode { - - override def output: Seq[Attribute] = child.output - - override def outputPartitioning: Partitioning = BroadcastPartitioning(mode) - - val timeout: Duration = { - val timeoutValue = sqlContext.conf.broadcastTimeout - if (timeoutValue < 0) { - Duration.Inf - } else { - timeoutValue.seconds - } - } - - @transient - private lazy val relationFuture: Future[broadcast.Broadcast[Any]] = { - // broadcastFuture is used in "doExecute". Therefore we can get the execution id correctly here. - val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - Future { - // This will run in another thread. Set the execution id so that we can connect these jobs - // with the correct execution. - SQLExecution.withExecutionId(sparkContext, executionId) { - // Note that we use .execute().collect() because we don't want to convert data to Scala - // types - val input: Array[InternalRow] = child.execute().map { row => - row.copy() - }.collect() - - // Construct and broadcast the relation. - sparkContext.broadcast(mode.transform(input)) - } - }(Broadcast.executionContext) - } - - override protected def doPrepare(): Unit = { - // Materialize the future. - relationFuture - } - - override protected def doExecute(): RDD[InternalRow] = { - throw new UnsupportedOperationException("Broadcast does not support the execute() code path.") - } - - override protected[sql] def doExecuteBroadcast[T](): broadcast.Broadcast[T] = { - val result = Await.result(relationFuture, timeout) - result.asInstanceOf[broadcast.Broadcast[T]] - } -} - -object Broadcast { - private[execution] val executionContext = ExecutionContext.fromExecutorService( - ThreadUtils.newDaemonCachedThreadPool("broadcast", 128)) -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala index 07015e5a5aaef..b096074a2fd1a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala @@ -33,9 +33,9 @@ import org.apache.spark.sql.catalyst.InternalRow * * A coordinator is constructed with three parameters, `numExchanges`, * `targetPostShuffleInputSize`, and `minNumPostShufflePartitions`. - * - `numExchanges` is used to indicated that how many [[Exchange]]s that will be registered to - * this coordinator. So, when we start to do any actual work, we have a way to make sure that - * we have got expected number of [[Exchange]]s. + * - `numExchanges` is used to indicated that how many [[ShuffleExchange]]s that will be registered + * to this coordinator. So, when we start to do any actual work, we have a way to make sure that + * we have got expected number of [[ShuffleExchange]]s. * - `targetPostShuffleInputSize` is the targeted size of a post-shuffle partition's * input data size. With this parameter, we can estimate the number of post-shuffle partitions. * This parameter is configured through @@ -45,26 +45,27 @@ import org.apache.spark.sql.catalyst.InternalRow * partitions. * * The workflow of this coordinator is described as follows: - * - Before the execution of a [[SparkPlan]], for an [[Exchange]] operator, + * - Before the execution of a [[SparkPlan]], for an [[ShuffleExchange]] operator, * if an [[ExchangeCoordinator]] is assigned to it, it registers itself to this coordinator. * This happens in the `doPrepare` method. - * - Once we start to execute a physical plan, an [[Exchange]] registered to this coordinator will - * call `postShuffleRDD` to get its corresponding post-shuffle [[ShuffledRowRDD]]. - * If this coordinator has made the decision on how to shuffle data, this [[Exchange]] will - * immediately get its corresponding post-shuffle [[ShuffledRowRDD]]. + * - Once we start to execute a physical plan, an [[ShuffleExchange]] registered to this + * coordinator will call `postShuffleRDD` to get its corresponding post-shuffle + * [[ShuffledRowRDD]]. + * If this coordinator has made the decision on how to shuffle data, this [[ShuffleExchange]] + * will immediately get its corresponding post-shuffle [[ShuffledRowRDD]]. * - If this coordinator has not made the decision on how to shuffle data, it will ask those - * registered [[Exchange]]s to submit their pre-shuffle stages. Then, based on the the size - * statistics of pre-shuffle partitions, this coordinator will determine the number of + * registered [[ShuffleExchange]]s to submit their pre-shuffle stages. Then, based on the the + * size statistics of pre-shuffle partitions, this coordinator will determine the number of * post-shuffle partitions and pack multiple pre-shuffle partitions with continuous indices * to a single post-shuffle partition whenever necessary. * - Finally, this coordinator will create post-shuffle [[ShuffledRowRDD]]s for all registered - * [[Exchange]]s. So, when an [[Exchange]] calls `postShuffleRDD`, this coordinator can - * lookup the corresponding [[RDD]]. + * [[ShuffleExchange]]s. So, when an [[ShuffleExchange]] calls `postShuffleRDD`, this coordinator + * can lookup the corresponding [[RDD]]. * * The strategy used to determine the number of post-shuffle partitions is described as follows. * To determine the number of post-shuffle partitions, we have a target input size for a * post-shuffle partition. Once we have size statistics of pre-shuffle partitions from stages - * corresponding to the registered [[Exchange]]s, we will do a pass of those statistics and + * corresponding to the registered [[ShuffleExchange]]s, we will do a pass of those statistics and * pack pre-shuffle partitions with continuous indices to a single post-shuffle partition until * the size of a post-shuffle partition is equal or greater than the target size. * For example, we have two stages with the following pre-shuffle partition size statistics: @@ -83,11 +84,11 @@ private[sql] class ExchangeCoordinator( extends Logging { // The registered Exchange operators. - private[this] val exchanges = ArrayBuffer[Exchange]() + private[this] val exchanges = ArrayBuffer[ShuffleExchange]() // This map is used to lookup the post-shuffle ShuffledRowRDD for an Exchange operator. - private[this] val postShuffleRDDs: JMap[Exchange, ShuffledRowRDD] = - new JHashMap[Exchange, ShuffledRowRDD](numExchanges) + private[this] val postShuffleRDDs: JMap[ShuffleExchange, ShuffledRowRDD] = + new JHashMap[ShuffleExchange, ShuffledRowRDD](numExchanges) // A boolean that indicates if this coordinator has made decision on how to shuffle data. // This variable will only be updated by doEstimationIfNecessary, which is protected by @@ -95,11 +96,11 @@ private[sql] class ExchangeCoordinator( @volatile private[this] var estimated: Boolean = false /** - * Registers an [[Exchange]] operator to this coordinator. This method is only allowed to be - * called in the `doPrepare` method of an [[Exchange]] operator. + * Registers an [[ShuffleExchange]] operator to this coordinator. This method is only allowed to + * be called in the `doPrepare` method of an [[ShuffleExchange]] operator. */ @GuardedBy("this") - def registerExchange(exchange: Exchange): Unit = synchronized { + def registerExchange(exchange: ShuffleExchange): Unit = synchronized { exchanges += exchange } @@ -199,7 +200,7 @@ private[sql] class ExchangeCoordinator( // Make sure we have the expected number of registered Exchange operators. assert(exchanges.length == numExchanges) - val newPostShuffleRDDs = new JHashMap[Exchange, ShuffledRowRDD](numExchanges) + val newPostShuffleRDDs = new JHashMap[ShuffleExchange, ShuffledRowRDD](numExchanges) // Submit all map stages val shuffleDependencies = ArrayBuffer[ShuffleDependency[Int, InternalRow, InternalRow]]() @@ -254,7 +255,7 @@ private[sql] class ExchangeCoordinator( } } - def postShuffleRDD(exchange: Exchange): ShuffledRowRDD = { + def postShuffleRDD(exchange: ShuffleExchange): ShuffledRowRDD = { doEstimationIfNecessary() if (!postShuffleRDDs.containsKey(exchange)) { 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 73fd22b38e1d6..a75c956fe6624 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 @@ -328,7 +328,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.Repartition(numPartitions, shuffle, child) => if (shuffle) { - execution.Exchange(RoundRobinPartitioning(numPartitions), planLater(child)) :: Nil + execution.ShuffleExchange(RoundRobinPartitioning(numPartitions), planLater(child)) :: Nil } else { execution.Coalesce(numPartitions, planLater(child)) :: Nil } @@ -367,7 +367,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case r @ logical.Range(start, end, step, numSlices, output) => execution.Range(start, step, numSlices, r.numElements, output) :: Nil case logical.RepartitionByExpression(expressions, child, nPartitions) => - execution.Exchange(HashPartitioning( + execution.ShuffleExchange(HashPartitioning( expressions, nPartitions.getOrElse(numPartitions)), planLater(child)) :: Nil case e @ python.EvaluatePython(udf, child, _) => python.BatchPythonEvaluation(udf, e.output, planLater(child)) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange.scala similarity index 86% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/exchange.scala index e91a82e0ddbb6..ee1466dd7eab6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange.scala @@ -19,6 +19,9 @@ package org.apache.spark.sql.execution import java.util.Random +import scala.concurrent.{Await, ExecutionContext, Future} +import scala.concurrent.duration._ + import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer @@ -30,12 +33,13 @@ import org.apache.spark.sql.catalyst.errors.attachTree import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.util.MutablePair +import org.apache.spark.util.{MutablePair, ThreadUtils} + /** * Performs a shuffle that will result in the desired `newPartitioning`. */ -case class Exchange( +case class ShuffleExchange( var newPartitioning: Partitioning, child: SparkPlan, @transient coordinator: Option[ExchangeCoordinator]) extends UnaryNode { @@ -80,7 +84,8 @@ case class Exchange( * the returned ShuffleDependency will be the input of shuffle. */ private[sql] def prepareShuffleDependency(): ShuffleDependency[Int, InternalRow, InternalRow] = { - Exchange.prepareShuffleDependency(child.execute(), child.output, newPartitioning, serializer) + ShuffleExchange.prepareShuffleDependency( + child.execute(), child.output, newPartitioning, serializer) } /** @@ -115,9 +120,9 @@ case class Exchange( } } -object Exchange { - def apply(newPartitioning: Partitioning, child: SparkPlan): Exchange = { - Exchange(newPartitioning, child, coordinator = None: Option[ExchangeCoordinator]) +object ShuffleExchange { + def apply(newPartitioning: Partitioning, child: SparkPlan): ShuffleExchange = { + ShuffleExchange(newPartitioning, child, coordinator = None: Option[ExchangeCoordinator]) } /** @@ -262,11 +267,72 @@ object Exchange { } } +/** + * A [[BroadcastExchange]] collects, transforms and finally broadcasts the result of a transformed + * SparkPlan. + */ +case class BroadcastExchange( + mode: BroadcastMode, + child: SparkPlan) extends UnaryNode { + + override def output: Seq[Attribute] = child.output + + override def outputPartitioning: Partitioning = BroadcastPartitioning(mode) + + val timeout: Duration = { + val timeoutValue = sqlContext.conf.broadcastTimeout + if (timeoutValue < 0) { + Duration.Inf + } else { + timeoutValue.seconds + } + } + + @transient + private lazy val relationFuture: Future[broadcast.Broadcast[Any]] = { + // broadcastFuture is used in "doExecute". Therefore we can get the execution id correctly here. + val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + Future { + // This will run in another thread. Set the execution id so that we can connect these jobs + // with the correct execution. + SQLExecution.withExecutionId(sparkContext, executionId) { + // Note that we use .execute().collect() because we don't want to convert data to Scala + // types + val input: Array[InternalRow] = child.execute().map { row => + row.copy() + }.collect() + + // Construct and broadcast the relation. + sparkContext.broadcast(mode.transform(input)) + } + }(BroadcastExchange.executionContext) + } + + override protected def doPrepare(): Unit = { + // Materialize the future. + relationFuture + } + + override protected def doExecute(): RDD[InternalRow] = { + throw new UnsupportedOperationException("Broadcast does not support the execute() code path.") + } + + override protected[sql] def doExecuteBroadcast[T](): broadcast.Broadcast[T] = { + val result = Await.result(relationFuture, timeout) + result.asInstanceOf[broadcast.Broadcast[T]] + } +} + +object BroadcastExchange { + private[execution] val executionContext = ExecutionContext.fromExecutorService( + ThreadUtils.newDaemonCachedThreadPool("broadcast-exchange", 128)) +} + /** * Ensures that the [[org.apache.spark.sql.catalyst.plans.physical.Partitioning Partitioning]] * of input data meets the * [[org.apache.spark.sql.catalyst.plans.physical.Distribution Distribution]] requirements for - * each operator by inserting [[Exchange]] Operators where required. Also ensure that the + * each operator by inserting [[ShuffleExchange]] Operators where required. Also ensure that the * input partition ordering requirements are met. */ private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[SparkPlan] { @@ -296,17 +362,17 @@ private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[ } /** - * Adds [[ExchangeCoordinator]] to [[Exchange]]s if adaptive query execution is enabled - * and partitioning schemes of these [[Exchange]]s support [[ExchangeCoordinator]]. + * Adds [[ExchangeCoordinator]] to [[ShuffleExchange]]s if adaptive query execution is enabled + * and partitioning schemes of these [[ShuffleExchange]]s support [[ExchangeCoordinator]]. */ private def withExchangeCoordinator( children: Seq[SparkPlan], requiredChildDistributions: Seq[Distribution]): Seq[SparkPlan] = { val supportsCoordinator = - if (children.exists(_.isInstanceOf[Exchange])) { + if (children.exists(_.isInstanceOf[ShuffleExchange])) { // Right now, ExchangeCoordinator only support HashPartitionings. children.forall { - case e @ Exchange(hash: HashPartitioning, _, _) => true + case e @ ShuffleExchange(hash: HashPartitioning, _, _) => true case child => child.outputPartitioning match { case hash: HashPartitioning => true @@ -333,7 +399,7 @@ private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[ targetPostShuffleInputSize, minNumPostShufflePartitions) children.zip(requiredChildDistributions).map { - case (e: Exchange, _) => + case (e: ShuffleExchange, _) => // This child is an Exchange, we need to add the coordinator. e.copy(coordinator = Some(coordinator)) case (child, distribution) => @@ -377,7 +443,7 @@ private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[ val targetPartitioning = createPartitioning(distribution, defaultNumPreShufflePartitions) assert(targetPartitioning.isInstanceOf[HashPartitioning]) - Exchange(targetPartitioning, child, Some(coordinator)) + ShuffleExchange(targetPartitioning, child, Some(coordinator)) } } else { // If we do not need ExchangeCoordinator, the original children are returned. @@ -399,9 +465,9 @@ private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[ case (child, distribution) if child.outputPartitioning.satisfies(distribution) => child case (child, BroadcastDistribution(mode)) => - Broadcast(mode, child) + BroadcastExchange(mode, child) case (child, distribution) => - Exchange(createPartitioning(distribution, defaultNumPreShufflePartitions), child) + ShuffleExchange(createPartitioning(distribution, defaultNumPreShufflePartitions), child) } // If the operator has multiple children and specifies child output distributions (e.g. join), @@ -456,8 +522,8 @@ private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[ child match { // If child is an exchange, we replace it with // a new one having targetPartitioning. - case Exchange(_, c, _) => Exchange(targetPartitioning, c) - case _ => Exchange(targetPartitioning, child) + case ShuffleExchange(_, c, _) => ShuffleExchange(targetPartitioning, c) + case _ => ShuffleExchange(targetPartitioning, child) } } } @@ -492,9 +558,9 @@ private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[ } def apply(plan: SparkPlan): SparkPlan = plan.transformUp { - case operator @ Exchange(partitioning, child, _) => + case operator @ ShuffleExchange(partitioning, child, _) => child.children match { - case Exchange(childPartitioning, baseChild, _)::Nil => + case ShuffleExchange(childPartitioning, baseChild, _)::Nil => if (childPartitioning.guarantees(partitioning)) child else operator case _ => operator } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashSemiJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashSemiJoin.scala index 5ae5d98541ab5..1cb6a00617c5e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashSemiJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashSemiJoin.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.BroadcastMode -import org.apache.spark.sql.execution.{Broadcast, SparkPlan} +import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.metric.LongSQLMetric diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala index 04daf9d0ce2a6..b50c79e4820fb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala @@ -37,7 +37,8 @@ case class CollectLimit(limit: Int, child: SparkPlan) extends UnaryNode { private val serializer: Serializer = new UnsafeRowSerializer(child.output.size) protected override def doExecute(): RDD[InternalRow] = { val shuffled = new ShuffledRowRDD( - Exchange.prepareShuffleDependency(child.execute(), child.output, SinglePartition, serializer)) + ShuffleExchange.prepareShuffleDependency( + child.execute(), child.output, SinglePartition, serializer)) shuffled.mapPartitionsInternal(_.take(limit)) } } @@ -111,7 +112,8 @@ case class TakeOrderedAndProject( } } val shuffled = new ShuffledRowRDD( - Exchange.prepareShuffleDependency(localTopK, child.output, SinglePartition, serializer)) + ShuffleExchange.prepareShuffleDependency( + localTopK, child.output, SinglePartition, serializer)) shuffled.mapPartitions { iter => val topK = org.apache.spark.util.collection.Utils.takeOrdered(iter.map(_.copy()), limit)(ord) if (projectList.isDefined) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index e8d0678989d88..f28632201a11c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -23,8 +23,8 @@ import scala.language.postfixOps import org.scalatest.concurrent.Eventually._ import org.apache.spark.Accumulators -import org.apache.spark.sql.execution.Exchange import org.apache.spark.sql.execution.PhysicalRDD +import org.apache.spark.sql.execution.ShuffleExchange import org.apache.spark.sql.execution.columnar._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} @@ -357,7 +357,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext * Verifies that the plan for `df` contains `expected` number of Exchange operators. */ private def verifyNumExchanges(df: DataFrame, expected: Int): Unit = { - assert(df.queryExecution.executedPlan.collect { case e: Exchange => e }.size == expected) + assert(df.queryExecution.executedPlan.collect { case e: ShuffleExchange => e }.size == expected) } test("A cached table preserves the partitioning and ordering of its cached SparkPlan") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 99ba2e2061258..65e0deada5f6f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -26,7 +26,7 @@ import org.scalatest.Matchers._ import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.plans.logical.{OneRowRelation, Union} -import org.apache.spark.sql.execution.Exchange +import org.apache.spark.sql.execution.ShuffleExchange import org.apache.spark.sql.execution.aggregate.TungstenAggregate import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSQLContext} @@ -1119,7 +1119,7 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { } atFirstAgg = true } - case e: Exchange => atFirstAgg = false + case e: ShuffleExchange => atFirstAgg = false case _ => } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala index 35ff1c40fe6ca..3d4080784b8c1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala @@ -297,13 +297,13 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. val exchanges = agg.queryExecution.executedPlan.collect { - case e: Exchange => e + case e: ShuffleExchange => e } assert(exchanges.length === 1) minNumPostShufflePartitions match { case Some(numPartitions) => exchanges.foreach { - case e: Exchange => + case e: ShuffleExchange => assert(e.coordinator.isDefined) assert(e.outputPartitioning.numPartitions === 3) case o => @@ -311,7 +311,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { case None => exchanges.foreach { - case e: Exchange => + case e: ShuffleExchange => assert(e.coordinator.isDefined) assert(e.outputPartitioning.numPartitions === 2) case o => @@ -348,13 +348,13 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. val exchanges = join.queryExecution.executedPlan.collect { - case e: Exchange => e + case e: ShuffleExchange => e } assert(exchanges.length === 2) minNumPostShufflePartitions match { case Some(numPartitions) => exchanges.foreach { - case e: Exchange => + case e: ShuffleExchange => assert(e.coordinator.isDefined) assert(e.outputPartitioning.numPartitions === 3) case o => @@ -362,7 +362,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { case None => exchanges.foreach { - case e: Exchange => + case e: ShuffleExchange => assert(e.coordinator.isDefined) assert(e.outputPartitioning.numPartitions === 2) case o => @@ -404,13 +404,13 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. val exchanges = join.queryExecution.executedPlan.collect { - case e: Exchange => e + case e: ShuffleExchange => e } assert(exchanges.length === 4) minNumPostShufflePartitions match { case Some(numPartitions) => exchanges.foreach { - case e: Exchange => + case e: ShuffleExchange => assert(e.coordinator.isDefined) assert(e.outputPartitioning.numPartitions === 3) case o => @@ -456,13 +456,13 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. val exchanges = join.queryExecution.executedPlan.collect { - case e: Exchange => e + case e: ShuffleExchange => e } assert(exchanges.length === 3) minNumPostShufflePartitions match { case Some(numPartitions) => exchanges.foreach { - case e: Exchange => + case e: ShuffleExchange => assert(e.coordinator.isDefined) assert(e.outputPartitioning.numPartitions === 3) case o => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala index 87bff3295f5be..c54f094a12a08 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala @@ -28,7 +28,7 @@ class ExchangeSuite extends SparkPlanTest with SharedSQLContext { val input = (1 to 1000).map(Tuple1.apply) checkAnswer( input.toDF(), - plan => Exchange(SinglePartition, plan), + plan => ShuffleExchange(SinglePartition, plan), input.map(Row.fromTuple) ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index 250ce8f86698f..c19a945097e9f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -212,7 +212,7 @@ class PlannerSuite extends SharedSQLContext { | JOIN tiny ON (small.key = tiny.key) """.stripMargin ).queryExecution.executedPlan.collect { - case exchange: Exchange => exchange + case exchange: ShuffleExchange => exchange }.length assert(numExchanges === 5) } @@ -227,7 +227,7 @@ class PlannerSuite extends SharedSQLContext { | JOIN tiny ON (normal.key = tiny.key) """.stripMargin ).queryExecution.executedPlan.collect { - case exchange: Exchange => exchange + case exchange: ShuffleExchange => exchange }.length assert(numExchanges === 5) } @@ -295,7 +295,7 @@ class PlannerSuite extends SharedSQLContext { ) val outputPlan = EnsureRequirements(sqlContext).apply(inputPlan) assertDistributionRequirementsAreSatisfied(outputPlan) - if (outputPlan.collect { case e: Exchange => true }.isEmpty) { + if (outputPlan.collect { case e: ShuffleExchange => true }.isEmpty) { fail(s"Exchange should have been added:\n$outputPlan") } } @@ -333,7 +333,7 @@ class PlannerSuite extends SharedSQLContext { ) val outputPlan = EnsureRequirements(sqlContext).apply(inputPlan) assertDistributionRequirementsAreSatisfied(outputPlan) - if (outputPlan.collect { case e: Exchange => true }.isEmpty) { + if (outputPlan.collect { case e: ShuffleExchange => true }.isEmpty) { fail(s"Exchange should have been added:\n$outputPlan") } } @@ -353,7 +353,7 @@ class PlannerSuite extends SharedSQLContext { ) val outputPlan = EnsureRequirements(sqlContext).apply(inputPlan) assertDistributionRequirementsAreSatisfied(outputPlan) - if (outputPlan.collect { case e: Exchange => true }.nonEmpty) { + if (outputPlan.collect { case e: ShuffleExchange => true }.nonEmpty) { fail(s"Exchange should not have been added:\n$outputPlan") } } @@ -376,7 +376,7 @@ class PlannerSuite extends SharedSQLContext { ) val outputPlan = EnsureRequirements(sqlContext).apply(inputPlan) assertDistributionRequirementsAreSatisfied(outputPlan) - if (outputPlan.collect { case e: Exchange => true }.nonEmpty) { + if (outputPlan.collect { case e: ShuffleExchange => true }.nonEmpty) { fail(s"No Exchanges should have been added:\n$outputPlan") } } @@ -435,7 +435,7 @@ class PlannerSuite extends SharedSQLContext { val finalPartitioning = HashPartitioning(Literal(1) :: Nil, 5) val childPartitioning = HashPartitioning(Literal(2) :: Nil, 5) assert(!childPartitioning.satisfies(distribution)) - val inputPlan = Exchange(finalPartitioning, + val inputPlan = ShuffleExchange(finalPartitioning, DummySparkPlan( children = DummySparkPlan(outputPartitioning = childPartitioning) :: Nil, requiredChildDistribution = Seq(distribution), @@ -444,7 +444,7 @@ class PlannerSuite extends SharedSQLContext { val outputPlan = EnsureRequirements(sqlContext).apply(inputPlan) assertDistributionRequirementsAreSatisfied(outputPlan) - if (outputPlan.collect { case e: Exchange => true }.size == 2) { + if (outputPlan.collect { case e: ShuffleExchange => true }.size == 2) { fail(s"Topmost Exchange should have been eliminated:\n$outputPlan") } } @@ -455,7 +455,7 @@ class PlannerSuite extends SharedSQLContext { val finalPartitioning = HashPartitioning(Literal(1) :: Nil, 8) val childPartitioning = HashPartitioning(Literal(2) :: Nil, 5) assert(!childPartitioning.satisfies(distribution)) - val inputPlan = Exchange(finalPartitioning, + val inputPlan = ShuffleExchange(finalPartitioning, DummySparkPlan( children = DummySparkPlan(outputPartitioning = childPartitioning) :: Nil, requiredChildDistribution = Seq(distribution), @@ -464,7 +464,7 @@ class PlannerSuite extends SharedSQLContext { val outputPlan = EnsureRequirements(sqlContext).apply(inputPlan) assertDistributionRequirementsAreSatisfied(outputPlan) - if (outputPlan.collect { case e: Exchange => true }.size == 1) { + if (outputPlan.collect { case e: ShuffleExchange => true }.size == 1) { fail(s"Topmost Exchange should not have been eliminated:\n$outputPlan") } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index 9ba645626fe72..615be729f84f3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -22,7 +22,7 @@ import java.io.File import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning -import org.apache.spark.sql.execution.{Exchange, PhysicalRDD} +import org.apache.spark.sql.execution.{PhysicalRDD, ShuffleExchange} import org.apache.spark.sql.execution.datasources.{BucketSpec, DataSourceStrategy} import org.apache.spark.sql.execution.joins.SortMergeJoin import org.apache.spark.sql.functions._ @@ -252,8 +252,8 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet assert(joined.queryExecution.executedPlan.isInstanceOf[SortMergeJoin]) val joinOperator = joined.queryExecution.executedPlan.asInstanceOf[SortMergeJoin] - assert(joinOperator.left.find(_.isInstanceOf[Exchange]).isDefined == shuffleLeft) - assert(joinOperator.right.find(_.isInstanceOf[Exchange]).isDefined == shuffleRight) + assert(joinOperator.left.find(_.isInstanceOf[ShuffleExchange]).isDefined == shuffleLeft) + assert(joinOperator.right.find(_.isInstanceOf[ShuffleExchange]).isDefined == shuffleRight) } } } @@ -312,7 +312,7 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet agged.sort("i", "j"), df1.groupBy("i", "j").agg(max("k")).sort("i", "j")) - assert(agged.queryExecution.executedPlan.find(_.isInstanceOf[Exchange]).isEmpty) + assert(agged.queryExecution.executedPlan.find(_.isInstanceOf[ShuffleExchange]).isEmpty) } } @@ -326,7 +326,7 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet agged.sort("i", "j"), df1.groupBy("i", "j").agg(max("k")).sort("i", "j")) - assert(agged.queryExecution.executedPlan.find(_.isInstanceOf[Exchange]).isEmpty) + assert(agged.queryExecution.executedPlan.find(_.isInstanceOf[ShuffleExchange]).isEmpty) } } @@ -339,7 +339,7 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet val agged = hiveContext.table("bucketed_table").groupBy("i").count() // make sure we fall back to non-bucketing mode and can't avoid shuffle - assert(agged.queryExecution.executedPlan.find(_.isInstanceOf[Exchange]).isDefined) + assert(agged.queryExecution.executedPlan.find(_.isInstanceOf[ShuffleExchange]).isDefined) checkAnswer(agged.sort("i"), df1.groupBy("i").count().sort("i")) } } From 9d526504aebcebc59d6eff32cb527da3d03c3097 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Sun, 21 Feb 2016 00:07:59 +0100 Subject: [PATCH 14/19] Revert renaming of variabels in LeftSemiJoinBNL. --- .../sql/execution/joins/LeftSemiJoinBNL.scala | 20 ++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala index ea20c086abad0..875e80f3612c6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala @@ -29,30 +29,32 @@ import org.apache.spark.sql.execution.metric.SQLMetrics * for hash join. */ case class LeftSemiJoinBNL( - left: SparkPlan, right: SparkPlan, condition: Option[Expression]) - extends BinaryNode { - // TODO: Override requiredChildDistribution. + streamed: SparkPlan, broadcast: SparkPlan, condition: Option[Expression]) extends BinaryNode { override private[sql] lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows")) + override def outputPartitioning: Partitioning = streamed.outputPartitioning + + override def output: Seq[Attribute] = left.output + + override def left: SparkPlan = streamed + + override def right: SparkPlan = broadcast + override def requiredChildDistribution: Seq[Distribution] = { UnspecifiedDistribution :: BroadcastDistribution(IdentityBroadcastMode) :: Nil } - override def outputPartitioning: Partitioning = left.outputPartitioning - - override def output: Seq[Attribute] = left.output - @transient private lazy val boundCondition = newPredicate(condition.getOrElse(Literal(true)), left.output ++ right.output) protected override def doExecute(): RDD[InternalRow] = { val numOutputRows = longMetric("numOutputRows") - val broadcastedRelation = right.executeBroadcast[Array[InternalRow]]() + val broadcastedRelation = broadcast.executeBroadcast[Array[InternalRow]]() - left.execute().mapPartitions { streamedIter => + streamed.execute().mapPartitions { streamedIter => val joinedRow = new JoinedRow val relation = broadcastedRelation.value From 54b558d1fee977177eea901d7f9a9dce942b2f8c Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Sun, 21 Feb 2016 00:14:54 +0100 Subject: [PATCH 15/19] Revert renaming of variabels in LeftSemiJoinBNL. --- .../org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala index 875e80f3612c6..df6dac88187cc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala @@ -38,8 +38,10 @@ case class LeftSemiJoinBNL( override def output: Seq[Attribute] = left.output + /** The Streamed Relation */ override def left: SparkPlan = streamed + /** The Broadcast relation */ override def right: SparkPlan = broadcast override def requiredChildDistribution: Seq[Distribution] = { From f33d2cbe324cc1eaed832eeb600ad68ff4447173 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Sun, 21 Feb 2016 14:27:45 +0100 Subject: [PATCH 16/19] Move all exchange related operators into the exchange package. --- .../org/apache/spark/sql/SQLContext.scala | 14 +- .../spark/sql/execution/SparkStrategies.scala | 8 +- .../apache/spark/sql/execution/exchange.scala | 567 ------------------ .../exchange/BroadcastExchange.scala | 90 +++ .../exchange/EnsureRequirements.scala | 261 ++++++++ .../{ => exchange}/ExchangeCoordinator.scala | 3 +- .../execution/exchange/ShuffleExchange.scala | 261 ++++++++ .../apache/spark/sql/execution/limit.scala | 1 + .../apache/spark/sql/CachedTableSuite.scala | 2 +- .../org/apache/spark/sql/DataFrameSuite.scala | 2 +- .../execution/ExchangeCoordinatorSuite.scala | 1 + .../spark/sql/execution/ExchangeSuite.scala | 1 + .../spark/sql/execution/PlannerSuite.scala | 1 + .../execution/joins/BroadcastJoinSuite.scala | 2 +- .../sql/execution/joins/InnerJoinSuite.scala | 1 + .../sql/execution/joins/OuterJoinSuite.scala | 3 +- .../sql/execution/joins/SemiJoinSuite.scala | 3 +- .../spark/sql/sources/BucketedReadSuite.scala | 3 +- 18 files changed, 644 insertions(+), 580 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/exchange.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchange.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala rename sql/core/src/main/scala/org/apache/spark/sql/execution/{ => exchange}/ExchangeCoordinator.scala (99%) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchange.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index c7d1096a1384c..ede4d84e83d25 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -41,6 +41,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.exchange.EnsureRequirements import org.apache.spark.sql.execution.ui.{SQLListener, SQLTab} import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.types._ @@ -59,7 +60,6 @@ import org.apache.spark.util.Utils * @groupname config Configuration * @groupname dataframes Custom DataFrame Creation * @groupname Ungrouped Support functions for language integrated queries - * * @since 1.0.0 */ class SQLContext private[sql]( @@ -305,6 +305,7 @@ class SQLContext private[sql]( /** * Returns true if the table is currently cached in-memory. + * * @group cachemgmt * @since 1.3.0 */ @@ -314,7 +315,8 @@ class SQLContext private[sql]( /** * Returns true if the [[Queryable]] is currently cached in-memory. - * @group cachemgmt + * + * @group cachemgmt * @since 1.3.0 */ private[sql] def isCached(qName: Queryable): Boolean = { @@ -323,6 +325,7 @@ class SQLContext private[sql]( /** * Caches the specified table in-memory. + * * @group cachemgmt * @since 1.3.0 */ @@ -332,6 +335,7 @@ class SQLContext private[sql]( /** * Removes the specified table from the in-memory cache. + * * @group cachemgmt * @since 1.3.0 */ @@ -339,6 +343,7 @@ class SQLContext private[sql]( /** * Removes all cached tables from the in-memory cache. + * * @since 1.3.0 */ def clearCache(): Unit = cacheManager.clearCache() @@ -364,6 +369,7 @@ class SQLContext private[sql]( /** * Converts $"col name" into an [[Column]]. + * * @since 1.3.0 */ // This must live here to preserve binary compatibility with Spark < 1.5. @@ -539,6 +545,7 @@ class SQLContext private[sql]( * * WARNING: Since there is no guaranteed ordering for fields in a Java Bean, * SELECT * queries will return the columns in an undefined order. + * * @group dataframes * @since 1.3.0 */ @@ -558,6 +565,7 @@ class SQLContext private[sql]( * * WARNING: Since there is no guaranteed ordering for fields in a Java Bean, * SELECT * queries will return the columns in an undefined order. + * * @group dataframes * @since 1.3.0 */ @@ -570,6 +578,7 @@ class SQLContext private[sql]( * * WARNING: Since there is no guaranteed ordering for fields in a Java Bean, * SELECT * queries will return the columns in an undefined order. + * * @group dataframes * @since 1.6.0 */ @@ -728,7 +737,6 @@ class SQLContext private[sql]( * cached/persisted before, it's also unpersisted. * * @param tableName the name of the table to be unregistered. - * * @group basic * @since 1.3.0 */ 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 3bddce741522a..7347156398674 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 @@ -17,7 +17,8 @@ package org.apache.spark.sql.execution -import org.apache.spark.sql.{execution, Strategy} +import org.apache.spark.sql.execution.exchange.ShuffleExchange +import org.apache.spark.sql.Strategy import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression @@ -25,6 +26,7 @@ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.{BroadcastHint, LogicalPlan} import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.execution import org.apache.spark.sql.execution.{DescribeCommand => RunnableDescribeCommand} import org.apache.spark.sql.execution.columnar.{InMemoryColumnarTableScan, InMemoryRelation} import org.apache.spark.sql.execution.datasources.{CreateTableUsing, CreateTempTableUsing, DescribeCommand => LogicalDescribeCommand, _} @@ -328,7 +330,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.Repartition(numPartitions, shuffle, child) => if (shuffle) { - execution.ShuffleExchange(RoundRobinPartitioning(numPartitions), planLater(child)) :: Nil + ShuffleExchange(RoundRobinPartitioning(numPartitions), planLater(child)) :: Nil } else { execution.Coalesce(numPartitions, planLater(child)) :: Nil } @@ -367,7 +369,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case r @ logical.Range(start, end, step, numSlices, output) => execution.Range(start, step, numSlices, r.numElements, output) :: Nil case logical.RepartitionByExpression(expressions, child, nPartitions) => - execution.ShuffleExchange(HashPartitioning( + exchange.ShuffleExchange(HashPartitioning( expressions, nPartitions.getOrElse(numPartitions)), planLater(child)) :: Nil case e @ python.EvaluatePython(udf, child, _) => python.BatchPythonEvaluation(udf, e.output, planLater(child)) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange.scala deleted file mode 100644 index 73854853793cc..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange.scala +++ /dev/null @@ -1,567 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution - -import java.util.Random - -import scala.concurrent.{Await, ExecutionContext, Future} -import scala.concurrent.duration._ - -import org.apache.spark._ -import org.apache.spark.rdd.RDD -import org.apache.spark.serializer.Serializer -import org.apache.spark.shuffle.hash.HashShuffleManager -import org.apache.spark.shuffle.sort.SortShuffleManager -import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.errors.attachTree -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering -import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.util.{MutablePair, ThreadUtils} - - -/** - * Performs a shuffle that will result in the desired `newPartitioning`. - */ -case class ShuffleExchange( - var newPartitioning: Partitioning, - child: SparkPlan, - @transient coordinator: Option[ExchangeCoordinator]) extends UnaryNode { - - override def nodeName: String = { - val extraInfo = coordinator match { - case Some(exchangeCoordinator) if exchangeCoordinator.isEstimated => - s"(coordinator id: ${System.identityHashCode(coordinator)})" - case Some(exchangeCoordinator) if !exchangeCoordinator.isEstimated => - s"(coordinator id: ${System.identityHashCode(coordinator)})" - case None => "" - } - - val simpleNodeName = "Exchange" - s"$simpleNodeName$extraInfo" - } - - override def outputPartitioning: Partitioning = newPartitioning - - override def output: Seq[Attribute] = child.output - - private val serializer: Serializer = new UnsafeRowSerializer(child.output.size) - - override protected def doPrepare(): Unit = { - // If an ExchangeCoordinator is needed, we register this Exchange operator - // to the coordinator when we do prepare. It is important to make sure - // we register this operator right before the execution instead of register it - // in the constructor because it is possible that we create new instances of - // Exchange operators when we transform the physical plan - // (then the ExchangeCoordinator will hold references of unneeded Exchanges). - // So, we should only call registerExchange just before we start to execute - // the plan. - coordinator match { - case Some(exchangeCoordinator) => exchangeCoordinator.registerExchange(this) - case None => - } - } - - /** - * Returns a [[ShuffleDependency]] that will partition rows of its child based on - * the partitioning scheme defined in `newPartitioning`. Those partitions of - * the returned ShuffleDependency will be the input of shuffle. - */ - private[sql] def prepareShuffleDependency(): ShuffleDependency[Int, InternalRow, InternalRow] = { - ShuffleExchange.prepareShuffleDependency( - child.execute(), child.output, newPartitioning, serializer) - } - - /** - * Returns a [[ShuffledRowRDD]] that represents the post-shuffle dataset. - * This [[ShuffledRowRDD]] is created based on a given [[ShuffleDependency]] and an optional - * partition start indices array. If this optional array is defined, the returned - * [[ShuffledRowRDD]] will fetch pre-shuffle partitions based on indices of this array. - */ - private[sql] def preparePostShuffleRDD( - shuffleDependency: ShuffleDependency[Int, InternalRow, InternalRow], - specifiedPartitionStartIndices: Option[Array[Int]] = None): ShuffledRowRDD = { - // If an array of partition start indices is provided, we need to use this array - // to create the ShuffledRowRDD. Also, we need to update newPartitioning to - // update the number of post-shuffle partitions. - specifiedPartitionStartIndices.foreach { indices => - assert(newPartitioning.isInstanceOf[HashPartitioning]) - newPartitioning = UnknownPartitioning(indices.length) - } - new ShuffledRowRDD(shuffleDependency, specifiedPartitionStartIndices) - } - - protected override def doExecute(): RDD[InternalRow] = attachTree(this, "execute") { - coordinator match { - case Some(exchangeCoordinator) => - val shuffleRDD = exchangeCoordinator.postShuffleRDD(this) - assert(shuffleRDD.partitions.length == newPartitioning.numPartitions) - shuffleRDD - case None => - val shuffleDependency = prepareShuffleDependency() - preparePostShuffleRDD(shuffleDependency) - } - } -} - -object ShuffleExchange { - def apply(newPartitioning: Partitioning, child: SparkPlan): ShuffleExchange = { - ShuffleExchange(newPartitioning, child, coordinator = None: Option[ExchangeCoordinator]) - } - - /** - * Determines whether records must be defensively copied before being sent to the shuffle. - * Several of Spark's shuffle components will buffer deserialized Java objects in memory. The - * shuffle code assumes that objects are immutable and hence does not perform its own defensive - * copying. In Spark SQL, however, operators' iterators return the same mutable `Row` object. In - * order to properly shuffle the output of these operators, we need to perform our own copying - * prior to sending records to the shuffle. This copying is expensive, so we try to avoid it - * whenever possible. This method encapsulates the logic for choosing when to copy. - * - * In the long run, we might want to push this logic into core's shuffle APIs so that we don't - * have to rely on knowledge of core internals here in SQL. - * - * See SPARK-2967, SPARK-4479, and SPARK-7375 for more discussion of this issue. - * - * @param partitioner the partitioner for the shuffle - * @param serializer the serializer that will be used to write rows - * @return true if rows should be copied before being shuffled, false otherwise - */ - private def needToCopyObjectsBeforeShuffle( - partitioner: Partitioner, - serializer: Serializer): Boolean = { - // Note: even though we only use the partitioner's `numPartitions` field, we require it to be - // passed instead of directly passing the number of partitions in order to guard against - // corner-cases where a partitioner constructed with `numPartitions` partitions may output - // fewer partitions (like RangePartitioner, for example). - val conf = SparkEnv.get.conf - val shuffleManager = SparkEnv.get.shuffleManager - val sortBasedShuffleOn = shuffleManager.isInstanceOf[SortShuffleManager] - val bypassMergeThreshold = conf.getInt("spark.shuffle.sort.bypassMergeThreshold", 200) - if (sortBasedShuffleOn) { - val bypassIsSupported = SparkEnv.get.shuffleManager.isInstanceOf[SortShuffleManager] - if (bypassIsSupported && partitioner.numPartitions <= bypassMergeThreshold) { - // If we're using the original SortShuffleManager and the number of output partitions is - // sufficiently small, then Spark will fall back to the hash-based shuffle write path, which - // doesn't buffer deserialized records. - // Note that we'll have to remove this case if we fix SPARK-6026 and remove this bypass. - false - } else if (serializer.supportsRelocationOfSerializedObjects) { - // SPARK-4550 and SPARK-7081 extended sort-based shuffle to serialize individual records - // prior to sorting them. This optimization is only applied in cases where shuffle - // dependency does not specify an aggregator or ordering and the record serializer has - // certain properties. If this optimization is enabled, we can safely avoid the copy. - // - // Exchange never configures its ShuffledRDDs with aggregators or key orderings, so we only - // need to check whether the optimization is enabled and supported by our serializer. - false - } else { - // Spark's SortShuffleManager uses `ExternalSorter` to buffer records in memory, so we must - // copy. - true - } - } else if (shuffleManager.isInstanceOf[HashShuffleManager]) { - // We're using hash-based shuffle, so we don't need to copy. - false - } else { - // Catch-all case to safely handle any future ShuffleManager implementations. - true - } - } - - /** - * Returns a [[ShuffleDependency]] that will partition rows of its child based on - * the partitioning scheme defined in `newPartitioning`. Those partitions of - * the returned ShuffleDependency will be the input of shuffle. - */ - private[sql] def prepareShuffleDependency( - rdd: RDD[InternalRow], - outputAttributes: Seq[Attribute], - newPartitioning: Partitioning, - serializer: Serializer): ShuffleDependency[Int, InternalRow, InternalRow] = { - val part: Partitioner = newPartitioning match { - case RoundRobinPartitioning(numPartitions) => new HashPartitioner(numPartitions) - case HashPartitioning(_, n) => - new Partitioner { - override def numPartitions: Int = n - // For HashPartitioning, the partitioning key is already a valid partition ID, as we use - // `HashPartitioning.partitionIdExpression` to produce partitioning key. - override def getPartition(key: Any): Int = key.asInstanceOf[Int] - } - case RangePartitioning(sortingExpressions, numPartitions) => - // Internally, RangePartitioner runs a job on the RDD that samples keys to compute - // partition bounds. To get accurate samples, we need to copy the mutable keys. - val rddForSampling = rdd.mapPartitionsInternal { iter => - val mutablePair = new MutablePair[InternalRow, Null]() - iter.map(row => mutablePair.update(row.copy(), null)) - } - implicit val ordering = new LazilyGeneratedOrdering(sortingExpressions, outputAttributes) - new RangePartitioner(numPartitions, rddForSampling, ascending = true) - case SinglePartition => - new Partitioner { - override def numPartitions: Int = 1 - override def getPartition(key: Any): Int = 0 - } - case _ => sys.error(s"Exchange not implemented for $newPartitioning") - // TODO: Handle BroadcastPartitioning. - } - def getPartitionKeyExtractor(): InternalRow => Any = newPartitioning match { - case RoundRobinPartitioning(numPartitions) => - // Distributes elements evenly across output partitions, starting from a random partition. - var position = new Random(TaskContext.get().partitionId()).nextInt(numPartitions) - (row: InternalRow) => { - // The HashPartitioner will handle the `mod` by the number of partitions - position += 1 - position - } - case h: HashPartitioning => - val projection = UnsafeProjection.create(h.partitionIdExpression :: Nil, outputAttributes) - row => projection(row).getInt(0) - case RangePartitioning(_, _) | SinglePartition => identity - case _ => sys.error(s"Exchange not implemented for $newPartitioning") - } - val rddWithPartitionIds: RDD[Product2[Int, InternalRow]] = { - if (needToCopyObjectsBeforeShuffle(part, serializer)) { - rdd.mapPartitionsInternal { iter => - val getPartitionKey = getPartitionKeyExtractor() - iter.map { row => (part.getPartition(getPartitionKey(row)), row.copy()) } - } - } else { - rdd.mapPartitionsInternal { iter => - val getPartitionKey = getPartitionKeyExtractor() - val mutablePair = new MutablePair[Int, InternalRow]() - iter.map { row => mutablePair.update(part.getPartition(getPartitionKey(row)), row) } - } - } - } - - // Now, we manually create a ShuffleDependency. Because pairs in rddWithPartitionIds - // are in the form of (partitionId, row) and every partitionId is in the expected range - // [0, part.numPartitions - 1]. The partitioner of this is a PartitionIdPassthrough. - val dependency = - new ShuffleDependency[Int, InternalRow, InternalRow]( - rddWithPartitionIds, - new PartitionIdPassthrough(part.numPartitions), - Some(serializer)) - - dependency - } -} - -/** - * A [[BroadcastExchange]] collects, transforms and finally broadcasts the result of a transformed - * SparkPlan. - */ -case class BroadcastExchange( - mode: BroadcastMode, - child: SparkPlan) extends UnaryNode { - - override def output: Seq[Attribute] = child.output - - override def outputPartitioning: Partitioning = BroadcastPartitioning(mode) - - val timeout: Duration = { - val timeoutValue = sqlContext.conf.broadcastTimeout - if (timeoutValue < 0) { - Duration.Inf - } else { - timeoutValue.seconds - } - } - - @transient - private lazy val relationFuture: Future[broadcast.Broadcast[Any]] = { - // broadcastFuture is used in "doExecute". Therefore we can get the execution id correctly here. - val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - Future { - // This will run in another thread. Set the execution id so that we can connect these jobs - // with the correct execution. - SQLExecution.withExecutionId(sparkContext, executionId) { - // Note that we use .execute().collect() because we don't want to convert data to Scala - // types - val input: Array[InternalRow] = child.execute().map { row => - row.copy() - }.collect() - - // Construct and broadcast the relation. - sparkContext.broadcast(mode.transform(input)) - } - }(BroadcastExchange.executionContext) - } - - override protected def doPrepare(): Unit = { - // Materialize the future. - relationFuture - } - - override protected def doExecute(): RDD[InternalRow] = { - throw new UnsupportedOperationException("Broadcast does not support the execute() code path.") - } - - override protected[sql] def doExecuteBroadcast[T](): broadcast.Broadcast[T] = { - val result = Await.result(relationFuture, timeout) - result.asInstanceOf[broadcast.Broadcast[T]] - } -} - -object BroadcastExchange { - private[execution] val executionContext = ExecutionContext.fromExecutorService( - ThreadUtils.newDaemonCachedThreadPool("broadcast-exchange", 128)) -} - -/** - * Ensures that the [[org.apache.spark.sql.catalyst.plans.physical.Partitioning Partitioning]] - * of input data meets the - * [[org.apache.spark.sql.catalyst.plans.physical.Distribution Distribution]] requirements for - * each operator by inserting [[ShuffleExchange]] Operators where required. Also ensure that the - * input partition ordering requirements are met. - */ -private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[SparkPlan] { - private def defaultNumPreShufflePartitions: Int = sqlContext.conf.numShufflePartitions - - private def targetPostShuffleInputSize: Long = sqlContext.conf.targetPostShuffleInputSize - - private def adaptiveExecutionEnabled: Boolean = sqlContext.conf.adaptiveExecutionEnabled - - private def minNumPostShufflePartitions: Option[Int] = { - val minNumPostShufflePartitions = sqlContext.conf.minNumPostShufflePartitions - if (minNumPostShufflePartitions > 0) Some(minNumPostShufflePartitions) else None - } - - /** - * Given a required distribution, returns a partitioning that satisfies that distribution. - */ - private def createPartitioning( - requiredDistribution: Distribution, - numPartitions: Int): Partitioning = { - requiredDistribution match { - case AllTuples => SinglePartition - case ClusteredDistribution(clustering) => HashPartitioning(clustering, numPartitions) - case OrderedDistribution(ordering) => RangePartitioning(ordering, numPartitions) - case dist => sys.error(s"Do not know how to satisfy distribution $dist") - } - } - - /** - * Adds [[ExchangeCoordinator]] to [[ShuffleExchange]]s if adaptive query execution is enabled - * and partitioning schemes of these [[ShuffleExchange]]s support [[ExchangeCoordinator]]. - */ - private def withExchangeCoordinator( - children: Seq[SparkPlan], - requiredChildDistributions: Seq[Distribution]): Seq[SparkPlan] = { - val supportsCoordinator = - if (children.exists(_.isInstanceOf[ShuffleExchange])) { - // Right now, ExchangeCoordinator only support HashPartitionings. - children.forall { - case e @ ShuffleExchange(hash: HashPartitioning, _, _) => true - case child => - child.outputPartitioning match { - case hash: HashPartitioning => true - case collection: PartitioningCollection => - collection.partitionings.forall(_.isInstanceOf[HashPartitioning]) - case _ => false - } - } - } else { - // In this case, although we do not have Exchange operators, we may still need to - // shuffle data when we have more than one children because data generated by - // these children may not be partitioned in the same way. - // Please see the comment in withCoordinator for more details. - val supportsDistribution = - requiredChildDistributions.forall(_.isInstanceOf[ClusteredDistribution]) - children.length > 1 && supportsDistribution - } - - val withCoordinator = - if (adaptiveExecutionEnabled && supportsCoordinator) { - val coordinator = - new ExchangeCoordinator( - children.length, - targetPostShuffleInputSize, - minNumPostShufflePartitions) - children.zip(requiredChildDistributions).map { - case (e: ShuffleExchange, _) => - // This child is an Exchange, we need to add the coordinator. - e.copy(coordinator = Some(coordinator)) - case (child, distribution) => - // If this child is not an Exchange, we need to add an Exchange for now. - // Ideally, we can try to avoid this Exchange. However, when we reach here, - // there are at least two children operators (because if there is a single child - // and we can avoid Exchange, supportsCoordinator will be false and we - // will not reach here.). Although we can make two children have the same number of - // post-shuffle partitions. Their numbers of pre-shuffle partitions may be different. - // For example, let's say we have the following plan - // Join - // / \ - // Agg Exchange - // / \ - // Exchange t2 - // / - // t1 - // In this case, because a post-shuffle partition can include multiple pre-shuffle - // partitions, a HashPartitioning will not be strictly partitioned by the hashcodes - // after shuffle. So, even we can use the child Exchange operator of the Join to - // have a number of post-shuffle partitions that matches the number of partitions of - // Agg, we cannot say these two children are partitioned in the same way. - // Here is another case - // Join - // / \ - // Agg1 Agg2 - // / \ - // Exchange1 Exchange2 - // / \ - // t1 t2 - // In this case, two Aggs shuffle data with the same column of the join condition. - // After we use ExchangeCoordinator, these two Aggs may not be partitioned in the same - // way. Let's say that Agg1 and Agg2 both have 5 pre-shuffle partitions and 2 - // post-shuffle partitions. It is possible that Agg1 fetches those pre-shuffle - // partitions by using a partitionStartIndices [0, 3]. However, Agg2 may fetch its - // pre-shuffle partitions by using another partitionStartIndices [0, 4]. - // So, Agg1 and Agg2 are actually not co-partitioned. - // - // It will be great to introduce a new Partitioning to represent the post-shuffle - // partitions when one post-shuffle partition includes multiple pre-shuffle partitions. - val targetPartitioning = - createPartitioning(distribution, defaultNumPreShufflePartitions) - assert(targetPartitioning.isInstanceOf[HashPartitioning]) - ShuffleExchange(targetPartitioning, child, Some(coordinator)) - } - } else { - // If we do not need ExchangeCoordinator, the original children are returned. - children - } - - withCoordinator - } - - private def ensureDistributionAndOrdering(operator: SparkPlan): SparkPlan = { - val requiredChildDistributions: Seq[Distribution] = operator.requiredChildDistribution - val requiredChildOrderings: Seq[Seq[SortOrder]] = operator.requiredChildOrdering - var children: Seq[SparkPlan] = operator.children - assert(requiredChildDistributions.length == children.length) - assert(requiredChildOrderings.length == children.length) - - // Ensure that the operator's children satisfy their output distribution requirements: - children = children.zip(requiredChildDistributions).map { - case (child, distribution) if child.outputPartitioning.satisfies(distribution) => - child - case (child, BroadcastDistribution(mode)) => - BroadcastExchange(mode, child) - case (child, distribution) => - ShuffleExchange(createPartitioning(distribution, defaultNumPreShufflePartitions), child) - } - - // If the operator has multiple children and specifies child output distributions (e.g. join), - // then the children's output partitionings must be compatible: - def requireCompatiblePartitioning(distribution: Distribution): Boolean = distribution match { - case UnspecifiedDistribution => false - case BroadcastDistribution(_) => false - case _ => true - } - if (children.length > 1 - && requiredChildDistributions.exists(requireCompatiblePartitioning) - && !Partitioning.allCompatible(children.map(_.outputPartitioning))) { - - // First check if the existing partitions of the children all match. This means they are - // partitioned by the same partitioning into the same number of partitions. In that case, - // don't try to make them match `defaultPartitions`, just use the existing partitioning. - val maxChildrenNumPartitions = children.map(_.outputPartitioning.numPartitions).max - val useExistingPartitioning = children.zip(requiredChildDistributions).forall { - case (child, distribution) => { - child.outputPartitioning.guarantees( - createPartitioning(distribution, maxChildrenNumPartitions)) - } - } - - children = if (useExistingPartitioning) { - // We do not need to shuffle any child's output. - children - } else { - // We need to shuffle at least one child's output. - // Now, we will determine the number of partitions that will be used by created - // partitioning schemes. - val numPartitions = { - // Let's see if we need to shuffle all child's outputs when we use - // maxChildrenNumPartitions. - val shufflesAllChildren = children.zip(requiredChildDistributions).forall { - case (child, distribution) => { - !child.outputPartitioning.guarantees( - createPartitioning(distribution, maxChildrenNumPartitions)) - } - } - // If we need to shuffle all children, we use defaultNumPreShufflePartitions as the - // number of partitions. Otherwise, we use maxChildrenNumPartitions. - if (shufflesAllChildren) defaultNumPreShufflePartitions else maxChildrenNumPartitions - } - - children.zip(requiredChildDistributions).map { - case (child, distribution) => { - val targetPartitioning = createPartitioning(distribution, numPartitions) - if (child.outputPartitioning.guarantees(targetPartitioning)) { - child - } else { - child match { - // If child is an exchange, we replace it with - // a new one having targetPartitioning. - case ShuffleExchange(_, c, _) => ShuffleExchange(targetPartitioning, c) - case _ => ShuffleExchange(targetPartitioning, child) - } - } - } - } - } - } - - // Now, we need to add ExchangeCoordinator if necessary. - // Actually, it is not a good idea to add ExchangeCoordinators while we are adding Exchanges. - // However, with the way that we plan the query, we do not have a place where we have a - // global picture of all shuffle dependencies of a post-shuffle stage. So, we add coordinator - // at here for now. - // Once we finish https://issues.apache.org/jira/browse/SPARK-10665, - // we can first add Exchanges and then add coordinator once we have a DAG of query fragments. - children = withExchangeCoordinator(children, requiredChildDistributions) - - // Now that we've performed any necessary shuffles, add sorts to guarantee output orderings: - children = children.zip(requiredChildOrderings).map { case (child, requiredOrdering) => - if (requiredOrdering.nonEmpty) { - // If child.outputOrdering is [a, b] and requiredOrdering is [a], we do not need to sort. - if (requiredOrdering != child.outputOrdering.take(requiredOrdering.length)) { - Sort(requiredOrdering, global = false, child = child) - } else { - child - } - } else { - child - } - } - - operator.withNewChildren(children) - } - - def apply(plan: SparkPlan): SparkPlan = plan.transformUp { - case operator @ ShuffleExchange(partitioning, child, _) => - child.children match { - case ShuffleExchange(childPartitioning, baseChild, _)::Nil => - if (childPartitioning.guarantees(partitioning)) child else operator - case _ => operator - } - case operator: SparkPlan => ensureDistributionAndOrdering(operator) - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchange.scala new file mode 100644 index 0000000000000..9e1574c2c009b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchange.scala @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.exchange + +import scala.concurrent.{Await, ExecutionContext, Future} +import scala.concurrent.duration._ + +import org.apache.spark.broadcast +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, BroadcastPartitioning, Partitioning} +import org.apache.spark.sql.execution.{SparkPlan, SQLExecution, UnaryNode} +import org.apache.spark.util.ThreadUtils + +/** + * A [[BroadcastExchange]] collects, transforms and finally broadcasts the result of a transformed + * SparkPlan. + */ +case class BroadcastExchange( + mode: BroadcastMode, + child: SparkPlan) extends UnaryNode { + + override def output: Seq[Attribute] = child.output + + override def outputPartitioning: Partitioning = BroadcastPartitioning(mode) + + val timeout: Duration = { + val timeoutValue = sqlContext.conf.broadcastTimeout + if (timeoutValue < 0) { + Duration.Inf + } else { + timeoutValue.seconds + } + } + + @transient + private lazy val relationFuture: Future[broadcast.Broadcast[Any]] = { + // broadcastFuture is used in "doExecute". Therefore we can get the execution id correctly here. + val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + Future { + // This will run in another thread. Set the execution id so that we can connect these jobs + // with the correct execution. + SQLExecution.withExecutionId(sparkContext, executionId) { + // Note that we use .execute().collect() because we don't want to convert data to Scala + // types + val input: Array[InternalRow] = child.execute().map { row => + row.copy() + }.collect() + + // Construct and broadcast the relation. + sparkContext.broadcast(mode.transform(input)) + } + }(BroadcastExchange.executionContext) + } + + override protected def doPrepare(): Unit = { + // Materialize the future. + relationFuture + } + + override protected def doExecute(): RDD[InternalRow] = { + throw new UnsupportedOperationException("Broadcast does not support the execute() code path.") + } + + override protected[sql] def doExecuteBroadcast[T](): broadcast.Broadcast[T] = { + val result = Await.result(relationFuture, timeout) + result.asInstanceOf[broadcast.Broadcast[T]] + } +} + +object BroadcastExchange { + private[execution] val executionContext = ExecutionContext.fromExecutorService( + ThreadUtils.newDaemonCachedThreadPool("broadcast-exchange", 128)) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala new file mode 100644 index 0000000000000..709a4246365dd --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -0,0 +1,261 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.exchange + +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution._ + +/** + * Ensures that the [[org.apache.spark.sql.catalyst.plans.physical.Partitioning Partitioning]] + * of input data meets the + * [[org.apache.spark.sql.catalyst.plans.physical.Distribution Distribution]] requirements for + * each operator by inserting [[ShuffleExchange]] Operators where required. Also ensure that the + * input partition ordering requirements are met. + */ +private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[SparkPlan] { + private def defaultNumPreShufflePartitions: Int = sqlContext.conf.numShufflePartitions + + private def targetPostShuffleInputSize: Long = sqlContext.conf.targetPostShuffleInputSize + + private def adaptiveExecutionEnabled: Boolean = sqlContext.conf.adaptiveExecutionEnabled + + private def minNumPostShufflePartitions: Option[Int] = { + val minNumPostShufflePartitions = sqlContext.conf.minNumPostShufflePartitions + if (minNumPostShufflePartitions > 0) Some(minNumPostShufflePartitions) else None + } + + /** + * Given a required distribution, returns a partitioning that satisfies that distribution. + */ + private def createPartitioning( + requiredDistribution: Distribution, + numPartitions: Int): Partitioning = { + requiredDistribution match { + case AllTuples => SinglePartition + case ClusteredDistribution(clustering) => HashPartitioning(clustering, numPartitions) + case OrderedDistribution(ordering) => RangePartitioning(ordering, numPartitions) + case dist => sys.error(s"Do not know how to satisfy distribution $dist") + } + } + + /** + * Adds [[ExchangeCoordinator]] to [[ShuffleExchange]]s if adaptive query execution is enabled + * and partitioning schemes of these [[ShuffleExchange]]s support [[ExchangeCoordinator]]. + */ + private def withExchangeCoordinator( + children: Seq[SparkPlan], + requiredChildDistributions: Seq[Distribution]): Seq[SparkPlan] = { + val supportsCoordinator = + if (children.exists(_.isInstanceOf[ShuffleExchange])) { + // Right now, ExchangeCoordinator only support HashPartitionings. + children.forall { + case e @ ShuffleExchange(hash: HashPartitioning, _, _) => true + case child => + child.outputPartitioning match { + case hash: HashPartitioning => true + case collection: PartitioningCollection => + collection.partitionings.forall(_.isInstanceOf[HashPartitioning]) + case _ => false + } + } + } else { + // In this case, although we do not have Exchange operators, we may still need to + // shuffle data when we have more than one children because data generated by + // these children may not be partitioned in the same way. + // Please see the comment in withCoordinator for more details. + val supportsDistribution = + requiredChildDistributions.forall(_.isInstanceOf[ClusteredDistribution]) + children.length > 1 && supportsDistribution + } + + val withCoordinator = + if (adaptiveExecutionEnabled && supportsCoordinator) { + val coordinator = + new ExchangeCoordinator( + children.length, + targetPostShuffleInputSize, + minNumPostShufflePartitions) + children.zip(requiredChildDistributions).map { + case (e: ShuffleExchange, _) => + // This child is an Exchange, we need to add the coordinator. + e.copy(coordinator = Some(coordinator)) + case (child, distribution) => + // If this child is not an Exchange, we need to add an Exchange for now. + // Ideally, we can try to avoid this Exchange. However, when we reach here, + // there are at least two children operators (because if there is a single child + // and we can avoid Exchange, supportsCoordinator will be false and we + // will not reach here.). Although we can make two children have the same number of + // post-shuffle partitions. Their numbers of pre-shuffle partitions may be different. + // For example, let's say we have the following plan + // Join + // / \ + // Agg Exchange + // / \ + // Exchange t2 + // / + // t1 + // In this case, because a post-shuffle partition can include multiple pre-shuffle + // partitions, a HashPartitioning will not be strictly partitioned by the hashcodes + // after shuffle. So, even we can use the child Exchange operator of the Join to + // have a number of post-shuffle partitions that matches the number of partitions of + // Agg, we cannot say these two children are partitioned in the same way. + // Here is another case + // Join + // / \ + // Agg1 Agg2 + // / \ + // Exchange1 Exchange2 + // / \ + // t1 t2 + // In this case, two Aggs shuffle data with the same column of the join condition. + // After we use ExchangeCoordinator, these two Aggs may not be partitioned in the same + // way. Let's say that Agg1 and Agg2 both have 5 pre-shuffle partitions and 2 + // post-shuffle partitions. It is possible that Agg1 fetches those pre-shuffle + // partitions by using a partitionStartIndices [0, 3]. However, Agg2 may fetch its + // pre-shuffle partitions by using another partitionStartIndices [0, 4]. + // So, Agg1 and Agg2 are actually not co-partitioned. + // + // It will be great to introduce a new Partitioning to represent the post-shuffle + // partitions when one post-shuffle partition includes multiple pre-shuffle partitions. + val targetPartitioning = + createPartitioning(distribution, defaultNumPreShufflePartitions) + assert(targetPartitioning.isInstanceOf[HashPartitioning]) + ShuffleExchange(targetPartitioning, child, Some(coordinator)) + } + } else { + // If we do not need ExchangeCoordinator, the original children are returned. + children + } + + withCoordinator + } + + private def ensureDistributionAndOrdering(operator: SparkPlan): SparkPlan = { + val requiredChildDistributions: Seq[Distribution] = operator.requiredChildDistribution + val requiredChildOrderings: Seq[Seq[SortOrder]] = operator.requiredChildOrdering + var children: Seq[SparkPlan] = operator.children + assert(requiredChildDistributions.length == children.length) + assert(requiredChildOrderings.length == children.length) + + // Ensure that the operator's children satisfy their output distribution requirements: + children = children.zip(requiredChildDistributions).map { + case (child, distribution) if child.outputPartitioning.satisfies(distribution) => + child + case (child, BroadcastDistribution(mode)) => + BroadcastExchange(mode, child) + case (child, distribution) => + ShuffleExchange(createPartitioning(distribution, defaultNumPreShufflePartitions), child) + } + + // If the operator has multiple children and specifies child output distributions (e.g. join), + // then the children's output partitionings must be compatible: + def requireCompatiblePartitioning(distribution: Distribution): Boolean = distribution match { + case UnspecifiedDistribution => false + case BroadcastDistribution(_) => false + case _ => true + } + if (children.length > 1 + && requiredChildDistributions.exists(requireCompatiblePartitioning) + && !Partitioning.allCompatible(children.map(_.outputPartitioning))) { + + // First check if the existing partitions of the children all match. This means they are + // partitioned by the same partitioning into the same number of partitions. In that case, + // don't try to make them match `defaultPartitions`, just use the existing partitioning. + val maxChildrenNumPartitions = children.map(_.outputPartitioning.numPartitions).max + val useExistingPartitioning = children.zip(requiredChildDistributions).forall { + case (child, distribution) => + child.outputPartitioning.guarantees( + createPartitioning(distribution, maxChildrenNumPartitions)) + } + + children = if (useExistingPartitioning) { + // We do not need to shuffle any child's output. + children + } else { + // We need to shuffle at least one child's output. + // Now, we will determine the number of partitions that will be used by created + // partitioning schemes. + val numPartitions = { + // Let's see if we need to shuffle all child's outputs when we use + // maxChildrenNumPartitions. + val shufflesAllChildren = children.zip(requiredChildDistributions).forall { + case (child, distribution) => + !child.outputPartitioning.guarantees( + createPartitioning(distribution, maxChildrenNumPartitions)) + } + // If we need to shuffle all children, we use defaultNumPreShufflePartitions as the + // number of partitions. Otherwise, we use maxChildrenNumPartitions. + if (shufflesAllChildren) defaultNumPreShufflePartitions else maxChildrenNumPartitions + } + + children.zip(requiredChildDistributions).map { + case (child, distribution) => + val targetPartitioning = createPartitioning(distribution, numPartitions) + if (child.outputPartitioning.guarantees(targetPartitioning)) { + child + } else { + child match { + // If child is an exchange, we replace it with + // a new one having targetPartitioning. + case ShuffleExchange(_, c, _) => ShuffleExchange(targetPartitioning, c) + case _ => ShuffleExchange(targetPartitioning, child) + } + } + } + } + } + + // Now, we need to add ExchangeCoordinator if necessary. + // Actually, it is not a good idea to add ExchangeCoordinators while we are adding Exchanges. + // However, with the way that we plan the query, we do not have a place where we have a + // global picture of all shuffle dependencies of a post-shuffle stage. So, we add coordinator + // at here for now. + // Once we finish https://issues.apache.org/jira/browse/SPARK-10665, + // we can first add Exchanges and then add coordinator once we have a DAG of query fragments. + children = withExchangeCoordinator(children, requiredChildDistributions) + + // Now that we've performed any necessary shuffles, add sorts to guarantee output orderings: + children = children.zip(requiredChildOrderings).map { case (child, requiredOrdering) => + if (requiredOrdering.nonEmpty) { + // If child.outputOrdering is [a, b] and requiredOrdering is [a], we do not need to sort. + if (requiredOrdering != child.outputOrdering.take(requiredOrdering.length)) { + Sort(requiredOrdering, global = false, child = child) + } else { + child + } + } else { + child + } + } + + operator.withNewChildren(children) + } + + def apply(plan: SparkPlan): SparkPlan = plan.transformUp { + case operator @ ShuffleExchange(partitioning, child, _) => + child.children match { + case ShuffleExchange(childPartitioning, baseChild, _)::Nil => + if (childPartitioning.guarantees(partitioning)) child else operator + case _ => operator + } + case operator: SparkPlan => ensureDistributionAndOrdering(operator) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala similarity index 99% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala index b096074a2fd1a..6f3bb0ad2bac1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution +package org.apache.spark.sql.execution.exchange import java.util.{HashMap => JHashMap, Map => JMap} import javax.annotation.concurrent.GuardedBy @@ -25,6 +25,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.{Logging, MapOutputStatistics, ShuffleDependency, SimpleFutureAction} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.{ShuffledRowRDD, SparkPlan} /** * A coordinator used to determines how we shuffle data between stages generated by Spark SQL. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchange.scala new file mode 100644 index 0000000000000..de21d7705e137 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchange.scala @@ -0,0 +1,261 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.exchange + +import java.util.Random + +import org.apache.spark._ +import org.apache.spark.rdd.RDD +import org.apache.spark.serializer.Serializer +import org.apache.spark.shuffle.hash.HashShuffleManager +import org.apache.spark.shuffle.sort.SortShuffleManager +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.errors._ +import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering +import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.execution._ +import org.apache.spark.util.MutablePair + +/** + * Performs a shuffle that will result in the desired `newPartitioning`. + */ +case class ShuffleExchange( + var newPartitioning: Partitioning, + child: SparkPlan, + @transient coordinator: Option[ExchangeCoordinator]) extends UnaryNode { + + override def nodeName: String = { + val extraInfo = coordinator match { + case Some(exchangeCoordinator) if exchangeCoordinator.isEstimated => + s"(coordinator id: ${System.identityHashCode(coordinator)})" + case Some(exchangeCoordinator) if !exchangeCoordinator.isEstimated => + s"(coordinator id: ${System.identityHashCode(coordinator)})" + case None => "" + } + + val simpleNodeName = "Exchange" + s"$simpleNodeName$extraInfo" + } + + override def outputPartitioning: Partitioning = newPartitioning + + override def output: Seq[Attribute] = child.output + + private val serializer: Serializer = new UnsafeRowSerializer(child.output.size) + + override protected def doPrepare(): Unit = { + // If an ExchangeCoordinator is needed, we register this Exchange operator + // to the coordinator when we do prepare. It is important to make sure + // we register this operator right before the execution instead of register it + // in the constructor because it is possible that we create new instances of + // Exchange operators when we transform the physical plan + // (then the ExchangeCoordinator will hold references of unneeded Exchanges). + // So, we should only call registerExchange just before we start to execute + // the plan. + coordinator match { + case Some(exchangeCoordinator) => exchangeCoordinator.registerExchange(this) + case None => + } + } + + /** + * Returns a [[ShuffleDependency]] that will partition rows of its child based on + * the partitioning scheme defined in `newPartitioning`. Those partitions of + * the returned ShuffleDependency will be the input of shuffle. + */ + private[sql] def prepareShuffleDependency(): ShuffleDependency[Int, InternalRow, InternalRow] = { + ShuffleExchange.prepareShuffleDependency( + child.execute(), child.output, newPartitioning, serializer) + } + + /** + * Returns a [[ShuffledRowRDD]] that represents the post-shuffle dataset. + * This [[ShuffledRowRDD]] is created based on a given [[ShuffleDependency]] and an optional + * partition start indices array. If this optional array is defined, the returned + * [[ShuffledRowRDD]] will fetch pre-shuffle partitions based on indices of this array. + */ + private[sql] def preparePostShuffleRDD( + shuffleDependency: ShuffleDependency[Int, InternalRow, InternalRow], + specifiedPartitionStartIndices: Option[Array[Int]] = None): ShuffledRowRDD = { + // If an array of partition start indices is provided, we need to use this array + // to create the ShuffledRowRDD. Also, we need to update newPartitioning to + // update the number of post-shuffle partitions. + specifiedPartitionStartIndices.foreach { indices => + assert(newPartitioning.isInstanceOf[HashPartitioning]) + newPartitioning = UnknownPartitioning(indices.length) + } + new ShuffledRowRDD(shuffleDependency, specifiedPartitionStartIndices) + } + + protected override def doExecute(): RDD[InternalRow] = attachTree(this, "execute") { + coordinator match { + case Some(exchangeCoordinator) => + val shuffleRDD = exchangeCoordinator.postShuffleRDD(this) + assert(shuffleRDD.partitions.length == newPartitioning.numPartitions) + shuffleRDD + case None => + val shuffleDependency = prepareShuffleDependency() + preparePostShuffleRDD(shuffleDependency) + } + } +} + +object ShuffleExchange { + def apply(newPartitioning: Partitioning, child: SparkPlan): ShuffleExchange = { + ShuffleExchange(newPartitioning, child, coordinator = None: Option[ExchangeCoordinator]) + } + + /** + * Determines whether records must be defensively copied before being sent to the shuffle. + * Several of Spark's shuffle components will buffer deserialized Java objects in memory. The + * shuffle code assumes that objects are immutable and hence does not perform its own defensive + * copying. In Spark SQL, however, operators' iterators return the same mutable `Row` object. In + * order to properly shuffle the output of these operators, we need to perform our own copying + * prior to sending records to the shuffle. This copying is expensive, so we try to avoid it + * whenever possible. This method encapsulates the logic for choosing when to copy. + * + * In the long run, we might want to push this logic into core's shuffle APIs so that we don't + * have to rely on knowledge of core internals here in SQL. + * + * See SPARK-2967, SPARK-4479, and SPARK-7375 for more discussion of this issue. + * + * @param partitioner the partitioner for the shuffle + * @param serializer the serializer that will be used to write rows + * @return true if rows should be copied before being shuffled, false otherwise + */ + private def needToCopyObjectsBeforeShuffle( + partitioner: Partitioner, + serializer: Serializer): Boolean = { + // Note: even though we only use the partitioner's `numPartitions` field, we require it to be + // passed instead of directly passing the number of partitions in order to guard against + // corner-cases where a partitioner constructed with `numPartitions` partitions may output + // fewer partitions (like RangePartitioner, for example). + val conf = SparkEnv.get.conf + val shuffleManager = SparkEnv.get.shuffleManager + val sortBasedShuffleOn = shuffleManager.isInstanceOf[SortShuffleManager] + val bypassMergeThreshold = conf.getInt("spark.shuffle.sort.bypassMergeThreshold", 200) + if (sortBasedShuffleOn) { + val bypassIsSupported = SparkEnv.get.shuffleManager.isInstanceOf[SortShuffleManager] + if (bypassIsSupported && partitioner.numPartitions <= bypassMergeThreshold) { + // If we're using the original SortShuffleManager and the number of output partitions is + // sufficiently small, then Spark will fall back to the hash-based shuffle write path, which + // doesn't buffer deserialized records. + // Note that we'll have to remove this case if we fix SPARK-6026 and remove this bypass. + false + } else if (serializer.supportsRelocationOfSerializedObjects) { + // SPARK-4550 and SPARK-7081 extended sort-based shuffle to serialize individual records + // prior to sorting them. This optimization is only applied in cases where shuffle + // dependency does not specify an aggregator or ordering and the record serializer has + // certain properties. If this optimization is enabled, we can safely avoid the copy. + // + // Exchange never configures its ShuffledRDDs with aggregators or key orderings, so we only + // need to check whether the optimization is enabled and supported by our serializer. + false + } else { + // Spark's SortShuffleManager uses `ExternalSorter` to buffer records in memory, so we must + // copy. + true + } + } else if (shuffleManager.isInstanceOf[HashShuffleManager]) { + // We're using hash-based shuffle, so we don't need to copy. + false + } else { + // Catch-all case to safely handle any future ShuffleManager implementations. + true + } + } + + /** + * Returns a [[ShuffleDependency]] that will partition rows of its child based on + * the partitioning scheme defined in `newPartitioning`. Those partitions of + * the returned ShuffleDependency will be the input of shuffle. + */ + private[sql] def prepareShuffleDependency( + rdd: RDD[InternalRow], + outputAttributes: Seq[Attribute], + newPartitioning: Partitioning, + serializer: Serializer): ShuffleDependency[Int, InternalRow, InternalRow] = { + val part: Partitioner = newPartitioning match { + case RoundRobinPartitioning(numPartitions) => new HashPartitioner(numPartitions) + case HashPartitioning(_, n) => + new Partitioner { + override def numPartitions: Int = n + // For HashPartitioning, the partitioning key is already a valid partition ID, as we use + // `HashPartitioning.partitionIdExpression` to produce partitioning key. + override def getPartition(key: Any): Int = key.asInstanceOf[Int] + } + case RangePartitioning(sortingExpressions, numPartitions) => + // Internally, RangePartitioner runs a job on the RDD that samples keys to compute + // partition bounds. To get accurate samples, we need to copy the mutable keys. + val rddForSampling = rdd.mapPartitionsInternal { iter => + val mutablePair = new MutablePair[InternalRow, Null]() + iter.map(row => mutablePair.update(row.copy(), null)) + } + implicit val ordering = new LazilyGeneratedOrdering(sortingExpressions, outputAttributes) + new RangePartitioner(numPartitions, rddForSampling, ascending = true) + case SinglePartition => + new Partitioner { + override def numPartitions: Int = 1 + override def getPartition(key: Any): Int = 0 + } + case _ => sys.error(s"Exchange not implemented for $newPartitioning") + // TODO: Handle BroadcastPartitioning. + } + def getPartitionKeyExtractor(): InternalRow => Any = newPartitioning match { + case RoundRobinPartitioning(numPartitions) => + // Distributes elements evenly across output partitions, starting from a random partition. + var position = new Random(TaskContext.get().partitionId()).nextInt(numPartitions) + (row: InternalRow) => { + // The HashPartitioner will handle the `mod` by the number of partitions + position += 1 + position + } + case h: HashPartitioning => + val projection = UnsafeProjection.create(h.partitionIdExpression :: Nil, outputAttributes) + row => projection(row).getInt(0) + case RangePartitioning(_, _) | SinglePartition => identity + case _ => sys.error(s"Exchange not implemented for $newPartitioning") + } + val rddWithPartitionIds: RDD[Product2[Int, InternalRow]] = { + if (needToCopyObjectsBeforeShuffle(part, serializer)) { + rdd.mapPartitionsInternal { iter => + val getPartitionKey = getPartitionKeyExtractor() + iter.map { row => (part.getPartition(getPartitionKey(row)), row.copy()) } + } + } else { + rdd.mapPartitionsInternal { iter => + val getPartitionKey = getPartitionKeyExtractor() + val mutablePair = new MutablePair[Int, InternalRow]() + iter.map { row => mutablePair.update(part.getPartition(getPartitionKey(row)), row) } + } + } + } + + // Now, we manually create a ShuffleDependency. Because pairs in rddWithPartitionIds + // are in the form of (partitionId, row) and every partitionId is in the expected range + // [0, part.numPartitions - 1]. The partitioner of this is a PartitionIdPassthrough. + val dependency = + new ShuffleDependency[Int, InternalRow, InternalRow]( + rddWithPartitionIds, + new PartitionIdPassthrough(part.numPartitions), + Some(serializer)) + + dependency + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala index a9aee8ba494b0..cd543d4195286 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.execution.exchange.ShuffleExchange /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index f28632201a11c..83d7953aaf700 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -24,8 +24,8 @@ import org.scalatest.concurrent.Eventually._ import org.apache.spark.Accumulators import org.apache.spark.sql.execution.PhysicalRDD -import org.apache.spark.sql.execution.ShuffleExchange import org.apache.spark.sql.execution.columnar._ +import org.apache.spark.sql.execution.exchange.ShuffleExchange import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} import org.apache.spark.storage.{RDDBlockId, StorageLevel} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 65e0deada5f6f..50a246489ee55 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -26,8 +26,8 @@ import org.scalatest.Matchers._ import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.plans.logical.{OneRowRelation, Union} -import org.apache.spark.sql.execution.ShuffleExchange import org.apache.spark.sql.execution.aggregate.TungstenAggregate +import org.apache.spark.sql.execution.exchange.ShuffleExchange import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSQLContext} import org.apache.spark.sql.test.SQLTestData.TestData2 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala index 3d4080784b8c1..b1c588a63d030 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala @@ -21,6 +21,7 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.{MapOutputStatistics, SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.sql._ +import org.apache.spark.sql.execution.exchange.{ExchangeCoordinator, ShuffleExchange} import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.TestSQLContext diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala index c54f094a12a08..d4f22de90c523 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.plans.physical.SinglePartition +import org.apache.spark.sql.execution.exchange.ShuffleExchange import org.apache.spark.sql.test.SharedSQLContext class ExchangeSuite extends SparkPlanTest with SharedSQLContext { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index c19a945097e9f..4de56783fabce 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, Literal, import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Repartition} import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.columnar.{InMemoryColumnarTableScan, InMemoryRelation} +import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ShuffleExchange} import org.apache.spark.sql.execution.joins.{BroadcastHashJoin, SortMergeJoin} import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala index 2738ca448cd68..a256ee95a153c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala @@ -23,7 +23,7 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.{AccumulatorSuite, SparkConf, SparkContext} import org.apache.spark.sql.{QueryTest, SQLContext} -import org.apache.spark.sql.execution.EnsureRequirements +import org.apache.spark.sql.execution.exchange.EnsureRequirements import org.apache.spark.sql.functions._ /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala index 4a2a4ec1d4639..6dfff3770b882 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys import org.apache.spark.sql.catalyst.plans.Inner import org.apache.spark.sql.catalyst.plans.logical.Join import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.exchange.EnsureRequirements import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{IntegerType, StringType, StructType} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala index f4b01fbad0585..cd6b6fcbb18af 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala @@ -22,7 +22,8 @@ import org.apache.spark.sql.catalyst.expressions.{And, Expression, LessThan} import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.Join -import org.apache.spark.sql.execution.{EnsureRequirements, SparkPlan, SparkPlanTest} +import org.apache.spark.sql.execution.{SparkPlan, SparkPlanTest} +import org.apache.spark.sql.execution.exchange.EnsureRequirements import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{DoubleType, IntegerType, StructType} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/SemiJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/SemiJoinSuite.scala index 9c86084f9b8a9..f3ad8409e5a3d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/SemiJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/SemiJoinSuite.scala @@ -22,7 +22,8 @@ import org.apache.spark.sql.catalyst.expressions.{And, Expression, LessThan} import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys import org.apache.spark.sql.catalyst.plans.Inner import org.apache.spark.sql.catalyst.plans.logical.Join -import org.apache.spark.sql.execution.{EnsureRequirements, SparkPlan, SparkPlanTest} +import org.apache.spark.sql.execution.{SparkPlan, SparkPlanTest} +import org.apache.spark.sql.execution.exchange.EnsureRequirements import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{DoubleType, IntegerType, StructType} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index 615be729f84f3..a05a57c0f5072 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -22,8 +22,9 @@ import java.io.File import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning -import org.apache.spark.sql.execution.{PhysicalRDD, ShuffleExchange} +import org.apache.spark.sql.execution.PhysicalRDD import org.apache.spark.sql.execution.datasources.{BucketSpec, DataSourceStrategy} +import org.apache.spark.sql.execution.exchange.ShuffleExchange import org.apache.spark.sql.execution.joins.SortMergeJoin import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive.test.TestHiveSingleton From 28363c82ffad939175b233d71089747c3ab76280 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Sun, 21 Feb 2016 15:25:41 +0100 Subject: [PATCH 17/19] CR --- .../spark/sql/execution/exchange/BroadcastExchange.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchange.scala index 9e1574c2c009b..20d998ee8bac1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchange.scala @@ -40,7 +40,8 @@ case class BroadcastExchange( override def outputPartitioning: Partitioning = BroadcastPartitioning(mode) - val timeout: Duration = { + @transient + private val timeout: Duration = { val timeoutValue = sqlContext.conf.broadcastTimeout if (timeoutValue < 0) { Duration.Inf @@ -75,7 +76,8 @@ case class BroadcastExchange( } override protected def doExecute(): RDD[InternalRow] = { - throw new UnsupportedOperationException("Broadcast does not support the execute() code path.") + throw new UnsupportedOperationException( + "BroadcastExchange does not support the execute() code path.") } override protected[sql] def doExecuteBroadcast[T](): broadcast.Broadcast[T] = { From 4b5978b57552083d701a431c4de9145aca86d677 Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Sun, 21 Feb 2016 15:54:04 +0100 Subject: [PATCH 18/19] put broadcast mode in a separate file. --- .../plans/physical/broadcastMode.scala | 35 +++++++++++++++++++ .../plans/physical/partitioning.scala | 15 -------- 2 files changed, 35 insertions(+), 15 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/broadcastMode.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/broadcastMode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/broadcastMode.scala new file mode 100644 index 0000000000000..c646dcfa11811 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/broadcastMode.scala @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.plans.physical + +import org.apache.spark.sql.catalyst.InternalRow + +/** + * Marker trait to identify the shape in which tuples are broadcasted. Typical examples of this are + * identity (tuples remain unchanged) or hashed (tuples are converted into some hash index). + */ +trait BroadcastMode { + def transform(rows: Array[InternalRow]): Any +} + +/** + * IdentityBroadcastMode requires that rows are broadcasted in their original form. + */ +case object IdentityBroadcastMode extends BroadcastMode { + override def transform(rows: Array[InternalRow]): Array[InternalRow] = rows +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index 61dc0fef4c4fc..45e2841ec9dd3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -76,21 +76,6 @@ case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { def clustering: Set[Expression] = ordering.map(_.child).toSet } -/** - * Marker trait to identify the shape in which tuples are broadcasted. Typical examples of this are - * identity (tuples remain unchanged) or hashed (tuples are converted into some hash index). - */ -trait BroadcastMode { - def transform(rows: Array[InternalRow]): Any -} - -/** - * IdentityBroadcastMode requires that rows are broadcasted in their original form. - */ -case object IdentityBroadcastMode extends BroadcastMode { - override def transform(rows: Array[InternalRow]): Array[InternalRow] = rows -} - /** * Represents data where tuples are broadcasted to every node. It is quite common that the * entire set of tuples is transformed into different data structure. From c8c175e91ad2896573a4d6efab9ee13d7f28103c Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Sun, 21 Feb 2016 16:02:22 +0100 Subject: [PATCH 19/19] Fix style in sqlcontext. --- .../scala/org/apache/spark/sql/SQLContext.scala | 14 +++----------- 1 file changed, 3 insertions(+), 11 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index cf9b8476778f4..a2f386850c1b5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -305,7 +305,6 @@ class SQLContext private[sql]( /** * Returns true if the table is currently cached in-memory. - * * @group cachemgmt * @since 1.3.0 */ @@ -314,18 +313,16 @@ class SQLContext private[sql]( } /** - * Returns true if the [[Queryable]] is currently cached in-memory. - * + * Returns true if the [[Queryable]] is currently cached in-memory. * @group cachemgmt - * @since 1.3.0 - */ + * @since 1.3.0 + */ private[sql] def isCached(qName: Queryable): Boolean = { cacheManager.lookupCachedData(qName).nonEmpty } /** * Caches the specified table in-memory. - * * @group cachemgmt * @since 1.3.0 */ @@ -335,7 +332,6 @@ class SQLContext private[sql]( /** * Removes the specified table from the in-memory cache. - * * @group cachemgmt * @since 1.3.0 */ @@ -343,7 +339,6 @@ class SQLContext private[sql]( /** * Removes all cached tables from the in-memory cache. - * * @since 1.3.0 */ def clearCache(): Unit = cacheManager.clearCache() @@ -545,7 +540,6 @@ class SQLContext private[sql]( * * WARNING: Since there is no guaranteed ordering for fields in a Java Bean, * SELECT * queries will return the columns in an undefined order. - * * @group dataframes * @since 1.3.0 */ @@ -565,7 +559,6 @@ class SQLContext private[sql]( * * WARNING: Since there is no guaranteed ordering for fields in a Java Bean, * SELECT * queries will return the columns in an undefined order. - * * @group dataframes * @since 1.3.0 */ @@ -578,7 +571,6 @@ class SQLContext private[sql]( * * WARNING: Since there is no guaranteed ordering for fields in a Java Bean, * SELECT * queries will return the columns in an undefined order. - * * @group dataframes * @since 1.6.0 */