[
https://issues.apache.org/jira/browse/BEAM-6751?focusedWorklogId=208919&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-208919
]
ASF GitHub Bot logged work on BEAM-6751:
----------------------------------------
Author: ASF GitHub Bot
Created on: 06/Mar/19 16:10
Start Date: 06/Mar/19 16:10
Worklog Time Spent: 10m
Work Description: mxm commented on pull request #7991: [BEAM-6751] Add
KafkaIO EOS support to Flink via @RequiresStableInput
URL: https://github.com/apache/beam/pull/7991#discussion_r263012597
##########
File path:
runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
##########
@@ -230,6 +239,25 @@ public DoFnOperator(
this.maxBundleSize = flinkOptions.getMaxBundleSize();
this.maxBundleTimeMills = flinkOptions.getMaxBundleTimeMills();
this.doFnSchemaInformation = doFnSchemaInformation;
+
+ this.requiresStableInput =
+ // WindowDoFnOperator does not use a DoFn
+ doFn != null
+ &&
DoFnSignatures.getSignature(doFn.getClass()).processElement().requiresStableInput();
+
+ if (requiresStableInput) {
+ Preconditions.checkState(
+ flinkOptions.getCheckpointingMode() ==
CheckpointingMode.EXACTLY_ONCE,
+ "Checkpointing mode is not set to exactly once but
@RequiresStableInput is used.");
+ Preconditions.checkState(
+ flinkOptions.getCheckpointingInterval() > 0,
+ "No checkpointing configured but pipeline uses
@RequiresStableInput");
+ LOG.warn(
+ "Enabling stable input for transform {}. Will only process elements
at most every {} milliseconds.",
Review comment:
I've added this warning here and the checks above.
----------------------------------------------------------------
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:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 208919)
Time Spent: 4h 20m (was: 4h 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: 4h 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)