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

Qingsheng Ren commented on FLINK-24851:
---------------------------------------

Thanks for the discussion [~tashoyan] [~liliwei] ! I get your concerns.

The reason _auto.offset.reset_ is overwritten in the case provided by 
[~tashoyan] is that the default value of starting offset is 
{_}OffsetsInitializer.earliest(){_}, which will force _auto.offset.reset_ to 
EARLIEST. I think we need to reconsider this default behavior. 

How about we follow these two rules:
 # Builder API has higher priority
 # Align with the default behavior of KafkaConsumer

Under these two rules:

a. If _KafkaSourceBuilder#setStartingOffsets(OffsetsInitializer)_ is used, I 
think there's no controversy: We take the logic in builder and ignore 
properties.

b. If _KafkaSourceBuilder#setStartingOffsets(OffsetsInitializer)_ is not used, 
we respect the default behavior of KafkaConsumer:
    i. If _group.id_ is set, we use 
{_}OffsetsInitializer.committedOffsets(OffsetsResetStrategy){_}. Here 
_OffsetsResetStrategy_ takes _auto.offset.reset_ in properties, or the default 
value from KafkaConsumer if not set
    ii. If _group.id_ is not set, we use _auto.offset.reset_ for initializing 
starting offset, or the default value from KafkaConsumer if not set in 
properties

Following the logic above:
{code:scala}
val props = new Properties()
props.put("bootstrap.servers", "localhost:9092")
props.put("group.id", "group1")
props.put("auto.offset.reset", "latest")

val kafkaSource = KafkaSource.builder[String]()
      .setProperties(props)
      .build() 
{code}
The behavior of the above code snippet will be trying to start from committed 
offset of group "group1", and fallback to latest as specified by 
_auto.offset.reset_ if committed offset doesn't exist.

What do you think? 

> KafkaSourceBuilder: auto.offset.reset is ignored
> ------------------------------------------------
>
>                 Key: FLINK-24851
>                 URL: https://issues.apache.org/jira/browse/FLINK-24851
>             Project: Flink
>          Issue Type: Bug
>          Components: Connectors / Kafka
>    Affects Versions: 1.14.0
>            Reporter: Arseniy Tashoyan
>            Assignee: liwei li
>            Priority: Major
>              Labels: pull-request-available
>
> Creating KafkaSource like this:
> {code:scala}
> val props = new Properties()
> props.put("bootstrap.servers", "localhost:9092")
> props.put("group.id", "group1")
> props.put("auto.offset.reset", "latest")
> val kafkaSource = KafkaSource.builder[String]()
>       .setProperties(props)
>       .build()
> {code}
> The actually used value for _"auto.offset.reset"_ is *"earliest"* instead of 
> configured *"latest"*.
> This occurs because _"auto.offset.reset"_ gets overridden by 
> _startingOffsetsInitializer.getAutoOffsetResetStrategy().name().toLowerCase()_.
>  The default value for _startingOffsetsInitializer_ is _"earliest"_.
> This behavior is misleading.
> This behavior imposes an inconvenience on configuring the Kafka connector. We 
> cannot use the Kafka setting _"auto.offset.reset"_ as-is. Instead we must 
> extract this particular setting from other settings and propagate to 
> _KafkaSourceBuilder.setStartingOffsets()_:
> {code:scala}
> val kafkaSource = KafkaSource.builder[String]()
>       .setProperties(props)
>       .setStartingOffsets(
>         OffsetsInitializer.committedOffsets(
>           OffsetResetStrategy.valueOf(
>             props.get(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG)
>               .asInstanceOf[String]
>               .toUpperCase(Locale.ROOT)
>           )
>         )
>       )
>       .build()
> {code}
> The expected behavior is to use the value of _"auto.offset.reset"_ provided 
> by _KafkaSourceBuilder.setProperties()_ - unless overridden via 
> _KafkaSourceBuilder. setStartingOffsets()_.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to