You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by David Morales de Frías <dm...@paradigmatecnologico.com> on 2014/03/13 10:42:46 UTC

consumer reaction to auto-create topics (0.8.1)

Hi there,

A few days ago i have written about an issue (0.8.0) when auto-create topic
is enabled and consumer starts before producer (consumer never sees any
message and they seem to be lost even by the brokers)

Now i have done the same test with 0.8.1 and the issue is partially fixed
but now the consumer never sees the first message in the topic.

These are the steps:

1) kafka-console-consumer --topic newTopic (it doesn't exist)
2) kafka-console-producer --topic newTopic
3) Send the first messages
4) I can see this in the log and the consumer never receives this first
message

[2014-03-13 10:24:07,223] WARN Error while fetching metadata
[{TopicMetadata for topic test3 ->
No partition metadata for topic test3 due to
kafka.common.LeaderNotAvailableException}] for topic [test3]: class
kafka.common.LeaderNotAvailableException
 (kafka.producer.BrokerPartitionInfo)
[2014-03-13 10:24:07,237] WARN Error while fetching metadata
[{TopicMetadata for topic test3 ->
No partition metadata for topic test3 due to
kafka.common.LeaderNotAvailableException}] for topic [test3]: class
kafka.common.LeaderNotAvailableException
 (kafka.producer.BrokerPartitionInfo)
[2014-03-13 10:24:07,238] ERROR Failed to collate messages by topic,
partition due to: Failed to fetch topic metadata for topic: test3
(kafka.producer.async.DefaultEventHandler)


5) send more messages
6) consumer receives all the messages (but the first one).


The same issue appears when you first start the producer and then the
consumer (on a new topic).

Re: consumer reaction to auto-create topics (0.8.1)

Posted by Jun Rao <ju...@gmail.com>.
If you set auto.offset.reset to smallest, the consumer should pick up the
first message on new topics.

Thanks,

Jun


On Thu, Mar 13, 2014 at 2:42 AM, David Morales de Frías <
dmorales@paradigmatecnologico.com> wrote:

> Hi there,
>
> A few days ago i have written about an issue (0.8.0) when auto-create topic
> is enabled and consumer starts before producer (consumer never sees any
> message and they seem to be lost even by the brokers)
>
> Now i have done the same test with 0.8.1 and the issue is partially fixed
> but now the consumer never sees the first message in the topic.
>
> These are the steps:
>
> 1) kafka-console-consumer --topic newTopic (it doesn't exist)
> 2) kafka-console-producer --topic newTopic
> 3) Send the first messages
> 4) I can see this in the log and the consumer never receives this first
> message
>
> [2014-03-13 10:24:07,223] WARN Error while fetching metadata
> [{TopicMetadata for topic test3 ->
> No partition metadata for topic test3 due to
> kafka.common.LeaderNotAvailableException}] for topic [test3]: class
> kafka.common.LeaderNotAvailableException
>  (kafka.producer.BrokerPartitionInfo)
> [2014-03-13 10:24:07,237] WARN Error while fetching metadata
> [{TopicMetadata for topic test3 ->
> No partition metadata for topic test3 due to
> kafka.common.LeaderNotAvailableException}] for topic [test3]: class
> kafka.common.LeaderNotAvailableException
>  (kafka.producer.BrokerPartitionInfo)
> [2014-03-13 10:24:07,238] ERROR Failed to collate messages by topic,
> partition due to: Failed to fetch topic metadata for topic: test3
> (kafka.producer.async.DefaultEventHandler)
>
>
> 5) send more messages
> 6) consumer receives all the messages (but the first one).
>
>
> The same issue appears when you first start the producer and then the
> consumer (on a new topic).
>

Re: consumer reaction to auto-create topics (0.8.1)

Posted by Neha Narkhede <ne...@gmail.com>.
I think it is worth fixing
https://issues.apache.org/jira/browse/KAFKA-1006since the behavior for
new topic discovery is broken in the new consumer.
Even if the consumer sets auto.offset.reset=largest, it is very unintuitive
to apply that behavior to the discovery of new topics. I think that new
topics should always be consumed from the beginning.

Thanks,
Neha


On Thu, Mar 13, 2014 at 10:20 AM, Guozhang Wang <wa...@gmail.com> wrote:

