You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by pnowojski <gi...@git.apache.org> on 2017/10/31 14:44:15 UTC

[GitHub] flink pull request #4928: [FLINK-7732][kafka-consumer] Do not commit to kafk...

GitHub user pnowojski opened a pull request:

    https://github.com/apache/flink/pull/4928

    [FLINK-7732][kafka-consumer] Do not commit to kafka Flink's sentinel offsets

    ## What is the purpose of the change
    
    This pull request fixes a bug in all versions of `KafkaConsumer`s, that resulted in application crash, if snapshot before `KafkaConsumer` was able to asynchronously properly initialize.
    
    ## Verifying this change
    
    This change adds additional unit test in `AbstractFetcherTest` and a check state to ensure/enforce contract that internal sentinel offsets will not be committed to Kafka. 
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): (yes / **no**)
      - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (yes / **no**)
      - The serializers: (yes / **no** / don't know)
      - The runtime per-record code paths (performance sensitive): (yes / **no** / don't know)
      - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (yes / **no** / don't know)
    
    ## Documentation
    
      - Does this pull request introduce a new feature? (yes / **no**)
      - If yes, how is the feature documented? (**not applicable** / docs / JavaDocs / not documented)
    


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/pnowojski/flink f7732

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/4928.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #4928
    
----
commit 9baea16ca69c27e5153808ce01592a47091c3131
Author: Piotr Nowojski <pi...@gmail.com>
Date:   2017-10-31T14:38:32Z

    [FLINK-7732][kafka-consumer] Do not commit to kafka Flink's sentinel offsets

----


---

[GitHub] flink pull request #4928: [FLINK-7732][kafka-consumer] Do not commit to kafk...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4928#discussion_r148226195
  
    --- Diff: flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java ---
    @@ -242,10 +243,25 @@ public void addDiscoveredPartitions(List<KafkaTopicPartition> newPartitions) thr
     	 * @param commitCallback The callback that the user should trigger when a commit request completes or fails.
     	 * @throws Exception This method forwards exceptions.
     	 */
    -	public abstract void commitInternalOffsetsToKafka(
    +	public final void commitInternalOffsetsToKafka(
    +			Map<KafkaTopicPartition, Long> offsets,
    +			@Nonnull KafkaCommitCallback commitCallback) throws Exception {
    +		// Ignore sentinels. They might appear here if snapshot has started before actual offsets values
    +		// replaced sentinels
    +		doCommitInternalOffsetsToKafka(filerOutSentinels(offsets), commitCallback);
    +	}
    +
    +	protected abstract void doCommitInternalOffsetsToKafka(
     			Map<KafkaTopicPartition, Long> offsets,
     			@Nonnull KafkaCommitCallback commitCallback) throws Exception;
     
    +	private Map<KafkaTopicPartition, Long> filerOutSentinels(Map<KafkaTopicPartition, Long> offsets) {
    --- End diff --
    
    typo: `filterOutSentinels`, missing `t`.


---

[GitHub] flink issue #4928: [FLINK-7732][kafka-consumer] Do not commit to kafka Flink...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on the issue:

    https://github.com/apache/flink/pull/4928
  
    By the way, what exactly was the error that caused the application crash in the described case?


---

[GitHub] flink pull request #4928: [FLINK-7732][kafka-consumer] Do not commit to kafk...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4928#discussion_r148204574
  
    --- Diff: flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateSentinel.java ---
    @@ -52,4 +52,7 @@
     	 */
     	public static final long GROUP_OFFSET = -915623761773L;
     
    +	public static boolean isSentinel(long offset) {
    +		return offset < 0;
    --- End diff --
    
    nit: this implementation could be a bit too broad. Could be a bit more specific by matching the static values in `KafkaTopicPartitionStateSentinel`.


---

[GitHub] flink issue #4928: [FLINK-7732][kafka-consumer] Do not commit to kafka Flink...

Posted by pnowojski <gi...@git.apache.org>.
Github user pnowojski commented on the issue:

    https://github.com/apache/flink/pull/4928
  
    Thanks!


---

[GitHub] flink pull request #4928: [FLINK-7732][kafka-consumer] Do not commit to kafk...

Posted by pnowojski <gi...@git.apache.org>.
Github user pnowojski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4928#discussion_r148221257
  
    --- Diff: flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateSentinel.java ---
    @@ -52,4 +52,7 @@
     	 */
     	public static final long GROUP_OFFSET = -915623761773L;
     
    +	public static boolean isSentinel(long offset) {
    +		return offset < 0;
    --- End diff --
    
    Kafka doesn't allow to commit any negative values 


---

[GitHub] flink pull request #4928: [FLINK-7732][kafka-consumer] Do not commit to kafk...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4928#discussion_r148204398
  
    --- Diff: flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java ---
    @@ -95,6 +95,10 @@ public final boolean isOffsetDefined() {
     		return offset != KafkaTopicPartitionStateSentinel.OFFSET_NOT_SET;
     	}
     
    +	public final boolean isSentinel() {
    --- End diff --
    
    nit: would `hasSentinelOffset` be a better name here?


---

[GitHub] flink pull request #4928: [FLINK-7732][kafka-consumer] Do not commit to kafk...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/flink/pull/4928


---

[GitHub] flink issue #4928: [FLINK-7732][kafka-consumer] Do not commit to kafka Flink...

Posted by pnowojski <gi...@git.apache.org>.
Github user pnowojski commented on the issue:

    https://github.com/apache/flink/pull/4928
  
    @tzulitai please check the details in the ticket: https://issues.apache.org/jira/browse/FLINK-7732
    
    I have changed the approach as we discussed and now we filtering out happens just before committing offsets.


---

[GitHub] flink issue #4928: [FLINK-7732][kafka-consumer] Do not commit to kafka Flink...

Posted by pnowojski <gi...@git.apache.org>.
Github user pnowojski commented on the issue:

    https://github.com/apache/flink/pull/4928
  
    @tzulitai @aljoscha could you take a look?


---

[GitHub] flink issue #4928: [FLINK-7732][kafka-consumer] Do not commit to kafka Flink...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on the issue:

    https://github.com/apache/flink/pull/4928
  
    Travis passes, merging ...


---

[GitHub] flink pull request #4928: [FLINK-7732][kafka-consumer] Do not commit to kafk...

Posted by pnowojski <gi...@git.apache.org>.
Github user pnowojski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4928#discussion_r148220943
  
    --- Diff: flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java ---
    @@ -95,6 +95,10 @@ public final boolean isOffsetDefined() {
     		return offset != KafkaTopicPartitionStateSentinel.OFFSET_NOT_SET;
     	}
     
    +	public final boolean isSentinel() {
    --- End diff --
    
    In new approach method was dropped


---