Controlled shutdown failure, retry settings

2013-10-25 Thread Jason Rosenberg
I'm running into an issue where sometimes, the controlled shutdown fails to complete after the default 3 retry attempts. This ended up in one case, with a broker under going an unclean shutdown, and then it was in a rather bad state after restart. Producers would connect to the metadata vip, stil

Please explain the behaviour of bin/kafka-list-topic.sh command

2013-10-25 Thread Monika Garg
Hi My zookeeper node in Kafka is storing the data for a partition like this: { "controller_epoch":3, "isr":[ 1, 3 ], "leader":1, "leader_epoch":1, "version":1 } So what I am thinking is the command *bin/kafka-list-topic.sh *uses the above data from zookeeper in fetching the its output* + *some a

Re: Does Kafka stores Producer/Consumer Details

2013-10-25 Thread Joel Koshy
> > I am wondering does metadata of Kafka stores producer/consumer information. > > As I start any producer,I provide a broker-list to it.But producer must be > connecting to one of the broker to send data(as per Im gettiing the > things).Similarly I can start more producer by giving a broker-list

Re: backoff.increment.ms support in kafka 0.8

2013-10-25 Thread Joel Koshy
> > kafka 0.7 consumer supported backoff.increment.ms to avoid repeatedly > polling a broker node which has no new data. It appears that this property > is no longer supported in 0.8. What is the reason? Kafka 0.7 did not support any polling policy which made fetching rather inefficient with a fix

Re: Controlled shutdown failure, retry settings

2013-10-25 Thread Joel Koshy
On Fri, Oct 25, 2013 at 1:18 AM, Jason Rosenberg wrote: > I'm running into an issue where sometimes, the controlled shutdown fails to > complete after the default 3 retry attempts. This ended up in one case, > with a broker under going an unclean shutdown, and then it was in a rather > bad state

Re: Controlled shutdown failure, retry settings

2013-10-25 Thread Neha Narkhede
Controlled shutdown can fail if the cluster has non zero under replicated partition count. Since that means the leaders may not move off of the broker being shutdown, causing controlled shutdown to fail. The backoff might help if the under replication is just temporary due to a spike in traffic. Th

Re: how request.required.acks works

2013-10-25 Thread Neha Narkhede
The producer acknowledgement is independent of the leader follower replication. So if the message is written to the leader and the followers are healthy, the message will get committed. Thanks, Neha On Oct 24, 2013 8:35 PM, "Kane Kane" wrote: > Hello Neha, does it mean even if not all replica ac

Re: Please explain the behaviour of bin/kafka-list-topic.sh command

2013-10-25 Thread Jun Rao
1. The information is obtained from ZK. For details, see https://cwiki.apache.org/confluence/display/KAFKA/Kafka+data+structures+in+Zookeeper 2. Broker 3 should be dropped our of isr. Are you running the latest code from the 0.8 branch? Any error in the controller and state-change log? Thanks, J

Re: Does Kafka stores Producer/Consumer Details

2013-10-25 Thread Jun Rao
Details of the consumer implementation can be found at http://research.microsoft.com/en-us/um/people/srikanth/netdb11/netdb11papers/netdb11-final12.pdf Thanks, Jun On Thu, Oct 24, 2013 at 11:13 PM, Monika Garg wrote: > Hi, > > > > I am wondering does metadata of Kafka stores producer/consumer

Re: how request.required.acks works

2013-10-25 Thread Kane Kane
Hello Neha, Can you explain please what this means: request.timeout.ms - The amount of time the broker will wait trying to meet the request.required.acks requirement before sending back an error to the client. So the message will be committed even though broker sends error to client? On Fri, Oct

Re: Controlled shutdown failure, retry settings

2013-10-25 Thread Jason Rosenberg
Ok, Looking at the controlled shutdown code, it appears that it can fail with an IOException too, in which case it won't report the remaining partitions to replicate, etc. (I think that might be what I'm seeing, since I never saw the log line for "controlled shutdown failed, X remaining partition

Re: Does Kafka stores Producer/Consumer Details

2013-10-25 Thread Monika Garg
Thanks Joel and Jun. So Joel,can we get the simple details that how many producers's connection a broker is maintaining? On Fri, Oct 25, 2013 at 8:25 PM, Jun Rao wrote: > Details of the consumer implementation can be found at > > http://research.microsoft.com/en-us/um/people/srikanth/netdb11/n

producer exceptions when broker dies

2013-10-25 Thread Kane Kane
I have cluster of 3 kafka brokers. With the following script I send some data to kafka and in the middle do the controlled shutdown of 1 broker. All 3 brokers are ISR before I start sending. When i shutdown the broker i get a couple of exceptions and I expect data shouldn't be written. Say, I send

Re: Controlled shutdown failure, retry settings