> Hi David,
>
> This issue should only happen if the following conditions are satisfied
>
> 1) The consumer does not have any offset stored in ZK for this topic when
> it first fetches messages for it.
> 2) The consumer's default offset reset value is "latest".
> 3) After the topic is created, it will take a bit time for the metadata to
> be propagated to all the brokers and hence the consumers know it can now
> consume the message, and then when the first fetch request comes with no
> specified offset, it will use the "latest" offset, and hence the first few
> messages that are already appended during the metadata propagation will be
> lost.
>
> For your case, if the services can go up and down but with the same topics
> all the time, and as long as you store the offsets somewhere (either in ZK
> or by yourself) you should not hit this issue.
>
> Guozhang
>
>
> On Thu, Mar 13, 2014 at 9:12 AM, David Morales de Frías <
> dmorales@paradigmatecnologico.com> wrote:
>
> > Hello Guozhang,
> >
> > We are using Kafka to connect several actors in our project (osgi, storm,
> > vert.x web app...), all of them being consumers and producers of
> different
> > topics. These services can go up and down in different  moments.
> >
> > Anyway, the issue is the same when you first start the consumer and then
> > the producer and the other way (tested with console producer and
> consumer).
> > So it doesn't depend on the order of services.
> >
> >
> > Thanks.
> >
> >
> >
> >
> >
> >
> >
> >
> > 2014-03-13 17:03 GMT+01:00 Guozhang Wang <wa...@gmail.com>:
> >
> > > Hello David,
> > >
> > > This is a known issue with consumer starting before the topic is
> created:
> > >
> > > https://issues.apache.org/jira/browse/KAFKA-1006
> > >
> > > And we are still figuring the right way to fix it. Could you describe
> > your
> > > use cases where you need to start the consumer beforehand?
> > >
> > > Guozhang
> > >
> > >
> > > On Thu, Mar 13, 2014 at 2:42 AM, David Morales de Frías <
> > > dmorales@paradigmatecnologico.com> wrote:
> > >
> > > > Hi there,
> > > >
> > > > A few days ago i have written about an issue (0.8.0) when auto-create
> > > topic
> > > > is enabled and consumer starts before producer (consumer never sees
> any
> > > > message and they seem to be lost even by the brokers)
> > > >
> > > > Now i have done the same test with 0.8.1 and the issue is partially
> > fixed
> > > > but now the consumer never sees the first message in the topic.
> > > >
> > > > These are the steps:
> > > >
> > > > 1) kafka-console-consumer --topic newTopic (it doesn't exist)
> > > > 2) kafka-console-producer --topic newTopic
> > > > 3) Send the first messages
> > > > 4) I can see this in the log and the consumer never receives this
> first
> > > > message
> > > >
> > > > [2014-03-13 10:24:07,223] WARN Error while fetching metadata
> > > > [{TopicMetadata for topic test3 ->
> > > > No partition metadata for topic test3 due to
> > > > kafka.common.LeaderNotAvailableException}] for topic [test3]: class
> > > > kafka.common.LeaderNotAvailableException
> > > >  (kafka.producer.BrokerPartitionInfo)
> > > > [2014-03-13 10:24:07,237] WARN Error while fetching metadata
> > > > [{TopicMetadata for topic test3 ->
> > > > No partition metadata for topic test3 due to
> > > > kafka.common.LeaderNotAvailableException}] for topic [test3]: class
> > > > kafka.common.LeaderNotAvailableException
> > > >  (kafka.producer.BrokerPartitionInfo)
> > > > [2014-03-13 10:24:07,238] ERROR Failed to collate messages by topic,
> > > > partition due to: Failed to fetch topic metadata for topic: test3
> > > > (kafka.producer.async.DefaultEventHandler)
> > > >
> > > >
> > > > 5) send more messages
> > > > 6) consumer receives all the messages (but the first one).
> > > >
> > > >
> > > > The same issue appears when you first start the producer and then the
> > > > consumer (on a new topic).
> > > >
> > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>
>
>
> --
> -- Guozhang
>

Re: consumer reaction to auto-create topics (0.8.1)

Posted by Guozhang Wang <wa...@gmail.com>.
Hi David,

This issue should only happen if the following conditions are satisfied

1) The consumer does not have any offset stored in ZK for this topic when
it first fetches messages for it.
2) The consumer's default offset reset value is "latest".
3) After the topic is created, it will take a bit time for the metadata to
be propagated to all the brokers and hence the consumers know it can now
consume the message, and then when the first fetch request comes with no
specified offset, it will use the "latest" offset, and hence the first few
messages that are already appended during the metadata propagation will be
lost.

