-
Notifications
You must be signed in to change notification settings - Fork 28.5k
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
[SPARK-4485][SQL] (1) Add broadcast hash outer join, (2) Fix SparkPlanTest #7162
Changes from all commits
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,121 @@ | ||
/* | ||
* 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.joins | ||
|
||
import org.apache.spark.annotation.DeveloperApi | ||
import org.apache.spark.rdd.RDD | ||
import org.apache.spark.sql.catalyst.expressions._ | ||
import org.apache.spark.sql.catalyst.plans.physical.{Distribution, UnspecifiedDistribution} | ||
import org.apache.spark.sql.catalyst.plans.{JoinType, LeftOuter, RightOuter} | ||
import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} | ||
import org.apache.spark.util.ThreadUtils | ||
|
||
import scala.collection.JavaConversions._ | ||
import scala.concurrent._ | ||
import scala.concurrent.duration._ | ||
|
||
/** | ||
* :: DeveloperApi :: | ||
* Performs a outer hash join for two child relations. When the output RDD of this operator is | ||
* being 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 BroadcastHashOuterJoin( | ||
leftKeys: Seq[Expression], | ||
rightKeys: Seq[Expression], | ||
joinType: JoinType, | ||
condition: Option[Expression], | ||
left: SparkPlan, | ||
right: SparkPlan) extends BinaryNode with HashOuterJoin { | ||
|
||
val timeout = { | ||
val timeoutValue = sqlContext.conf.broadcastTimeout | ||
if (timeoutValue < 0) { | ||
Duration.Inf | ||
} else { | ||
timeoutValue.seconds | ||
} | ||
} | ||
|
||
override def requiredChildDistribution: Seq[Distribution] = | ||
UnspecifiedDistribution :: UnspecifiedDistribution :: Nil | ||
|
||
private[this] lazy val (buildPlan, streamedPlan) = joinType match { | ||
case RightOuter => (left, right) | ||
case LeftOuter => (right, left) | ||
case x => | ||
throw new IllegalArgumentException( | ||
s"BroadcastHashOuterJoin should not take $x as the JoinType") | ||
} | ||
|
||
private[this] lazy val (buildKeys, streamedKeys) = joinType match { | ||
case RightOuter => (leftKeys, rightKeys) | ||
case LeftOuter => (rightKeys, leftKeys) | ||
case x => | ||
throw new IllegalArgumentException( | ||
s"BroadcastHashOuterJoin should not take $x as the JoinType") | ||
} | ||
|
||
@transient | ||
private val broadcastFuture = future { | ||
// Note that we use .execute().collect() because we don't want to convert data to Scala types | ||
val input: Array[InternalRow] = buildPlan.execute().map(_.copy()).collect() | ||
// buildHashTable uses code-generated rows as keys, which are not serializable | ||
val hashed = | ||
buildHashTable(input.iterator, new InterpretedProjection(buildKeys, buildPlan.output)) | ||
sparkContext.broadcast(hashed) | ||
}(BroadcastHashOuterJoin.broadcastHashOuterJoinExecutionContext) | ||
|
||
override def doExecute(): RDD[InternalRow] = { | ||
val broadcastRelation = Await.result(broadcastFuture, timeout) | ||
|
||
streamedPlan.execute().mapPartitions { streamedIter => | ||
val joinedRow = new JoinedRow() | ||
val hashTable = broadcastRelation.value | ||
val keyGenerator = newProjection(streamedKeys, streamedPlan.output) | ||
|
||
joinType match { | ||
case LeftOuter => | ||
streamedIter.flatMap(currentRow => { | ||
val rowKey = keyGenerator(currentRow) | ||
joinedRow.withLeft(currentRow) | ||
leftOuterIterator(rowKey, joinedRow, hashTable.getOrElse(rowKey, EMPTY_LIST)) | ||
}) | ||
|
||
case RightOuter => | ||
streamedIter.flatMap(currentRow => { | ||
val rowKey = keyGenerator(currentRow) | ||
joinedRow.withRight(currentRow) | ||
rightOuterIterator(rowKey, hashTable.getOrElse(rowKey, EMPTY_LIST), joinedRow) | ||
}) | ||
|
||
case x => | ||
throw new IllegalArgumentException( | ||
s"BroadcastHashOuterJoin should not take $x as the JoinType") | ||
} | ||
} | ||
} | ||
} | ||
|
||
object BroadcastHashOuterJoin { | ||
|
||
private val broadcastHashOuterJoinExecutionContext = ExecutionContext.fromExecutorService( | ||
ThreadUtils.newDaemonCachedThreadPool("broadcast-hash-outer-join", 128)) | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,85 @@ | ||
/* | ||
* 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.joins | ||
|
||
import org.apache.spark.annotation.DeveloperApi | ||
import org.apache.spark.rdd.RDD | ||
import org.apache.spark.sql.catalyst.expressions._ | ||
import org.apache.spark.sql.catalyst.plans.physical.{Distribution, ClusteredDistribution} | ||
import org.apache.spark.sql.catalyst.plans.{FullOuter, JoinType, LeftOuter, RightOuter} | ||
import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} | ||
|
||
import scala.collection.JavaConversions._ | ||
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. @kai-zeng I'm in the process of refactoring some parts of this file as part of another PR of mine and wanted to briefly call out two minor performance considerations that I noticed while looking at this code. The first is the use of 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. I agree with your first concern. It is really necessary to refactor the outer join as now we have both shuffled and broadcast outer join. |
||
|
||
/** | ||
* :: DeveloperApi :: | ||
* Performs a hash based outer join for two child relations by shuffling the data using | ||
* the join keys. This operator requires loading the associated partition in both side into memory. | ||
*/ | ||
@DeveloperApi | ||
case class ShuffledHashOuterJoin( | ||
leftKeys: Seq[Expression], | ||
rightKeys: Seq[Expression], | ||
joinType: JoinType, | ||
condition: Option[Expression], | ||
left: SparkPlan, | ||
right: SparkPlan) extends BinaryNode with HashOuterJoin { | ||
|
||
override def requiredChildDistribution: Seq[Distribution] = | ||
ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil | ||
|
||
protected override def doExecute(): RDD[InternalRow] = { | ||
val joinedRow = new JoinedRow() | ||
left.execute().zipPartitions(right.execute()) { (leftIter, rightIter) => | ||
// TODO this probably can be replaced by external sort (sort merged join?) | ||
joinType match { | ||
case LeftOuter => | ||
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. Second possible perf. concern: I think that the 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. Here joinType matching is performed once per partition, not once per row. So I don't think it is a big issue. |
||
val rightHashTable = buildHashTable(rightIter, newProjection(rightKeys, right.output)) | ||
val keyGenerator = newProjection(leftKeys, left.output) | ||
leftIter.flatMap( currentRow => { | ||
val rowKey = keyGenerator(currentRow) | ||
joinedRow.withLeft(currentRow) | ||
leftOuterIterator(rowKey, joinedRow, rightHashTable.getOrElse(rowKey, EMPTY_LIST)) | ||
}) | ||
|
||
case RightOuter => | ||
val leftHashTable = buildHashTable(leftIter, newProjection(leftKeys, left.output)) | ||
val keyGenerator = newProjection(rightKeys, right.output) | ||
rightIter.flatMap ( currentRow => { | ||
val rowKey = keyGenerator(currentRow) | ||
joinedRow.withRight(currentRow) | ||
rightOuterIterator(rowKey, leftHashTable.getOrElse(rowKey, EMPTY_LIST), joinedRow) | ||
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. It looks like Davies ended up cleaning up these instances of the MapWrapper issue as part of one of his Tungsten PRs and I'll fix the remaining FullOuter case as part of my PR. |
||
}) | ||
|
||
case FullOuter => | ||
val leftHashTable = buildHashTable(leftIter, newProjection(leftKeys, left.output)) | ||
val rightHashTable = buildHashTable(rightIter, newProjection(rightKeys, right.output)) | ||
(leftHashTable.keySet ++ rightHashTable.keySet).iterator.flatMap { key => | ||
fullOuterIterator(key, | ||
leftHashTable.getOrElse(key, EMPTY_LIST), | ||
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. Here, doing an explicit Java-style null check on the Java HashMaps would avoid the cost of allocating a MapWrapper for each row. |
||
rightHashTable.getOrElse(key, EMPTY_LIST), | ||
joinedRow) | ||
} | ||
|
||
case x => | ||
throw new IllegalArgumentException( | ||
s"ShuffledHashOuterJoin should not take $x as the JoinType") | ||
} | ||
} | ||
} | ||
} |
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 would probably be reasonable to have a single threadpool that we share for all broadcasting.