You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2022/03/14 08:28:16 UTC

[GitHub] [spark] xuanyuanking commented on a change in pull request #35673: [SPARK-38204][SS] Use StatefulOpClusteredDistribution for stateful operators with respecting backward compatibility

xuanyuanking commented on a change in pull request #35673:
URL: https://github.com/apache/spark/pull/35673#discussion_r825682652



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala
##########
@@ -153,7 +179,8 @@ object AggUtils {
     // If we have session window expression in aggregation, we add UpdatingSessionsExec to
     // calculate sessions for input rows and update rows' session column, so that further
     // aggregations can aggregate input rows for the same session.
-    val maySessionChild = mayAppendUpdatingSessionExec(groupingExpressions, child)
+    val maySessionChild = mayAppendUpdatingSessionExec(groupingExpressions, child,

Review comment:
       ditto

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala
##########
@@ -447,8 +475,10 @@ object AggUtils {
       val aggregateExpressions = functionsWithoutDistinct.map(_.copy(mode = PartialMerge))
       val aggregateAttributes = aggregateExpressions.map(_.resultAttribute)
       MergingSessionsExec(
-        requiredChildDistributionExpressions = None,
-        requiredChildDistributionOption = Some(restored.requiredChildDistribution),
+        requiredChildDistributionExpressions = Some(groupingWithoutSessionAttributes),
+        isStreaming = true,
+        // This will be replaced with actual value in state rule.

Review comment:
       let's link the state rule class name here?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala
##########
@@ -491,10 +521,15 @@ object AggUtils {
 
   private def mayAppendUpdatingSessionExec(
       groupingExpressions: Seq[NamedExpression],
-      maybeChildPlan: SparkPlan): SparkPlan = {
+      maybeChildPlan: SparkPlan,
+      isStreaming: Boolean): SparkPlan = {
     groupingExpressions.find(_.metadata.contains(SessionWindow.marker)) match {
       case Some(sessionExpression) =>
         UpdatingSessionsExec(
+          isStreaming = isStreaming,
+          // numShufflePartitions will be set to None, and replaced to the actual value in the
+          // state rule if the query is streaming.

Review comment:
       ditto

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulOperatorPartitioning.scala
##########
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.streaming
+
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Distribution, StatefulOpClusteredDistribution}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.SQLConf.STATEFUL_OPERATOR_USE_STRICT_DISTRIBUTION
+
+/**
+ * This object is to provide clustered distribution for stateful operator with ensuring backward
+ * compatibility. Please read through the NOTE on the classdoc of
+ * [[StatefulOpClusteredDistribution]] before making any changes. Please refer SPARK-38204
+ * for details.
+ *
+ * Do not use methods in this object for stateful operators which already uses
+ * StatefulOpClusteredDistribution as its required child distribution.

Review comment:
       nit: [[StatefulOpClusteredDistribution]]

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -1773,6 +1773,23 @@ object SQLConf {
       .booleanConf
       .createWithDefault(true)
 
+  val STATEFUL_OPERATOR_USE_STRICT_DISTRIBUTION =
+    buildConf("spark.sql.streaming.statefulOperator.useStrictDistribution")
+      .internal()
+      .doc("When true, the stateful operator for streaming query will use " +
+        "StatefulOpClusteredDistribution which guarantees stable state partitioning as long as " +
+        "the operator provides consistent grouping keys across the lifetime of query. " +
+        "When false, the stateful operator for streaming query will use ClusteredDistribution " +
+        "which is not sufficient to guarantee stable state partitioning despite the operator " +
+        "provides consistent grouping keys across the lifetime of query. " +
+        "This config will be set to true for new streaming queries to guarantee stable state " +
+        "partitioning, and set to false for existing streaming queries to not break queries " +
+        "which are restored from existing checkpoints. Please refer SPARK-38204 for details. " +
+        "The purpose of this config is only compatibility; DO NOT MANUALLY CHANGE THIS!!!")

Review comment:
       let's move the description of `This config is for compatility only; DO NOT MANUALLY CHANGE THIS!!!` as the first sentence in the doc.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala
##########
@@ -121,7 +147,7 @@ object AggUtils {
     // If we have session window expression in aggregation, we add MergingSessionExec to
     // merge sessions with calculating aggregation values.
     val interExec: SparkPlan = mayAppendMergingSessionExec(groupingExpressions,
-      aggregateExpressions, partialAggregate)
+      aggregateExpressions, partialAggregate, isStreaming = false)

Review comment:
       nit: how about using default value in `mayAppendMergingSessionExec` and let here unchange?




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