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-6352) FlinkKafkaConsumer should support to use timestamp to set up start offset
Date Fri, 30 Jun 2017 05:33:00 GMT

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

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_r124966340
  
    --- Diff: flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
---
    @@ -697,13 +738,19 @@ protected static void initializeSubscribedPartitionsToStartOffsets(
     			int indexOfThisSubtask,
     			int numParallelSubtasks,
     			StartupMode startupMode,
    +			Date specificStartupDate,
     			Map<KafkaTopicPartition, Long> specificStartupOffsets) {
     
     		for (int i = 0; i < kafkaTopicPartitions.size(); i++) {
     			if (i % numParallelSubtasks == indexOfThisSubtask) {
    -				if (startupMode != StartupMode.SPECIFIC_OFFSETS) {
    -					subscribedPartitionsToStartOffsets.put(kafkaTopicPartitions.get(i), startupMode.getStateSentinel());
    -				} else {
    +				if (startupMode == StartupMode.SPECIFIC_TIMESTAMP) {
    +					if (specificStartupDate == null) {
    +						throw new IllegalArgumentException(
    +							"Startup mode for the consumer set to " + StartupMode.SPECIFIC_TIMESTAMP +
    +								", but no specific timestamp were specified");
    +					}
    +					subscribedPartitionsToStartOffsets.put(kafkaTopicPartitions.get(i), specificStartupDate.getTime());
    --- End diff --
    
    This is the main problem:
    following the original design pattern, it would be better to place a `KafkaTopicPartitionStateSentinel`
here instead of eagerly converting the `Date` to a specific offset. We only convert the date
to specific offsets when we're about to start consuming the partition (i.e. in `KafkaConsumer`
thread).


> 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.4.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)

Mime
View raw message