Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-44846

PushFoldableIntoBranches in complex grouping expressions may cause bindReference error

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 3.4.1
    • 3.4.2, 3.5.0, 4.0.0
    • SQL
    • None

    Description

      SQL:

      select c*2 as d from
      (select if(b > 1, 1, b) as c from
      (select if(a < 0, 0 ,a) as b from t group by b) t1
      group by c) t2 

      ERROR:

      Couldn't find _groupingexpression#15 in [if ((_groupingexpression#15 > 1)) 1 else _groupingexpression#15#16]
      java.lang.IllegalStateException: Couldn't find _groupingexpression#15 in [if ((_groupingexpression#15 > 1)) 1 else _groupingexpression#15#16]
          at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:80)
          at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:73)
          at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:461)
          at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:76)
          at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:461)
          at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$3(TreeNode.scala:466)
          at org.apache.spark.sql.catalyst.trees.BinaryLike.mapChildren(TreeNode.scala:1241)
          at org.apache.spark.sql.catalyst.trees.BinaryLike.mapChildren$(TreeNode.scala:1240)
          at org.apache.spark.sql.catalyst.expressions.BinaryExpression.mapChildren(Expression.scala:653)
          at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:466)
          at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$3(TreeNode.scala:466)
          at org.apache.spark.sql.catalyst.trees.TernaryLike.mapChildren(TreeNode.scala:1272)
          at org.apache.spark.sql.catalyst.trees.TernaryLike.mapChildren$(TreeNode.scala:1271)
          at org.apache.spark.sql.catalyst.expressions.If.mapChildren(conditionalExpressions.scala:41)
          at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:466)
          at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$3(TreeNode.scala:466)
          at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1215)
          at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1214)
          at org.apache.spark.sql.catalyst.expressions.UnaryExpression.mapChildren(Expression.scala:533)
          at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:466)
          at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:437)
          at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:405)
          at org.apache.spark.sql.catalyst.expressions.BindReferences$.bindReference(BoundAttribute.scala:73)
          at org.apache.spark.sql.catalyst.expressions.BindReferences$.$anonfun$bindReferences$1(BoundAttribute.scala:94)
          at scala.collection.immutable.List.map(List.scala:293)
          at org.apache.spark.sql.catalyst.expressions.BindReferences$.bindReferences(BoundAttribute.scala:94)
          at org.apache.spark.sql.execution.aggregate.HashAggregateExec.generateResultFunction(HashAggregateExec.scala:360)
          at org.apache.spark.sql.execution.aggregate.HashAggregateExec.doProduceWithKeys(HashAggregateExec.scala:538)
          at org.apache.spark.sql.execution.aggregate.AggregateCodegenSupport.doProduce(AggregateCodegenSupport.scala:69)
          at org.apache.spark.sql.execution.aggregate.AggregateCodegenSupport.doProduce$(AggregateCodegenSupport.scala:65)
          at org.apache.spark.sql.execution.aggregate.HashAggregateExec.doProduce(HashAggregateExec.scala:49)
          at org.apache.spark.sql.execution.CodegenSupport.$anonfun$produce$1(WholeStageCodegenExec.scala:97)
          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.CodegenSupport.produce(WholeStageCodegenExec.scala:92)
          at org.apache.spark.sql.execution.CodegenSupport.produce$(WholeStageCodegenExec.scala:92)
          at org.apache.spark.sql.execution.aggregate.HashAggregateExec.produce(HashAggregateExec.scala:49)
          at org.apache.spark.sql.execution.WholeStageCodegenExec.doCodeGen(WholeStageCodegenExec.scala:660)
          at org.apache.spark.sql.execution.WholeStageCodegenExec.doExecute(WholeStageCodegenExec.scala:723)
          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.DeserializeToObjectExec.doExecute(objects.scala:93)
          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.adaptive.AdaptiveSparkPlanExec.$anonfun$doExecute$1(AdaptiveSparkPlanExec.scala:386)
          at org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.withFinalPlanUpdate(AdaptiveSparkPlanExec.scala:402)
          at org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.doExecute(AdaptiveSparkPlanExec.scala:386)
          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.QueryExecution.toRdd$lzycompute(QueryExecution.scala:207)
          at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:206)
          at org.apache.spark.sql.Dataset.rdd$lzycompute(Dataset.scala:3857)
          at org.apache.spark.sql.Dataset.rdd(Dataset.scala:3855)
          at org.apache.spark.sql.QueryTest$.$anonfun$getErrorMessageInCheckAnswer$1(QueryTest.scala:266)
          at scala.runtime.java8.JFunction0$mcJ$sp.apply(JFunction0$mcJ$sp.java:23)
          at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:209)
          at org.apache.spark.sql.QueryTest$.getErrorMessageInCheckAnswer(QueryTest.scala:266)
          at org.apache.spark.sql.QueryTest$.checkAnswer(QueryTest.scala:243)
          at org.apache.spark.sql.QueryTest.checkAnswer(QueryTest.scala:151)
          at org.apache.spark.sql.DataFrameSuite.$anonfun$new$737(DataFrameSuite.scala:3676)
          at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
          at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(SparkErrorUtils.scala:64)
          at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(SparkErrorUtils.scala:61)
          at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:95)
          at org.apache.spark.sql.test.SQLTestUtilsBase.withTempView(SQLTestUtils.scala:276)
          at org.apache.spark.sql.test.SQLTestUtilsBase.withTempView$(SQLTestUtils.scala:274)
          at org.apache.spark.sql.DataFrameSuite.withTempView(DataFrameSuite.scala:60)
          at org.apache.spark.sql.DataFrameSuite.$anonfun$new$736(DataFrameSuite.scala:3667)
          at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
          at org.scalatest.enablers.Timed$$anon$1.timeoutAfter(Timed.scala:127)
          at org.scalatest.concurrent.TimeLimits$.failAfterImpl(TimeLimits.scala:282)
          at org.scalatest.concurrent.TimeLimits.failAfter(TimeLimits.scala:231)
          at org.scalatest.concurrent.TimeLimits.failAfter$(TimeLimits.scala:230)
          at org.apache.spark.SparkFunSuite.failAfter(SparkFunSuite.scala:69)
          at org.apache.spark.SparkFunSuite.$anonfun$test$2(SparkFunSuite.scala:155)
          at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
          at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
          at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
          at org.scalatest.Transformer.apply(Transformer.scala:22)
          at org.scalatest.Transformer.apply(Transformer.scala:20)
          at org.scalatest.funsuite.AnyFunSuiteLike$$anon$1.apply(AnyFunSuiteLike.scala:226)
          at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:227)
          at org.scalatest.funsuite.AnyFunSuiteLike.invokeWithFixture$1(AnyFunSuiteLike.scala:224)
          at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTest$1(AnyFunSuiteLike.scala:236)
          at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
          at org.scalatest.funsuite.AnyFunSuiteLike.runTest(AnyFunSuiteLike.scala:236)
          at org.scalatest.funsuite.AnyFunSuiteLike.runTest$(AnyFunSuiteLike.scala:218)
          at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:69)
          at org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:234)
          at org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:227)
          at org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:69)
          at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTests$1(AnyFunSuiteLike.scala:269)
          at org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:413)
          at scala.collection.immutable.List.foreach(List.scala:431)
          at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401)
          at org.scalatest.SuperEngine.runTestsInBranch(Engine.scala:396)
          at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:475)
          at org.scalatest.funsuite.AnyFunSuiteLike.runTests(AnyFunSuiteLike.scala:269)
          at org.scalatest.funsuite.AnyFunSuiteLike.runTests$(AnyFunSuiteLike.scala:268)
          at org.scalatest.funsuite.AnyFunSuite.runTests(AnyFunSuite.scala:1564)
          at org.scalatest.Suite.run(Suite.scala:1114)
          at org.scalatest.Suite.run$(Suite.scala:1096)
          at org.scalatest.funsuite.AnyFunSuite.org$scalatest$funsuite$AnyFunSuiteLike$$super$run(AnyFunSuite.scala:1564)
          at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$run$1(AnyFunSuiteLike.scala:273)
          at org.scalatest.SuperEngine.runImpl(Engine.scala:535)
          at org.scalatest.funsuite.AnyFunSuiteLike.run(AnyFunSuiteLike.scala:273)
          at org.scalatest.funsuite.AnyFunSuiteLike.run$(AnyFunSuiteLike.scala:272)
          at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterAll$$super$run(SparkFunSuite.scala:69)
          at org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:213)
          at org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210)
          at org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208)
          at org.apache.spark.SparkFunSuite.run(SparkFunSuite.scala:69)
          at org.scalatest.tools.SuiteRunner.run(SuiteRunner.scala:47)
          at org.scalatest.tools.Runner$.$anonfun$doRunRunRunDaDoRunRun$13(Runner.scala:1321)
          at org.scalatest.tools.Runner$.$anonfun$doRunRunRunDaDoRunRun$13$adapted(Runner.scala:1315)
          at scala.collection.immutable.List.foreach(List.scala:431)
          at org.scalatest.tools.Runner$.doRunRunRunDaDoRunRun(Runner.scala:1315)
          at org.scalatest.tools.Runner$.$anonfun$runOptionallyWithPassFailReporter$24(Runner.scala:992)
          at org.scalatest.tools.Runner$.$anonfun$runOptionallyWithPassFailReporter$24$adapted(Runner.scala:970)
          at org.scalatest.tools.Runner$.withClassLoaderAndDispatchReporter(Runner.scala:1481)
          at org.scalatest.tools.Runner$.runOptionallyWithPassFailReporter(Runner.scala:970)
          at org.scalatest.tools.Runner$.run(Runner.scala:798)
          at org.scalatest.tools.Runner.run(Runner.scala)
          at org.jetbrains.plugins.scala.testingSupport.scalaTest.ScalaTestRunner.runScalaTest2or3(ScalaTestRunner.java:38)
          at org.jetbrains.plugins.scala.testingSupport.scalaTest.ScalaTestRunner.main(ScalaTestRunner.java:25)
       

      Attachments

        Activity

          People

            zhuml Mingliang Zhu
            zhuml Mingliang Zhu
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: