You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
Copy file name to clipboardExpand all lines: CHANGES.md
+1Lines changed: 1 addition & 0 deletions
Display the source diff
Display the rich diff
Original file line number
Diff line number
Diff line change
@@ -70,6 +70,7 @@
70
70
71
71
## New Features / Improvements
72
72
73
+
* Capability introduces an indicator for aggregations and timers firing during a pipeline drain, allowing users and sinks to recognize and appropriately handle potentially incomplete or partial data ([#36884](https://github.com/apache/beam/issues/36884)).
73
74
* Added support for setting disk provisioned IOPS and throughput in Dataflow runner via `--diskProvisionedIops` and `--diskProvisionedThroughputMibps` pipeline options (Java/Go) ([#38349](https://github.com/apache/beam/issues/38349)).
74
75
* X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)).
75
76
* TriggerStateMachineRunner changes from BitSetCoder to SentinelBitSetCoder to
@@ -347,4 +354,4 @@ support for dealing with this use case in production.
347
354
348
355
349
356
Runner specific notes:
350
-
Google Cloud Dataflow Runners Drain feature does not support looping timers (Link to matrix)
357
+
Support for cancelling looping timers on drain is currently limited to Dataflow and is being implemented (see [Issue #36884](https://github.com/apache/beam/issues/36884)).
Looping timers are a pattern where a timer sets another timer for a future time, creating a loop. This is useful for producing periodic outputs or heartbeats in the absence of data for a specific key.
7507
+
7508
+
When draining a pipeline, it is important to terminate these loops to allow the pipeline to finish. In the Java SDK, you can use the `CausedByDrain` parameter in the `@OnTimer` method to check if the timer firing was induced by a drain operation. **Note:**`CausedByDrain` will be set only in certain runners. Check the [capability matrix](/documentation/runners/capability-matrix/) for more details.
7509
+
7510
+
{{< highlight java >}}
7511
+
public static class LoopingStatefulTimer extends DoFn<KV<String, Integer>, KV<String, Integer>> {
7512
+
@StateId("key") private final StateSpec<ValueState<String>> key = StateSpecs.value();
7513
+
@TimerId("loopingTimer") private final TimerSpec loopingTimer = TimerSpecs.timer(TimeDomain.EVENT_TIME);
0 commit comments