HeartSaVioR commented on a change in pull request #31570: URL: https://github.com/apache/spark/pull/31570#discussion_r596631016
########## 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: And if I understand correctly, this needs to copy all input rows because of the buffering. Now I could reload the context of my patch a bit; MergingSessionsIterator only copies the rows which are the first row of session, as it only needs to retain the last session to compare with current input row. That was the reason I chose such complexity. Performance wise, and also there's concern on JIRA issue about memory usage on the flight. ---------------------------------------------------------------- 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]
