Details
-
Bug
-
Status: Resolved
-
Major
-
Resolution: Fixed
-
3.2.1, 3.5.1, 3.3.4
Description
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
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)