KAFKA-10866: Add metadata to ConsumerRecords#9836
Conversation
There was a problem hiding this comment.
Hey @guozhangwang , do you have time to review this?
I'll send a follow-on PR for the Streams code.
By the way, I pushed this branch to the Apache repo so that I can concurrently send the follow-on PR based on it.
| testCompile libs.junitJupiterApi | ||
| testCompile libs.junitVintageEngine | ||
| testCompile libs.mockitoCore | ||
| testCompile libs.hamcrest |
There was a problem hiding this comment.
To get access to more Matchers
There was a problem hiding this comment.
We actually removed these from clients when we migrated to JUnit 5...
There was a problem hiding this comment.
Ah, my apologies. I'll send a new PR to back this out.
| files="MemoryRecordsTest|MetricsTest|RequestResponseTest|TestSslUtils|AclAuthorizerBenchmark|MockConsumer"/> | ||
|
|
||
| <suppress checks="CyclomaticComplexity" | ||
| files="MockConsumer"/> |
There was a problem hiding this comment.
I've added a couple more branches to MockConsumer, which pushed it over the line.
| private final Map<TopicPartition, List<ConsumerRecord<K, V>>> records; | ||
| private final Map<TopicPartition, Metadata> metadata; | ||
|
|
||
| public static final class Metadata { |
There was a problem hiding this comment.
I made this a static inner class, since "metadata" is such an abstract concept. This way, the scope is clear.
| } | ||
| } | ||
|
|
||
| private static <K, V> Map<TopicPartition, Metadata> extractMetadata(final FetchedRecords<K, V> fetchedRecords) { |
There was a problem hiding this comment.
This is where we translate from the internal result container to the public one. It could be moved to a utility class, but this seemed fine.
| * @throws KafkaException if the rebalance callback throws exception | ||
| */ | ||
| private Map<TopicPartition, List<ConsumerRecord<K, V>>> pollForFetches(Timer timer) { | ||
| private FetchedRecords<K, V> pollForFetches(Timer timer) { |
There was a problem hiding this comment.
Added an internal struct so that the Fetcher can also return the desired metadata.
| if (!records.isEmpty()) { | ||
| fetched.addRecords(partition, records); | ||
| recordsRemaining -= records.size(); | ||
| } |
There was a problem hiding this comment.
Then, we get the records (note the main logic is now internal to FetchedRecords).
|
|
||
| KafkaConsumer<String, String> consumer = newConsumerNoAutoCommit(time, client, subscription, metadata); | ||
| consumer.assign(Arrays.asList(tp0, tp1)); | ||
| consumer.assign(asList(tp0, tp1)); |
There was a problem hiding this comment.
A side effect of adding static imports.
|
|
||
| ConsumerRecords<String, String> records = consumer.poll(Duration.ZERO); | ||
| assertEquals(0, records.count()); | ||
| assertThat(records.metadata(), equalTo(emptyMap())); |
There was a problem hiding this comment.
I dropped in a couple of extra assertions where it seemed appropriate. There are also some new tests farther down to really exercise the new code.
| long logFirstOffset; | ||
| long logLastOffset; |
There was a problem hiding this comment.
I considered inferring these from the sequence of other mock interactions, but it seemed more sensible to just have a way to specify them explicitly.
|
|
||
| // the first fetchedRecords() should return the first valid message | ||
| assertEquals(1, fetcher.fetchedRecords().get(tp0).size()); | ||
| assertEquals(1, fetcher.fetchedRecords().records().get(tp0).size()); |
There was a problem hiding this comment.
Just a consequence of moving the records inside of a response struct.
41900e1 to
ddef3d1
Compare
|
@vvcephei I'm starting to review this PR now (sorry for the late delay!). Could you rebase the PR a bit? |
552131c to
2e46bce
Compare
guozhangwang
left a comment
There was a problem hiding this comment.
Mostly minor comments, otherwise LGTM!
|
|
||
| final Map<TopicPartition, ConsumerRecords.Metadata> metadata = new HashMap<>(); | ||
| for (final TopicPartition partition : subscriptions.assignedPartitions()) { | ||
| if (subscriptions.hasValidPosition(partition) && beginningOffsets.containsKey(partition) && endOffsets.containsKey(partition)) { |
There was a problem hiding this comment.
Why also check beginningOffsets here?
There was a problem hiding this comment.
Ah, this is from before I removed it.
| if (fetchMetadata == null | ||
| || !fetchMetadata.position().offsetEpoch.isPresent() | ||
| || fetchPosition.offsetEpoch.isPresent() | ||
| && fetchMetadata.position().offsetEpoch.get() <= fetchPosition.offsetEpoch.get()) { |
There was a problem hiding this comment.
Interesting, why we do not want to update the metadata if epoch is stale?
There was a problem hiding this comment.
Also, would <= accept null from fetchMetadata.position().offsetEpoch.get() on left hand side?
There was a problem hiding this comment.
Ah, good catch. It looks like this was also leftover from a previous version.
I used to directly populate the returned metadata from the fetch response, but now I'm just populating the returned metadata from the subscription state, which initializeCompletedFetch has already updated.
The benefit is that we don't have to worry about cases like this, since they've already been checked.
| fetched.put(partition, newRecords); | ||
| TopicPartition partition = nextInLineFetch.partition; | ||
|
|
||
| if (subscriptions.isAssigned(partition)) { |
There was a problem hiding this comment.
Could it ever happen that this condition failed except mocking tests?
There was a problem hiding this comment.
I copied this check from fetchRecords, which says "this can happen when a rebalance happened before fetched records are returned to the consumer's poll call". I.e., it seems like it can actually happen, but a comment is called for, at least. I'll add it.
|
|
||
|
|
||
| @Test | ||
| public void testPollMetadataWithExtraPartitions() { |
There was a problem hiding this comment.
Does the test cover 1) stale epoch, 2) no prev value, cases?
There was a problem hiding this comment.
Good question. Thanks to your other comment, I've removed the stale epoch check.
By the "no prev value" case, are you referring to what happens when we get a fetch response for a partition for the first time? This is actually all we're testing here.
|
Huh, I can't get the PlaintextConsumerTest to fail locally... |
|
Made another pass on the latest commit and it LGTM. Also triggered the test again. Once it passed we can merge as-is. |
1cc0bb7 to
ffbaeab
Compare
|
Thanks, @guozhangwang ! I ran the PlaintextConsumerTest a bunch more times, and also searched through the trunk build logs. I think this PR did make the test more flaky, and I suspect the reason is that the consumer long-poll will now return "early" if we get a metadata-only fetch response. I've adjusted the test to account for this, and we'll see how the build does now. |
Got it. How about this: In We change the condition to |
|
Thanks @guozhangwang , I considered that, but I think for this work, we actually do want to return the metadata if we have fetched some. Since the semantics of All that is wrong is that this particular test expects to get records back after a single poll. Among our integration tests, this is a pretty unique expectation, so I felt good about relaxing it. Does that seem ok to you? |
|
Hey @vvcephei I was concerned about not the tests actually, it just rang to me if we should be paranoid about any side-effects for a tighter consumer loop in user code like: Before this loop is iterated say N times every second, and now it could be M times every second where M >> N but we are still returning the same number of records in every sec. If user's process logic does not have, e.g., But I also realized that the above change I made would effectively "kill" the purpose of this approach. So probably we do not have a better solution ATM without even larger API changes, and we'd just bite it and see if there's any surprises. |
|
Thanks @guozhangwang . I share that concern. It's a little different than what you pointed out, but to call attention to it: I actually didn't change the definition of To your point, it's true that And to the final point, I really doubt that any real code could have an expectation that they can just call |
|
Yup, I think we are on the same page for:
For |
ffbaeab to
bb23614
Compare
|
That last build was green, but it was a few days ago. Just rebased and pushed again to make sure I don't break the build when I merge this. |
Exposed the fetched metadata via the ConsumerRecords object as described in KIP-695
bb23614 to
fc3ec40
Compare
|
Most of those failures were known flaky tests, but one was an EasyMock error. I'm not able to repro it locally after a rebase, though. Rebased, pushed, and trying one more time to get a clean build. |
|
Flaky test failures: The most concerning one is the FetcherTest, but it's also failing on trunk. |
|
Have we verified that this doesn't cause perf regressions? It seems to have caused system test failures: https://issues.apache.org/jira/browse/KAFKA-12268 |
|
Thanks @vvcephei ! |
This reverts commit fdcf8fb.
This reverts commit fdcf8fb.
Implements KIP-695 Reverts a previous behavior change to Consumer.poll and replaces it with a new Consumer.currentLag API, which returns the client's currently cached lag. Uses this new API to implement the desired task idling semantics improvement from KIP-695. Reverts fdcf8fb / KAFKA-10866: Add metadata to ConsumerRecords (#9836) Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Guozhang Wang <guozhang@apache.org>
…he#10137) Implements KIP-695 Reverts a previous behavior change to Consumer.poll and replaces it with a new Consumer.currentLag API, which returns the client's currently cached lag. Uses this new API to implement the desired task idling semantics improvement from KIP-695. Reverts fdcf8fb / KAFKA-10866: Add metadata to ConsumerRecords (apache#9836) Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Guozhang Wang <guozhang@apache.org>
Expose fetched metadata via the ConsumerRecords
object as described in KIP-695.
Committer Checklist (excluded from commit message)