-
Notifications
You must be signed in to change notification settings - Fork 28.4k
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-32421][SQL] Add code-gen for shuffled hash join #29277
Conversation
cc @cloud-fan and @sameeragarwal if you guys can help take a look, thanks! |
relationTerm = ctx.addMutableState( | ||
"org.apache.spark.sql.execution.joins.HashedRelation", "relation", forceInline = true) |
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.
As you already use mutable state for the hashed relation here, why don't just follow BroadcastHashJoinExec to call buildHashedRelation at prepareRelation and set it to mutable state? Then BroadcastHashJoinExec and ShuffledHashJoinExec look more consistent.
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.
@viirya - BroadcastHashJoinExec
needs to broadcast build side in prepareBroadcast()
. I feel it's hard to refactor there. Wondering do you have any idea to make it cleaner?
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.
I have the same feeling with @viirya .
Can we broadcast build side in BroadcastHashJoinExec.prepareRelation
?
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.
We can always call prepareRelation
in doProduce
as the first step
override def doProduce(ctx: CodegenContext): String = {
prepareRelation(ctx) // which sets 2 vars: relationTerm and isUniqueKey
streamedPlan.asInstanceOf[CodegenSupport].produce(ctx, this)
}
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.
Sorry I may not understand the real problem. What's wrong with
override def doProduce(ctx: CodegenContext): String = {
streamedPlan.asInstanceOf[CodegenSupport].produce(ctx, this)
}
override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = {
joinType match {
case _: InnerLike => codegenInner(ctx, input)
case LeftOuter | RightOuter => codegenOuter(ctx, input)
case LeftSemi => codegenSemi(ctx, input)
case LeftAnti => codegenAnti(ctx, input)
case j: ExistenceJoin => codegenExistence(ctx, input)
case x =>
throw new IllegalArgumentException(
s"BroadcastHashJoin 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.
@cloud-fan and @viirya -
Updated code with following change:
(1). ShuffledHashJoinExec.prepareRelation
will do buildRelation
to build hash map.
(1).BroadcastHashJoinExec.prepareRelation
will do prepareBroadcast
to broadcast build side and build hash map.
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.
re #29277 (comment):
wondering do you think keeping two vars would look much better, than the current approach for calling prepareRelation
separately in each codegenInner/codegenOuter/...
? If yes, I can make the change accordingly, thanks.
re #29277 (comment):
By design, doConsume()
generates code for processing one input row. BroadcastHashJoinExec
can do codegen work in doConsume()
with only stream side input, because it just broadcasts executing its build side query plan, and generates per-row-processing codegen for stream side in doConsume()
.
However, ShuffledHashJoinExec
cannot do codegen work in doConsume()
with stream and build side input, because it needs to first read all build side input rows and build a hash map, before processing each row from stream side input. We cannot generate code in doConsume()
with simply a pair of stream and build side input row. Similar to SortMergeJoinExec
, where it needs to stream only one row in one side, and buffer the other side for multiple rows. ShuffledHashJoinExec
has to generate code in doProduce()
, and its children have to do codegen separately in their own iterator classes.
|$isNull = $row.isNullAt($i); | ||
|$value = $isNull ? $defaultValue : ($valueCode); | ||
""".stripMargin | ||
val leftVarsDecl = |
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.
Since you remove left concept, we better clean up these leftXXX variables too.
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.
@viirya - sorry for missing this, done.
* the variables should be declared separately from accessing the columns, we can't use the | ||
* codegen of BoundReference here. | ||
*/ | ||
protected def createVars( |
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.
Original createLeftVars
is created to defer accessing of row fields after condition evaluation. But I look at the usage of this createVars
in HashJoin
, I don't see such thing. If you don't do defer there, you can simply use BoundReference
codegen, it is much simpler.
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.
@viirya - I was actually originally using BoundReference
, but got compilation error with variable redefinition. E.g., for code branch c21:codegen-fail
(with change to BoundReference
compare to this PR), and example query in JoinBenchmark
.
The BoundReference
version generated code like this:
while (shj_streamedInput_0.hasNext()) {
shj_streamedRow_0 = (InternalRow) shj_streamedInput_0.next();
// generate join key for stream side
long shj_value_0 = shj_streamedRow_0.getLong(0); // 1st definition here
// find matches from HashRelation
scala.collection.Iterator shj_matches_0 = false ? null : (scala.collection.Iterator)shj_relation_0.get(shj_value_0);
while (shj_matches_0.hasNext()) {
UnsafeRow shj_matched_0 = (UnsafeRow) shj_matches_0.next();
long shj_value_0 = shj_streamedRow_0.getLong(0); // 2nd definition here and compilation error
shj_mutableStateArray_0[0].write(0, shj_value_0);
}
}
So basically the variable shj_value_0
here (stream side key ) needs to be defer accessed in HashJoin.consume()
, and it was first accessed in HashJoin.genStreamSideJoinKey()
. So it seems that BoundReference
not work for me out of box.
Let me know if it makes sense or there would be any other better approach, thanks.
Test build #126737 has finished for PR 29277 at commit
|
@@ -316,6 +318,387 @@ trait HashJoin extends BaseJoinExec { | |||
resultProj(r) | |||
} | |||
} | |||
|
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 PR comments to highlight the real changes? Seems most of the diff is just moving code around.
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.
@cloud-fan - sorry about that. yes mostly of them is moving code around without change. Highlighted change with comments, thanks.
val (matched, checkCondition, _) = getJoinCondition(ctx, input) | ||
val numOutput = metricTerm(ctx, "numOutputRows") | ||
|
||
protected override def codegenAnti(ctx: CodegenContext, input: Seq[ExprCode]): String = { |
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.
codegenAnti
is changed to keep NULL-aware anti join separately here, and move other logic to HashJoin.codegenAnti
.
* Generates the code for left semi join. | ||
*/ | ||
private def codegenSemi(ctx: CodegenContext, input: Seq[ExprCode]): String = { | ||
protected override def prepareRelation(ctx: CodegenContext): (String, Boolean) = { |
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.
a new method prepareRelation
is added to call prepareBroadcast()
and get to know whether the key is known to be unique during codegen time.
@@ -23,6 +23,7 @@ import org.apache.spark.TaskContext | |||
import org.apache.spark.rdd.RDD |
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.
all change in ShuffledHashJoinExec
here are real change, not refactoring.
* the variables should be declared separately from accessing the columns, we can't use the | ||
* codegen of BoundReference here. | ||
*/ | ||
protected def createVars( |
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.
createVars
is copied from SortMergeJoinExec.createLeftVars()
to be usable from SortMergeJoinExec
and ShuffledHashJoinExec
for generating code for stream side input.
* Returns the code for generating join key for stream side, and expression of whether the key | ||
* has any null in it or not. | ||
*/ | ||
protected def genStreamSideJoinKey( |
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.
this method is copied from BroadcastHashJoinExec
without change.
val (matched, checkCondition, _) = getJoinCondition(ctx, input) | ||
val numOutput = metricTerm(ctx, "numOutputRows") | ||
|
||
if (keyIsKnownUnique) { |
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.
added keyIsKnownUnique
to support unique-key code path for BroadcastHashJoinExec
.
val (matched, checkCondition, _) = getJoinCondition(ctx, input) | ||
val numOutput = metricTerm(ctx, "numOutputRows") | ||
|
||
if (keyIsKnownUnique) { |
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.
added keyIsKnownUnique
to support unique-key code path for BroadcastHashJoinExec
.
val resultVar = input ++ Seq(ExprCode.forNonNullValue( | ||
JavaCode.variable(existsVar, BooleanType))) | ||
|
||
if (keyIsKnownUnique) { |
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.
added keyIsKnownUnique
to support unique-key code path for BroadcastHashJoinExec
.
@@ -903,6 +904,10 @@ case class CollapseCodegenStages( | |||
// The children of SortMergeJoin should do codegen separately. | |||
j.withNewChildren(j.children.map( | |||
child => InputAdapter(insertWholeStageCodegen(child)))) | |||
case j: ShuffledHashJoinExec => |
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.
codegen children of ShuffledHashJoinExec
separately same as SortMergeJoinExec
.
@@ -316,6 +318,387 @@ trait HashJoin extends BaseJoinExec { | |||
resultProj(r) | |||
} | |||
} | |||
|
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.
@cloud-fan - sorry about that. yes mostly of them is moving code around without change. Highlighted change with comments, thanks.
Test build #126741 has finished for PR 29277 at commit
|
|
||
val thisPlan = ctx.addReferenceObj("plan", this) | ||
val (relationTerm, _) = prepareRelation(ctx) | ||
val buildRelation = s"$relationTerm = $thisPlan.buildHashedRelation($buildInput);" |
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.
why not include it when we create the relationTerm
in prepareRelation
?
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.
@cloud-fan - sorry if we include buildRelation
inside prepareRelation
, how do we use buildRelation
in final code-gen code? Do you mean creating a private var to keep buildRelation
after prepareRelation
is called?
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.
something like
protected def prepareRelation(ctx: CodegenContext): String = {
val thisPlan = ctx.addReferenceObj("plan", this)
val clsName = classOf[HashedRelation].getName
ctx.addMutableState(clsName, "relation", v =>
s"""
| $v = $thisPlan.buildHashedRelation(inputs[1]);
""".stripMargin, forceInline = true)
}
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.
Then we don't need the if (!$initRelation)
stuff.
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.
@cloud-fan - yes I got your point. Updated to do initialization in prepareRelation
, thanks.
@cloud-fan - updated the PR with addressing comments, and it is ready for review. Also updated the PR description for latest codegen code of example query. Thanks. |
@@ -903,6 +904,10 @@ case class CollapseCodegenStages( | |||
// The children of SortMergeJoin should do codegen separately. | |||
j.withNewChildren(j.children.map( | |||
child => InputAdapter(insertWholeStageCodegen(child)))) | |||
case j: ShuffledHashJoinExec => | |||
// The children of ShuffledHashJoin should do codegen separately. |
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.
This and SortMergeJoin
. I actually re-think about it and then figure out, because codegen related code rarely changes recently. It would be nice to add more comments here to explain it.
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.
@viirya - sure, wondering what kind of wording you are expecting here? does it look better with:
// The children of ShuffledHashJoin should do codegen separately,
// because codegen for ShuffledHashJoin depends on more than one row
// from the build side input.
// The children of SortMergeJoin should do codegen separately,
// because codegen for SortMergeJoin depends on more than one row
// from the buffer side input.
Test build #126796 has finished for PR 29277 at commit
|
} | ||
|
||
s""" | ||
|while ($streamedInput.hasNext()) { |
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.
This is exactly what will be generated by streamedPlan.asInstanceOf[CodegenSupport].produce(ctx, this)
. See InputAdapter.doProduce
. I don't understand why we can't put the join part in doConsume
.
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.
Note: https://github.com/apache/spark/pull/29277/files#diff-db4ffe4f0196a9d7cf1f04c350ee3381R124
We actually build the relation in the class constructor, the codegen flow should be the same with broadcast hash join.
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.
@cloud-fan - sorry I got your point now, updated to do join part in doConsume()
now. Moved BHJ's doProduce()
and doConsume()
into HashJoin
, to be shared between BHJ and SHJ. Also updated PR with latest codegen code for example query. Thanks for suggestion.
retest this please |
1 similar comment
retest this please |
@@ -903,6 +904,10 @@ case class CollapseCodegenStages( | |||
// The children of SortMergeJoin should do codegen separately. | |||
j.withNewChildren(j.children.map( | |||
child => InputAdapter(insertWholeStageCodegen(child)))) | |||
case j: ShuffledHashJoinExec => | |||
// The children of ShuffledHashJoin should do codegen separately. | |||
j.withNewChildren(j.children.map( |
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.
We can remove this now. ShuffledHashJoin now does codegen like BroadcastHashJoin.
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.
@viirya - I don't think we can remove this. We have to do shuffled hash join codegen separately, as we have a hardcoded dependency for build side input input[1]
when building relation. This can go wrong if we have multiple shuffled hash join in one query.
E.g.
test("ShuffledHashJoin should be included in WholeStageCodegen") {
withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "30",
SQLConf.SHUFFLE_PARTITIONS.key -> "2",
SQLConf.PREFER_SORTMERGEJOIN.key -> "false") {
val df1 = spark.range(5).select($"id".as("k1"))
val df2 = spark.range(15).select($"id".as("k2"))
val df3 = spark.range(6).select($"id".as("k3"))
val twoJoinsDF = df1.join(df2, $"k1" === $"k2").join(df3, $"k1" === $"k3")
}
}
If we don't codegen shuffled hash join children separately, we will get something like:
/* 018 */ public void init(int index, scala.collection.Iterator[] inputs) {
/* 019 */ partitionIndex = index;
/* 020 */ this.inputs = inputs;
/* 021 */ inputadapter_input_0 = inputs[0];
/* 022 */ shj_relation_0 = ((org.apache.spark.sql.execution.joins.ShuffledHashJoinExec) references[0] /* plan */).buildHashedRelation(inputs[1]);
/* 023 */ shj_mutableStateArray_0[0] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(2, 0);
/* 024 */ shj_relation_1 = ((org.apache.spark.sql.execution.joins.ShuffledHashJoinExec) references[2] /* plan */).buildHashedRelation(inputs[1]);
/* 025 */ shj_mutableStateArray_0[1] = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(3, 0);
/* 026 */
/* 027 */ }
shj_relation_0
and shj_relation_1
will try to build hash relation on same input (but shouldn't), as the input[1]
is hardcoded there. On the other hand, I couldn't think of an alternative way not to hardcode input[1]
here in codegen. Let me know if you have any better options. Thanks. I also updated WholeStageCodegenSuite.scala
to have a unit test for this kind of multiple joins query.
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.
Seems we only need to do it for the build side?
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.
Yea, that's true.
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.
@cloud-fan, @viirya - if we only codegen separately for build side, we would still have the same problem as above for multiple SHJs right? Essentially we would fuse multiple stream sides codegen together in one codegen method, so we will have multiple build side initialized in init(), and naming collision as above. Let me know if it doesn't make sense, or I can create a counter example here.
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.
@cloud-fan - sounds good, non-trivial for me now as well. Will try to resolve it in the future. Thanks.
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.
And there are more problems if we have many shuffle hash join stay together. We need to accumulate the CodegenSupport.inputRDDs
, but WholeStageCodegenExec
only supports up to 2 input RDDs for now.
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.
We need to accumulate the CodegenSupport.inputRDDs, but WholeStageCodegenExec only supports up to 2 input RDDs for now.
Yes. Agreed. SortMergeJoinExec
took the decision to do codegen for children separately, it's just simpler without getting into these limitations.
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.
Once we only codegen separately for build side, we should not build hash relation using inputs[1]
anymore. It will work similarly like BroadcastHashJoin
, we just build hash relation using buildPlan
. And ShuffledHashJoinExec
has only one input RDD now, like BroadcastHashJoin
's inputRDDs
.
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.
Hm, it still has some problems. Will think about it if I have more time.
/** | ||
* Returns a tuple of variable name for HashedRelation, | ||
* and boolean false to indicate key not to be known unique in code-gen time. | ||
*/ |
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.
Move the comment to HashJoin
?
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.
@viirya - sure, updated.
@c21 Jenkins seems not working now. You can sync with master to fix the SparkR issue in GitHub Actions. We can also rely on GitHub Actions to test. |
@viirya - thanks for tips, rebased to master now. |
Regarding the comment from @viirya, I've one last comment: #29277 (comment) Otherwise, I think it's good to go. |
Test build #126813 has finished for PR 29277 at commit
|
thanks, merging to master! |
Thank you @cloud-fan and @viirya for review and discussion! |
@cloud-fan - thanks for discussion. Creating https://issues.apache.org/jira/browse/SPARK-32505 as followup task, and I will take a shot. Thanks. |
What changes were proposed in this pull request?
Adding codegen for shuffled hash join. Shuffled hash join codegen is very similar to broadcast hash join codegen. So most of code change is to refactor existing codegen in
BroadcastHashJoinExec
toHashJoin
.Example codegen for query in
JoinBenchmark
:Shuffled hash join codegen:
Broadcast hash join codegen for the same query (for reference here):
Why are the changes needed?
Codegen shuffled hash join can help save CPU cost. We added shuffled hash join codegen internally in our fork, and seeing obvious improvement in benchmark compared to current non-codegen code path.
Test example query in
JoinBenchmark
, seeing 30% wall clock time improvement compared to existing non-codegen code path:Enable shuffled hash join code-gen:
Disable shuffled hash join codegen:
Does this PR introduce any user-facing change?
No.
How was this patch tested?
Added unit test in
WholeStageCodegenSuite
.