kafka icon indicating copy to clipboard operation
kafka copied to clipboard

KAFKA-19397: Not relaying on metadata to map between topic id and name.

Open OmniaGM opened this issue 5 months ago • 3 comments

  • Metadata doesn't have the full view of topicNames to ids during rebootstrap of client or when topic has been deleted/recreated. The solution is to pass down topic id and stop trying to figure it out later in the logic.

OmniaGM avatar Jun 13 '25 14:06 OmniaGM

@junrao I redeployed the soak with the fix. I will report back if the problem reoccurs

lucasbru avatar Jun 16 '25 09:06 lucasbru

@OmniaGM @junrao I have been running the soak for 24h, and it's looking good.

lucasbru avatar Jun 17 '25 08:06 lucasbru

I have added some test in ProducerSendWhileDeletionTest to cover recreation while producing as well hope this will be enough to cover these cases.

OmniaGM avatar Jun 17 '25 16:06 OmniaGM

@OmniaGM could you please fix the conflicts?

chia7712 avatar Jun 22 '25 19:06 chia7712

Also, regarding the title of the PR. "Not relaying on metadata to map between topic id and name".

We are still relying on the metadata to map topic id and name. We just want to use consistent metadata between generating the produce request and handing the produce response.

junrao avatar Jun 23 '25 21:06 junrao

I will run perf test one more time this time with some topic recreation in the background while am waiting for the pipline to finish to ensure everything is okay.

OmniaGM avatar Jun 27 '25 11:06 OmniaGM

last perf tests results 5000000 records sent, 2378.4 records/sec (2.32 MB/sec), 12647.78 ms avg latency, 55251.00 ms max latency, 11555 ms 50th, 25864 ms 95th, 32131 ms 99th, 39096 ms 99.9th.

OmniaGM avatar Jun 27 '25 12:06 OmniaGM

Will wait for @lucasbru confirmation before merging this

OmniaGM avatar Jun 29 '25 20:06 OmniaGM

I deployed the test again with this change. I think running it for 24 hours should give us reasonable confidence that the problem does not reappear. If we are fairly confident that the fix is still valid, we could merge it and worst case block the first RC on this.

lucasbru avatar Jun 30 '25 09:06 lucasbru

Unfortunately, this version again seems to be triggering a problem with producers getting stuck during flushing. We are running with Kafka commitId 405d1b7ed8b6c452 which should be the latest version of this PR.

In the thread dump, I see producers being stuck here:

   java.lang.Thread.State: WAITING (parking)
	at jdk.internal.misc.Unsafe.park([email protected]/Native Method)
	- parking to wait for  <0x000000072a34ed28> (a java.util.concurrent.CountDownLatch$Sync)
	at java.util.concurrent.locks.LockSupport.park([email protected]/LockSupport.java:211)
	at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire([email protected]/AbstractQueuedSynchronizer.java:715)
	at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly([email protected]/AbstractQueuedSynchronizer.java:1047)
	at java.util.concurrent.CountDownLatch.await([email protected]/CountDownLatch.java:230)
	at org.apache.kafka.clients.producer.internals.ProduceRequestResult.await(ProduceRequestResult.java:76)
	at org.apache.kafka.clients.producer.internals.RecordAccumulator.awaitFlushCompletion(RecordAccumulator.java:1075)
	at org.apache.kafka.clients.producer.KafkaProducer.flush(KafkaProducer.java:1325)
	at ...

The last sign of life from the producer is this:

soak_44.244.234.161/streams.log.2025-06-30-22:59:[2025-06-30 22:50:32,044] ERROR [kafka-producer-network-thread | i-0a00080f294c267cf-StreamThread-3-producer] [Producer clientId=i-0a00080f294c267cf-StreamThread-3-producer] Uncaught error in request completion: (org.apache.kafka.clients.NetworkClient)
soak_44.244.234.161/streams.log.2025-06-30-22:59-java.lang.IllegalStateException: batch created for -1 can't be found.
soak_44.244.234.161/streams.log.2025-06-30-22:59-	at org.apache.kafka.clients.producer.internals.Sender.lambda$handleProduceResponse$2(Sender.java:620)
soak_44.244.234.161/streams.log.2025-06-30-22:59-	at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
soak_44.244.234.161/streams.log.2025-06-30-22:59-	at org.apache.kafka.clients.producer.internals.Sender.lambda$handleProduceResponse$3(Sender.java:597)
soak_44.244.234.161/streams.log.2025-06-30-22:59-	at java.base/java.lang.Iterable.forEach(Iterable.java:75)
soak_44.244.234.161/streams.log.2025-06-30-22:59-	at org.apache.kafka.clients.producer.internals.Sender.handleProduceResponse(Sender.java:597)
soak_44.244.234.161/streams.log.2025-06-30-22:59-	at org.apache.kafka.clients.producer.internals.Sender.lambda$sendProduceRequest$9(Sender.java:904)
soak_44.244.234.161/streams.log.2025-06-30-22:59-	at org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:154)
soak_44.244.234.161/streams.log.2025-06-30-22:59-	at org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:669)
soak_44.244.234.161/streams.log.2025-06-30-22:59-	at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:661)
soak_44.244.234.161/streams.log.2025-06-30-22:59-	at org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:340)
soak_44.244.234.161/streams.log.2025-06-30-22:59-	at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:242)
soak_44.244.234.161/streams.log.2025-06-30-22:59-	at java.base/java.lang.Thread.run(Thread.java:840)

lucasbru avatar Jul 01 '25 09:07 lucasbru

Unfortunately, this version again seems to be triggering a problem with producers getting stuck during flushing. We are running with Kafka commitId 405d1b7ed8b6c452 which should be the latest version of this PR.

In the thread dump, I see producers being stuck here:

   java.lang.Thread.State: WAITING (parking)
	at jdk.internal.misc.Unsafe.park([email protected]/Native Method)
	- parking to wait for  <0x000000072a34ed28> (a java.util.concurrent.CountDownLatch$Sync)
	at java.util.concurrent.locks.LockSupport.park([email protected]/LockSupport.java:211)
	at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire([email protected]/AbstractQueuedSynchronizer.java:715)
	at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly([email protected]/AbstractQueuedSynchronizer.java:1047)
	at java.util.concurrent.CountDownLatch.await([email protected]/CountDownLatch.java:230)
	at org.apache.kafka.clients.producer.internals.ProduceRequestResult.await(ProduceRequestResult.java:76)
	at org.apache.kafka.clients.producer.internals.RecordAccumulator.awaitFlushCompletion(RecordAccumulator.java:1075)
	at org.apache.kafka.clients.producer.KafkaProducer.flush(KafkaProducer.java:1325)
	at ...

The last sign of life from the producer is this:

soak_44.244.234.161/streams.log.2025-06-30-22:59:[2025-06-30 22:50:32,044] ERROR [kafka-producer-network-thread | i-0a00080f294c267cf-StreamThread-3-producer] [Producer clientId=i-0a00080f294c267cf-StreamThread-3-producer] Uncaught error in request completion: (org.apache.kafka.clients.NetworkClient)
soak_44.244.234.161/streams.log.2025-06-30-22:59-java.lang.IllegalStateException: batch created for -1 can't be found.
soak_44.244.234.161/streams.log.2025-06-30-22:59-	at org.apache.kafka.clients.producer.internals.Sender.lambda$handleProduceResponse$2(Sender.java:620)
soak_44.244.234.161/streams.log.2025-06-30-22:59-	at java.base/java.util.ArrayList.forEach(ArrayList.java:1511)
soak_44.244.234.161/streams.log.2025-06-30-22:59-	at org.apache.kafka.clients.producer.internals.Sender.lambda$handleProduceResponse$3(Sender.java:597)
soak_44.244.234.161/streams.log.2025-06-30-22:59-	at java.base/java.lang.Iterable.forEach(Iterable.java:75)
soak_44.244.234.161/streams.log.2025-06-30-22:59-	at org.apache.kafka.clients.producer.internals.Sender.handleProduceResponse(Sender.java:597)
soak_44.244.234.161/streams.log.2025-06-30-22:59-	at org.apache.kafka.clients.producer.internals.Sender.lambda$sendProduceRequest$9(Sender.java:904)
soak_44.244.234.161/streams.log.2025-06-30-22:59-	at org.apache.kafka.clients.ClientResponse.onComplete(ClientResponse.java:154)
soak_44.244.234.161/streams.log.2025-06-30-22:59-	at org.apache.kafka.clients.NetworkClient.completeResponses(NetworkClient.java:669)
soak_44.244.234.161/streams.log.2025-06-30-22:59-	at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:661)
soak_44.244.234.161/streams.log.2025-06-30-22:59-	at org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:340)
soak_44.244.234.161/streams.log.2025-06-30-22:59-	at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:242)
soak_44.244.234.161/streams.log.2025-06-30-22:59-	at java.base/java.lang.Thread.run(Thread.java:840)

