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

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

                Author: ASF GitHub Bot
            Created on: 15/Oct/18 10:30
            Start Date: 15/Oct/18 10:30
    Worklog Time Spent: 10m 
      Work Description: mxm closed pull request #6654: [BEAM-5713] Make 
ImpulseSourceFunction execute in parallel
URL: https://github.com/apache/beam/pull/6654
 
 
   

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/functions/ImpulseSourceFunction.java
 
b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/ImpulseSourceFunction.java
index 420d90e749a..b717d200916 100644
--- 
a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/ImpulseSourceFunction.java
+++ 
b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/ImpulseSourceFunction.java
@@ -19,13 +19,13 @@
 package org.apache.beam.runners.flink.translation.functions;
 
 import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
 
 /**
  * Source function which sends an impulse to a downstream operator. It may 
keep the source alive
  * although its work is already done. It will only shutdown when requested by 
the JobManager.
  */
-public class ImpulseSourceFunction implements 
SourceFunction<WindowedValue<byte[]>> {
+public class ImpulseSourceFunction implements 
ParallelSourceFunction<WindowedValue<byte[]>> {
 
   /** Keep source running even after it has done all the work. */
   private final boolean keepSourceAlive;
diff --git 
a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/ImpulseInputFormat.java
 
b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/ImpulseInputFormat.java
index d9c4cfd3ed7..c7160322a64 100644
--- 
a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/ImpulseInputFormat.java
+++ 
b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/ImpulseInputFormat.java
@@ -62,8 +62,13 @@ public float getAverageRecordWidth() {
 
   @Override
   public GenericInputSplit[] createInputSplits(int numSplits) {
-    // Always return a single split because only one global "impulse" will 
ever be sent.
-    return new GenericInputSplit[] {new GenericInputSplit(1, 1)};
+    // Generate one input split per partition (numSplits) to trigger
+    // an impulse for each parallel instance.
+    GenericInputSplit[] splits = new GenericInputSplit[numSplits];
+    for (int i = 0; i < splits.length; i++) {
+      splits[i] = new GenericInputSplit(i, numSplits);
+    }
+    return splits;
   }
 
   @Override
diff --git 
a/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableExecutionTest.java
 
b/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableExecutionTest.java
index 0f89bc4b94c..5d6253b7109 100644
--- 
a/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableExecutionTest.java
+++ 
b/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableExecutionTest.java
@@ -89,6 +89,7 @@ public void testExecution() throws Exception {
     options.setRunner(CrashingRunner.class);
     options.as(FlinkPipelineOptions.class).setFlinkMaster("[local]");
     options.as(FlinkPipelineOptions.class).setStreaming(isStreaming);
+    options.as(FlinkPipelineOptions.class).setParallelism(2);
     options
         .as(PortablePipelineOptions.class)
         .setDefaultEnvironmentType(Environments.ENVIRONMENT_EMBEDDED);
@@ -150,6 +151,6 @@ public void process(ProcessContext ctx) {
 
     assertEquals(1, outputValues.size());
     assertEquals("foo", outputValues.get(0).getKey());
-    assertThat(outputValues.get(0).getValue(), containsInAnyOrder(4L, 3L, 3L));
+    assertThat(outputValues.get(0).getValue(), containsInAnyOrder(4L, 3L, 3L, 
4L, 3L, 3L));
   }
 }
diff --git 
a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/ImpulseSourceFunctionTest.java
 
b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/ImpulseSourceFunctionTest.java
new file mode 100644
index 00000000000..598bc8935ee
--- /dev/null
+++ 
b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/ImpulseSourceFunctionTest.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.runners.flink.translation.functions;
+
+import static org.hamcrest.core.Is.is;
+import static org.hamcrest.core.IsInstanceOf.instanceOf;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.verify;
+
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.junit.After;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Tests for {@link ImpulseSourceFunction}. */
+public class ImpulseSourceFunctionTest {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ImpulseSourceFunctionTest.class);
+
+  @Rule public TestName testName = new TestName();
+
+  private final SourceFunction.SourceContext<WindowedValue<byte[]>> 
sourceContext;
+
+  public ImpulseSourceFunctionTest() {
+    this.sourceContext = Mockito.mock(SourceFunction.SourceContext.class);
+  }
+
+  @After
+  public void after() {
+    if 
(testName.getMethodName().equals("testInstanceOfParallelSourceFunction")) {
+      return;
+    }
+    verify(sourceContext).collect(Mockito.any());
+  }
+
+  @Test
+  public void testInstanceOfParallelSourceFunction() {
+    assertThat(new ImpulseSourceFunction(false), 
instanceOf(ParallelSourceFunction.class));
+  }
+
+  @Test(timeout = 10_000)
+  public void testKeepAlive() throws Exception {
+    ImpulseSourceFunction source = new ImpulseSourceFunction(true);
+    Thread sourceThread =
+        new Thread(
+            new Runnable() {
+              @Override
+              public void run() {
+                try {
+                  source.run(sourceContext);
+                  // should not finish
+                } catch (Exception e) {
+                  LOG.error("Exception while executing ImpulseSourceFunction", 
e);
+                }
+              }
+            });
+    try {
+      sourceThread.start();
+      source.cancel();
+      // should finish
+      sourceThread.join();
+    } finally {
+      sourceThread.interrupt();
+      sourceThread.join();
+    }
+  }
+
+  @Test(timeout = 10_000)
+  public void testKeepAliveDuringInterrupt() throws Exception {
+    ImpulseSourceFunction source = new ImpulseSourceFunction(true);
+    Thread sourceThread =
+        new Thread(
+            new Runnable() {
+              @Override
+              public void run() {
+                try {
+                  source.run(sourceContext);
+                  // should not finish
+                } catch (Exception e) {
+                  LOG.error("Exception while executing ImpulseSourceFunction", 
e);
+                }
+              }
+            });
+
+    sourceThread.start();
+    sourceThread.interrupt();
+    Thread.sleep(200);
+    assertThat(sourceThread.isAlive(), is(true));
+    // should quit
+    source.cancel();
+    sourceThread.interrupt();
+    sourceThread.join();
+  }
+
+  @Test(timeout = 10_000)
+  public void testKeepAliveDisabled() throws Exception {
+    ImpulseSourceFunction source = new ImpulseSourceFunction(false);
+    source.run(sourceContext);
+    // should finish
+  }
+}


 

----------------------------------------------------------------
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: 154244)
    Time Spent: 1h  (was: 50m)

> Flink portable runner schedules all tasks of streaming job on same task 
> manager
> -------------------------------------------------------------------------------
>
>                 Key: BEAM-5713
>                 URL: https://issues.apache.org/jira/browse/BEAM-5713
>             Project: Beam
>          Issue Type: Bug
>          Components: runner-flink
>    Affects Versions: 2.8.0
>            Reporter: Thomas Weise
>            Assignee: Maximilian Michels
>            Priority: Major
>              Labels: portability, portability-flink
>         Attachments: Different SlotSharingGroup.png, With 
> RichParallelSourceFunction and parallelism 5.png, 
> image-2018-10-11-11-43-50-333.png, image-2018-10-11-16-20-45-221.png
>
>          Time Spent: 1h
>  Remaining Estimate: 0h
>
> The cluster has 9 task managers and 144 task slots total. A simple streaming 
> pipeline with parallelism of 8 will get all tasks scheduled on the same task 
> manager, causing the host to be fully booked and the remaining cluster idle.



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

Reply via email to