Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-48155

PropagateEmpty relation cause LogicalQueryStage only with broadcast without join then execute failed

    XMLWordPrintableJSON

Details

    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) 

      Attachments

        Activity

          People

            angerszhuuu angerszhu
            angerszhuuu angerszhu
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: