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

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

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

 ##########
 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()));
 
 Review comment:
   Do we need to handle key collision in different 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: 218227)
    Time Spent: 40m  (was: 0.5h)

> 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: 40m
>  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