Skip to content

Commit

Permalink
[SPARK-34702][SQL] Avoid unnecessary code generation in JoinCodegenSu…
Browse files Browse the repository at this point in the history
…pport.genBuildSideVars

### What changes were proposed in this pull request?

As a followup from code review in #31736 (comment) , for `JoinCodegenSupport.genBuildSideVars`, we only need to generate build side variables with default values for LEFT OUTER and RIGHT OUTER join, but not for other join types (i.e. LEFT SEMI and LEFT ANTI). Create this PR to clean up the code.

In addition, change `BroadcastNestedLoopJoinExec` unit test to cover both whole stage code-gen enabled and disabled. Harden the unit tests to exercise all code paths.

### Why are the changes needed?

Avoid unnecessary code generation.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Existing unit tests.
* BHJ and SHJ inner join is covered in `InnerJoinSuite.scala`
* BHJ and SHJ left outer and right outer join are covered in `OuterJoinSuite.scala`
* BHJ and SHJ left semi, left anti and existence join are covered in `ExistenceJoinSuite.scala`

Closes #31802 from c21/join-codegen-fix.

Authored-by: Cheng Su <chengsu@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
  • Loading branch information
c21 authored and cloud-fan committed Mar 11, 2021
1 parent fa1cf5c commit da086a8
Show file tree
Hide file tree
Showing 2 changed files with 24 additions and 20 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.joins
import org.apache.spark.sql.catalyst.expressions.{BindReferences, BoundReference}
import org.apache.spark.sql.catalyst.expressions.codegen._
import org.apache.spark.sql.catalyst.expressions.codegen.Block._
import org.apache.spark.sql.catalyst.plans.InnerLike
import org.apache.spark.sql.catalyst.plans.{ExistenceJoin, InnerLike, LeftAnti, LeftOuter, LeftSemi, RightOuter}
import org.apache.spark.sql.execution.{CodegenSupport, SparkPlan}

/**
Expand Down Expand Up @@ -73,23 +73,27 @@ trait JoinCodegenSupport extends CodegenSupport with BaseJoinExec {
ctx.INPUT_ROW = buildRow
buildPlan.output.zipWithIndex.map { case (a, i) =>
val ev = BoundReference(i, a.dataType, a.nullable).genCode(ctx)
if (joinType.isInstanceOf[InnerLike]) {
ev
} else {
// the variables are needed even there is no matched rows
val isNull = ctx.freshName("isNull")
val value = ctx.freshName("value")
val javaType = CodeGenerator.javaType(a.dataType)
val code = code"""
|boolean $isNull = true;
|$javaType $value = ${CodeGenerator.defaultValue(a.dataType)};
|if ($buildRow != null) {
| ${ev.code}
| $isNull = ${ev.isNull};
| $value = ${ev.value};
|}
""".stripMargin
ExprCode(code, JavaCode.isNullVariable(isNull), JavaCode.variable(value, a.dataType))
joinType match {
case _: InnerLike | LeftSemi | LeftAnti | _: ExistenceJoin =>
ev
case LeftOuter | RightOuter =>
// the variables are needed even there is no matched rows
val isNull = ctx.freshName("isNull")
val value = ctx.freshName("value")
val javaType = CodeGenerator.javaType(a.dataType)
val code = code"""
|boolean $isNull = true;
|$javaType $value = ${CodeGenerator.defaultValue(a.dataType)};
|if ($buildRow != null) {
| ${ev.code}
| $isNull = ${ev.isNull};
| $value = ${ev.value};
|}
""".stripMargin
ExprCode(code, JavaCode.isNullVariable(isNull), JavaCode.variable(value, a.dataType))
case _ =>
throw new IllegalArgumentException(
s"JoinCodegenSupport.genBuildSideVars should not take $joinType as the JoinType")
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -198,7 +198,7 @@ class InnerJoinSuite extends SparkPlanTest with SharedSparkSession {
}
}

test(s"$testName using BroadcastNestedLoopJoin build left") {
testWithWholeStageCodegenOnAndOff(s"$testName using BroadcastNestedLoopJoin build left") { _ =>
withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") {
checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) =>
BroadcastNestedLoopJoinExec(left, right, BuildLeft, Inner, Some(condition())),
Expand All @@ -207,7 +207,7 @@ class InnerJoinSuite extends SparkPlanTest with SharedSparkSession {
}
}

test(s"$testName using BroadcastNestedLoopJoin build right") {
testWithWholeStageCodegenOnAndOff(s"$testName using BroadcastNestedLoopJoin build right") { _ =>
withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") {
checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) =>
BroadcastNestedLoopJoinExec(left, right, BuildRight, Inner, Some(condition())),
Expand Down

0 comments on commit da086a8

Please sign in to comment.