viirya commented on a change in pull request #29342:
URL: https://github.com/apache/spark/pull/29342#discussion_r467251404



##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala
##########
@@ -71,8 +88,122 @@ case class ShuffledHashJoinExec(
     val numOutputRows = longMetric("numOutputRows")
     streamedPlan.execute().zipPartitions(buildPlan.execute()) { (streamIter, 
buildIter) =>
       val hashed = buildHashedRelation(buildIter)
-      join(streamIter, hashed, numOutputRows)
+      joinType match {
+        case FullOuter => fullOuterJoin(streamIter, hashed, numOutputRows)
+        case _ => join(streamIter, hashed, numOutputRows)
+      }
+    }
+  }
+
+  /**
+   * Full outer shuffled hash join has three steps:
+   * 1. Construct hash relation from build side,
+   *    with extra boolean value at the end of row to track look up information
+   *    (done in `buildHashedRelation`).
+   * 2. Process rows from stream side by looking up hash relation,
+   *    and mark the matched rows from build side be looked up.
+   * 3. Process rows from build side by iterating hash relation,
+   *    and filter out rows from build side being looked up already.
+   */
+  private def fullOuterJoin(
+      streamIter: Iterator[InternalRow],
+      hashedRelation: HashedRelation,
+      numOutputRows: SQLMetric): Iterator[InternalRow] = {
+    val joinRow = new JoinedRow
+    val (joinRowWithStream, joinRowWithBuild) = {
+      buildSide match {
+        case BuildLeft => (joinRow.withRight _, joinRow.withLeft _)
+        case BuildRight => (joinRow.withLeft _, joinRow.withRight _)
+      }
+    }
+    val joinKeys = streamSideKeyGenerator()
+    val buildRowGenerator = UnsafeProjection.create(buildOutput, buildOutput)
+    val buildNullRow = new GenericInternalRow(buildOutput.length)
+    val streamNullRow = new GenericInternalRow(streamedOutput.length)
+
+    def markRowLookedUp(row: UnsafeRow): Unit =
+      row.setBoolean(row.numFields() - 1, true)
+
+    // Process stream side with looking up hash relation
+    val streamResultIter =
+      if (hashedRelation.keyIsUnique) {
+        streamIter.map { srow =>
+          joinRowWithStream(srow)
+          val keys = joinKeys(srow)
+          if (keys.anyNull) {
+            joinRowWithBuild(buildNullRow)
+          } else {
+            val matched = hashedRelation.getValue(keys)
+            if (matched != null) {
+              val buildRow = buildRowGenerator(matched)
+              if (boundCondition(joinRowWithBuild(buildRow))) {
+                markRowLookedUp(matched.asInstanceOf[UnsafeRow])
+                joinRow
+              } else {
+                joinRowWithBuild(buildNullRow)
+              }
+            } else {
+              joinRowWithBuild(buildNullRow)
+            }
+          }
+        }
+      } else {
+        streamIter.flatMap { srow =>
+          joinRowWithStream(srow)
+          val keys = joinKeys(srow)
+          if (keys.anyNull) {
+            Iterator.single(joinRowWithBuild(buildNullRow))
+          } else {
+            val buildIter = hashedRelation.get(keys)
+            new RowIterator {
+              private var found = false
+              override def advanceNext(): Boolean = {
+                while (buildIter != null && buildIter.hasNext) {
+                  val matched = buildIter.next()
+                  val buildRow = buildRowGenerator(matched)
+                  if (boundCondition(joinRowWithBuild(buildRow))) {
+                    markRowLookedUp(matched.asInstanceOf[UnsafeRow])
+                    found = true
+                    return true
+                  }
+                }
+                if (!found) {
+                  joinRowWithBuild(buildNullRow)
+                  found = true
+                  return true
+                }
+                false
+              }
+              override def getRow: InternalRow = joinRow
+            }.toScala
+          }
+        }
+      }
+
+    // Process build side with filtering out rows looked up already

Review comment:
       not only looked up, but also passed join condition?




----------------------------------------------------------------
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:
us...@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to