You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
There's an assert in GpuBroadcastNestedLoopJoinExec that could fire on a legitimate query:
// Existence join should have a condition
assert(!joinType.isInstanceOf[ExistenceJoin])
This assert indicates it's not valid to have a nested loop existence join without a condition, but it can happen. I ran across this attempting to fabricate an InSubqueryExec test that was turned into a broadcast nested loop join instead:
Caused by: java.lang.AssertionError: assertion failed
at scala.Predef$.assert(Predef.scala:208)
at org.apache.spark.sql.rapids.execution.GpuBroadcastNestedLoopJoinExecBase.doUnconditionalJoin(GpuBroadcastNestedLoopJoinExecBase.scala:537)
at org.apache.spark.sql.rapids.execution.GpuBroadcastNestedLoopJoinExecBase.internalDoExecuteColumnar(GpuBroadcastNestedLoopJoinExecBase.scala:513)
at com.nvidia.spark.rapids.GpuExec.doExecuteColumnar(GpuExec.scala:349)
at com.nvidia.spark.rapids.GpuExec.doExecuteColumnar$(GpuExec.scala:348)
at org.apache.spark.sql.rapids.execution.GpuBroadcastNestedLoopJoinExecBase.doExecuteColumnar(GpuBroadcastNestedLoopJoinExecBase.scala:380)
at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeColumnar$1(SparkPlan.scala:222)
at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:246)
at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:243)
at org.apache.spark.sql.execution.SparkPlan.executeColumnar(SparkPlan.scala:218)
at com.nvidia.spark.rapids.GpuProjectExec.internalDoExecuteColumnar(basicPhysicalOperators.scala:371)
at com.nvidia.spark.rapids.GpuExec.doExecuteColumnar(GpuExec.scala:349)
at com.nvidia.spark.rapids.GpuExec.doExecuteColumnar$(GpuExec.scala:348)
at com.nvidia.spark.rapids.GpuProjectExec.doExecuteColumnar(basicPhysicalOperators.scala:343)
at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeColumnar$1(SparkPlan.scala:222)
at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:246)
at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:243)
at org.apache.spark.sql.execution.SparkPlan.executeColumnar(SparkPlan.scala:218)
at org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBase.inputBatchRDD$lzycompute(GpuShuffleExchangeExecBase.scala:221)
at org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBase.inputBatchRDD(GpuShuffleExchangeExecBase.scala:221)
at org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBase.shuffleDependencyColumnar$lzycompute(GpuShuffleExchangeExecBase.scala:231)
at org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBase.shuffleDependencyColumnar(GpuShuffleExchangeExecBase.scala:229)
at org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBase.$anonfun$internalDoExecuteColumnar$1(GpuShuffleExchangeExecBase.scala:255)
at com.nvidia.spark.rapids.shims.Spark320PlusShims.attachTreeIfSupported(Spark320PlusShims.scala:296)
at com.nvidia.spark.rapids.shims.Spark320PlusShims.attachTreeIfSupported$(Spark320PlusShims.scala:291)
at com.nvidia.spark.rapids.shims.SparkShimImpl$.attachTreeIfSupported(SparkShims.scala:23)
at org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBase.internalDoExecuteColumnar(GpuShuffleExchangeExecBase.scala:252)
at com.nvidia.spark.rapids.GpuExec.doExecuteColumnar(GpuExec.scala:349)
at com.nvidia.spark.rapids.GpuExec.doExecuteColumnar$(GpuExec.scala:348)
at org.apache.spark.sql.rapids.execution.GpuShuffleExchangeExecBase.doExecuteColumnar(GpuShuffleExchangeExecBase.scala:152)
at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeColumnar$1(SparkPlan.scala:222)
at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:246)
at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:243)
at org.apache.spark.sql.execution.SparkPlan.executeColumnar(SparkPlan.scala:218)
at com.nvidia.spark.rapids.GpuShuffleCoalesceExec.internalDoExecuteColumnar(GpuShuffleCoalesceExec.scala:69)
at com.nvidia.spark.rapids.GpuExec.doExecuteColumnar(GpuExec.scala:349)
at com.nvidia.spark.rapids.GpuExec.doExecuteColumnar$(GpuExec.scala:348)
at com.nvidia.spark.rapids.GpuShuffleCoalesceExec.doExecuteColumnar(GpuShuffleCoalesceExec.scala:42)
at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeColumnar$1(SparkPlan.scala:222)
at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:246)
at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:243)
at org.apache.spark.sql.execution.SparkPlan.executeColumnar(SparkPlan.scala:218)
at com.nvidia.spark.rapids.GpuColumnarToRowExec.doExecute(GpuColumnarToRowExec.scala:333)
at org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:195)
at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:246)
at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:243)
at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:191)
at org.apache.spark.sql.execution.SparkPlan.getByteArrayRdd(SparkPlan.scala:364)
at org.apache.spark.sql.execution.SparkPlan.executeCollect(SparkPlan.scala:445)
at org.apache.spark.sql.Dataset.collectFromPlan(Dataset.scala:4177)
at org.apache.spark.sql.Dataset.$anonfun$collect$1(Dataset.scala:3418)
at org.apache.spark.sql.Dataset.$anonfun$withAction$2(Dataset.scala:4167)
at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:526)
... 70 more
There needs to be an implementation of broadcast nested loop join for an unconditional existence join. The implementation is pretty straightforward, it should treat the condition as essentially broadcast is nonEmpty (i.e.: the existence column is all true if there's any rows at all in the broadcast table, otherwise the column is all false).
The text was updated successfully, but these errors were encountered:
There's an assert in GpuBroadcastNestedLoopJoinExec that could fire on a legitimate query:
This assert indicates it's not valid to have a nested loop existence join without a condition, but it can happen. I ran across this attempting to fabricate an InSubqueryExec test that was turned into a broadcast nested loop join instead:
This test asserts like this:
Stacktrace
There needs to be an implementation of broadcast nested loop join for an unconditional existence join. The implementation is pretty straightforward, it should treat the condition as essentially broadcast is nonEmpty (i.e.: the existence column is all true if there's any rows at all in the broadcast table, otherwise the column is all false).
The text was updated successfully, but these errors were encountered: