mxm commented on a change in pull request #12759:
URL: https://github.com/apache/beam/pull/12759#discussion_r482841893



##########
File path: 
runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
##########
@@ -75,7 +80,14 @@
 public class FlinkStateInternals<K> implements StateInternals {
 
   private final KeyedStateBackend<ByteBuffer> flinkStateBackend;
-  private Coder<K> keyCoder;
+  private final Coder<K> keyCoder;
+
+  /**
+   * A set which contains all state descriptors created in the global window. 
Used for cleanup on

Review comment:
       Yes, that's right but that doesn't matter because all other state should 
also be cleaned up with the global window.

##########
File path: 
runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
##########
@@ -75,7 +80,14 @@
 public class FlinkStateInternals<K> implements StateInternals {
 
   private final KeyedStateBackend<ByteBuffer> flinkStateBackend;
-  private Coder<K> keyCoder;
+  private final Coder<K> keyCoder;
+
+  /**
+   * A set which contains all state descriptors created in the global window. 
Used for cleanup on

Review comment:
       I'll remove "global window".

##########
File path: 
runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
##########
@@ -139,17 +163,27 @@ private FlinkStateBinder(StateNamespace namespace, 
StateContext<?> stateContext)
     @Override
     public <T2> ValueState<T2> bindValue(
         String id, StateSpec<ValueState<T2>> spec, Coder<T2> coder) {
-      return new FlinkValueState<>(flinkStateBackend, id, namespace, coder);
+      ValueStateDescriptor<T2> valueStateDescriptor =
+          new ValueStateDescriptor<>(id, new CoderTypeSerializer<>(coder));
+      globalWindowStateDescriptors.add(valueStateDescriptor);
+      return new FlinkValueState<>(flinkStateBackend, id, namespace, 
valueStateDescriptor);
     }
 
     @Override
     public <T2> BagState<T2> bindBag(String id, StateSpec<BagState<T2>> spec, 
Coder<T2> elemCoder) {
-      return new FlinkBagState<>(flinkStateBackend, id, namespace, elemCoder);
+      ListStateDescriptor<T2> listStateDescriptor =
+          new ListStateDescriptor<>(id, new CoderTypeSerializer<>(elemCoder));
+      globalWindowStateDescriptors.add(listStateDescriptor);

Review comment:
       This is unrelated to the changes here. This object was just created a 
layer down before (FlinkBagState). Let's handle such optimizations in a 
follow-up. 




----------------------------------------------------------------
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]


Reply via email to