Skip to content

Commit

Permalink
[SPARK-47960][SS] Allow chaining other stateful operators after trans…
Browse files Browse the repository at this point in the history
…formWithState operator

### What changes were proposed in this pull request?

This PR adds support to define event time column in the output dataset of `TransformWithState` operator. The new event time column will be used to evaluate watermark expressions in downstream operators.

1. Note that the transformWithState operator does not enforce that values generated by user's computation adhere to the watermark semantics. (no output rows are generated which have event time less than watermark).
2. Updated the watermark value passed in TimerInfo as evictionWatermark, rather than lateEventsWatermark.
3. Ensure that event time column can only be defined in output if a watermark has been defined previously.

### Why are the changes needed?

This change is required to support chaining of stateful operators after `transformWithState`. Event time column is required to evaluate watermark expressions in downstream stateful operators.

### Does this PR introduce _any_ user-facing change?

Yes. Adds a new version of transformWithState API which allows redefining the event time column.

### How was this patch tested?

Added unit test cases.

### Was this patch authored or co-authored using generative AI tooling?

No

Closes apache#45376 from sahnib/tws-chaining-stateful-operators.

Authored-by: Bhuwan Sahni <[email protected]>
Signed-off-by: Jungtaek Lim <[email protected]>
  • Loading branch information
