[ https://issues.apache.org/jira/browse/BEAM-5707?focusedWorklogId=154383&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-154383 ]
ASF GitHub Bot logged work on BEAM-5707: ---------------------------------------- Author: ASF GitHub Bot Created on: 15/Oct/18 17:34 Start Date: 15/Oct/18 17:34 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_r225253456 ########## File path: runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/StreamingImpulseSource.java ########## @@ -0,0 +1,63 @@ +/* + * 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[]>> { Review comment: I guess what I'm trying to say is this implementation does not behave like `Impulse`. We should keep that in mind when writing test cases against this native source. I thought it doesn't really make a difference for scheduling. That's why I closed #6654. We can still let the source run in parallel but only emit elements on one of them using the subtask index. ---------------------------------------------------------------- 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: 154383) Time Spent: 3h 20m (was: 3h 10m) > 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: 3h 20m > 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)