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

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

                Author: ASF GitHub Bot
            Created on: 12/Oct/18 09:03
            Start Date: 12/Oct/18 09:03
    Worklog Time Spent: 10m 
      Work Description: mxm commented on a change in pull request #6637: 
[BEAM-5707] Add a periodic, streaming impulse source for Flink portable 
pipelines
URL: https://github.com/apache/beam/pull/6637#discussion_r224718069
 
 

 ##########
 File path: 
runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPortablePipelineTranslator.java
 ##########
 @@ -406,6 +417,56 @@ private void translateImpulse(
     
context.addDataStream(Iterables.getOnlyElement(pTransform.getOutputsMap().values()),
 source);
   }
 
+  @AutoService(NativeTransforms.IsNativeTransform.class)
+  public static class IsFlinkNativeTransform implements 
NativeTransforms.IsNativeTransform {
+    @Override
+    public boolean test(RunnerApi.PTransform pTransform) {
+      return 
STREAMING_IMPULSE_TRANSFORM_URL.equals(PTransformTranslation.urnForTransformOrNull(pTransform));
+    }
+  }
+
+  private void translateStreamingImpulse(
+      String id, RunnerApi.Pipeline pipeline, StreamingTranslationContext 
context) {
+    RunnerApi.PTransform pTransform = 
pipeline.getComponents().getTransformsOrThrow(id);
+
+    ObjectMapper objectMapper = new ObjectMapper();
+
+    int intervalMillis;
+    int messageCount;
+    try {
+      JsonNode config = 
objectMapper.readTree(pTransform.getSpec().getPayload().toByteArray());
+      intervalMillis = config.path("interval_ms").asInt(100);
+      messageCount = config.path("message_count").asInt(0);
+    } catch (IOException e) {
+        throw new RuntimeException("Failed to parse configuration for 
streaming impulse", e);
+    }
+
+    DataStreamSource<WindowedValue<byte[]>> source =
+        context
+            .getExecutionEnvironment()
+            .addSource(
+                new RichParallelSourceFunction<WindowedValue<byte[]>>() {
+                  private AtomicBoolean cancelled = new AtomicBoolean(false);
+                  private AtomicLong count = new AtomicLong();
+
+                  @Override
+                  public void run(SourceContext<WindowedValue<byte[]>> ctx) 
throws Exception {
+                    while (!cancelled.get() && (messageCount == 0 || 
count.getAndIncrement() < messageCount)) {
+                      ctx.collect(WindowedValue.valueInGlobalWindow(new byte[] 
{}));
+                      Thread.sleep(intervalMillis);
 
 Review comment:
   You could also handle `InterruptedException` here since we typically want to 
continue processing the source, unless `cancel()` has been called.

----------------------------------------------------------------
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: 153837)
    Time Spent: 1h 40m  (was: 1.5h)

> Add a portable Flink streaming synthetic source for testing
> -----------------------------------------------------------
>
>                 Key: BEAM-5707
>                 URL: https://issues.apache.org/jira/browse/BEAM-5707
>             Project: Beam
>          Issue Type: Improvement
>          Components: runner-flink
>            Reporter: Micah Wylde
>            Assignee: Aljoscha Krettek
>            Priority: Minor
>          Time Spent: 1h 40m
>  Remaining Estimate: 0h
>
> Currently there are no built-in streaming sources for portable pipelines. 
> This makes it hard to test streaming functionality in the Python SDK.
> It would be very useful to add a periodic impulse source that (with some 
> configurable frequency) outputs an empty byte array, which can then be 
> transformed as desired inside the python pipeline. More context in this 
> [mailing list 
> discussion|https://lists.apache.org/thread.html/b44a648ab1d0cb200d8bfe4b280e9dad6368209c4725609cbfbbe410@%3Cdev.beam.apache.org%3E].



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

Reply via email to