Skip to content

[SPARK-35351][SQL] Add code-gen for left anti sort merge join #32547

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

Closed
wants to merge 3 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -364,7 +364,7 @@ case class SortMergeJoinExec(
}

private lazy val ((streamedPlan, streamedKeys), (bufferedPlan, bufferedKeys)) = joinType match {
case _: InnerLike | LeftOuter | LeftSemi => ((left, leftKeys), (right, rightKeys))
case _: InnerLike | LeftOuter | LeftSemi | LeftAnti => ((left, leftKeys), (right, rightKeys))
case RightOuter => ((right, rightKeys), (left, leftKeys))
case x =>
throw new IllegalArgumentException(
Expand All @@ -375,7 +375,7 @@ case class SortMergeJoinExec(
private lazy val bufferedOutput = bufferedPlan.output

override def supportCodegen: Boolean = joinType match {
case _: InnerLike | LeftOuter | RightOuter | LeftSemi => true
case _: InnerLike | LeftOuter | RightOuter | LeftSemi | LeftAnti => true
case _ => false
}

Expand Down Expand Up @@ -453,7 +453,7 @@ case class SortMergeJoinExec(
|$streamedRow = null;
|continue;
""".stripMargin
case LeftOuter | RightOuter =>
case LeftOuter | RightOuter | LeftAnti =>
// Eagerly return streamed row. Only call `matches.clear()` when `matches.isEmpty()` is
// false, to reduce unnecessary computation.
s"""
Expand All @@ -472,7 +472,7 @@ case class SortMergeJoinExec(
case _: InnerLike | LeftSemi =>
// Skip streamed row.
s"$streamedRow = null;"
case LeftOuter | RightOuter =>
case LeftOuter | RightOuter | LeftAnti =>
// Eagerly return with streamed row.
"return false;"
case x =>
Expand Down Expand Up @@ -509,17 +509,17 @@ case class SortMergeJoinExec(
// 1. Inner and Left Semi join: skip the row. `matches` will be cleared later when
// hitting the next `streamedRow` with non-null join
// keys.
// 2. Left/Right Outer join: clear the previous `matches` if needed, keep the row,
// and return false.
// 2. Left/Right Outer and Left Anti join: clear the previous `matches` if needed,
// keep the row, and return false.
//
// - Step 2: Find the `matches` from buffered side having same join keys with `streamedRow`.
// Clear `matches` if we hit a new `streamedRow`, as we need to find new matches.
// Use `bufferedRow` to iterate buffered side to put all matched rows into
// `matches` (`addRowToBuffer`). Return true when getting all matched rows.
// For `streamedRow` without `matches` (`handleStreamedWithoutMatch`):
// 1. Inner and Left Semi join: skip the row.
// 2. Left/Right Outer join: keep the row and return false (with `matches` being
// empty).
// 2. Left/Right Outer and Left Anti join: keep the row and return false (with
// `matches` being empty).
val findNextJoinRowsFuncName = ctx.freshName("findNextJoinRows")
ctx.addNewFunction(findNextJoinRowsFuncName,
s"""
Expand Down Expand Up @@ -664,14 +664,14 @@ case class SortMergeJoinExec(
streamedVars ++ bufferedVars
case RightOuter =>
bufferedVars ++ streamedVars
case LeftSemi =>
case LeftSemi | LeftAnti =>
streamedVars
case x =>
throw new IllegalArgumentException(
s"SortMergeJoin.doProduce should not take $x as the JoinType")
}

val (streamedBeforeLoop, condCheck) = if (condition.isDefined) {
val (streamedBeforeLoop, condCheck, loadStreamed) = if (condition.isDefined) {
// Split the code of creating variables based on whether it's used by condition or not.
val loaded = ctx.freshName("loaded")
val (streamedBefore, streamedAfter) = splitVarsByCondition(streamedOutput, streamedVars)
Expand All @@ -680,13 +680,36 @@ case class SortMergeJoinExec(
ctx.currentVars = streamedVars ++ bufferedVars
val cond = BindReferences.bindReference(
condition.get, streamedPlan.output ++ bufferedPlan.output).genCode(ctx)
// evaluate the columns those used by condition before loop
// Evaluate the columns those used by condition before loop
val before =
s"""
|boolean $loaded = false;
|$streamedBefore
""".stripMargin

val loadStreamed =
s"""
|if (!$loaded) {
| $loaded = true;
| $streamedAfter
|}
""".stripMargin
Comment on lines +683 to +696
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: seems loaded is not needed for LeftAnti case.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

loadStreamed is not used by LeftAnti.
I think you are referring to boolean $loaded = false; in before should not be needed, right?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yea, looks like for LeftAnti, it doesn't rely on loaded to do streamedAfter.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Created #32681 as followup, thanks.


val loadStreamedAfterCondition = joinType match {
case LeftAnti =>
// No need to evaluate columns not used by condition from streamed side, as for Left Anti
// join, streamed row with match is not outputted.
""
case _ => loadStreamed
}

val loadBufferedAfterCondition = joinType match {
case LeftSemi | LeftAnti =>
// No need to evaluate columns not used by condition from buffered side
""
case _ => bufferedAfter
}

val checking =
s"""
|$bufferedBefore
Expand All @@ -696,15 +719,12 @@ case class SortMergeJoinExec(
| continue;
| }
|}
|if (!$loaded) {
| $loaded = true;
| $streamedAfter
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For left anti, streamedAfter will appear twice in the code. How large is it?

Copy link
Contributor Author

@c21 c21 May 14, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@cloud-fan - good call for code size. Actually I just figured we don't need streamedAfter here for left anti, because we can skip streamed row when there's a match here.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@cloud-fan - updated. Also avoid unnecessary code for bufferedAfter and LEFT SEMI join.

|}
|$bufferedAfter
|$loadStreamedAfterCondition
|$loadBufferedAfterCondition
""".stripMargin
(before, checking.trim)
(before, checking.trim, loadStreamed)
} else {
(evaluateVariables(streamedVars), "")
(evaluateVariables(streamedVars), "", "")
}

val beforeLoop =
Expand Down Expand Up @@ -732,6 +752,9 @@ case class SortMergeJoinExec(
case LeftSemi =>
codegenSemi(findNextJoinRows, beforeLoop, iterator, bufferedRow, condCheck,
ctx.freshName("hasOutputRow"), outputRow, eagerCleanup)
case LeftAnti =>
codegenAnti(streamedInput, findNextJoinRows, beforeLoop, iterator, bufferedRow, condCheck,
loadStreamed, ctx.freshName("hasMatchedRow"), outputRow, eagerCleanup)
case x =>
throw new IllegalArgumentException(
s"SortMergeJoin.doProduce should not take $x as the JoinType")
Expand Down Expand Up @@ -825,6 +848,44 @@ case class SortMergeJoinExec(
""".stripMargin
}

/**
* Generates the code for Left Anti join.
*/
private def codegenAnti(
streamedInput: String,
findNextJoinRows: String,
beforeLoop: String,
matchIterator: String,
bufferedRow: String,
conditionCheck: String,
loadStreamed: String,
hasMatchedRow: String,
outputRow: String,
eagerCleanup: String): String = {
s"""
|while ($streamedInput.hasNext()) {
| $findNextJoinRows;
| $beforeLoop
| boolean $hasMatchedRow = false;
|
| while (!$hasMatchedRow && $matchIterator.hasNext()) {
| InternalRow $bufferedRow = (InternalRow) $matchIterator.next();
| $conditionCheck
| $hasMatchedRow = true;
| }
|
| if (!$hasMatchedRow) {
| // load all values of streamed row, because the values not in join condition are not
| // loaded yet.
| $loadStreamed
| $outputRow
| }
| if (shouldStop()) return;
|}
|$eagerCleanup
""".stripMargin
}

override protected def withNewChildrenInternal(
newLeft: SparkPlan, newRight: SparkPlan): SortMergeJoinExec =
copy(left = newLeft, right = newRight)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,7 @@
: +- * BroadcastHashJoin Inner BuildRight (32)
: :- * Project (26)
: : +- * BroadcastHashJoin Inner BuildRight (25)
: : :- SortMergeJoin LeftAnti (19)
: : :- * SortMergeJoin LeftAnti (19)
: : : :- * Project (13)
: : : : +- * SortMergeJoin LeftSemi (12)
: : : : :- * Sort (6)
Expand Down Expand Up @@ -124,7 +124,7 @@ Arguments: hashpartitioning(cr_order_number#14, 5), ENSURE_REQUIREMENTS, [id=#16
Input [1]: [cr_order_number#14]
Arguments: [cr_order_number#14 ASC NULLS FIRST], false, 0

(19) SortMergeJoin
(19) SortMergeJoin [codegen id : 11]
Left keys [1]: [cs_order_number#5]
Right keys [1]: [cr_order_number#14]
Join condition: None
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,8 +13,8 @@ WholeStageCodegen (12)
BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk]
Project [cs_ship_date_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit]
BroadcastHashJoin [cs_ship_addr_sk,ca_address_sk]
InputAdapter
SortMergeJoin [cs_order_number,cr_order_number]
SortMergeJoin [cs_order_number,cr_order_number]
InputAdapter
WholeStageCodegen (5)
Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit]
SortMergeJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk]
Expand All @@ -39,6 +39,7 @@ WholeStageCodegen (12)
ColumnarToRow
InputAdapter
Scan parquet default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk]
InputAdapter
WholeStageCodegen (7)
Sort [cr_order_number]
InputAdapter
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,7 @@
: +- * BroadcastHashJoin Inner BuildRight (32)
: :- * Project (26)
: : +- * BroadcastHashJoin Inner BuildRight (25)
: : :- SortMergeJoin LeftAnti (19)
: : :- * SortMergeJoin LeftAnti (19)
: : : :- * Project (13)
: : : : +- * SortMergeJoin LeftSemi (12)
: : : : :- * Sort (6)
Expand Down Expand Up @@ -124,7 +124,7 @@ Arguments: hashpartitioning(cr_order_number#14, 5), ENSURE_REQUIREMENTS, [id=#16
Input [1]: [cr_order_number#14]
Arguments: [cr_order_number#14 ASC NULLS FIRST], false, 0

(19) SortMergeJoin
(19) SortMergeJoin [codegen id : 11]
Left keys [1]: [cs_order_number#5]
Right keys [1]: [cr_order_number#14]
Join condition: None
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,8 +13,8 @@ WholeStageCodegen (12)
BroadcastHashJoin [cs_ship_addr_sk,ca_address_sk]
Project [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit]
BroadcastHashJoin [cs_ship_date_sk,d_date_sk]
InputAdapter
SortMergeJoin [cs_order_number,cr_order_number]
SortMergeJoin [cs_order_number,cr_order_number]
InputAdapter
WholeStageCodegen (5)
Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit]
SortMergeJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk]
Expand All @@ -39,6 +39,7 @@ WholeStageCodegen (12)
ColumnarToRow
InputAdapter
Scan parquet default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk]
InputAdapter
WholeStageCodegen (7)
Sort [cr_order_number]
InputAdapter
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -9,8 +9,8 @@ TakeOrderedAndProject (51)
: +- * Project (41)
: +- * BroadcastHashJoin Inner BuildRight (40)
: :- * Project (34)
: : +- SortMergeJoin LeftAnti (33)
: : :- SortMergeJoin LeftAnti (25)
: : +- * SortMergeJoin LeftAnti (33)
: : :- * SortMergeJoin LeftAnti (25)
: : : :- * SortMergeJoin LeftSemi (17)
: : : : :- * Sort (5)
: : : : : +- Exchange (4)
Expand Down Expand Up @@ -158,7 +158,7 @@ Arguments: hashpartitioning(ws_bill_customer_sk#13, 5), ENSURE_REQUIREMENTS, [id
Input [1]: [ws_bill_customer_sk#13]
Arguments: [ws_bill_customer_sk#13 ASC NULLS FIRST], false, 0

(25) SortMergeJoin
(25) SortMergeJoin [codegen id : 10]
Left keys [1]: [c_customer_sk#1]
Right keys [1]: [ws_bill_customer_sk#13]
Join condition: None
Expand All @@ -170,35 +170,35 @@ Location: InMemoryFileIndex []
PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#7)]
ReadSchema: struct<cs_ship_customer_sk:int>

(27) ColumnarToRow [codegen id : 11]
(27) ColumnarToRow [codegen id : 12]
Input [2]: [cs_ship_customer_sk#17, cs_sold_date_sk#18]

(28) ReusedExchange [Reuses operator id: 12]
Output [1]: [d_date_sk#19]

(29) BroadcastHashJoin [codegen id : 11]
(29) BroadcastHashJoin [codegen id : 12]
Left keys [1]: [cs_sold_date_sk#18]
Right keys [1]: [d_date_sk#19]
Join condition: None

(30) Project [codegen id : 11]
(30) Project [codegen id : 12]
Output [1]: [cs_ship_customer_sk#17]
Input [3]: [cs_ship_customer_sk#17, cs_sold_date_sk#18, d_date_sk#19]

(31) Exchange
Input [1]: [cs_ship_customer_sk#17]
Arguments: hashpartitioning(cs_ship_customer_sk#17, 5), ENSURE_REQUIREMENTS, [id=#20]

(32) Sort [codegen id : 12]
(32) Sort [codegen id : 13]
Input [1]: [cs_ship_customer_sk#17]
Arguments: [cs_ship_customer_sk#17 ASC NULLS FIRST], false, 0

(33) SortMergeJoin
(33) SortMergeJoin [codegen id : 15]
Left keys [1]: [c_customer_sk#1]
Right keys [1]: [cs_ship_customer_sk#17]
Join condition: None

(34) Project [codegen id : 14]
(34) Project [codegen id : 15]
Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3]
Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3]

Expand All @@ -209,27 +209,27 @@ Location [not included in comparison]/{warehouse_dir}/customer_address]
PushedFilters: [In(ca_state, [KY,GA,NM]), IsNotNull(ca_address_sk)]
ReadSchema: struct<ca_address_sk:int,ca_state:string>

(36) ColumnarToRow [codegen id : 13]
(36) ColumnarToRow [codegen id : 14]
Input [2]: [ca_address_sk#21, ca_state#22]

(37) Filter [codegen id : 13]
(37) Filter [codegen id : 14]
Input [2]: [ca_address_sk#21, ca_state#22]
Condition : (ca_state#22 IN (KY,GA,NM) AND isnotnull(ca_address_sk#21))

(38) Project [codegen id : 13]
(38) Project [codegen id : 14]
Output [1]: [ca_address_sk#21]
Input [2]: [ca_address_sk#21, ca_state#22]

(39) BroadcastExchange
Input [1]: [ca_address_sk#21]
Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23]

(40) BroadcastHashJoin [codegen id : 14]
(40) BroadcastHashJoin [codegen id : 15]
Left keys [1]: [c_current_addr_sk#3]
Right keys [1]: [ca_address_sk#21]
Join condition: None

(41) Project [codegen id : 14]
(41) Project [codegen id : 15]
Output [1]: [c_current_cdemo_sk#2]
Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#21]

Expand All @@ -251,16 +251,16 @@ Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_stat
Input [6]: [cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30]
Condition : isnotnull(cd_demo_sk#25)

(46) BroadcastHashJoin [codegen id : 15]
(46) BroadcastHashJoin [codegen id : 16]
Left keys [1]: [c_current_cdemo_sk#2]
Right keys [1]: [cd_demo_sk#25]
Join condition: None

(47) Project [codegen id : 15]
(47) Project [codegen id : 16]
Output [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30]
Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#25, cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30]

(48) HashAggregate [codegen id : 15]
(48) HashAggregate [codegen id : 16]
Input [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30]
Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30]
Functions [1]: [partial_count(1)]
Expand All @@ -271,7 +271,7 @@ Results [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_pur
Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, count#32]
Arguments: hashpartitioning(cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, 5), ENSURE_REQUIREMENTS, [id=#33]

(50) HashAggregate [codegen id : 16]
(50) HashAggregate [codegen id : 17]
Input [6]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30, count#32]
Keys [5]: [cd_gender#26, cd_marital_status#27, cd_education_status#28, cd_purchase_estimate#29, cd_credit_rating#30]
Functions [1]: [count(1)]
Expand Down
Loading