2013-10-25 Thread Neha Narkhede
Jason, The state change log tool is described here - https://cwiki.apache.org/confluence/display/KAFKA/Replication+tools#Replicationtools-7.StateChangeLogMergerTool I'm curious what the IOException is and if we can improve error reporting. Can you send around the stack trace ? Thanks, Neha On

Re: Controlled shutdown failure, retry settings

2013-10-25 Thread Jason Rosenberg
Neha, It looks like the StateChangeLogMergerTool takes state change logs as input. I'm not sure I know where those live? (Maybe update the doc on that wiki page to describe!). Thanks, Jason On Fri, Oct 25, 2013 at 12:38 PM, Neha Narkhede wrote: > Jason, > > The state change log tool is desc

Re: how request.required.acks works

2013-10-25 Thread Guozhang Wang
Hi Kane, You can find details of request.timeout.ms here: http://kafka.apache.org/documentation.html#configuration And yes, if a "timeout" response to sent back to the producer, it means the message may or may not have been committed. Guozhang On Fri, Oct 25, 2013 at 8:05 AM, Kane Kane wrote

Re: producer exceptions when broker dies

2013-10-25 Thread Guozhang Wang
Hello Kane, As discussed in the other thread, even if a timeout response is sent back to the producer, the message may still be committed. Did you shut down the leader broker of the partition or a follower broker? Guozhang On Fri, Oct 25, 2013 at 8:45 AM, Kane Kane wrote: > I have cluster of

Re: producer exceptions when broker dies

2013-10-25 Thread Kane Kane
Hello Guozhang, My partitions are split almost evenly between broker, so, yes - broker that I shutdown is the leader for some of them. Does it mean i can get an exception and data is still being written? Is there any setting on the broker where i can control this? I.e. can i make broker replicatio

Re: producer exceptions when broker dies

2013-10-25 Thread Kane Kane
Or, to rephrase it more generally, is there a way to know exactly if message was committed or no? On Fri, Oct 25, 2013 at 10:43 AM, Kane Kane wrote: > Hello Guozhang, > > My partitions are split almost evenly between broker, so, yes - broker > that I shutdown is the leader for some of them. Doe

Re: how request.required.acks works

2013-10-25 Thread Kane Kane
Thanks a lot Guohang, this makes sense. On Fri, Oct 25, 2013 at 10:32 AM, Guozhang Wang wrote: > Hi Kane, > > You can find details of request.timeout.ms here: > > http://kafka.apache.org/documentation.html#configuration > > And yes, if a "timeout" response to sent back to the producer, it means

Re: producer exceptions when broker dies

2013-10-25 Thread Aniket Bhatnagar
As per my understanding, if the broker says the msg is committed, its guaranteed to have been committed as per ur ack config. If it says it did not get committed, then its very hard to figure out if this was just a false error. Since there is concept of unique ids for messages, a replay of the sam

Re: producer exceptions when broker dies

2013-10-25 Thread Kane Kane
Hello Aniket, Thanks for the answer, this totally makes sense and implementing that layer on consumer side to check for dups sound like a good solution to this issue. Can we get a confirmation from kafka devs that this is how kafka supposed to work (by design) and how we should implement the solu

Re: producer exceptions when broker dies

2013-10-25 Thread Guozhang Wang
Aniket is exactly right. In general, Kafka provides "at least once" guarantee instead of "exactly once". Guozhang On Fri, Oct 25, 2013 at 11:13 AM, Aniket Bhatnagar < aniket.bhatna...@gmail.com> wrote: > As per my understanding, if the broker says the msg is committed, its > guaranteed to have

Re: producer exceptions when broker dies

2013-10-25 Thread Kane Kane
I.e. from the documentation: So effectively Kafka guarantees at-least-once delivery by default and allows the user to implement at most once delivery by disabling retries on the producer I've disabled retries but it's not at-most-once which my test proves. It's still at-least-once. On Fri, Oct

Re: producer exceptions when broker dies

2013-10-25 Thread Kane Kane
Guozhang, but i've posted a piece from kafka documentation above: So effectively Kafka guarantees at-least-once delivery by default and allows the user to implement at most once delivery by disabling retries on the producer. What i want is at-most-once and docs claim it's possible with certain set

Re: producer exceptions when broker dies

2013-10-25 Thread Steve Morin
Kane and Aniket, I am interested in knowing what the pattern/solution that people usually use to implement exactly once as well. -Steve On Fri, Oct 25, 2013 at 11:39 AM, Kane Kane wrote: > Guozhang, but i've posted a piece from kafka documentation above: > So effectively Kafka guarantees at-l

Re: producer exceptions when broker dies

2013-10-25 Thread Guozhang Wang
Kane, If you set message.send.max.retries to 0 it should be at-most-once, and I saw your props have the right config. What are the exceptions you got from the send() call? Guozhang On Fri, Oct 25, 2013 at 12:54 PM, Steve Morin wrote: > Kane and Aniket, > I am interested in knowing what the

