You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2021/03/04 14:04:30 UTC

[GitHub] [spark] maropu commented on a change in pull request #31736: [SPARK-34620][SQL] Code-gen broadcast nested loop join (inner/cross)

maropu commented on a change in pull request #31736:
URL: https://github.com/apache/spark/pull/31736#discussion_r587461811



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala
##########
@@ -92,6 +92,39 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession
       Seq(Row(0, 0, 0), Row(1, 1, 1), Row(2, 2, 2), Row(3, 3, 3), Row(4, 4, 4)))
   }
 
+  test("Inner/Cross BroadcastNestedLoopJoinExec should be included in WholeStageCodegen") {
+    val df1 = spark.range(4).select($"id".as("k1"))
+    val df2 = spark.range(3).select($"id".as("k2"))
+    val df3 = spark.range(2).select($"id".as("k3"))
+
+    // test broadcast nested loop join without condition
+    val oneJoinDF = df1.join(df2)
+    assert(oneJoinDF.queryExecution.executedPlan.collect {
+      case WholeStageCodegenExec(_ : BroadcastNestedLoopJoinExec) => true
+    }.size === 1)
+    checkAnswer(oneJoinDF,

Review comment:
       For better test coverage, could you run tests with codegen enabled/disabled?

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala
##########
@@ -166,6 +166,19 @@ object JoinBenchmark extends SqlBasedBenchmark {
     }
   }
 
+  def broadcastNestedLoopJoin(): Unit = {
+    val N = 20 << 20
+    val M = 1 << 4
+
+    val dim = broadcast(spark.range(M).selectExpr("id as k", "cast(id as string) as v"))
+    codegenBenchmark("broadcast nested loop join", N) {

Review comment:
       Could you update `benchmarks/JoinBenchmark-results.txt`, too?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala
##########
@@ -393,4 +394,65 @@ case class BroadcastNestedLoopJoinExec(
       }
     }
   }
+
+  override def supportCodegen: Boolean = {
+    joinType.isInstanceOf[InnerLike]
+  }
+
+  override def inputRDDs(): Seq[RDD[InternalRow]] = {
+    streamed.asInstanceOf[CodegenSupport].inputRDDs()
+  }
+
+  override def needCopyResult: Boolean = true
+
+  override def doProduce(ctx: CodegenContext): String = {
+    streamed.asInstanceOf[CodegenSupport].produce(ctx, this)
+  }
+
+  override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = {
+    joinType match {
+      case _: InnerLike => codegenInner(ctx, input)
+      case x =>
+        throw new IllegalArgumentException(
+          s"BroadcastNestedLoopJoin code-gen should not take $x as the JoinType")
+    }
+  }
+
+  /**
+   * Returns the variable name for [[Broadcast]] side.
+   */
+  private def prepareBroadcast(ctx: CodegenContext): String = {
+    // Create a name for broadcast side
+    val broadcastArray = broadcast.executeBroadcast[Array[InternalRow]]()
+    val broadcastTerm = ctx.addReferenceObj("broadcastTerm", broadcastArray)
+
+    // Inline mutable state since not many join operations in a task
+    ctx.addMutableState("InternalRow[]", "broadcastArray",
+      v => s"$v = (InternalRow[]) $broadcastTerm.value();", forceInline = true)
+  }
+
+  private def codegenInner(ctx: CodegenContext, input: Seq[ExprCode]): String = {
+    val arrayTerm = prepareBroadcast(ctx)

Review comment:
       nit: `arrayTerm` -> `buildRowArrayTerm`, `buildRowsTerm`, ...?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala
##########
@@ -393,4 +394,65 @@ case class BroadcastNestedLoopJoinExec(
       }
     }
   }
+
+  override def supportCodegen: Boolean = {
+    joinType.isInstanceOf[InnerLike]
+  }
+
+  override def inputRDDs(): Seq[RDD[InternalRow]] = {
+    streamed.asInstanceOf[CodegenSupport].inputRDDs()
+  }
+
+  override def needCopyResult: Boolean = true
+
+  override def doProduce(ctx: CodegenContext): String = {
+    streamed.asInstanceOf[CodegenSupport].produce(ctx, this)
+  }
+
+  override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = {
+    joinType match {
+      case _: InnerLike => codegenInner(ctx, input)
+      case x =>
+        throw new IllegalArgumentException(
+          s"BroadcastNestedLoopJoin code-gen should not take $x as the JoinType")
+    }
+  }
+
+  /**
+   * Returns the variable name for [[Broadcast]] side.
+   */
+  private def prepareBroadcast(ctx: CodegenContext): String = {
+    // Create a name for broadcast side
+    val broadcastArray = broadcast.executeBroadcast[Array[InternalRow]]()
+    val broadcastTerm = ctx.addReferenceObj("broadcastTerm", broadcastArray)
+
+    // Inline mutable state since not many join operations in a task
+    ctx.addMutableState("InternalRow[]", "broadcastArray",

Review comment:
       nit: `broadcastArray` -> `buildRows`, `buidlRowArray`, ...?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala
##########
@@ -393,4 +394,65 @@ case class BroadcastNestedLoopJoinExec(
       }
     }
   }
+
+  override def supportCodegen: Boolean = {
+    joinType.isInstanceOf[InnerLike]
+  }
+
+  override def inputRDDs(): Seq[RDD[InternalRow]] = {
+    streamed.asInstanceOf[CodegenSupport].inputRDDs()
+  }
+
+  override def needCopyResult: Boolean = true

Review comment:
       always true? Is there a trade-off between the overheads of uniqueness checks and result copys?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala
##########
@@ -393,4 +394,65 @@ case class BroadcastNestedLoopJoinExec(
       }
     }
   }
+
+  override def supportCodegen: Boolean = {
+    joinType.isInstanceOf[InnerLike]
+  }
+
+  override def inputRDDs(): Seq[RDD[InternalRow]] = {
+    streamed.asInstanceOf[CodegenSupport].inputRDDs()
+  }
+
+  override def needCopyResult: Boolean = true
+
+  override def doProduce(ctx: CodegenContext): String = {
+    streamed.asInstanceOf[CodegenSupport].produce(ctx, this)
+  }
+
+  override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = {
+    joinType match {
+      case _: InnerLike => codegenInner(ctx, input)
+      case x =>
+        throw new IllegalArgumentException(
+          s"BroadcastNestedLoopJoin code-gen should not take $x as the JoinType")

Review comment:
       nit:
   ```
         case _ =>
           throw new IllegalArgumentException(
             s"BroadcastNestedLoopJoin code-gen should not take $joinType as the JoinType")
   ```
   ?




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



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