Yes it is supported in 0.8.2-beta. It is documented on the site - you
will need to set offsets.storage to kafka.
On Thu, Feb 19, 2015 at 03:57:31PM -0500, Matthew Butt wrote:
I'm having a hard time figuring out if the new Kafka-based offset
management in the high-level Scala Consumer is
On Fri, Feb 06, 2015 at 12:43:37AM -0500, Jason Rosenberg wrote:
I'm not sure what you mean by 'default' behavior 'only if' offset.storage
is kafka. Does that mean the 'default' behavior is 'false' if
offset.storage is 'zookeeper'? Can that be clarified in the config
documentation section?
On Thu, Feb 05, 2015 at 11:57:15PM -0800, Joel Koshy wrote:
On Fri, Feb 06, 2015 at 12:43:37AM -0500, Jason Rosenberg wrote:
I'm not sure what you mean by 'default' behavior 'only if' offset.storage
is kafka. Does that mean the 'default' behavior is 'false' if
offset.storage is
On Thu, Feb 5, 2015 at 9:52 PM, Joel Koshy jjkosh...@gmail.com wrote:
Ok, so it looks like the default settings are:
offset.storage = zookeeper
dual.commit.enabled = true
The doc for 'dual.commit.enabled' seems to imply (but doesn't clearly
state) that it will only apply if
Ok, so it looks like the default settings are:
offset.storage = zookeeper
dual.commit.enabled = true
The doc for 'dual.commit.enabled' seems to imply (but doesn't clearly
state) that it will only apply if offset.storage = kafka. Is that right?
(I'm guessing not)
*If you are using kafka* as
This is documented in the official docs:
http://kafka.apache.org/documentation.html#distributionimpl
On Thu, Feb 05, 2015 at 01:23:01PM -0500, Jason Rosenberg wrote:
What are the defaults for those settings (I assume it will be to continue
using only zookeeper by default)?
Also, if I have a
This is what I've found so far.
https://cwiki.apache.org/confluence/display/KAFKA/Committing+and+fetching+consumer+offsets+in+Kafka
The high-level consumer just worked for me by setting offsets.storage =
kafka.
Scroll down to the offsets.* config params.
Hi,
For 0.8.2, one of the features listed is:
- Kafka-based offset storage.
Is there documentation on this (I've heard discussion of it of course)?
Also, is it something that will be used by existing consumers when they
migrate up to 0.8.2? What is the migration process?
Thanks,
Jason
There should probably be a wiki page started for this so we have the details in
one place. The same question was asked on Freenode IRC a few minutes ago. :)
A summary of the migration procedure is:
1) Upgrade your brokers and set dual.commit.enabled=false and
offsets.storage=zookeeper (Commit
Ok, so it looks like the default settings are:
offset.storage = zookeeper
dual.commit.enabled = true
The doc for 'dual.commit.enabled' seems to imply (but doesn't clearly
state) that it will only apply if offset.storage = kafka. Is that right?
(I'm guessing not)
dual.commit.enabled
Thanks Jon. I updated the FAQ with your procedure:
https://cwiki.apache.org/confluence/display/KAFKA/FAQ#FAQ-HowdowemigratetocommittingoffsetstoKafka(ratherthanZookeeper)in0.8.2
?
On Thu, Feb 5, 2015 at 9:16 AM, Jon Bringhurst
jbringhu...@linkedin.com.invalid wrote:
There should probably be a
What are the defaults for those settings (I assume it will be to continue
using only zookeeper by default)?
Also, if I have a cluster of consumers sharing the same groupId, and I
update them via a rolling release, will it be a problem during the rolling
restart if there is inconsistency in the
12 matches
Mail list logo