Re: Kafka startup/restart process

2013-08-20 Thread Neha Narkhede
Vadim, The controlled shutdown command proceeds to shutting down the broker after it runs of controlled shutdown retries. Since the shutdown call is blocking, its return will indicate the broker has successfully shut down. If the under replicated partition count drops to 0, that is a good enough i

Re: issue with kafka-preferred-replica-election.sh

2013-08-20 Thread Guozhang Wang
Libo, Sorry for the late reply. I will file a JIRA for this one. Could you please provide the full process and environment to reproduce this issue? Guozhang On Tue, Aug 20, 2013 at 1:10 PM, Yu, Libo wrote: > Hi Guozhang, > > Thanks for the detailed reply. All the brokers are running. > The pa

Re: Offset committing on rebalance

2013-08-20 Thread Ian Friedman
Hey just reporting that the ZK disconnect tip on the FAQ was in fact right on the money. After tweaking our GC settings and zk timeout settings, I'm no longer seeing the flood of rebalances. -- Ian Friedman On Tuesday, August 20, 2013 at 2:26 AM, Ian Friedman wrote: > Sorry, ignore that f

RE: Possible corrupted index (Kafka 0.8)

2013-08-20 Thread Bello, Bob
I have resolved the issue. Our default consumer fetch size was the default (1024*1024) which was too small for some of the messages. Thanks -Bob

Re: Correlation id

2013-08-20 Thread Timothy Chen
Thanks Tejas! That's very helpful. Tim On Mon, Aug 19, 2013 at 11:40 PM, Tejas Patil wrote: > Multiple produce requests are sent asynchronously over the same socket. > Suppose you send 2 requests and get back single response, how do you figure > out which one it corresponds to of those 2 reques

Re: use kafka as dependency in sbt from central maven repository

2013-08-20 Thread Lorenz Knies
hi joe, thanks for the hint with the apache repo. the from directive did not work for me. while it stopped sbt from complaining during the update, it did not include the jar in the classpath either. this may be correct behaviour since the sbt documentation states that you may specify a direct UR

RE: issue with kafka-preferred-replica-election.sh

2013-08-20 Thread Yu, Libo
Hi Guozhang, Thanks for the detailed reply. All the brokers are running. The path not created is /brokers/topics/uattoqaaa.default/partitions not /brokers/topics/uattoqaaa.default/partitions/[partition-id]/state Actually, I found quite a few topics don't have the issue. The controller must have f

Re: issue with kafka-preferred-replica-election.sh

2013-08-20 Thread Guozhang Wang
The create command tool will only create the path /brokers/topics/uattoqaaa.default, and then controller, once noticing the change in ZK about the added topic, will elect leaders from the assigned replicas written in /brokers/topics/uattoqaaa.default and then create /brokers/topics/uattoqaaa.defaul

Possible corrupted index (Kafka 0.8)

2013-08-20 Thread Bello, Bob
Hello Kafka Club, We are running a July 29th git pull of 0.8 Kafka. Linux Sun JDK1.7.0_25 64bit We have a what appears to be a corrupted index for log file. This has occurred on a low volume topic on a single partition: - The leader Kafka broker thinks this topic is at offset: 1808 -

RE: issue with kafka-preferred-replica-election.sh

2013-08-20 Thread Yu, Libo
One more question: "The create topic command will not immediately create the path; it will be created by the controller later during the creation of partitions and assign replicas to partitions." So when will the controller create the path and partitions? When the first message is published to

Re: Avro serialization

2013-08-20 Thread Mark
Ok someone answered a similar question in the Avro forum. It *sounds* like that the Avro messages sent to Kafka are wrapped and/or prepended with the SHA which is used by the consumer to lookup the schema. That makes more sense. On Aug 20, 2013, at 11:09 AM, Mark wrote: > Thanks Jay I've alr

RE: issue with kafka-preferred-replica-election.sh

2013-08-20 Thread Yu, Libo
This is from my very first email: > > I checked zookeeper and there is no > > /brokers/topics/uattoqaaa.default/partitions. All I found is > > /brokers/topics/uattoqaaa.default. The topic is not newly created and it has been there for a long time. So this seems to be an issue from create topic com

Re: Avro serialization

2013-08-20 Thread Mark
Thanks Jay I've already read the paper and Jira ticket (haven't read the code) but I'm still confused on how to integrate this with Kafka. Say we write an Avro message (the message contains a SHA of the shcmea) to Kafka and a consumer pulls of this message. How does the consume know how to des

Re: Avro serialization

2013-08-20 Thread Guozhang Wang
The magic byte is already in the Kafka Message data format, all you need to do is upon reading the magic byte for Avro, use your Avro resolver which treat the first 16 bytes as the schemaId to read the schema. Guozhang On Tue, Aug 20, 2013 at 10:54 AM, Mark wrote: > So basically prepend the in

Re: Kafka Mirroring setup

2013-08-20 Thread Andrew Otto
In our case, our aggregator/analytics cluster is in our main datacenter, so there's no risk of the main producers becoming disconnected from it. It seems nicer to have a dedicated aggregator cluster, that only only gets its data via MirrorMaker (Option A), but in our case this isn't necessary.

Re: Avro serialization

2013-08-20 Thread Mark
So basically prepend the information in the header of each message? Wouldn't this require changes in all readers/writers? On Aug 20, 2013, at 9:19 AM, Guozhang Wang wrote: > Hello Mark, > > The Kafka message format starts with a magic byte indicating what kind of > serialization is used for th

Re: issue with kafka-preferred-replica-election.sh

2013-08-20 Thread Guozhang Wang
Could you check that before you run kafka-preferred-replica-election.sh the /brokers/topics/uattoqaaa.default/partitions Node has been created or not? The create topic command will not immediately create the path; it will be created by the controller later during the creation of partitions and assi

