[
https://issues.apache.org/jira/browse/SPARK-37667?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Kellan B Cummings updated SPARK-37667:
--------------------------------------
Summary: Spark throws TreeNodeException ("Couldn't find gen_alias") during
wildcard column expansion (was: Spark throws TreeNodeException during wildcard
column expansion)
> Spark throws TreeNodeException ("Couldn't find gen_alias") during wildcard
> column expansion
> -------------------------------------------------------------------------------------------
>
> Key: SPARK-37667
> URL: https://issues.apache.org/jira/browse/SPARK-37667
> Project: Spark
> Issue Type: Bug
> Components: SQL
> Affects Versions: 3.1.2
> Reporter: Kellan B Cummings
> Priority: Major
>
> 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*
> [email protected]:kellanburket/spark3bug.git
>
> {code:java}
> val rightDf = spark.read.parquet("right.parquet")
> val leftDf = spark.read.parquet("left.parquet"){code}
>
> *Schemas*
> {code:java}
> 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){code}
> {code:java}
> rightDf.printSchema()
> root
> |-- id: string (nullable = true)
> |-- s2_cell_id: long (nullable = true){code}
>
> *Breaking Code*
> {code:java}
> leftDf.join(rightDf, "s2_cell_id").filter(
> "id != row.start.latitude"
> ).select(
> col("row.*"), col("id")
> ).groupBy(
> "start"
> ).agg(
> min("id")
> ).show(){code}
>
> *Working Examples*
> The following examples don't seem to be effected by the bug
> Works without group by:
> {code:java}
> leftDf.join(rightDf, "s2_cell_id").filter(
> "id != row.start.latitude"
> ).select(
> col("row.*"), col("id")
> ).show(){code}
> Works without filter
> {code:java}
> leftDf.join(rightDf, "s2_cell_id").select(
> col("row.*"), col("id")
> ).groupBy(
> "start"
> ).agg(
> min("id")
> ).show(){code}
> Works without variable expansion
> {code:java}
> leftDf.join(rightDf, "s2_cell_id").filter(
> "id != row.start.latitude"
> ).select(
> col("row.start"), col("id")
> ).groupBy(
> "start"
> ).agg(
> min("id")
> ).show(){code}
> Works with caching
> {code:java}
> leftDf.join(rightDf, "s2_cell_id").filter(
> "id != row.start.latitude"
> ).cache().select(
> col("row.*"),
> col("id")
> ).groupBy(
> "start"
> ).agg(
> min("id")
> ).show(){code}
> *Error message*
>
>
> {code:java}
> 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{code}
>
--
This message was sent by Atlassian Jira
(v8.20.1#820001)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]