How do votes works?

I feel there are 3 options right here, and I’d like a pre vote before a real 
vote? 
1) Adding constructors. Could get messy over time, especially with headers 
coming into play, and future possible improvement to the message format
2) Adding a builder / nicer looking API (like fluent) to help build a 
ProducerRecord in a safe way. Issue here are two ways of building a 
ProducerRecord can bring confusion
3) Same as 2), but deprecating all the constructors. May be too much of an 
aggressive strategy
 

I’m happy to go over 2), update the docs, and tell people this is the 
“preferred” way. Won’t outdate all the literature on Kafka, but I feel this set 
people up for success in the future.
Thoughts  / pre vote? 

On 3/5/17, 4:20 pm, "Ewen Cheslack-Postava" <e...@confluent.io> wrote:

    I understand the convenience of pointing at a JIRA/PR, but can we put the
    concrete changes proposed in the JIRA (under "Proposed Changes"). I don't
    think voting on the KIP would be reasonable otherwise since the changes
    under vote could change arbitrarily...
    
    I'm increasingly skeptical of adding more convenience constructors -- the
    current patch adds timestamps, we're about to add headers as well (for
    core, for Connect we have
    
https://cwiki.apache.org/confluence/display/KAFKA/KIP-145+-+Expose+Record+Headers+in+Kafka+Connect
    in flight). It just continues to get messier over time.
    
    I think builders in the right context are useful, as long as they exceed a
    certain number of parameters (SchemaBuilder in Connect is an artifact of
    that position). I don't think a transition period with 2 ways to construct
    an object is actually a problem -- if there's always an "all N parameters"
    version of the constructor, all other constructors are just convenience
    shortcuts, but the Builder provides a shorthand.
    
    I also agree w/ Ismael that deprecating to aggressively is bad -- we added
    the APIs instead of a builder and there's not any real maintenance cost, so
    why add the deprecation? I don't want to suggest actually adding such an
    annotation, but the real issue here is that one API will become "preferred"
    for some time.
    
    -Ewen
    
    On Tue, May 2, 2017 at 1:15 AM, Ismael Juma <ism...@juma.me.uk> wrote:
    
    > Hi Matthias,
    >
    > Deprecating widely used APIs is a big deal. Build warnings are a nuisance
    > and can potentially break the build for those who have a zero-warnings
    > policy (which is good practice). It creates a bunch of busy work for our
    > users and various resources like books, blog posts, etc. become out of
    > date.
    >
    > This does not mean that we should not do it, but the benefit has to be
    > worth it and we should not do it lightly.
    >
    > Ismael
    >
    > On Sat, Apr 29, 2017 at 6:52 PM, Matthias J. Sax <matth...@confluent.io>
    > wrote:
    >
    > > I understand that we cannot just break stuff (btw: also not for
    > > Streams!). But deprecating does not break anything, so I don't think
    > > it's a big deal to change the API as long as we keep the old API as
    > > deprecated.
    > >
    > >
    > > -Matthias
    > >
    > > On 4/29/17 9:28 AM, Jay Kreps wrote:
    > > > Hey Matthias,
    > > >
    > > > Yeah I agree, I'm not against change as a general thing! I also think
    > if
    > > > you look back on the last two years, we completely rewrote the 
producer
    > > and
    > > > consumer APIs, reworked the binary protocol many times over, and added
    > > the
    > > > connector and stream processing apis, both major new additions. So I
    > > don't
    > > > think we're in too much danger of stagnating!
    > > >
    > > > My two cents was just around breaking compatibility for trivial 
changes
    > > > like constructor => builder. I think this only applies to the 
producer,
    > > > consumer, and connect apis which are heavily embedded in hundreds of
    > > > ecosystem components that depend on them. This is different from 
direct
    > > > usage. If we break the streams api it is really no big deal---apps 
just
    > > > need to rebuild when they upgrade, not the end of the world at all.
    > > However
    > > > because many intermediate things depend on the Kafka producer you can
    > > cause
    > > > these weird situations where your app depends on two third party 
things
    > > > that use Kafka and each requires different, incompatible versions. We
    > did
    > > > this a lot in earlier versions of Kafka and it was the cause of much
    > > angst
    > > > (and an ingrained general reluctance to upgrade) from our users.
    > > >
    > > > I still think we may have to break things, i just don't think we 
should
    > > do
    > > > it for things like builders vs direct constructors which i think are
    > kind
    > > > of a debatable matter of taste.
    > > >
    > > > -Jay
    > > >
    > > >
    > > >
    > > > On Mon, Apr 24, 2017 at 9:40 AM, Matthias J. Sax <
    > matth...@confluent.io>
    > > > wrote:
    > > >
    > > >> Hey Jay,
    > > >>
    > > >> I understand your concern, and for sure, we will need to keep the
    > > >> current constructors deprecated for a long time (ie, many years).
    > > >>
    > > >> But if we don't make the move, we will not be able to improve. And I
    > > >> think warnings about using deprecated APIs is an acceptable price to
    > > >> pay. And the API improvements will help new people who adopt Kafka to
    > > >> get started more easily.
    > > >>
    > > >> Otherwise Kafka might end up as many other enterprise software with a
    > > >> lots of old stuff that is kept forever because nobody has the guts to
    > > >> improve/change it.
    > > >>
    > > >> Of course, we can still improve the docs of the deprecated
    > constructors,
    > > >> too.
    > > >>
    > > >> Just my two cents.
    > > >>
    > > >>
    > > >> -Matthias
    > > >>
    > > >> On 4/23/17 3:37 PM, Jay Kreps wrote:
    > > >>> Hey guys,
    > > >>>
    > > >>> I definitely think that the constructors could have been better
    > > designed,
    > > >>> but I think given that they're in heavy use I don't think this
    > proposal
    > > >>> will improve things. Deprecating constructors just leaves everyone
    > with
    > > >>> lots of warnings and crossed out things. We can't actually delete 
the
    > > >>> methods because lots of code needs to be usable across multiple 
Kafka
    > > >>> versions, right? So we aren't picking between the original approach
    > > >> (worse)
    > > >>> and the new approach (better); what we are proposing is a perpetual
    > > >>> mingling of the original style and the new style with a bunch of
    > > >> deprecated
    > > >>> stuff, which I think is worst of all.
    > > >>>
    > > >>> I'd vote for just documenting the meaning of null in the
    > ProducerRecord
    > > >>> constructor.
    > > >>>
    > > >>> -Jay
    > > >>>
    > > >>> On Wed, Apr 19, 2017 at 3:34 PM, 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