offset storage as kafka with zookeeper 3.4.6

2015-06-11 Thread Kris K
I am trying to migrate the offset storage to kafka (3 brokers of version 0.8.2.1) using the consumer property offsets.storage=kafka. I noticed that a new topic, __consumer_offsets got created. But nothing is being written to this topic, while the consumer offsets continue to reside on zookeeper.

Kafka 0.8.1.1 arbitrarily increased replication factor

2015-06-11 Thread Yury Ruchin
Hello, I've run into a weird situation with Kafka 0.8.1.1. I had an operating cluster which I wanted to extend with new brokers. The sequence was as follows: 1. I added the brokers to cluster and ensured that they appeared under /brokers/ids. 2. Ran reassign-partitions tool to redistribute the d

Re: If you run Kafka in AWS or Docker, how do you persist data?

2015-06-11 Thread Domen Pogacnik
@Jeff Schroeder: I’m trying to do a similar thing: running Kafka brokers in docker under marathon under Mesos. I’m wondering if you're able to do a rolling deploy in a way that Marathon waits for the old broker instances to replicate the data to the new ones before killing them? Best, Domen

Re: offset storage as kafka with zookeeper 3.4.6

2015-06-11 Thread Joel Koshy
> Is it mandatory to use the zookeeper that comes with kafka for offset > storage to be migrated to kafka? If you want to "move" offsets from zookeeper to Kafka then yes you need to have a phase where all consumers in your group set dual commit to true. If you are starting a fresh consumer group th

Re: How to prevent custom Partitioner from increasing the number of producer's requests?

2015-06-11 Thread Sebastien Falquier
Thanks Jiangjie. The new producer is exactly what I was looking for and it works perfectly in production. It should be more documented on the official site. Jason : you're right, I missed a point with my AtomicIntegers. Regards, Sebastien 2015-06-04 20:02 GMT+02:00 Jason Rosenberg : > Sebastien

Re: Increased replication factor. Replication didn't happen!

2015-06-11 Thread Joel Koshy
This may help: http://kafka.apache.org/documentation.html#basic_ops_increase_replication_factor On Thu, Jun 11, 2015 at 11:20:05AM +0800, Shady Xu wrote: > Right now, Kafka topics do not support changing replication factor or > partition number after creation. The kafka-reassign-partitions.sh too

Re: offset storage as kafka with zookeeper 3.4.6

2015-06-11 Thread Kris K
>If you want to "move" offsets from zookeeper to Kafka then yes you >need to have a phase where all consumers in your group set dual commit >to true. If you are starting a fresh consumer group then you can >turn off dual-commit. I followed these steps to move the offsets from zookeeper to kafka: 1.

Re: If you run Kafka in AWS or Docker, how do you persist data?

2015-06-11 Thread Joe Stein
If your running kafka on mesos you should use the framework https://github.com/mesos/kafka as the scheduler is better opinionated than generic launch on marathon. ~ Joestein On Jun 11, 2015 10:26 AM, "Domen Pogacnik" < domen.pogac...@rocket-internet.de> wrote: > @Jeff Schroeder: > > I’m trying to

Re: Kafka 0.8.3 - New Consumer - user implemented partition.assignment.strategies?

2015-06-11 Thread Johansson, Olof
Thanks Guozhang, I agree that it seems to be a common reassignment strategy that should be one of the pre-defined strategies. Do you have a Jira ticket for this specific case, and/or a Jira ticket to add user defined partitions.assignment.strategies that I can watch? / Olof On 10/06/2015 14:35,

Re: How to manage the consumer group id?

2015-06-11 Thread Todd Palino
When we need to delete a group, we do it in Zookeeper directly. When we need to roll back offsets, we use the Import/Export tool classes to do it, because it's a little more efficient than working in Zookeeper. You can find the details on the tools at https://cwiki.apache.org/confluence/display/KAF

Re: Increased replication factor. Replication didn't happen!

2015-06-11 Thread Dillian Murphey
Oh, hmm. There is even documentation on it: http://kafka.apache.org/documentation.html#basic_ops_increase_replication_factor Having a difficult time sifting through the logs. Is this not a common operation? Do users normally just delete the topic and create a new one?? On Wed, Jun 10, 2015

Re: Increased replication factor. Replication didn't happen!

2015-06-11 Thread Adam Dubiel
I just tried it out on my 0.8.2 cluster and it worked just fine - the ISR grew, replica factor changed and data was physically moved to new brokers. Was there not output/no logs? I see things like INFO Created log for partition [topicName,7] in /opt/kafka/ with properties {.. some json} in se

Re: Kafka as an event store for Event Sourcing

2015-06-11 Thread Ben Kirwin
As it happens, I submitted a ticket for this feature a couple days ago: https://issues.apache.org/jira/browse/KAFKA-2260 Couldn't find any existing proposals for similar things, but it's certainly possible they're out there... On the other hand, I think you can solve your particular issue by ref

Re: Kafka as an event store for Event Sourcing

2015-06-11 Thread Gwen Shapira
Hi Ben, Thanks for creating the ticket. Having check-and-set capability will be sweet :) Are you planning to implement this yourself? Or is it just an idea for the community? Gwen On Thu, Jun 11, 2015 at 8:01 PM, Ben Kirwin wrote: > As it happens, I submitted a ticket for this feature a couple