You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "HeartSaVioR (via GitHub)" <gi...@apache.org> on 2023/09/13 04:46:41 UTC

[GitHub] [spark] HeartSaVioR commented on a diff in pull request #42822: [SPARK-45084][SS] ProgressReport to include accurate effective shuffle partition number

HeartSaVioR commented on code in PR #42822:
URL: https://github.com/apache/spark/pull/42822#discussion_r1323963218


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala:
##########
@@ -264,9 +265,16 @@ trait ProgressReporter extends Logging {
     if (lastExecution == null) return Nil
     // lastExecution could belong to one of the previous triggers if `!hasExecuted`.
     // Walking the plan again should be inexpensive.
+
+    val shufflePartitionValue = sparkSession.conf.getOption(SHUFFLE_PARTITIONS.key).getOrElse("-1")
+    val numShufflePartitions: Long = try {
+      shufflePartitionValue.toLong
+    } catch {
+      case e: NumberFormatException => -1L
+    }
     lastExecution.executedPlan.collect {
       case p if p.isInstanceOf[StateStoreWriter] =>
-        val progress = p.asInstanceOf[StateStoreWriter].getProgress()
+        val progress = p.asInstanceOf[StateStoreWriter].getProgress(numShufflePartitions)

Review Comment:
   Shall we read the value from the physical plan? StateStoreWriter has a method `stateInfo` which you can find a number of shuffle partitions. That value is closer to the reality, as we build a child distribution requirement based on that value.
   
   https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulOperatorPartitioning.scala
   
   This is also future-proof - if we want to set the different shuffle partitions per operator, we will set the value differently for stateInfo.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org