[
https://issues.apache.org/jira/browse/FLINK-6352?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16011944#comment-16011944
]
ASF GitHub Bot commented on FLINK-6352:
---------------------------------------
Github user tzulitai commented on a diff in the pull request:
https://github.com/apache/flink/pull/3915#discussion_r116674925
--- Diff:
flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java
---
@@ -181,12 +181,6 @@ public FlinkKafkaConsumer09(List<String> topics,
KeyedDeserializationSchema<T> d
boolean useMetrics = !PropertiesUtil.getBoolean(properties,
KEY_DISABLE_METRICS, false);
- // make sure that auto commit is disabled when our offset
commit mode is ON_CHECKPOINTS;
- // this overwrites whatever setting the user configured in the
properties
- if (offsetCommitMode == OffsetCommitMode.ON_CHECKPOINTS ||
offsetCommitMode == OffsetCommitMode.DISABLED) {
-
properties.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false");
- }
--- End diff --
This shouldn't be removed (I assume you accidentally removed it when
rebasing?).
> FlinkKafkaConsumer should support to use timestamp to set up start offset
> -------------------------------------------------------------------------
>
> Key: FLINK-6352
> URL: https://issues.apache.org/jira/browse/FLINK-6352
> Project: Flink
> Issue Type: Improvement
> Components: Kafka Connector
> Reporter: Fang Yong
> Assignee: Fang Yong
> Fix For: 1.3.0
>
>
> Currently "auto.offset.reset" is used to initialize the start offset of
> FlinkKafkaConsumer, and the value should be earliest/latest/none. This method
> can only let the job comsume the beginning or the most recent data, but can
> not specify the specific offset of Kafka began to consume.
> So, there should be a configuration item (such as
> "flink.source.start.time" and the format is "yyyy-MM-dd HH:mm:ss") that
> allows user to configure the initial offset of Kafka. The action of
> "flink.source.start.time" is as follows:
> 1) job start from checkpoint / savepoint
> a> offset of partition can be restored from checkpoint/savepoint,
> "flink.source.start.time" will be ignored.
> b> there's no checkpoint/savepoint for the partition (For example, this
> partition is newly increased), the "flink.kafka.start.time" will be used to
> initialize the offset of the partition
> 2) job has no checkpoint / savepoint, the "flink.source.start.time" is used
> to initialize the offset of the kafka
> a> the "flink.source.start.time" is valid, use it to set the offset of kafka
> b> the "flink.source.start.time" is out-of-range, the same as it does
> currently with no initial offset, get kafka's current offset and start reading
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)