Description
I'm seeing a TreeNodeException ("Couldn't find gen_alias") when running certain operations in Spark 3.1.2.
A few conditions need to be met to trigger the bug:
- a DF with a nested struct joins to a second DF
- a filter that compares a column in the right DF to a column in the left DF
- wildcard column expansion of the nested struct
- a group by statement on a struct column
Data
git@github.com:kellanburket/spark3bug.git
val rightDf = spark.read.parquet("right.parquet") val leftDf = spark.read.parquet("left.parquet")
Schemas
leftDf.printSchema() root |-- row: struct (nullable = true) | |-- mid: string (nullable = true) | |-- start: struct (nullable = true) | | |-- latitude: double (nullable = true) | | |-- longitude: double (nullable = true) |-- s2_cell_id: long (nullable = true)
rightDf.printSchema() root |-- id: string (nullable = true) |-- s2_cell_id: long (nullable = true)
Breaking Code
leftDf.join(rightDf, "s2_cell_id").filter( "id != row.start.latitude" ).select( col("row.*"), col("id") ).groupBy( "start" ).agg( min("id") ).show()
Working Examples
The following examples don't seem to be effected by the bug
Works without group by:
leftDf.join(rightDf, "s2_cell_id").filter( "id != row.start.latitude" ).select( col("row.*"), col("id") ).show()
Works without filter
leftDf.join(rightDf, "s2_cell_id").select( col("row.*"), col("id") ).groupBy( "start" ).agg( min("id") ).show()
Works without wildcard expansion
leftDf.join(rightDf, "s2_cell_id").filter( "id != row.start.latitude" ).select( col("row.start"), col("id") ).groupBy( "start" ).agg( min("id") ).show()
Works with caching
leftDf.join(rightDf, "s2_cell_id").filter( "id != row.start.latitude" ).cache().select( col("row.*"), col("id") ).groupBy( "start" ).agg( min("id") ).show()
Error message
org.apache.spark.sql.catalyst.errors.package$TreeNodeException: execute, tree: Exchange hashpartitioning(start#2116, 1024), ENSURE_REQUIREMENTS, [id=#3849] +- SortAggregate(key=[knownfloatingpointnormalized(if (isnull(start#2116)) null else named_struct(latitude, knownfloatingpointnormalized(normalizenanandzero(start#2116.latitude)), longitude, knownfloatingpointnormalized(normalizenanandzero(start#2116.longitude)))) AS start#2116], functions=[partial_min(id#2103)], output=[start#2116, min#2138]) +- *(2) Sort [knownfloatingpointnormalized(if (isnull(start#2116)) null else named_struct(latitude, knownfloatingpointnormalized(normalizenanandzero(start#2116.latitude)), longitude, knownfloatingpointnormalized(normalizenanandzero(start#2116.longitude)))) AS start#2116 ASC NULLS FIRST], false, 0 +- *(2) Project [_gen_alias_2133#2133 AS start#2116, id#2103] +- *(2) !BroadcastHashJoin [s2_cell_id#2108L], [s2_cell_id#2104L], Inner, BuildLeft, NOT (cast(id#2103 as double) = _gen_alias_2134#2134), false :- BroadcastQueryStage 0 : +- BroadcastExchange HashedRelationBroadcastMode(List(input[1, bigint, false]),false), [id=#3768] : +- *(1) Project [row#2107.start AS _gen_alias_2133#2133, s2_cell_id#2108L] : +- *(1) Filter isnotnull(s2_cell_id#2108L) : +- FileScan parquet [row#2107,s2_cell_id#2108L] Batched: false, DataFilters: [isnotnull(s2_cell_id#2108L)], Format: Parquet, Location: InMemoryFileIndex[s3://co.mira.public/spark3_bug/left], PartitionFilters: [], PushedFilters: [IsNotNull(s2_cell_id)], ReadSchema: struct<row:struct<start:struct<latitude:double,longitude:double>>,s2_cell_id:bigint> +- *(2) Filter (isnotnull(id#2103) AND isnotnull(s2_cell_id#2104L)) +- *(2) ColumnarToRow +- FileScan parquet [id#2103,s2_cell_id#2104L] Batched: true, DataFilters: [isnotnull(id#2103), isnotnull(s2_cell_id#2104L)], Format: Parquet, Location: InMemoryFileIndex[s3://co.mira.public/spark3_bug/right], PartitionFilters: [], PushedFilters: [IsNotNull(id), IsNotNull(s2_cell_id)], ReadSchema: struct<id:string,s2_cell_id:bigint> at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:56) at org.apache.spark.sql.execution.exchange.ShuffleExchangeLike.$anonfun$materializeFuture$1(ShuffleExchangeExec.scala:101) at org.apache.spark.sql.util.LazyValue.getOrInit(LazyValue.scala:41) at org.apache.spark.sql.execution.exchange.Exchange.getOrInitMaterializeFuture(Exchange.scala:71) at org.apache.spark.sql.execution.exchange.ShuffleExchangeLike.materializeFuture(ShuffleExchangeExec.scala:97) at org.apache.spark.sql.execution.exchange.ShuffleExchangeLike.materialize(ShuffleExchangeExec.scala:85) at org.apache.spark.sql.execution.exchange.ShuffleExchangeLike.materialize$(ShuffleExchangeExec.scala:84) at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.materialize(ShuffleExchangeExec.scala:129) at org.apache.spark.sql.execution.adaptive.ShuffleQueryStageExec.doMaterialize(QueryStageExec.scala:161) at org.apache.spark.sql.execution.adaptive.QueryStageExec.$anonfun$materialize$1(QueryStageExec.scala:74) at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:232) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:229) at org.apache.spark.sql.execution.adaptive.QueryStageExec.materialize(QueryStageExec.scala:74) at org.apache.spark.sql.execution.adaptive.MaterializeExecutable.tryStart(AdaptiveExecutable.scala:396) at org.apache.spark.sql.execution.adaptive.AdaptiveExecutorRuntime.startChild(AdaptiveExecutor.scala:225) at org.apache.spark.sql.execution.adaptive.ExecutionHelper.start(ExecutionHelper.scala:47) at org.apache.spark.sql.execution.adaptive.QueryStageExecutable$$anon$2.$anonfun$new$1(AdaptiveExecutable.scala:251) at org.apache.spark.sql.execution.adaptive.ExecutionHelper$Listener.$anonfun$onChildSuccess$2(ExecutionHelper.scala:55) at org.apache.spark.sql.execution.adaptive.ExecutionHelper$Listener.$anonfun$onChildSuccess$2$adapted(ExecutionHelper.scala:54) at scala.Option.foreach(Option.scala:407) at org.apache.spark.sql.execution.adaptive.ExecutionHelper$Listener.$anonfun$onChildSuccess$1(ExecutionHelper.scala:54) at org.apache.spark.sql.execution.adaptive.ExecutionHelper$Listener.$anonfun$onChildSuccess$1$adapted(ExecutionHelper.scala:53) at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62) at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49) at org.apache.spark.sql.execution.adaptive.ExecutionHelper$Listener.onChildSuccess(ExecutionHelper.scala:53) at org.apache.spark.sql.execution.adaptive.AdaptiveExecutorRuntime.$anonfun$onActiveChildSuccess$2(AdaptiveExecutor.scala:314) at org.apache.spark.sql.execution.adaptive.AdaptiveExecutorRuntime.$anonfun$onActiveChildSuccess$2$adapted(AdaptiveExecutor.scala:314) at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62) at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49) at org.apache.spark.sql.execution.adaptive.AdaptiveExecutorRuntime.onActiveChildSuccess(AdaptiveExecutor.scala:314) at org.apache.spark.sql.execution.adaptive.AdaptiveExecutorRuntime.onChildSuccess(AdaptiveExecutor.scala:284) at org.apache.spark.sql.execution.adaptive.AdaptiveExecutor.$anonfun$doRun$1(AdaptiveExecutor.scala:92) at org.apache.spark.sql.execution.adaptive.AdaptiveExecutor.$anonfun$doRun$1$adapted(AdaptiveExecutor.scala:91) at scala.collection.mutable.HashMap.$anonfun$foreach$1(HashMap.scala:149) at scala.collection.mutable.HashTable.foreachEntry(HashTable.scala:237) at scala.collection.mutable.HashTable.foreachEntry$(HashTable.scala:230) at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:44) at scala.collection.mutable.HashMap.foreach(HashMap.scala:149) at org.apache.spark.sql.execution.adaptive.AdaptiveExecutor.doRun(AdaptiveExecutor.scala:91) at org.apache.spark.sql.execution.adaptive.AdaptiveExecutor.tryRunningAndGetFuture(AdaptiveExecutor.scala:66) at org.apache.spark.sql.execution.adaptive.AdaptiveExecutor.execute(AdaptiveExecutor.scala:57) at org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.$anonfun$getFinalPhysicalPlan$1(AdaptiveSparkPlanExec.scala:184) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:775) at org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.getFinalPhysicalPlan(AdaptiveSparkPlanExec.scala:183) at org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.executeCollect(AdaptiveSparkPlanExec.scala:405) at org.apache.spark.sql.Dataset.collectFromPlan(Dataset.scala:3760) at org.apache.spark.sql.Dataset.$anonfun$head$1(Dataset.scala:2763) at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3751) at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:107) at org.apache.spark.sql.execution.SQLExecution$.withTracker(SQLExecution.scala:232) at org.apache.spark.sql.execution.SQLExecution$.executeQuery$1(SQLExecution.scala:110) at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:135) at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:107) at org.apache.spark.sql.execution.SQLExecution$.withTracker(SQLExecution.scala:232) at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:135) at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:253) at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:134) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:775) at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:68) at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3749) at org.apache.spark.sql.Dataset.head(Dataset.scala:2763) at org.apache.spark.sql.Dataset.take(Dataset.scala:2970) at org.apache.spark.sql.Dataset.getRows(Dataset.scala:303) at org.apache.spark.sql.Dataset.showString(Dataset.scala:340) at org.apache.spark.sql.Dataset.show(Dataset.scala:866) at org.apache.spark.sql.Dataset.show(Dataset.scala:825) at org.apache.spark.sql.Dataset.show(Dataset.scala:834) ... 74 elided Caused by: org.apache.spark.sql.catalyst.errors.package$TreeNodeException: execute, tree: SortAggregate(key=[knownfloatingpointnormalized(if (isnull(start#2116)) null else named_struct(latitude, knownfloatingpointnormalized(normalizenanandzero(start#2116.latitude)), longitude, knownfloatingpointnormalized(normalizenanandzero(start#2116.longitude)))) AS start#2116], functions=[partial_min(id#2103)], output=[start#2116, min#2138]) +- *(2) Sort [knownfloatingpointnormalized(if (isnull(start#2116)) null else named_struct(latitude, knownfloatingpointnormalized(normalizenanandzero(start#2116.latitude)), longitude, knownfloatingpointnormalized(normalizenanandzero(start#2116.longitude)))) AS start#2116 ASC NULLS FIRST], false, 0 +- *(2) Project [_gen_alias_2133#2133 AS start#2116, id#2103] +- *(2) !BroadcastHashJoin [s2_cell_id#2108L], [s2_cell_id#2104L], Inner, BuildLeft, NOT (cast(id#2103 as double) = _gen_alias_2134#2134), false :- BroadcastQueryStage 0 : +- BroadcastExchange HashedRelationBroadcastMode(List(input[1, bigint, false]),false), [id=#3768] : +- *(1) Project [row#2107.start AS _gen_alias_2133#2133, s2_cell_id#2108L] : +- *(1) Filter isnotnull(s2_cell_id#2108L) : +- FileScan parquet [row#2107,s2_cell_id#2108L] Batched: false, DataFilters: [isnotnull(s2_cell_id#2108L)], Format: Parquet, Location: InMemoryFileIndex[s3://co.mira.public/spark3_bug/left], PartitionFilters: [], PushedFilters: [IsNotNull(s2_cell_id)], ReadSchema: struct<row:struct<start:struct<latitude:double,longitude:double>>,s2_cell_id:bigint> +- *(2) Filter (isnotnull(id#2103) AND isnotnull(s2_cell_id#2104L)) +- *(2) ColumnarToRow +- FileScan parquet [id#2103,s2_cell_id#2104L] Batched: true, DataFilters: [isnotnull(id#2103), isnotnull(s2_cell_id#2104L)], Format: Parquet, Location: InMemoryFileIndex[s3://co.mira.public/spark3_bug/right], PartitionFilters: [], PushedFilters: [IsNotNull(id), IsNotNull(s2_cell_id)], ReadSchema: struct<id:string,s2_cell_id:bigint> at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:56) at org.apache.spark.sql.execution.aggregate.SortAggregateExec.doExecute(SortAggregateExec.scala:54) at org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:194) at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:232) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:229) at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:190) at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.inputRDD$lzycompute(ShuffleExchangeExec.scala:171) at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.inputRDD(ShuffleExchangeExec.scala:171) at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.mapOutputStatisticsFuture$lzycompute(ShuffleExchangeExec.scala:175) at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.mapOutputStatisticsFuture(ShuffleExchangeExec.scala:174) at org.apache.spark.sql.execution.exchange.ShuffleExchangeLike.$anonfun$materializeFuture$2(ShuffleExchangeExec.scala:101) at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:52) ... 143 more Caused by: org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree: _gen_alias_2134#2134 at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:56) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:75) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:74) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$1(TreeNode.scala:329) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:75) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:329) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$3(TreeNode.scala:334) at org.apache.spark.sql.catalyst.trees.TreeNode.applyFunctionIfChanged$1(TreeNode.scala:388) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$mapChildren$1(TreeNode.scala:424) at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:256) at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:422) at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:370) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:334) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$3(TreeNode.scala:334) at org.apache.spark.sql.catalyst.trees.TreeNode.applyFunctionIfChanged$1(TreeNode.scala:388) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$mapChildren$1(TreeNode.scala:424) at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:256) at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:422) at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:370) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:334) at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:318) at org.apache.spark.sql.catalyst.expressions.BindReferences$.bindReference(BoundAttribute.scala:74) at org.apache.spark.sql.execution.joins.JoinCodegenSupport.getJoinCondition(JoinCodegenSupport.scala:52) at org.apache.spark.sql.execution.joins.JoinCodegenSupport.getJoinCondition$(JoinCodegenSupport.scala:38) at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.getJoinCondition(BroadcastHashJoinExec.scala:40) at org.apache.spark.sql.execution.joins.HashJoin.codegenInner(HashJoin.scala:392) at org.apache.spark.sql.execution.joins.HashJoin.codegenInner$(HashJoin.scala:389) at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.codegenInner(BroadcastHashJoinExec.scala:40) at org.apache.spark.sql.execution.joins.HashJoin.doConsume(HashJoin.scala:356) at org.apache.spark.sql.execution.joins.HashJoin.doConsume$(HashJoin.scala:354) at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.doConsume(BroadcastHashJoinExec.scala:40) at org.apache.spark.sql.execution.CodegenSupport.constructDoConsumeFunction(WholeStageCodegenExec.scala:282) at org.apache.spark.sql.execution.CodegenSupport.consume(WholeStageCodegenExec.scala:253) at org.apache.spark.sql.execution.CodegenSupport.consume$(WholeStageCodegenExec.scala:210) at org.apache.spark.sql.execution.FilterExec.consume(basicPhysicalOperators.scala:161) at org.apache.spark.sql.execution.FilterExec.doConsume(basicPhysicalOperators.scala:283) at org.apache.spark.sql.execution.CodegenSupport.consume(WholeStageCodegenExec.scala:255) at org.apache.spark.sql.execution.CodegenSupport.consume$(WholeStageCodegenExec.scala:210) at org.apache.spark.sql.execution.ColumnarToRowExec.consume(Columnar.scala:66) at org.apache.spark.sql.execution.ColumnarToRowExec.doProduce(Columnar.scala:191) 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:232) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:229) 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.ColumnarToRowExec.produce(Columnar.scala:66) at org.apache.spark.sql.execution.FilterExec.doProduce(basicPhysicalOperators.scala:198) 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:232) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:229) 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.FilterExec.produce(basicPhysicalOperators.scala:161) at org.apache.spark.sql.execution.joins.HashJoin.doProduce(HashJoin.scala:351) at org.apache.spark.sql.execution.joins.HashJoin.doProduce$(HashJoin.scala:350) 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:232) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:229) 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:54) 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:232) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:229) 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:41) at org.apache.spark.sql.execution.SortExec.doProduce(SortExec.scala:173) 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:232) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:229) 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.SortExec.produce(SortExec.scala:41) at org.apache.spark.sql.execution.WholeStageCodegenExec.doCodeGen(WholeStageCodegenExec.scala:726) at org.apache.spark.sql.execution.WholeStageCodegenExec.doExecute(WholeStageCodegenExec.scala:795) at org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:194) at org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:232) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:229) at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:190) at org.apache.spark.sql.execution.aggregate.SortAggregateExec.$anonfun$doExecute$1(SortAggregateExec.scala:56) at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:52) ... 155 more Caused by: java.lang.RuntimeException: Couldn't find _gen_alias_2134#2134 in [id#2103,s2_cell_id#2104L,_gen_alias_2133#2133,s2_cell_id#2108L] at scala.sys.package$.error(package.scala:30) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.$anonfun$applyOrElse$1(BoundAttribute.scala:81) at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:52) ... 244 more