Re: Connection reset by peer

2013-03-28 Thread Yonghui Zhao
I used zookeeper-3.3.4 in kafka. Default tickTime is 3 seconds, minSesstionTimeOut is 6 seconds. Now I change tickTime to 5 seconds. minSesstionTimeOut to 10 seconds But if we change timeout to a larger one, "you have shutdown this broker and restarted it faster than the zookeeper timeout so it ap

log file flushing, flushall on clean shutdown...

2013-03-28 Thread Jason Rosenberg
Hi, We are managing our kafka clusters by doing rolling restarts (e.g. we cleanly shutdown and restart each broker one at a time). I'm working through an issue, whereby we are losing messages sent to a kafka broker, right before the broker is shutdown cleanly. I'm still using 0.7.2, so not sure

race condition with log flush interval settings...

2013-03-28 Thread Jason Rosenberg
It looks like there is a race condition between the settings for the 2 properties: log.default.flush.scheduler.interval.ms & log.default.flush.interval.ms. I'm using 0.7.2. By default, both of these get set to 3000ms (and in the docs, it recommends setting flushInterval to be a multiple of the f

Re:log file flushing, flushall on clean shutdown...

2013-03-28 Thread SuoNayi
Hi, the mail list has been moved to users@kafka.apache.org long time ago. You may resubscribe the list or the list may be removed in future. At 2013-03-28 15:39:07,"Jason Rosenberg" wrote: >Hi, > >We are managing our kafka clusters by doing rolling restarts (e.g. we >cleanly shutdown and resta

Re: log file flushing, flushall on clean shutdown...

2013-03-28 Thread Neha Narkhede
Jason, How do you know the server had written the lost data to its log ? In Kafka 0.7, data could be lost from the producer's or server's socket buffer. You can verify this by running DumpLogSegments before and after shutdown. Thanks, Neha On Thursday, March 28, 2013, SuoNayi wrote: > Hi, the m

Re: C/C++ Client

2013-03-28 Thread Magnus Edenhill
Hi Matt, regarding librdkafka: - the AF_NETLINK stuff has now been removed, a helper function had support for it but it had no relevance to Kafka operation. - TLS: it would be possible to remove the need for TLS in rdkafka, but it does feel like a leap back in time; even NetBSD has support for T

Re: Connection reset by peer

2013-03-28 Thread Jun Rao
The zk session timeout only kicks in if you force kill the consumer. Otherwise, consumer will close ZK session properly on clean shutdown. The problem with GC is that if the consumer pauses for a long time, ZK server won't receive pings from the client and thus can expire a still existing session.

Re: race condition with log flush interval settings...

2013-03-28 Thread Jun Rao
Jason, Could you file a jira so that we can track it? Thanks, Jun On Thu, Mar 28, 2013 at 12:43 AM, Jason Rosenberg wrote: > It looks like there is a race condition between the settings for the 2 > properties: log.default.flush.scheduler.interval.ms & > log.default.flush.interval.ms. I'm us

Re: log file flushing, flushall on clean shutdown...

2013-03-28 Thread Jason Rosenberg
Hi Neha, I enabled TRACE logging on the producer, and verified that it successfully wrote out the bytes to the server (but this was after the last log flush on the server, where trace logging was also enabled). I'm quite certain that the server is receiving the lost data, and then shutting down,

Re: Connection reset by peer

2013-03-28 Thread Yonghui Zhao
Thanks Jun. But I can't understand how consumer GC trigger kafka server issue: java.lang.RuntimeException: A broker is already registered on the path /brokers/ids/0. This probably indicates that you either have configured a brokerid that is already in use, or else you have shutdown this broker and

Re: Transactional writing

2013-03-28 Thread Jun Rao
Jonathan, With a single writer, the producer can achieve exact once write. If a send request fails, the producer first checks the end of the log to see if the previous write succeeded or not. The producer will only resend if the previous write fails. To do this, the producer needs the offset of a

Re: Connection reset by peer

2013-03-28 Thread Jun Rao
Do you see lots of ZK session expiration in the broker too? If so, that suggests a GC issue in the broker too. So, you may need to tune the GC in the broker as well. Thanks, Jun On Thu, Mar 28, 2013 at 8:20 AM, Yonghui Zhao wrote: > Thanks Jun. > > But I can't understand how consumer GC trigge

Re: log file flushing, flushall on clean shutdown...

2013-03-28 Thread Jun Rao
Jason, Our shutdown sequence in 0.7 is the following in KafkaServer.shutdown(): 1. close socket channel 2. close log manager 2.1 for each log 2.2 flush last segment and close file channel So, it does what you are suggesting. All unflushed data are supposed to be written to disk on a clean shut

Re: Connection reset by peer

2013-03-28 Thread Yonghui Zhao
Will do a check, I just wonder why broker need re-regiester and it failed, so broker service is stopped. 2013/3/28 Jun Rao > Do you see lots of ZK session expiration in the broker too? If so, that > suggests a GC issue in the broker too. So, you may need to tune the GC in > the broker as well. >

Re: C/C++ Client

2013-03-28 Thread Rohit Prasad
I have a working copy of kafka 0.8 producer in c++. I have not yet published it on github, since I did not get time to clean it up properly. If you wait till the weekend, I can clean up and share the repo. After that you can either use it or modify and use it. Thanks, Rohit On Thu, Mar 28, 2013

Re: C/C++ Client

2013-03-28 Thread mrevilgnome
I'm on OSX. It's not going to production on OSX, but I have a requirement that the entire stack be deployable on a developer's machine. OSX does have TLS, but not the local variable __thread. It would need to be reworked to use the pthreads API. That by itself isn't much work it's just I don't know

Re: C/C++ Client

2013-03-28 Thread mrevilgnome
I'm definitely interested. If you publish it please let me know. On Thu, Mar 28, 2013 at 8:33 AM, Rohit Prasad wrote: > I have a working copy of kafka 0.8 producer in c++. > I have not yet published it on github, since I did not get time to clean it > up properly. > If you wait till the weekend,

Re: Transactional writing

2013-03-28 Thread Jonathan Hodges
Awesome! Thanks for confirmation and continued great work on Kafka! On Thu, Mar 28, 2013 at 9:22 AM, Jun Rao wrote: > Jonathan, > > With a single writer, the producer can achieve exact once write. If a send > request fails, the producer first checks the end of the log to see if the > previous

Re: log file flushing, flushall on clean shutdown...

2013-03-28 Thread Jason Rosenberg
Ok, sorry, I see now that in fact, it does close all logs during LogManager.close(), which deeper in the code flushes logSegments. So it doesn't do so as explicitly as in LogManager.flushAllLogs during the normally scheduled flush interval. The confusing thing, is that I clearly see my message b

Re: race condition with log flush interval settings...

2013-03-28 Thread Jason Rosenberg
Filed: https://issues.apache.org/jira/browse/KAFKA-839 By the way, the jira queue needs to be updated to know that 0.7.2 is now a released version, etc. Jason On Thu, Mar 28, 2013 at 7:58 AM, Jun Rao wrote: > Jason, > > Could you file a jira so that we can track it? > > Thanks, > > Jun > > O

Re: log file flushing, flushall on clean shutdown...

2013-03-28 Thread Neha Narkhede
>> So, I am guessing now that the 499 bytes written message was going to a buffered socket channel, that received the bytes, but didn't actually send them out at the os level. No exception is thrown, so I am wondering whether this socket is ever flushed, or if it just quietly fails to send its dat

Re: log file flushing, flushall on clean shutdown...

2013-03-28 Thread Jason Rosenberg
Thanks Neha for the detailed comments. I'll look forward to 0.8 (maybe start playing with the latest version next week). Jason On Thu, Mar 28, 2013 at 2:18 PM, Neha Narkhede wrote: > >> So, I am guessing now that the 499 bytes written message was going to a > buffered socket channel, that recei

Re: Connection reset by peer

2013-03-28 Thread Jun Rao
Not sure why re-registering in broker fails. Normall, when the broker registers, the ZK path should already be gone. Thanks, Jun On Thu, Mar 28, 2013 at 8:31 AM, Yonghui Zhao wrote: > Will do a check, I just wonder why broker need re-regiester and it failed, > so broker service is stopped. > >