Thanks Mike, I was just about to do the same!

I think the benefit of the builder pattern, or the `with` pattern, is
greatly improved if you take interceptor implementations into account.

On Fri, 19 May 2017 at 08:29 Michael Pearce <michael.pea...@ig.com> wrote:

> And I just failed at a basic copy/paste exercise (bangs head on wall)
>
> Try again:
>
>
> > On 18 May 2017, at 18:29, Andrew Coates <andy.coa...@apple.com> wrote:
> >
> > Hi all,
> >
> > The `ProducerRecord` type has many optional fields and the list has
> grown over different revisions of Kafka. Kafka supports
> `ProducerInterceptor`s, which often need to construct new
> `ProducerRecord`s, based on those passed in, copying most fields from the
> old to the new record, e.g.:
> >
> > ```java
> >   public ProducerRecord<K,V> onSend(ProducerRecord<K,V> record) {
> >       return new ProducerRecord<>(record.topic(), record.partition(),
> getSpecificTimestampIWantToSet(), record.key(), record.value())
> >   }
> > ```
> >
> > If/when a new field is next added to the `ProducerRecord` all existing
> interceptor implementations will fail to copy across the new field,
> assuming a backwards compatible constructors exist that allow the old code
> to compile, (which the tend to do). This makes the code brittle and leaves
> me with a bad taste in my mouth.
> >
> > Additionally, the set of `ProducerRecord` constructors is multiplying as
> new optional fields are being added and not all combinations are supported,
> though they may be valid.
> >
> > I was wondering what peoples thoughts would be to introducing a builder
> pattern on the producer record?  If we did and a pre-initialised builder
> could be obtained from any existing record, then interceptors can just
> set/oeverwrite the fields they care about, without additional fields being
> lost, so the above code becomes:
> >
> > ```java
> >   public ProducerRecord<K,V> onSend(ProducerRecord<K,V> record) {
> >       return record.asBuilder()
> >            .setTimestamp(getSpecificTimestampIWantToSet())
> >      .build();
> >   }
> > ```
> >
> > This has the benefits of less and more clear interceptor code, and the
> code will pass along new fields, added in a newer version, without
> modification. (Though interceptor authors can still make the choice to use
> a constructor instead, dropping new fields - but now they’d have a choice).
> >
> > If people like this idea then I can create a Jira and a PR. (Would a KIP
> be required also?). If people don’t, I’ll move along quietly…
> >
> > Thanks,
> >
> > Andy
> >
> >
>
> On 19/05/2017, 08:27, "Michael Pearce" <michael.pea...@ig.com> wrote:
>
>     Just copying in from the mail list, another use case / +1 for the
> builder pattern, so its kept with the KIP thread.
>
>
>
>     On 18/05/2017, 22:06, "Andrew Coates" <big.andy.coa...@gmail.com>
> wrote:
>
>         Thanks Mike
>         On Thu, 18 May 2017 at 21:33, Michael André Pearce <
>         michael.andre.pea...@me.com> wrote:
>
>         >Hi Andrew,
>         >
>         >There is already a kip discussion exactly around this if you look
> for KIP
>         >141 discuss thread.
>         >
>         >Cheers
>         >Mike
>         >
>         >Sent from my iPhone
>         >
>         >> On 18 May 2017, at 18:29, Andrew Coates <andy.coa...@apple.com>
> wrote:
>         >>
>         >> Hi all,
>         >>
>         >> The `ProducerRecord` type has many optional fields and the list
> has
>         >grown over different revisions of Kafka. Kafka supports
>         >`ProducerInterceptor`s, which often need to construct new
>         >`ProducerRecord`s, based on those passed in, copying most fields
> from the
>         >old to the new record, e.g.:
>         >>
>         >> ```java
>         >>   public ProducerRecord<K,V> onSend(ProducerRecord<K,V> record)
> {
>         >>       return new ProducerRecord<>(record.topic(),
> record.partition(),
>         >getSpecificTimestampIWantToSet(), record.key(), record.value())
>         >>   }
>         >> ```
>         >>
>         >> If/when a new field is next added to the `ProducerRecord` all
> existing
>         >interceptor implementations will fail to copy across the new
> field,
>         >assuming a backwards compatible constructors exist that allow the
> old code
>         >to compile, (which the tend to do). This makes the code
>
>
>     On 08/05/2017, 18:26, "Colin McCabe" <cmcc...@apache.org> wrote:
>
>         Hadoop had a very similar issue in two places: in the constructor
> of
>         MiniDFSCluster, and with the FileSystem#create API.  In both cases,
>         people kept adding more and more function overloads until the APIs
> got
>         very ugly and hard to understand.  This is especially the case
> when some
>         of the parameters were just ints or other basic types.  Do you
> need the
>         Path, FsPermission, boolean, int, short, long, Progressable,
>         InetSocketAddress[] overload or the Path, FsPermission,
>         EnumSet<CreateFlag>, int, short, long, Progressable, ChecksumOpt
> one?
>         Or one of the other literally dozen+ overloads?  Ever since that
>         experience I've been in favor of creating builder objects whenever
> it
>         seems like you will end up in that situation.
>
>         As Jay mentions, Hadoop also had a very difficult time deprecating
> APIs.
>          We tried to deprecate org.apache.hadoop.fs.FileSystem in favor of
>         org.apache.hadoop.fs.FileContext, and it just didn't take (people
> kept
>         using the old API, and still are).  A similar story could be told
> with
>         the mapred vs. mapreduce APIs.
>
>         There were a few issues that made this fail.  The first was that
> there
>         was a big installed base of existing programs, and people often
> used
>         those existing programs as examples.  And they all used the old
> APIs.
>         The second is that all the projects built on top of Hadoop's APIs,
> like
>         HBase, Spark, etc., wanted to use an API that could support
> multiple
>         versions, which meant using the old API.  Finally, the new APIs
> were not
>         better for users.
>
>         Hopefully, someday Java will get named arguments with default
>         parameters, similar to Scala.  Until that point, I think it's
> sensible
>         to just use a builder whenever you are in the scenario "I have
> half a
>         dozen arguments to this public API, and I think I might have half a
>         dozen more in a year or two".  So my vote would be to add the
> builders
>         and keep the existing functions as well.
>
>         best,
>         Colin
>
>
>         On Thu, May 4, 2017, at 21:23, Michael André Pearce wrote:
>         > My vote would be with 2, then 3 then 1.
>         >
>         > Could I suggest maybe an option 4.
>         >
>         > that is option 2 but with a note that there is an intent in 1/2
> years
>         > time to deprecate the old way (under another kip). This way books
>         > materials can be updated over a period, code won't be making
> compile
>         > warnings today. And also we can judge if two ways is really
> causing an
>         > issue or not, but does hint to users to start actively using the
> new way.
>         >
>         > Think when the new Java clients were released they released as
> like a
>         > beta, it was a release or two after that the old clients were
> deprecated,
>         > and still to be removed.
>         >
>         > Sent from my iPhone
>         >
>         > > On 4 May 2017, at 21:18, Matthias J. Sax <
> matth...@confluent.io> wrote:
>         > >
>         > > We can go either way. I just pointed out, what I would prefer
> -- it's
>         > > also quite subjective.
>         > >
>         > > The least invasive change would be to add new constructors and
> update
>         > > the JavaDocs to point out the semantics of `partition`
> parameter.
>         > >
>         > > However, I still like the builder pattern: ProducerRecord has 6
>         > > parameters with only 2 being mandatory (topic and either key
> or value).
>         > > Thus, to have a complete set of overloads we would need many
> more
>         > > constructors. Right now, it feels to be "incomplete" and as if
> the
>         > > offered constructors got picked "randomly".
>         > >
>         > > I got convinced though, that deprecation is not strictly
> required for
>         > > this change. If we go with option (2), it might be good to
> update the
>         > > JavaDocs of the current API to point to the new one as
> "recommended to use".
>         > >
>         > >
>         > >
>         > > -Matthias
>         > >
>         > >
>         > >> On 5/3/17 10:47 PM, Ewen Cheslack-Postava wrote:
>         > >> Stephane,
>         > >>
>         > >> VOTES are really on-demand based on the author, but obviously
> it's good to
>         > >> come to some level of consensus in the DISCUSS thread before
> initiating a
>         > >> vote. I think the request for comments/votes on your 3
> options is a
>         > >> reasonable way to gauge current opinions.
>         > >>
>         > >> For myself, I think either 1 or 3 are good options, and I
> think at least
>         > >> Matthias & Jay are in agreement -- basically have one
> preferred, but
>         > >> possibly support 2 approaches for awhile.
>         > >>
>         > >> I think 3 is the right way to go long term -- I don't expect
> so many more
>         > >> built-in fields to be added, but then again I didn't expect
> this much churn
>         > >> this quickly (headers were a surprise for me). We've gotten
> to enough
>         > >> parameters that a builder is more effective. It sucks a bit
> for existing
>         > >> users that rely on the constructors, but a full major release
> cycle (at the
>         > >> minimum) is a pretty significant window, and we can always
> choose to extend
>         > >> the window longer if we want to give people more time to
> transition. To me,
>         > >> the biggest problem is all the tutorials and content that we
> *can't*
>         > >> control -- there's a ton of code and tutorials out there that
> will still
>         > >> reference the constructors, and those will last far longer
> than any
>         > >> deprecation period we put in place.
>         > >>
>         > >> -Ewen
>         > >>
>         > >> On Wed, May 3, 2017 at 5:46 PM, Stephane Maarek <
>         > >> steph...@simplemachines.com.au> wrote:
>         > >>
>         > >>> 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
>         > >>>>>>>>>
>         > >>>>>>>>
>         > >>>>>>>
>         > >>>>>>>
>         > >>>>>>
>         > >>>>>
>         > >>>>>
>         > >>>>
>         > >>>
>         > >>>
>         > >>>
>         > >>>
>         > >>
>         > >
>
>
>     The information contained in this email is strictly confidential and
> for the use of the addressee only, unless otherwise indicated. If you are
> not the intended recipient, please do not read, copy, use or disclose to
> others this message or any attachment. Please also notify the sender by
> replying to this email or by telephone (+44(020 7896 0011) and then delete
> the email and any copies of it. Opinions, conclusion (etc) that do not
> relate to the official business of this company shall be understood as
> neither given nor endorsed by it. IG is a trading name of IG Markets
> Limited (a company registered in England and Wales, company number
> 04008957) and IG Index Limited (a company registered in England and Wales,
> company number 01190902). Registered address at Cannon Bridge House, 25
> Dowgate Hill, London EC4R 2YA. Both IG Markets Limited (register number
> 195355) and IG Index Limited (register number 114059) are authorised and
> regulated by the Financial Conduct Authority.
>
>
>

Reply via email to