-
Notifications
You must be signed in to change notification settings - Fork 28.1k
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 2 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,120 @@ | ||
/* | ||
* 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.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 = new GeneralHashedRelation( | ||
buildHashTable(input.iterator, newProjection(buildKeys, buildPlan.output))) | ||
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. 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. @marmbrus Yeah, that's why I used GeneralHashedRelation to wrap the hash table. Which way do you think is better? 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. This still fails for me when I run it on a real cluster. I'd just change this to 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. Sure |
||
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)) | ||
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 would probably be reasonable to have a single threadpool that we share for all broadcasting. |
||
} |
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.
can you use CanBroadcast here