Skip to content

KAFKA-5075; Defer exception to the next pollOnce() if consumer's fetch position has already increased#2859

Closed
lindong28 wants to merge 5 commits intoapache:trunkfrom
lindong28:KAFKA-5075
Closed

KAFKA-5075; Defer exception to the next pollOnce() if consumer's fetch position has already increased#2859
lindong28 wants to merge 5 commits intoapache:trunkfrom
lindong28:KAFKA-5075

Conversation

@lindong28
Copy link
Copy Markdown
Member

No description provided.

@lindong28
Copy link
Copy Markdown
Member Author

In Fetcher.fetchRecords() we iterate over the partition data to collect the ConsumerRecords, after we collect some consumer records from a partition, we advance the position of that partition then move on to the next partition. If the next partition throws exceptions (e.g. OffsetOutOfRangeException), the messages that have already been read out of the buffer will not be delivered to the users. Since the positions of the previous partitions have been be updated, those messages will not be consumed again either.

This patch fixes the problem by deferring exception to the next pollOnce() if consumer's fetch position has already increased.

Ping @becketqin for review.

@asfbot
Copy link
Copy Markdown

asfbot commented Apr 16, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.10/2953/
Test FAILed (JDK 7 and Scala 2.10).

* the defaultResetPolicy is NONE
*/
public Map<TopicPartition, List<ConsumerRecord<K, V>>> fetchedRecords() {
if (nextInLineException != null) {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What if the subscription changes and the partition which had exception is not assigned anymore? Or if user seeks to a new position for the partition in question which should no longer throw exception? It seems we need to verify whether the partition state has changed or not before throw the cached exception.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good point. I have updated the patch to throw exception only if the partition is fetchable and the subscription position equals the fetched offset of the record that causes the exception.

}
}
} catch (KafkaException e) {
if (fetched.isEmpty()) {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Kafka does not use {} when the if statement has only one line.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure. Fixed now.

assertEquals(0, fetcherNoAutoReset.fetchedRecords().size());
}

@Test
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we add some simple comments. So readers can understand the test more easily?

Copy link
Copy Markdown
Member Author

@lindong28 lindong28 Apr 16, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure. I added the following comment:

"verify the advancement in the next fetch offset equals the number of fetched records when some of the fetched partition cause Exception. This ensures that consumer won't lose record upon exception"

@asfbot
Copy link
Copy Markdown

asfbot commented Apr 16, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.11/2958/
Test FAILed (JDK 8 and Scala 2.11).

@asfbot
Copy link
Copy Markdown

asfbot commented Apr 16, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/2954/
Test FAILed (JDK 8 and Scala 2.12).

@becketqin
Copy link
Copy Markdown
Contributor

@lindong28 The newly added test has failed also. Can you check?

@lindong28
Copy link
Copy Markdown
Member Author

lindong28 commented Apr 16, 2017

@becketqin I have fixed the newly added tests. I am still investigating why testPatternSubscriptionMatchingInternalTopicWithDescribeOnlyPermission fails randomly. Can you check if the patch addresses the problem you mentioned?

@asfbot
Copy link
Copy Markdown

asfbot commented Apr 16, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.11/2961/
Test FAILed (JDK 8 and Scala 2.11).

@asfbot
Copy link
Copy Markdown

asfbot commented Apr 16, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.10/2956/
Test FAILed (JDK 7 and Scala 2.10).

@asfbot
Copy link
Copy Markdown

asfbot commented Apr 16, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/2957/
Test FAILed (JDK 8 and Scala 2.12).

private final Deserializer<V> valueDeserializer;

private PartitionRecords nextInLineRecords = null;
private ExceptionMetadata nextInLineExceptionMetadata = null;
Copy link
Copy Markdown
Contributor

@becketqin becketqin Apr 16, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am wondering that would it be simpler if we just peek the first CompletedFetch in the completedFetches, and only remove it after it is parsed. The logic would be:

  1. If parseCompleted(completedFetches.peek()) did not throw exception, remove the CompletedFetch.
  2. If parseCompleted(completedFetches.peek()) threw exception and fetched.isEmpty() == false, catch the exception, return the ConsumerRecords that has already been fetched.
  3. If parseCompleted(completedFetches.peek()) threw exception and fetched.isEmpty() == true, catch the exception, remove the first entry in completedFetches and re-throw the exception.

This way we don't need to cache any result.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

But I think we still need to cache these to handle the exception thrown from Fetcher.fetchRecords(), which in turn comes from Fetcher.maybeEnsureValid(). What do you think?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

BTW, it is probably OK to cache this exception since we already cache results such as nextInLineRecords.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If an InvalidRecordException is thrown in Fetcher.fetchRecords() the position of that partition won't advance and no message for that partition would be added to fetched either. So it seems we can apply the same rule there as well?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Discussed offline. We will keep the fix as is.

@asfbot
Copy link
Copy Markdown

asfbot commented Apr 16, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.11/2962/
Test PASSed (JDK 8 and Scala 2.11).

@asfbot
Copy link
Copy Markdown

asfbot commented Apr 16, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.10/2957/
Test PASSed (JDK 7 and Scala 2.10).

@asfbot
Copy link
Copy Markdown

asfbot commented Apr 16, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/2958/
Test PASSed (JDK 8 and Scala 2.12).

@asfbot
Copy link
Copy Markdown

asfbot commented Apr 16, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.11/2963/
Test FAILed (JDK 8 and Scala 2.11).

@asfbot
Copy link
Copy Markdown

asfbot commented Apr 16, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/2959/
Test FAILed (JDK 8 and Scala 2.12).

@asfbot
Copy link
Copy Markdown

asfbot commented Apr 16, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.10/2958/
Test FAILed (JDK 7 and Scala 2.10).

@asfbot
Copy link
Copy Markdown

asfbot commented Apr 16, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.10/2959/
Test FAILed (JDK 7 and Scala 2.10).

@asfbot
Copy link
Copy Markdown

asfbot commented Apr 16, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.11/2964/
Test PASSed (JDK 8 and Scala 2.11).

@asfbot
Copy link
Copy Markdown

asfbot commented Apr 16, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/2960/
Test PASSed (JDK 8 and Scala 2.12).

@asfbot
Copy link
Copy Markdown

asfbot commented Apr 16, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.11/2967/
Test PASSed (JDK 8 and Scala 2.11).

@asfbot
Copy link
Copy Markdown

asfbot commented Apr 16, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.10/2962/
Test PASSed (JDK 7 and Scala 2.10).

@asfbot
Copy link
Copy Markdown

asfbot commented Apr 16, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/2963/
Test PASSed (JDK 8 and Scala 2.12).

try {
consumer.subscribe(Pattern.compile(".*"), new NoOpConsumerRebalanceListener)
consumeRecords(consumer)
consumer.subscribe(singletonList(kafka.common.Topic.GroupMetadataTopicName), new NoOpConsumerRebalanceListener)
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It seems the test was trying to test the pattern subscription, so we probably want to keep that unchanged.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure. It is fixed now.

assertEquals(0, fetcherNoAutoReset.fetchedRecords().size());
}

@Test
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we add a unit test about subscription change / offset seek after an exception is cached?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure. Added now.

private final Deserializer<V> valueDeserializer;

private PartitionRecords nextInLineRecords = null;
private ExceptionMetadata nextInLineExceptionMetadata = null;
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Discussed offline. We will keep the fix as is.

@asfbot
Copy link
Copy Markdown

asfbot commented Apr 17, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.11/2971/
Test PASSed (JDK 8 and Scala 2.11).

@asfbot
Copy link
Copy Markdown

asfbot commented Apr 17, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.10/2966/
Test PASSed (JDK 7 and Scala 2.10).

@asfbot
Copy link
Copy Markdown

asfbot commented Apr 17, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/2967/
Test PASSed (JDK 8 and Scala 2.12).

@asfbot
Copy link
Copy Markdown

asfbot commented Apr 17, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.11/2973/
Test PASSed (JDK 8 and Scala 2.11).

@asfbot
Copy link
Copy Markdown

asfbot commented Apr 17, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.10/2968/
Test PASSed (JDK 7 and Scala 2.10).

@becketqin
Copy link
Copy Markdown
Contributor

Thanks for the patch. LGTM.

@asfgit asfgit closed this in 17ce2a7 Apr 17, 2017
@asfbot
Copy link
Copy Markdown

asfbot commented Apr 17, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/2969/
Test PASSed (JDK 8 and Scala 2.12).

@hachikuji
Copy link
Copy Markdown
Contributor

Good catch @lindong28. Thanks for fixing!

@lindong28 lindong28 deleted the KAFKA-5075 branch April 17, 2017 19:40
@lindong28
Copy link
Copy Markdown
Member Author

@hachikuji My pleasure :)

asfgit pushed a commit that referenced this pull request Apr 18, 2017
…h position has already increased

Author: Dong Lin <lindong28@gmail.com>
Author: Dong Lin <lindong28@users.noreply.github.com>

Reviewers: Jiangjie Qin <becket.qin@gmail.com>

Closes #2859 from lindong28/KAFKA-5075

This is a backport patch for 0.10.2 after resolving the following conflict.
Conflicts:
	clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants