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

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

                Author: ASF GitHub Bot
            Created on: 18/Oct/18 21:00
            Start Date: 18/Oct/18 21:00
    Worklog Time Spent: 10m 
      Work Description: tweise closed pull request #6723: [BEAM-5760] Add 
support for multi-element bundles to portable Flink runner.
URL: https://github.com/apache/beam/pull/6723
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java
 
b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java
index e0982b4cab8..ee49ef451db 100644
--- 
a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java
+++ 
b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java
@@ -83,6 +83,7 @@
   private transient StageBundleFactory stageBundleFactory;
   private transient LinkedBlockingQueue<KV<String, OutputT>> outputQueue;
   private transient ExecutableStage executableStage;
+  private transient RemoteBundle remoteBundle;
 
   public ExecutableStageDoFnOperator(
       String stepName,
@@ -159,49 +160,16 @@ private StateRequestHandler 
getStateRequestHandler(ExecutableStage executableSta
     }
   }
 
-  // TODO: currently assumes that every element is a separate bundle,
-  // but this can be changed by pushing some of this logic into the 
"DoFnRunner"
-  private void processElementWithSdkHarness(WindowedValue<InputT> element) 
throws Exception {
-    checkState(
-        stageBundleFactory != null, "%s not yet prepared", 
StageBundleFactory.class.getName());
-    checkState(
-        stateRequestHandler != null, "%s not yet prepared", 
StateRequestHandler.class.getName());
-
-    OutputReceiverFactory receiverFactory =
-        new OutputReceiverFactory() {
-          @Override
-          public FnDataReceiver<OutputT> create(String pCollectionId) {
-            return (receivedElement) -> {
-              // handover to queue, do not block the grpc thread
-              outputQueue.put(KV.of(pCollectionId, receivedElement));
-            };
-          }
-        };
-
-    try (RemoteBundle bundle =
-        stageBundleFactory.getBundle(receiverFactory, stateRequestHandler, 
progressHandler)) {
-      LOG.debug(String.format("Sending value: %s", element));
-      // TODO(BEAM-4681): Add support to Flink to support portable timers.
-      
Iterables.getOnlyElement(bundle.getInputReceivers().values()).accept(element);
-      // TODO: it would be nice to emit results as they arrive, can thread 
wait non-blocking?
-    }
-
-    // RemoteBundle close blocks until all results are received
-    KV<String, OutputT> result;
-    while ((result = outputQueue.poll()) != null) {
-      outputManager.output(outputMap.get(result.getKey()), (WindowedValue) 
result.getValue());
-    }
-  }
-
   @Override
-  public void close() throws Exception {
+  public void dispose() throws Exception {
+    // DoFnOperator generates another "bundle" for the final watermark
+    super.dispose();
     // Remove the reference to stageContext and make stageContext available 
for garbage collection.
     try (@SuppressWarnings("unused")
             AutoCloseable bundleFactoryCloser = stageBundleFactory;
         @SuppressWarnings("unused")
             AutoCloseable closable = stageContext) {}
     stageContext = null;
-    super.close();
   }
 
   @Override
@@ -213,7 +181,6 @@ protected void 
addSideInputValue(StreamRecord<RawUnionValue> streamRecord) {
     sideInputHandler.addSideInputValue(sideInput, 
value.withValue(value.getValue().getValue()));
   }
 
-  // TODO: remove single element bundle assumption
   @Override
   protected DoFnRunner<InputT, OutputT> createWrappingDoFnRunner(
       DoFnRunner<InputT, OutputT> wrappedRunner) {
@@ -222,15 +189,41 @@ protected void 
addSideInputValue(StreamRecord<RawUnionValue> streamRecord) {
 
   private class SdkHarnessDoFnRunner implements DoFnRunner<InputT, OutputT> {
     @Override
-    public void startBundle() {}
+    public void startBundle() {
+      checkState(
+          stageBundleFactory != null, "%s not yet prepared", 
StageBundleFactory.class.getName());
+      checkState(
+          stateRequestHandler != null, "%s not yet prepared", 
StateRequestHandler.class.getName());
+      OutputReceiverFactory receiverFactory =
+          new OutputReceiverFactory() {
+            @Override
+            public FnDataReceiver<OutputT> create(String pCollectionId) {
+              return (receivedElement) -> {
+                // handover to queue, do not block the grpc thread
+                outputQueue.put(KV.of(pCollectionId, receivedElement));
+              };
+            }
+          };
+
+      try {
+        remoteBundle =
+            stageBundleFactory.getBundle(receiverFactory, stateRequestHandler, 
progressHandler);
+      } catch (Exception e) {
+        throw new RuntimeException("Failed to start remote bundle", e);
+      }
+    }
 
     @Override
-    public void processElement(WindowedValue<InputT> elem) {
+    public void processElement(WindowedValue<InputT> element) {
+      checkState(remoteBundle != null, "%s not yet prepared", 
RemoteBundle.class.getName());
       try {
-        processElementWithSdkHarness(elem);
+        LOG.debug(String.format("Sending value: %s", element));
+        // TODO(BEAM-4681): Add support to Flink to support portable timers.
+        
Iterables.getOnlyElement(remoteBundle.getInputReceivers().values()).accept(element);
       } catch (Exception e) {
-        throw new RuntimeException(e);
+        throw new RuntimeException("Failed to process element with SDK 
harness.", e);
       }
+      emitResults();
     }
 
     @Override
@@ -238,7 +231,23 @@ public void onTimer(
         String timerId, BoundedWindow window, Instant timestamp, TimeDomain 
timeDomain) {}
 
     @Override
-    public void finishBundle() {}
+    public void finishBundle() {
+      try {
+        // TODO: it would be nice to emit results as they arrive, can thread 
wait non-blocking?
+        // close blocks until all results are received
+        remoteBundle.close();
+        emitResults();
+      } catch (Exception e) {
+        throw new RuntimeException("Failed to finish remote bundle", e);
+      }
+    }
+
+    private void emitResults() {
+      KV<String, OutputT> result;
+      while ((result = outputQueue.poll()) != null) {
+        outputManager.output(outputMap.get(result.getKey()), (WindowedValue) 
result.getValue());
+      }
+    }
 
     @Override
     public DoFn<InputT, OutputT> getFn() {
diff --git 
a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/ExecutableStageDoFnOperatorTest.java
 
b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/ExecutableStageDoFnOperatorTest.java
index fc71d0c0c0d..7888b4dd23f 100644
--- 
a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/ExecutableStageDoFnOperatorTest.java
+++ 
b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/ExecutableStageDoFnOperatorTest.java
@@ -59,6 +59,7 @@
 import org.apache.flink.api.common.cache.DistributedCache;
 import org.apache.flink.api.common.functions.RuntimeContext;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
 import org.apache.flink.util.OutputTag;
@@ -125,11 +126,12 @@ public void sdkErrorsSurfaceOnClose() throws Exception {
     FnDataReceiver<WindowedValue<?>> receiver = 
Mockito.mock(FnDataReceiver.class);
     
when(bundle.getInputReceivers()).thenReturn(ImmutableMap.of("pCollectionId", 
receiver));
 
-    Exception expected = new Exception();
+    Exception expected = new RuntimeException(new Exception());
     doThrow(expected).when(bundle).close();
     thrown.expectCause(is(expected));
 
     operator.processElement(new 
StreamRecord<>(WindowedValue.valueInGlobalWindow(0)));
+    testHarness.close();
   }
 
   @Test
@@ -254,10 +256,12 @@ public void close() {}
         new OneInputStreamOperatorTestHarness<>(operator);
 
     testHarness.open();
-
     testHarness.processElement(new StreamRecord<>(zero));
+    testHarness.close(); // triggers finish bundle
 
-    assertThat(testHarness.getOutput(), contains(new StreamRecord<>(three)));
+    assertThat(
+        testHarness.getOutput(),
+        contains(new StreamRecord<>(three), new Watermark(Long.MAX_VALUE)));
 
     assertThat(
         testHarness.getSideOutput(tagsToOutputTags.get(additionalOutput1)),
@@ -266,8 +270,6 @@ public void close() {}
     assertThat(
         testHarness.getSideOutput(tagsToOutputTags.get(additionalOutput2)),
         contains(new StreamRecord<>(five)));
-
-    testHarness.close();
   }
 
   @Test
@@ -280,14 +282,18 @@ public void testStageBundleClosed() throws Exception {
 
     OneInputStreamOperatorTestHarness<WindowedValue<Integer>, 
WindowedValue<Integer>> testHarness =
         new OneInputStreamOperatorTestHarness<>(operator);
+
+    RemoteBundle bundle = Mockito.mock(RemoteBundle.class);
+    when(stageBundleFactory.getBundle(any(), any(), any())).thenReturn(bundle);
+
     testHarness.open();
+    testHarness.close();
 
-    operator.close();
     verify(stageBundleFactory).close();
     verify(stageContext).close();
+    // DoFnOperator generates a final watermark, which triggers a new bundle..
+    verify(stageBundleFactory).getBundle(any(), any(), any());
     verifyNoMoreInteractions(stageBundleFactory);
-
-    testHarness.close();
   }
 
   @Test


 

----------------------------------------------------------------
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: 156133)
    Time Spent: 4h 50m  (was: 4h 40m)

> Portable Flink support for maxBundleSize/maxBundleMillis
> --------------------------------------------------------
>
>                 Key: BEAM-5760
>                 URL: https://issues.apache.org/jira/browse/BEAM-5760
>             Project: Beam
>          Issue Type: Improvement
>          Components: runner-flink
>    Affects Versions: 2.8.0
>            Reporter: Thomas Weise
>            Assignee: Thomas Weise
>            Priority: Major
>              Labels: portability-flink
>             Fix For: 2.9.0
>
>          Time Spent: 4h 50m
>  Remaining Estimate: 0h
>
> The portable runner needs to support larger bundles in streaming mode. 
> Currently every element is a separate bundle, which is very inefficient due 
> to the per bundle SDK worker overhead. The old Java SDK runner already 
> supports these parameters.



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

Reply via email to