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 2021/03/18 02:55:00 UTC

[GitHub] [spark] xuanyuanking commented on a change in pull request #31570: [SPARK-10816][SS] SessionWindow support for Structure Streaming

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreHandler.scala
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.state
+
+import org.apache.hadoop.conf.Configuration
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.execution.streaming.StatefulOperatorStateInfo
+import org.apache.spark.sql.types.StructType
+
+trait StateStoreType
+
+/** Helper trait for invoking common functionalities of a state store. */
+abstract class StateStoreHandler extends Logging {

Review comment:
       It would be great if we can describe a little more how we reuse the original streaming join `StateStoreHandler`. Seems it's an important implementation detail.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala
##########
@@ -262,6 +262,17 @@ object AggUtils {
    *  - PartialMerge (now there is at most 1 tuple per group)
    *  - StateStoreSave (saves the tuple for the next batch)
    *  - Complete (output the current result of the aggregation)
+   *
+   * Plans a streaming aggregation with Session Window using the following progression:
+   *  - (Shuffle + Session Window Assignment, see `SessionWindowExec`)
+   *  - Partial Aggregation (now there is at most 1 tuple per group)
+   *  - SessionStateStoreRestore (now there is 1 tuple from this batch + optionally one from

Review comment:
       Yes. If we see a bottleneck in the future. We can add the pre-shuffle PartialMerge back.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/SessionWindowMergeExec.scala
##########
@@ -0,0 +1,180 @@
+/*
+ * 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
+
+import scala.collection.mutable
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Distribution, Partitioning}
+
+/**
+ * The physical plan for streaming query, merge session window after restore from state store.
+ * Note: the end time of window that restore from statestore has already contain session windowGap
+ *
+ * @param windowExpressions
+ * @param sessionSpec
+ * @param child
+ */
+case class SessionWindowMergeExec(
+    windowExpressions: NamedExpression,
+    sessionSpec: Seq[Expression],
+    child: SparkPlan)
+  extends UnaryExecNode {
+
+  override def requiredChildDistribution: Seq[Distribution] = {
+    ClusteredDistribution(sessionSpec) :: Nil
+  }
+
+  // Data should be sorted, so we can merge session window directly.
+  // TODO: use this requirement for simplicity, not necessary to sort the whole dataset,
+  // try better way later.

Review comment:
       Seems we can already get rid of this ordering. Due to the session window in state store is in order.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/SessionWindowExec.scala
##########
@@ -0,0 +1,203 @@
+/*
+ * 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
+
+import scala.collection.mutable
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Distribution, Partitioning}
+
+/**
+ * Used for calculating the session window start and end for each row, so this plan requires
+ * child distributed by sessionSpec and sorted by time column in each part. The value for
+ * window start is time value of the first row in this window, the value for window end is
+ * time value of the last row plus the windowGap.
+ *
+ * @param windowExpressions session window expression for the exec node.
+ * @param sessionSpec the partition key of this session window, it is the rest column of
+ *                    groupingExpr in parent aggregate node.
+ * @param windowGap window gap in micro second.
+ * @param child child plan for this node.
+ */
+case class SessionWindowExec(
+    windowExpressions: NamedExpression,
+    timeColumn: Expression,
+    sessionSpec: Seq[Expression],
+    windowGap: Long,
+    child: SparkPlan)
+  extends UnaryExecNode {
+
+  override def requiredChildDistribution: Seq[Distribution] = {
+    ClusteredDistribution(sessionSpec) :: Nil
+  }
+
+  override def requiredChildOrdering: Seq[Seq[SortOrder]] =
+    Seq(sessionSpec.map(SortOrder(_, Ascending)) :+ SortOrder(timeColumn, Ascending))
+
+  override def producedAttributes: AttributeSet = AttributeSet(windowExpressions.toAttribute)
+
+  override def output: Seq[Attribute] = child.output ++ Seq(windowExpressions.toAttribute)
+
+  override def outputPartitioning: Partitioning = child.outputPartitioning
+
+  override def outputOrdering: Seq[SortOrder] = child.outputOrdering
+
+  /**
+   * Produces the result of the query as an `RDD[InternalRow]`
+   *
+   * Overridden by concrete implementations of SparkPlan.
+   */
+  override protected def doExecute(): RDD[InternalRow] = {

Review comment:
       This is another difference with the @HeartSaVioR's. We use the pattern of SQL window function, while Jungtaek's one is a sub-class for aggregation iterator.
   Pros:
   Less code change. Don't mess up with aggregation logic.
   Cons:
   We keep the original row for a while. Might involve more memory cost.




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

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