kafka-users mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Sean Glover <sean.glo...@lightbend.com>
Subject Re: Producer send blocking when destination partition does not exist
Date Mon, 04 Nov 2019 18:51:53 GMT
Hi Jamie,

It is enabled, because that is the default, but that is just coincidental.
My use case which reproduced this error was the following:

1. Launch a single Kafka broker & ZK node with docker (testcontainers-java)
2. Create a topic with 2 partitions by using the admin client.  I block
during this step before proceeding.
3. Produce messages to this topic and specify partitions 0, 1, and 2 (this
was my mistake) in ProducerRecords
4. My first two messages sent successfully, then I was blocked when
attempting to send a message to partition 2 which does not exist.

Regards,
Sean

On Mon, Nov 4, 2019 at 1:37 PM Jamie <jamiedd13@aol.co.uk.invalid> wrote:

> Hi Sean,
> Out of interest, is auto topic creation enabled on the brokers?
> Thanks,
> Jamie
>
>
> -----Original Message-----
> From: Sean Glover <sean.glover@lightbend.com>
> To: users <users@kafka.apache.org>
> Sent: Mon, Nov 4, 2019 04:21 PM
> Subject: Producer send blocking when destination partition does not exist
>
>
> Hi,
>
> I accidentally created a scenario where I was attempting to produce a
> record to a partition that did not exist, because I was manually overriding
> the destination partition, and I noticed that the producer.send blocked for
> 60s (producer property max.block.ms).  During this time the producer was
> internally polling the broker with metadata requests, I assume to wait for
> this partition to eventually exist.  After the timeout the failure is
> returned.  I've since learned the same thing happens if the topic does not
> exist.
>
> I have a few questions:
>
> Q: Why does the producer block in this scenario?  It could "fail fast" if
> the producer record contains incompatible topic metadata such as incorrect
> topic name or invalid partition number.
>
> Q: Should we consider adding a more informative error message that
> indicates what the root cause of the problem is?  For example, that the
> partition does not exist.
>
> Logs:
>
> 11:15:08.176 DEBUG [-thread | producer-1]
> o.apache.kafka.clients.NetworkClient  [Producer clientId=producer-1]
> Sending metadata request (type=MetadataRequest, topics=topic-1-1) to node
> localhost:33263 (id: 0 rack: null)
> 11:15:08.221 DEBUG [-thread | producer-1] o.a.k.c.producer.internals.Sender
>     [Producer clientId=producer-1] ProducerId: 0; Set last ack'd sequence
> number for topic-partition topic-1-1-1 to 0
> 11:15:08.270 DEBUG [-thread | producer-1] o.a.k.c.producer.internals.Sender
>     [Producer clientId=producer-1] ProducerId: 0; Set last ack'd sequence
> number for topic-partition topic-1-1-0 to 0
> 11:15:08.273 DEBUG [-thread | producer-1] org.apache.kafka.clients.Metadata
>     Updated cluster metadata version 3 to Cluster(id =
> 5zRzH-78RkmjmB5Ud8J6VQ, nodes = [localhost:33263 (id: 0 rack: null)],
> partitions = [Partition(topic = topic-1-1, partition = 1, leader = 0,
> replicas = [0], isr = [0], offlineReplicas = []), Partition(topic =
> topic-1-1, partition = 0, leader = 0, replicas = [0], isr = [0],
> offlineReplicas = [])], controller = localhost:33263 (id: 0 rack: null))
> 11:15:08.372 DEBUG [-thread | producer-1]
> o.apache.kafka.clients.NetworkClient  [Producer clientId=producer-1]
> Sending metadata request (type=MetadataRequest, topics=topic-1-1) to node
> localhost:33263 (id: 0 rack: null)
> 11:15:08.374 DEBUG [-thread | producer-1] org.apache.kafka.clients.Metadata
>     Updated cluster metadata version 4 to Cluster(id =
> 5zRzH-78RkmjmB5Ud8J6VQ, nodes = [localhost:33263 (id: 0 rack: null)],
> partitions = [Partition(topic = topic-1-1, partition = 1, leader = 0,
> replicas = [0], isr = [0], offlineReplicas = []), Partition(topic =
> topic-1-1, partition = 0, leader = 0, replicas = [0], isr = [0],
> offlineReplicas = [])], controller = localhost:33263 (id: 0 rack: null))
> 11:15:08.474 DEBUG [-thread | producer-1]
> o.apache.kafka.clients.NetworkClient  [Producer clientId=producer-1]
> Sending metadata request (type=MetadataRequest, topics=topic-1-1) to node
> localhost:33263 (id: 0 rack: null)
> 11:15:08.478 DEBUG [-thread | producer-1] org.apache.kafka.clients.Metadata
>     Updated cluster metadata version 5 to Cluster(id =
> 5zRzH-78RkmjmB5Ud8J6VQ, nodes = [localhost:33263 (id: 0 rack: null)],
> partitions = [Partition(topic = topic-1-1, partition = 1, leader = 0,
> replicas = [0], isr = [0], offlineReplicas = []), Partition(topic =
> topic-1-1, partition = 0, leader = 0, replicas = [0], isr = [0],
> offlineReplicas = [])], controller = localhost:33263 (id: 0 rack: null))
> ...
> 11:16:08.095 DEBUG [-thread | producer-1]
> o.apache.kafka.clients.NetworkClient  [Producer clientId=producer-1]
> Sending metadata request (type=MetadataRequest, topics=topic-1-1) to node
> localhost:33263 (id: 0 rack: null)
> 11:16:08.096 DEBUG [-thread | producer-1] org.apache.kafka.clients.Metadata
>     Updated cluster metadata version 582 to Cluster(id =
> 5zRzH-78RkmjmB5Ud8J6VQ, nodes = [localhost:33263 (id: 0 rack: null)],
> partitions = [Partition(topic = topic-1-1, partition = 1, leader = 0,
> replicas = [0], isr = [0], offlineReplicas = []), Partition(topic =
> topic-1-1, partition = 0, leader = 0, replicas = [0], isr = [0],
> offlineReplicas = [])], controller = localhost:33263 (id: 0 rack: null))
> 11:16:08.099 DEBUG [default-dispatcher-7]
> o.a.k.clients.producer.KafkaProducer  [Producer clientId=producer-1]
> Exception occurred during message send:
> org.apache.kafka.common.errors.TimeoutException: Topic topic-1-1 not
> present in metadata after 60000 ms.
>
> --
> Sean Glover
> Principal Engineer, Alpakka, Lightbend, Inc. <https://lightbend.com>
> @seg1o <https://twitter.com/seg1o>, in/seanaglover
> <https://www.linkedin.com/in/seanaglover/>
>

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message