Re: Using Custom Partitioner in Streams

2017-12-19 Thread Sameer Kumar
Tue, Dec 19, 2017 at 1:17 AM, Matthias J. Sax <matth...@confluent.io > > > >> wrote: > >> > >>>> need to map the keys, modify them > >>>>> and then do a join. > >>> > >>> This will always trigger a rebalance. There i

Re: Using Custom Partitioner in Streams

2017-12-19 Thread Matthias J. Sax
. >> >> On Tue, Dec 19, 2017 at 1:17 AM, Matthias J. Sax <matth...@confluent.io> >> wrote: >> >>>> need to map the keys, modify them >>>>> and then do a join. >>> >>> This will always trigger a rebalance. There is no A

Re: Using Custom Partitioner in Streams

2017-12-19 Thread Sameer Kumar
ue, Dec 19, 2017 at 1:17 AM, Matthias J. Sax <matth...@confluent.io> > wrote: > >> > need to map the keys, modify them >> >> and then do a join. >> >> This will always trigger a rebalance. There is no API atm to tell KS >> that partitioning is prese

Re: Using Custom Partitioner in Streams

2017-12-18 Thread Sameer Kumar
need to map the keys, modify them > >> and then do a join. > > This will always trigger a rebalance. There is no API atm to tell KS > that partitioning is preserved. > > Custom partitioner won't help for your case as far as I understand it. > > > -Matthias > > On 12

Re: Using Custom Partitioner in Streams

2017-12-18 Thread Matthias J. Sax
> need to map the keys, modify them >> and then do a join. This will always trigger a rebalance. There is no API atm to tell KS that partitioning is preserved. Custom partitioner won't help for your case as far as I understand it. -Matthias On 12/17/17 9:48 PM, Sameer Kumar wrote: &

Re: Using Custom Partitioner in Streams

2017-12-17 Thread Sameer Kumar
> read the key but only modify the value (-> "data is still local") a > custom partitioner won't help. Also, we are improving this in upcoming > version 1.1 and allows read access to a key in mapValue() (cf. KIP-149 > for details). > > Hope this helps. > > >

Re: Using Custom Partitioner in Streams

2017-12-17 Thread Matthias J. Sax
ead of map()? If you use map() to only read the key but only modify the value (-> "data is still local") a custom partitioner won't help. Also, we are improving this in upcoming version 1.1 and allows read access to a key in mapValue() (cf. KIP-149 for details). Hope this helps. -Matthi

Re: Using Custom Partitioner in Streams

2017-12-17 Thread Sameer Kumar
<matth...@confluent.io> wrote: > It's not recommended to write a custom partitioner because it's pretty > difficult to write a correct one. There are many dependencies and you > need deep knowledge of Kafka Streams internals to get it write. > Otherwise, your custom partitioner br

Re: Using Custom Partitioner in Streams

2017-12-15 Thread Matthias J. Sax
It's not recommended to write a custom partitioner because it's pretty difficult to write a correct one. There are many dependencies and you need deep knowledge of Kafka Streams internals to get it write. Otherwise, your custom partitioner breaks Kafka Streams. That is the reason why it's

Using Custom Partitioner in Streams

2017-12-15 Thread Sameer Kumar
Hi, I want to use the custom partitioner in streams, I couldnt find the same in the documentation. I want to make sure that during map phase, the keys produced adhere to the customized partitioner. -Sameer.

Re: Custom Partitioner explanation?

2016-11-26 Thread Ewen Cheslack-Postava
that the way to implement a custom Partitioner has changed > In 0.8, I had implemented this interface:kafka.producer.Partitioner > with this implementation of the partition() method - where the goal is to > equally distribute events by their custom IDs over all available > partitions:@O

Custom Partitioner explanation?

2016-11-22 Thread Marina
Hi,I'm trying to upgrade my 0.8 producer to 0.9(0.10) APIs, and noticed that the way to implement a custom Partitioner has changed In 0.8, I had implemented this interface:kafka.producer.Partitioner with this implementation of the partition() method - where the goal is to equally distribute

Regarding Custom Partitioner

2015-10-19 Thread Gaurav Agarwal
Hello I created Custom partitioner for my need implemented Partitioner interface Override this method public int partition(Object key, int a_numPartitions){ return partitionId; } We have something called as We are using key as correlationId, That will be unique for each message

Implementing a custom partitioner

