You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by Francis Chuang <fr...@apache.org> on 2022/04/02 08:56:52 UTC

Request was for a topic or partition that does not exist on this broker error when using Kafka in Kraft mode

Hey Everyone,

I have a Go library that publishes and consumes from Kafka using 
Shopify's sarama library.

Initially, my test environment used Kafka and Zookeeper. Recently, I 
updated it to use Kafka in Kraft (no-zookeeper) mode. I noticed 
previously passing and stable tests started failing with the "kafka 
server: Request was for a topic or partition that does not exist on this 
broker." error message.

Here's what I am doing:
- Creating a Kafka consumer
- Using the consumer or consumer group to subscribe to a topic that does 
not exist

Note, in my project it is valid to subscribe to a topic that does not 
exist yet and receive messages from it when publishers write to it later.

The error message does not reproduce consistently, so it takes a few 
tries for the error message to show up. However, the error will 
eventually show for Kafka in Kraft mode, but never in Zookeeper mode.

I have a repository demonstrating this problem here: 
https://github.com/F21/kafka-topic-does-not-exist

It basically creates a kafka consumer and consumers a topic that does 
not exist multiple times to trigger the error.

To run it with Kafka and Zookeeper, use `docker compose -f 
docker-compose-zk.yml`

To run it in Kraft mode, use `docker compose run test`

After running each type of test, use `docker compose down` to tear down 
the environment completely.

Can someone please shed some light on why this is happening?

Thanks,
Francis



Re: Request was for a topic or partition that does not exist on this broker error when using Kafka in Kraft mode

Posted by Francis Chuang <fr...@apache.org>.
Hmm, upon further investigation, the client also returns "kafka server: 
Request was for a topic or partition that does not exist on this 
broker." if I am not using a consumer group.

There doesn't seem to be anything interesting in kafka's logs:

Broker 1:
[2022-04-04 23:58:42,480] INFO [Controller 1] Created topic mytopic-3 
with topic ID ykUAlXknQR24xpF1BlxY_A. 
(org.apache.kafka.controller.ReplicationControlManager)
[2022-04-04 23:58:42,480] INFO [Controller 1] Created partition 
mytopic-3-0 with topic ID ykUAlXknQR24xpF1BlxY_A and 
PartitionRegistration(replicas=[3], isr=[3], removingReplicas=[], 
addingReplicas=[], leader=3, leaderEpoch=0, partitionEpoch=0). 
(org.apache.kafka.controller.ReplicationControlManager)

Broker 2:
[2022-04-04 23:58:41,933] INFO Sent auto-creation request for 
Set(mytopic-3) to the active controller. 
(kafka.server.DefaultAutoTopicCreationManager)
[2022-04-04 23:58:41,936] INFO [Controller 2] CreateTopics result(s): 
CreatableTopic(name='mytopic-3', numPartitions=1, replicationFactor=1, 
assignments=[], configs=[]): SUCCESS 
(org.apache.kafka.controller.ReplicationControlManager)
[2022-04-04 23:58:41,937] INFO [Controller 2] Created topic mytopic-3 
with topic ID ykUAlXknQR24xpF1BlxY_A. 
(org.apache.kafka.controller.ReplicationControlManager)
[2022-04-04 23:58:41,937] INFO [Controller 2] Created partition 
mytopic-3-0 with topic ID ykUAlXknQR24xpF1BlxY_A and 
PartitionRegistration(replicas=[3], isr=[3], removingReplicas=[], 
addingReplicas=[], leader=3, leaderEpoch=0, partitionEpoch=0). 
(org.apache.kafka.controller.ReplicationControlManager)

Broker 3:
[2022-04-04 23:58:42,479] INFO [Controller 3] Created topic mytopic-3 
with topic ID ykUAlXknQR24xpF1BlxY_A. 
(org.apache.kafka.controller.ReplicationControlManager)
[2022-04-04 23:58:42,479] INFO [Controller 3] Created partition 
mytopic-3-0 with topic ID ykUAlXknQR24xpF1BlxY_A and 
PartitionRegistration(replicas=[3], isr=[3], removingReplicas=[], 
addingReplicas=[], leader=3, leaderEpoch=0, partitionEpoch=0). 
(org.apache.kafka.controller.ReplicationControlManager)
[2022-04-04 23:58:42,480] INFO [Broker id=3] Transitioning 1 
partition(s) to local leaders. (state.change.logger)
[2022-04-04 23:58:42,480] INFO [ReplicaFetcherManager on broker 3] 
Removed fetcher for partitions Set(mytopic-3-0) 
(kafka.server.ReplicaFetcherManager)
[2022-04-04 23:58:42,480] INFO [Broker id=3] Creating new partition 
mytopic-3-0 with topic id ykUAlXknQR24xpF1BlxY_A. (state.change.logger)
[2022-04-04 23:58:42,485] INFO [LogLoader partition=mytopic-3-0, 
dir=/bitnami/kafka/data] Loading producer state till offset 0 with 
message format version 2 (kafka.log.UnifiedLog$)
[2022-04-04 23:58:42,486] INFO Created log for partition mytopic-3-0 in 
/bitnami/kafka/data/mytopic-3-0 with properties {} (kafka.log.LogManager)
[2022-04-04 23:58:42,487] INFO [Partition mytopic-3-0 broker=3] No 
checkpointed highwatermark is found for partition mytopic-3-0 
(kafka.cluster.Partition)
[2022-04-04 23:58:42,487] INFO [Partition mytopic-3-0 broker=3] Log 
loaded for partition mytopic-3-0 with initial high watermark 0 
(kafka.cluster.Partition)
[2022-04-04 23:58:42,487] INFO [Broker id=3] Leader mytopic-3-0 starts 
at leader epoch 0 from offset 0 with high watermark 0 ISR [3] 
addingReplicas [] removingReplicas []. Previous leader epoch was -1. 
(state.change.logger)

Error from the sarama client:
2022/04/04 23:58:42 Could not consume partition 0 in topic mytopic-3: 
kafka server: Request was for a topic or partition that does not exist 
on this broker.
         Topics in broker: [mytopic-2 mytopic-1 mytopic-0 mytopic-3]
         Partitions for topic: [0]

This is using the set up reproducing the problem here: 
https://github.com/F21/kafka-topic-does-not-exist

Francis

