[
https://issues.apache.org/jira/browse/FLINK-6352?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16322119#comment-16322119
]
ASF GitHub Bot commented on FLINK-6352:
---------------------------------------
Github user tzulitai commented on the issue:
https://github.com/apache/flink/pull/3915
Hi @zjureel, I went ahead to address my own review / concerns with the
change in another PR that is based on your current work: #5282. I hope that is
okay, and would be great if you would like to review that.
The main changes are:
- We eagerly determine the timestamp offsets. `LATEST`, `EARLIEST`,
`GROUP_OFFSETS` startup modes still determines the offsets lazily, while
`TIMESTAMP` and `SPECIFIC_OFFSETS` will have actual offsets already before they
handled by the `KafkaConsumerThread`. It dawned on me that actually there is no
reason to lazily determine the offset for timestamp-based startup, since the
actual offset in the end in this case does not vary depending on when we fetch
the startup offsets.
- Don't use `Date` to define timestamp, just use Longs. The Kafka APIs
actually take long value timestamps, so I figured it would make sense that we
follow.
> 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
> Priority: Blocker
> Fix For: 1.5.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.4.14#64029)