leaderless topicparts after single node failure: how to repair?

2014-12-09 Thread Neil Harkins
Hi. We've suffered a single node HW failure (broker_id 4) with at least 2 replicas of each topic partition, but some topic parts are now leaderless (all were across 4,5): Topic: topic.with.two.replicas Partition: 0Leader: -1 Replicas: 4,5 Isr: on broker 5, we see warnings like this in t

Re: nulls found in topic, created by recovery?

2014-11-06 Thread Neil Harkins
ou try to run a consumer through the topic's > data or alternately the DumpLogSegments tool? > > On Thu, Nov 6, 2014 at 12:56 PM, Neil Harkins wrote: >> >> Hi all. I saw something weird yesterday on our "leaf" instances >> which run kafka 0.7.2 (and mirr

nulls found in topic, created by recovery?

2014-11-06 Thread Neil Harkins
Hi all. I saw something weird yesterday on our "leaf" instances which run kafka 0.7.2 (and mirror to kafka 0.8 via our custom code). I fully realize everyone's instinctual response is "upgrade, already.", but I'd like to have an internals discussion to better understand what happened, as I suspect

Re: How many partition can one single machine handle in Kafka?

2014-10-22 Thread Neil Harkins
I've been thinking about this recently. If kafka provided cmdline hooks to be executed on segment rotation, similar to postgres' wal 'archive_command', configurations could store only the current segments and all their random i/o on flash, then once rotated, copy them sequentially onto larger/slowe

Re: How many partition can one single machine handle in Kafka?

2014-10-21 Thread Neil Harkins
On Tue, Oct 21, 2014 at 2:10 PM, Todd Palino wrote: > As far as the number of partitions a single broker can handle, we've set > our cap at 4000 partitions (including replicas). Above that we've seen some > performance and stability issues. How many brokers? I'm curious: what kinds of problems wo

frequent periods of ~1500 replicas not in sync

2014-10-21 Thread Neil Harkins
Hi. I've got a 5 node cluster running Kafka 0.8.1, with 4697 partitions (2 replicas each) across 564 topics. I'm sending it about 1% of our total messaging load now, and several times a day there is a period where 1~1500 partitions have one replica not in sync. Is this normal? If a consumer is read

Re: rate of producers writing to a topic?

2014-10-07 Thread Neil Harkins
> On Tue, Oct 7, 2014 at 3:56 PM, Josh J wrote: >> Is there a way to monitor the rate of rates to a particular topic? I wish >> to monitor the frequency of incoming tuples in order to consume from the >> topic in particular ways depending on the incoming write throughput. we've got a cron which i

Re: kafka.SocketServerStats write rate metrics seem wrong

2014-08-05 Thread Neil Harkins
No compression is configured, the spool files can be /usr/bin/strings'd to show the msgs. On Tue, Aug 5, 2014 at 2:13 PM, Guozhang Wang wrote: > Hello Neil, > > Did you use any compression on your producers? > > Guozhang > > > On Tue, Aug 5, 2014 at 12:23 PM,

kafka.SocketServerStats write rate metrics seem wrong

2014-08-05 Thread Neil Harkins
Hi. I'm seeing some odd numbers from kafka.SocketServerStats. Ideally, I'd like to have stats broken down per-topic, e.g. what's our most written/read topics? For write rates, I've got a separate process iterating topics every minute, doing this: (head_offset_now - head_offset_last) / (time_now -