leanken commented on a change in pull request #29104:
URL: https://github.com/apache/spark/pull/29104#discussion_r472587295
##########
File path:
sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala
##########
@@ -454,6 +490,40 @@ case class BroadcastHashJoinExec(
val (matched, checkCondition, _) = getJoinCondition(ctx, input)
val numOutput = metricTerm(ctx, "numOutputRows")
+ if (isNullAwareAntiJoin) {
+ if (broadcastRelation.value == EmptyHashedRelation) {
+ return s"""
+ |// If the right side is empty, NAAJ simply returns the left
side.
+ |$numOutput.add(1);
+ |${consume(ctx, input)}
+ """.stripMargin
+ } else if (broadcastRelation.value ==
EmptyHashedRelationWithAllNullKeys) {
+ return s"""
+ |// If the right side contains any all-null key, NAAJ simply
returns Nothing.
+ """.stripMargin
+ } else {
+ val found = ctx.freshName("found")
+ return s"""
+ |boolean $found = false;
+ |// generate join key for stream side
+ |${keyEv.code}
+ |if ($anyNull) {
+ | $found = true;
+ |} else {
+ | UnsafeRow $matched =
(UnsafeRow)$relationTerm.getValue(${keyEv.value});
+ | if ($matched != null) {
+ | $found = true;
+ | }
+ |}
+ |
+ |if (!$found) {
+ | $numOutput.add(1);
+ | ${consume(ctx, input)}
+ |}
Review comment:
how about
```
s"""
|// generate join key for stream side
|${keyEv.code}
|if ($anyNull || $relationTerm.getValue(${keyEv.value}) != null) {
| // do nothing
|} else {
| $numOutput.add(1);
| ${consume(ctx, input)}
|}
""".stripMargin
```
maybe I could update these code as well with the new HashedRelation Name in
next PR.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]