Z1Wu opened a new issue, #9279:
URL: https://github.com/apache/incubator-gluten/issues/9279

   ### Backend
   
   VL (Velox)
   
   ### Bug description
   
   Pull out expression from `PartialMerge` aggregate function results in 
invalid reference binding in `ProjectExecTransformer ` when there are 
`merge_sum` and `partial_count` aggreagate function with non-attribute 
expression in same `HashAggregate`.
   
   Error Physical Plan looks like below:
   
   ``` sql
    ProjectExecTransformer [(sum(cast(f1#13 as double))#17 / cast(count(f2#14, 
f3#15)#18L as double)) AS f#12]                                                 
                                                                                
                                                                                
                                           ProjectExecTransformer 
[(sum(cast(f1#13 as double))#17 / cast(count(f2#14, f3#15)#18L as double)) AS 
f#12]
    +- HashAggregateTransformer(keys=[f4#16], functions=[sum(cast(f1#13 as 
double)), count(distinct f2#14, f3#15)], isStreamingAgg=false, output=[f4#16, 
sum(cast(f1#13 as double))#17, count(f2#14, f3#15)#18L])                        
                                                                                
                                                 +- 
HashAggregateTransformer(keys=[f4#16], functions=[sum(cast(f1#13 as double)), 
count(distinct f2#14, f3#15)], isStreamingAgg=false, output=[f4#16, 
sum(cast(f1#13 as double))#17, count(f2#14, f3#15)#18L])
      +- Exchange hashpartitioning(f4#16, 5), ENSURE_REQUIREMENTS, 
[plan_id=588]                                                                   
                                                                                
                                                                                
                                                         +- Exchange 
hashpartitioning(f4#16, 5), ENSURE_REQUIREMENTS, [plan_id=602]
         +- HashAggregateTransformer(keys=[f4#16], 
functions=[merge_sum(_pre_1#31), partial_count(distinct _pre_2#32)], 
isStreamingAgg=false, output=[f4#16, sum#22, count#25L])                        
                                                                                
                                                                                
       +- FlushableHashAggregateTransformer(keys=[f4#16], 
functions=[merge_sum(_pre_1#31), partial_count(distinct _pre_2#32)], 
isStreamingAgg=false, output=[f4#16, sum#22, count#25L])
             +- !Project [f4#16, sum#22, cast(f1#13 as double) AS _pre_1#31, if 
((isnull(f2#14) OR isnull(f3#15))) null else 1 AS _pre_2#32]                    
                                                                                
                                                                                
                                                   +- !Project [f4#16, sum#22, 
cast(f1#13 as double) AS _pre_1#31, if ((isnull(f2#14) OR isnull(f3#15))) null 
else 1 AS _pre_2#32]
                +- HashAggregateTransformer(keys=[f4#16, f2#14, f3#15], 
functions=[merge_sum(cast(f1#13 as double))], isStreamingAgg=false, 
output=[f4#16, f2#14, f3#15, sum#22])                                           
                                                                                
                                                                          +- 
HashAggregateTransformer(keys=[f4#16, f2#14, f3#15], 
functions=[merge_sum(cast(f1#13 as double))], isStreamingAgg=false, 
output=[f4#16, f2#14, f3#15, sum#22])
                  +- Exchange hashpartitioning(f4#16, f2#14, f3#15, 5), 
ENSURE_REQUIREMENTS, [plan_id=582]                                              
                                                                                
                                                                                
                                                                +- Exchange 
hashpartitioning(f4#16, f2#14, f3#15, 5), ENSURE_REQUIREMENTS, [plan_id=596]
                     +- HashAggregateTransformer(keys=[f4#16, f2#14, f3#15], 
functions=[partial_sum(_pre_0#27)], isStreamingAgg=false, output=[f4#16, f2#14, 
f3#15, sum#22])                                                                 
                                                                                
                                                              +- 
FlushableHashAggregateTransformer(keys=[f4#16, f2#14, f3#15], 
functions=[partial_sum(_pre_0#27)], isStreamingAgg=false, output=[f4#16, f2#14, 
f3#15, sum#22])
                         +- ProjectExecTransformer [f2#14, f3#15, f4#16, 
cast(f1#13 as double) AS _pre_0#27]                                             
                                                                                
                                                                                
                                                                      +- 
ProjectExecTransformer [f2#14, f3#15, f4#16, cast(f1#13 as double) AS _pre_0#27]
                            +- FileScan csv 
spark_catalog.default.tmp_table[f1#13,f2#14,f3#15,f4#16] Batched: false, 
DataFilters: [], Format: CSV, Location: InMemoryFileIndex(1 
paths)[file:/home/wuzy/my_code/incubator-gluten/spark-warehouse/org.apache.gl...,
 PartitionFilters: [], PushedFilters: [], ReadSchema: 
struct<f1:string,f2:string,f3:string,f4:string>                           +- 
FileScan csv spark_catalog.default.tmp_table[f1#13,f2#14,f3#15,f4#16] Batched: 
false, DataFilters: [], Format: CSV, Location: InMemoryFileIndex(1 
paths)[file:/home/wuzy/my_code/incubator-gluten/spark-warehouse/org.apache.gl...,
 PartitionFilters: [], PushedFilters: [], ReadSchema: 
struct<f1:string,f2:string,f3:string,f4:string>
   
   ```
   
   Sample SQL to reproduce: 
   
   ```sql
   withTable("t") {
       sql("CREATE TABLE t(f1 String, f2 String, f3 String, f4 String) USING 
CSV")
       sql("INSERT INTO t values ('1', '2', '3', '4'), ('11' ,'22', '33', '4')")
       val df = sql(
         """
           |SELECT SUM(f1) / COUNT(DISTINCT f2, f3) FROM t GROUP BY f4;
           |""".stripMargin)
       assert(df.collect()(0).getDouble(0) == 6)
     }
   ```
   
   
   
   ### Spark version
   
   Spark-3.5.x
   
   ### Spark configurations
   
   _No response_
   
   ### System information
   
   Gluten Version: 1.5.0-SNAPSHOT
   Commit: 90df6ee3e75223334657fa67604d5570b5d4c2a2
   CMake Version: 3.28.3
   System: Linux-5.15.167.4-microsoft-standard-WSL2
   Arch: x86_64
   CPU Name: Model name:                           12th Gen Intel(R) Core(TM) 
