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/10/19 01:18:18 UTC

[GitHub] [spark] HeartSaVioR commented on a diff in pull request #38288: [SPARK-40821][SQL][CORE][PYTHON][SS] Introduce window_time function to extract event time from the window column

HeartSaVioR commented on code in PR #38288:
URL: https://github.com/apache/spark/pull/38288#discussion_r998806180


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -4201,6 +4219,73 @@ object SessionWindowing extends Rule[LogicalPlan] {
   }
 }
 
+/**
+ * Resolves the window_time expression which extracts the correct window time from the
+ * window column generated as the output of the window aggregating operators. The
+ * window column is of type struct { start: TimestampType, end: TimestampType }.
+ * The correct window time for further aggregations is window.end - 1.
+ * */
+object ResolveWindowTime extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
+    case p: LogicalPlan if p.children.size == 1 =>
+      val child = p.children.head
+      val windowTimeExpressions =
+        p.expressions.flatMap(_.collect { case w: WindowTime => w }).toSet
+
+      if (windowTimeExpressions.size == 1 &&
+        windowTimeExpressions.head.windowColumn.resolved &&
+        windowTimeExpressions.head.checkInputDataTypes().isSuccess) {
+
+        val windowTime = windowTimeExpressions.head
+
+        val metadata = windowTime.windowColumn match {
+          case a: Attribute => a.metadata
+          case _ => Metadata.empty
+        }
+
+        if (!metadata.contains(TimeWindow.marker) &&
+          !metadata.contains(SessionWindow.marker)) {
+          // FIXME: error framework?
+          throw new AnalysisException("The input is not a correct window column!")

Review Comment:
   Let's put some information of column/expression in the error message. It's not friendly to users to react based on the error message.
   
   Btw I don't have a good suggestion for error framework - we can ping some folks to get some guidance cc. @MaxGekk 



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -4201,6 +4219,73 @@ object SessionWindowing extends Rule[LogicalPlan] {
   }
 }
 
+/**
+ * Resolves the window_time expression which extracts the correct window time from the
+ * window column generated as the output of the window aggregating operators. The
+ * window column is of type struct { start: TimestampType, end: TimestampType }.
+ * The correct window time for further aggregations is window.end - 1.

Review Comment:
   nit: Maybe "further aggregations" is too narrow explanation compared to what it can enable, if we consider deduplication as not an aggregation. Maybe "the correct event time of window", or "the correct representative time of window"?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WindowTime.scala:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.types._
+
+// scalastyle:off line.size.limit line.contains.tab
+@ExpressionDescription(
+  usage = """
+    _FUNC_(window_column) - Extract the time value from time/session window column which can be used for event time value of window.
+      The extracted time is (window.end - 1) which reflects the fact that the the aggregating
+      windows have exclusive upper bound - [start, end)
+      See <a href="https://spark.apache.org/docs/latest/structured-streaming-programming-guide.html#window-operations-on-event-time">'Window Operations on Event Time'</a> in Structured Streaming guide doc for detailed explanation and examples.
+  """,
+  arguments = """
+    Arguments:
+      * window_column - The column representing time/session window.
+  """,
+  examples = """
+    Examples:
+      > SELECT a, window.start as start, window.end as end, _FUNC_(window), cnt FROM (SELECT a, window, count(*) as cnt FROM VALUES ('A1', '2021-01-01 00:00:00'), ('A1', '2021-01-01 00:04:30'), ('A1', '2021-01-01 00:06:00'), ('A2', '2021-01-01 00:01:00') AS tab(a, b) GROUP by a, window(b, '5 minutes') ORDER BY a, window.start);
+        A1	2021-01-01 00:00:00	2021-01-01 00:05:00	2021-01-01 00:04:59.999999	2
+        A1	2021-01-01 00:05:00	2021-01-01 00:10:00	2021-01-01 00:09:59.999999	1
+        A2	2021-01-01 00:00:00	2021-01-01 00:05:00	2021-01-01 00:04:59.999999	1
+  """,
+  group = "datetime_funcs",
+  since = "3.3.0")
+// scalastyle:on line.size.limit line.contains.tab
+case class WindowTime(windowColumn: Expression)
+  extends UnaryExpression
+    with ImplicitCastInputTypes
+    with Unevaluable
+    with NonSQLExpression {
+
+  // FIXME: Should we check that windowColumn is a correct one - generated by

Review Comment:
   I think we are fine as long as the rule checks the marker.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -4201,6 +4219,73 @@ object SessionWindowing extends Rule[LogicalPlan] {
   }
 }
 
+/**
+ * Resolves the window_time expression which extracts the correct window time from the
+ * window column generated as the output of the window aggregating operators. The
+ * window column is of type struct { start: TimestampType, end: TimestampType }.
+ * The correct window time for further aggregations is window.end - 1.
+ * */
+object ResolveWindowTime extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
+    case p: LogicalPlan if p.children.size == 1 =>
+      val child = p.children.head
+      val windowTimeExpressions =
+        p.expressions.flatMap(_.collect { case w: WindowTime => w }).toSet
+
+      if (windowTimeExpressions.size == 1 &&
+        windowTimeExpressions.head.windowColumn.resolved &&
+        windowTimeExpressions.head.checkInputDataTypes().isSuccess) {
+
+        val windowTime = windowTimeExpressions.head
+
+        val metadata = windowTime.windowColumn match {
+          case a: Attribute => a.metadata
+          case _ => Metadata.empty
+        }
+
+        if (!metadata.contains(TimeWindow.marker) &&
+          !metadata.contains(SessionWindow.marker)) {
+          // FIXME: error framework?
+          throw new AnalysisException("The input is not a correct window column!")
+        }
+
+        val newMetadata = new MetadataBuilder()
+          .withMetadata(metadata)
+          .remove(TimeWindow.marker)
+          .remove(SessionWindow.marker)
+          .build()
+
+        val attr = AttributeReference(
+          "window_time", windowTime.dataType, metadata = newMetadata)()
+
+        // NOTE: "window.end" is "exclusive" upper bound of window, so if we use this value as
+        // it is, it is going to be bound to the different window even if we apply the same window
+        // spec. Decrease 1 microsecond from window.end to let the window_time be bound to the
+        // correct window range.
+        val subtractExpr =
+        PreciseTimestampConversion(
+          Subtract(PreciseTimestampConversion(
+            // FIXME: better handling of window.end
+            GetStructField(windowTime.windowColumn, 1),
+            windowTime.dataType, LongType), Literal(1L)),
+          LongType,
+          windowTime.dataType)
+
+        // FIXME: Can there already be a window_time column? Will this lead to conflict?

Review Comment:
   This is valid concern, although we do the same for window() as well.
   
   E.g. Spark tends to name the column for function as the function name with entire parameters. (Unless we alias the result of function explicitly, it should be something like `window(event_time, '5 seconds')`, but we just give the column name of the result as `window`.)



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala:
##########
@@ -4201,6 +4219,73 @@ object SessionWindowing extends Rule[LogicalPlan] {
   }
 }
 
+/**
+ * Resolves the window_time expression which extracts the correct window time from the
+ * window column generated as the output of the window aggregating operators. The
+ * window column is of type struct { start: TimestampType, end: TimestampType }.
+ * The correct window time for further aggregations is window.end - 1.
+ * */
+object ResolveWindowTime extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
+    case p: LogicalPlan if p.children.size == 1 =>
+      val child = p.children.head
+      val windowTimeExpressions =
+        p.expressions.flatMap(_.collect { case w: WindowTime => w }).toSet
+
+      if (windowTimeExpressions.size == 1 &&
+        windowTimeExpressions.head.windowColumn.resolved &&
+        windowTimeExpressions.head.checkInputDataTypes().isSuccess) {
+
+        val windowTime = windowTimeExpressions.head
+
+        val metadata = windowTime.windowColumn match {
+          case a: Attribute => a.metadata
+          case _ => Metadata.empty
+        }
+
+        if (!metadata.contains(TimeWindow.marker) &&
+          !metadata.contains(SessionWindow.marker)) {
+          // FIXME: error framework?
+          throw new AnalysisException("The input is not a correct window column!")
+        }
+
+        val newMetadata = new MetadataBuilder()
+          .withMetadata(metadata)
+          .remove(TimeWindow.marker)
+          .remove(SessionWindow.marker)
+          .build()
+
+        val attr = AttributeReference(
+          "window_time", windowTime.dataType, metadata = newMetadata)()
+
+        // NOTE: "window.end" is "exclusive" upper bound of window, so if we use this value as
+        // it is, it is going to be bound to the different window even if we apply the same window
+        // spec. Decrease 1 microsecond from window.end to let the window_time be bound to the
+        // correct window range.
+        val subtractExpr =
+        PreciseTimestampConversion(
+          Subtract(PreciseTimestampConversion(
+            // FIXME: better handling of window.end

Review Comment:
   I guess this is OK as long as we don't make a change in window struct. (And technically we can't, due to the backward compatibility.)



##########
sql/core/src/main/scala/org/apache/spark/sql/functions.scala:
##########
@@ -3777,6 +3777,23 @@ object functions {
     window(timeColumn, windowDuration, windowDuration, "0 second")
   }
 
+  /**
+   * Extracts the event time from the window column of a record produced by window aggregation

Review Comment:
   I may need to be strict here to avoid further confusion based on comment. We do not have window aggregation operator. Technically saying, window function is effectively a TVF like explode(), and we apply the "normal" aggregation against the grouped Dataset which group contains window.
   
   That said, `produced by window aggregation` seems to be unnecessary.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WindowTime.scala:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.types._
+
+// scalastyle:off line.size.limit line.contains.tab
+@ExpressionDescription(
+  usage = """
+    _FUNC_(window_column) - Extract the time value from time/session window column which can be used for event time value of window.
+      The extracted time is (window.end - 1) which reflects the fact that the the aggregating
+      windows have exclusive upper bound - [start, end)
+      See <a href="https://spark.apache.org/docs/latest/structured-streaming-programming-guide.html#window-operations-on-event-time">'Window Operations on Event Time'</a> in Structured Streaming guide doc for detailed explanation and examples.
+  """,
+  arguments = """
+    Arguments:
+      * window_column - The column representing time/session window.
+  """,
+  examples = """
+    Examples:
+      > SELECT a, window.start as start, window.end as end, _FUNC_(window), cnt FROM (SELECT a, window, count(*) as cnt FROM VALUES ('A1', '2021-01-01 00:00:00'), ('A1', '2021-01-01 00:04:30'), ('A1', '2021-01-01 00:06:00'), ('A2', '2021-01-01 00:01:00') AS tab(a, b) GROUP by a, window(b, '5 minutes') ORDER BY a, window.start);
+        A1	2021-01-01 00:00:00	2021-01-01 00:05:00	2021-01-01 00:04:59.999999	2
+        A1	2021-01-01 00:05:00	2021-01-01 00:10:00	2021-01-01 00:09:59.999999	1
+        A2	2021-01-01 00:00:00	2021-01-01 00:05:00	2021-01-01 00:04:59.999999	1
+  """,
+  group = "datetime_funcs",
+  since = "3.3.0")
+// scalastyle:on line.size.limit line.contains.tab
+case class WindowTime(windowColumn: Expression)
+  extends UnaryExpression
+    with ImplicitCastInputTypes
+    with Unevaluable
+    with NonSQLExpression {
+
+  // FIXME: Should we check that windowColumn is a correct one - generated by
+  // a window or session_window aggregation? Do we need to check for the corresponding marker?
+
+  override def child: Expression = windowColumn
+  override def inputTypes: Seq[AbstractDataType] = Seq(StructType)
+
+  // FIXME: pick either "start" or "end" explicitly here

Review Comment:
   Let's leave a comment instead; window struct has start and end column which has identical data type, hence choosing the first will work.



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