Skip to content

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

Merged
merged 67 commits into from
Apr 29, 2025

Conversation

OmniaGM
Copy link
Contributor

@OmniaGM OmniaGM commented May 15, 2024

  • Add support topicId in ProduceRequest/ProduceResponse. Topic name
    and Topic Id will become ignorable following the footstep of
    FetchRequest/FetchResponse
  • ReplicaManager still look for HostedPartition using TopicPartition
    and doesn't check topic id. This is an [OPEN QUESTION] if we should
    address this in this pr or wait for
    KAFKA-16212 as this
    will update ReplicaManager::getPartition to use TopicIdParittion
    once we update the cache. Other option is that we compare provided
    topicId with Partition topic id and return UNKNOW_TOPIC_ID or
    UNKNOW_TOPIC_PARTITION if we can't find partition with matched topic
    id.

Reviewers: Jun Rao [email protected], Justine Olshan
[email protected]

@OmniaGM OmniaGM marked this pull request as draft May 15, 2024 15:53
@OmniaGM OmniaGM force-pushed the KAFKA-10551-produceRequest branch from 1805c4f to a8f0c91 Compare May 15, 2024 22:05
@OmniaGM OmniaGM marked this pull request as ready for review May 15, 2024 22:58
@OmniaGM OmniaGM changed the title Kafka-10551: Add topic id support to produce request and response KAFKA-10551: Add topic id support to produce request and response May 15, 2024
@OmniaGM
Copy link
Contributor Author

OmniaGM commented May 16, 2024

Few of the failed tests are related to this change and am working on fixing them

@OmniaGM OmniaGM force-pushed the KAFKA-10551-produceRequest branch from 63a6032 to 8c3602b Compare May 16, 2024 15:06
@OmniaGM
Copy link
Contributor Author

OmniaGM commented May 20, 2024

Few of the failed tests are related to this change and am working on fixing them

I believe that failed tests now are unrelated

@jolshan
Copy link
Member

jolshan commented May 20, 2024

Topic name and Topic Id will become optional following the footstep of FetchRequest/FetchResponse

My understanding is that all requests going forward will use ID and not name similar to fetch request. I believe that is what is in the PR, but the comment suggests otherwise.

@@ -610,7 +611,9 @@ private void handleProduceResponse(ClientResponse response, Map<TopicPartition,
// This will be set by completeBatch.
Map<TopicPartition, Metadata.LeaderIdAndEpoch> partitionsWithUpdatedLeaderInfo = new HashMap<>();
produceResponse.data().responses().forEach(r -> r.partitionResponses().forEach(p -> {
TopicPartition tp = new TopicPartition(r.name(), p.index());
// Version 12 drop topic name and add support to topic id. However, metadata can be used to map topic id to topic name.
String topicName = (r.name() == null || r.name().isEmpty()) ? metadata.topicNames().get(r.topicId()) : r.name();
Copy link
Member

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.

Copy link
Contributor Author

@OmniaGM OmniaGM May 28, 2024

Choose a reason for hiding this comment

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

  • If topic has been recreated and topic id is out of date, the client will get UNKNOWN_TOPIC_ID and on the retry the topic id will be updated
  • If topic has been reassigned to another broker then the client will get NOT_LEADER_OR_FOLLOWER and then the client can retry with the right broker.
  • Am not sure what upgrade case/downgrade you refer too here Do you mean the client and broker IBP combination? If yes then some of these are covered in ProduceRequestTest and RequestResponseTest

I added two test cases to cover the first two and the producer seem to self recover on retry.

Copy link
Member

@jolshan jolshan May 31, 2024

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.

Copy link
Contributor Author

@OmniaGM OmniaGM Jun 11, 2024

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 tests tests/kafkatest/tests/core/compatibility_test_new_broker_test.py and tests/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?

Copy link
Member

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.

@@ -1361,10 +1366,10 @@ class ReplicaManager(val config: KafkaConfig,
*/
private def appendToLocalLog(internalTopicsAllowed: Boolean,
origin: AppendOrigin,
entriesPerPartition: Map[TopicPartition, MemoryRecords],
entriesPerPartition: Map[TopicIdPartition, MemoryRecords],
Copy link
Member

Choose a reason for hiding this comment

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

is there a reason to pass this data structure here if we are not using the ID to check the append at the log level?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

two reasons here

  1. I didn't want to keep convert between TopicIdPartitions to TopicPartition
  2. KAFKA-16212 will eventually use TopicIdPartitions to getPartitionOrException

Copy link
Member

Choose a reason for hiding this comment

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

Ok -- once we start using these across the log layer it makes sense.

Copy link
Member

Choose a reason for hiding this comment

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

Did we end up doing the getPartitionOrException in this PR?

Copy link
Contributor

Choose a reason for hiding this comment

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

We introduced getPartitionOrException(topicIdPartition: TopicIdPartition) that verifies the topicId if provided.

@jolshan
Copy link
Member

jolshan commented May 20, 2024

I would recommend taking a look at where we are passing the topic ID through and the checks we do. If we think it is useful to ensure we are writing to the right topic, we should do it, but if it is just adding complexity, we may want to consider changing.

@OmniaGM
Copy link
Contributor Author

OmniaGM commented May 22, 2024

Topic name and Topic Id will become optional following the footstep of FetchRequest/FetchResponse

My understanding is that all requests going forward will use ID and not name similar to fetch request. I believe that is what is in the PR, but the comment suggests otherwise.

I meant that in Json files both will be marked ignorable

@OmniaGM OmniaGM force-pushed the KAFKA-10551-produceRequest branch from 8daeb62 to 1abc2ac Compare May 28, 2024 15:43
@OmniaGM OmniaGM force-pushed the KAFKA-10551-produceRequest branch from 1abc2ac to 35dba4b Compare May 28, 2024 16:02
@OmniaGM OmniaGM force-pushed the KAFKA-10551-produceRequest branch from 79814ff to 916e471 Compare June 11, 2024 21:22
@OmniaGM OmniaGM force-pushed the KAFKA-10551-produceRequest branch from 86c1f71 to c59b10c Compare June 11, 2024 21:25
Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

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

@OmniaGM : Thanks for the updated PR. A few more comments.

Copy link
Member

@jolshan jolshan left a comment

Choose a reason for hiding this comment

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

Thanks Omnia for your hard work!

Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

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

@OmniaGM : Thanks for the updated PR. A couple of more comments.

Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

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

@OmniaGM : Thanks for the updated PR. One more comment. Also, could you rebase?

@OmniaGM OmniaGM force-pushed the KAFKA-10551-produceRequest branch from 43e93b4 to 0c3e509 Compare April 29, 2025 09:01
@OmniaGM OmniaGM force-pushed the KAFKA-10551-produceRequest branch from 0c3e509 to 9abc966 Compare April 29, 2025 14:34
Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

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

@OmniaGM : Thanks for the updated PR. The code LGTM. Is the test failure PlaintextConsumerTest related? If not, perhaps you could rebase to trigger another run of the test.

@jolshan
Copy link
Member

jolshan commented Apr 29, 2025

I triggered the rerun 👍

@jolshan
Copy link
Member

jolshan commented Apr 29, 2025

@junrao build is green now! I noticed you didn't hit the approve button on GH, so I will wait to merge just to double check.

Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

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

@OmniaGM : Thanks for addressing all the comments. LGTM

@jolshan : Feel free to merge it.

@jolshan jolshan merged commit 6f783f8 into apache:trunk Apr 29, 2025
26 checks passed
@OmniaGM
Copy link
Contributor Author

OmniaGM commented Apr 30, 2025

@OmniaGM : Thanks for addressing all the comments. LGTM

@jolshan : Feel free to merge it.

Thanks for reviewing this and merging. It is finally merged before hitting exactly 1 year :D

if (topicName.isEmpty)
nonExistingTopicResponses += new TopicIdPartition(topicId, topicPartition) -> new PartitionResponse(Errors.UNKNOWN_TOPIC_ID)
else if (!metadataCache.contains(topicPartition))
nonExistingTopicResponses += new TopicIdPartition(topicId, topicPartition) -> new PartitionResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION)
Copy link
Contributor

Choose a reason for hiding this comment

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

This seems to be a regression. Since topic name is sensitive, if the topic name doesn't exist and is not authorized, we should return TOPIC_AUTHORIZATION_FAILED, instead of UNKNOWN_TOPIC_OR_PARTITION. Filed https://issues.apache.org/jira/browse/KAFKA-19234 to track it.

Copy link
Member

Choose a reason for hiding this comment

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

Hmm -- I recall some discussion around this for fetch requests...Is it different for produce?

Copy link
Member

Choose a reason for hiding this comment

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

Copy link
Member

Choose a reason for hiding this comment

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

Its funny because you can re-arrange the numbers to get the jira number 😂

Copy link
Member

Choose a reason for hiding this comment

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

I also see that this is a slightly different case, so makes sense. 👍

Copy link
Contributor

Choose a reason for hiding this comment

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

It seems that the Fetch request does the right thing. If an old version of the fetch request includes topic name that doesn't exist, it will go through the authorization check first.

      // Regular Kafka consumers need READ permission on each partition they are fetching.
      val partitionDatas = new mutable.ArrayBuffer[(TopicIdPartition, FetchRequest.PartitionData)]
      fetchContext.foreachPartition { (topicIdPartition, partitionData) =>
        if (topicIdPartition.topic == null)
          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_ID)
        else
          partitionDatas += topicIdPartition -> partitionData
      }
      val authorizedTopics = authHelper.filterByAuthorized(request.context, READ, TOPIC, partitionDatas)(_._1.topicPartition.topic)
      partitionDatas.foreach { case (topicIdPartition, data) =>
        if (!authorizedTopics.contains(topicIdPartition.topic))
          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.TOPIC_AUTHORIZATION_FAILED)
        else if (!metadataCache.contains(topicIdPartition.topicPartition))
          erroneous += topicIdPartition -> FetchResponse.partitionResponse(topicIdPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION)

shmily7829 pushed a commit to shmily7829/kafka that referenced this pull request May 7, 2025
…ache#15968)

- Add support topicId in `ProduceRequest`/`ProduceResponse`. Topic name
and Topic Id will become `ignorable` following the footstep of
`FetchRequest`/`FetchResponse`
- ReplicaManager still look for `HostedPartition` using `TopicPartition`
and doesn't check topic id. This is an **[OPEN QUESTION]** if we should
address this in this pr or wait for
[KAFKA-16212](https://issues.apache.org/jira/browse/KAFKA-16212) as this
will update `ReplicaManager::getPartition` to use `TopicIdParittion`
once we update the cache. Other option is that we compare provided
`topicId` with `Partition` topic id and return `UNKNOW_TOPIC_ID` or
`UNKNOW_TOPIC_PARTITION` if we can't find partition with matched topic
id.

Reviewers: Jun Rao <[email protected]>, Justine Olshan
 <[email protected]>
@lucasbru
Copy link
Member

lucasbru commented Jun 3, 2025

@OmniaGM Hey, I detected the following NPE during Kafka Streams soak testing (on trunk):

[2025-05-29 04:06:05,855] ERROR [kafka-producer-network-thread | i-07bbab180f6062ba3-StreamThread-3-producer] [Producer clientId=i-07bbab180f6062ba3-StreamThread-3-producer] Uncaught error in request completion: (org.apache.kafka.clients.NetworkClient)
java.lang.NullPointerException: Cannot read field "topicPartition" because "batch" is null
	at org.apache.kafka.clients.producer.internals.TransactionManager.handleCompletedBatch(TransactionManager.java:748)
	at org.apache.kafka.clients.producer.internals.Sender.completeBatch(Sender.java:736)
	at org.apache.kafka.clients.producer.internals.Sender.completeBatch(Sender.java:710)
	at org.apache.kafka.clients.producer.internals.Sender.lambda$handleProduceResponse$2(Sender.java:613)
	at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
	at org.apache.kafka.clients.producer.internals.Sender.lambda$handleProduceResponse$3(Sender.java:597)
	at java.base/java.lang.Iterable.forEach(Iterable.java:75)
	at org.apache.kafka.clients.producer.internals.Sender.handleProduceResponse(Sender.java:597)
	at org.apache.kafka.clients.producer.internals.Sender.lambda$sendProduceRequest$9(Sender.java:895)
	at org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:154)
	at org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:669)
	at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:661)
	at org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:340)
	at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:242)
	at java.base/java.lang.Thread.run(Thread.java:840)