This is basically same state as NPE but with IllegalStateException where tooic name is empty and topic id not part of initial metadata passed down from sendProduceRequest

OmniaGM avatar Jul 01 '25 15:07 OmniaGM

@lucasbru can you give this another go please?

OmniaGM avatar Jul 01 '25 16:07 OmniaGM

I am soaking commit a5644e92f72e8fd8. In the first 12 hours, the problem did not reappear.

lucasbru avatar Jul 02 '25 09:07 lucasbru

@OmniaGM Still looking good - problem seems to be resovled, but the integration test doesn't seem to pass now.

lucasbru avatar Jul 02 '25 16:07 lucasbru

@lucasbru : Thanks for the update. As for the integration test failure. I saw the following locally.

Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-13068287464958241505/controller_3000 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-13068287464958241505/broker_1_data0 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-13068287464958241505/broker_0_data0 with metadata.version 4.2-IV1.
[2025-07-02 09:49:51,304] WARN [QuorumController id=3000] Performing controller activation. The metadata log appears to be empty. Appending 6 bootstrap record(s) in metadata transaction at metadata.version 4.2-IV1 from bootstrap source 'testkit'. (org.apache.kafka.controller.QuorumController:106)
[2025-07-02 09:49:51,469] WARN [Producer clientId=producer-2] The metadata response from the cluster reported a recoverable issue with correlation id 1 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-02 09:49:51,677] WARN [Producer clientId=producer-2] The metadata response from the cluster reported a recoverable issue with correlation id 7 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-02 09:49:52,372] WARN [NodeToControllerChannelManager id=3000 name=registration] Attempting to close NetworkClient that has already been closed. (org.apache.kafka.clients.NetworkClient:744)
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-14231399981533981299/controller_3000 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-14231399981533981299/broker_1_data0 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-14231399981533981299/broker_0_data0 with metadata.version 4.2-IV1.
[2025-07-02 09:49:52,491] WARN [QuorumController id=3000] Performing controller activation. The metadata log appears to be empty. Appending 6 bootstrap record(s) in metadata transaction at metadata.version 4.2-IV1 from bootstrap source 'testkit'. (org.apache.kafka.controller.QuorumController:106)
[2025-07-02 09:49:52,648] WARN [Producer clientId=producer-3] The metadata response from the cluster reported a recoverable issue with correlation id 1 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-02 09:49:52,858] WARN [Producer clientId=producer-3] The metadata response from the cluster reported a recoverable issue with correlation id 8 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-02 09:49:53,568] WARN [NodeToControllerChannelManager id=3000 name=registration] Attempting to close NetworkClient that has already been closed. (org.apache.kafka.clients.NetworkClient:744)
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-6584854305397606511/broker_0_data0 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-6584854305397606511/controller_3000 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-6584854305397606511/broker_1_data0 with metadata.version 4.2-IV1.
[2025-07-02 09:49:53,671] WARN [QuorumController id=3000] Performing controller activation. The metadata log appears to be empty. Appending 6 bootstrap record(s) in metadata transaction at metadata.version 4.2-IV1 from bootstrap source 'testkit'. (org.apache.kafka.controller.QuorumController:106)
[2025-07-02 09:49:53,820] WARN [Producer clientId=producer-4] The metadata response from the cluster reported a recoverable issue with correlation id 1 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-02 09:49:54,566] WARN [NodeToControllerChannelManager id=3000 name=registration] Attempting to close NetworkClient that has already been closed. (org.apache.kafka.clients.NetworkClient:744)
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-7033202108695639212/controller_3000 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-7033202108695639212/broker_0_data0 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-7033202108695639212/broker_1_data0 with metadata.version 4.2-IV1.
[2025-07-02 09:49:54,652] WARN [QuorumController id=3000] Performing controller activation. The metadata log appears to be empty. Appending 6 bootstrap record(s) in metadata transaction at metadata.version 4.2-IV1 from bootstrap source 'testkit'. (org.apache.kafka.controller.QuorumController:106)
[2025-07-02 09:49:54,811] WARN [Producer clientId=producer-5] The metadata response from the cluster reported a recoverable issue with correlation id 1 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-02 09:49:55,023] WARN [Producer clientId=producer-5] The metadata response from the cluster reported a recoverable issue with correlation id 7 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-02 09:49:55,590] WARN [NodeToControllerChannelManager id=3000 name=registration] Attempting to close NetworkClient that has already been closed. (org.apache.kafka.clients.NetworkClient:744)
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-4645572128043181693/controller_3000 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-4645572128043181693/broker_1_data0 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-4645572128043181693/broker_0_data0 with metadata.version 4.2-IV1.
[2025-07-02 09:49:55,693] WARN [QuorumController id=3000] Performing controller activation. The metadata log appears to be empty. Appending 6 bootstrap record(s) in metadata transaction at metadata.version 4.2-IV1 from bootstrap source 'testkit'. (org.apache.kafka.controller.QuorumController:106)
[2025-07-02 09:49:55,847] WARN [Producer clientId=producer-6] The metadata response from the cluster reported a recoverable issue with correlation id 1 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-02 09:49:56,058] WARN [Producer clientId=producer-6] The metadata response from the cluster reported a recoverable issue with correlation id 7 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-02 09:49:56,568] WARN [NodeToControllerChannelManager id=3000 name=registration] Attempting to close NetworkClient that has already been closed. (org.apache.kafka.clients.NetworkClient:744)

