Re: Producer message ordering problem

2013-08-22 Thread Philip O'Toole
I am curious. What is it about your design that requires you track order so tightly? Maybe there is another way to meet your needs instead of relying on Kafka to do it. Philip On Aug 22, 2013, at 9:32 PM, Ross Black wrote: > Hi, > > I am using Kafka 0.7.1, and using the low-level SyncProduc

Producer message ordering problem

2013-08-22 Thread Ross Black
Hi, I am using Kafka 0.7.1, and using the low-level SyncProducer to send messages to a *single* partition from a *single* thread. The client sends messages that contain sequential numbers so it is obvious at the consumer when message order is shuffled. I have noticed that messages can be saved out

Re: Differences in size of data replicated by mirror maker

2013-08-22 Thread Jun Rao
We have JMX beans that report #messages per topic? Does the total count match btw the two clusters? Thanks, Jun On Thu, Aug 22, 2013 at 2:14 PM, Rajasekar Elango wrote: > Hi, > > We are using mirrormaker to replicate data between two kafka clusters. I am > seeing huge difference in size of log

Re: repeated for request offset out of range

2013-08-22 Thread Jun Rao
Are you using the ZK-based consumer or SimpleConsumer? Only the former automatically resets offsets on OffsetOutOfRangeException. Thanks, Jun On Thu, Aug 22, 2013 at 10:23 AM, Wu, Jiang2 wrote: > Hi, > > In our log we see there are repeated errors due to the same request whose > offset is out

Re: Tuning mirror maker performance

2013-08-22 Thread Jun Rao
You have to determine whether the bottleneck is in the consumer or the producer. To improve the performance of the latter, you can increase the # of total consumer streams. # streams is capped by total # partitions. So, you may need to increase the # of partitions. To improve the performance of t

Re: Questions about the leader election

2013-08-22 Thread Neha Narkhede
The replication state machine and leader election mechanism is described here - http://kafka.apache.org/documentation.html#replication Let us know how the docs can be improved. Thanks, Neha On Thu, Aug 22, 2013 at 8:51 PM, James Wu wrote: > Hi, > > I am wondering what is the mechanism that Ka

Re: No leaders elected after topics created

2013-08-22 Thread Jun Rao
Any error in the controller and state-change log? Thanks, Jun On Thu, Aug 22, 2013 at 10:01 AM, Vadim Keylis wrote: > I ran command to create 300 topics sequentially. Kafka create topic script > return with topic created successfully message. However running > kafka-list-topics.sh script showe

Re: Ganglia Metrics Reporter

2013-08-22 Thread Jun Rao
I added a JMX reporter wiki page: https://cwiki.apache.org/confluence/display/KAFKA/JMX+Reporters , which is linked from our main wiki page. Could you add a short description (including the supported version) and a link there? Thanks, Jun On Thu, Aug 22, 2013 at 8:42 AM, Maxime Brugidou wrote:

Questions about the leader election

2013-08-22 Thread James Wu
Hi, I am wondering what is the mechanism that Kafka elects the leader of partitions ? Does it handle by the controller process or ? If the leader crashed, who will decide the new leader ? and the process is running on Zookeeper or Kafka? Thanks. -- -- Friendly regards, *James Wu

Re: Broker does not publish offset in zookeeper