It seems likely to me that this may be caused by this PR, given the changes in Sender.handleProduceResponse. Could be a race-condition around topic metadata, but I don't understand the details well enough to fix it myself. Could you please take a look?

cc @jolshan

@OmniaGM
Copy link
Contributor Author

OmniaGM commented Jun 10, 2025

@OmniaGM Hey, I detected the following NPE during Kafka Streams soak testing (on trunk):


[2025-05-29 04:06:05,855] ERROR [kafka-producer-network-thread | i-07bbab180f6062ba3-StreamThread-3-producer] [Producer clientId=i-07bbab180f6062ba3-StreamThread-3-producer] Uncaught error in request completion: (org.apache.kafka.clients.NetworkClient)

java.lang.NullPointerException: Cannot read field "topicPartition" because "batch" is null

	at org.apache.kafka.clients.producer.internals.TransactionManager.handleCompletedBatch(TransactionManager.java:748)

	at org.apache.kafka.clients.producer.internals.Sender.completeBatch(Sender.java:736)

	at org.apache.kafka.clients.producer.internals.Sender.completeBatch(Sender.java:710)

	at org.apache.kafka.clients.producer.internals.Sender.lambda$handleProduceResponse$2(Sender.java:613)

	at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)

	at org.apache.kafka.clients.producer.internals.Sender.lambda$handleProduceResponse$3(Sender.java:597)

	at java.base/java.lang.Iterable.forEach(Iterable.java:75)

	at org.apache.kafka.clients.producer.internals.Sender.handleProduceResponse(Sender.java:597)

	at org.apache.kafka.clients.producer.internals.Sender.lambda$sendProduceRequest$9(Sender.java:895)

	at org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:154)

	at org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:669)

	at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:661)

	at org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:340)

	at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:242)

	at java.base/java.lang.Thread.run(Thread.java:840)

