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

IllegalStateException due to nested column aliasing

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 4.0.0, 3.5.1, 3.4.3
    • 4.0.0, 3.5.2
    • SQL

    Description

      val f = udf[((Int, Int), Int), ((Int, Int), Int)](identity)
      val ds1 = Seq(((1, 2), 1)).toDS
      
      val rhs1 = ds1.select(ds1("_1._1"))
      val tmp1 = ds1.join(rhs1, ds1("_1._1") === rhs1("_1")).select(f(struct(ds1("_1"), ds1("_2"))).as("tmp1")).select($"tmp1.*")
      
      tmp1.select($"_1._2").collect() 

      crashes with

      java.lang.IllegalStateException: Couldn't find _1#6 in [_extract__1#35,_2#7,_1#11]
        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 scala.collection.immutable.ArraySeq.map(ArraySeq.scala:75)
        at scala.collection.immutable.ArraySeq.map(ArraySeq.scala:35)
        at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:699)
        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 scala.collection.immutable.ArraySeq.map(ArraySeq.scala:75)
        at scala.collection.immutable.ArraySeq.map(ArraySeq.scala:35)
        at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:699)
        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.$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.$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.ArraySeq.map(ArraySeq.scala:75)
        at scala.collection.immutable.ArraySeq.map(ArraySeq.scala:35)
        at org.apache.spark.sql.catalyst.expressions.BindReferences$.bindReferences(BoundAttribute.scala:94)
        at org.apache.spark.sql.execution.ProjectExec.doConsume(basicPhysicalOperators.scala:69)
        at org.apache.spark.sql.execution.CodegenSupport.consume(WholeStageCodegenExec.scala:196)
        at org.apache.spark.sql.execution.CodegenSupport.consume$(WholeStageCodegenExec.scala:151)
        at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.consume(BroadcastHashJoinExec.scala:40)
        at org.apache.spark.sql.execution.joins.HashJoin.codegenInner(HashJoin.scala:414)
        at org.apache.spark.sql.execution.joins.HashJoin.codegenInner$(HashJoin.scala:390)
        at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.codegenInner(BroadcastHashJoinExec.scala:40)
        at org.apache.spark.sql.execution.joins.HashJoin.doConsume(HashJoin.scala:357)
        at org.apache.spark.sql.execution.joins.HashJoin.doConsume$(HashJoin.scala:355)
        at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.doConsume(BroadcastHashJoinExec.scala:40)
        at org.apache.spark.sql.execution.CodegenSupport.consume(WholeStageCodegenExec.scala:196)
        at org.apache.spark.sql.execution.CodegenSupport.consume$(WholeStageCodegenExec.scala:151)
        at org.apache.spark.sql.execution.LocalTableScanExec.consume(LocalTableScanExec.scala:32)
        at org.apache.spark.sql.execution.InputRDDCodegen.doProduce(WholeStageCodegenExec.scala:485)
        at org.apache.spark.sql.execution.InputRDDCodegen.doProduce$(WholeStageCodegenExec.scala:458)
        at org.apache.spark.sql.execution.LocalTableScanExec.doProduce(LocalTableScanExec.scala:32)
        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.LocalTableScanExec.produce(LocalTableScanExec.scala:32)
        at org.apache.spark.sql.execution.joins.HashJoin.doProduce(HashJoin.scala:352)
        at org.apache.spark.sql.execution.joins.HashJoin.doProduce$(HashJoin.scala:351)
        at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.doProduce(BroadcastHashJoinExec.scala:40)
        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.joins.BroadcastHashJoinExec.produce(BroadcastHashJoinExec.scala:40)
        at org.apache.spark.sql.execution.ProjectExec.doProduce(basicPhysicalOperators.scala:55)
        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.ProjectExec.produce(basicPhysicalOperators.scala:42)
        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.SparkPlan.getByteArrayRdd(SparkPlan.scala:364)
        at org.apache.spark.sql.execution.SparkPlan.executeCollect(SparkPlan.scala:445)
        at org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.$anonfun$executeCollect$1(AdaptiveSparkPlanExec.scala:374)
        at org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.withFinalPlanUpdate(AdaptiveSparkPlanExec.scala:402)
        at org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.executeCollect(AdaptiveSparkPlanExec.scala:374)
        at org.apache.spark.sql.Dataset.collectFromPlan(Dataset.scala:4344)
        at org.apache.spark.sql.Dataset.$anonfun$collect$1(Dataset.scala:3585)
        at org.apache.spark.sql.Dataset.$anonfun$withAction$2(Dataset.scala:4334)
        at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:546)
        at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:4332)
        at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:125)
        at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:201)
        at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:108)
        at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900)
        at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:66)
        at org.apache.spark.sql.Dataset.withAction(Dataset.scala:4332)
        at org.apache.spark.sql.Dataset.collect(Dataset.scala:3585)
        ... 42 elided

      Setting

      spark.sql.optimizer.expression.nestedPruning.enabled=false 
      spark.sql.optimizer.nestedSchemaPruning.enabled=false 

      Makes the plan run correctly.

      Attachments

        Issue Links

          Activity

            People

              eejbyfeldt Emil Ejbyfeldt
              eejbyfeldt Emil Ejbyfeldt
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: