RE: async

2015-06-01 Thread Aditya Auradkar
My bad. The exposed option is sync, so omitting that should default to async. Aditya From: ram kumar [ramkumarro...@gmail.com] Sent: Sunday, May 31, 2015 10:43 PM To: users@kafka.apache.org Subject: Re: async async option is not available in 0.8.2.1 On

Re: How to correctly handle offsets?

2015-06-01 Thread svante karlsson
1) correlationId is just a number that you get back in your reply. you can safely set it to anything. If you have some kind of call identification is your system that you want to trace through logs - this is what you would use. 2) You can safely use any external offset management you like. just

How to correctly handle offsets?

2015-06-01 Thread luo.fucong
Hi all: I am using Kafka 0.8.2 and SimpleConsumer in maven: dependency groupIdorg.apache.kafka/groupId artifactIdkafka_2.11/artifactId version0.8.2.1/version /dependency I follow the SimpleConsumer example in the wiki, and there are some questions: 1. There seems lacking of how to

Re: Cascading failures on running out of disk space

2015-06-01 Thread Jason Rosenberg
Hi Jananee, Do you for sure that you ran out of disk space completely? Did you see an IOExceptions failing to write? Normally, when that happens, the broker is supposed to immediately shut itself down. Did the one broker shut itself down? The NotLeaderForPartitionException's are normal when

Re: Broker error: failed due to Leader not local for partition

2015-06-01 Thread Jason Rosenberg
failed due to leader not local for partition usually occurs in response to client requests that make a fetch or produce request to a partition, to the wrong broker (e.g. to a follower and not the leader for the partition). Clients need to make a meta-data request first to determine the leader

Re: Kafka broker - Ip-address instead of host naem

2015-06-01 Thread Jason Rosenberg
Daniel, Are you sure about that (it's not what I would have understood). Generally, the way to do it is use a round-robin dns entry, which returns successive nodes for successive requests. Kafka will retry a zookeeper request on failure (and in the process get re-routed to a new zk node). If a

Re: leader update partitions fail with KeeperErrorCode = BadVersion,kafka version=0.8.1.1

2015-06-01 Thread Jason Rosenberg
I've seen this problem now too with 0.8.2.1. It happened after we had a disk failure (but the server failed to shutdown: KAFKA-). After that happened, subsequently, several ISR's underwent I think 'unclean leader election', but I'm not 100% sure. But I did see lots of those same error

Kafka issue with FetchRequest buffer size

2015-06-01 Thread Zhenzhong Xu
Hi, We recently ran into a scenario where we initiate a FetechRequest with a fixed fetchSize (64k) shown below using Simple Consumer. When the broker contains an unusually large sized message, this resulted in the broker returns an empty message set *without any error code*. According to the

Re: aborting a repartition assignment

2015-06-01 Thread Jason Rosenberg
As Lance mentioned, the best course of action in such a case (since version 0.8.X) is to keep the failed broker down, and bring up a new node (with the same broker id as the failed broker), and it will automatically re-sync its replicas (which may take some time). You don't want to try to

Re: Ordered Message Queue with Pool of Consumers

2015-06-01 Thread Jason Rosenberg
How would you apply total ordering if multiple messages are being consumed in parallel? If message-1 and message-2 are being consumed in parallel, do you really mean you want to guarantee that message-1 is consumed before the consumption of message-2 begins? On Tue, May 26, 2015 at 1:34 PM,

Re: consumer poll returns no records unless called more than once, why?

2015-06-01 Thread Jason Rosenberg
Ben, It could also be related to how you initialize auto.offset.reset. In unit tests, you generally want to set it to 'smallest' to avoid race conditions between producing and consuming. Jason On Wed, May 20, 2015 at 2:32 PM, Padgett, Ben bpadg...@illumina.com wrote: Thanks for the

Re: KafkaException: Size of FileMessageSet has been truncated during write

2015-06-01 Thread Jason Rosenberg
Might be good to have a more friendly error message though! On Thu, May 28, 2015 at 4:32 PM, Andrey Yegorov andrey.yego...@gmail.com wrote: Thank you! -- Andrey Yegorov On Wed, May 27, 2015 at 4:42 PM, Jiangjie Qin j...@linkedin.com.invalid wrote: This should be just a message

Re: Kafka partitions unbalanced

2015-06-01 Thread Jason Rosenberg
Andrew Otto, This is a known problem (and which I have run into as well). Generally, my solution has been to increase the number of partitions such that the granularity of partitions is much higher than the number of disks, such that its more unlikely for the imbalance to be significant. I

Re: Offset management: client vs broker side responsibility

2015-06-01 Thread Jason Rosenberg
Stevo, Both of the main solutions used by the high-level consumer are standardized and supported directly by the kafka client libraries (e.g. maintaining offsets in zookeeper or in kafka itself). And for the zk case, there is the consumer offset checker (which is good for monitoring). Consumer

potential bug with offset request and just rolled log segment

2015-06-01 Thread Alfred Landrum
I'm running 0.8.2 in a setup that's rolling segments relatively often. Using my non-Kafka client that's using librdkafka, I occasionally see this in my kafka logs: ... [2015-05-30 00:30:26,149] INFO Rolled new log segment for 'receiver-db1fc9b7-85b2-4016-8eda-66a356cb2ae8-35' in 1 ms.

Re: potential bug with offset request and just rolled log segment

2015-06-01 Thread Guozhang Wang
Hi Alfred, I check the 0.8.2 code and may probably find a bug related to your issue. Basically, segsArray.last.size is called multiple times during handling offset requests, while segsArray.last could get concurrent appends. Hence it is possible that in line 461, if(segsArray.last.size 0)