HeartSaVioR commented on code in PR #52428: URL: https://github.com/apache/spark/pull/52428#discussion_r2375749544
########## sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkWithWatermarkDefInSelectSuite.scala: ########## @@ -0,0 +1,267 @@ +/* + * 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 + +import java.{util => ju} +import java.text.SimpleDateFormat + +import org.scalatest.BeforeAndAfter + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.UTC +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream +import org.apache.spark.sql.functions.{col, timestamp_seconds} +import org.apache.spark.sql.streaming.StateStoreMetricsTest + +class EventTimeWatermarkWithWatermarkDefInSelectSuite + extends StateStoreMetricsTest + with BeforeAndAfter + with Logging { + + import testImplicits._ + + after { + sqlContext.streams.active.foreach(_.stop()) + } + + test("event time and watermark metrics with watermark in select DML - case 1") { + // All event time metrics where watermarking is set + val inputData = MemoryStream[Int] + val df = inputData.toDF() + .withColumn("eventTime", timestamp_seconds(col("value"))) + df.createOrReplaceTempView("stream_src") + val aggWithWatermark = spark.sql( + """ + |SELECT + | CAST(window.start AS LONG), CAST(count(*) AS LONG) AS count + |FROM + | stream_src WATERMARK eventTime DELAY OF INTERVAL 10 seconds + |GROUP BY window(eventTime, '5 seconds') + |""".stripMargin) + + testWindowedAggregation(inputData, aggWithWatermark) + } + + test("event time and watermark metrics with watermark in select DML - case 2") { + // All event time metrics where watermarking is set + val inputData = MemoryStream[Int] + val df = inputData.toDF() + df.createOrReplaceTempView("stream_src") + val aggWithWatermark = spark.sql( + """ + |SELECT + | CAST(window.start AS LONG), CAST(count(*) AS LONG) AS count + |FROM + | stream_src + | WATERMARK timestamp_seconds(value) AS eventTime DELAY OF INTERVAL 10 seconds + |GROUP BY window(eventTime, '5 seconds') + |""".stripMargin) + + testWindowedAggregation(inputData, aggWithWatermark) + } + + private def testWindowedAggregation( + inputData: MemoryStream[Int], + dataFrame: DataFrame): Unit = { + testStream(dataFrame)( + AddData(inputData, 15), + CheckAnswer(), + assertEventStats(min = 15, max = 15, avg = 15, wtrmark = 0), + AddData(inputData, 10, 12, 14), + CheckAnswer(), + assertEventStats(min = 10, max = 14, avg = 12, wtrmark = 5), + AddData(inputData, 25), + CheckAnswer((10, 3)), + assertEventStats(min = 25, max = 25, avg = 25, wtrmark = 5) + ) + } + + test("stream-stream join with watermark in select DML - case 1") { + val leftInput = MemoryStream[(Int, Int)] + val rightInput = MemoryStream[(Int, Int)] + + val df1 = leftInput.toDF().toDF("leftKey", "time") + .select($"leftKey", timestamp_seconds($"time") as "leftTime", + ($"leftKey" * 2) as "leftValue") + val df2 = rightInput.toDF().toDF("rightKey", "time") + .select($"rightKey", timestamp_seconds($"time") as "rightTime", + ($"rightKey" * 3) as "rightValue") + + df1.createOrReplaceTempView("stream_left") + df2.createOrReplaceTempView("stream_right") + + val joined = spark.sql( + """ + |SELECT + | leftKey, rightKey, CAST(leftTime AS INTEGER), CAST(rightTime AS INTEGER) + |FROM + | stream_left WATERMARK leftTime DELAY OF INTERVAL 0 second + |FULL OUTER JOIN + | stream_right WATERMARK rightTime DELAY OF INTERVAL 0 second Review Comment: You can set watermark "before" the first stateful operator has appeared. But conceptually, it is advised to define watermark closer to stream scan. The concept of watermark is to mark that the engine has seen (processed) all the data before the timestamp ts, which is closely related to the input data from stream scan. -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
