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

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

                Author: ASF GitHub Bot
            Created on: 14/Nov/18 13:55
            Start Date: 14/Nov/18 13:55
    Worklog Time Spent: 10m 
      Work Description: mxm commented on a change in pull request #6981: 
[BEAM-4681] Add support for portable timers in Flink streaming mode 
URL: https://github.com/apache/beam/pull/6981#discussion_r233443898
 
 

 ##########
 File path: 
runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java
 ##########
 @@ -359,13 +533,100 @@ public void finishBundle() {
         emitResults();
       } catch (Exception e) {
         throw new RuntimeException("Failed to finish remote bundle", e);
+      } finally {
+        remoteBundle = null;
+      }
+      if (bundleFinishedCallback != null) {
+        bundleFinishedCallback.run();
+        bundleFinishedCallback = null;
       }
     }
 
+    /** Key for timer which has not been registered yet. */
+    Object getTimerKeyForRegistration() {
+      return keyForTimerToBeSet;
+    }
+
+    /** Key for timer which is about to be fired. */
+    void setTimerKeyForFire(Object key) {
+      this.keyForTimerToBeFired = key;
+    }
+
+    boolean isBundleInProgress() {
+      return remoteBundle != null;
+    }
+
+    void setBundleFinishedCallback(Runnable callback) {
+      this.bundleFinishedCallback = callback;
+    }
+
     private void emitResults() {
       KV<String, OutputT> result;
       while ((result = outputQueue.poll()) != null) {
-        outputManager.output(outputMap.get(result.getKey()), (WindowedValue) 
result.getValue());
+        final String outputPCollectionId = 
Preconditions.checkNotNull(result.getKey());
+        TupleTag<?> tag = outputMap.get(outputPCollectionId);
+        WindowedValue windowedValue =
+            Preconditions.checkNotNull(
+                (WindowedValue) result.getValue(),
+                "Received a null value from the SDK harness for %s",
+                outputPCollectionId);
+        if (tag != null) {
+          // process regular elements
+          outputManager.output(tag, windowedValue);
+        } else {
+          final String timerCollectionId = 
extractTimerPCollectionId(outputPCollectionId);
+          TimerSpec timerSpec = extractTimerSpec(timerCollectionId);
+          Timer timer =
+              Preconditions.checkNotNull(
+                  (Timer) ((KV) windowedValue.getValue()).getValue(),
+                  "Received null Timer from SDK harness: %s",
+                  windowedValue);
+          LOG.debug("Timer received: {} {}", outputPCollectionId, timer);
+          for (Object window : windowedValue.getWindows()) {
+            StateNamespace namespace = StateNamespaces.window(windowCoder, 
(BoundedWindow) window);
+            TimerInternals.TimerData timerData =
+                TimerInternals.TimerData.of(
+                    timerCollectionId, namespace, timer.getTimestamp(), 
timerSpec.getTimeDomain());
+            setTimer(windowedValue, timerData);
+          }
+        }
+      }
+    }
+
+    private String extractTimerPCollectionId(String outputPCollectionId) {
+      return stageBundleFactory
+          .getProcessBundleDescriptor()
+          .getProcessBundleDescriptor()
+          .getPcollectionsMap()
+          .get(outputPCollectionId)
+          .getUniqueName();
 
 Review comment:
   It is still necessary. The timerReferenceMap gives me `ParDo.timer.foo` but 
while processing the timer elements, I get `ParDo.timer.foo.out:0`. The issue 
seems to be that the output collection ids are not set correctly.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on 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: 165882)
    Time Spent: 16h 20m  (was: 16h 10m)

> Integrate support for timers using the portability APIs into Flink
> ------------------------------------------------------------------
>
>                 Key: BEAM-4681
>                 URL: https://issues.apache.org/jira/browse/BEAM-4681
>             Project: Beam
>          Issue Type: Sub-task
>          Components: runner-flink
>            Reporter: Luke Cwik
>            Assignee: Maximilian Michels
>            Priority: Major
>              Labels: portability, portability-flink
>          Time Spent: 16h 20m
>  Remaining Estimate: 0h
>
> Consider using the code produced in BEAM-4658 to support timers.



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

Reply via email to