2015-07-21 Thread JIEFU GONG
Hi all, If I wanted to write my own partitioner, all I would need to do is write a class that extends Partitioner and override the partition function, correct? I am currently doing so, at least, with a class in the package 'services', yet when I use: properties.put(partitioner.class,

Re: Implementing a custom partitioner

2015-07-21 Thread Sriharsha Chintalapani
Hi,      Are you using the latest trunk for Producer API?.  Did you implement the interface here  https://cwiki.apache.org/confluence/display/KAFKA/KIP-+22+-+Expose+a+Partitioner+interface+in+the+new+producer --  Harsha On July 21, 2015 at 2:27:05 PM, JIEFU GONG (jg...@berkeley.edu) wrote: Hi

Re: Implementing a custom partitioner

2015-07-21 Thread JIEFU GONG
Sriharsha, thanks for your response. I'm using version 0.8.2, and I am implementing kafka.producer.Partitioner. I noticed that in the latest trunk the line I specified above is replaced with: this.partitioner = config.getConfiguredInstance(ProducerConfig. PARTITIONER_CLASS_CONFIG,

Re: Implementing a custom partitioner

2015-07-21 Thread Sriharsha Chintalapani
If you are using the new producer api from kafka 0.8.2 there is no pluggable partitioner in it for this you need to use the latest trunk. But in 0.8.2 if you are using old producer code you can implement a pluggable partitioner 

Re: How to prevent custom Partitioner from increasing the number of producer's requests?

2015-06-11 Thread Sebastien Falquier
Thanks Jiangjie. The new producer is exactly what I was looking for and it works perfectly in production. It should be more documented on the official site. Jason : you're right, I missed a point with my AtomicIntegers. Regards, Sebastien 2015-06-04 20:02 GMT+02:00 Jason Rosenberg

Re: How to prevent custom Partitioner from increasing the number of producer's requests?

2015-06-04 Thread Jiangjie Qin
From the code you pasted, that is old producer. The new producer class is org.apache.kafka.clients.producer.KafkaProducer. The new producer does not have sticky partition behavior. The default partitioner use round-robin like way to send non-keyed messages to partitions. Jiangjie (Becket) Qin

Re: How to prevent custom Partitioner from increasing the number of producer's requests?

2015-06-04 Thread Jason Rosenberg
Sebastien, I think you may have an off by 1 error (e.g. batch should be 0-199, not 1-200). Thus you are sending 2 batches each time (one for 0, another for 1-199). Jason On Thu, Jun 4, 2015 at 1:33 PM, Jiangjie Qin j...@linkedin.com.invalid wrote: From the code you pasted, that is old

Re: How to prevent custom Partitioner from increasing the number of producer's requests?

2015-06-04 Thread Sebastien Falquier
I am using this code (from org.apache.kafka % kafka_2.10 % 0.8.2.0), no idea if it is the old producer or the new one import kafka.producer.Produced import kafka.producer.ProducerConfig val prodConfig : ProducerConfig = new ProducerConfig(properties) val producer : Producer[Integer,String] =

Re: How to prevent custom Partitioner from increasing the number of producer's requests?

2015-06-03 Thread Sebastien Falquier
Hi Jason, The default partitioner does not make the job since my producers haven't a smooth traffic. What I mean is that they can deliver lots of messages during 10 minutes and less during the next 10 minutes, that is too say the first partition will have stacked most of the messages of the last

Re: How to prevent custom Partitioner from increasing the number of producer's requests?

2015-06-03 Thread Jiangjie Qin
Are you using new producer or old producer? The old producer has 10 min sticky partition behavior while the new producer does not. Thanks, Jiangjie (Becket) Qin On 6/2/15, 11:58 PM, Sebastien Falquier sebastien.falqu...@teads.tv wrote: Hi Jason, The default partitioner does not make the job

How to prevent custom Partitioner from increasing the number of producer's requests?

2015-06-02 Thread Sebastien Falquier
Hi guys, I am new to Kafka and I am facing a problem I am not able to sort out. To smooth traffic over all my brokers' partitions, I have coded a custom Paritioner for my producers, using a simple round robin algorithm that jumps from a partition to another on every batch of messages

Re: How to prevent custom Partitioner from increasing the number of producer's requests?

2015-06-02 Thread Jason Rosenberg
Hi Sebastien, You might just try using the default partitioner (which is random). It works by choosing a random partition each time it re-polls the meta-data for the topic. By default, this happens every 10 minutes for each topic you produce to (so it evenly distributes load at a granularity of

Custom partitioner in kafka-0.8.2.0

2015-02-19 Thread sunil kalva
Hi I could not find a way to customize Partitioner class in new KafaProducer class, is it intentional ? tx SunilKalva

Re: Custom partitioner in kafka-0.8.2.0

2015-02-19 Thread Manikumar Reddy
Hi, In new producer, we can specify the partition number as part of ProducerRecord. From javadocs : *If a valid partition number is specified that partition will be used when sending the record. If no partition is specified but a key is present a partition will be chosen using a hash of the key.

Re: Custom partitioner in kafka-0.8.2.0

2015-02-19 Thread sunil kalva
Hi I could not find a way to customize Partitioner class in new KafaProducer class, is it intentional ? tx SunilKalva

Re: Custom partitioner in kafka-0.8.2.0

2015-02-19 Thread sunil kalva
thanks mani for quick response, sorry some how i missed this javadoc :) t SunilKalva On Thu, Feb 19, 2015 at 6:14 PM, Manikumar Reddy ku...@nmsworks.co.in wrote: Hi, In new producer, we can specify the partition number as part of ProducerRecord. From javadocs : *If a valid partition

Re: Custom partitioner

2013-06-01 Thread Jun Rao
: Are you using the 0.8 branch code? That error is for the 0.7 release. Thanks, Jun On Fri, May 31, 2013 at 1:42 PM, Timothy Chen tnac...@gmail.com wrote: Hi, I'm trying to add my own custom partitioner and saw the example in the 0.8 producer example in the wiki. However

Custom partitioner

2013-05-31 Thread Timothy Chen
Hi, I'm trying to add my own custom partitioner and saw the example in the 0.8 producer example in the wiki. However, when I set a broker list and set the custom partitioner class name I did in the client, I see this error: Partitioner cannot be used when broker list is set Does this means

Re: Custom partitioner

2013-05-31 Thread Tim Chen (343)
, I'm trying to add my own custom partitioner and saw the example in the 0.8 producer example in the wiki. However, when I set a broker list and set the custom partitioner class name I did in the client, I see this error: Partitioner cannot be used when broker list is set Does