Typical log.retention.bytes value

2016-08-26 Thread chris snow
When configuring a new Kafka cluster, is it common practice to leave the l og.retention.bytes setting at -1? If not -1, it would be interesting to know what size partitions other users have. Many thanks, Chris

Re: KIP-33 Opt out from Time Based indexing

2016-08-26 Thread Jan Filipiak
Hi Jun, thanks for taking the time to answer on such a detailed level. You are right Log.fetchOffsetByTimestamp works, the comment is just confusing "// Get all the segments whose largest timestamp is smaller than target timestamp" wich is apparently is not what takeWhile does (I am more on

Re: KIP-33 Opt out from Time Based indexing

2016-08-26 Thread Becket Qin
Jun, Good point about new log rolling behavior issue when move replicas. Keeping the old behavior sounds reasonable to me. Currently the time index entry points to the exact shallow message with the indexed timestamp, are you suggesting we change it to point to the starting offset of the

Re: Batch Expired

2016-08-26 Thread R Krishna
Are any requests at all making it? That is a pretty big timeout. However, I noticed if there is no connections made to broker, you can still get batch expiry. On Fri, Aug 26, 2016 at 6:32 AM, Ghosh, Achintya (Contractor) < achintya_gh...@comcast.com> wrote: > Hi there, > > What is the

Re: Recommendation for producer batch size

2016-08-26 Thread R Krishna
Don't think it always fills up buffer size before sending and then there is linger, that can add to the delay. From what I read, it depends on your use case. This link talks about

Monitoring the max lag of a kafka streams application.

2016-08-26 Thread Rohit Valsakumar
Hi all, I want to monitor the max lag of a kafka streams job which is consuming from three topics and to do that I have implemented the MetricsReporter interface which I pass through the Streams Configuration to the KafkaStreams object. In the implementation’s metricChange() method I have

Regarding log.retention.bytes config

2016-08-26 Thread Amit Karyekar
Hi, We’re using Kafka 0.9 Wanted to check whether log.retention.bytes works on per partition basis or is it cumulative of all partitions? Regards, Amit Information contained in this e-mail message is confidential. This e-mail message is intended only for the personal use of the recipient(s)

Re: Joining Streams with Kafka Streams

2016-08-26 Thread Guozhang Wang
I would suggest you trying the high-level Streams DSL as Michael mentioned, it implements this kind of joins in a slight different way (i.e. as two join processors against each other's materialized view, followed by a merge processor node), and hence the above two issues you mentioned are

Re: KTable aggregations send intermediate results downstream?

2016-08-26 Thread Guozhang Wang
The comment from Kasier Chen on the patch is correct: 1. Objects.equals() depends on the typed class object.equal() function, which may not be safe: for example users can override the equals function to only compare a subset of all the fields, whereas for Kafka Streams we need these two objects

Recommendation for producer batch size

2016-08-26 Thread Andrew Jorgensen
I was wonder if there are any good rules of thumb for determining the optimal batch size for the producer. For example lets say I have a group of producers that are in aggregate producing messages at about 40 million per minute with an average size of 700 bytes per message. With the default of

Re: KIP-33 Opt out from Time Based indexing

2016-08-26 Thread Jun Rao
Jiangjie, I am not sure about changing the default to LogAppendTime since CreateTime is probably what most people want. It also doesn't solve the problem completely. For example, if you do partition reassignment and need to copy a bunch of old log segments to a new broker, this may cause log

Re: Networking errors and durability settings

2016-08-26 Thread Bryan Baugher
Yes its quite likely we saw many zk session losses for the brokers around the same time. I'll keep an eye on that JIRA and let you know if we come up with anything else On Fri, Aug 26, 2016 at 11:44 AM Jun Rao wrote: > Bryan, > > Were there multiple brokers losing ZK session

Re: Networking errors and durability settings

2016-08-26 Thread Jun Rao
Bryan, Were there multiple brokers losing ZK session around the same time? There is one known issue https://issues.apache.org/jira/browse/KAFKA-1211. Basically, if the leader changes too quickly, it's possible for a follower to truncate some previous committed messages and then immediately

RE: unexpected consumer rebalance 0.9.0.1

2016-08-26 Thread Burtsev, Kirill
I also have seen this happening. Though, error said that it could be because of commit intervals were greater than configured timeout, it is not the case. -Original Message- From: Jaikiran Pai [mailto:jai.forums2...@gmail.com] Sent: Friday, August 26, 2016 8:50 AM To:

Re: kafka client on cloud

2016-08-26 Thread Jaikiran Pai
Can you explain what exactly you mean by "cloud" and what kind of restrictions you are running into in trying to point to the truststore location? -Jaikiran On Friday 19 August 2016 08:09 PM, Nomar Morado wrote: kafka consumer/producer currently require path to keystore/truststore. my client

Re: Networking errors and durability settings

2016-08-26 Thread Bryan Baugher
We didn't suffer any data loss nor was there any power outage that I know of. On Fri, Aug 26, 2016 at 5:14 AM Khurrum Nasim wrote: > On Tue, Aug 23, 2016 at 9:00 AM, Bryan Baugher wrote: > > > > > Hi everyone, > > > > > > Yesterday we had lots of

Re: unexpected consumer rebalance 0.9.0.1

2016-08-26 Thread Jaikiran Pai
What's the heartbeat interval that you have set on these consumer configs (if any)? Can you also paste a snippet of your code to show what the consumer code looks like (including the poll and commit calls)? -Jaikiran On Tuesday 23 August 2016 07:55 PM, Franco Giacosa wrote: Hi I am

Re: Kafka 0.8.2.2 - CLOSE_WAITS on broker

2016-08-26 Thread Jaikiran Pai
Which Java vendor and version are you using in runtime? Also what OS is this? Can you get the lsof output (on Linux) and paste the output of that to some place (like gist) to show us what descriptors are open etc... -Jaikiran On Friday 26 August 2016 02:49 AM, Bharath Srinivasan wrote:

Batch Expired

2016-08-26 Thread Ghosh, Achintya (Contractor)
Hi there, What is the recommended Producer setting for Producer as I see a lot of Batch Expired exception even though I put request.timeout=6. Producer settings: acks=1 retries=3 batch.size=16384 linger.ms=5 buffer.memory=33554432 request.timeout.ms=6 timeout.ms=6 Thanks Achintya

Re: consumer with version 0.10.0

2016-08-26 Thread Jaikiran Pai
Is anyone producing any (new) messages to the topics you are subscribing to in that consumer? -Jaikiran On Friday 26 August 2016 10:14 AM, Jack Yang wrote: Hi all, I am using kafka 0.10.0.1, and I set up my listeners like: listeners=PLAINTEXT://myhostName:9092 then I have one consumer going

Re: Kafka streams

2016-08-26 Thread Matthias J. Sax
Hi, If you need multiple polls to receive more data before you start processing, you should disable auto commit (via `auto.commit.enable=false`). Thus, no commit happens on poll() -- of course, you need to do commits manually. Kafka Streams also uses this strategy internally. About KTable: if a

Re: Networking errors and durability settings

2016-08-26 Thread Khurrum Nasim
On Tue, Aug 23, 2016 at 9:00 AM, Bryan Baugher wrote: > > > Hi everyone, > > > > Yesterday we had lots of network failures running our Kafka cluster > > (0.9.0.1 ~40 nodes). We run everything using the higher durability > settings > > in order to avoid in data loss, producers

Re: Joining Streams with Kafka Streams

2016-08-26 Thread Michael Noll
First a follow-up question, just in case (sorry if that was obvious to you already): Have you considered using Kafka Streams DSL, which has much more convenient join functionality built-in out of the box? The reason I am asking is that you didn't specifically mention that you did try using the

Re: Kafka streams

2016-08-26 Thread Abhishek Agarwal
Thanks Matthias and Eno. For at least once guarantees to be effective in any system, the source (Kafka receiver) needs to know that the message has been successfully processed. What I understood is that since processing happens in single thread, if another record is polled, it is implicitly

Re: Question regarding functionality of MirrorMaker

2016-08-26 Thread cs user
Hi Umesh, I haven't had that problem, it seems to work fine for me. The only issue I found, which kind of makes sense, it that it doesn't mirror existing topics immediately, only when messages are first set to the topic after mirror maker connects. It doesn't start from the first offset

Re: oom of kafka

2016-08-26 Thread 黄川
thanks for your reply, Kirsch. yes , i have analyzed using MAT, the output for memory leak is as follows: [image: 内嵌图片 1] 2016-08-26 14:34 GMT+08:00 Harald Kirsch : > There is hardly any way anyone can guess what happens there from just the > numbers. > > What you

Re: oom of kafka

2016-08-26 Thread Harald Kirsch
There is hardly any way anyone can guess what happens there from just the numbers. What you should do is start Kafka with -XX:+HeapDumpOnOutOfMemoryError, possibly even reduce the Xmx to 500MB and let it bomb out. Then you take a look at the generated heap dump with the Eclipse Memory