Github user marmbrus commented on a diff in the pull request: https://github.com/apache/spark/pull/15702#discussion_r86259192 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala --- @@ -0,0 +1,93 @@ +/* + * 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 org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeProjection} +import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.types.MetadataBuilder +import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.util.AccumulatorV2 + +class MaxLong(protected var currentValue: Long = 0) + extends AccumulatorV2[Long, Long] + with Serializable { + + override def isZero: Boolean = value == 0 + override def value: Long = currentValue + override def copy(): AccumulatorV2[Long, Long] = new MaxLong(currentValue) + + override def reset(): Unit = { + currentValue = 0 + } + + override def add(v: Long): Unit = { + if (value < v) { currentValue = v } + } + + override def merge(other: AccumulatorV2[Long, Long]): Unit = { + if (currentValue < other.value) { + currentValue = other.value + } + } +} + +/** + * Used to mark a column as the containing the event time for a given record. In addition to + * adding appropriate metadata to this column, this operator also tracks the maximum observed event + * time. Based on the maximum observed time and a user specified delay, we can calculate the + * `watermark` after which we assume we will no longer see late records for a particular time + * period. + */ +case class EventTimeWatermarkExec( + eventTime: Attribute, + delay: CalendarInterval, + child: SparkPlan) extends SparkPlan { + + // TODO: Use Spark SQL Metrics? + val maxEventTime = new MaxLong + sparkContext.register(maxEventTime) + + override protected def doExecute(): RDD[InternalRow] = { + child.execute().mapPartitions { iter => + val getEventTime = UnsafeProjection.create(eventTime :: Nil, child.output) + iter.map { row => + maxEventTime.add(getEventTime(row).getLong(0)) --- End diff -- Added to checkAnalysis.
--- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org