[ 
https://issues.apache.org/jira/browse/BEAM-6876?focusedWorklogId=218283&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-218283
 ]

ASF GitHub Bot logged work on BEAM-6876:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 25/Mar/19 21:37
            Start Date: 25/Mar/19 21:37
    Worklog Time Spent: 10m 
      Work Description: mxm commented on pull request #8118: [BEAM-6876] 
Cleanup user state in portable Flink Runner
URL: https://github.com/apache/beam/pull/8118#discussion_r268862438
 
 

 ##########
 File path: 
runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java
 ##########
 @@ -713,6 +705,86 @@ private void emitResults() {
     }
   }
 
+  private DoFnRunner<InputT, OutputT> ensureStateCleanup(
+      SdkHarnessDoFnRunner<InputT, OutputT> sdkHarnessRunner) {
+    if (keyCoder == null) {
+      // There won't be any state to clean up
+      // (stateful functions have to be keyed)
+      return sdkHarnessRunner;
+    }
+    // Takes care of state cleanup via StatefulDoFnRunner
+    Coder windowCoder = windowingStrategy.getWindowFn().windowCoder();
+    StatefulDoFnRunner.CleanupTimer<InputT> cleanupTimer =
+        new StatefulDoFnRunner.CleanupTimer<InputT>() {
+
+          private static final String GC_TIMER_ID = "__user-state-cleanup__";
+
+          @Override
+          public Instant currentInputWatermarkTime() {
+            return timerInternals.currentInputWatermarkTime();
+          }
+
+          @Override
+          public void setForWindow(InputT input, BoundedWindow window) {
+            Preconditions.checkNotNull(input, "Null input passed to 
CleanupTimer");
+            // make sure this fires after any window.maxTimestamp() timers
+            Instant gcTime = LateDataUtils.garbageCollectionTime(window, 
windowingStrategy).plus(1);
+            ByteBuffer key;
+            try {
+              key =
+                  ByteBuffer.wrap(
+                      CoderUtils.encodeToByteArray((Coder) keyCoder, ((KV) 
input).getKey()));
+            } catch (CoderException e) {
+              throw new RuntimeException("Failed to encode key for Flink state 
backend", e);
+            }
+            // Ensure the state backend is not concurrently accessed by the 
state requests
+            try {
+              stateBackendLock.lock();
+              // Set these two to ensure correct timer registration
+              // 1) For the timer setting
+              sdkHarnessRunner.setCurrentTimerKey(key);
+              // 2) For the timer deduplication
+              getKeyedStateBackend().setCurrentKey(key);
+              timerInternals.setTimer(
+                  StateNamespaces.window(windowCoder, window),
+                  GC_TIMER_ID,
+                  gcTime,
 
 Review comment:
   The state namespace is assumed to be unique for each window, if not then 
they are the same window and the GC time should be the same. Internally, 
`StateNamespaces$WindowNamespace#stringKey()` serializes the Window to a byte 
array and base64 encodes it to a string. That's a bit crazy but that's how Beam 
state works for windows. 
 
----------------------------------------------------------------
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:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 218283)
    Time Spent: 1h  (was: 50m)

> User state cleanup in portable Flink runner
> -------------------------------------------
>
>                 Key: BEAM-6876
>                 URL: https://issues.apache.org/jira/browse/BEAM-6876
>             Project: Beam
>          Issue Type: Bug
>          Components: runner-flink
>    Affects Versions: 2.11.0
>            Reporter: Thomas Weise
>            Assignee: Maximilian Michels
>            Priority: Major
>              Labels: portability-flink
>          Time Spent: 1h
>  Remaining Estimate: 0h
>
> State is currently not being cleaned up by the runner.
> [https://lists.apache.org/thread.html/86f0809fbfa3da873051287b9ff249d6dd5a896b45409db1e484cf38@%3Cdev.beam.apache.org%3E]
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to