Re: rebalancing partitions across a jbod

2016-10-20 Thread Jeremy Hansen
After running, I see partitions on the new volumes that were added to my brokers. Both mount points are listed in log.dirs comma separated. After running kafka-reassign-partitions.sh, I now see partitions in the second additional volume. -jeremy > On Oct 20, 2016, at 10:30 PM, Manikumar w

Re: rebalancing partitions across a jbod

2016-10-20 Thread Manikumar
kafka-reassign-partitions.sh is for migrating data to new/other brokers. How it is useful for migrating data within same broker? you reassigned some partitions to other broker and moved back to original broker? On Fri, Oct 21, 2016 at 10:46 AM, Jeremy Hansen wrote: > So I seemed to have solved t

Re: rebalancing partitions across a jbod

2016-10-20 Thread Jeremy Hansen
So I seemed to have solved this issue just by running kafka-reassign-partitions.sh. I now see even distribution of partitions across both disk volumes. Does this make sense? Thanks -jeremy > On Oct 20, 2016, at 10:07 PM, Manikumar wrote: > > Kafka does not automatically migrate the the exis

RE: Mirror multi-embedded consumer's configuration

2016-10-20 Thread ZHU Hua B
Hi, Anybody could help to answer below question? Thanks! Best Regards Johnny From: ZHU Hua B Sent: 2016年10月19日 16:22 To: 'users@kafka.apache.org' Subject: Mirror multi-embedded consumer's configuration Hi, I launch Kafka mirror maker with multi-embedded consumer's configuration but fai

Re: rebalancing partitions across a jbod

2016-10-20 Thread Manikumar
Kafka does not automatically migrate the the existing partition data to new volumes. Only new partitions will placed on on new volumes. For now, you can manually copy some the partition dirs(careful with checkpoint files) to new disk, or you can increase the partitions. Or we can just delete the

Error Failed to update metadata after 60000 ms: Kafka 0.10 + Kerberos + HDP 2.5

2016-10-20 Thread Jagat Singh
We are trying to run a simple Kafka code with new api to test Kerberos integration. The code is present at https://pastebin.ubuntu.com/23356675/ We are using ticket based mechanism to test first and then we will use Keytabs java -Djava.security.krb5.conf=/etc/krb5.conf -Djava.security.auth.login

Re: Kafka Streaming

2016-10-20 Thread Mohit Anchlia
Any idea of when 3.2 is coming? On Thu, Oct 20, 2016 at 4:53 PM, Matthias J. Sax wrote: > -BEGIN PGP SIGNED MESSAGE- > Hash: SHA512 > > No problem. Asking questions is the purpose of mailing lists. :) > > The issue will be fixed in next version of examples branch. > > Examples branch is

Re: Kafka Streaming

2016-10-20 Thread Matthias J. Sax
-BEGIN PGP SIGNED MESSAGE- Hash: SHA512 I mixed up version numbers... Current CP version is 3.0.1 and not 3.1 Ie. 3.0.1 contains Kafka 0.10.0.1 and 3.1 will be released soon will contain Kafka 0.10.1.0 examples master uses CP-3.1-SNAPSHOT. Sorry for the confusion. On 10/20/16 4:53 P

Re: Kafka consumer rate

2016-10-20 Thread Hans Jespersen
Yes. See the description of quotas. https://kafka.apache.org/documentation#design_quotas -hans /** * Hans Jespersen, Principal Systems Engineer, Confluent Inc. * h...@confluent.io (650)924-2670 */ On Thu, Oct 20, 2016 at 3:20 PM, Adrienne Kole wrote: > Hi, > > Is there a way to limit the c

Re: Kafka Streaming

2016-10-20 Thread Matthias J. Sax
-BEGIN PGP SIGNED MESSAGE- Hash: SHA512 No problem. Asking questions is the purpose of mailing lists. :) The issue will be fixed in next version of examples branch. Examples branch is build with CP dependency and not with Kafka dependency. CP-3.2 is not available yet; only Kafka 0.10.1.0

Re: Kafka Streaming

2016-10-20 Thread Mohit Anchlia
So this issue I am seeing is fixed in the next version of example branch? Can I change my pom to point it the higher version of Kafka if that is the issue? Or do I need to wait until new branch is made available? Sorry lot of questions :) On Thu, Oct 20, 2016 at 3:56 PM, Matthias J. Sax wrote: >

Re: Kafka Streaming

2016-10-20 Thread Matthias J. Sax
-BEGIN PGP SIGNED MESSAGE- Hash: SHA512 The branch is 0.10.0.1 and not 0.10.1.0 (sorry for so many zeros and ones -- super easy to mix up) However, examples master branch uses CP-3.1-SNAPSHOT (ie, Kafka 0.10.1.0) -- there will be a 0.10.1 examples branch, after CP-3.1 was released - -Ma

Re: Kafka Streaming

2016-10-20 Thread Mohit Anchlia
I just now cloned this repo. It seems to be using 10.1 https://github.com/confluentinc/examples and running examples in https://github.com/confluentinc/examples/tree/kafka-0.10.0.1-cp-3.0.1/kafka-streams On Thu, Oct 20, 2016 at 3:10 PM, Michael Noll wrote: > I suspect you are running Kafka 0.10

Kafka consumer rate

2016-10-20 Thread Adrienne Kole
Hi, Is there a way to limit the consumer rate from kafka? (say, max 400K p/s from whole topic including n partitions) By default I think it is not limited by any parameter but only by network, performance and etc. Cheers Adrienne

Re: Kafka Streaming