sahnib authored and HeartSaVioR committed May 8, 2024
1 parent 3d9d1f3 commit 5e49665
Show file tree
Hide file tree
Showing 16 changed files with 866 additions and 51 deletions.
14 changes: 14 additions & 0 deletions common/utils/src/main/resources/error/error-conditions.json
Original file line number Diff line number Diff line change
Expand Up @@ -125,6 +125,12 @@
],
"sqlState" : "428FR"
},
"CANNOT_ASSIGN_EVENT_TIME_COLUMN_WITHOUT_WATERMARK" : {
"message" : [
"Watermark needs to be defined to reassign event time column. Failed to find watermark definition in the streaming query."
],
"sqlState" : "42611"
},
"CANNOT_CAST_DATATYPE" : {
"message" : [
"Cannot cast <sourceType> to <targetType>."
Expand Down Expand Up @@ -1057,6 +1063,14 @@
},
"sqlState" : "4274K"
},
"EMITTING_ROWS_OLDER_THAN_WATERMARK_NOT_ALLOWED" : {
"message" : [
"Previous node emitted a row with eventTime=<emittedRowEventTime> which is older than current_watermark_value=<currentWatermark>",
"This can lead to correctness issues in the stateful operators downstream in the execution pipeline.",
"Please correct the operator logic to emit rows after current global watermark value."
],
"sqlState" : "42815"
},
"EMPTY_JSON_FIELD_VALUE" : {
"message" : [
"Failed to parse an empty string for data type <dataType>."
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -331,6 +331,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
Seq(
ResolveWithCTE,
ExtractDistributedSequenceID) ++
Seq(ResolveUpdateEventTimeWatermarkColumn) ++
extendedResolutionRules : _*),
Batch("Remove TempResolvedColumn", Once, RemoveTempResolvedColumn),
Batch("Post-Hoc Resolution", Once,
Expand Down Expand Up @@ -4003,6 +4004,8 @@ object EliminateEventTimeWatermark extends Rule[LogicalPlan] {
override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
_.containsPattern(EVENT_TIME_WATERMARK)) {
case EventTimeWatermark(_, _, child) if child.resolved && !child.isStreaming => child
case UpdateEventTimeWatermarkColumn(_, _, child) if child.resolved && !child.isStreaming =>
child
}
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,52 @@
/*
* 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.analysis

import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, LogicalPlan, UpdateEventTimeWatermarkColumn}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.catalyst.trees.TreePattern.UPDATE_EVENT_TIME_WATERMARK_COLUMN
import org.apache.spark.sql.errors.QueryCompilationErrors

/**
* Extracts the watermark delay and adds it to the UpdateEventTimeWatermarkColumn
* logical node (if such a node is present). [[UpdateEventTimeWatermarkColumn]] node updates
* the eventTimeColumn for upstream operators.
*
* If the logical plan contains a [[UpdateEventTimeWatermarkColumn]] node, but no watermark
* has been defined, the query will fail with a compilation error.
*/
object ResolveUpdateEventTimeWatermarkColumn extends Rule[LogicalPlan] {

override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUpWithPruning(
_.containsPattern(UPDATE_EVENT_TIME_WATERMARK_COLUMN), ruleId) {
case u: UpdateEventTimeWatermarkColumn if u.delay.isEmpty && u.childrenResolved =>
val existingWatermarkDelay = u.child.collect {
case EventTimeWatermark(_, delay, _) => delay
}

if (existingWatermarkDelay.isEmpty) {
// input dataset needs to have a event time column, we transfer the
// watermark delay from this column to user specified eventTimeColumnName
// in the output dataset.
throw QueryCompilationErrors.cannotAssignEventTimeColumn()
}

val delay = existingWatermarkDelay.head
u.copy(delay = Some(delay))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,8 @@ package org.apache.spark.sql.catalyst.plans.logical
import java.util.concurrent.TimeUnit

import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.trees.TreePattern.{EVENT_TIME_WATERMARK, TreePattern}
import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark.updateEventTimeColumn
import org.apache.spark.sql.catalyst.trees.TreePattern.{EVENT_TIME_WATERMARK, TreePattern, UPDATE_EVENT_TIME_WATERMARK_COLUMN}
import org.apache.spark.sql.catalyst.util.IntervalUtils
import org.apache.spark.sql.types.MetadataBuilder
import org.apache.spark.unsafe.types.CalendarInterval
Expand All @@ -32,6 +33,36 @@ object EventTimeWatermark {
def getDelayMs(delay: CalendarInterval): Long = {
IntervalUtils.getDuration(delay, TimeUnit.MILLISECONDS)
}

/**
* Adds watermark delay to the metadata for newEventTime in provided attributes.
*
* If any other existing attributes have watermark delay present in their metadata, watermark
* delay will be removed from their metadata.
*/
def updateEventTimeColumn(
attributes: Seq[Attribute],
delayMs: Long,
newEventTime: Attribute): Seq[Attribute] = {
attributes.map { a =>
if (a semanticEquals newEventTime) {
val updatedMetadata = new MetadataBuilder()
.withMetadata(a.metadata)
.putLong(EventTimeWatermark.delayKey, delayMs)
.build()
a.withMetadata(updatedMetadata)
} else if (a.metadata.contains(EventTimeWatermark.delayKey)) {
// Remove existing columns tagged as eventTime for watermark
val updatedMetadata = new MetadataBuilder()
.withMetadata(a.metadata)
.remove(EventTimeWatermark.delayKey)
.build()
a.withMetadata(updatedMetadata)
} else {
a
}
}
}
}

/**
Expand All @@ -49,26 +80,38 @@ case class EventTimeWatermark(
// logic here because we also maintain the compatibility flag. (See
// SQLConf.STATEFUL_OPERATOR_ALLOW_MULTIPLE for details.)
// TODO: Disallow updating the metadata once we remove the compatibility flag.
override val output: Seq[Attribute] = child.output.map { a =>
if (a semanticEquals eventTime) {
val delayMs = EventTimeWatermark.getDelayMs(delay)
val updatedMetadata = new MetadataBuilder()
.withMetadata(a.metadata)
.putLong(EventTimeWatermark.delayKey, delayMs)
.build()
a.withMetadata(updatedMetadata)
} else if (a.metadata.contains(EventTimeWatermark.delayKey)) {
// Remove existing watermark
val updatedMetadata = new MetadataBuilder()
.withMetadata(a.metadata)
.remove(EventTimeWatermark.delayKey)
.build()
a.withMetadata(updatedMetadata)
override val output: Seq[Attribute] = {
val delayMs = EventTimeWatermark.getDelayMs(delay)
updateEventTimeColumn(child.output, delayMs, eventTime)
}

override protected def withNewChildInternal(newChild: LogicalPlan): EventTimeWatermark =
copy(child = newChild)
}

/**
* Updates the event time column to [[eventTime]] in the child output.
*
* Any watermark calculations performed after this node will use the
* updated eventTimeColumn.
*/
case class UpdateEventTimeWatermarkColumn(
eventTime: Attribute,
delay: Option[CalendarInterval],
child: LogicalPlan) extends UnaryNode {

final override val nodePatterns: Seq[TreePattern] = Seq(UPDATE_EVENT_TIME_WATERMARK_COLUMN)

override def output: Seq[Attribute] = {
if (delay.isDefined) {
val delayMs = EventTimeWatermark.getDelayMs(delay.get)
updateEventTimeColumn(child.output, delayMs, eventTime)
} else {
a
child.output
}
}

override protected def withNewChildInternal(newChild: LogicalPlan): EventTimeWatermark =
override protected def withNewChildInternal(
newChild: LogicalPlan): UpdateEventTimeWatermarkColumn =
copy(child = newChild)
}
Original file line number Diff line number Diff line change
Expand Up @@ -579,7 +579,7 @@ object TransformWithState {
child: LogicalPlan): LogicalPlan = {
val keyEncoder = encoderFor[K]
val mapped = new TransformWithState(
UnresolvedDeserializer(encoderFor[K].deserializer, groupingAttributes),
UnresolvedDeserializer(keyEncoder.deserializer, groupingAttributes),
UnresolvedDeserializer(encoderFor[V].deserializer, dataAttributes),
groupingAttributes,
dataAttributes,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -104,6 +104,7 @@ object RuleIdCollection {
"org.apache.spark.sql.catalyst.analysis.TypeCoercionBase$CombinedTypeCoercionRule" ::
"org.apache.spark.sql.catalyst.analysis.UpdateOuterReferences" ::
"org.apache.spark.sql.catalyst.analysis.UpdateAttributeNullability" ::
"org.apache.spark.sql.catalyst.analysis.ResolveUpdateEventTimeWatermarkColumn" ::
// Catalyst Optimizer rules
"org.apache.spark.sql.catalyst.optimizer.BooleanSimplification" ::
"org.apache.spark.sql.catalyst.optimizer.CollapseProject" ::
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -133,6 +133,7 @@ object TreePattern extends Enumeration {
val UNION: Value = Value
val UNRESOLVED_RELATION: Value = Value
val UNRESOLVED_WITH: Value = Value
val UPDATE_EVENT_TIME_WATERMARK_COLUMN: Value = Value
val TEMP_RESOLVED_COLUMN: Value = Value
val TYPED_FILTER: Value = Value
val WINDOW: Value = Value
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4061,4 +4061,11 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
callDeprecatedMethodError("createTable(..., StructType, ...)",
"createTable(..., Array[Column], ...)")
}

def cannotAssignEventTimeColumn(): Throwable = {
new AnalysisException(
errorClass = "CANNOT_ASSIGN_EVENT_TIME_COLUMN_WITHOUT_WATERMARK",
messageParameters = Map()
)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -2762,4 +2762,15 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE
"numFields" -> numFields.toString,
"schemaLen" -> schemaLen.toString))
}

def emittedRowsAreOlderThanWatermark(
currentWatermark: Long, emittedRowEventTime: Long): SparkRuntimeException = {
new SparkRuntimeException(
errorClass = "EMITTING_ROWS_OLDER_THAN_WATERMARK_NOT_ALLOWED",
messageParameters = Map(
"currentWatermark" -> currentWatermark.toString,
"emittedRowEventTime" -> emittedRowEventTime.toString
)
)
}
}
Loading

0 comments on commit 5e49665

Please sign in to comment.