For your case, if the services can go up and down but with the same topics
all the time, and as long as you store the offsets somewhere (either in ZK
or by yourself) you should not hit this issue.

Guozhang


On Thu, Mar 13, 2014 at 9:12 AM, David Morales de Frías <
dmorales@paradigmatecnologico.com> wrote:

> Hello Guozhang,
>
> We are using Kafka to connect several actors in our project (osgi, storm,
> vert.x web app...), all of them being consumers and producers of different
> topics. These services can go up and down in different  moments.
>
> Anyway, the issue is the same when you first start the consumer and then
> the producer and the other way (tested with console producer and consumer).
> So it doesn't depend on the order of services.
>
>
> Thanks.
>
>
>
>
>
>
>
>
> 2014-03-13 17:03 GMT+01:00 Guozhang Wang <wa...@gmail.com>:
>
> > Hello David,
> >
> > This is a known issue with consumer starting before the topic is created:
> >
> > https://issues.apache.org/jira/browse/KAFKA-1006
> >
> > And we are still figuring the right way to fix it. Could you describe
> your
> > use cases where you need to start the consumer beforehand?
> >
> > Guozhang
> >
> >
> > On Thu, Mar 13, 2014 at 2:42 AM, David Morales de Frías <
> > dmorales@paradigmatecnologico.com> wrote:
> >
> > > Hi there,
> > >
> > > A few days ago i have written about an issue (0.8.0) when auto-create
> > topic
> > > is enabled and consumer starts before producer (consumer never sees any
> > > message and they seem to be lost even by the brokers)
> > >
> > > Now i have done the same test with 0.8.1 and the issue is partially
> fixed
> > > but now the consumer never sees the first message in the topic.
> > >
> > > These are the steps:
> > >
> > > 1) kafka-console-consumer --topic newTopic (it doesn't exist)
> > > 2) kafka-console-producer --topic newTopic
> > > 3) Send the first messages
> > > 4) I can see this in the log and the consumer never receives this first
> > > message
> > >
> > > [2014-03-13 10:24:07,223] WARN Error while fetching metadata
> > > [{TopicMetadata for topic test3 ->
> > > No partition metadata for topic test3 due to
> > > kafka.common.LeaderNotAvailableException}] for topic [test3]: class
> > > kafka.common.LeaderNotAvailableException
> > >  (kafka.producer.BrokerPartitionInfo)
> > > [2014-03-13 10:24:07,237] WARN Error while fetching metadata
> > > [{TopicMetadata for topic test3 ->
> > > No partition metadata for topic test3 due to
> > > kafka.common.LeaderNotAvailableException}] for topic [test3]: class
> > > kafka.common.LeaderNotAvailableException
> > >  (kafka.producer.BrokerPartitionInfo)
> > > [2014-03-13 10:24:07,238] ERROR Failed to collate messages by topic,
> > > partition due to: Failed to fetch topic metadata for topic: test3
> > > (kafka.producer.async.DefaultEventHandler)
> > >
> > >
> > > 5) send more messages
> > > 6) consumer receives all the messages (but the first one).
> > >
> > >
> > > The same issue appears when you first start the producer and then the
> > > consumer (on a new topic).
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>



-- 
-- Guozhang

Re: consumer reaction to auto-create topics (0.8.1)

Posted by Joe Stein <jo...@stealth.ly>.
One system I am working on we have something similar however we are
creating topics like how the TopicCommand does but different.  We keep
those operations persisted in a meta store so consumers that have to-do
something know that everything is all ready for them (as often it is not
just Kafka) for them to-do "their thing".  In our case we are spinning up
consumers within our Mesos slaves... we don't want that to happen unless
there is a topic present and is showing up in our monitoring first (test
messages going in first).

Think of it like the producer having to request permission for a new topic
to be created and blocking waiting for that to happen (first one creates
the topic and the other request ack back right away as topic is created).
 There may be good reason for that topic to not be created i.e. half your
hardware just "conked" out and Mesos is rebalancing all non priority
services killing them to start up more priority services... you also have
to make sure then that no new low priority services are created.

