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 the logs:

/var/log/kafka/kafka.log.2:[2014-12-05 05:21:28,216] 19186668
[kafka-request-handler-4] WARN  kafka.server.ReplicaManager  -
[Replica Manager on Broker 5]: While recording the follower position,
the partition [topic.with.two.replicas,0] hasn't been created, skip
updating leader HW

/var/log/kafka/kafka.log.2:[2014-12-05 05:21:28,219] 19186671
[kafka-request-handler-4] WARN  kafka.server.KafkaApis  - [KafkaApi-5]
Fetch request with correlation id 36397 from client
ReplicaFetcherThread-1-5 on partition [topic.with.two.replicas,0]
failed due to Topic topic.with.two.replicas either doesn't exist or is
in the process of being deleted

We also have some topics which had 3 replicas also now leaderless:

Topic:topic.with.three.replicas PartitionCount:6 ReplicationFactor:3 Configs:
Topic: topic.with.three.replicas Partition: 0 Leader: none Replicas: 3,1,2 Isr:

whose 'state' in zookeeper apparently disappeared:
'/brokers/topics/topic.with.three.replicas/partitions/3/state':
NoNodeError((), {})

Our versions are:
kafka 0.8.1
zookeeper 3.4.5

From searching archives of this list, the recommended fix
is to blow away the topic(s) and recreate. At this point in time,
that's an option, but it's not really acceptable for a reliable
data pipeline. Are there options to repair specific partitions?

-neil


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 it's still relevant in 0.8.

Basically, in one of our topics there was an 8k stretch of nulls.
Correlating timestamps from the messages bracketing the nulls
to the kafka log, I see that the server restarted during that time,
and here are the recovery lines related to the topic with the nulls:

[2014-11-04 00:48:07,602] INFO zookeeper state changed (SyncConnected)
(org.I0Itec.zkclient.ZkClient)
[2014-11-04 01:00:35,806] INFO Shutting down Kafka server
(kafka.server.KafkaServer)
[2014-11-04 01:00:35,813] INFO shutdown scheduler kafka-logcleaner-
(kafka.utils.KafkaScheduler)
[2014-11-04 01:01:38,411] INFO Starting Kafka server...
(kafka.server.KafkaServer)
...
[2014-11-04 01:01:49,146] INFO Loading log 'foo.bar-0' (kafka.log.LogManager)
[2014-11-04 01:01:49,147] INFO Loading the last segment
/var/kafka-leaf-spool/foo.bar-0/002684355423.kafka in mutable
mode, recovery true (kafka.log.Log)
[2014-11-04 01:01:55,877] INFO recover high water mark:414004449
(kafka.message.FileMessageSet)
[2014-11-04 01:01:55,877] INFO Recovery succeeded in 6 seconds. 0
bytes truncated. (kafka.message.FileMessageSet)

The only hypothesis I can come up with is that the shutdown
(graceful?) did not wait for all messages to flush to disk
(we're configured with: log.flush.interval=1,
log.default.flush.interval.ms=500, and
log.default.flush.scheduler.interval.ms=500),
but the max offset was recorded, so that when it came back up,
it filled the gap with nulls to reach the valid max offset in case
any consumers were at the end.

But for consumers with a position prior to all the nulls,
are they guaranteed to get back on the rails so-to-speak?
Nulls appear as v0(i.e. magic) messages of 0 length,
but the messages replaced could be variable length.

Thanks in advance for any input,
-neil


Re: nulls found in topic, created by recovery?

2014-11-06 Thread Neil Harkins
The topic is not compressed. The consumer used our fork of the python lib,
which I had to modify to get over the nulls.

-neil


On Thu, Nov 6, 2014 at 2:16 PM, Neha Narkhede neha.narkh...@gmail.com wrote:
 IIRC, the bug that introduced the nulls was related to compressed data. Is
 this topic compressed? Did you 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 nhark...@gmail.com wrote:

 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 it's still relevant in 0.8.

 Basically, in one of our topics there was an 8k stretch of nulls.
 Correlating timestamps from the messages bracketing the nulls
 to the kafka log, I see that the server restarted during that time,
 and here are the recovery lines related to the topic with the nulls:

 [2014-11-04 00:48:07,602] INFO zookeeper state changed (SyncConnected)
 (org.I0Itec.zkclient.ZkClient)
 [2014-11-04 01:00:35,806] INFO Shutting down Kafka server
 (kafka.server.KafkaServer)
 [2014-11-04 01:00:35,813] INFO shutdown scheduler kafka-logcleaner-
 (kafka.utils.KafkaScheduler)
 [2014-11-04 01:01:38,411] INFO Starting Kafka server...
 (kafka.server.KafkaServer)
 ...
 [2014-11-04 01:01:49,146] INFO Loading log 'foo.bar-0'
 (kafka.log.LogManager)
 [2014-11-04 01:01:49,147] INFO Loading the last segment
 /var/kafka-leaf-spool/foo.bar-0/002684355423.kafka in mutable
 mode, recovery true (kafka.log.Log)
 [2014-11-04 01:01:55,877] INFO recover high water mark:414004449
 (kafka.message.FileMessageSet)
 [2014-11-04 01:01:55,877] INFO Recovery succeeded in 6 seconds. 0
 bytes truncated. (kafka.message.FileMessageSet)

 The only hypothesis I can come up with is that the shutdown
 (graceful?) did not wait for all messages to flush to disk
 (we're configured with: log.flush.interval=1,
 log.default.flush.interval.ms=500, and
 log.default.flush.scheduler.interval.ms=500),
 but the max offset was recorded, so that when it came back up,
 it filled the gap with nulls to reach the valid max offset in case
 any consumers were at the end.

 But for consumers with a position prior to all the nulls,
 are they guaranteed to get back on the rails so-to-speak?
 Nulls appear as v0(i.e. magic) messages of 0 length,
 but the messages replaced could be variable length.

 Thanks in advance for any input,
 -neil




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

