Re: are kafka consumer apps guaranteed to see msgs at least once?

2013-11-22 Thread Imran Rashid
MessageAndMetadata includes the partition and offset along w/ each message. So this is enough for the workers to track which partitions they need to update, and by what amount. There is the possibility that you'd commit an update for a partition that you *used* to own, but has since be rebalanced

Re: are kafka consumer apps guaranteed to see msgs at least once?

2013-11-22 Thread Jason Rosenberg
I think that the problem is, you don't know which partitions a thread is currently 'owning', and therefore, you don't know which partitions you can commit to. On Fri, Nov 22, 2013 at 5:28 PM, Imran Rashid wrote: > I don't think I need control over which partitions are processed by a > thread --

Re: are kafka consumer apps guaranteed to see msgs at least once?

2013-11-22 Thread Imran Rashid
I don't think I need control over which partitions are processed by a thread -- however, I do need the partition --> thread assignment to be consistent. I didn't realize that assignment could change, I guess that could mean my solution might not work. It depends a bit on what happens during reass

Re: are kafka consumer apps guaranteed to see msgs at least once?

2013-11-22 Thread Jason Rosenberg
With the high-level consumer, the design is that you really don't have programmatic control over which threads will process which partitions (and usually multiple per thread). Furthermore, the high-level consumer can re-balance and redistribute which threads (and which other processes) are consumi

Re: any roadmap for releases?

2013-11-22 Thread Demian Berjman
Neha and Jun, thanks for the quick response! Demian On Fri, Nov 22, 2013 at 12:50 PM, Jun Rao wrote: > Demian, > > I added the tentative future release plan in the following wiki. > > https://cwiki.apache.org/confluence/display/KAFKA/Future+release+plan > > Thanks, > > Jun > > > On Fri, Nov 2

RE: retention size

2013-11-22 Thread Yu, Libo
I did a restart and the issue was gone. It could be that we changed the retention size and did not restart the brokers to pick up the change. Thanks for your help. Regards, Libo -Original Message- From: Yu, Libo [ICG-IT] Sent: Friday, November 22, 2013 10:53 AM To: 'users@kafka.apache

Re: Kafka/Hadoop consumers and producers

2013-11-22 Thread Abhi Basu
I agree with you. We are looking for a simple solution for data from Kafka to Hadoop. I have tried using Camus earlier (Non-Avro) and documentation is lacking to make it work correctly, as we do not need to introduce another component to the solution. In the meantime, can the Kafka Hadoop Consu

RE: retention size

2013-11-22 Thread Yu, Libo
I am using 0.8-beta1. I used only one producer and compression was not turned on. The topic was newly created. The default retention size applied so each of three partitions Is 3Gb. I checked log files for all three partitions and all of them are 1.4G. Regards, Libo -Original Message-

Re: any roadmap for releases?

2013-11-22 Thread Jun Rao
Demian, I added the tentative future release plan in the following wiki. https://cwiki.apache.org/confluence/display/KAFKA/Future+release+plan Thanks, Jun On Fri, Nov 22, 2013 at 6:46 AM, Demian Berjman wrote: > Hi. > > There is any plan for a beta release of 0.8.1? > > We have special inter

Re: any roadmap for releases?

2013-11-22 Thread Neha Narkhede
We are in the middle of a deployment of 0.8.1 at LinkedIn. I think we may be able to release something by the end of the year. Thanks, Neha On Nov 22, 2013 9:47 AM, "Demian Berjman" wrote: > Hi. > > There is any plan for a beta release of 0.8.1? > > We have special interest in this improvement >

Re: Consumer not consuming for 1st time

2013-11-22 Thread Jun Rao
That error means the consumer has never checkpointed any offset in ZK. Thanks, Jun On Thu, Nov 21, 2013 at 10:31 PM, Tarang Dawer wrote: > Hi Jun > > I tried the ConsumerOffsetChecker script for checking the offsets > > but there also , got an exception , with no information regarding the > of

any roadmap for releases?

2013-11-22 Thread Demian Berjman
Hi. There is any plan for a beta release of 0.8.1? We have special interest in this improvement https://issues.apache.org/jira/browse/KAFKA-554 Thanks,

Re: are kafka consumer apps guaranteed to see msgs at least once?

2013-11-22 Thread Imran Rashid
hmm, ok, after some more thought I see what you mean.I was thinking async because of a more complicated use case -- my app actually doesn't process messages one at a time, it builds a batch of messages, merges the batches from all threads, and process those batches, and thats when it shoudl commit.

Re: Consumer not consuming for 1st time

2013-11-22 Thread Tarang Dawer
The data is being produced in the topic, as i can view the topic-partitions directories in the data directory. I also noticed that the above scenario happens only when consumer is first started and later the data is inserted for the first time, but not when data is already present, then it consume