-
Notifications
You must be signed in to change notification settings - Fork 29k
[SQL] SPARK-1800 Add broadcast hash join operator #734
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
a8420ca
cf6b381
76ca434
a92ed0c
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 |
|---|---|---|
|
|
@@ -18,13 +18,16 @@ | |
| package org.apache.spark.sql.execution | ||
|
|
||
| import scala.collection.mutable.{ArrayBuffer, BitSet} | ||
| import scala.concurrent._ | ||
| import scala.concurrent.duration._ | ||
| import scala.concurrent.ExecutionContext.Implicits.global | ||
|
|
||
| import org.apache.spark.SparkContext | ||
|
|
||
| import org.apache.spark.annotation.DeveloperApi | ||
| import org.apache.spark.sql.catalyst.expressions._ | ||
| import org.apache.spark.sql.catalyst.plans._ | ||
| import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Partitioning} | ||
| import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Partitioning, UnspecifiedDistribution} | ||
|
|
||
| @DeveloperApi | ||
| sealed abstract class BuildSide | ||
|
|
@@ -35,21 +38,13 @@ case object BuildLeft extends BuildSide | |
| @DeveloperApi | ||
| case object BuildRight extends BuildSide | ||
|
|
||
| /** | ||
| * :: DeveloperApi :: | ||
| */ | ||
| @DeveloperApi | ||
| case class HashJoin( | ||
| leftKeys: Seq[Expression], | ||
| rightKeys: Seq[Expression], | ||
| buildSide: BuildSide, | ||
| left: SparkPlan, | ||
| right: SparkPlan) extends BinaryNode { | ||
|
|
||
| override def outputPartitioning: Partitioning = left.outputPartitioning | ||
|
|
||
| override def requiredChildDistribution = | ||
| ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil | ||
| trait HashJoin { | ||
| val leftKeys: Seq[Expression] | ||
| val rightKeys: Seq[Expression] | ||
| val buildSide: BuildSide | ||
| val left: SparkPlan | ||
| val right: SparkPlan | ||
|
|
||
| val (buildPlan, streamedPlan) = buildSide match { | ||
| case BuildLeft => (left, right) | ||
|
|
@@ -67,79 +62,140 @@ case class HashJoin( | |
| @transient lazy val streamSideKeyGenerator = | ||
| () => new MutableProjection(streamedKeys, streamedPlan.output) | ||
|
|
||
| def execute() = { | ||
|
|
||
| buildPlan.execute().zipPartitions(streamedPlan.execute()) { (buildIter, streamIter) => | ||
| // TODO: Use Spark's HashMap implementation. | ||
| val hashTable = new java.util.HashMap[Row, ArrayBuffer[Row]]() | ||
| var currentRow: Row = null | ||
|
|
||
| // Create a mapping of buildKeys -> rows | ||
| while (buildIter.hasNext) { | ||
| currentRow = buildIter.next() | ||
| val rowKey = buildSideKeyGenerator(currentRow) | ||
| if(!rowKey.anyNull) { | ||
| val existingMatchList = hashTable.get(rowKey) | ||
| val matchList = if (existingMatchList == null) { | ||
| val newMatchList = new ArrayBuffer[Row]() | ||
| hashTable.put(rowKey, newMatchList) | ||
| newMatchList | ||
| } else { | ||
| existingMatchList | ||
| } | ||
| matchList += currentRow.copy() | ||
| def joinIterators(buildIter: Iterator[Row], streamIter: Iterator[Row]): Iterator[Row] = { | ||
| // TODO: Use Spark's HashMap implementation. | ||
|
|
||
| val hashTable = new java.util.HashMap[Row, ArrayBuffer[Row]]() | ||
| var currentRow: Row = null | ||
|
|
||
| // Create a mapping of buildKeys -> rows | ||
| while (buildIter.hasNext) { | ||
| currentRow = buildIter.next() | ||
| val rowKey = buildSideKeyGenerator(currentRow) | ||
| if(!rowKey.anyNull) { | ||
| val existingMatchList = hashTable.get(rowKey) | ||
| val matchList = if (existingMatchList == null) { | ||
| val newMatchList = new ArrayBuffer[Row]() | ||
| hashTable.put(rowKey, newMatchList) | ||
| newMatchList | ||
| } else { | ||
| existingMatchList | ||
| } | ||
| matchList += currentRow.copy() | ||
| } | ||
| } | ||
|
|
||
| new Iterator[Row] { | ||
| private[this] var currentStreamedRow: Row = _ | ||
| private[this] var currentHashMatches: ArrayBuffer[Row] = _ | ||
| private[this] var currentMatchPosition: Int = -1 | ||
| new Iterator[Row] { | ||
| private[this] var currentStreamedRow: Row = _ | ||
| private[this] var currentHashMatches: ArrayBuffer[Row] = _ | ||
| private[this] var currentMatchPosition: Int = -1 | ||
|
|
||
| // Mutable per row objects. | ||
| private[this] val joinRow = new JoinedRow | ||
| // Mutable per row objects. | ||
| private[this] val joinRow = new JoinedRow | ||
|
|
||
| private[this] val joinKeys = streamSideKeyGenerator() | ||
| private[this] val joinKeys = streamSideKeyGenerator() | ||
|
|
||
| override final def hasNext: Boolean = | ||
| (currentMatchPosition != -1 && currentMatchPosition < currentHashMatches.size) || | ||
| override final def hasNext: Boolean = | ||
| (currentMatchPosition != -1 && currentMatchPosition < currentHashMatches.size) || | ||
| (streamIter.hasNext && fetchNext()) | ||
|
|
||
| override final def next() = { | ||
| val ret = joinRow(currentStreamedRow, currentHashMatches(currentMatchPosition)) | ||
| currentMatchPosition += 1 | ||
| ret | ||
| } | ||
| override final def next() = { | ||
| val ret = joinRow(currentStreamedRow, currentHashMatches(currentMatchPosition)) | ||
| currentMatchPosition += 1 | ||
| ret | ||
| } | ||
|
|
||
| /** | ||
| * Searches the streamed iterator for the next row that has at least one match in hashtable. | ||
| * | ||
| * @return true if the search is successful, and false the streamed iterator runs out of | ||
| * tuples. | ||
| */ | ||
| private final def fetchNext(): Boolean = { | ||
| currentHashMatches = null | ||
| currentMatchPosition = -1 | ||
|
|
||
| while (currentHashMatches == null && streamIter.hasNext) { | ||
| currentStreamedRow = streamIter.next() | ||
| if (!joinKeys(currentStreamedRow).anyNull) { | ||
| currentHashMatches = hashTable.get(joinKeys.currentValue) | ||
| } | ||
| /** | ||
| * Searches the streamed iterator for the next row that has at least one match in hashtable. | ||
| * | ||
| * @return true if the search is successful, and false the streamed iterator runs out of | ||
| * tuples. | ||
| */ | ||
| private final def fetchNext(): Boolean = { | ||
| currentHashMatches = null | ||
| currentMatchPosition = -1 | ||
|
|
||
| while (currentHashMatches == null && streamIter.hasNext) { | ||
| currentStreamedRow = streamIter.next() | ||
| if (!joinKeys(currentStreamedRow).anyNull) { | ||
| currentHashMatches = hashTable.get(joinKeys.currentValue) | ||
| } | ||
| } | ||
|
|
||
| if (currentHashMatches == null) { | ||
| false | ||
| } else { | ||
| currentMatchPosition = 0 | ||
| true | ||
| } | ||
| if (currentHashMatches == null) { | ||
| false | ||
| } else { | ||
| currentMatchPosition = 0 | ||
| true | ||
| } | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * :: DeveloperApi :: | ||
| * Performs and inner hash join of two child relations by first shuffling the data using the join | ||
| * keys. | ||
| */ | ||
| @DeveloperApi | ||
| case class ShuffledHashJoin( | ||
| leftKeys: Seq[Expression], | ||
| rightKeys: Seq[Expression], | ||
| buildSide: BuildSide, | ||
| left: SparkPlan, | ||
| right: SparkPlan) extends BinaryNode with HashJoin { | ||
|
|
||
| override def outputPartitioning: Partitioning = left.outputPartitioning | ||
|
|
||
| override def requiredChildDistribution = | ||
| ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil | ||
|
|
||
|
|
||
| def execute() = { | ||
| buildPlan.execute().zipPartitions(streamedPlan.execute()) { | ||
| (buildIter, streamIter) => joinIterators(buildIter, streamIter) | ||
| } | ||
| } | ||
| } | ||
|
|
||
|
|
||
| /** | ||
| * :: DeveloperApi :: | ||
| * Performs an inner hash join of two child relations. When the operator is constructed, a Spark | ||
| * job is asynchronously started to calculate the values for the broadcasted relation. This data | ||
| * is then placed in a Spark broadcast variable. The streamed relation is not shuffled. | ||
| */ | ||
| @DeveloperApi | ||
| case class BroadcastHashJoin( | ||
| leftKeys: Seq[Expression], | ||
| rightKeys: Seq[Expression], | ||
| buildSide: BuildSide, | ||
| left: SparkPlan, | ||
| right: SparkPlan)(@transient sc: SparkContext) extends BinaryNode with HashJoin { | ||
|
|
||
| override def otherCopyArgs = sc :: Nil | ||
|
|
||
| override def outputPartitioning: Partitioning = left.outputPartitioning | ||
|
|
||
| override def requiredChildDistribution = | ||
| UnspecifiedDistribution :: UnspecifiedDistribution :: Nil | ||
|
|
||
| @transient | ||
| lazy val broadcastFuture = future { | ||
| sc.broadcast(buildPlan.executeCollect()) | ||
|
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. Hi, will you plan to clean up broadcast variables after the operation or leave it in the context?
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. In Spark 1.0, with the newly added garbage collection mechanism, when the query plan itself goes out of scope, the broadcast variable should also be cleaned automatically. Another way we can do this is to have some query context object we pass around the entire physical query plan which tracks the stuff we need to clean up. 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. Hi Reynold, thanks for the reply. Does spark has a plan to port this PR in to the repo?
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. We definitely want to merge this PR (assuming you are talking about the broadcast hash join PR). 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. Yep, the broadcast join. We were experiencing the perf problem when join between a big table with a small table. Look forward to the merge. Do you know when it will approximately be, assuming it goes to 1.1.0?
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. 1.0 is already going through voting now so this won't make it into 1.0. It will be in 1.0.1/1.1; However, if you need this functionality, you can just cherry pick this pull request and do a custom build. 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. Good to know. Thanks for the headsup |
||
| } | ||
|
|
||
| def execute() = { | ||
| val broadcastRelation = Await.result(broadcastFuture, 5.minute) | ||
|
|
||
| streamedPlan.execute().mapPartitions { streamedIter => | ||
| joinIterators(broadcastRelation.value.iterator, streamedIter) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * :: DeveloperApi :: | ||
| */ | ||
|
|
||
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.
The bodies of
BroadcastHashJoinand ofHashJoindo not strictly referencebroadcastFuture, right? If so, the Spark job isn't launched during the constructor?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.
It is only run on Line 191 during execute.
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.
Yep, we should update the comment "When the operator is constructed" then.
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.
Yeah i guess it should be when the RDD is constructed.