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 <
dmora...@paradigmatecnologico.com> wrote:
> Hi there,
>
> A few days ago i have written about an issue (0.8.0) when auto-
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
to
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
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
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
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
Thanks for your support.
I have explained the issue by using the console producer&consumer, in terms
of simplicity, but i have just found it in my application.
I will update this thread later with more tests.
Regards.
2014-03-06 18:56 GMT+01:00 Neha Narkhede :
> I've seen this behavior whe
I've seen this behavior when the broker was not functional. Basically what
you see is that the console producer appears to have sent some messages.
These messages may not have reached the server. Due to the bad state of the
server, the consumer cannot get these messages and also runs into issues
wh
Ok. The is fixed in the 0.8.1 release, which is being voted now.
Thanks,
Jun
On Thu, Mar 6, 2014 at 9:34 AM, David Morales de Frías <
dmora...@paradigmatecnologico.com> wrote:
> 0.8, thanks.
>
>
> 2014-03-06 18:27 GMT+01:00 Jun Rao :
>
> > Which version of Kafka are you using?
> >
> > Thanks,
0.8, thanks.
2014-03-06 18:27 GMT+01:00 Jun Rao :
> Which version of Kafka are you using?
>
> Thanks,
>
> Jun
>
>
> On Thu, Mar 6, 2014 at 2:10 AM, David Morales de Frías <
> dmora...@paradigmatecnologico.com> wrote:
>
> > Hi there,
> >
> > If i start a consumer in a non-existent topic (auto-cre
Which version of Kafka are you using?
Thanks,
Jun
On Thu, Mar 6, 2014 at 2:10 AM, David Morales de Frías <
dmora...@paradigmatecnologico.com> wrote:
> Hi there,
>
> If i start a consumer in a non-existent topic (auto-create true) before the
> producer, the consumer never gets the messages.
>
>
11 matches
Mail list logo