Not sure why the cluster keeps reformatting. However, the default delivery.timeout.ms is 120 secs, which is too long for the test. In our case, it's ok for a producer.send() to fail. We just want to make sure that every callback is called. So, we could at least reduce delivery.timeout.ms.

junrao avatar Jul 02 '25 17:07 junrao

Not sure why the cluster keeps reformatting. However, the default delivery.timeout.ms is 120 secs, which is too long for the test. In our case, it's ok for a producer.send() to fail. We just want to make sure that every callback is called. So, we could at least reduce delivery.timeout.ms.

The root cause is that the default partition count is '2', while the re-created partition count is '1'. Hence, the produce request sent to topic-1, which the topic is auto-created, can never be completed.

chia7712 avatar Jul 02 '25 19:07 chia7712

@OmniaGM : Thanks for the updated PR. I ran the new test until failure. After 10 minutes, I still saw the timeout.

java.util.concurrent.TimeoutException: testSendWhileTopicGetRecreated() timed out after 120 seconds

	at java.base/java.util.Optional.ifPresent(Optional.java:178)
	at java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1708)
	at java.base/java.util.stream.ReferencePipeline$Head.forEach(ReferencePipeline.java:762)
	at java.base/java.util.ArrayList.forEach(ArrayList.java:1596)
	at java.base/java.util.ArrayList.forEach(ArrayList.java:1596)

Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-6089077873146292020/broker_0_data0 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-6089077873146292020/broker_1_data0 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-6089077873146292020/controller_3000 with metadata.version 4.2-IV1.
[2025-07-03 08:32:04,927] WARN [QuorumController id=3000] Performing controller activation. The metadata log appears to be empty. Appending 6 bootstrap record(s) in metadata transaction at metadata.version 4.2-IV1 from bootstrap source 'testkit'. (org.apache.kafka.controller.QuorumController:106)
[2025-07-03 08:32:05,103] WARN [Producer clientId=producer-2] The metadata response from the cluster reported a recoverable issue with correlation id 1 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-03 08:32:05,214] WARN [Producer clientId=producer-2] The metadata response from the cluster reported a recoverable issue with correlation id 7 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-03 08:32:05,438] WARN [Producer clientId=producer-2] The metadata response from the cluster reported a recoverable issue with correlation id 9 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-03 08:32:06,165] WARN [NodeToControllerChannelManager id=3000 name=registration] Attempting to close NetworkClient that has already been closed. (org.apache.kafka.clients.NetworkClient:744)
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-6608140775505501629/controller_3000 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-6608140775505501629/broker_1_data0 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-6608140775505501629/broker_0_data0 with metadata.version 4.2-IV1.
[2025-07-03 08:32:06,290] WARN [QuorumController id=3000] Performing controller activation. The metadata log appears to be empty. Appending 6 bootstrap record(s) in metadata transaction at metadata.version 4.2-IV1 from bootstrap source 'testkit'. (org.apache.kafka.controller.QuorumController:106)
[2025-07-03 08:32:06,450] WARN [Producer clientId=producer-3] The metadata response from the cluster reported a recoverable issue with correlation id 1 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-03 08:32:06,558] WARN [Producer clientId=producer-3] The metadata response from the cluster reported a recoverable issue with correlation id 7 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-03 08:32:07,360] WARN [NodeToControllerChannelManager id=3000 name=registration] Attempting to close NetworkClient that has already been closed. (org.apache.kafka.clients.NetworkClient:744)
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-1689995402898229137/controller_3000 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-1689995402898229137/broker_1_data0 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-1689995402898229137/broker_0_data0 with metadata.version 4.2-IV1.
[2025-07-03 08:32:07,468] WARN [QuorumController id=3000] Performing controller activation. The metadata log appears to be empty. Appending 6 bootstrap record(s) in metadata transaction at metadata.version 4.2-IV1 from bootstrap source 'testkit'. (org.apache.kafka.controller.QuorumController:106)
[2025-07-03 08:32:07,628] WARN [Producer clientId=producer-4] The metadata response from the cluster reported a recoverable issue with correlation id 1 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-03 08:32:07,733] WARN [Producer clientId=producer-4] The metadata response from the cluster reported a recoverable issue with correlation id 7 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-03 08:32:07,956] WARN [Producer clientId=producer-4] The metadata response from the cluster reported a recoverable issue with correlation id 8 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-03 08:32:08,770] WARN [NodeToControllerChannelManager id=3000 name=registration] Attempting to close NetworkClient that has already been closed. (org.apache.kafka.clients.NetworkClient:744)
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-17048505300526058547/controller_3000 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-17048505300526058547/broker_1_data0 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-17048505300526058547/broker_0_data0 with metadata.version 4.2-IV1.
[2025-07-03 08:32:08,880] WARN [QuorumController id=3000] Performing controller activation. The metadata log appears to be empty. Appending 6 bootstrap record(s) in metadata transaction at metadata.version 4.2-IV1 from bootstrap source 'testkit'. (org.apache.kafka.controller.QuorumController:106)
[2025-07-03 08:32:09,145] WARN [Producer clientId=producer-5] The metadata response from the cluster reported a recoverable issue with correlation id 2 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-03 08:32:09,272] WARN [Producer clientId=producer-5] Got error produce response with correlation id 8 on topic-partition topic-0, retrying (2147483646 attempts left). Error: UNKNOWN_TOPIC_OR_PARTITION (org.apache.kafka.clients.producer.internals.Sender:680)
[2025-07-03 08:32:09,273] WARN [Producer clientId=producer-5] Received unknown topic or partition error in produce request on partition topic-0. The topic-partition may not exist or the user may not have Describe access to it (org.apache.kafka.clients.producer.internals.Sender:702)
[2025-07-03 08:32:09,365] WARN [Producer clientId=producer-5] The metadata response from the cluster reported a recoverable issue with correlation id 9 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-03 08:32:09,484] WARN [Producer clientId=producer-5] The metadata response from the cluster reported a recoverable issue with correlation id 10 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-03 08:32:10,164] WARN [NodeToControllerChannelManager id=3000 name=registration] Attempting to close NetworkClient that has already been closed. (org.apache.kafka.clients.NetworkClient:744)
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-12586289379138957056/controller_3000 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-12586289379138957056/broker_0_data0 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-12586289379138957056/broker_1_data0 with metadata.version 4.2-IV1.
[2025-07-03 08:32:10,281] WARN [QuorumController id=3000] Performing controller activation. The metadata log appears to be empty. Appending 6 bootstrap record(s) in metadata transaction at metadata.version 4.2-IV1 from bootstrap source 'testkit'. (org.apache.kafka.controller.QuorumController:106)
[2025-07-03 08:32:10,440] WARN [Producer clientId=producer-6] The metadata response from the cluster reported a recoverable issue with correlation id 2 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-03 08:32:10,545] WARN [Producer clientId=producer-6] The metadata response from the cluster reported a recoverable issue with correlation id 8 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-03 08:32:10,782] WARN [Producer clientId=producer-6] The metadata response from the cluster reported a recoverable issue with correlation id 9 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-03 08:32:11,569] WARN [NodeToControllerChannelManager id=3000 name=registration] Attempting to close NetworkClient that has already been closed. (org.apache.kafka.clients.NetworkClient:744)
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-2477307315647262884/controller_3000 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-2477307315647262884/broker_0_data0 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-2477307315647262884/broker_1_data0 with metadata.version 4.2-IV1.
[2025-07-03 08:32:11,712] WARN [QuorumController id=3000] Performing controller activation. The metadata log appears to be empty. Appending 6 bootstrap record(s) in metadata transaction at metadata.version 4.2-IV1 from bootstrap source 'testkit'. (org.apache.kafka.controller.QuorumController:106)
[2025-07-03 08:32:11,857] WARN [Producer clientId=producer-7] The metadata response from the cluster reported a recoverable issue with correlation id 2 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-03 08:32:12,066] WARN [Producer clientId=producer-7] The metadata response from the cluster reported a recoverable issue with correlation id 7 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-03 08:32:12,760] WARN [NodeToControllerChannelManager id=3000 name=registration] Attempting to close NetworkClient that has already been closed. (org.apache.kafka.clients.NetworkClient:744)
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-14757478012604431632/controller_3000 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-14757478012604431632/broker_0_data0 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-14757478012604431632/broker_1_data0 with metadata.version 4.2-IV1.
[2025-07-03 08:32:12,860] WARN [QuorumController id=3000] Performing controller activation. The metadata log appears to be empty. Appending 6 bootstrap record(s) in metadata transaction at metadata.version 4.2-IV1 from bootstrap source 'testkit'. (org.apache.kafka.controller.QuorumController:106)
[2025-07-03 08:32:13,007] WARN [Producer clientId=producer-8] The metadata response from the cluster reported a recoverable issue with correlation id 1 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-03 08:32:13,217] WARN [Producer clientId=producer-8] Got error produce response with correlation id 9 on topic-partition topic-1, retrying (2147483646 attempts left). Error: UNKNOWN_TOPIC_ID (org.apache.kafka.clients.producer.internals.Sender:680)
[2025-07-03 08:32:13,217] WARN [Producer clientId=producer-8] Received invalid metadata error in produce request on partition topic-1 due to org.apache.kafka.common.errors.UnknownTopicIdException: This server does not host this topic ID. Going to request metadata update now (org.apache.kafka.clients.producer.internals.Sender:706)
[2025-07-03 08:32:13,331] WARN [Producer clientId=producer-8] Got error produce response with correlation id 12 on topic-partition topic-1, retrying (2147483645 attempts left). Error: UNKNOWN_TOPIC_ID (org.apache.kafka.clients.producer.internals.Sender:680)
[2025-07-03 08:32:13,331] WARN [Producer clientId=producer-8] Received invalid metadata error in produce request on partition topic-1 due to org.apache.kafka.common.errors.UnknownTopicIdException: This server does not host this topic ID. Going to request metadata update now (org.apache.kafka.clients.producer.internals.Sender:706)
[2025-07-03 08:32:13,332] WARN [Producer clientId=producer-8] The metadata response from the cluster reported a recoverable issue with correlation id 13 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-03 08:32:13,551] WARN [Producer clientId=producer-8] Got error produce response with correlation id 15 on topic-partition topic-1, retrying (2147483644 attempts left). Error: UNKNOWN_TOPIC_ID (org.apache.kafka.clients.producer.internals.Sender:680)
[2025-07-03 08:32:13,551] WARN [Producer clientId=producer-8] Received invalid metadata error in produce request on partition topic-1 due to org.apache.kafka.common.errors.UnknownTopicIdException: This server does not host this topic ID. Going to request metadata update now (org.apache.kafka.clients.producer.internals.Sender:706)
[2025-07-03 08:32:14,376] WARN [NodeToControllerChannelManager id=3000 name=registration] Attempting to close NetworkClient that has already been closed. (org.apache.kafka.clients.NetworkClient:744)
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-321361512033974143/controller_3000 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-321361512033974143/broker_0_data0 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-321361512033974143/broker_1_data0 with metadata.version 4.2-IV1.
[2025-07-03 08:32:14,462] WARN [QuorumController id=3000] Performing controller activation. The metadata log appears to be empty. Appending 6 bootstrap record(s) in metadata transaction at metadata.version 4.2-IV1 from bootstrap source 'testkit'. (org.apache.kafka.controller.QuorumController:106)
[2025-07-03 08:32:14,620] WARN [Producer clientId=producer-9] The metadata response from the cluster reported a recoverable issue with correlation id 1 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-03 08:32:14,726] WARN [Producer clientId=producer-9] The metadata response from the cluster reported a recoverable issue with correlation id 7 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-03 08:32:14,965] WARN [Producer clientId=producer-9] The metadata response from the cluster reported a recoverable issue with correlation id 9 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-03 08:32:15,774] WARN [NodeToControllerChannelManager id=3000 name=registration] Attempting to close NetworkClient that has already been closed. (org.apache.kafka.clients.NetworkClient:744)
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-543793708133122389/controller_3000 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-543793708133122389/broker_0_data0 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-543793708133122389/broker_1_data0 with metadata.version 4.2-IV1.
[2025-07-03 08:32:15,883] WARN [QuorumController id=3000] Performing controller activation. The metadata log appears to be empty. Appending 6 bootstrap record(s) in metadata transaction at metadata.version 4.2-IV1 from bootstrap source 'testkit'. (org.apache.kafka.controller.QuorumController:106)
[2025-07-03 08:32:16,033] WARN [Producer clientId=producer-10] The metadata response from the cluster reported a recoverable issue with correlation id 1 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-03 08:32:16,241] WARN [Producer clientId=producer-10] The metadata response from the cluster reported a recoverable issue with correlation id 7 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-03 08:32:16,472] WARN [Producer clientId=producer-10] The metadata response from the cluster reported a recoverable issue with correlation id 9 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-03 08:32:17,360] WARN [NodeToControllerChannelManager id=3000 name=registration] Attempting to close NetworkClient that has already been closed. (org.apache.kafka.clients.NetworkClient:744)
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-18238593010590373133/controller_3000 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-18238593010590373133/broker_1_data0 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-18238593010590373133/broker_0_data0 with metadata.version 4.2-IV1.
[2025-07-03 08:32:17,454] WARN [QuorumController id=3000] Performing controller activation. The metadata log appears to be empty. Appending 6 bootstrap record(s) in metadata transaction at metadata.version 4.2-IV1 from bootstrap source 'testkit'. (org.apache.kafka.controller.QuorumController:106)
[2025-07-03 08:32:17,634] WARN [Producer clientId=producer-11] The metadata response from the cluster reported a recoverable issue with correlation id 2 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-03 08:32:17,739] WARN [Producer clientId=producer-11] The metadata response from the cluster reported a recoverable issue with correlation id 7 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-03 08:32:18,369] WARN [NodeToControllerChannelManager id=3000 name=registration] Attempting to close NetworkClient that has already been closed. (org.apache.kafka.clients.NetworkClient:744)
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-1007440053127284728/controller_3000 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-1007440053127284728/broker_0_data0 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-1007440053127284728/broker_1_data0 with metadata.version 4.2-IV1.
[2025-07-03 08:32:18,459] WARN [QuorumController id=3000] Performing controller activation. The metadata log appears to be empty. Appending 6 bootstrap record(s) in metadata transaction at metadata.version 4.2-IV1 from bootstrap source 'testkit'. (org.apache.kafka.controller.QuorumController:106)
[2025-07-03 08:32:18,636] WARN [Producer clientId=producer-12] The metadata response from the cluster reported a recoverable issue with correlation id 1 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-03 08:32:18,845] WARN [Producer clientId=producer-12] The metadata response from the cluster reported a recoverable issue with correlation id 8 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-03 08:32:19,035] WARN [Producer clientId=producer-12] The metadata response from the cluster reported a recoverable issue with correlation id 10 : {topic=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1295)
[2025-07-03 08:32:19,997] WARN [NodeToControllerChannelManager id=3000 name=registration] Attempting to close NetworkClient that has already been closed. (org.apache.kafka.clients.NetworkClient:744)
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-11984181535268137544/broker_0_data0 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-11984181535268137544/controller_3000 with metadata.version 4.2-IV1.
Formatting metadata directory /var/folders/t1/_lmg790n4dg46yk7lzvzg45r0000gp/T/kafka-11984181535268137544/broker_1_data0 with metadata.version 4.2-IV1.

