mxm commented on a change in pull request #12759:
URL: https://github.com/apache/beam/pull/12759#discussion_r484720811
##########
File path:
runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
##########
@@ -111,19 +128,43 @@ public K getKey() {
@Override
public <T extends State> T state(
StateNamespace namespace, StateTag<T> address, StateContext<?> context) {
+ if (globalWindowNamespace.equals(namespace)) {
+ // Take note of state bound to the global window for cleanup in
clearGlobalState below.
+ globalWindowStateTags.add(address);
+ }
return address.getSpec().bind(address.getId(), new
FlinkStateBinder(namespace, context));
}
- public void clearBagStates(StateNamespace namespace, StateTag<? extends
BagState> address)
- throws Exception {
- CoderTypeSerializer typeSerializer = new
CoderTypeSerializer<>(VoidCoder.of());
- flinkStateBackend.applyToAllKeys(
- namespace.stringKey(),
- StringSerializer.INSTANCE,
- new ListStateDescriptor<>(address.getId(), typeSerializer),
- (key, state) -> {
+ /**
+ * Allows to clear all state for the global watermark when the maximum
watermark arrives. We do
+ * not clean up the global window state via timers which would lead to an
unbounded number of keys
+ * and cleanup timers. Instead, the cleanup code below should be run when we
finally receive the
+ * max watermark.
+ */
+ public void clearGlobalState() {
+ try {
+ for (StateTag stateTag : globalWindowStateTags) {
+ State state =
+ state(
+ globalWindowNamespace,
+ stateTag,
+ StateContexts.windowOnlyContext(GlobalWindow.INSTANCE));
+ // We collect all keys in the global window for a particular state
+ // Note that the alternative method applyToAllKeys(..) does the same
internally.
Review comment:
Not really, other than using Beam's state descriptors instead of
Flink's. Good point on other state backends potentially allowing to iterate
over the keys without materializing first. Have reverted this back to the old
behavior.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]