-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-13136][SQL] Create a dedicated Broadcast exchange operator #11083
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
aa7120e
c2b7533
6a5568a
9adecdd
d0194fb
02a61b8
c12c8e6
c7dd7ae
e847383
d73f11c
9c0f4bf
da4a966
681f347
7db240a
3ad839d
1116768
a5501cf
c7429bb
b12bbc2
9d52650
54b558d
f33d2cb
28363c8
f812a31
4b5978b
c8c175e
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -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 | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -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(mode: BroadcastMode) extends Distribution | ||
|
|
||
| /** | ||
| * Describes how an operator's output is split across partitions. The `compatibleWith`, | ||
| * `guarantees`, and `satisfies` methods describe relationships between child partitionings, | ||
|
|
@@ -213,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 | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. i think this is ok for now, but technically we don't need to introduce an exchange if both sides of the join have only one partition. i guess this framework does not currently handle that. |
||
| case _ => true | ||
| } | ||
|
|
||
| override def compatibleWith(other: Partitioning): Boolean = other.numPartitions == 1 | ||
|
|
||
|
|
@@ -351,3 +361,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 { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. and partitioning here isn't really just about partitioning anymore. it also includes how data is physically organized on each partition. |
||
| 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 | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,89 @@ | ||
| /* | ||
| * 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) | ||
|
|
||
| @transient | ||
| private 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 .executeCollect() because we don't want to convert data to Scala types | ||
| val input: Array[InternalRow] = child.executeCollect() | ||
|
|
||
| // 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( | ||
| "BroadcastExchange 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)) | ||
| } |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ok for now, but technically once we introduce this, "Distribution" is not just about "distribution" anymore, but rather some data property.