HeartSaVioR commented on a change in pull request #31570:
URL: https://github.com/apache/spark/pull/31570#discussion_r596514052
##########
File path:
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -3964,6 +3965,80 @@ object TimeWindowing extends Rule[LogicalPlan] {
}
}
+/**
+ * Replace the [[SessionWindowExpression]] in Aggregate node, this rule will
add [[SessionWindow]]
+ * as the current Aggregate's new child. It will throw [[AnalysisException]]
while
+ * [[SessionWindowExpression]] is the only column in group by.
+ */
+object ResolveSessionWindow extends Rule[LogicalPlan] {
+
+ private def hasWindowFunction(groupList: Seq[Expression]): Boolean =
+ groupList.exists(hasWindowFunction)
+
+ private def hasWindowFunction(expr: Expression): Boolean = {
+ expr.find {
+ case window: SessionWindowExpression => true
+ case _ => false
+ }.isDefined
+ }
+
+ private final val WINDOW_COL_NAME = "session_window"
+
+ def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperators {
+ case p @ Aggregate(groupingExpr, aggregateExpr, _) if
hasWindowFunction(groupingExpr) =>
Review comment:
`hasWindowFunction` and `windowExpressions` should count time window as
well. I guess it'll be simpler to resolve when we consolidate, but it would
depend on how more complicated it will become.
##########
File path:
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -3964,6 +3965,80 @@ object TimeWindowing extends Rule[LogicalPlan] {
}
}
+/**
+ * Replace the [[SessionWindowExpression]] in Aggregate node, this rule will
add [[SessionWindow]]
+ * as the current Aggregate's new child. It will throw [[AnalysisException]]
while
+ * [[SessionWindowExpression]] is the only column in group by.
+ */
+object ResolveSessionWindow extends Rule[LogicalPlan] {
Review comment:
I know the logic in TimeWindowing is a bit long and you may not want to
add additional complexity there, but I feel this still has to be consolidated
with TimeWindowing (that said TimeWindowing needs to refactor a bit after
adding logic on session window). Because there're lots of similarities and the
limitation is applied altogether, like only allowing a time window should
include session window, and vice versa.
##########
File path:
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala
##########
@@ -94,7 +94,47 @@ case class TimeWindow(
}
}
-object TimeWindow {
+case class SessionWindowExpression(
Review comment:
I feel it's OK to just say `SessionWindow` like we do for `TimeWindow`,
but if we don't feel it's clear, we need to rename `TimeWindow` to
`TimeWindowExpression` as well for consistency.
##########
File path:
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -3964,6 +3965,80 @@ object TimeWindowing extends Rule[LogicalPlan] {
}
}
+/**
+ * Replace the [[SessionWindowExpression]] in Aggregate node, this rule will
add [[SessionWindow]]
+ * as the current Aggregate's new child. It will throw [[AnalysisException]]
while
+ * [[SessionWindowExpression]] is the only column in group by.
+ */
+object ResolveSessionWindow extends Rule[LogicalPlan] {
+
+ private def hasWindowFunction(groupList: Seq[Expression]): Boolean =
+ groupList.exists(hasWindowFunction)
+
+ private def hasWindowFunction(expr: Expression): Boolean = {
+ expr.find {
+ case window: SessionWindowExpression => true
+ case _ => false
+ }.isDefined
+ }
+
+ private final val WINDOW_COL_NAME = "session_window"
Review comment:
I see how to deal with meta fields (start, end) is slightly different
from time window. Would the difference be bugging when we consolidate twos into
one? I have no specific preference, but would like to be consistent with both
so that we don't deal with differences in maintenance.
That said, let's change both (OK to do it on follow-up PR), or follow the
current approach.
##########
File path:
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -3964,6 +3965,80 @@ object TimeWindowing extends Rule[LogicalPlan] {
}
}
+/**
+ * Replace the [[SessionWindowExpression]] in Aggregate node, this rule will
add [[SessionWindow]]
+ * as the current Aggregate's new child. It will throw [[AnalysisException]]
while
+ * [[SessionWindowExpression]] is the only column in group by.
+ */
+object ResolveSessionWindow extends Rule[LogicalPlan] {
+
+ private def hasWindowFunction(groupList: Seq[Expression]): Boolean =
+ groupList.exists(hasWindowFunction)
+
+ private def hasWindowFunction(expr: Expression): Boolean = {
+ expr.find {
+ case window: SessionWindowExpression => true
+ case _ => false
+ }.isDefined
+ }
+
+ private final val WINDOW_COL_NAME = "session_window"
+
+ def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperators {
+ case p @ Aggregate(groupingExpr, aggregateExpr, _) if
hasWindowFunction(groupingExpr) =>
+ val child = p.child
+ val windowExpressions =
+ p.expressions.flatMap(_.collect { case t: SessionWindowExpression => t
}).toSet
+
+ val numWindowExpr = windowExpressions.size
+ // Only support a single session window expression for now
+ if (numWindowExpr == 1 &&
+ windowExpressions.head.timeColumn.resolved &&
+ windowExpressions.head.checkInputDataTypes().isSuccess) {
+
+ val window = windowExpressions.head
+
+ val metadata = window.timeColumn match {
+ case a: Attribute => a.metadata
+ case _ => Metadata.empty
+ }
+
+ val windowAttr = AttributeReference(
+ WINDOW_COL_NAME, window.dataType, metadata = metadata)()
+
+ // check partitionExpression in groupingExpr
+ val partitionExpression = groupingExpr.filterNot(hasWindowFunction)
+ if (partitionExpression.isEmpty) {
+ p.failAnalysis("Cannot use session_window as the only group by
column.")
Review comment:
I feel this is less clearer, probably better to mention another key
column(s) are required? Like "Cannot use session_window without additional key
column(s)", or if we assume end users know about the concept of "global
aggregation", "Cannot apply session_window on global aggregation".
##########
File path:
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
##########
@@ -673,6 +673,19 @@ case class Window(
def windowOutputSet: AttributeSet =
AttributeSet(windowExpressions.map(_.toAttribute))
}
+case class SessionWindow(
Review comment:
Ah OK this is conflicting with SessionWindowExpression when renaming. OK
to leave as it is.
##########
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
Review comment:
add explanation on each param, or simply remove
##########
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
Review comment:
This made me stop to think, and looks like this is guaranteed per
micro-batch. Probably ideal to leave a brief explanation how it is guaranteed.
##########
File path:
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
##########
@@ -348,7 +348,6 @@ abstract class SparkStrategies extends
QueryPlanner[SparkPlan] {
rewrittenResultExpressions,
stateVersion,
planLater(child))
-
Review comment:
nit: unnecessary change
##########
File path:
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -3964,6 +3965,80 @@ object TimeWindowing extends Rule[LogicalPlan] {
}
}
+/**
+ * Replace the [[SessionWindowExpression]] in Aggregate node, this rule will
add [[SessionWindow]]
+ * as the current Aggregate's new child. It will throw [[AnalysisException]]
while
+ * [[SessionWindowExpression]] is the only column in group by.
+ */
+object ResolveSessionWindow extends Rule[LogicalPlan] {
Review comment:
Or, at least counting windows should consider both.
##########
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
+
+ /**
Review comment:
The comment doesn't seem to be needed.
----------------------------------------------------------------
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]