I think a lot of this comes down to different domain specific build systems
and implementing these behaviors within them, once integration hooks are
understood (and easily available). Maybe we need to make the command tools
more plug-able?  Without trying to become "plug-able happy" we could
broaden the convo looking at command tools as a whole.  There is thread
going for that lets lump this ticket into it? At least so it can solved as
part of a larger piece?  Or maybe just make it more like an API and the
project ships with a shell wrapper for the API?  Not far off from that now.

There was also a patch for a different restart mechanism
https://issues.apache.org/jira/browse/KAFKA-1300  that looked interesting
that would be part of it too I would think? So folks could share parts of
the implementation but have a way for it to work smoothly in their build
process? Tools out of the box and then also ones that can be plugged in.

/*******************************************
 Joe Stein
 Founder, Principal Consultant
 Big Data Open Source Security LLC
 http://www.stealth.ly
 Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
********************************************/


On Thu, Mar 13, 2014 at 12:12 PM, David Morales de Frías <
dmorales@paradigmatecnologico.com> wrote:

> Hello Guozhang,
>
> We are using Kafka to connect several actors in our project (osgi, storm,
> vert.x web app...), all of them being consumers and producers of different
> topics. These services can go up and down in different  moments.
>
> Anyway, the issue is the same when you first start the consumer and then
> the producer and the other way (tested with console producer and consumer).
> So it doesn't depend on the order of services.
>
>
> Thanks.
>
>
>
>
>
>
>
>
> 2014-03-13 17:03 GMT+01:00 Guozhang Wang <wa...@gmail.com>:
>
> > Hello David,
> >
> > This is a known issue with consumer starting before the topic is created:
> >
> > https://issues.apache.org/jira/browse/KAFKA-1006
> >
> > And we are still figuring the right way to fix it. Could you describe
> your
> > use cases where you need to start the consumer beforehand?
> >
> > Guozhang
> >
> >
> > On Thu, Mar 13, 2014 at 2:42 AM, David Morales de Frías <
> > dmorales@paradigmatecnologico.com> wrote:
> >
> > > Hi there,
> > >
> > > A few days ago i have written about an issue (0.8.0) when auto-create
> > topic
> > > is enabled and consumer starts before producer (consumer never sees any
> > > message and they seem to be lost even by the brokers)
> > >
> > > Now i have done the same test with 0.8.1 and the issue is partially
> fixed
> > > but now the consumer never sees the first message in the topic.
> > >
> > > These are the steps:
> > >
> > > 1) kafka-console-consumer --topic newTopic (it doesn't exist)
> > > 2) kafka-console-producer --topic newTopic
> > > 3) Send the first messages
> > > 4) I can see this in the log and the consumer never receives this first
> > > message
> > >
> > > [2014-03-13 10:24:07,223] WARN Error while fetching metadata
> > > [{TopicMetadata for topic test3 ->
> > > No partition metadata for topic test3 due to
> > > kafka.common.LeaderNotAvailableException}] for topic [test3]: class
> > > kafka.common.LeaderNotAvailableException
> > >  (kafka.producer.BrokerPartitionInfo)
> > > [2014-03-13 10:24:07,237] WARN Error while fetching metadata
> > > [{TopicMetadata for topic test3 ->
> > > No partition metadata for topic test3 due to
> > > kafka.common.LeaderNotAvailableException}] for topic [test3]: class
> > > kafka.common.LeaderNotAvailableException
> > >  (kafka.producer.BrokerPartitionInfo)
> > > [2014-03-13 10:24:07,238] ERROR Failed to collate messages by topic,
> > > partition due to: Failed to fetch topic metadata for topic: test3
> > > (kafka.producer.async.DefaultEventHandler)
> > >
> > >
> > > 5) send more messages
> > > 6) consumer receives all the messages (but the first one).
> > >
> > >
> > > The same issue appears when you first start the producer and then the
> > > consumer (on a new topic).
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>

Re: consumer reaction to auto-create topics (0.8.1)

Posted by David Morales de Frías <dm...@paradigmatecnologico.com>.
Hello Guozhang,

We are using Kafka to connect several actors in our project (osgi, storm,
vert.x web app...), all of them being consumers and producers of different
topics. These services can go up and down in different  moments.

Anyway, the issue is the same when you first start the consumer and then
the producer and the other way (tested with console producer and consumer).
So it doesn't depend on the order of services.


Thanks.








2014-03-13 17:03 GMT+01:00 Guozhang Wang <wa...@gmail.com>:

> Hello David,
>
> This is a known issue with consumer starting before the topic is created:
>
> https://issues.apache.org/jira/browse/KAFKA-1006
>
> And we are still figuring the right way to fix it. Could you describe your
> use cases where you need to start the consumer beforehand?
>
> Guozhang
>
>
> On Thu, Mar 13, 2014 at 2:42 AM, David Morales de Frías <
> dmorales@paradigmatecnologico.com> wrote:
>
> > Hi there,
> >
> > A few days ago i have written about an issue (0.8.0) when auto-create
> topic
> > is enabled and consumer starts before producer (consumer never sees any
> > message and they seem to be lost even by the brokers)
> >
> > Now i have done the same test with 0.8.1 and the issue is partially fixed
> > but now the consumer never sees the first message in the topic.
> >
> > These are the steps:
> >
> > 1) kafka-console-consumer --topic newTopic (it doesn't exist)
> > 2) kafka-console-producer --topic newTopic
> > 3) Send the first messages
> > 4) I can see this in the log and the consumer never receives this first
> > message
> >
> > [2014-03-13 10:24:07,223] WARN Error while fetching metadata
> > [{TopicMetadata for topic test3 ->
> > No partition metadata for topic test3 due to
> > kafka.common.LeaderNotAvailableException}] for topic [test3]: class
> > kafka.common.LeaderNotAvailableException
> >  (kafka.producer.BrokerPartitionInfo)
> > [2014-03-13 10:24:07,237] WARN Error while fetching metadata
> > [{TopicMetadata for topic test3 ->
> > No partition metadata for topic test3 due to
> > kafka.common.LeaderNotAvailableException}] for topic [test3]: class
> > kafka.common.LeaderNotAvailableException
> >  (kafka.producer.BrokerPartitionInfo)
> > [2014-03-13 10:24:07,238] ERROR Failed to collate messages by topic,
> > partition due to: Failed to fetch topic metadata for topic: test3
> > (kafka.producer.async.DefaultEventHandler)
> >
> >
> > 5) send more messages
> > 6) consumer receives all the messages (but the first one).
> >
> >
> > The same issue appears when you first start the producer and then the
> > consumer (on a new topic).
> >
>
>
>
> --
> -- Guozhang
>

Re: consumer reaction to auto-create topics (0.8.1)

Posted by Guozhang Wang <wa...@gmail.com>.
Hello David,

This is a known issue with consumer starting before the topic is created:

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

And we are still figuring the right way to fix it. Could you describe your
use cases where you need to start the consumer beforehand?

Guozhang


On Thu, Mar 13, 2014 at 2:42 AM, David Morales de Frías <
dmorales@paradigmatecnologico.com> wrote:

> Hi there,
>
> A few days ago i have written about an issue (0.8.0) when auto-create topic
> is enabled and consumer starts before producer (consumer never sees any
> message and they seem to be lost even by the brokers)
>
> Now i have done the same test with 0.8.1 and the issue is partially fixed
> but now the consumer never sees the first message in the topic.
>
> These are the steps:
>
> 1) kafka-console-consumer --topic newTopic (it doesn't exist)
> 2) kafka-console-producer --topic newTopic
> 3) Send the first messages
> 4) I can see this in the log and the consumer never receives this first
> message
>
> [2014-03-13 10:24:07,223] WARN Error while fetching metadata
> [{TopicMetadata for topic test3 ->
> No partition metadata for topic test3 due to
> kafka.common.LeaderNotAvailableException}] for topic [test3]: class
> kafka.common.LeaderNotAvailableException
>  (kafka.producer.BrokerPartitionInfo)
> [2014-03-13 10:24:07,237] WARN Error while fetching metadata
> [{TopicMetadata for topic test3 ->
> No partition metadata for topic test3 due to
> kafka.common.LeaderNotAvailableException}] for topic [test3]: class
> kafka.common.LeaderNotAvailableException
>  (kafka.producer.BrokerPartitionInfo)
> [2014-03-13 10:24:07,238] ERROR Failed to collate messages by topic,
> partition due to: Failed to fetch topic metadata for topic: test3
> (kafka.producer.async.DefaultEventHandler)
>
>
> 5) send more messages
> 6) consumer receives all the messages (but the first one).
>
>
> The same issue appears when you first start the producer and then the
> consumer (on a new topic).
>



-- 
-- Guozhang