Details
-
Bug
-
Status: Resolved
-
Major
-
Resolution: Fixed
-
4.0.0, 3.5.1, 3.4.3
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
- links to