Offset Storage

2015-09-18 Thread Andy Chambers
Hi All, I'm a little bit confused when I read about offset management in kafka. A google search turns up the wiki page that provides some example code about how to use the offset API[1] but also buried in the docs is the configuration option "offset.storage"[2]. So my question is "can we achieve

aggregate clusters and mirror maker

2015-09-18 Thread Doug Tomm
hello, i have some questions about setting up an aggregate cluster in a data center for mirror maker. i haven't found answers for these, so i'm appealing to the community at large :-) does the aggregate cluster need its own zookeeper service, or can it use the same service used by the local

Re: OffsetCommitRequest not received

2015-09-18 Thread Vadim Bobrov
Interestingly, just noticed - at the exact time the commit request should have arrived at the server (15:44:14,365) there are a few moments of silence in both kafka-request.log and server.log: kafka-request.log: [2015-09-18 15:44:14,355] TRACE 500 bytes read from /192.168.221.91:60967 (kafka.netwo

Will Mirror Maker only support 1 to 1?

2015-09-18 Thread Xiang Zhou (Samuel)
Hi, folks, I found that the Mirror Maker docs should be updated(KAFKA-2449) since it mentions N source to 1 destination will not be supported. So is that true it only support 1 source to 1 destination in 0.9.0? Or it will be extended to support N source to N dest? Thanks, Samuel

OffsetCommitRequest not received

2015-09-18 Thread Vadim Bobrov
Hi, I am trying to commit offsets manually to kafka and it sometimes works sometimes not. What I found in kafka-request.log for the successful attempt is this: [2015-09-18 15:22:38,031] TRACE [KafkaApi-0] Handling request: Name: OffsetCommitRequest; Version: 1; CorrelationId: 0; ClientId: chaos-t

Re: 0.9.0.0 remaining jiras

2015-09-18 Thread Ismael Juma
Do we want to deprecate anything before 0.9.0 is released? Maybe the old producer? Ismael On Thu, Sep 17, 2015 at 8:07 PM, Dong Lin wrote: > Hey Jun, > > Should we also include https://issues.apache.org/jira/browse/KAFKA-2390 in > 0.9.0? Becket told me that this is one of those patches ( > http

Re: automatically consume from all topics

2015-09-18 Thread Alexis Midon
Thanks Tao. for reference, here is the corresponding listener https://github.com/apache/kafka/blob/9dbeb71ab258955e04b46991c1baf880b07633f4/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala#L1000-L1002 On Sat, Sep 12, 2015 at 5:42 AM, tao xiao wrote: > It is handled by the impl

Re: Log Cleaner Thread Stops

2015-09-18 Thread Todd Palino
I think the last major issue with log compaction (that it couldn't handle compressed messages) was committed as part of https://issues.apache.org/jira/browse/KAFKA-1374 in August, but I'm not certain what version this will end up in. It may be part of 0.8.2.2. Regardless, you'll probably be OK now

Re: Log Cleaner Thread Stops

2015-09-18 Thread John Holland
Thanks! I did what you suggested and it worked except it was necessary for me to remove the cleaner-offset-checkpoint file from the data directory and restart the servers. The log indicates all is well. Do you know what version the fix to this will be in? I'm not looking forward to dealing with

Re: Log Cleaner Thread Stops

2015-09-18 Thread Todd Palino
Yes, this is a known concern, and it should be fixed with recent commits. In the meantime, you'll have to do a little manual cleanup. The problem you're running into is a corrupt message in the offsets topic. We've seen this a lot. What you need to do is set the topic configuration to remove the c

Useful metric to check slow ISR catchup

2015-09-18 Thread Prabhjot Bharaj
Hi, I've noticed that 1 follower replica node out of my kafka cluster catches up to the data form the leader pretty slowly. My topic has just 1 partition with 3 replicas. One other follower replica gets the full data from the leader pretty instantly It takes around 22 minutes to catch up 500MB of

Log Cleaner Thread Stops

2015-09-18 Thread John Holland
I've been experiencing this issue across several of our environments ever since we enabled the log cleaner for the __consumer_offsets topic. We are on version 0.8.2.1 of kafka, using the new producer. All of our consumers are set to commit to kafka only. Below is the stack trace in the log I've

MirrorMaker kafka.message.InvalidMessageException

2015-09-18 Thread Jörg Wagner
Hey everyone! One of my Mirrormakers is exiting with the following error: [2015-09-18 11:27:35,591] FATAL [mirrormaker-consumer-0] Stream unexpectedly exited. (kafka.tools.MirrorMaker$ConsumerThread) kafka.message.InvalidMessageException: Message is corrupt (stored crc = 3256823012, computed c

Re: Does Kafka support adding custom metadata to commits?

2015-09-18 Thread Petr Novak
Thanks so much. On Thu, Sep 17, 2015 at 8:36 PM, Helleren, Erik wrote: > Looking at the docs here ( > https://cwiki.apache.org/confluence/display/KAFKA/Committing+and+fetching+c > onsumer+offsets+in+Kafka), its possible to attach metadata as a string to > each partition for the consumer group us