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

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

                Author: ASF GitHub Bot
            Created on: 16/Oct/18 17:19
            Start Date: 16/Oct/18 17:19
    Worklog Time Spent: 10m 
      Work Description: mwylde 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_r225633678
 
 

 ##########
 File path: 
runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/StreamingImpulseSource.java
 ##########
 @@ -0,0 +1,73 @@
+/*
+ * 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.wrappers.streaming.io;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.beam.sdk.util.WindowedValue;
+import 
org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A streaming source that periodically produces an empty byte array. This is 
mostly useful for
+ * debugging, or for triggering periodic behavior in a portable pipeline.
+ */
+public class StreamingImpulseSource extends 
RichParallelSourceFunction<WindowedValue<byte[]>> {
+  private static final Logger LOG = 
LoggerFactory.getLogger(StreamingImpulseSource.class);
+
+  private final AtomicBoolean cancelled = new AtomicBoolean(false);
+  private long count = 0;
+  private final int intervalMillis;
+  private final int messageCount;
+
+  public StreamingImpulseSource(int intervalMillis, int messageCount) {
+    this.intervalMillis = intervalMillis;
+    this.messageCount = messageCount;
+  }
+
+  @Override
+  public void run(SourceContext<WindowedValue<byte[]>> ctx) {
+    // in order to produce messageCount messages across all parallel subtasks, 
we divide by
+    // the total number of subtasks
+    int subtaskCount = messageCount / 
getRuntimeContext().getNumberOfParallelSubtasks();
 
 Review comment:
   Yes, that's the default. If messageCount is 0 we'll run forever.

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