flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-4280) New Flink-specific option to set starting position of Kafka consumer without respecting external offsets in ZK / Broker
Date Tue, 14 Feb 2017 14:37:43 GMT

    [ https://issues.apache.org/jira/browse/FLINK-4280?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15865860#comment-15865860
] 

ASF GitHub Bot commented on FLINK-4280:
---------------------------------------

Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2509#discussion_r101000341
  
    --- Diff: docs/dev/connectors/kafka.md ---
    @@ -161,6 +161,46 @@ For convenience, Flink provides the following schemas:
         The KeyValue objectNode contains a "key" and "value" field which contain all fields,
as well as
         an optional "metadata" field that exposes the offset/partition/topic for this message.
     
    +#### Kafka Consumers Start Position Configuration
    +
    +By default, the Flink Kafka Consumer starts reading partitions from the consumer group's
(`group.id` setting in the
    +consumer properties) committed offsets in Kafka brokers (or Zookeeper for Kafka 0.8).
    +
    +This behaviour can be explicitly overriden, as demonstrated below:
    +
    +<div class="codetabs" markdown="1">
    +<div data-lang="java" markdown="1">
    +{% highlight java %}
    +final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +FlinkKafkaConsumer08<String> myConsumer = new FlinkKafkaConsumer08<>(...);
    +myConsumer.setStartFromEarliest();     // start from the earliest record possible
    +myConsumer.setStartFromLatest();       // start from the latest record
    +myConsumer.setStartFromGroupOffsets(); // the default behaviour
    --- End diff --
    
    Does the "the default behaviour" also mean that we only respect the "auto.offset.reset"
configs in that case?


> New Flink-specific option to set starting position of Kafka consumer without respecting
external offsets in ZK / Broker
> -----------------------------------------------------------------------------------------------------------------------
>
>                 Key: FLINK-4280
>                 URL: https://issues.apache.org/jira/browse/FLINK-4280
>             Project: Flink
>          Issue Type: New Feature
>          Components: Kafka Connector
>            Reporter: Tzu-Li (Gordon) Tai
>            Assignee: Tzu-Li (Gordon) Tai
>             Fix For: 1.3.0
>
>
> Currently, to start reading from the "earliest" and "latest" position in topics for the
Flink Kafka consumer, users set the Kafka config {{auto.offset.reset}} in the provided properties
configuration.
> However, the way this config actually works might be a bit misleading if users were trying
to find a way to "read topics from a starting position". The way the {{auto.offset.reset}}
config works in the Flink Kafka consumer resembles Kafka's original intent for the setting:
first, existing external offsets committed to the ZK / brokers will be checked; if none exists,
then will {{auto.offset.reset}} be respected.
> I propose to add Flink-specific ways to define the starting position, without taking
into account the external offsets. The original behaviour (reference external offsets first)
can be changed to be a user option, so that the behaviour can be retained for frequent Kafka
users that may need some collaboration with existing non-Flink Kafka consumer applications.
> How users will interact with the Flink Kafka consumer after this is added, with a newly
introduced {{flink.starting-position}} config:
> {code}
> Properties props = new Properties();
> props.setProperty("flink.starting-position", "earliest/latest");
> props.setProperty("auto.offset.reset", "..."); // this will be ignored (log a warning)
> props.setProperty("group.id", "...") // this won't have effect on the starting position
anymore (may still be used in external offset committing)
> ...
> {code}
> Or, reference external offsets in ZK / broker:
> {code}
> Properties props = new Properties();
> props.setProperty("flink.starting-position", "external-offsets");
> props.setProperty("auto.offset.reset", "earliest/latest"); // default will be latest
> props.setProperty("group.id", "..."); // will be used to lookup external offsets in ZK
/ broker on startup
> ...
> {code}
> A thing we would need to decide on is what would the default value be for {{flink.starting-position}}.
> Two merits I see in adding this:
> 1. This compensates the way users generally interpret "read from a starting position".
As the Flink Kafka connector is somewhat essentially a "high-level" Kafka consumer for Flink
users, I think it is reasonable to add Flink-specific functionality that users will find useful,
although it wasn't supported in Kafka's original consumer designs.
> 2. By adding this, the idea that "the Kafka offset store (ZK / brokers) is used only
to expose progress to the outside world, and not used to manipulate how Kafka topics are read
in Flink (unless users opt to do so)" is even more definite and solid. There was some discussion
in this PR (https://github.com/apache/flink/pull/1690, FLINK-3398) on this aspect. I think
adding this "decouples" more Flink's internal offset checkpointing from the external Kafka's
offset store.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Mime
View raw message