forked from apache/spark
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[SPARK-47960][SS] Allow chaining other stateful operators after trans…
…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
1 parent
3d9d1f3
commit 5e49665
Showing
16 changed files
with
866 additions
and
51 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
52 changes: 52 additions & 0 deletions
52
.../scala/org/apache/spark/sql/catalyst/analysis/ResolveUpdateEventTimeWatermarkColumn.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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)) | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.