2014-10-23 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/slower spinning disks,
or even S3.

-neil

On Wed, Oct 22, 2014 at 10:09 PM, Xiaobin She xiaobin...@gmail.com wrote:
 Todd,

 Thank you for the information.

 With 28,000+ files and 14 disks, that makes there are averagely about 4000
 open files on two disk ( which is treated as one single disk) , am I right?

 How do you manage to make the all the write operation to thest 4000 open
 files be sequential to the disk?

 As far as I know, write operation to different files on the same disk will
 cause random write, which is not good for performance.

 xiaobinshe




 2014-10-23 1:00 GMT+08:00 Todd Palino tpal...@gmail.com:

 In fact there are many more than 4000 open files. Many of our brokers run
 with 28,000+ open files (regular file handles, not network connections). In
 our case, we're beefing up the disk performance as much as we can by
 running in a RAID-10 configuration with 14 disks.

 -Todd

 On Tue, Oct 21, 2014 at 7:58 PM, Xiaobin She xiaobin...@gmail.com wrote:

  Todd,
 
  Actually I'm wondering how kafka handle so much partition, with one
  partition there is at least one file on disk, and with 4000 partition,
  there will be at least 4000 files.
 
  When all these partitions have write request, how did Kafka make the
 write
  operation on the disk to be sequential (which is emphasized in the design
  document of Kafka) and make sure the disk access is effective?
 
  Thank you for your reply.
 
  xiaobinshe
 
 
 
  2014-10-22 5:10 GMT+08:00 Todd Palino tpal...@gmail.com:
 
   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.
  
   -Todd
  
   On Tue, Oct 21, 2014 at 12:15 AM, Xiaobin She xiaobin...@gmail.com
   wrote:
  
hello, everyone
   
I'm new to kafka, I'm wondering what's the max num of partition can
 one
siggle machine handle in Kafka?
   
Is there an sugeest num?
   
Thanks.
   
xiaobinshe
   
  
 



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 reading from a replica that gets deemed
not in sync, does it get redirected to the good replica?
Is there a #partitions over which maintenance tasks
become infeasible?

Relevant config bits:
auto.leader.rebalance.enable=true
leader.imbalance.per.broker.percentage=20
leader.imbalance.check.interval.seconds=30
replica.lag.time.max.ms=1
replica.lag.max.messages=4000
num.replica.fetchers=4
replica.fetch.max.bytes=10485760

Not necessarily correlated to those periods,
I see a lot of these errors in the logs:

[2014-10-20 21:23:26,999] 21963614 [ReplicaFetcherThread-3-1] ERROR
kafka.server.ReplicaFetcherThread  - [ReplicaFetcherThread-3-1], Error
in fetch Name: FetchRequest; Version: 0; CorrelationId: 77423;
ClientId: ReplicaFetcherThread-3-1; ReplicaId: 2; MaxWait: 500 ms;
MinBytes: 1 bytes; RequestInfo: ...

And a few of these:

[2014-10-20 21:23:39,555] 3467527 [kafka-scheduler-2] ERROR
kafka.utils.ZkUtils$  - Conditional update of path
/brokers/topics/foo.bar/partitions/3/state with data
{controller_epoch:11,leader:3,version:1,leader_epoch:109,isr:[3]}
and expected version 197 failed due to
org.apache.zookeeper.KeeperException$BadVersionException:
KeeperErrorCode = BadVersion for
/brokers/topics/foo.bar/partitions/3/state

And this one I assume is a client closing the connection non-gracefully,
thus should probably be a warning, not an error?:

[2014-10-20 21:54:15,599] 23812214 [kafka-processor-9092-3] ERROR
kafka.network.Processor  - Closing socket for /10.31.0.224 because of
error

-neil


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 tpal...@gmail.com 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 would affect
a single broker with a large number of partitions, but not affect the
entire cluster with even more partitions?


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 joshjd...@gmail.com 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 iterates each topic+partition and
writes an index of timestamps-byte offset, so it was
easy to hook from there into graphite. but graphite couldn't
do 2000+ derivative()s for a reasonable pageload time,
so we put the pre-calculated write rate into graphite.

what i'd very much like to have is per-topic read/write
stats by client ip.

-neil


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 wangg...@gmail.com wrote:
 Hello Neil,

 Did you use any compression on your producers?

 Guozhang


 On Tue, Aug 5, 2014 at 12:23 PM, Neil Harkins nhark...@gmail.com wrote:

 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 - time_last)
 and inserting that into graphite (was previously inserting the
 head_offset, but graphite's derivative() on 2000+ topics
 was a bit much).

 a graphite sumSeries() across them shows 20~25 MB/s
 on a single-instance kafka 0.7.2, which sounds correct,
 but when I compare that to the kafka.SocketServerStats
 we're collecting in graphite, derivative(TotalBytesWritten)
 shows 3.0GB/s, and BytesWrittenPerSecond shows
 200KB/s, neither of which jive with my offset rate.

 Is anyone aware of any weirdness with those stats?

 thanks in advance for any insight,
 -neil




 --
 -- Guozhang