junrao avatar Jul 03 '25 16:07 junrao

I made the following change and the test hasn't failed after running for 15 minutes. It uses a shorter delivery timeout and request timeout. We probably also want to reduce the test time.

         AtomicInteger numAcks = new AtomicInteger(0);
         var producerFuture = CompletableFuture.runAsync(() -> {
-            try (var producer = cluster.producer()) {
+            try (var producer = createProducer()) {

junrao avatar Jul 03 '25 16:07 junrao

I made the following change and the test hasn't failed after running for 15 minutes. It uses a shorter delivery timeout and request timeout. We probably also want to reduce the test time.

         AtomicInteger numAcks = new AtomicInteger(0);
         var producerFuture = CompletableFuture.runAsync(() -> {
-            try (var producer = cluster.producer()) {
+            try (var producer = createProducer()) {

Updated the pr now, and removed the timeouts seems to be running way faster now and should be within the default 60 timeout for the test (at least locally will wait for the pipeline)

OmniaGM avatar Jul 03 '25 17:07 OmniaGM

I ran the new test until failure. After 10 minutes, I still saw the timeout.

@junrao Could you share more logs with me if they exist. I'm curious about the root cause of the timeout, since it does not happen on my machine even after over 1000 attempts for the https://github.com/apache/kafka/commit/6419e079e439f312de5b2ea4c33def457ef4f8fb

chia7712 avatar Jul 03 '25 18:07 chia7712

test_failure.txt

@chia7712 : Attached is the test failure output. It still happened after the latest fix.

junrao avatar Jul 03 '25 18:07 junrao

test_failure.txt

@chia7712 : Attached is the test failure output. It still happened after the latest fix.

I can't reproduce it locally as well. @chia7712 did you had any luck reproducing the timeout locally

OmniaGM avatar Jul 04 '25 14:07 OmniaGM

I ran an existing test testSendWithTopicReassignmentIsMidWay until failure and hit the same failure (test_failure2.txt). So, this seems to be issue in the test framework on my local environment. We can merge the PR as it is.

junrao avatar Jul 04 '25 16:07 junrao

@OmniaGM thanks for handling that crisis. BTW, we have a commit tool https://github.com/apache/kafka/blob/trunk/committer-tools/reviewers.py which can easily add reviewers to the commit message. Perhaps you could give it a try next time.

chia7712 avatar Jul 04 '25 16:07 chia7712

@OmniaGM thanks for handling that crisis. BTW, we have a commit tool https://github.com/apache/kafka/blob/trunk/committer-tools/reviewers.py which can easily add reviewers to the commit message. Perhaps you could give it a try next time.

I was wondering about this just now as I noticed the commit went without them. I thought it runs automatically. My bad will try it next time

OmniaGM avatar Jul 04 '25 16:07 OmniaGM

@OmniaGM could you please backport it to 4.1 branch?

chia7712 avatar Jul 04 '25 21:07 chia7712