Thanks Jay. I removed partitioner.metadata from KIP. I’ll send an updated patch.
-- Harsha Sent with Airmail On May 5, 2015 at 6:31:47 AM, Sriharsha Chintalapani (harsh...@fastmail.fm) wrote: Thanks for the comments everyone. Hi Jay, I do have a question regarding configurable interface on how to pass a Map<String, ?> properties. I couldn’t find any other classes using it. JMX reporter overrides it but doesn’t implement it. So with configurable partitioner how can a user pass in partitioner configuration since its getting instantiated within the producer. Thanks, Harsha On May 4, 2015 at 10:36:45 AM, Jay Kreps (jay.kr...@gmail.com) wrote: Hey Harsha, That proposal sounds good. One minor thing--I don't think we need to have the partitioner.metadata property. Our reason for using string properties is exactly to make config extensible at runtime. So a given partitioner can add whatever properties make sense using the configure() api it defines. -Jay On Sun, May 3, 2015 at 5:57 PM, Harsha <ka...@harsha.io> wrote: > Thanks Jay & Gianmarco for the comments. I picked the option A, if user > sends a partition id than it will applied and partitioner.class method > will only called if partition id is null . > Please take a look at the updated KIP here > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-+22+-+Expose+a+Partitioner+interface+in+the+new+producer > . Let me know if you see anything missing. > > Thanks, > Harsha > > On Fri, Apr 24, 2015, at 02:15 AM, Gianmarco De Francisci Morales wrote: > > Hi, > > > > > > Here are the questions I think we should consider: > > > 1. Do we need this at all given that we have the partition argument in > > > ProducerRecord which gives full control? I think we do need it because > this > > > is a way to plug in a different partitioning strategy at run time and > do it > > > in a fairly transparent way. > > > > > > > Yes, we need it if we want to support different partitioning strategies > > inside Kafka rather than requiring the user to code them externally. > > > > > > > 3. Do we need to add the value? I suspect people will have uses for > > > computing something off a few fields in the value to choose the > partition. > > > This would be useful in cases where the key was being used for log > > > compaction purposes and did not contain the full information for > computing > > > the partition. > > > > > > > I am not entirely sure about this. I guess that most partitioners should > > not use it. > > I think it makes it easier to reason about the system if the partitioner > > only works on the key. > > Hoever, if the value (and its serialization) are already available, there > > is not much harm in passing them along. > > > > > > > 4. This interface doesn't include either an init() or close() method. > It > > > should implement Closable and Configurable, right? > > > > > > > Right now the only application I can think of to have an init() and > > close() > > is to read some state information (e.g., load information) that is > > published on some external distributed storage (e.g., zookeeper) by the > > brokers. > > It might be useful also for reconfiguration and state migration. > > > > I think it's not a very common use case right now, but if the added > > complexity is not too much it might be worth to have support for these > > methods. > > > > > > > > > 5. What happens if the user both sets the partition id in the > > > ProducerRecord and sets a partitioner? Does the partition id just get > > > passed in to the partitioner (as sort of implied in this interface?). > This > > > is a bit weird since if you pass in the partition id you kind of > expect it > > > to get used, right? Or is it the case that if you specify a partition > the > > > partitioner isn't used at all (in which case no point in including > > > partition in the Partitioner api). > > > > > > > > The user should be able to override the partitioner on a per-record basis > > by explicitly setting the partition id. > > I don't think it makes sense for the partitioners to take "hints" on the > > partition. > > > > I would even go the extra step, and have a default logic that accepts > > both > > key and partition id (current interface) and calls partition() only if > > the > > partition id is not set. The partition() method does *not* take the > > partition ID as input (only key-value). > > > > > > Cheers, > > -- > > Gianmarco > > > > > > > > > Cheers, > > > > > > -Jay > > > > > > On Thu, Apr 23, 2015 at 6:55 AM, Sriharsha Chintalapani < > ka...@harsha.io> > > > wrote: > > > > > > > Hi, > > > > Here is the KIP for adding a partitioner interface for > producer. > > > > > > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-+22+-+Expose+a+Partitioner+interface+in+the+new+producer > > > > There is one open question about how interface should look like. > Please > > > > take a look and let me know if you prefer one way or the other. > > > > > > > > Thanks, > > > > Harsha > > > > > > > > > > > >