It seems likely to me that this may be caused by this PR, given the changes in Sender.handleProduceResponse. Could be a race-condition around topic metadata, but I don't understand the details well enough to fix it myself. Could you please take a look?

cc @jolshan

Hi can you please share which test can I run to reproduce this? I will have a look this week.

@lucasbru
Copy link
Member

@OmniaGM . This is being triggered in a long-running test, so I don't really have a directly reproducable test. However, I have DEBUG logs, which show the METADATA / PRODUCE requests / responses involved, attached to the ticket here:

https://issues.apache.org/jira/browse/KAFKA-19397

This should help with the debugging.

@junrao
Copy link
Contributor

junrao commented Jun 10, 2025

@lucasbru : Thanks for reporting this issue. Does the test delete and recreate the same topic?

@OmniaGM : If a topic is recreated with a different topic id while processing a produce response, the following code may not handle it well. topicName will be empty if the topic id has changed in the metadata and batch will be null.

                    // Version 13 drop topic name and add support to topic id. However, metadata can be used to map topic id to topic name.
                    String topicName = metadata.topicNames().getOrDefault(r.topicId(), r.name());
                    TopicPartition tp = new TopicPartition(topicName, p.index());
                    ProduceResponse.PartitionResponse partResp = new ProduceResponse.PartitionResponse(
                            Errors.forCode(p.errorCode()),
                            p.baseOffset(),
                            p.logAppendTimeMs(),
                            p.logStartOffset(),
                            p.recordErrors()
                                .stream()
                                .map(e -> new ProduceResponse.RecordError(e.batchIndex(), e.batchIndexErrorMessage()))
                                .collect(Collectors.toList()),
                            p.errorMessage(),
                            p.currentLeader());
                    ProducerBatch batch = batches.get(tp);

I am wondering if we should include topicId in recordsByPartition in sendProduceRequest(). This way, we don't need to check metadata for topicId since it could change over time.

@OmniaGM
Copy link
Contributor Author

OmniaGM commented Jun 10, 2025

@lucasbru : Thanks for reporting this issue. Does the test delete and recreate the same topic?

@OmniaGM : If a topic is recreated with a different topic id while processing a produce response, the following code may not handle it well. topicName will be empty if the topic id has changed in the metadata and batch will be null.

                    // Version 13 drop topic name and add support to topic id. However, metadata can be used to map topic id to topic name.
                    String topicName = metadata.topicNames().getOrDefault(r.topicId(), r.name());
                    TopicPartition tp = new TopicPartition(topicName, p.index());
                    ProduceResponse.PartitionResponse partResp = new ProduceResponse.PartitionResponse(
                            Errors.forCode(p.errorCode()),
                            p.baseOffset(),
                            p.logAppendTimeMs(),
                            p.logStartOffset(),
                            p.recordErrors()
                                .stream()
                                .map(e -> new ProduceResponse.RecordError(e.batchIndex(), e.batchIndexErrorMessage()))
                                .collect(Collectors.toList()),
                            p.errorMessage(),
                            p.currentLeader());
                    ProducerBatch batch = batches.get(tp);

I am wondering if we should include topicId in recordsByPartition in sendProduceRequest(). This way, we don't need to check metadata for topicId since it could change over time.

This was same thinking I had. Will raise a PR for this

@chia7712
Copy link
Member

Does the test delete and recreate the same topic?