Re: Controlled shutdown failure, retry settings

2013-10-25 Thread Jason Rosenberg
It looks like when the controlled shutdown failes with an IOException, the exception is swallowed, and we see nothing in the logs: catch { case ioe: java.io.IOException => channel.disconnect() channel = null // ignore and tr

Re: Broker bind address versus published hostname in ZooKeeper

2013-10-25 Thread Roger Hoover
Ok. I'm working on it. On Thu, Oct 24, 2013 at 10:02 AM, Timothy Chen wrote: > Hi Folks/Roger, > > Unfortunately I don't have legal clearance to contribute patches yet back > to Kafka for code done at work, so Roger it will be great if you can > provide this patch. > > Thanks! > > Tim > > > >

How to run a single unit test with ./sbt

2013-10-25 Thread Roger Hoover
Hi, I'm new to Scala but working on a simple patch for a configuration change and want to run just my unit tests. When I run ./sbt test-only, it executes all sorts of other tests but not the one I want. Is there an easy way to run a single test? Any help is appreciated. $ ./sbt test-only kafka

Re: producer exceptions when broker dies

2013-10-25 Thread Kane Kane
There are a lot of exceptions, I will try to pick an example of each: ERROR async.DefaultEventHandler - Failed to send requests for topics benchmark with correlation ids in [879,881] WARN async.DefaultEventHandler - Produce request with correlation id 874 failed due to [benchmark,43]: kafka.common

Re: producer exceptions when broker dies

2013-10-25 Thread Guozhang Wang
As we have said, the timeout exception does not actually mean the message is not committed to the broker. When message.send.max.retries is 0 Kafka does guarantee "at-most-once" which means that you will not have duplicates, but not means that all your exceptions can be treated as "message not deliv

Re: Does Kafka stores Producer/Consumer Details

2013-10-25 Thread Joel Koshy
> So Joel,can we get the simple details that how many producers's connection > a broker is maintaining? I don't think we directly maintain that information but it is readily available through standard OS-level tools (e.g., netstat). Thanks, Joel > > > On Fri, Oct 25, 2013 at 8:25 PM, Jun Rao w

Re: Controlled shutdown failure, retry settings

2013-10-25 Thread Joel Koshy
On Fri, Oct 25, 2013 at 3:22 PM, Jason Rosenberg wrote: > It looks like when the controlled shutdown failes with an IOException, the > exception is swallowed, and we see nothing in the logs: > > catch { > case ioe: java.io.IOException => > channel.disconne

Re: How to run a single unit test with ./sbt

2013-10-25 Thread Joel Koshy
In the sbt shell: > projects (to see the available projects) > project core > test-only Although lately if there is a test failure it isn't very helpful in saying exactly where the test failed; my environment is probably messed up but I know of one or two others who are having similar issues. J

Re: Flush configuration per topic

2013-10-25 Thread Felix GV
Hi guys, I don't see log.flush.interval being available on per-topic basis in 0.7.2 either (only the time interval is configurable per topic), as in 0.8. Is it correct to say that this is only going to be available in 0.8.1 and isn't available in 0.7.x? Thanks a lot guys (: ... -- Felix On Fr

Re: producer exceptions when broker dies

2013-10-25 Thread Kane Kane
Thanks Guozhang, it makes sense if it's by design. Just wanted to ensure i'm not doing something wrong. On Fri, Oct 25, 2013 at 5:57 PM, Guozhang Wang wrote: > As we have said, the timeout exception does not actually mean the message > is not committed to the broker. When message.send.max.retri

Re: Flush configuration per topic

2013-10-25 Thread Jun Rao
Yes, there won't be any feature development in 0.7. Thanks, Jun On Fri, Oct 25, 2013 at 6:32 PM, Felix GV wrote: > Hi guys, > > I don't see log.flush.interval being available on per-topic basis in 0.7.2 > either (only the time interval is configurable per topic), as in 0.8. > > Is it correct

Re: Controlled shutdown failure, retry settings

2013-10-25 Thread Jason Rosenberg
On Fri, Oct 25, 2013 at 9:16 PM, Joel Koshy wrote: > > Unclean shutdown could result in data loss - since you are moving > leadership to a replica that has fallen out of ISR. i.e., it's log end > offset is behind the last committed message to this partition. > > But if data is written with 'reque

Re: producer exceptions when broker dies

2013-10-25 Thread Jason Rosenberg
Just to clarify, I think in order to get 'at least once' guarantees, you must produce messages with 'request.required.acks=-1'. Otherwise, you can't be 100% sure the message was received by all ISR replicas. On Fri, Oct 25, 2013 at 9:56 PM, Kane Kane wrote: > Thanks Guozhang, it makes sense if