2013-08-22 Thread Jun Rao
Yes, that's 0.8 specific. You can find the docs in http://kafka.apache.org/documentation.html#monitoring Thanks, Jun On Thu, Aug 22, 2013 at 8:42 AM, Pablo Barrera González < pablo.barr...@gmail.com> wrote: > Is that 0.8 specific? > > Sorry, I can't find the documentation you mention (there ar

Re: Differences in size of data replicated by mirror maker

2013-08-22 Thread Guozhang Wang
When you state the numbers, are they the same across instances in the cluster, meaning that Topic-0 would have 910*5 GB in source cluster and 25*5 GB in target cluster? Another possibility is that MirrorMaker uses compression on the producer side, but I would be surprised if the compression rate c

Re: Differences in size of data replicated by mirror maker

2013-08-22 Thread Rajasekar Elango
Yes, both source and target clusters have 5 brokers in cluster. Sent from my iPhone On Aug 22, 2013, at 6:11 PM, Guozhang Wang wrote: > Hello Rajasekar, > > Are the size of the source cluster and target cluster the same? > > Guozhang > > > On Thu, Aug 22, 2013 at 2:14 PM, Rajasekar Elango > wr

Re: Differences in size of data replicated by mirror maker

2013-08-22 Thread Guozhang Wang
Hello Rajasekar, Are the size of the source cluster and target cluster the same? Guozhang On Thu, Aug 22, 2013 at 2:14 PM, Rajasekar Elango wrote: > Hi, > > We are using mirrormaker to replicate data between two kafka clusters. I am > seeing huge difference in size of log in data dir between t

Re: partition does not exist (but it is there).

2013-08-22 Thread Neha Narkhede
Libo, Did you see this error in the broker 3's log, right after it was restarted? One of the reasons for this error is that the broker hasn't received the first LeaderAndIsr request from the controller yet. When a broker starts up, the only way it knows what partitions it owns is through the first

Re: message loss

2013-08-22 Thread Neha Narkhede
I agree with you. If we include that knob, applications can choose their consistency vs availability tradeoff according to the respective requirements. I will file a JIRA for this. Thanks, Neha On Thu, Aug 22, 2013 at 2:10 PM, Scott Clasen wrote: > +1 for that knob on a per topic basis, choosi

Differences in size of data replicated by mirror maker

2013-08-22 Thread Rajasekar Elango
Hi, We are using mirrormaker to replicate data between two kafka clusters. I am seeing huge difference in size of log in data dir between the broker in source cluster vs broker in destination cluster: For eg: Size of ~/data/Topic-0/ is about 910 G in source broker, but only its only 25G in destin

Re: message loss

2013-08-22 Thread Scott Clasen
+1 for that knob on a per topic basis, choosing consistency over availability would open kafka to more use cases no? Sent from my iPhone On Aug 22, 2013, at 1:59 PM, Neha Narkhede wrote: > Scott, > > Kafka replication aims to guarantee that committed writes are not lost. In > other words, as

Re: repeated for request offset out of range

2013-08-22 Thread Neha Narkhede
I think your request log is empty because you might not have a directory at /logs. I think it should be logs/kafka/kafka-request.log Thanks, Neha On Thu, Aug 22, 2013 at 1:02 PM, Wu, Jiang2 wrote: > The kafka-request.log is empty. > > This is the related log4j.properties part: > > log4j.append

Re: delete a topic

2013-08-22 Thread Neha Narkhede
In production, we are not deleting topics yet. In test environments, if we have to delete topics, we wipe out the cluster. If this is a feature that most users are blocked on, I think it makes sense to prioritize https://issues.apache.org/jira/browse/KAFKA-330. Thanks, Neha On Thu, Aug 22, 2013

Re: Tuning mirror maker performance

2013-08-22 Thread Rajasekar Elango
Hi, I am trying to tune mirrormaker configurations based on this doc and would like know your recommendations. Our configuration: We are doin

Re: message loss

2013-08-22 Thread Neha Narkhede
Scott, Kafka replication aims to guarantee that committed writes are not lost. In other words, as long as leader can be transitioned to a broker that was in the ISR, no data will be lost. For increased availability, if there are no other brokers in the ISR, we fall back to electing a broker that i

Tuning mirror maker performance

2013-08-22 Thread Rajasekar Elango
I am trying to tune mirrormaker configurations based on this doc and would like know your recommendations. Our configuration: We are doing int

Re: No leaders elected after topics created

2013-08-22 Thread Vadim Keylis
I only need to delete topics when I want to start from clean state. I still unclear why leaders were not elected after create script printed message that topic was created successfully Thanks, Vadim On Thu, Aug 22, 2013 at 12:28 PM, Neha Narkhede wrote: > I see the issue. I think the 0.8-beta in

RE: repeated for request offset out of range

2013-08-22 Thread Wu, Jiang2
The kafka-request.log is empty. This is the related log4j.properties part: log4j.appender.requestAppender=org.apache.log4j.RollingFileAppender #log4j.appender.requestAppender.DatePattern='.'-MM-dd-HH log4j.appender.requestAppender.MaxFileSize=10MB log4j.appender.requestAppender.MaxBackupIndex

Re: delete a topic

2013-08-22 Thread Vadim Keylis
How do you guys delete a topic if such need arise? On Thu, Aug 22, 2013 at 12:29 PM, Neha Narkhede wrote: > I mentioned this in a different thread. I think the 0.8.0-beta1-candidate1 > includes the script, but we have removed it later. I think when we publish > 0.8-final, this should be resolved

message loss

2013-08-22 Thread Scott Clasen
So looks like there is a jespen post coming on kafka 0.8 replication, based on this thats circulating on twitter. https://www.refheap.com/17932/raw Understanding that kafka isnt designed particularly to be partition tolerant, the result is not completely surprising. But my question is, is there s

partition does not exist (but it is there).

2013-08-22 Thread Yu, Libo
This is from the broker 3's log: [2013-08-22 15:40:02,984] WARN [KafkaApi-3] Fetch request: Partition [tes t.replica1.default,0] doesn't exist on 3 (kafka.server.KafkaApis) Here is what list topic command shows: topic: test.replica1.defaultpartition: 0leader: 3 replicas: 3 isr:

RE: delete a topic

2013-08-22 Thread Yu, Libo
After I deleted a topic using the delete script and deleted all the log files, the brokers still tried to sync for that topic. It seems a cluster restart is required. Regards, Libo -Original Message- From: Vadim Keylis [mailto:vkeylis2...@gmail.com] Sent: Thursday, August 22, 2013 2:

Re: repeated for request offset out of range

2013-08-22 Thread Neha Narkhede
Could you check the kafka-request.log for these FetchRequests and see if they are sent by the same clientId ? Also, from the same kafka-request.log, could you check if the Kafka server received an OffsetRequest after these FetchRequests ? Thanks, Neha On Thu, Aug 22, 2013 at 10:23 AM, Wu, Jiang2

Re: delete a topic

2013-08-22 Thread Neha Narkhede
I mentioned this in a different thread. I think the 0.8.0-beta1-candidate1 includes the script, but we have removed it later. I think when we publish 0.8-final, this should be resolved. Thanks, Neha On Thu, Aug 22, 2013 at 12:24 PM, Neha Narkhede wrote: > Vadim, > > The JIRA tracking this featu

Re: No leaders elected after topics created

2013-08-22 Thread Neha Narkhede
I see the issue. I think the 0.8-beta included the script, but we later removed it. We still don't have support for the delete topic feature, so I recommend not using it. How long did you wait for the leaders to get elected before you attempted to delete the topics? Thanks, Neha On Thu, Aug 22,

Re: delete a topic

2013-08-22 Thread Neha Narkhede
Vadim, The JIRA tracking this feature is https://issues.apache.org/jira/browse/KAFKA-330. Until we have this feature, we should remove any scripts that give the impression of deleting a topic, as Jay suggested. I tried looking under bin/ but couldn't find any delete topic or partition script. Can

Re: delete a topic

2013-08-22 Thread Vadim Keylis
Jay or Neha. What is the process of deleting the topic if the script included in the repository creates false impression? Thanks, Vadim On Thu, Aug 22, 2013 at 11:14 AM, Jay Kreps wrote: > We should really remove the delete script from 0.8 if we plan to release > without delete support. Right

Re: No leaders elected after topics created

2013-08-22 Thread Vadim Keylis
Hello Neha. The topics should of been available since I ran list-topics script 5 hours later after creating topics. I am reffering to kafka-delete-topic.sh script that I got from kafka git repository. Thanks, Vadim On Thu, Aug 22, 2013 at 11:05 AM, Neha Narkhede wrote: > As part of topic creati

Re: delete a topic

2013-08-22 Thread Jay Kreps
We should really remove the delete script from 0.8 if we plan to release without delete support. Right now it appears to work but doesn't which is clearly not good. -Jay On Thu, Aug 22, 2013 at 10:57 AM, Neha Narkhede wrote: > We currently don't have the delete topic feature in Kafka 0.8. So an

Re: load-balancing consumers question

2013-08-22 Thread Guozhang Wang
Hello Michal, This FAQ entry may help you understanding the rebalance logic: https://cwiki.apache.org/confluence/display/KAFKA/FAQ#FAQ-CanIpredicttheresultsoftheconsumerrebabalance%3F In a word, since we use a deterministic range partition in rebalance, unevenly or customized partition assignmen

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

2013-08-22 Thread Yu, Libo
Thanks, Guozhang Regards, Libo -Original Message- From: Guozhang Wang [mailto:wangg...@gmail.com] Sent: Thursday, August 22, 2013 12:42 PM To: users@kafka.apache.org Subject: Re: issue with kafka-preferred-replica-election.sh Hello Libo, I have created the Jira for this issue: https

RE: delete a topic

2013-08-22 Thread Yu, Libo
Thanks, Neha. Regards, Libo -Original Message- From: Neha Narkhede [mailto:neha.narkh...@gmail.com] Sent: Thursday, August 22, 2013 1:57 PM To: users@kafka.apache.org Subject: Re: delete a topic We currently don't have the delete topic feature in Kafka 0.8. So any manual attempts to

Re: No leaders elected after topics created

2013-08-22 Thread Neha Narkhede
As part of topic creation, the controller broker has to elect leader for each of the topics and this involves a few writes to zookeeper. So it might take several seconds to complete leader election for 300 topics and that time would increase if you have many partitions per topic. Which delete part

load-balancing consumers question

2013-08-22 Thread Michal Haris
Hi, is it possible or has anybody tried/needed to balance partitions between consumers unevenly or based on some custom function ? Ideally with Kafka 0.7 Michal Haris

Re: delete a topic

2013-08-22 Thread Neha Narkhede
We currently don't have the delete topic feature in Kafka 0.8. So any manual attempts to do so might have a negative impact on functionality. Thanks, Neha On Thu, Aug 22, 2013 at 10:30 AM, Yu, Libo wrote: > Hi team, > > When I delete a topic, the topic is deleted from zookeeper but its log > f

Re: How does Kafka decide which Consumer out of multiple Consumer clients to assign to a single topic partition

2013-08-22 Thread Dennis Haller
Thanks Guozhang. It's good to have this in the FAQ. On Thu, Aug 22, 2013 at 10:22 AM, Guozhang Wang wrote: > Thanks Neha for the clarification. I have created a new entry in FAQ for > this question: > > > https://cwiki.apache.org/confluence/display/KAFKA/FAQ#FAQ-CanIpredicttheresultsoftheconsu

Re: How does Kafka decide which Consumer out of multiple Consumer clients to assign to a single topic partition

2013-08-22 Thread Dennis Haller
The sorted list makes sense with what I'm seeing. The consumers are named with the group-name appended with the server hostname plus some other string such as this: redis-indexer_ip-10-122-123-214.ec2.internal-1377184713770-a6aa2f8e-0 Because in AWS the hostname is coming up with a different inte

delete a topic

2013-08-22 Thread Yu, Libo
Hi team, When I delete a topic, the topic is deleted from zookeeper but its log files are not deleted from Brokers. When I restart a broker, the broker will try to sync the log files whose topic has been deleted. Manually deleting the log files will resolve the issue. Should broker ignore log

Re: How does Kafka decide which Consumer out of multiple Consumer clients to assign to a single topic partition

2013-08-22 Thread Guozhang Wang
Thanks Neha for the clarification. I have created a new entry in FAQ for this question: https://cwiki.apache.org/confluence/display/KAFKA/FAQ#FAQ-CanIpredicttheresultsoftheconsumerrebabalance%3F Dennis, please let me know if this does not fully answer your question. Guozhang On Thu, Aug 22, 20

repeated for request offset out of range

2013-08-22 Thread Wu, Jiang2
Hi, In our log we see there are repeated errors due to the same request whose offset is out of range. As I understand when out of range happens, the consumer should reset its offset, therefore such error should happen only once. Does the log indicate a bug? Thanks, Jiang [2013-08-22 09:23:20

RE: broker never comes back to ISR

2013-08-22 Thread Wu, Jiang2
Hi Neha, The ReplicaFetcherThreads is Runnable at that moment - I'm working with Libo checking the problem. Could some operations such as manually deleting kafka log files cause the problem? Thanks, Jiang -Original Message- From: Neha Narkhede [mailto:neha.narkh...@gmail.com] Sent: We

No leaders elected after topics created

2013-08-22 Thread Vadim Keylis
I ran command to create 300 topics sequentially. Kafka create topic script return with topic created successfully message. However running kafka-list-topics.sh script showed to me that no leaders been selected and no servers listed in ISR. First inspection of the logs showed no obvious signs of the

Re: More questions on avro serialization

2013-08-22 Thread Mark
… or is the payload of the message prepending with a magic byte followed by the SHA? On Aug 22, 2013, at 9:49 AM, Mark wrote: > Are you referring to the same message class as: > https://github.com/apache/kafka/blob/0.7/core/src/main/scala/kafka/message/Message.scala > or are you talking bout

Re: More questions on avro serialization

2013-08-22 Thread Mark
Are you referring to the same message class as: https://github.com/apache/kafka/blob/0.7/core/src/main/scala/kafka/message/Message.scala or are you talking bout a wrapper around this message class which has its own magic byte followed by SHA of schema? If its the former, I'm confused. FYI, Lo

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

2013-08-22 Thread Guozhang Wang
Hello Libo, I have created the Jira for this issue: https://issues.apache.org/jira/browse/KAFKA-1019 Guozhang On Wed, Aug 21, 2013 at 11:27 AM, Yu, Libo wrote: > We never deleted it. Either it was never created or deleted somehow. > > Regards, > > Libo > > > -Original Message- > From

Re: More questions on avro serialization

2013-08-22 Thread Neha Narkhede
The point of the magic byte is to indicate the current version of the message format. One part of the format is the fact that it is Avro encoded. I'm not sure how Camus gets a 4 byte id, but at LinkedIn we use the 16 byte MD5 hash of the schema. Since AVRO-1124 is not resolved yet, I'm not sure if

Re: How does Kafka decide which Consumer out of multiple Consumer clients to assign to a single topic partition

2013-08-22 Thread Guozhang Wang
Hello Dennis, The rebalance on each consumer works by first release their owned partitions first (releasePartitionOwnership in ZookeeperConsumerConnector.scala) and then compute the new ownership. Hence in your scenario it is equally possible for each one of the two consumers to own the partition

Re: How does Kafka decide which Consumer out of multiple Consumer clients to assign to a single topic partition

2013-08-22 Thread Neha Narkhede
We range partition a sorted list of topic-partitions over a sorted list of consumer instances. This makes the rebalancing algorithm deterministic. As soon as you bring up the 2nd consumer instance, if its position in the sorted list is before the position of the 1st consumer client, it will end up

Re: Ganglia Metrics Reporter

2013-08-22 Thread Andrew Otto
Cool! At WMF, we use jmxtrans to do this: https://github.com/jmxtrans/jmxtrans And, if you use puppet, here's a nice little module to help generate jmxtrans json files, and an example of metrics we were sending to Ganglia from Kafka 0.7.2. https://github.com/wikimedia/puppet-jmxtrans#kafka-bro

How does Kafka decide which Consumer out of multiple Consumer clients to assign to a single topic partition

2013-08-22 Thread Dennis Haller
I have a situation where two high level consumers are being created to consume a single topic. There is only one partition for the topic, so I understand that only one Consumer will end up owning the topic and receiving messages. The two consumers are created from two servers in a redundant master

Re: Broker does not publish offset in zookeeper

2013-08-22 Thread Pablo Barrera González
Is that 0.8 specific? Sorry, I can't find the documentation you mention (there are no references to the variable maxLag). I still find more problems in having a dual connection (jmx and zookeeper) instead of only one (zookeeper). 2013/8/22 Jun Rao > Are you trying to monitor the lag in the

Ganglia Metrics Reporter

2013-08-22 Thread Maxime Brugidou
Hi all, Since I couldn't find any other way to publish kafka metrics to ganglia from kafka 0.8 (beta), I just published on github a super-simple ganglia metrics reporter for Kafka. It is configurable through the kafka config file and you can use it on the broker side and on your consumers/producer

Re: ordering

2013-08-22 Thread Jun Rao
A consumer can consume more than one partition. Messages from the same partition will still be delivered to the consumer in order. There is no ordering guarantee for messages across partitions. Thanks, Jun On Thu, Aug 22, 2013 at 6:52 AM, Yu, Libo wrote: > Hi Jun, > > This is my original conc

Re: Broker does not publish offset in zookeeper

2013-08-22 Thread Jun Rao
Are you trying to monitor the lag in the consumer? If so, you can use the maxLag/minFetch JMX described in http://kafka.apache.org/documentation.html#monitoring Thanks, Jun On Thu, Aug 22, 2013 at 7:41 AM, Pablo Barrera González < pablo.barr...@gmail.com> wrote: > Hello > > I am trying to moni

Broker does not publish offset in zookeeper

2013-08-22 Thread Pablo Barrera González
Hello I am trying to monitor our kafka cluster. For that I use a zookeeper client that reads the information of all the consumer and a jmx agent that connects to the brokers. I try to look for discrepancies between the data being produced and the data being consumed. I dislike the idea of using t

RE: ordering

2013-08-22 Thread Yu, Libo
Hi Jun, This is my original concern: Assume the number of consumer threads is the same as the number of partitions of a topic. And we want to leverage the fact that messages in a partition are ordered. But when starting multiple consumer threads or rebalancing, it is possible that one consumer

RE: broker never comes back to ISR

2013-08-22 Thread Yu, Libo
Thanks for investigating the thread dump. Our log only covers about 5 hours. So I didn't find any error log about fetcher thread. After a night, we notice for one topic, that broker goes back to its isr but for the other topics, it is still not in their isr. I did a restart for that broker. Af

Re: Kafka Mirroring setup

2013-08-22 Thread Andrew Otto
Great, thanks for the answers all! What about just for redundancy/maintenance purposes? I know that with replication, individual brokers are redundant anyway, but might it ever be nice to be able to take the analytics kafka cluster offline completely without worrying about losing data from the