2016-10-20 Thread Michael Noll
I suspect you are running Kafka 0.10.0.x on Windows? If so, this is a known issue that is fixed in Kafka 0.10.1 that was just released today. Also: which examples are you referring to? And, to confirm: which git branch / Kafka version / OS in case my guess above was wrong. On Thursday, October

Kafka Streaming

2016-10-20 Thread Mohit Anchlia
I am trying to run the examples from git. While running the wordcount example I see this error: Caused by: *java.lang.RuntimeException*: librocksdbjni-win64.dll was not found inside JAR. Am I expected to include this jar locally?

rebalancing partitions across a jbod

2016-10-20 Thread Jeremy Hansen
I’ve read several posts on this but it seemed to be old versions of kafka. I wanted to get the latest proper way to do this. If I add a volume to my brokers running kafka 0.10.0 and I’ve adjusted log.dirs with the new volume mount point, what do I have to do to rebalance so kafka takes advan

Re: [ANNOUNCE] Apache Kafka 0.10.1.0 Released

2016-10-20 Thread Jason Gustafson
Had the wrong address for dev and users (haven't sent from this account before). On Thu, Oct 20, 2016 at 11:05 AM, Jason Gustafson wrote: > The Apache Kafka community is pleased to announce the release for Apache > Kafka 0.10.1.0. This is a feature release which includes the completion of > 15 K

Re: Dismissing late messages in Kafka Streams

2016-10-20 Thread Guozhang Wang
Hello Nicolas, Thanks for reporting this, and your observations about the window retention mechanism are correct. I think our documentation do have a few lacks about the windowing operations in that: 1. We should mention about the default retention period of the window as one day (or should the d

Re: Dismissing late messages in Kafka Streams

2016-10-20 Thread Matthias J. Sax
-BEGIN PGP SIGNED MESSAGE- Hash: SHA512 Nicolas, you can filter "old" messages using KStream#transform(). It provides a ProcessorContext object that allows you to access the timestamp of the currently processed record. Thus, you "only" need to maintain the largest timestamp you can ever s

Start 128 producers simultaneously

2016-10-20 Thread Hendrik Dev
We have a Kafka 0.10.0.0 cluster which consists of 5 nodes on aws c4.2xlarge instances. When we connect and send data to that cluster with 128 simultaneous producers we saw that Kafka immediately drops, every time we start that scenario, 6-12 of these connections. Is there a way to circumvent this

Re: Need to add & remove consumers dynamically in diffrent group and consume consecutively.

2016-10-20 Thread Kaushil Rambhia
I know we can have that thing but my concern is i don’t want to read every new consumer joining in to read from starting, i want them to read from where any last message consumed and that too consecutively by all consumers. On Wed, Oct 19, 2016 at 9:34 PM, Sharninder Khera wrote: > Do you have o

Re: Dismissing late messages in Kafka Streams

2016-10-20 Thread Nicolas Fouché
I forgot to mention that the default maintain duration of a window is 1 day. Would it be useful to warn the developer is the current maintain duration is "not compatible" with the current window size and interval ? 2016-10-20 14:49 GMT+02:00 Nicolas Fouché : > Hi Michael, > > thanks for the quic

Re: Dismissing late messages in Kafka Streams

2016-10-20 Thread Nicolas Fouché
Hi Michael, thanks for the quick reply. Let's try to explain things a bit better: Within a call to `aggregateByKey`, I specified a window with the size of 15 days and an interval (hop) of 1 day, without setting a maintain duration. I produced a message, the timestamp being the current clock time,

Re: Dismissing late messages in Kafka Streams

2016-10-20 Thread Michael Noll
Nicolas, > I set the maintain duration of the window to 30 days. > If it consumes a message older than 30 days, then a new aggregate is created for this old window. I assume you mean: If a message should have been included in the original ("old") window but that message happens to arrive late (a

Re: How to block tests of Kafka Streams until messages processed?

2016-10-20 Thread Michael Noll
> Would there be any advantage to using the kafka connect method? The advantage is to decouple the data processing (which you do in your app) from the responsibility of making the processing results available to one or more downstream systems, like Cassandra. For example, what will your applicati

Re: How to block tests of Kafka Streams until messages processed?

2016-10-20 Thread Ali Akhtar
Michael, Would there be any advantage to using the kafka connect method? Seems like it'd just add an extra step of overhead? On Thu, Oct 20, 2016 at 12:35 PM, Michael Noll wrote: > Ali, > > my main feedback is similar to what Eno and Dave have already said. In > your situation, options like th

Dismissing late messages in Kafka Streams

2016-10-20 Thread Nicolas Fouché
Hi, I aggregate some data with `aggregateByKey` and a `TimeWindows`. I set the maintain duration of the window to 30 days. If it consumes a message older than 30 days, then a new aggregate is created for this old window. The problem is that this old windowed aggregate is of course incomplete and

Re: How to block tests of Kafka Streams until messages processed?

2016-10-20 Thread Michael Noll
Ali, my main feedback is similar to what Eno and Dave have already said. In your situation, options like these are what you'd currently need to do since you are writing directly from your Kafka Stream app to Cassandra, rather than writing from your app to Kafka and then using Kafka Connect to ing

Re: kafka streams metadata request fails on 0.10.0.1 broker/topic from 0.10.1.0 client

2016-10-20 Thread Michael Noll
Absolutely, Sai. That's exactly why we want to improve the upgrade/compatibility story. On Thu, Oct 20, 2016 at 12:28 AM, saiprasad mishra < saiprasadmis...@gmail.com> wrote: > Thanks Michael > Hopefully the upgrade story evolves as 0.10.1+ advances to maturity > > Just my 2 cents > > Decouplin