Re: Kafka Mirroring setup

2013-08-20 Thread Jay Kreps
We do something like A (though I'm not sure I understand B): http://kafka.apache.org/documentation.html#datacenters Essentially what we wanted was that each datacenter stood alone so that we would not lose data if the datacenters became disconnected. Network partitions within our data centers are

Kafka Mirroring setup

2013-08-20 Thread Andrew Otto
Hi all! Wikimedia is investigating how best to set up Broker clusters in multiple data centers. Our main analytics Broker cluster is currently in our main datacenter. It is possible for all of the main DC's frontend producers to produce directly to our analytics cluster, but we're not sure if

RE: issue with kafka-preferred-replica-election.sh

2013-08-20 Thread Yu, Libo
It is created with 3 partitions. Regards, Libo -Original Message- From: Guozhang Wang [mailto:wangg...@gmail.com] Sent: Tuesday, August 20, 2013 12:21 PM To: users@kafka.apache.org Subject: Re: issue with kafka-preferred-replica-election.sh Hello Libo, Before you call the kafka-prefe

Re: Avro serialization

2013-08-20 Thread Jay Kreps
This paper has more information on what we are doing at LinkedIn: http://sites.computer.org/debull/A12june/pipeline.pdf This Avro JIRA has a schema repository implementation similar to the one LinkedIn uses: https://issues.apache.org/jira/browse/AVRO-1124 -Jay On Tue, Aug 20, 2013 at 7:08 AM, M

Re: Failed to start preferred replica election

2013-08-20 Thread Jay Kreps
Is there any way to channel these many excellent email threads into documentation improvements :-) -Jay On Mon, Aug 19, 2013 at 8:55 PM, Jun Rao wrote: > We also have a jmx bean that tracks the lag in messages per partition in > the follower broker. > > Thanks, > > Jun > > > On Mon, Aug 19, 20

Re: issue with kafka-preferred-replica-election.sh

2013-08-20 Thread Guozhang Wang
Hello Libo, Before you call the kafka-preferred-replica-election.sh have you already created the topic uattoqaaa.default with the number of partitions? Guozhang On Tue, Aug 20, 2013 at 8:35 AM, Yu, Libo wrote: > Hi, > > I tried to run kafka-preferred-replica-election.sh on our kafka cluster.

Re: Avro serialization

2013-08-20 Thread Guozhang Wang
Hello Mark, The Kafka message format starts with a magic byte indicating what kind of serialization is used for this message. And if this byte indicates Avro, you can layout your message as starting with the schemaId and then followed by message payload. Upon consumption, you can first get the sch

issue with kafka-preferred-replica-election.sh

2013-08-20 Thread Yu, Libo
Hi, I tried to run kafka-preferred-replica-election.sh on our kafka cluster. But I got this expection: Failed to start preferred replica election org.I0Itec.zkclient.exception.ZkNoNodeException: org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /brokers/topics/ua

Re: How to get broker back to ISR

2013-08-20 Thread Neha Narkhede
Once the broker is restarted, the controller broker will send it a list of partitions that it should follow. The broker starts fetching from the respective leaders and enters the ISR. Depending on the duration of shutdown, the broker can take some time to enter ISR. Thanks, Neha On Aug 20, 2013 4:

Avro serialization

2013-08-20 Thread Mark
Can someone break down how message serialization would work with Avro? I've read instead of adding a schema to every single event it would be wise to add some sort of fingerprint with each message to identify which schema it should used. What I'm having trouble understanding is, how do we read t

Re: use kafka as dependency in sbt from central maven repository

2013-08-20 Thread Joe Stein
Lorenz, The Maven central repo got bonked ( https://issues.apache.org/jira/browse/KAFKA-974 ) and it looks like it is affecting you too Try this "org.apache.kafka" % "kafka_2.8.2" % "0.8.0-beta1" intransitive() from " https://repository.apache.org/content/repositories/releases"; I just tried it

Re: use kafka as dependency in sbt from central maven repository

2013-08-20 Thread Lorenz Knies
thanks for the reply, but that unfortunately does not help. to make my problem easily reproducible: creating an empty directory, put just a build.sbt file with the following content into it: name := "kafka-dependency-test" scalaVersion := "2.8.2" libraryDependencies += "org.apache.kafka" % "k

Re: questtion about log.retention.bytes

2013-08-20 Thread Paul Christian
Neha, Correct, that is my question. We want to investigate capping our disk usage so we don't fill up our hds. If you have any recommended configurations or documents on these setting, please let us know. Thank you, Paul On Tue, Aug 20, 2013 at 6:16 AM, Paul Christian wrote: > Jun, > > For m

re: questtion about log.retention.bytes

2013-08-20 Thread Paul Christian
Jun, For my first example is that syntax correct? I.e. log.retention.bytes.per.topic.A = 15MB log.retention.bytes.per.topic.B = 20MB I totally guessed there and was wondering if I guessed right? Otherwise is there a document with the proper formatting to full out this map? Thank you, Paul

Re: use kafka as dependency in sbt from central maven repository

2013-08-20 Thread Joe Stein
you need to put intransitive() at the end so it ignores dependencies that are defined to be ignored in maven so your line would be libraryDependencies += "org.apache.kafka" % "kafka_2.9.2" % "0.8.0-beta1" intransitive() /*** Joe Stein Founder, Principal

How to get broker back to ISR

2013-08-20 Thread James Wu
Hi, I am wondering if my leader broker crash, how to get it back to ISR after restart kafak ? In the initial status the kafka-list-topic.sh shows: topic: failover-test partition: 0 leader: 0 replicas: 0,1 isr: 0,1 If I terminate the leader and kafka-list-topic.sh shows: topic: failover-test part