[
https://issues.apache.org/jira/browse/BEAM-4681?focusedWorklogId=165873&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-165873
]
ASF GitHub Bot logged work on BEAM-4681:
----------------------------------------
Author: ASF GitHub Bot
Created on: 14/Nov/18 12:29
Start Date: 14/Nov/18 12:29
Worklog Time Spent: 10m
Work Description: robertwb 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_r233426740
##########
File path:
runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java
##########
@@ -359,13 +534,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());
Review comment:
The notion "timerCollectionId" is ambiguous, as timers are associated with
two distinct PCollections at execution time.
Timers are absolutely determined by their transform name and local name;
probably it makes sense to use that (let's call it fullyQualifiedTimerName) for
the TimerData here, and then also have a mapping fullyQualifiedTimerName ->
timerInputPCollectionId for plumbing timers to the SDK, and a map
timerOutputPCollectionId -> fullyQualifiedTimerName for recording timers from
the SDK (like we do here).
----------------------------------------------------------------
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:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 165873)
Time Spent: 16h 10m (was: 16h)
> 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 10m
> 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)