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/02/10 07:53:18 UTC

[GitHub] [spark] HeartSaVioR commented on a diff in pull request #39931: [SPARK-42376][SS] Introduce watermark propagation among operators

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala:
##########
@@ -113,20 +114,28 @@ class IncrementalExecution(
       numStateStores)
   }
 
-  // Watermarks to use for late record filtering and state eviction in stateful operators.
-  // Using the previous watermark for late record filtering is a Spark behavior change so we allow
-  // this to be disabled.
-  val eventTimeWatermarkForEviction = offsetSeqMetadata.batchWatermarkMs
-  val eventTimeWatermarkForLateEvents =
-    if (sparkSession.conf.get(SQLConf.STATEFUL_OPERATOR_ALLOW_MULTIPLE)) {
-      prevOffsetSeqMetadata.getOrElse(offsetSeqMetadata).batchWatermarkMs
-    } else {
-      eventTimeWatermarkForEviction
-    }
-
   /** Locates save/restore pairs surrounding aggregation. */
-  val state = new Rule[SparkPlan] {
+  val shufflePartitionsRule = new Rule[SparkPlan] {
+    override def apply(plan: SparkPlan): SparkPlan = plan transform {
+      // NOTE: we should include all aggregate execs here which are used in streaming aggregations
+      case a: SortAggregateExec if a.isStreaming =>

Review Comment:
   The amount of change would be similar - I see one way to do this, `def withNewNumShufflePartitions(numStateStores: Long): BaseAggregateExec` but this may need to be implemented per class. All logical/physical nodes are case class.



-- 
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