[ https://issues.apache.org/jira/browse/BEAM-6751?focusedWorklogId=208043&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-208043 ]
ASF GitHub Bot logged work on BEAM-6751: ---------------------------------------- Author: ASF GitHub Bot Created on: 05/Mar/19 20:39 Start Date: 05/Mar/19 20:39 Worklog Time Spent: 10m Work Description: kennknowles commented on pull request #7991: [BEAM-6751] Add KafkaIO EOS support to Flink via @RequiresStableInput URL: https://github.com/apache/beam/pull/7991#discussion_r262641597 ########## File path: runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java ########## @@ -171,6 +175,9 @@ private final DoFnSchemaInformation doFnSchemaInformation; + /** If true, we must not emit elements while a checkpoint is pending. */ Review comment: Just reading through this PR in order, you should not emit elements when downstream requires stable input, right? Looking at the code below, it looks like if this is true you don't process _input_ elements while checkpoint is pending, which seems right. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to 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: 208043) Time Spent: 2h 20m (was: 2h 10m) > KafkaIO's EOS mode does not work with FlinkRunner > ------------------------------------------------- > > Key: BEAM-6751 > URL: https://issues.apache.org/jira/browse/BEAM-6751 > Project: Beam > Issue Type: Bug > Components: io-java-kafka, runner-flink > Reporter: Maximilian Michels > Assignee: Maximilian Michels > Priority: Major > Fix For: 2.12.0 > > Time Spent: 2h 20m > Remaining Estimate: 0h > > KafkaIO has a validation check which whitelists certain runners capable of > provide exactly-once semantics: > {noformat} > if ("org.apache.beam.runners.direct.DirectRunner".equals(runner) > || runner.startsWith("org.apache.beam.runners.dataflow.") > || runner.startsWith("org.apache.beam.runners.spark.") { > ... > {noformat} > The Flink supports exactly-once checkpointing but the Flink Runner can't > utilize it in the way KafkaIO intends it. > I think we should remove the check in favor of checking for translation of > {{@RequiresStableInput}}. Changes to KafkaIO might have to be made to support > EOS efficiently with the Flink Runner. -- This message was sent by Atlassian JIRA (v7.6.3#76005)