Re: what response does broker send when there is not message on the partition

2015-04-21 Thread Alexey Borschenko
We trying to implement *offset rewind* procedure. For this we need to be able to get offset for topic and partition for specified timestamp. On Tue, Apr 21, 2015 at 4:36 AM, Gwen Shapira wrote: > I believe it doesn't take consumers into account at all. Just the > offset available on the partitio

Re: Kafka Zookeeper queries

2015-04-21 Thread Achanta Vamsi Subhash
We are planning to have ~2 partitions. Will it be a bottleneck? On Mon, Apr 20, 2015 at 10:48 PM, Jiangjie Qin wrote: > Producers usually do not query zookeeper at all. > Consumers usually query zookeeper at beginning or rebalance. It is > supposed to be in frequent if you don¹t have consume

[ANNOUNCEMENT] Kafka Tool Beta Version Released

2015-04-21 Thread marko
Hi Everyone, Kafka Tool is a new cross-platform GUI tool for working with Kafka. Some of the main features include -Quickly view all your Kafka clusters, including their brokers, topics and consumers -View contents of messages in your partitions and add new messages -View offsets of the Kafka con

Fetch API Offset

2015-04-21 Thread Piotr Husiatyński
According to documentation, sending fetch request with offset value result in messages starting with given offset (including) and greater. https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-FetchAPI > Logically one specifies the topics, partit

Re: Fetch API Offset

2015-04-21 Thread Manoj Khangaonkar
Hi, I suspect If some message from the given offset have expired, then they will not be returned. regards On Tue, Apr 21, 2015 at 5:14 AM, Piotr Husiatyński wrote: > According to documentation, sending fetch request with offset value > result in messages starting with given offset (including)

Re: Fetch API Offset

2015-04-21 Thread Piotr Husiatyński
What do you mean that message has expired? If I will do fetch request with offset 4, I will receive first message with offset 5. I think fetch is always excluding requested offset, while documentation is saying it will be included in response. On Tue, Apr 21, 2015 at 3:35 PM, Manoj Khangaonkar w

Re: what response does broker send when there is not message on the partition

2015-04-21 Thread Gwen Shapira
So "offsetRequest" should give you what you need. Or the SimpleConsumer getOffsetsBefore. You don't really need a specific consumer for this - just a topic, partition and timestamp. On Tue, Apr 21, 2015 at 12:12 AM, Alexey Borschenko wrote: > We trying to implement *offset rewind* procedure. For

Re: Fetch API Offset

2015-04-21 Thread Manoj Khangaonkar
Hi, I have used code like FetchRequest req = new FetchRequestBuilder() .clientId(clientname) .addFetch(topic, partition, offset_in_partition, 10).build(); FetchResponse fetchResponse = consumer.fetch(req); And it returns the message that I was looking for. The offset_in_

Re: Fetch API Offset

2015-04-21 Thread Piotr Husiatyński
I'm using Go with kafka library** and sending fetch request as described in wire protocol documentation returns messages off by one. Are you sure that code that you wrote is not sending fetch request with offset being one less than you pass? ** https://github.com/optiopay/kafka On Tue, Apr 21, 20

Kafka start server throws could not reserve enough space for object heap

2015-04-21 Thread Rajesh Datla
Hi All, Kafka start server throws could not reserve enough space for object heap used command: bin/kafka-server-start.sh config/server.properties Thanks Raj

partition reassignment stuck

2015-04-21 Thread Wes Chow
I started a partition reassignment (this is a 8.1.1 cluster) some time ago and it seems to be stuck. Partitions are no longer getting moved around, but it seems like the cluster is operational otherwise. The stuck nodes have a lot of .index files, and their logs show errors

Re: Kafka server - conflicted ephemeral node

2015-04-21 Thread Mayuresh Gharat
This happens due to a bug in zookeeper, sometimes the znode does not get deleted automatically.We have seen it many times at Linkedin and are trying to investigate further. Thanks, Mayuresh On Mon, Apr 20, 2015 at 8:52 PM, 小宇 wrote: > Thanks for your response gharatmayuresh1, but I don't know

Re: Kafka Zookeeper queries

2015-04-21 Thread Jiangjie Qin
2 partitions should be OK. On 4/21/15, 12:33 AM, "Achanta Vamsi Subhash" wrote: >We are planning to have ~2 partitions. Will it be a bottleneck? > >On Mon, Apr 20, 2015 at 10:48 PM, Jiangjie Qin >wrote: > >> Producers usually do not query zookeeper at all. >> Consumers usually query zoo

Re: partition reassignment stuck

2015-04-21 Thread Jiangjie Qin
Those .index files are for different partitions and they should be generated if new replicas is assigned to the broker. We might want to know what caused the UnknownException. Did you see any error log on broker 28? Jiangjie (Becket) Qin On 4/21/15, 9:16 AM, "Wes Chow" wrote

Re: partition reassignment stuck

2015-04-21 Thread Wes Chow
Not for that particular partition, but I am seeing these errors on 28: kafka.common.NotAssignedReplicaException: Leader 28 failed to record follower 25's position 0 for partition [click_engage,116] since the replica 25 is not recognized to be one of the assigned r eplicas for partition [clic

Re: partition reassignment stuck

2015-04-21 Thread Jiangjie Qin
This means that the broker 0 thought broker 28 was leader for that partition but broker 28 has actually already received StopReplicaRequest from controller and stopped serving as a replica for that partition. This might happen transiently but broker 0 will be able to find the new leader for the

Re: Kafka Zookeeper queries

2015-04-21 Thread pushkar priyadarshi
In my knowledge if you are using 0.8.2.1 which is latest stable you can sync up your consumer offsets in kafka itself instead of Zk which further brings down write load on ZKs. Regards, Pushkar On Tue, Apr 21, 2015 at 1:13 PM, Jiangjie Qin wrote: > 2 partitions should be OK. > > On 4/21/15,

Re: Kafka Zookeeper queries

2015-04-21 Thread Achanta Vamsi Subhash
@Qin Thanks for the inputs. Could you point us what we should monitor in zookeeper if we have 20k partitions (this might go ~1 lakh in the coming 6 months)? We are not running zookeeper on SSDs and we were facing ZkConnectionLoss in HighLevelConsumer, sometimes for all the topics. What are the crit

Re: Fetch API Offset

2015-04-21 Thread Andrei
@Piotr I can confirm that in native (Java/Scala) library returned message offsets start with the one specified in request, not the next one. Note sure about underlying protocol, though. As long as you also know Python, I would suggest you looking at kafka-python [1] library that is pretty readable

Warn No Checkpointed highwatermark is found for partition

2015-04-21 Thread pushkar priyadarshi
I Get warnings in server log saying "No checkpointed highwatermark is found for partition" in server.log when trying to create a new topic. What does this mean?Though this is warning was curious to know if it implies of any potential problem. Thanks And Regards, Pushkar

Re: Warn No Checkpointed highwatermark is found for partition

2015-04-21 Thread pushkar priyadarshi
I think its ok for this to come in the start when topic is created as there are no high watermark(offset of last commited message) check pointed. Got to understand this from this blog https://engineering.linkedin.com/kafka/intra-cluster-replication-apache-kafka Thanks And Regards, Pushkar On Tue,

what to do if replicas are not in sync

2015-04-21 Thread Thomas Kwan
We have 5 kafka brokers available, and created a topic with replication factor of 3. After a few broker issues (e.g. went out of file descriptors), running kafkacat on the producer node shows the following: Command: kafkacat-CentOS-6.5-x86_64 -L -b "kafka01-east.manage.com, kafka02-east.manage.co

Re: Kafka start server throws could not reserve enough space for object heap

2015-04-21 Thread Otis Gospodnetic
Short version: probably don't have enough free memory relative to what you have in -Xmx. top, free -m, and other monitoring tools and utilities will share more light on this. Otis -- Monitoring * Alerting * Anomaly Detection * Centralized Log Management Solr & Elasticsearch Support * http://semate

Re: Kafka Monitoring using JMX

2015-04-21 Thread Otis Gospodnetic
Hi, Not sure what you are using to get the data out, but if you use SPM you don't have to worry about any of that... assuming you use either 0.7.x or 0.8.2.x. Kafka metrics have changed drastically in the past and it looks like more chan

Re: Fw: How to measure performance of Mirror Maker

2015-04-21 Thread Otis Gospodnetic
Hi, I don't know much about MM yet, but what sort of MM performance are you looking for? If MM acts as just another producer for another Kafka cluster (just a guess!) then you could look at the metrics of the consuming cluster to see how fast it's ingesting but if MM works differently, then

Re: partition reassignment stuck

2015-04-21 Thread Wes Chow
Quick clarification: you say broker 0, but do you actually mean broker 25? 25 one of the replicas for the partition, is currently the one having trouble getting into sync, and 28 is the leader for the partition. Unfortunately, the logs of rotated off so I can't get to what happened around th

Re: Fw: How to measure performance of Mirror Maker

2015-04-21 Thread Daniel Compton
>From memory, MirrorMaker is just using Kafka Producers and Consumers to send the data from one DC to the other. So the meaningful performance metric I would be looking at is how far behind your mirror queues are from your source queues. Your other performance metrics are going to be very dependent

Re: [DISCUSS] KIP-12 - Kafka Sasl/Kerberos implementation

2015-04-21 Thread Jay Kreps
Hey Sriharsha, Thanks for the excellent write-up. Couple of minor questions: 1. Isn't the blocking handshake going to be a performance concern? Can we do the handshake non-blocking instead? If anything that causes connections to drop can incur blocking network roundtrips won't that eat up all th

Re: [DISCUSS] KIP-12 - Kafka Sasl/Kerberos implementation

2015-04-21 Thread Sriharsha Chintalapani
Hi Jay,       Thanks for the review.     1. Isn't the blocking handshake going to be a performance concern? Can we  do the handshake non-blocking instead? If anything that causes connections  to drop can incur blocking network roundtrips won't that eat up all the  network threads immediately? I gu

Re: Kafka Zookeeper queries

2015-04-21 Thread Jiangjie Qin
I might not be the best one to answer this question. It will be better if someone has more operation experience can help out here. Jiangjie (Becket) Qin On 4/21/15, 12:00 PM, "Achanta Vamsi Subhash" wrote: >@Qin >Thanks for the inputs. Could you point us what we should monitor in >zookeeper if

Re: partition reassignment stuck

2015-04-21 Thread Jiangjie Qin
Yes, should be broker 25 thread 0 from the log. This needs to be resolved, you might need to bounce both of the brokers who think itself as controller respectively. The new controller should be able to continue the partition reassignment. From: Wes Chow mailto:w...@chartbeat.com>> Reply-To: "use

Re: what to do if replicas are not in sync

2015-04-21 Thread Gwen Shapira
They should be trying to get back into sync on their own. Do you see any errors in broker logs? Gwen On Tue, Apr 21, 2015 at 10:15 AM, Thomas Kwan wrote: > We have 5 kafka brokers available, and created a topic with replication > factor of 3. After a few broker issues (e.g. went out of file desc

Re: partition reassignment stuck

2015-04-21 Thread Wesley Chow
There is only one broker that thinks it's the controller right now. The double controller situation happened earlier this morning. Do the other brokers have to be bounced after the controller situation is fixed? I did not do that for all brokers. Wes On Apr 21, 2015 8:25 PM, "Jiangjie Qin" wrot

Re: partition reassignment stuck

2015-04-21 Thread Jiangjie Qin
Hard to say, but if you have producers keeping producing data and they work well then probably you don¹t need to. On 4/21/15, 5:34 PM, "Wesley Chow" wrote: >There is only one broker that thinks it's the controller right now. The >double controller situation happened earlier this morning. Do the

Offset management: client vs broker side responsibility

2015-04-21 Thread Stevo Slavić
Hello Apache Kafka community, Please correct me if wrong, AFAIK currently (Kafka 0.8.2.x) offset management responsibility is mainly client/consumer side responsibility. Wouldn't it be better if it was broker side only responsibility? E.g. now if one wants to use custom offset management, any of

Re: running kafka/zookeeper running locally

2015-04-21 Thread Maciej Jaśkowski
I creatively reused this code to have embedded kafka and zookeeper working for test purposes: https://github.com/apache/samza/blob/master/samza-test/src/test/scala/org/apache/samza/test/integration/TestStatefulTask.scala Works like a charm! MJ On 20 April 2015 at 10:29, sunil kalva wrote: > Wa