Haven't used either one now. Sounds like 0.8.2.1 will help.
We are using the High Level Consumer generally but are thinking to use the 
SimpleConsumer on failure recovery to set the offsets.
Is that the recommended approach for this use case?
Thanks.
-Suren
 

     On Thursday, February 19, 2015 9:40 AM, Joel Koshy <jjkosh...@gmail.com> 
wrote:
   

 Are you using it from Java or Scala? i.e., are you using the
javaapi.SimpleConsumer or kafka.consumer.SimpleConsumer

In 0.8.2 javaapi we explicitly set version 0 of the
OffsetCommitRequest/OffsetFetchRequest which means it will
commit/fetch to/from ZooKeeper only. If you use the scala API you can
create an OffsetCommitRequest with version set to 1 (which will allow
you to commit to Kafka).

Since we are doing an 0.8.2.1 release we will make the above more
consistent. i.e., you can create OffsetCommitRequests with version 1
even from the javaapi. I will be updating the documentation on this to
make it clearer.

Thanks,

Joel

On Thu, Feb 19, 2015 at 02:28:32PM +0000, Suren wrote:
> Joel,
> Looking at SimpleConsumer in the 0.8.2 code, it is using OffsetCommitRequest 
> and sending that over to a broker.
> Is the broker storing that in ZK?
> -Suren
>  
> 
>      On Tuesday, February 17, 2015 12:22 PM, Joel Koshy <jjkosh...@gmail.com> 
>wrote:
>    
> 
>  Hi Chris,
> 
> In 0.8.2, the simple consumer Java API supports committing/fetching
> offsets that are stored in ZooKeeper. You don't need to issue any
> ConsumerMetadataRequest for this. Unfortunately, the API currently
> does not support fetching offsets that are stored in Kafka.
> 
> Thanks,
> 
> Joel
> 
> On Mon, Feb 16, 2015 at 05:02:08PM -0500, Christopher Piggott wrote:
> > Hi,
> > 
> > I am still using 0.8.1.1 because of the CPU use concerns.
> > 
> > I'm confused about why the SimpleConsumer has:
> > 
> > OffsetCommitResponse commitOffsets(OffsetCommitRequest request)
> > 
> > and
> > 
> > OffsetFetchResponse fetchOffsets(OffsetFetchRequest request)
> > 
> > but no way that I can see to issue a ConsumerMetadataRequest, which is
> > what I think when restarting my consumers so that they can begin
> > working where they last left off (in the event that they were stopped
> > for a while then restarted some time later, and new messages had come
> > in).
> > 
> > The fetchOffsets() works on time, usually it looks like you send it
> > Earliest or Latest (beginning or end of what's currently in the
> > stream).
> > 
> > I realize the documentation says this:
> > 
> > 
> > > *Downsides of using SimpleConsumer*The SimpleConsumer does require a 
> > > significant amount of work not needed in the Consumer Groups:
> > >
> > >    1. You must keep track of the offsets in your application to know 
> > >where you left off consuming.
> > >
> > > But that's not really quite true ... not as long as commitOffsets() has 
> > > been provided.  It seems the SimpleConsumer provides you with a solution 
> > > to only one half of the problem of offset management.
> > 
> > Using some zookeeper python scripts I wrote I can see that the
> > commitOffsets() is doing its job and writing to
> > 
> > 
> > /consumers/myGroupId/offsets/myTopic/0
> > 
> > 
> > That has this value:
> > 
> > ('32757408', ZnodeStat(czxid=2211679, mzxid=14779964, ctime=1423777630972,
> > > mtime=1424122117397, version=12568262, cversion=0, aversion=0,
> > > ephemeralOwner=0, dataLength=8, numChildren=0, pzxid=2211679))
> > 
> > 
> > Now the question is just how to retrieve that - do I really have to
> > have my client connect to ZK directly?  If that's the case, future
> > upgrades would break (e.g. 0.8.2 having its own storage for commit
> > watermarks).
> > 
> > 
> > What was the intent here, and what's the advice on how to proceed
> > being that 0.8.2 is in an iffy state right now?
> > 
> > 
> > --Chris
> 
> 
> 
>    



   

Reply via email to