Hi Ismael,

Good points
I think I was headed in that direction: 
https://github.com/apache/kafka/pull/2894 
1. That’s a possibility. I’m just unsure about how the message format will 
evolve in future versions, because adding constructors is painful if more 
parameters come into play. The approach above (KIP / PR) can easily allow for 
message format extensions
2. /3 Not a bad idea, maybe I’ll explore that as well. Just looking for 
feedback on the KIP / PR first as it’s been updated 15 minutes ago


On 22/4/17, 10:38 am, "Ismael Juma" <isma...@gmail.com on behalf of 
ism...@juma.me.uk> wrote:

    Thanks for the KIP. A possible alternative:
    
    1. Add constructor ProducerRecord(String topic, K key, V value, Long
    timestamp). This provides an unambiguous constructor that allows one to
    pass a timestamp without a partition, which is the main requirement of the
    KIP.
    
    We could also consider:
    
    2. Add a couple of `createWithPartition` static factory methods to replace
    the existing constructors that take a partition. The idea being that
    passing a partition is different enough that it should be called out
    specifically.
    
    3. Deprecate the existing constructors that take a partition so that we can
    remove them (or make one of them private/protected) in a future release
    
    Because ProducerRecord is used so widely, we should make sure that there is
    real value in doing 2 and 3. Otherwise, we should stick to 1.
    
    Ismael
    
    On Fri, Apr 21, 2017 at 12:57 AM, Stephane Maarek <
    steph...@simplemachines.com.au> wrote:
    
    > Matthias: I was definitely on board with you at first, but Ismael made the
    > comment that for:
    >
    > public ProducerRecord(String topic, K key, V value, Integer partition)
    > public ProducerRecord(String topic, K key, V value, Long timestamp)
    >
    > Integer and Long are way too close in terms of meaning, and could provide
    > a strong misuse of the timestamp / partition field.
    > Therefore I started with a builder pattern for explicit argument
    > declaration. Seems like a lot of boilerplate, but it makes things quite
    > easy to understand.
    >
    > I like your point about the necessity of the key, and that users should
    > set it to null explicitely.
    >
    > Damian: I like your idea of public ProducerRecordBuilder(String topic, V
    > value)
    > Finally, I also chose the withForcedPartition because in my learning of
    > Kafka, I was always told that the key is solely the determining factor to
    > know how a messages makes it to a partition. I find it incredibly
    > unintuitive and dangerous to provide the users the ability to force a
    > partition. If anything they should be providing their own key -> partition
    > mapping, but I’m really against letting users force a partition within the
    > producerRecord. What do you think?
    >
    >
    > What do you both think of the more opiniated:
    >
    > public ProducerRecordBuilder(String topic, K key, V value)
    >
    > coming with withPartition and withTimestamp?
    >
    >
    >
    > On 21/4/17, 2:24 am, "Matthias J. Sax" <matth...@confluent.io> wrote:
    >
    >     Thanks for the KIP!
    >
    >     While I agree, that the current API is not perfect, I am not sure if a
    >     builder pattern does make sense here, because it's not too many
    > parameters.
    >
    >     IMHO, builder pattern makes sense if there are many optional
    > parameters.
    >     For a ProducerRecord, I think there are only 2 optional parameters:
    >     partition and timestamp.
    >
    >     I don't think key should be optional, because uses should be "forced"
    > to
    >     think about the key argument as it effects the partitioning. Thus,
    >     providing an explicit `null` if there is no key seems reasonable to 
me.
    >
    >     Overall I think that providing 3 overloads would be sufficient:
    >
    >     > public ProducerRecord(String topic, K key, V value, Integer
    > partition)
    >     > public ProducerRecord(String topic, K key, V value, Long timestamp)
    >     > public ProducerRecord(String topic, K key, V value, Integer
    > partition, Long timestamp)
    >
    >
    >     Just my 2 cents.
    >
    >     -Matthias
    >
    >
    >     On 4/20/17 4:20 AM, Damian Guy wrote:
    >     > Hi Stephane,
    >     >
    >     > Thanks for the KIP.  Overall it looks ok, though i think the builder
    > should
    >     > enforce the required parameters by supplying them via the
    > constructor, i.e,
    >     >
    >     > public ProducerRecordBuilder(String topic, V value)
    >     >
    >     > You can then remove the withValue and withTopic methods
    >     >
    >     > I also think withForcedPartition should just be withPartition
    >     >
    >     > Thanks,
    >     > Damian
    >     >
    >     > On Wed, 19 Apr 2017 at 23:34 Stephane Maarek <
    > steph...@simplemachines.com.au>
    >     > wrote:
    >     >
    >     >> Hi all,
    >     >>
    >     >> My first KIP, let me know your thoughts!
    >     >>
    >     >> https://cwiki.apache.org/confluence/display/KAFKA/KIP+
    > 141+-+ProducerRecordBuilder+Interface
    >     >>
    >     >>
    >     >> Cheers,
    >     >> Stephane
    >     >>
    >     >
    >
    >
    >
    >
    >
    


Reply via email to