I write a rough test according to this comment, and it does reproduce the NPE.

    public void test(ClusterInstance clusterInstance) {
        var topic = "chia";
        var f = CompletableFuture.runAsync(() -> {
            try (var admin = clusterInstance.admin()) {
                for (int i = 1; i <= 1000; i++) {
                    try {
                        admin.deleteTopics(List.of(topic)).all().get();
                    } catch (Exception ignore) {
                    }
                    try {
                        admin.createTopics(List.of(new NewTopic(topic, 10, (short) 1))).all().get();
                    } catch (Exception ignore) {
                    }
                }
            }
        });
        try {
            var fs = IntStream.range(0, 10).mapToObj(ignored -> CompletableFuture.runAsync(() -> {
                try (var producer = clusterInstance.producer()) {
                    for (int i = 1; i <= 1000; i++) {
                        producer.send(new ProducerRecord<>(topic, "value".getBytes()));
                    }
                }
            })).toList();
            fs.forEach(CompletableFuture::join);
        } finally {
            f.join();
        }
    }
[2025-06-11 02:08:07,497] ERROR [Producer clientId=producer-5] Uncaught error in request completion: (org.apache.kafka.clients.NetworkClient:671)
java.lang.NullPointerException: Cannot invoke "org.apache.kafka.clients.producer.internals.ProducerBatch.hasReachedDeliveryTimeout(long, long)" because "batch" is null
	at org.apache.kafka.clients.producer.internals.Sender.canRetry(Sender.java:835) ~[kafka-clients-4.1.0-SNAPSHOT.jar:?]
	at org.apache.kafka.clients.producer.internals.Sender.completeBatch(Sender.java:669) ~[kafka-clients-4.1.0-SNAPSHOT.jar:?]
	at org.apache.kafka.clients.producer.internals.Sender.lambda$handleProduceResponse$2(Sender.java:613) ~[kafka-clients-4.1.0-SNAPSHOT.jar:?]
	at java.base/java.util.ArrayList.forEach(ArrayList.java:1596) ~[?:?]
	at org.apache.kafka.clients.producer.internals.Sender.lambda$handleProduceResponse$3(Sender.java:597) ~[kafka-clients-4.1.0-SNAPSHOT.jar:?]
	at java.base/java.lang.Iterable.forEach(Iterable.java:75) ~[?:?]
	at org.apache.kafka.clients.producer.internals.Sender.handleProduceResponse(Sender.java:597) ~[kafka-clients-4.1.0-SNAPSHOT.jar:?]
	at org.apache.kafka.clients.producer.internals.Sender.lambda$sendProduceRequest$9(Sender.java:895) ~[kafka-clients-4.1.0-SNAPSHOT.jar:?]
	at org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:154) ~[kafka-clients-4.1.0-SNAPSHOT.jar:?]
	at org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:669) ~[kafka-clients-4.1.0-SNAPSHOT.jar:?]
	at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:661) ~[kafka-clients-4.1.0-SNAPSHOT.jar:?]
	at org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:340) ~[kafka-clients-4.1.0-SNAPSHOT.jar:?]
	at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:255) ~[kafka-clients-4.1.0-SNAPSHOT.jar:?]
	at java.base/java.lang.Thread.run(Thread.java:1583) [?:?]

@lucasbru
Copy link
Member

@OmniaGM I don't think the test recreates topics. What I have seen is that sometimes a rebootstrap of the producer happened around the same time. But indeed, if we can avoid the additional lookup in the metadata, this may solve the problem. I can restart the streams soak once your fix is merged, and we should see if the problem reappears.

@junrao
Copy link
Contributor

junrao commented Jun 11, 2025

@lucasbru : Thanks for the explanation. During rebootstrap, it's possible that the metadata is empty when an inflight response is being processed. So, this has the same effect of a topic being deleted.

@OmniaGM
Copy link
Contributor Author

OmniaGM commented Jun 13, 2025

I am wondering if we should include topicId in recordsByPartition in sendProduceRequest(). This way, we don't need to check metadata for topicId since it could change over time.

I have been looking into replacing this

String topicName = metadata.topicNames().getOrDefault(r.topicId(), r.name());
                    TopicPartition tp = new TopicPartition(topicName, p.index());

which is used later to batches.get(tp) to the following TopicIdPartition which I will be sending in recordsByPartitioninstead of TopicPartition. The challenge am hitting now is the the response r in this method already doesn't have topic name and only have topic id.

@OmniaGM
Copy link
Contributor Author

OmniaGM commented Jun 13, 2025

I have raised this #19964 pr, trying to update ProducerSendWhileDeletionTest to cover this case better now.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

7 participants