HeartSaVioR commented on a change in pull request #31570: URL: https://github.com/apache/spark/pull/31570#discussion_r596534327
########## 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: That may not just bring additional memory cost. That may bring spill, which is something we'd like to avoid at all cost. There's a trade-off, complexity vs optimization. I'm OK to move forward to make it work, and evaluate the value of the trade-off. Except state format we could change everything afterwards, so OK with that. ########## File path: sql/core/src/test/scala/org/apache/spark/sql/DataFrameSessionWindowingSuite.scala ########## @@ -0,0 +1,150 @@ +/* + * 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 + +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.sql.execution.SessionWindowExec +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.StringType + +class DataFrameSessionWindowingSuite + extends QueryTest with SharedSparkSession with AdaptiveSparkPlanHelper with BeforeAndAfterEach { + + import testImplicits._ + + private def withTempTable(f: String => Unit): Unit = { + val tableName = "temp" + Seq( + ("2018-08-22 19:39:27", "a", 4), + ("2018-08-22 19:39:34", "a", 1), + ("2018-08-22 19:39:56", "a", 3), + ("2018-08-22 19:39:56", "b", 2) + ).toDF("time", "key", "value").createOrReplaceTempView(tableName) + try { + f(tableName) + } finally { + spark.catalog.dropTempView(tableName) + } + } + + test("session window in SQL with single key as session window key") { + withTempTable { table => + val a = spark.sql( Review comment: dead code? ########## 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: As @xuanyuanking stated, we can say that's a simplification, but it is arguable that PartialMerge is "unnecessary". The sort operation is necessary to aggregate before shuffling so that is a trade-off, but it's known that less amount of shuffle data brings performance benefits. ---------------------------------------------------------------- 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: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
