-
Notifications
You must be signed in to change notification settings - Fork 28.2k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-18124] Observed delay based Event Time Watermarks #15702
Changes from 11 commits
e6e3bbe
9232072
5b92132
14a728e
311e7c0
2685771
379255d
7a9b6dd
554119a
1d4784f
de601bb
87d8618
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,51 @@ | ||
/* | ||
* 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.plans.logical | ||
|
||
import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} | ||
import org.apache.spark.sql.types.MetadataBuilder | ||
import org.apache.spark.unsafe.types.CalendarInterval | ||
|
||
object EventTimeWatermark { | ||
/** The [[org.apache.spark.sql.types.Metadata]] key used to hold the eventTime watermark delay. */ | ||
val delayKey = "spark.watermarkDelayMs" | ||
} | ||
|
||
/** | ||
* Used to mark a user specified column as holding the event time for a row. | ||
*/ | ||
case class EventTimeWatermark( | ||
eventTime: Attribute, | ||
delay: CalendarInterval, | ||
child: LogicalPlan) extends LogicalPlan { | ||
|
||
// Update the metadata on the eventTime column to include the desired delay. | ||
override val output: Seq[Attribute] = child.output.map { a => | ||
if (a semanticEquals eventTime) { | ||
val updatedMetadata = new MetadataBuilder() | ||
.withMetadata(a.metadata) | ||
.putLong(EventTimeWatermark.delayKey, delay.milliseconds) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm a bit confused. Normally Spark SQL uses microsecond precision for TimestampType. When it converts it to LongType, it uses second precision. Here we're using milliseconds. Wouldn't that be super confusing to reason about? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I switched it to using There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Updating the key to include |
||
.build() | ||
a.withMetadata(updatedMetadata) | ||
} else { | ||
a | ||
} | ||
} | ||
|
||
override val children: Seq[LogicalPlan] = child :: Nil | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -50,6 +50,7 @@ import org.apache.spark.sql.execution.python.EvaluatePython | |
import org.apache.spark.sql.streaming.DataStreamWriter | ||
import org.apache.spark.sql.types._ | ||
import org.apache.spark.storage.StorageLevel | ||
import org.apache.spark.unsafe.types.CalendarInterval | ||
import org.apache.spark.util.Utils | ||
|
||
private[sql] object Dataset { | ||
|
@@ -476,7 +477,7 @@ class Dataset[T] private[sql]( | |
* `collect()`, will throw an [[AnalysisException]] when there is a streaming | ||
* source present. | ||
* | ||
* @group basic | ||
* @group streaming | ||
* @since 2.0.0 | ||
*/ | ||
@Experimental | ||
|
@@ -496,8 +497,6 @@ class Dataset[T] private[sql]( | |
/** | ||
* Returns a checkpointed version of this Dataset. | ||
* | ||
* @param eager When true, materializes the underlying checkpointed RDD eagerly. | ||
* | ||
* @group basic | ||
* @since 2.1.0 | ||
*/ | ||
|
@@ -535,6 +534,41 @@ class Dataset[T] private[sql]( | |
)(sparkSession)).as[T] | ||
} | ||
|
||
/** | ||
* :: Experimental :: | ||
* Defines an event time watermark for this [[Dataset]]. A watermark tracks a point in time | ||
* before which we assume no more late data is going to arrive. | ||
* | ||
* Spark will use this watermark for several purposes: | ||
* - To know when a given time window aggregation can be finalized and thus can be emitted when | ||
* using output modes that do not allow updates. | ||
* - To minimize the amount of state that we need to keep for on-going aggregations. | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. For append, this sounds like the intention is emit only once watermark has passed, and drop state. |
||
* | ||
* The current watermark is computed by looking at the `MAX(eventTime)` seen across | ||
* all of the partitions in the query minus a user specified `delayThreshold`. Due to the cost | ||
* of coordinating this value across partitions, the actual watermark used is only guaranteed | ||
* to be at least `delayThreshold` behind the actual event time. In some cases we may still | ||
* process records that arrive more than `delayThreshold` late. | ||
* | ||
* @param eventTime the name of the column that contains the event time of the row. | ||
* @param delayThreshold the minimum delay to wait to data to arrive late, relative to the latest | ||
* record that has been processed in the form of an interval | ||
* (e.g. "1 minute" or "5 hours"). | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Should this make it clear what the minimum useful granularity is (ms)? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. That seems like more of an implementation detail, rather than a contract of the API. The real contract is stated above as |
||
* | ||
* @group streaming | ||
* @since 2.1.0 | ||
*/ | ||
@Experimental | ||
@InterfaceStability.Evolving | ||
// We only accept an existing column name, not a derived column here as a watermark that is | ||
// defined on a derived column cannot referenced elsewhere in the plan. | ||
def withWatermark(eventTime: String, delayThreshold: String): Dataset[T] = withTypedPlan { | ||
val parsedDelay = | ||
Option(CalendarInterval.fromString("interval " + delayThreshold)) | ||
.getOrElse(throw new AnalysisException(s"Unable to parse time delay '$delayThreshold'")) | ||
EventTimeWatermark(UnresolvedAttribute(eventTime), parsedDelay, logicalPlan) | ||
} | ||
|
||
/** | ||
* Displays the Dataset in a tabular form. Strings more than 20 characters will be truncated, | ||
* and all cells will be aligned right. For example: | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Which high level case is caught by this condition?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The result of a window operation.