On 5/04/2022 9:43 am, Liam Clarke-Hutchinson wrote:
> Hi Francis,
> 
> The rebalancing is your consumer group, it looks like your consumer is
> repeatedly entering and leaving when hitting that error. Do you have logs
> from the consumer side?
> 
> Cheers,
> 
> Liam
> 
> On Tue, 5 Apr 2022 at 11:30, Francis Chuang <fr...@apache.org>
> wrote:
> 
>> Not sure if this is due to the broker rebalancing multiple times every
>> second for the topic. Relevant logs for the 3 brokers below:
>>
>> Broker 1:
>> [2022-04-04 23:14:24,232] INFO Sent auto-creation request for
>> Set(test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704) to the
>> active controller. (kafka.server.DefaultAutoTopicCreationManager)
>> [2022-04-04 23:14:24,234] INFO [Controller 2] CreateTopics result(s):
>> CreatableTopic(name='test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704',
>>
>> numPartitions=1, replicationFactor=1, assignments=[], configs=[]):
>> SUCCESS (org.apache.kafka.controller.ReplicationControlManager)
>> [2022-04-04 23:14:24,234] INFO [Controller 2] Created topic
>> test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 with topic
>> ID JujvHxuwSF65PjxapwU4KA.
>> (org.apache.kafka.controller.ReplicationControlManager)
>> [2022-04-04 23:14:24,235] INFO [Controller 2] Created partition
>> test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704-0 with topic
>> ID JujvHxuwSF65PjxapwU4KA and PartitionRegistration(replicas=[3],
>> isr=[3], removingReplicas=[], addingReplicas=[], leader=3,
>> leaderEpoch=0, partitionEpoch=0).
>> (org.apache.kafka.controller.ReplicationControlManager)
>>
>> Broker 2 (there are a lot more rebalancing entries in the log, so this
>> is just a sample):
>> [2022-04-04 23:14:23,787] INFO [GroupCoordinator 3]: Dynamic Member with
>> unknown member id joins group
>> test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 in Empty
>> state. Created a new member id
>> sarama-f36324b2-ec03-4c44-a8dc-67188c1986c8 for this member and add to
>> the group. (kafka.coordinator.group.GroupCoordinator)
>> [2022-04-04 23:14:23,799] INFO [GroupCoordinator 3]: Preparing to
>> rebalance group
>> test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 in state
>> PreparingRebalance with old generation 0 (__consumer_offsets-15)
>> (reason: Adding new member sarama-f36324b2-ec03-4c44-a8dc-67188c1986c8
>> with group instance id None) (kafka.coordinator.group.GroupCoordinator)
>> [2022-04-04 23:14:23,808] INFO [GroupCoordinator 3]: Stabilized group
>> test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 generation 1
>> (__consumer_offsets-15) with 1 members
>> (kafka.coordinator.group.GroupCoordinator)
>> [2022-04-04 23:14:23,818] INFO [GroupCoordinator 3]: Assignment received
>> from leader sarama-f36324b2-ec03-4c44-a8dc-67188c1986c8 for group
>> test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 for
>> generation 1. The group has 1 members, 0 of which are static.
>> (kafka.coordinator.group.GroupCoordinator)
>> [2022-04-04 23:14:24,030] INFO [Controller 3] Created topic
>> test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 with topic
>> ID zVBVJQfOSrGmpjlYPEvv0w.
>> (org.apache.kafka.controller.ReplicationControlManager)
>> [2022-04-04 23:14:24,030] INFO [Controller 3] Created partition
>> test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71-0 with topic
>> ID zVBVJQfOSrGmpjlYPEvv0w and PartitionRegistration(replicas=[2],
>> isr=[2], removingReplicas=[], addingReplicas=[], leader=2,
>> leaderEpoch=0, partitionEpoch=0).
>> (org.apache.kafka.controller.ReplicationControlManager)
>> [2022-04-04 23:14:24,208] INFO [GroupCoordinator 3]: Preparing to
>> rebalance group
>> test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 in state
>> PreparingRebalance with old generation 1 (__consumer_offsets-15)
>> (reason: Removing member sarama-f36324b2-ec03-4c44-a8dc-67188c1986c8 on
>> LeaveGroup) (kafka.coordinator.group.GroupCoordinator)
>> [2022-04-04 23:14:24,210] INFO [GroupCoordinator 3]: Group
>> test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 with
>> generation 2 is now empty (__consumer_offsets-15)
>> (kafka.coordinator.group.GroupCoordinator)
>> [2022-04-04 23:14:24,216] INFO [GroupCoordinator 3]: Member
>> MemberMetadata(memberId=sarama-f36324b2-ec03-4c44-a8dc-67188c1986c8,
>> groupInstanceId=None, clientId=sarama, clientHost=/172.21.0.6,
>> sessionTimeoutMs=10000, rebalanceTimeoutMs=60000,
>> supportedProtocols=List(range)) has left group
>> test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 through
>> explicit `LeaveGroup` request (kafka.coordinator.group.GroupCoordinator)
>> [2022-04-04 23:14:24,494] INFO [GroupCoordinator 3]: Dynamic Member with
>> unknown member id joins group
>> test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 in Empty
>> state. Created a new member id
>> sarama-9affaa64-3233-4deb-978e-771fc790cf34 for this member and add to
>> the group. (kafka.coordinator.group.GroupCoordinator)
>> [2022-04-04 23:14:24,495] INFO [GroupCoordinator 3]: Preparing to
>> rebalance group
>> test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 in state
>> PreparingRebalance with old generation 0 (__consumer_offsets-5) (reason:
>> Adding new member sarama-9affaa64-3233-4deb-978e-771fc790cf34 with group
>> instance id None) (kafka.coordinator.group.GroupCoordinator)
>> [2022-04-04 23:14:24,499] INFO [GroupCoordinator 3]: Stabilized group
>> test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 generation 1
>> (__consumer_offsets-5) with 1 members
>> (kafka.coordinator.group.GroupCoordinator)
>> [2022-04-04 23:14:24,504] INFO [GroupCoordinator 3]: Assignment received
>> from leader sarama-9affaa64-3233-4deb-978e-771fc790cf34 for group
>> test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 for
>> generation 1. The group has 1 members, 0 of which are static.
>> (kafka.coordinator.group.GroupCoordinator)
>> ...
>>
>> Broker 3:
>> [2022-04-04 23:14:24,772] INFO [Controller 1] Created topic
>> test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 with topic
>> ID JujvHxuwSF65PjxapwU4KA.
>> (org.apache.kafka.controller.ReplicationControlManager)
>> [2022-04-04 23:14:24,772] INFO [Controller 1] Created partition
>> test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704-0 with topic
>> ID JujvHxuwSF65PjxapwU4KA and PartitionRegistration(replicas=[3],
>> isr=[3], removingReplicas=[], addingReplicas=[], leader=3,
>> leaderEpoch=0, partitionEpoch=0).
>> (org.apache.kafka.controller.ReplicationControlManager)
>>
>> On 2/04/2022 7:56 pm, Francis Chuang wrote:
>>> Hey Everyone,
>>>
>>> I have a Go library that publishes and consumes from Kafka using
>>> Shopify's sarama library.
>>>
>>> Initially, my test environment used Kafka and Zookeeper. Recently, I
>>> updated it to use Kafka in Kraft (no-zookeeper) mode. I noticed
>>> previously passing and stable tests started failing with the "kafka
>>> server: Request was for a topic or partition that does not exist on this
>>> broker." error message.
>>>
>>> Here's what I am doing:
>>> - Creating a Kafka consumer
>>> - Using the consumer or consumer group to subscribe to a topic that does
>>> not exist
>>>
>>> Note, in my project it is valid to subscribe to a topic that does not
>>> exist yet and receive messages from it when publishers write to it later.
>>>
>>> The error message does not reproduce consistently, so it takes a few
>>> tries for the error message to show up. However, the error will
>>> eventually show for Kafka in Kraft mode, but never in Zookeeper mode.
>>>
>>> I have a repository demonstrating this problem here:
>>> https://github.com/F21/kafka-topic-does-not-exist
>>>
>>> It basically creates a kafka consumer and consumers a topic that does
>>> not exist multiple times to trigger the error.
>>>
>>> To run it with Kafka and Zookeeper, use `docker compose -f
>>> docker-compose-zk.yml`
>>>
>>> To run it in Kraft mode, use `docker compose run test`
>>>
>>> After running each type of test, use `docker compose down` to tear down
>>> the environment completely.
>>>
>>> Can someone please shed some light on why this is happening?
>>>
>>> Thanks,
>>> Francis
>>>
>>>
>>
>>
> 

