Skip to content
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

[BUG] GpuBroadcastNestedLoopJoinExec can assert in doUnconditionalJoin #9086

Closed
jlowe opened this issue Aug 21, 2023 · 0 comments · Fixed by #9100
Closed

[BUG] GpuBroadcastNestedLoopJoinExec can assert in doUnconditionalJoin #9086

jlowe opened this issue Aug 21, 2023 · 0 comments · Fixed by #9100
Assignees
Labels
bug Something isn't working

Comments

@jlowe
Copy link
Contributor

jlowe commented Aug 21, 2023

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:

  testSparkResultsAreEqual(
    s"InSubqueryExec",
    nullableStringsIntsDf,
    new SparkConf().set("spark.rapids.sql.debug.logTransformations", "true")
      .set("spark.sql.autoBroadcastJoinThreshold", "1")) { df =>
    val spark = df.sparkSession
    val df2 = nullableStringsDf(spark)
    val filter = Filter(
      DynamicPruningExpression(InSubquery(Seq(Literal("400.0")),
        ListQuery(df2.logicalPlan.select(UnresolvedAttribute("strings"))))),
      df.logicalPlan)
    Dataset.ofRows(spark, filter)
  }

This test asserts like this:

Stacktrace
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).

@jlowe jlowe added bug Something isn't working ? - Needs Triage Need team to review and classify labels Aug 21, 2023
@sameerz sameerz removed the ? - Needs Triage Need team to review and classify label Aug 22, 2023
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something isn't working
Projects
None yet
Development

Successfully merging a pull request may close this issue.

2 participants