[ 
https://issues.apache.org/jira/browse/SPARK-48155?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Wenchen Fan reassigned SPARK-48155:
-----------------------------------

    Assignee: angerszhu

> PropagateEmpty relation cause LogicalQueryStage only with broadcast without 
> join then execute failed
> ----------------------------------------------------------------------------------------------------
>
>                 Key: SPARK-48155
>                 URL: https://issues.apache.org/jira/browse/SPARK-48155
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 3.2.1, 3.5.1, 3.3.4
>            Reporter: angerszhu
>            Assignee: angerszhu
>            Priority: Major
>              Labels: pull-request-available
>
> {code:java}
> 24/05/07 09:48:55 ERROR [main] PlanChangeLogger:
> === Applying Rule 
> org.apache.spark.sql.execution.adaptive.AQEPropagateEmptyRelation ===
>  Project [date#124, station_name#0, shipment_id#14]
>  +- Filter (status#2L INSET 1, 149, 2, 36, 400, 417, 418, 419, 49, 5, 50, 581 
> AND station_type#1 IN (3,12))
>     +- Aggregate [date#124, shipment_id#14], [date#124, shipment_id#14, ... 3 
> more fields] 
> !      +- Join LeftOuter, ((cast(date#124 as timestamp) >= 
> cast(from_unixtime((ctime#27L - 0), yyyy-MM-dd HH:mm:ss, 
> Some(Asia/Singapore)) as timestamp)) AND (cast(date#124 as timestamp) + 
> INTERVAL '-4' DAY <= cast(from_unixtime((ctime#27L - 0), yyyy-MM-dd HH:mm:ss, 
> Some(Asia/Singapore)) as timestamp)))
> !         :- LogicalQueryStage Generate 
> explode(org.apache.spark.sql.catalyst.expressions.UnsafeArrayData@3a191e40), 
> false, [date#124], BroadcastQueryStage 0
> !         +- LocalRelation <empty>, [shipment_id#14, station_name#5, ... 3 
> more fields]24/05/07 09:48:55 ERROR [main] 
> Project [date#124, station_name#0, shipment_id#14]
>  +- Filter (status#2L INSET 1, 149, 2, 36, 400, 417, 418, 419, 49, 5, 50, 581 
> AND station_type#1 IN (3,12))
>     +- Aggregate [date#124, shipment_id#14], [date#124, shipment_id#14, ... 3 
> more fields]
> !      +- Project [date#124, cast(null as string) AS shipment_id#14, ... 4 
> more fields]
> !         +- LogicalQueryStage Generate 
> explode(org.apache.spark.sql.catalyst.expressions.UnsafeArrayData@3a191e40), 
> false, [date#124], BroadcastQueryStage 0 {code}
> {code:java}
> java.util.concurrent.ExecutionException: java.lang.RuntimeException: 
> java.lang.UnsupportedOperationException: BroadcastExchange does not support 
> the execute() code path.    at 
> org.apache.spark.sql.errors.QueryExecutionErrors$.executeCodePathUnsupportedError(QueryExecutionErrors.scala:1652)
>     at 
> org.apache.spark.sql.execution.exchange.BroadcastExchangeExec.doExecute(BroadcastExchangeExec.scala:203)
>     at 
> org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:184)
>     at 
> org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:222)
>     at 
> org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
>     at 
> org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:219)    
> at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:180)    
> at 
> org.apache.spark.sql.execution.adaptive.QueryStageExec.doExecute(QueryStageExec.scala:119)
>     at 
> org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:184)
>     at 
> org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:222)
>     at 
> org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
>     at 
> org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:219)    
> at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:180)    
> at 
> org.apache.spark.sql.execution.InputAdapter.inputRDD(WholeStageCodegenExec.scala:526)
>     at 
> org.apache.spark.sql.execution.InputRDDCodegen.inputRDDs(WholeStageCodegenExec.scala:454)
>     at 
> org.apache.spark.sql.execution.InputRDDCodegen.inputRDDs$(WholeStageCodegenExec.scala:453)
>     at 
> org.apache.spark.sql.execution.InputAdapter.inputRDDs(WholeStageCodegenExec.scala:497)
>     at 
> org.apache.spark.sql.execution.ProjectExec.inputRDDs(basicPhysicalOperators.scala:50)
>     at org.apache.spark.sql.execution.SortExec.inputRDDs(SortExec.scala:132)  
>   at 
> org.apache.spark.sql.execution.WholeStageCodegenExec.doExecute(WholeStageCodegenExec.scala:750)
>     at 
> org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:184)
>     at 
> org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:222)
>     at 
> org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
>     at 
> org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:219)    
> at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:180)    
> at 
> org.apache.spark.sql.execution.aggregate.SortAggregateExec.doExecute(SortAggregateExec.scala:55)
>     at 
> org.apache.spark.sql.execution.SparkPlan.$anonfun$execute$1(SparkPlan.scala:184)
>     at 
> org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:222)
>     at 
> org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
>     at 
> org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:219)    
> at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:180)    
> at 
> org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.inputRDD$lzycompute(ShuffleExchangeExec.scala:144)
>     at 
> org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.inputRDD(ShuffleExchangeExec.scala:144)
>     at 
> org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.mapOutputStatisticsFuture$lzycompute(ShuffleExchangeExec.scala:149)
>     at 
> org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.mapOutputStatisticsFuture(ShuffleExchangeExec.scala:148)
>     at 
> org.apache.spark.sql.execution.exchange.ShuffleExchangeLike.$anonfun$submitShuffleJob$1(ShuffleExchangeExec.scala:70)
>     at 
> org.apache.spark.sql.execution.SparkPlan.$anonfun$executeQuery$1(SparkPlan.scala:222)
>     at 
> org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
>     at 
> org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:219)    
> at 
> org.apache.spark.sql.execution.exchange.ShuffleExchangeLike.submitShuffleJob(ShuffleExchangeExec.scala:70)
>     at 
> org.apache.spark.sql.execution.exchange.ShuffleExchangeLike.submitShuffleJob$(ShuffleExchangeExec.scala:69)
>     at 
> org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.submitShuffleJob(ShuffleExchangeExec.scala:117)
>     at 
> org.apache.spark.sql.execution.adaptive.ShuffleQueryStageExec.shuffleFuture$lzycompute(QueryStageExec.scala:170)
>     at 
> org.apache.spark.sql.execution.adaptive.ShuffleQueryStageExec.shuffleFuture(QueryStageExec.scala:170)
>     at 
> org.apache.spark.sql.execution.adaptive.ShuffleQueryStageExec.doMaterialize(QueryStageExec.scala:172)
>     at 
> org.apache.spark.sql.execution.adaptive.QueryStageExec.materialize(QueryStageExec.scala:82)
>     at 
> org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.$anonfun$getFinalPhysicalPlan$5(AdaptiveSparkPlanExec.scala:257)
>     at 
> org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.$anonfun$getFinalPhysicalPlan$5$adapted(AdaptiveSparkPlanExec.scala:255)
>     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 
> org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.$anonfun$getFinalPhysicalPlan$1(AdaptiveSparkPlanExec.scala:255)
>     at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:782)   
>  at 
> org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.getFinalPhysicalPlan(AdaptiveSparkPlanExec.scala:227)
>     at 
> org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.withFinalPlanUpdate(AdaptiveSparkPlanExec.scala:366)
>     at 
> org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec.executeCollect(AdaptiveSparkPlanExec.scala:339)
>     at org.apache.spark.sql.Dataset.collectFromPlan(Dataset.scala:3715)    at 
> org.apache.spark.sql.Dataset.$anonfun$collectAsList$1(Dataset.scala:2983)    
> at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3706)    
> at 
> org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:104)
>     at 
> org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:170)
>     at 
> org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:91)
>     at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:782) 
> {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org
For additional commands, e-mail: issues-h...@spark.apache.org

Reply via email to