Re: Request was for a topic or partition that does not exist on this broker error when using Kafka in Kraft mode

Posted by Francis Chuang <fr...@apache.org>.
Hi Liam,

These are the logs I have for the consumer (I'll see if I can turn on 
debug mode for sarama to get more info):
2022-04-04T23:14:24.519Z        ERROR   client/client.go:410    Kafka 
consumer error    {"error": "kafka: error while consuming 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704/0: kafka 
server: Request was for a topic or partition that does not exist on this 
broker."}
github.com/Boostport/event-messaging/client.(*Client).startKafkaClient.func1
         /source/client/client.go:410
2022-04-04T23:14:24.534Z        ERROR   client/client.go:410    Kafka 
consumer error    {"error": "kafka: error while consuming 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704/0: kafka 
server: Request was for a topic or partition that does not exist on this 
broker."}
github.com/Boostport/event-messaging/client.(*Client).startKafkaClient.func1
         /source/client/client.go:410
2022-04-04T23:14:24.547Z        ERROR   client/client.go:410    Kafka 
consumer error    {"error": "kafka: error while consuming 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704/0: kafka 
server: Request was for a topic or partition that does not exist on this 
broker."}
github.com/Boostport/event-messaging/client.(*Client).startKafkaClient.func1
         /source/client/client.go:410
2022-04-04T23:14:24.561Z        ERROR   client/client.go:410    Kafka 
consumer error    {"error": "kafka: error while consuming 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704/0: kafka 
server: Request was for a topic or partition that does not exist on this 
broker."}
github.com/Boostport/event-messaging/client.(*Client).startKafkaClient.func1
         /source/client/client.go:410
2022-04-04T23:14:24.574Z        ERROR   client/client.go:410    Kafka 
consumer error    {"error": "kafka: error while consuming 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704/0: kafka 
server: Request was for a topic or partition that does not exist on this 
broker."}
github.com/Boostport/event-messaging/client.(*Client).startKafkaClient.func1
         /source/client/client.go:410
2022-04-04T23:14:24.588Z        ERROR   client/client.go:410    Kafka 
consumer error    {"error": "kafka: error while consuming 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704/0: kafka 
server: Request was for a topic or partition that does not exist on this 
broker."}
github.com/Boostport/event-messaging/client.(*Client).startKafkaClient.func1
         /source/client/client.go:410
2022-04-04T23:14:24.602Z        ERROR   client/client.go:410    Kafka 
consumer error    {"error": "kafka: error while consuming 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704/0: kafka 
server: Request was for a topic or partition that does not exist on this 
broker."}
github.com/Boostport/event-messaging/client.(*Client).startKafkaClient.func1
         /source/client/client.go:410
2022-04-04T23:14:24.615Z        ERROR   client/client.go:410    Kafka 
consumer error    {"error": "kafka: error while consuming 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704/0: kafka 
server: Request was for a topic or partition that does not exist on this 
broker."}
github.com/Boostport/event-messaging/client.(*Client).startKafkaClient.func1
         /source/client/client.go:410
2022-04-04T23:14:24.627Z        ERROR   client/client.go:410    Kafka 
consumer error    {"error": "kafka: error while consuming 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704/0: kafka 
server: Request was for a topic or partition that does not exist on this 
broker."}
github.com/Boostport/event-messaging/client.(*Client).startKafkaClient.func1
         /source/client/client.go:410
2022-04-04T23:14:24.639Z        ERROR   client/client.go:410    Kafka 
consumer error    {"error": "kafka: error while consuming 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704/0: kafka 
server: Request was for a topic or partition that does not exist on this 
broker."}
github.com/Boostport/event-messaging/client.(*Client).startKafkaClient.func1
         /source/client/client.go:410
2022-04-04T23:14:24.650Z        ERROR   client/client.go:410    Kafka 
consumer error    {"error": "kafka: error while consuming 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704/0: kafka 
server: Request was for a topic or partition that does not exist on this 
broker."}
github.com/Boostport/event-messaging/client.(*Client).startKafkaClient.func1
         /source/client/client.go:410
2022-04-04T23:14:24.661Z        ERROR   client/client.go:410    Kafka 
consumer error    {"error": "kafka: error while consuming 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704/0: kafka 
server: Request was for a topic or partition that does not exist on this 
broker."}
github.com/Boostport/event-messaging/client.(*Client).startKafkaClient.func1
         /source/client/client.go:410
2022-04-04T23:14:24.672Z        ERROR   client/client.go:410    Kafka 
consumer error    {"error": "kafka: error while consuming 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704/0: kafka 
server: Request was for a topic or partition that does not exist on this 
broker."}
github.com/Boostport/event-messaging/client.(*Client).startKafkaClient.func1
         /source/client/client.go:410
2022-04-04T23:14:24.682Z        ERROR   client/client.go:410    Kafka 
consumer error    {"error": "kafka: error while consuming 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704/0: kafka 
server: Request was for a topic or partition that does not exist on this 
broker."}
github.com/Boostport/event-messaging/client.(*Client).startKafkaClient.func1
         /source/client/client.go:410
2022-04-04T23:14:24.693Z        ERROR   client/client.go:410    Kafka 
consumer error    {"error": "kafka: error while consuming 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704/0: kafka 
server: Request was for a topic or partition that does not exist on this 
broker."}
github.com/Boostport/event-messaging/client.(*Client).startKafkaClient.func1
         /source/client/client.go:410
2022-04-04T23:14:24.703Z        ERROR   client/client.go:410    Kafka 
consumer error    {"error": "kafka: error while consuming 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704/0: kafka 
server: Request was for a topic or partition that does not exist on this 
broker."}
github.com/Boostport/event-messaging/client.(*Client).startKafkaClient.func1
         /source/client/client.go:410
2022-04-04T23:14:24.714Z        ERROR   client/client.go:410    Kafka 
consumer error    {"error": "kafka: error while consuming 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704/0: kafka 
server: Request was for a topic or partition that does not exist on this 
broker."}
github.com/Boostport/event-messaging/client.(*Client).startKafkaClient.func1
         /source/client/client.go:410
2022-04-04T23:14:24.723Z        ERROR   client/client.go:410    Kafka 
consumer error    {"error": "kafka: error while consuming 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704/0: kafka 
server: Request was for a topic or partition that does not exist on this 
broker."}
github.com/Boostport/event-messaging/client.(*Client).startKafkaClient.func1
         /source/client/client.go:410
2022-04-04T23:14:24.733Z        ERROR   client/client.go:410    Kafka 
consumer error    {"error": "kafka: error while consuming 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704/0: kafka 
server: Request was for a topic or partition that does not exist on this 
broker."}
github.com/Boostport/event-messaging/client.(*Client).startKafkaClient.func1
         /source/client/client.go:410
2022-04-04T23:14:24.742Z        ERROR   client/client.go:410    Kafka 
consumer error    {"error": "kafka: error while consuming 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704/0: kafka 
server: Request was for a topic or partition that does not exist on this 
broker."}
github.com/Boostport/event-messaging/client.(*Client).startKafkaClient.func1
         /source/client/client.go:410
2022-04-04T23:14:24.753Z        ERROR   client/client.go:410    Kafka 
consumer error    {"error": "kafka: error while consuming 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704/0: kafka 
server: Request was for a topic or partition that does not exist on this 
broker."}
github.com/Boostport/event-messaging/client.(*Client).startKafkaClient.func1
         /source/client/client.go:410
2022-04-04T23:14:24.762Z        ERROR   client/client.go:410    Kafka 
consumer error    {"error": "kafka: error while consuming 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704/0: kafka 
server: Request was for a topic or partition that does not exist on this 
broker."}
github.com/Boostport/event-messaging/client.(*Client).startKafkaClient.func1
         /source/client/client.go:410
2022-04-04T23:14:24.770Z        ERROR   client/client.go:410    Kafka 
consumer error    {"error": "kafka: error while consuming 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704/0: kafka 
server: Request was for a topic or partition that does not exist on this 
broker."}
github.com/Boostport/event-messaging/client.(*Client).startKafkaClient.func1
         /source/client/client.go:410

It seems that eventually it does stabalize, logs in the broker:

[2022-04-04 23:14:24,764] INFO [GroupCoordinator 3]: Preparing to 
rebalance group 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 in state 
PreparingRebalance with old generation 22 (__consumer_offsets-5) 
(reason: Leader sarama-9affaa64-3233-4deb-978e-771fc790cf34 re-joining 
group during Stable) (kafka.coordinator.group.GroupCoordinator)
[2022-04-04 23:14:24,764] INFO [GroupCoordinator 3]: Stabilized group 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 generation 
23 (__consumer_offsets-5) with 1 members 
(kafka.coordinator.group.GroupCoordinator)
[2022-04-04 23:14:24,765] INFO [GroupCoordinator 3]: Assignment received 
from leader sarama-9affaa64-3233-4deb-978e-771fc790cf34 for group 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 for 
generation 23. The group has 1 members, 0 of which are static. 
(kafka.coordinator.group.GroupCoordinator)
[2022-04-04 23:14:24,771] INFO [Controller 3] Created topic 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 with topic 
ID JujvHxuwSF65PjxapwU4KA. 
(org.apache.kafka.controller.ReplicationControlManager)
[2022-04-04 23:14:24,771] INFO [Controller 3] Created partition 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704-0 with topic 
ID JujvHxuwSF65PjxapwU4KA and PartitionRegistration(replicas=[3], 
isr=[3], removingReplicas=[], addingReplicas=[], leader=3, 
leaderEpoch=0, partitionEpoch=0). 
(org.apache.kafka.controller.ReplicationControlManager)
[2022-04-04 23:14:24,773] INFO [Broker id=3] Transitioning 1 
partition(s) to local leaders. (state.change.logger)
[2022-04-04 23:14:24,773] INFO [ReplicaFetcherManager on broker 3] 
Removed fetcher for partitions 
Set(test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704-0) 
(kafka.server.ReplicaFetcherManager)
[2022-04-04 23:14:24,773] INFO [Broker id=3] Creating new partition 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704-0 with topic 
id JujvHxuwSF65PjxapwU4KA. (state.change.logger)
[2022-04-04 23:14:24,773] INFO [GroupCoordinator 3]: Preparing to 
rebalance group 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 in state 
PreparingRebalance with old generation 23 (__consumer_offsets-5) 
(reason: Leader sarama-9affaa64-3233-4deb-978e-771fc790cf34 re-joining 
group during Stable) (kafka.coordinator.group.GroupCoordinator)
[2022-04-04 23:14:24,773] INFO [GroupCoordinator 3]: Stabilized group 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 generation 
24 (__consumer_offsets-5) with 1 members 
(kafka.coordinator.group.GroupCoordinator)
[2022-04-04 23:14:24,775] INFO [GroupCoordinator 3]: Assignment received 
from leader sarama-9affaa64-3233-4deb-978e-771fc790cf34 for group 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 for 
generation 24. The group has 1 members, 0 of which are static. 
(kafka.coordinator.group.GroupCoordinator)
[2022-04-04 23:14:24,777] INFO [LogLoader 
partition=test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704-0, 
dir=/tmp/logs] Loading producer state till offset 0 with message format 
version 2 (kafka.log.UnifiedLog$)
[2022-04-04 23:14:24,778] INFO Created log for partition 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704-0 in 
/tmp/logs/test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704-0 
with properties {} (kafka.log.LogManager)
[2022-04-04 23:14:24,779] INFO [Partition 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704-0 broker=3] 
No checkpointed highwatermark is found for partition 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704-0 
(kafka.cluster.Partition)
[2022-04-04 23:14:24,780] INFO [Partition 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704-0 broker=3] 
Log loaded for partition 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704-0 with 
initial high watermark 0 (kafka.cluster.Partition)
[2022-04-04 23:14:24,780] INFO [Broker id=3] Leader 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704-0 starts at 
leader epoch 0 from offset 0 with high watermark 0 ISR [3] 
addingReplicas [] removingReplicas []. Previous leader epoch was -1. 
(state.change.logger)
[2022-04-04 23:14:45,016] INFO [GroupCoordinator 3]: Preparing to 
rebalance group 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 in state 
PreparingRebalance with old generation 24 (__consumer_offsets-5) 
(reason: Removing member sarama-9affaa64-3233-4deb-978e-771fc790cf34 on 
LeaveGroup) (kafka.coordinator.group.GroupCoordinator)
[2022-04-04 23:14:45,017] INFO [GroupCoordinator 3]: Group 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 with 
generation 25 is now empty (__consumer_offsets-5) 
(kafka.coordinator.group.GroupCoordinator)
[2022-04-04 23:14:45,020] INFO [GroupCoordinator 3]: Member 
MemberMetadata(memberId=sarama-9affaa64-3233-4deb-978e-771fc790cf34, 
groupInstanceId=None, clientId=sarama, clientHost=/172.21.0.6, 
sessionTimeoutMs=10000, rebalanceTimeoutMs=60000, 
supportedProtocols=List(range)) has left group 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 through 
explicit `LeaveGroup` request (kafka.coordinator.group.GroupCoordinator)

Thanks,
Francis

On 5/04/2022 9:43 am, Liam Clarke-Hutchinson wrote:
> Hi Francis,
> 
> The rebalancing is your consumer group, it looks like your consumer is
> repeatedly entering and leaving when hitting that error. Do you have logs
> from the consumer side?
> 
> Cheers,
> 
> Liam
> 
> On Tue, 5 Apr 2022 at 11:30, Francis Chuang <fr...@apache.org>
> wrote:
> 
>> Not sure if this is due to the broker rebalancing multiple times every
>> second for the topic. Relevant logs for the 3 brokers below:
>>
>> Broker 1:
>> [2022-04-04 23:14:24,232] INFO Sent auto-creation request for
>> Set(test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704) to the
>> active controller. (kafka.server.DefaultAutoTopicCreationManager)
>> [2022-04-04 23:14:24,234] INFO [Controller 2] CreateTopics result(s):
>> CreatableTopic(name='test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704',
>>
>> numPartitions=1, replicationFactor=1, assignments=[], configs=[]):
>> SUCCESS (org.apache.kafka.controller.ReplicationControlManager)
>> [2022-04-04 23:14:24,234] INFO [Controller 2] Created topic
>> test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 with topic
>> ID JujvHxuwSF65PjxapwU4KA.
>> (org.apache.kafka.controller.ReplicationControlManager)
>> [2022-04-04 23:14:24,235] INFO [Controller 2] Created partition
>> test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704-0 with topic
>> ID JujvHxuwSF65PjxapwU4KA and PartitionRegistration(replicas=[3],
>> isr=[3], removingReplicas=[], addingReplicas=[], leader=3,
>> leaderEpoch=0, partitionEpoch=0).
>> (org.apache.kafka.controller.ReplicationControlManager)
>>
>> Broker 2 (there are a lot more rebalancing entries in the log, so this
>> is just a sample):
>> [2022-04-04 23:14:23,787] INFO [GroupCoordinator 3]: Dynamic Member with
>> unknown member id joins group
>> test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 in Empty
>> state. Created a new member id
>> sarama-f36324b2-ec03-4c44-a8dc-67188c1986c8 for this member and add to
>> the group. (kafka.coordinator.group.GroupCoordinator)
>> [2022-04-04 23:14:23,799] INFO [GroupCoordinator 3]: Preparing to
>> rebalance group
>> test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 in state
>> PreparingRebalance with old generation 0 (__consumer_offsets-15)
>> (reason: Adding new member sarama-f36324b2-ec03-4c44-a8dc-67188c1986c8
>> with group instance id None) (kafka.coordinator.group.GroupCoordinator)
>> [2022-04-04 23:14:23,808] INFO [GroupCoordinator 3]: Stabilized group
>> test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 generation 1
>> (__consumer_offsets-15) with 1 members
>> (kafka.coordinator.group.GroupCoordinator)
>> [2022-04-04 23:14:23,818] INFO [GroupCoordinator 3]: Assignment received
>> from leader sarama-f36324b2-ec03-4c44-a8dc-67188c1986c8 for group
>> test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 for
>> generation 1. The group has 1 members, 0 of which are static.
>> (kafka.coordinator.group.GroupCoordinator)
>> [2022-04-04 23:14:24,030] INFO [Controller 3] Created topic
>> test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 with topic
>> ID zVBVJQfOSrGmpjlYPEvv0w.
>> (org.apache.kafka.controller.ReplicationControlManager)
>> [2022-04-04 23:14:24,030] INFO [Controller 3] Created partition
>> test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71-0 with topic
>> ID zVBVJQfOSrGmpjlYPEvv0w and PartitionRegistration(replicas=[2],
>> isr=[2], removingReplicas=[], addingReplicas=[], leader=2,
>> leaderEpoch=0, partitionEpoch=0).
>> (org.apache.kafka.controller.ReplicationControlManager)
>> [2022-04-04 23:14:24,208] INFO [GroupCoordinator 3]: Preparing to
>> rebalance group
>> test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 in state
>> PreparingRebalance with old generation 1 (__consumer_offsets-15)
>> (reason: Removing member sarama-f36324b2-ec03-4c44-a8dc-67188c1986c8 on
>> LeaveGroup) (kafka.coordinator.group.GroupCoordinator)
>> [2022-04-04 23:14:24,210] INFO [GroupCoordinator 3]: Group
>> test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 with
>> generation 2 is now empty (__consumer_offsets-15)
>> (kafka.coordinator.group.GroupCoordinator)
>> [2022-04-04 23:14:24,216] INFO [GroupCoordinator 3]: Member
>> MemberMetadata(memberId=sarama-f36324b2-ec03-4c44-a8dc-67188c1986c8,
>> groupInstanceId=None, clientId=sarama, clientHost=/172.21.0.6,
>> sessionTimeoutMs=10000, rebalanceTimeoutMs=60000,
>> supportedProtocols=List(range)) has left group
>> test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 through
>> explicit `LeaveGroup` request (kafka.coordinator.group.GroupCoordinator)
>> [2022-04-04 23:14:24,494] INFO [GroupCoordinator 3]: Dynamic Member with
>> unknown member id joins group
>> test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 in Empty
>> state. Created a new member id
>> sarama-9affaa64-3233-4deb-978e-771fc790cf34 for this member and add to
>> the group. (kafka.coordinator.group.GroupCoordinator)
>> [2022-04-04 23:14:24,495] INFO [GroupCoordinator 3]: Preparing to
>> rebalance group
>> test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 in state
>> PreparingRebalance with old generation 0 (__consumer_offsets-5) (reason:
>> Adding new member sarama-9affaa64-3233-4deb-978e-771fc790cf34 with group
>> instance id None) (kafka.coordinator.group.GroupCoordinator)
>> [2022-04-04 23:14:24,499] INFO [GroupCoordinator 3]: Stabilized group
>> test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 generation 1
>> (__consumer_offsets-5) with 1 members
>> (kafka.coordinator.group.GroupCoordinator)
>> [2022-04-04 23:14:24,504] INFO [GroupCoordinator 3]: Assignment received
>> from leader sarama-9affaa64-3233-4deb-978e-771fc790cf34 for group
>> test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 for
>> generation 1. The group has 1 members, 0 of which are static.
>> (kafka.coordinator.group.GroupCoordinator)
>> ...
>>
>> Broker 3:
>> [2022-04-04 23:14:24,772] INFO [Controller 1] Created topic
>> test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 with topic
>> ID JujvHxuwSF65PjxapwU4KA.
>> (org.apache.kafka.controller.ReplicationControlManager)
>> [2022-04-04 23:14:24,772] INFO [Controller 1] Created partition
>> test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704-0 with topic
>> ID JujvHxuwSF65PjxapwU4KA and PartitionRegistration(replicas=[3],
>> isr=[3], removingReplicas=[], addingReplicas=[], leader=3,
>> leaderEpoch=0, partitionEpoch=0).
>> (org.apache.kafka.controller.ReplicationControlManager)
>>
>> On 2/04/2022 7:56 pm, Francis Chuang wrote:
>>> Hey Everyone,
>>>
>>> I have a Go library that publishes and consumes from Kafka using
>>> Shopify's sarama library.
>>>
>>> Initially, my test environment used Kafka and Zookeeper. Recently, I
>>> updated it to use Kafka in Kraft (no-zookeeper) mode. I noticed
>>> previously passing and stable tests started failing with the "kafka
>>> server: Request was for a topic or partition that does not exist on this
>>> broker." error message.
>>>
>>> Here's what I am doing:
>>> - Creating a Kafka consumer
>>> - Using the consumer or consumer group to subscribe to a topic that does
>>> not exist
>>>
>>> Note, in my project it is valid to subscribe to a topic that does not
>>> exist yet and receive messages from it when publishers write to it later.
>>>
>>> The error message does not reproduce consistently, so it takes a few
>>> tries for the error message to show up. However, the error will
>>> eventually show for Kafka in Kraft mode, but never in Zookeeper mode.
>>>
>>> I have a repository demonstrating this problem here:
>>> https://github.com/F21/kafka-topic-does-not-exist
>>>
>>> It basically creates a kafka consumer and consumers a topic that does
>>> not exist multiple times to trigger the error.
>>>
>>> To run it with Kafka and Zookeeper, use `docker compose -f
>>> docker-compose-zk.yml`
>>>
>>> To run it in Kraft mode, use `docker compose run test`
>>>
>>> After running each type of test, use `docker compose down` to tear down
>>> the environment completely.
>>>
>>> Can someone please shed some light on why this is happening?
>>>
>>> Thanks,
>>> Francis
>>>
>>>
>>
>>
> 

Re: Request was for a topic or partition that does not exist on this broker error when using Kafka in Kraft mode

Posted by Liam Clarke-Hutchinson <lc...@redhat.com>.
Hi Francis,

The rebalancing is your consumer group, it looks like your consumer is
repeatedly entering and leaving when hitting that error. Do you have logs
from the consumer side?

Cheers,

Liam

On Tue, 5 Apr 2022 at 11:30, Francis Chuang <fr...@apache.org>
wrote:

> Not sure if this is due to the broker rebalancing multiple times every
> second for the topic. Relevant logs for the 3 brokers below:
>
> Broker 1:
> [2022-04-04 23:14:24,232] INFO Sent auto-creation request for
> Set(test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704) to the
> active controller. (kafka.server.DefaultAutoTopicCreationManager)
> [2022-04-04 23:14:24,234] INFO [Controller 2] CreateTopics result(s):
> CreatableTopic(name='test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704',
>
> numPartitions=1, replicationFactor=1, assignments=[], configs=[]):
> SUCCESS (org.apache.kafka.controller.ReplicationControlManager)
> [2022-04-04 23:14:24,234] INFO [Controller 2] Created topic
> test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 with topic
> ID JujvHxuwSF65PjxapwU4KA.
> (org.apache.kafka.controller.ReplicationControlManager)
> [2022-04-04 23:14:24,235] INFO [Controller 2] Created partition
> test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704-0 with topic
> ID JujvHxuwSF65PjxapwU4KA and PartitionRegistration(replicas=[3],
> isr=[3], removingReplicas=[], addingReplicas=[], leader=3,
> leaderEpoch=0, partitionEpoch=0).
> (org.apache.kafka.controller.ReplicationControlManager)
>
> Broker 2 (there are a lot more rebalancing entries in the log, so this
> is just a sample):
> [2022-04-04 23:14:23,787] INFO [GroupCoordinator 3]: Dynamic Member with
> unknown member id joins group
> test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 in Empty
> state. Created a new member id
> sarama-f36324b2-ec03-4c44-a8dc-67188c1986c8 for this member and add to
> the group. (kafka.coordinator.group.GroupCoordinator)
> [2022-04-04 23:14:23,799] INFO [GroupCoordinator 3]: Preparing to
> rebalance group
> test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 in state
> PreparingRebalance with old generation 0 (__consumer_offsets-15)
> (reason: Adding new member sarama-f36324b2-ec03-4c44-a8dc-67188c1986c8
> with group instance id None) (kafka.coordinator.group.GroupCoordinator)
> [2022-04-04 23:14:23,808] INFO [GroupCoordinator 3]: Stabilized group
> test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 generation 1
> (__consumer_offsets-15) with 1 members
> (kafka.coordinator.group.GroupCoordinator)
> [2022-04-04 23:14:23,818] INFO [GroupCoordinator 3]: Assignment received
> from leader sarama-f36324b2-ec03-4c44-a8dc-67188c1986c8 for group
> test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 for
> generation 1. The group has 1 members, 0 of which are static.
> (kafka.coordinator.group.GroupCoordinator)
> [2022-04-04 23:14:24,030] INFO [Controller 3] Created topic
> test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 with topic
> ID zVBVJQfOSrGmpjlYPEvv0w.
> (org.apache.kafka.controller.ReplicationControlManager)
> [2022-04-04 23:14:24,030] INFO [Controller 3] Created partition
> test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71-0 with topic
> ID zVBVJQfOSrGmpjlYPEvv0w and PartitionRegistration(replicas=[2],
> isr=[2], removingReplicas=[], addingReplicas=[], leader=2,
> leaderEpoch=0, partitionEpoch=0).
> (org.apache.kafka.controller.ReplicationControlManager)
> [2022-04-04 23:14:24,208] INFO [GroupCoordinator 3]: Preparing to
> rebalance group
> test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 in state
> PreparingRebalance with old generation 1 (__consumer_offsets-15)
> (reason: Removing member sarama-f36324b2-ec03-4c44-a8dc-67188c1986c8 on
> LeaveGroup) (kafka.coordinator.group.GroupCoordinator)
> [2022-04-04 23:14:24,210] INFO [GroupCoordinator 3]: Group
> test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 with
> generation 2 is now empty (__consumer_offsets-15)
> (kafka.coordinator.group.GroupCoordinator)
> [2022-04-04 23:14:24,216] INFO [GroupCoordinator 3]: Member
> MemberMetadata(memberId=sarama-f36324b2-ec03-4c44-a8dc-67188c1986c8,
> groupInstanceId=None, clientId=sarama, clientHost=/172.21.0.6,
> sessionTimeoutMs=10000, rebalanceTimeoutMs=60000,
> supportedProtocols=List(range)) has left group
> test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 through
> explicit `LeaveGroup` request (kafka.coordinator.group.GroupCoordinator)
> [2022-04-04 23:14:24,494] INFO [GroupCoordinator 3]: Dynamic Member with
> unknown member id joins group
> test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 in Empty
> state. Created a new member id
> sarama-9affaa64-3233-4deb-978e-771fc790cf34 for this member and add to
> the group. (kafka.coordinator.group.GroupCoordinator)
> [2022-04-04 23:14:24,495] INFO [GroupCoordinator 3]: Preparing to
> rebalance group
> test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 in state
> PreparingRebalance with old generation 0 (__consumer_offsets-5) (reason:
> Adding new member sarama-9affaa64-3233-4deb-978e-771fc790cf34 with group
> instance id None) (kafka.coordinator.group.GroupCoordinator)
> [2022-04-04 23:14:24,499] INFO [GroupCoordinator 3]: Stabilized group
> test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 generation 1
> (__consumer_offsets-5) with 1 members
> (kafka.coordinator.group.GroupCoordinator)
> [2022-04-04 23:14:24,504] INFO [GroupCoordinator 3]: Assignment received
> from leader sarama-9affaa64-3233-4deb-978e-771fc790cf34 for group
> test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 for
> generation 1. The group has 1 members, 0 of which are static.
> (kafka.coordinator.group.GroupCoordinator)
> ...
>
> Broker 3:
> [2022-04-04 23:14:24,772] INFO [Controller 1] Created topic
> test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 with topic
> ID JujvHxuwSF65PjxapwU4KA.
> (org.apache.kafka.controller.ReplicationControlManager)
> [2022-04-04 23:14:24,772] INFO [Controller 1] Created partition
> test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704-0 with topic
> ID JujvHxuwSF65PjxapwU4KA and PartitionRegistration(replicas=[3],
> isr=[3], removingReplicas=[], addingReplicas=[], leader=3,
> leaderEpoch=0, partitionEpoch=0).
> (org.apache.kafka.controller.ReplicationControlManager)
>
> On 2/04/2022 7:56 pm, Francis Chuang wrote:
> > Hey Everyone,
> >
> > I have a Go library that publishes and consumes from Kafka using
> > Shopify's sarama library.
> >
> > Initially, my test environment used Kafka and Zookeeper. Recently, I
> > updated it to use Kafka in Kraft (no-zookeeper) mode. I noticed
> > previously passing and stable tests started failing with the "kafka
> > server: Request was for a topic or partition that does not exist on this
> > broker." error message.
> >
> > Here's what I am doing:
> > - Creating a Kafka consumer
> > - Using the consumer or consumer group to subscribe to a topic that does
> > not exist
> >
> > Note, in my project it is valid to subscribe to a topic that does not
> > exist yet and receive messages from it when publishers write to it later.
> >
> > The error message does not reproduce consistently, so it takes a few
> > tries for the error message to show up. However, the error will
> > eventually show for Kafka in Kraft mode, but never in Zookeeper mode.
> >
> > I have a repository demonstrating this problem here:
> > https://github.com/F21/kafka-topic-does-not-exist
> >
> > It basically creates a kafka consumer and consumers a topic that does
> > not exist multiple times to trigger the error.
> >
> > To run it with Kafka and Zookeeper, use `docker compose -f
> > docker-compose-zk.yml`
> >
> > To run it in Kraft mode, use `docker compose run test`
> >
> > After running each type of test, use `docker compose down` to tear down
> > the environment completely.
> >
> > Can someone please shed some light on why this is happening?
> >
> > Thanks,
> > Francis
> >
> >
>
>

Re: Request was for a topic or partition that does not exist on this broker error when using Kafka in Kraft mode

Posted by Francis Chuang <fr...@apache.org>.
Not sure if this is due to the broker rebalancing multiple times every 
second for the topic. Relevant logs for the 3 brokers below:

Broker 1:
[2022-04-04 23:14:24,232] INFO Sent auto-creation request for 
Set(test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704) to the 
active controller. (kafka.server.DefaultAutoTopicCreationManager)
[2022-04-04 23:14:24,234] INFO [Controller 2] CreateTopics result(s): 
CreatableTopic(name='test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704', 
numPartitions=1, replicationFactor=1, assignments=[], configs=[]): 
SUCCESS (org.apache.kafka.controller.ReplicationControlManager)
[2022-04-04 23:14:24,234] INFO [Controller 2] Created topic 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 with topic 
ID JujvHxuwSF65PjxapwU4KA. 
(org.apache.kafka.controller.ReplicationControlManager)
[2022-04-04 23:14:24,235] INFO [Controller 2] Created partition 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704-0 with topic 
ID JujvHxuwSF65PjxapwU4KA and PartitionRegistration(replicas=[3], 
isr=[3], removingReplicas=[], addingReplicas=[], leader=3, 
leaderEpoch=0, partitionEpoch=0). 
(org.apache.kafka.controller.ReplicationControlManager)

Broker 2 (there are a lot more rebalancing entries in the log, so this 
is just a sample):
[2022-04-04 23:14:23,787] INFO [GroupCoordinator 3]: Dynamic Member with 
unknown member id joins group 
test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 in Empty 
state. Created a new member id 
sarama-f36324b2-ec03-4c44-a8dc-67188c1986c8 for this member and add to 
the group. (kafka.coordinator.group.GroupCoordinator)
[2022-04-04 23:14:23,799] INFO [GroupCoordinator 3]: Preparing to 
rebalance group 
test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 in state 
PreparingRebalance with old generation 0 (__consumer_offsets-15) 
(reason: Adding new member sarama-f36324b2-ec03-4c44-a8dc-67188c1986c8 
with group instance id None) (kafka.coordinator.group.GroupCoordinator)
[2022-04-04 23:14:23,808] INFO [GroupCoordinator 3]: Stabilized group 
test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 generation 1 
(__consumer_offsets-15) with 1 members 
(kafka.coordinator.group.GroupCoordinator)
[2022-04-04 23:14:23,818] INFO [GroupCoordinator 3]: Assignment received 
from leader sarama-f36324b2-ec03-4c44-a8dc-67188c1986c8 for group 
test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 for 
generation 1. The group has 1 members, 0 of which are static. 
(kafka.coordinator.group.GroupCoordinator)
[2022-04-04 23:14:24,030] INFO [Controller 3] Created topic 
test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 with topic 
ID zVBVJQfOSrGmpjlYPEvv0w. 
(org.apache.kafka.controller.ReplicationControlManager)
[2022-04-04 23:14:24,030] INFO [Controller 3] Created partition 
test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71-0 with topic 
ID zVBVJQfOSrGmpjlYPEvv0w and PartitionRegistration(replicas=[2], 
isr=[2], removingReplicas=[], addingReplicas=[], leader=2, 
leaderEpoch=0, partitionEpoch=0). 
(org.apache.kafka.controller.ReplicationControlManager)
[2022-04-04 23:14:24,208] INFO [GroupCoordinator 3]: Preparing to 
rebalance group 
test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 in state 
PreparingRebalance with old generation 1 (__consumer_offsets-15) 
(reason: Removing member sarama-f36324b2-ec03-4c44-a8dc-67188c1986c8 on 
LeaveGroup) (kafka.coordinator.group.GroupCoordinator)
[2022-04-04 23:14:24,210] INFO [GroupCoordinator 3]: Group 
test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 with 
generation 2 is now empty (__consumer_offsets-15) 
(kafka.coordinator.group.GroupCoordinator)
[2022-04-04 23:14:24,216] INFO [GroupCoordinator 3]: Member 
MemberMetadata(memberId=sarama-f36324b2-ec03-4c44-a8dc-67188c1986c8, 
groupInstanceId=None, clientId=sarama, clientHost=/172.21.0.6, 
sessionTimeoutMs=10000, rebalanceTimeoutMs=60000, 
supportedProtocols=List(range)) has left group 
test-bounded-context-1-c42b0ec0-7646-4f9e-b82c-17f380e07f71 through 
explicit `LeaveGroup` request (kafka.coordinator.group.GroupCoordinator)
[2022-04-04 23:14:24,494] INFO [GroupCoordinator 3]: Dynamic Member with 
unknown member id joins group 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 in Empty 
state. Created a new member id 
sarama-9affaa64-3233-4deb-978e-771fc790cf34 for this member and add to 
the group. (kafka.coordinator.group.GroupCoordinator)
[2022-04-04 23:14:24,495] INFO [GroupCoordinator 3]: Preparing to 
rebalance group 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 in state 
PreparingRebalance with old generation 0 (__consumer_offsets-5) (reason: 
Adding new member sarama-9affaa64-3233-4deb-978e-771fc790cf34 with group 
instance id None) (kafka.coordinator.group.GroupCoordinator)
[2022-04-04 23:14:24,499] INFO [GroupCoordinator 3]: Stabilized group 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 generation 1 
(__consumer_offsets-5) with 1 members 
(kafka.coordinator.group.GroupCoordinator)
[2022-04-04 23:14:24,504] INFO [GroupCoordinator 3]: Assignment received 
from leader sarama-9affaa64-3233-4deb-978e-771fc790cf34 for group 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 for 
generation 1. The group has 1 members, 0 of which are static. 
(kafka.coordinator.group.GroupCoordinator)
...

Broker 3:
[2022-04-04 23:14:24,772] INFO [Controller 1] Created topic 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704 with topic 
ID JujvHxuwSF65PjxapwU4KA. 
(org.apache.kafka.controller.ReplicationControlManager)
[2022-04-04 23:14:24,772] INFO [Controller 1] Created partition 
test-bounded-context-1-6cdaff21-6562-418a-92e8-1f2d8fa0c704-0 with topic 
ID JujvHxuwSF65PjxapwU4KA and PartitionRegistration(replicas=[3], 
isr=[3], removingReplicas=[], addingReplicas=[], leader=3, 
leaderEpoch=0, partitionEpoch=0). 
(org.apache.kafka.controller.ReplicationControlManager)

On 2/04/2022 7:56 pm, Francis Chuang wrote:
> Hey Everyone,
> 
> I have a Go library that publishes and consumes from Kafka using 
> Shopify's sarama library.
> 
> Initially, my test environment used Kafka and Zookeeper. Recently, I 
> updated it to use Kafka in Kraft (no-zookeeper) mode. I noticed 
> previously passing and stable tests started failing with the "kafka 
> server: Request was for a topic or partition that does not exist on this 
> broker." error message.
> 
> Here's what I am doing:
> - Creating a Kafka consumer
> - Using the consumer or consumer group to subscribe to a topic that does 
> not exist
> 
> Note, in my project it is valid to subscribe to a topic that does not 
> exist yet and receive messages from it when publishers write to it later.
> 
> The error message does not reproduce consistently, so it takes a few 
> tries for the error message to show up. However, the error will 
> eventually show for Kafka in Kraft mode, but never in Zookeeper mode.
> 
> I have a repository demonstrating this problem here: 
> https://github.com/F21/kafka-topic-does-not-exist
> 
> It basically creates a kafka consumer and consumers a topic that does 
> not exist multiple times to trigger the error.
> 
> To run it with Kafka and Zookeeper, use `docker compose -f 
> docker-compose-zk.yml`
> 
> To run it in Kraft mode, use `docker compose run test`
> 
> After running each type of test, use `docker compose down` to tear down 
> the environment completely.
> 
> Can someone please shed some light on why this is happening?
> 
> Thanks,
> Francis
> 
>