i5-12600KF
   C++ Compiler: /usr/bin/c++
   C++ Compiler Version: 11.4.0
   C Compiler: /usr/bin/cc
   C Compiler Version: 11.4.0
   CMake Prefix Path: 
/usr/local;/usr;/;/usr/local/lib/python3.8/dist-packages/cmake/data;/usr/local;/usr/X11R6;/usr/pkg;/opt
   
   
   ### Relevant logs
   
   ```bash
   Couldn't find f1#13 in [f4#16,f2#14,f3#15,sum#25]
   java.lang.IllegalStateException: Couldn't find f1#13 in 
[f4#16,f2#14,f3#15,sum#25]
        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.UnaryLike.mapChildren(TreeNode.scala:1216)
        at 
org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1215)
        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:1216)
        at 
org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1215)
        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.TraversableLike.$anonfun$map$1(TraversableLike.scala:286)
        at scala.collection.Iterator.foreach(Iterator.scala:943)
        at scala.collection.Iterator.foreach$(Iterator.scala:943)
        at scala.collection.AbstractIterator.foreach(Iterator.scala:1431)
        at scala.collection.IterableLike.foreach(IterableLike.scala:74)
        at scala.collection.IterableLike.foreach$(IterableLike.scala:73)
        at scala.collection.AbstractIterable.foreach(Iterable.scala:56)
        at scala.collection.TraversableLike.map(TraversableLike.scala:286)
        at scala.collection.TraversableLike.map$(TraversableLike.scala:279)
        at scala.collection.AbstractTraversable.map(Traversable.scala:108)
        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.InputAdapter.consume(WholeStageCodegenExec.scala:498)
        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.InputAdapter.doProduce(WholeStageCodegenExec.scala:498)
        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.InputAdapter.produce(WholeStageCodegenExec.scala:498)
        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.gluten.execution.RowToVeloxColumnarExec.doExecuteColumnarInternal(RowToVeloxColumnarExec.scala:55)
        at 
org.apache.gluten.execution.RowToColumnarExecBase.doExecuteColumnar(RowToColumnarExecBase.scala:68)
        at 
org.apache.spark.sql.execution.SparkPlan.$anonfun$executeColumnar$1(SparkPlan.scala:222)
        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.executeColumnar(SparkPlan.scala:218)
        at 
org.apache.spark.sql.execution.ColumnarInputAdapter.doExecuteColumnar(ColumnarCollapseTransformStages.scala:193)
        at 
org.apache.spark.sql.execution.SparkPlan.$anonfun$executeColumnar$1(SparkPlan.scala:222)
        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.executeColumnar(SparkPlan.scala:218)
        at 
org.apache.gluten.execution.TransformSupport.getColumnarInputRDDs(WholeStageTransformer.scala:197)
        at 
org.apache.gluten.execution.TransformSupport.getColumnarInputRDDs$(WholeStageTransformer.scala:192)
        at 
org.apache.spark.sql.execution.InputIteratorTransformer.getColumnarInputRDDs(ColumnarCollapseTransformStages.scala:48)
        at 
org.apache.gluten.execution.UnaryTransformSupport.columnarInputRDDs(WholeStageTransformer.scala:213)
        at 
org.apache.gluten.execution.UnaryTransformSupport.columnarInputRDDs$(WholeStageTransformer.scala:212)
        at 
org.apache.spark.sql.execution.InputIteratorTransformer.columnarInputRDDs(ColumnarCollapseTransformStages.scala:48)
        at 
org.apache.gluten.execution.TransformSupport.getColumnarInputRDDs(WholeStageTransformer.scala:195)
        at 
org.apache.gluten.execution.TransformSupport.getColumnarInputRDDs$(WholeStageTransformer.scala:192)
        at 
org.apache.gluten.execution.HashAggregateExecBaseTransformer.getColumnarInputRDDs(HashAggregateExecBaseTransformer.scala:36)
        at 
org.apache.gluten.execution.UnaryTransformSupport.columnarInputRDDs(WholeStageTransformer.scala:213)
        at 
org.apache.gluten.execution.UnaryTransformSupport.columnarInputRDDs$(WholeStageTransformer.scala:212)
        at 
org.apache.gluten.execution.HashAggregateExecBaseTransformer.columnarInputRDDs(HashAggregateExecBaseTransformer.scala:36)
        at 
org.apache.gluten.execution.TransformSupport.getColumnarInputRDDs(WholeStageTransformer.scala:195)
        at 
org.apache.gluten.execution.TransformSupport.getColumnarInputRDDs$(WholeStageTransformer.scala:192)
        at 
org.apache.gluten.execution.WholeStageTransformer.getColumnarInputRDDs(WholeStageTransformer.scala:217)
        at 
org.apache.gluten.execution.UnaryTransformSupport.columnarInputRDDs(WholeStageTransformer.scala:213)
        at 
org.apache.gluten.execution.UnaryTransformSupport.columnarInputRDDs$(WholeStageTransformer.scala:212)
        at 
org.apache.gluten.execution.WholeStageTransformer.columnarInputRDDs(WholeStageTransformer.scala:217)
        at 
org.apache.gluten.execution.WholeStageTransformer.doExecuteColumnar(WholeStageTransformer.scala:533)
        at 
org.apache.spark.sql.execution.SparkPlan.$anonfun$executeColumnar$1(SparkPlan.scala:222)
        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.executeColumnar(SparkPlan.scala:218)
        at 
org.apache.gluten.execution.VeloxColumnarToRowExec.doExecuteInternal(VeloxColumnarToRowExec.scala:75)
        at 
org.apache.gluten.execution.ColumnarToRowExecBase.doExecute(ColumnarToRowExecBase.scala:61)
        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.exchange.ShuffleExchangeExec.inputRDD$lzycompute(ShuffleExchangeExec.scala:141)
        at 
org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.inputRDD(ShuffleExchangeExec.scala:141)
        at 
org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.shuffleDependency$lzycompute(ShuffleExchangeExec.scala:175)
        at 
org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.shuffleDependency(ShuffleExchangeExec.scala:173)
        at 
org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.doExecute(ShuffleExchangeExec.scala:195)
        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.gluten.execution.RowToVeloxColumnarExec.doExecuteColumnarInternal(RowToVeloxColumnarExec.scala:55)
        at 
org.apache.gluten.execution.RowToColumnarExecBase.doExecuteColumnar(RowToColumnarExecBase.scala:68)
        at 
org.apache.spark.sql.execution.SparkPlan.$anonfun$executeColumnar$1(SparkPlan.scala:222)
        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.executeColumnar(SparkPlan.scala:218)
        at 
org.apache.spark.sql.execution.ColumnarInputAdapter.doExecuteColumnar(ColumnarCollapseTransformStages.scala:193)
        at 
org.apache.spark.sql.execution.SparkPlan.$anonfun$executeColumnar$1(SparkPlan.scala:222)
        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.executeColumnar(SparkPlan.scala:218)
        at 
org.apache.gluten.execution.TransformSupport.getColumnarInputRDDs(WholeStageTransformer.scala:197)
        at 
org.apache.gluten.execution.TransformSupport.getColumnarInputRDDs$(WholeStageTransformer.scala:192)
        at 
org.apache.spark.sql.execution.InputIteratorTransformer.getColumnarInputRDDs(ColumnarCollapseTransformStages.scala:48)
        at 
org.apache.gluten.execution.UnaryTransformSupport.columnarInputRDDs(WholeStageTransformer.scala:213)
        at 
org.apache.gluten.execution.UnaryTransformSupport.columnarInputRDDs$(WholeStageTransformer.scala:212)
        at 
org.apache.spark.sql.execution.InputIteratorTransformer.columnarInputRDDs(ColumnarCollapseTransformStages.scala:48)
        at 
org.apache.gluten.execution.TransformSupport.getColumnarInputRDDs(WholeStageTransformer.scala:195)
        at 
org.apache.gluten.execution.TransformSupport.getColumnarInputRDDs$(WholeStageTransformer.scala:192)
        at 
org.apache.gluten.execution.HashAggregateExecBaseTransformer.getColumnarInputRDDs(HashAggregateExecBaseTransformer.scala:36)
        at 
org.apache.gluten.execution.UnaryTransformSupport.columnarInputRDDs(WholeStageTransformer.scala:213)
        at 
org.apache.gluten.execution.UnaryTransformSupport.columnarInputRDDs$(WholeStageTransformer.scala:212)
        at 
org.apache.gluten.execution.HashAggregateExecBaseTransformer.columnarInputRDDs(HashAggregateExecBaseTransformer.scala:36)
        at 
org.apache.gluten.execution.TransformSupport.getColumnarInputRDDs(WholeStageTransformer.scala:195)
        at 
org.apache.gluten.execution.TransformSupport.getColumnarInputRDDs$(WholeStageTransformer.scala:192)
        at 
org.apache.gluten.execution.WholeStageTransformer.getColumnarInputRDDs(WholeStageTransformer.scala:217)
        at 
org.apache.gluten.execution.UnaryTransformSupport.columnarInputRDDs(WholeStageTransformer.scala:213)
        at 
org.apache.gluten.execution.UnaryTransformSupport.columnarInputRDDs$(WholeStageTransformer.scala:212)
        at 
org.apache.gluten.execution.WholeStageTransformer.columnarInputRDDs(WholeStageTransformer.scala:217)
        at 
org.apache.gluten.execution.WholeStageTransformer.doExecuteColumnar(WholeStageTransformer.scala:533)
        at 
org.apache.spark.sql.execution.SparkPlan.$anonfun$executeColumnar$1(SparkPlan.scala:222)
        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.executeColumnar(SparkPlan.scala:218)
        at 
org.apache.gluten.execution.VeloxColumnarToRowExec.doExecuteInternal(VeloxColumnarToRowExec.scala:75)
        at 
org.apache.gluten.execution.ColumnarToRowExecBase.doExecute(ColumnarToRowExecBase.scala:61)
        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.InputAdapter.inputRDD(WholeStageCodegenExec.scala:527)
        at 
org.apache.spark.sql.execution.InputRDDCodegen.inputRDDs(WholeStageCodegenExec.scala:455)
        at 
org.apache.spark.sql.execution.InputRDDCodegen.inputRDDs$(WholeStageCodegenExec.scala:454)
        at 
org.apache.spark.sql.execution.InputAdapter.inputRDDs(WholeStageCodegenExec.scala:498)
        at 
org.apache.spark.sql.execution.ProjectExec.inputRDDs(basicPhysicalOperators.scala:51)
        at 
org.apache.spark.sql.execution.WholeStageCodegenExec.doExecute(WholeStageCodegenExec.scala:751)
        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.executeTake(SparkPlan.scala:498)
        at 
org.apache.spark.sql.execution.SparkPlan.executeTake(SparkPlan.scala:483)
        at 
org.apache.spark.sql.execution.CollectLimitExec.executeCollect(limit.scala:61)
        at org.apache.spark.sql.Dataset.collectFromPlan(Dataset.scala:4334)
        at org.apache.spark.sql.Dataset.$anonfun$head$1(Dataset.scala:3316)
        at 
org.apache.spark.sql.Dataset.$anonfun$withAction$2(Dataset.scala:4324)
        at 
org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:546)
        at 
org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:4322)
        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:4322)
        at org.apache.spark.sql.Dataset.head(Dataset.scala:3316)
        at org.apache.spark.sql.Dataset.take(Dataset.scala:3539)
        at org.apache.spark.sql.Dataset.getRows(Dataset.scala:280)
        at org.apache.spark.sql.Dataset.showString(Dataset.scala:315)
        at org.apache.spark.sql.Dataset.show(Dataset.scala:838)
        at org.apache.spark.sql.Dataset.show(Dataset.scala:797)
        at org.apache.spark.sql.Dataset.show(Dataset.scala:806)
        at 
org.apache.gluten.extension.GlutenExtensionRewriteRuleSuite.$anonfun$new$3(GlutenExtensionRewriteRuleSuite.scala:89)
        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:43)
        at 
org.jetbrains.plugins.scala.testingSupport.scalaTest.ScalaTestRunner.main(ScalaTestRunner.java:26)
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to