-
Notifications
You must be signed in to change notification settings - Fork 14.3k
KAFKA-10551: Add topic id support to produce request and response #15968
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: trunk
Are you sure you want to change the base?
Changes from 6 commits
d3acdf0
ddeac1b
a92abe6
afede12
a8f0c91
8c3602b
cba0a2d
27ed97b
35dba4b
916e471
c59b10c
65d4a41
646839d
29adbd3
7ed70b1
fef8fd6
1c12e48
d388d8a
a2e858c
30f0aa0
1e87719
d1b026c
c97212f
dc72f92
8b69253
c3d2c24
776375f
f7f0a04
654a967
315b6a3
ce85e4f
048f6ff
67e7bd2
270ab27
4166eb0
bed9c8e
f8830b1
d90580e
33ba2b0
92bad70
f662cf7
c57cc63
239e321
44da5ba
410fd1f
4114067
ba30756
6e281fd
900c460
b091e28
9b6a892
0ba7431
628371a
6cd5d4e
2f37078
aa811f1
f158714
cb66c85
6932bd1
54ecdf5
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,7 +17,8 @@ | |
package org.apache.kafka.common.requests; | ||
|
||
import org.apache.kafka.common.InvalidRecordException; | ||
import org.apache.kafka.common.TopicPartition; | ||
import org.apache.kafka.common.TopicIdPartition; | ||
import org.apache.kafka.common.Uuid; | ||
import org.apache.kafka.common.errors.UnsupportedCompressionTypeException; | ||
import org.apache.kafka.common.message.ProduceRequestData; | ||
import org.apache.kafka.common.message.ProduceResponseData; | ||
|
@@ -51,13 +52,20 @@ public static Builder forMagic(byte magic, ProduceRequestData data) { | |
if (magic < RecordBatch.MAGIC_VALUE_V2) { | ||
minVersion = 2; | ||
maxVersion = 2; | ||
} else if (canNotSupportTopicId(data)) { | ||
minVersion = 3; | ||
maxVersion = 11; | ||
} else { | ||
minVersion = 3; | ||
maxVersion = ApiKeys.PRODUCE.latestVersion(); | ||
} | ||
return new Builder(minVersion, maxVersion, data); | ||
} | ||
|
||
private static boolean canNotSupportTopicId(ProduceRequestData data) { | ||
return data.topicData().stream().anyMatch(d -> d.topicId() == null || d.topicId() == Uuid.ZERO_UUID); | ||
OmniaGM marked this conversation as resolved.
Show resolved
Hide resolved
|
||
} | ||
|
||
public static Builder forCurrentMagic(ProduceRequestData data) { | ||
return forMagic(RecordBatch.CURRENT_MAGIC_VALUE, data); | ||
} | ||
|
@@ -117,7 +125,7 @@ public String toString() { | |
// Care should be taken in methods that use this field. | ||
private volatile ProduceRequestData data; | ||
// the partitionSizes is lazily initialized since it is used by server-side in production. | ||
private volatile Map<TopicPartition, Integer> partitionSizes; | ||
private volatile Map<TopicIdPartition, Integer> partitionSizes; | ||
|
||
public ProduceRequest(ProduceRequestData produceRequestData, short version) { | ||
super(ApiKeys.PRODUCE, version); | ||
|
@@ -128,15 +136,15 @@ public ProduceRequest(ProduceRequestData produceRequestData, short version) { | |
} | ||
|
||
// visible for testing | ||
Map<TopicPartition, Integer> partitionSizes() { | ||
Map<TopicIdPartition, Integer> partitionSizes() { | ||
if (partitionSizes == null) { | ||
// this method may be called by different thread (see the comment on data) | ||
synchronized (this) { | ||
if (partitionSizes == null) { | ||
Map<TopicPartition, Integer> tmpPartitionSizes = new HashMap<>(); | ||
Map<TopicIdPartition, Integer> tmpPartitionSizes = new HashMap<>(); | ||
data.topicData().forEach(topicData -> | ||
topicData.partitionData().forEach(partitionData -> | ||
tmpPartitionSizes.compute(new TopicPartition(topicData.name(), partitionData.index()), | ||
tmpPartitionSizes.compute(new TopicIdPartition(topicData.topicId(), partitionData.index(), topicData.name()), | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. will we ever need the name and ID for this data structure? I know fetch had something where we pass in a map to convert IDs to names if needed. Just want to make sure folks won't use this info expecting the name to be there. If we don't think it is needed, maybe just include a comment about it. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. In the context of
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ok -- it is fine if we don't need the name. But it might be good to leave a comment or something that says the name will not be present for older versions. |
||
(ignored, previousValue) -> | ||
partitionData.records().sizeInBytes() + (previousValue == null ? 0 : previousValue)) | ||
) | ||
|
@@ -183,9 +191,9 @@ public ProduceResponse getErrorResponse(int throttleTimeMs, Throwable e) { | |
ApiError apiError = ApiError.fromThrowable(e); | ||
ProduceResponseData data = new ProduceResponseData().setThrottleTimeMs(throttleTimeMs); | ||
partitionSizes().forEach((tp, ignored) -> { | ||
ProduceResponseData.TopicProduceResponse tpr = data.responses().find(tp.topic()); | ||
ProduceResponseData.TopicProduceResponse tpr = data.responses().find(tp.topic(), tp.topicId()); | ||
if (tpr == null) { | ||
tpr = new ProduceResponseData.TopicProduceResponse().setName(tp.topic()); | ||
tpr = new ProduceResponseData.TopicProduceResponse().setName(tp.topic()).setTopicId(tp.topicId()); | ||
data.responses().add(tpr); | ||
} | ||
tpr.partitionResponses().add(new ProduceResponseData.PartitionProduceResponse() | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
What do we do if metadata has refreshed and is no longer in the metadata?
For fetch it is a bit different since we have the session logic, and can handle missing topics.
I would recommend writing through a few cases where the server and client have/don't have the topic ID to reason about the upgrade case/downgrade case/deletions/reassignments.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
UNKNOWN_TOPIC_ID
and on the retry the topic id will be updatedNOT_LEADER_OR_FOLLOWER
and then the client can retry with the right broker.upgrade case/downgrade
you refer too here Do you mean the client and broker IBP combination? If yes then some of these are covered inProduceRequestTest
andRequestResponseTest
I added two test cases to cover the first two and the producer seem to self recover on retry.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes. For the fetch request for example, there is code to make sure that all topics have IDs before we can send the fetch request. This is a bit less of an issue now, but if we have a cluster that is running on a MV < 2.8, topics will not have IDs. So when we decide which version of produce we want to send, we want to be aware of this.
Not only that, but even if the broker supports topic IDs on all topics, we also may have a case where we need to do a rolling upgrade to get the code that supports handling the latest API version. This may be less complicated for Produce since it is a client only API and doesn't rely on MV/IBP, so the apiVersions exchange between the client and the broker may be enough to ensure api compatibility.
We just want to confirm these upgrade paths are compatible since produce is the hot path and we don't want any (or at least not extended) downtime in the middle of an upgrade.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hi @jolshan sorry for taking too long to get back to you on this. I have added
ApiVersions::maxSupportedProduceVersion
to determine what the supported version supported on the cluster. Regarding the tests for these combination of old/new producer with old/new broker are all covered in the system teststests/kafkatest/tests/core/compatibility_test_new_broker_test.py
andtests/kafkatest/tests/client/client_compatibility_produce_consume_test.py
. I am trying to add another one to test the same while deleting topic and leadership changes but not sure if we really need to cover this or not as these are covered in the integration tests. WDYT?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No problem. I've also been super busy so not sure if I would have had time to review.
As long as the tests are between an older version and a newer version (ie not upgrade tests where the issue can just seem transient) then this should be ok.
Whenever I implemented the fetch change, I also did some sanity checks by looking at the logs when upgrading to see the change in request version without errors. I'd recommend doing this at least once as a manual test.
Thanks for the change and I will take a look.