Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-05-22 Thread Andrew Coates
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  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  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 onSend(ProducerRecord 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 onSend(ProducerRecord 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"  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" 
> 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 
> 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 onSend(ProducerRecord 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"  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,
> 

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-05-19 Thread Michael Pearce
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  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 onSend(ProducerRecord 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 onSend(ProducerRecord 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"  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"  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  
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 onSend(ProducerRecord 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"  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, 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 

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-05-19 Thread Michael Pearce
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"  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  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 onSend(ProducerRecord 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"  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, 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  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 

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-05-08 Thread Colin McCabe
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, 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  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 <
> >> 

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-05-04 Thread Michael André Pearce
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  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"  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 

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-05-04 Thread Matthias J. Sax
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"  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  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 

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-05-03 Thread Ewen Cheslack-Postava
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"  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  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 

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-05-03 Thread Stephane Maarek
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"  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  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 
> 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 

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-05-03 Thread Ewen Cheslack-Postava
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  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 
> 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
> 

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-05-02 Thread Ismael Juma
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 
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 
> > 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
> 
> >>>
> >>
> >>
> >
>
>


Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-05-01 Thread Jay Kreps
Hey Matthias,

Yeah, I think we mostly agree. I think if we think the new way is a lot
better, then I agree keeping it deprecated for a long while then removing
is a viable path. To me this case just seemed like a minor thing so the 1+
years of having a deprecated way in common use and a new way and the
resulting confusion just seems unnecessary, especially since this is so
much in the common path of the most basic kafka usage. My experience with
other apis like MapReduce or Google Collections that did this was pretty
negative.

Big time agreement that doing both is not the right thing to do. People
aren't looking for 4 different ways of creating a ProducerRecord, that just
generates confusion.

-Jay

On Mon, May 1, 2017 at 10:34 AM, Matthias J. Sax <matth...@confluent.io>
wrote:

> Hi,
>
> I am personally not a big fan of providing two APIs to do the same
> thing. If we believe that one API is better than the other, we should
> indicate this by deprecating the old API IMHO.
>
> Just my two cents.
>
>
> -Matthias
>
>
> On 4/30/17 11:05 PM, Michael Pearce wrote:
> > See
> >
> > https://hc.apache.org/httpcomponents-client-ga/tutorial/html/fluent.html
> >
> > Doesn't cause much issue over there where you have a fluent api wrapper
> for those who like that style, and the original more verbose api.
> >
> > Surely it would be better than removing a way of doing things that
> everyone has got used to and built their code around ala constructors
> approach. And simply provide a wrapper to provide a per field way of doing
> things.
> >
> >
> >
> >
> >
> > Sent using OWA for iPhone
> > 
> > From: Stephane Maarek <steph...@simplemachines.com.au>
> > Sent: Monday, May 1, 2017 6:37:44 AM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface
> >
> > I’m not sure how people would feel about having two distinct methods to
> build the same object?
> > An API wrapper may be useful, but it doesn’t bring opinion about how one
> should program, that’s just driven by the docs.
> > I’m okay with that, but we need concensus
> >
> >
> > On 1/5/17, 6:08 am, "Michael Pearce" <michael.pea...@ig.com> wrote:
> >
> > Why not, instead of deprecating or removing whats there, as noted,
> its a point of preference, think about something that could wrap the
> existing, but provide an api that for you is cleaner?
> >
> > e.g. here's a sample idea building on a fluent api way. (this wraps
> the producer and producer records so no changes needed)
> >
> > https://gist.github.com/michaelandrepearce/
> de0f5ad4aa7d39d243781741c58c293e
> >
> > In future as new items further add to Producer Record, they just
> become new methods in the fluent API, as it builds the ProducerRecord using
> the most exhaustive constructor.
> >
> >
> >
> > 
> > From: Matthias J. Sax <matth...@confluent.io>
> > Sent: Saturday, April 29, 2017 6:52 PM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface
> >
> > 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 d

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-05-01 Thread Michael André Pearce
If it's a choice of either or. I would vote keep as is. At least then people 
can write their own api wrappers easily with not many lines of code, like the 
one supplied.

Sent from my iPhone

> On 1 May 2017, at 18:34, Matthias J. Sax <matth...@confluent.io> wrote:
> 
> Hi,
> 
> I am personally not a big fan of providing two APIs to do the same
> thing. If we believe that one API is better than the other, we should
> indicate this by deprecating the old API IMHO.
> 
> Just my two cents.
> 
> 
> -Matthias
> 
> 
>> On 4/30/17 11:05 PM, Michael Pearce wrote:
>> See
>> 
>> https://hc.apache.org/httpcomponents-client-ga/tutorial/html/fluent.html
>> 
>> Doesn't cause much issue over there where you have a fluent api wrapper for 
>> those who like that style, and the original more verbose api.
>> 
>> Surely it would be better than removing a way of doing things that everyone 
>> has got used to and built their code around ala constructors approach. And 
>> simply provide a wrapper to provide a per field way of doing things.
>> 
>> 
>> 
>> 
>> 
>> Sent using OWA for iPhone
>> ________
>> From: Stephane Maarek <steph...@simplemachines.com.au>
>> Sent: Monday, May 1, 2017 6:37:44 AM
>> To: dev@kafka.apache.org
>> Subject: Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface
>> 
>> I’m not sure how people would feel about having two distinct methods to 
>> build the same object?
>> An API wrapper may be useful, but it doesn’t bring opinion about how one 
>> should program, that’s just driven by the docs.
>> I’m okay with that, but we need concensus
>> 
>> 
>> On 1/5/17, 6:08 am, "Michael Pearce" <michael.pea...@ig.com> wrote:
>> 
>>Why not, instead of deprecating or removing whats there, as noted, its a 
>> point of preference, think about something that could wrap the existing, but 
>> provide an api that for you is cleaner?
>> 
>>e.g. here's a sample idea building on a fluent api way. (this wraps the 
>> producer and producer records so no changes needed)
>> 
>>
>> https://gist.github.com/michaelandrepearce/de0f5ad4aa7d39d243781741c58c293e
>> 
>>In future as new items further add to Producer Record, they just become 
>> new methods in the fluent API, as it builds the ProducerRecord using the 
>> most exhaustive constructor.
>> 
>> 
>> 
>>
>>From: Matthias J. Sax <matth...@confluent.io>
>>Sent: Saturday, April 29, 2017 6:52 PM
>>To: dev@kafka.apache.org
>>Subject: Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface
>> 
>>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.
>>> 
>&

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-05-01 Thread Matthias J. Sax
Hi,

I am personally not a big fan of providing two APIs to do the same
thing. If we believe that one API is better than the other, we should
indicate this by deprecating the old API IMHO.

Just my two cents.


-Matthias


On 4/30/17 11:05 PM, Michael Pearce wrote:
> See
> 
> https://hc.apache.org/httpcomponents-client-ga/tutorial/html/fluent.html
> 
> Doesn't cause much issue over there where you have a fluent api wrapper for 
> those who like that style, and the original more verbose api.
> 
> Surely it would be better than removing a way of doing things that everyone 
> has got used to and built their code around ala constructors approach. And 
> simply provide a wrapper to provide a per field way of doing things.
> 
> 
> 
> 
> 
> Sent using OWA for iPhone
> 
> From: Stephane Maarek <steph...@simplemachines.com.au>
> Sent: Monday, May 1, 2017 6:37:44 AM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface
> 
> I’m not sure how people would feel about having two distinct methods to build 
> the same object?
> An API wrapper may be useful, but it doesn’t bring opinion about how one 
> should program, that’s just driven by the docs.
> I’m okay with that, but we need concensus
> 
> 
> On 1/5/17, 6:08 am, "Michael Pearce" <michael.pea...@ig.com> wrote:
> 
> Why not, instead of deprecating or removing whats there, as noted, its a 
> point of preference, think about something that could wrap the existing, but 
> provide an api that for you is cleaner?
> 
> e.g. here's a sample idea building on a fluent api way. (this wraps the 
> producer and producer records so no changes needed)
> 
> 
> https://gist.github.com/michaelandrepearce/de0f5ad4aa7d39d243781741c58c293e
> 
> In future as new items further add to Producer Record, they just become 
> new methods in the fluent API, as it builds the ProducerRecord using the most 
> exhaustive constructor.
> 
> 
> 
> ________________
> From: Matthias J. Sax <matth...@confluent.io>
> Sent: Saturday, April 29, 2017 6:52 PM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface
> 
> 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
&g

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-05-01 Thread Michael Pearce
See

https://hc.apache.org/httpcomponents-client-ga/tutorial/html/fluent.html

Doesn't cause much issue over there where you have a fluent api wrapper for 
those who like that style, and the original more verbose api.

Surely it would be better than removing a way of doing things that everyone has 
got used to and built their code around ala constructors approach. And simply 
provide a wrapper to provide a per field way of doing things.





Sent using OWA for iPhone

From: Stephane Maarek <steph...@simplemachines.com.au>
Sent: Monday, May 1, 2017 6:37:44 AM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

I’m not sure how people would feel about having two distinct methods to build 
the same object?
An API wrapper may be useful, but it doesn’t bring opinion about how one should 
program, that’s just driven by the docs.
I’m okay with that, but we need concensus


On 1/5/17, 6:08 am, "Michael Pearce" <michael.pea...@ig.com> wrote:

Why not, instead of deprecating or removing whats there, as noted, its a 
point of preference, think about something that could wrap the existing, but 
provide an api that for you is cleaner?

e.g. here's a sample idea building on a fluent api way. (this wraps the 
producer and producer records so no changes needed)

https://gist.github.com/michaelandrepearce/de0f5ad4aa7d39d243781741c58c293e

In future as new items further add to Producer Record, they just become new 
methods in the fluent API, as it builds the ProducerRecord using the most 
exhaustive constructor.




From: Matthias J. Sax <matth...@confluent.io>
Sent: Saturday, April 29, 2017 6:52 PM
To: dev@kafka.apache.org
    Subject: Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

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,
&

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-30 Thread Stephane Maarek
I’m not sure how people would feel about having two distinct methods to build 
the same object?
An API wrapper may be useful, but it doesn’t bring opinion about how one should 
program, that’s just driven by the docs. 
I’m okay with that, but we need concensus
 

On 1/5/17, 6:08 am, "Michael Pearce" <michael.pea...@ig.com> wrote:

Why not, instead of deprecating or removing whats there, as noted, its a 
point of preference, think about something that could wrap the existing, but 
provide an api that for you is cleaner?

e.g. here's a sample idea building on a fluent api way. (this wraps the 
producer and producer records so no changes needed)

https://gist.github.com/michaelandrepearce/de0f5ad4aa7d39d243781741c58c293e

In future as new items further add to Producer Record, they just become new 
methods in the fluent API, as it builds the ProducerRecord using the most 
exhaustive constructor.




From: Matthias J. Sax <matth...@confluent.io>
Sent: Saturday, April 29, 2017 6:52 PM
To: dev@kafka.apache.org
    Subject: Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

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
>

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-30 Thread Michael Pearce
Why not, instead of deprecating or removing whats there, as noted, its a point 
of preference, think about something that could wrap the existing, but provide 
an api that for you is cleaner?

e.g. here's a sample idea building on a fluent api way. (this wraps the 
producer and producer records so no changes needed)

https://gist.github.com/michaelandrepearce/de0f5ad4aa7d39d243781741c58c293e

In future as new items further add to Producer Record, they just become new 
methods in the fluent API, as it builds the ProducerRecord using the most 
exhaustive constructor.




From: Matthias J. Sax <matth...@confluent.io>
Sent: Saturday, April 29, 2017 6:52 PM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

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 informa

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-29 Thread Matthias J. Sax
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 
> 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

>>>
>>
>>
> 



signature.asc
Description: OpenPGP digital signature


Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-29 Thread Jay Kreps
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 
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
> >>
> >
>
>


Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-28 Thread Michael Pearce
This is ref to this section that's still there

"
The ProducerRecord constructors will all be deprecated, except the main long 
explicit one:
"

Sent using OWA for iPhone

From: Michael Pearce <michael.pea...@ig.com>
Sent: Saturday, April 29, 2017 6:17:04 AM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

Probably should remove the words in the kip the other constructors are going to 
be deprecated if they aren't any more with just the tactical change

Sent using OWA for iPhone

From: Michael Pearce <michael.pea...@ig.com>
Sent: Saturday, April 29, 2017 6:04:00 AM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

+1

N.b. it is a shame we don't work on the contention to work a better way 
forwards.

From: Stephane Maarek <steph...@simplemachines.com.au>
Sent: Saturday, April 29, 2017 4:39:32 AM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

Hi

Thanks for all your input. My initial intent was to just add a “timestamp” 
constructor for the ProducerRecord, and it seems greater changes are too 
contentious.
I’ve just rolled back the PR to add the missing constructors, in the way Ismael 
suggested.

Updated KIP here: 
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=69406838
Added constructors here: https://github.com/apache/kafka/pull/2894

Let me know your thoughts,

Regards,
Stephane

On 25/4/17, 2:58 am, "Michael Pearce" <michael.pea...@ig.com> wrote:

Why not simply make a cleaner client fluent API wrapper? The internals use 
and send via current api, but provide a cleaner more fluent api.

A good example here is HTTP compontents where they did this.

https://hc.apache.org/httpcomponents-client-ga/tutorial/html/fluent.html



On 24/04/2017, 17:40, "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 regis

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-28 Thread Michael Pearce
Probably should remove the words in the kip the other constructors are going to 
be deprecated if they aren't any more with just the tactical change

Sent using OWA for iPhone

From: Michael Pearce <michael.pea...@ig.com>
Sent: Saturday, April 29, 2017 6:04:00 AM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

+1

N.b. it is a shame we don't work on the contention to work a better way 
forwards.

From: Stephane Maarek <steph...@simplemachines.com.au>
Sent: Saturday, April 29, 2017 4:39:32 AM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

Hi

Thanks for all your input. My initial intent was to just add a “timestamp” 
constructor for the ProducerRecord, and it seems greater changes are too 
contentious.
I’ve just rolled back the PR to add the missing constructors, in the way Ismael 
suggested.

Updated KIP here: 
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=69406838
Added constructors here: https://github.com/apache/kafka/pull/2894

Let me know your thoughts,

Regards,
Stephane

On 25/4/17, 2:58 am, "Michael Pearce" <michael.pea...@ig.com> wrote:

Why not simply make a cleaner client fluent API wrapper? The internals use 
and send via current api, but provide a cleaner more fluent api.

A good example here is HTTP compontents where they did this.

https://hc.apache.org/httpcomponents-client-ga/tutorial/html/fluent.html



On 24/04/2017, 17:40, "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.





Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-28 Thread Michael Pearce
+1

N.b. it is a shame we don't work on the contention to work a better way 
forwards.

From: Stephane Maarek <steph...@simplemachines.com.au>
Sent: Saturday, April 29, 2017 4:39:32 AM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

Hi

Thanks for all your input. My initial intent was to just add a “timestamp” 
constructor for the ProducerRecord, and it seems greater changes are too 
contentious.
I’ve just rolled back the PR to add the missing constructors, in the way Ismael 
suggested.

Updated KIP here: 
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=69406838
Added constructors here: https://github.com/apache/kafka/pull/2894

Let me know your thoughts,

Regards,
Stephane

On 25/4/17, 2:58 am, "Michael Pearce" <michael.pea...@ig.com> wrote:

Why not simply make a cleaner client fluent API wrapper? The internals use 
and send via current api, but provide a cleaner more fluent api.

A good example here is HTTP compontents where they did this.

https://hc.apache.org/httpcomponents-client-ga/tutorial/html/fluent.html



On 24/04/2017, 17:40, "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.





Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-28 Thread Stephane Maarek
Hi

Thanks for all your input. My initial intent was to just add a “timestamp” 
constructor for the ProducerRecord, and it seems greater changes are too 
contentious.
I’ve just rolled back the PR to add the missing constructors, in the way Ismael 
suggested.

Updated KIP here: 
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=69406838 
Added constructors here: https://github.com/apache/kafka/pull/2894  
 
Let me know your thoughts,

Regards,
Stephane

On 25/4/17, 2:58 am, "Michael Pearce"  wrote:

Why not simply make a cleaner client fluent API wrapper? The internals use 
and send via current api, but provide a cleaner more fluent api.

A good example here is HTTP compontents where they did this.

https://hc.apache.org/httpcomponents-client-ga/tutorial/html/fluent.html



On 24/04/2017, 17:40, "Matthias J. Sax"  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.





Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-24 Thread Michael Pearce
Why not simply make a cleaner client fluent API wrapper? The internals use and 
send via current api, but provide a cleaner more fluent api.

A good example here is HTTP compontents where they did this.

https://hc.apache.org/httpcomponents-client-ga/tutorial/html/fluent.html



On 24/04/2017, 17:40, "Matthias J. Sax"  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.


Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-24 Thread Matthias J. Sax
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
>>
> 



signature.asc
Description: OpenPGP digital signature


Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-24 Thread Jay Kreps
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
>


Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-23 Thread Michal Borowiecki
However, if you're used to working with immutables, it can feel natural 
that withPartition would return a new object, so it could be more prone 
to mistakes.


Cheers,
Michal

On 23/04/17 10:41, Michal Borowiecki wrote:


IMHO, the ProducerRecord is anyway not immutable until send, since key 
and value don't have to immutable, until serialized on send.


So if immutability is a concern, I think it would have to be enforced 
in send as Mike suggested, don't see much point in enforcing 
immutability prior to send.


Therefore the with pattern should not be discarded just for that reason.

Thanks,

Michal


On 23/04/17 06:34, Stephane Maarek wrote:

Good call.
That’s when I heavily miss Scala Case classes and options. You get clarity on 
optional vs mandatory fields, one constructor, and immutability. If losing 
Immutability is an issue, then the with pattern is a no-go and then I’ll just 
add a missing constructor the way Ismael described it. That’ll make the PR way 
simpler, with limited refactoring.

Regarding the ConsumerRecord, I’m happy to have a look, but it’s the first time 
I see it concretely. When would you manually construct such a record? Isn’t the 
client handling all that for you behind the scene?
  


On 23/4/17, 3:21 pm, "Michael Pearce"<michael.pea...@ig.com>  wrote:

 If moving to a wither pattern instead of a builder. How will this enforce 
immutability? Eg current PR it is now changing to allow possible change values 
once set.
 
 Or are you proposing to change it to a mutable record? And move to a closable record similar to the closing of the headers on send.
 
 How about also the consumer record, is this also being looked at so we don't have two very different styles.
 
 Cheers

 Mike
 
 
 
 Sent using OWA for iPhone

 
 From:isma...@gmail.com  <isma...@gmail.com>  on behalf of Ismael 
Juma<ism...@juma.me.uk>
 Sent: Saturday, April 22, 2017 11:53:45 PM
 To:dev@kafka.apache.org
     Subject: Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface
 
 On Sat, Apr 22, 2017 at 6:16 PM, Matthias J. Sax<matth...@confluent.io>

 wrote:
 
 > I think Ismael's suggestion is a valid alternative.

 >
 > However, `timestamp` is an optional field and thus we should have at
 > least two constructors for this:
 >
 >  - ProducerRecord(String topic, K key, V value)
 
  - ProducerRecord(String topic, K key, V value, Long timestamp)

 >
 
 Yes, the other one already exists.
 
 Ismael

 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.
 





--
Signature
<http://www.openbet.com/> Michal Borowiecki
Senior Software Engineer L4
T:  +44 208 742 1600


+44 203 249 8448



E:  michal.borowie...@openbet.com
W:  www.openbet.com <http://www.openbet.com/>


OpenBet Ltd

Chiswick Park Building 9

566 Chiswick High Rd

London

W4 5XT

UK


<https://www.openbet.com/email_promo>

This message is confidential and intended only for the addressee. If 
you have received this message in error, please immediately notify the 
postmas...@openbet.com <mailto:postmas...@openbet.com> and delete it 
from your system as well as any copies. The content of e-mails as well 
as traffic data may be monitored by OpenBet for employment and 
security purposes. To protect the environment please do not print this 
e-mail unless necessary. OpenBet Ltd. Registered Office: Chiswick Park 
Building 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A 
company registered in England and Wales. Registered no. 3134634. VAT 
no. GB927523612




--
Signature
<http://www.openbet.com/> Michal Borowiecki
Senior Software Engineer L4
T:  +44 208 742 1600


+44 203 249 8448


   

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-23 Thread Michal Borowiecki
IMHO, the ProducerRecord is anyway not immutable until send, since key 
and value don't have to immutable, until serialized on send.


So if immutability is a concern, I think it would have to be enforced in 
send as Mike suggested, don't see much point in enforcing immutability 
prior to send.


Therefore the with pattern should not be discarded just for that reason.

Thanks,

Michal


On 23/04/17 06:34, Stephane Maarek wrote:

Good call.
That’s when I heavily miss Scala Case classes and options. You get clarity on 
optional vs mandatory fields, one constructor, and immutability. If losing 
Immutability is an issue, then the with pattern is a no-go and then I’ll just 
add a missing constructor the way Ismael described it. That’ll make the PR way 
simpler, with limited refactoring.

Regarding the ConsumerRecord, I’m happy to have a look, but it’s the first time 
I see it concretely. When would you manually construct such a record? Isn’t the 
client handling all that for you behind the scene?
  


On 23/4/17, 3:21 pm, "Michael Pearce" <michael.pea...@ig.com> wrote:

 If moving to a wither pattern instead of a builder. How will this enforce 
immutability? Eg current PR it is now changing to allow possible change values 
once set.
 
 Or are you proposing to change it to a mutable record? And move to a closable record similar to the closing of the headers on send.
 
 How about also the consumer record, is this also being looked at so we don't have two very different styles.
 
 Cheers

 Mike
 
 
 
 Sent using OWA for iPhone

 
 From: isma...@gmail.com <isma...@gmail.com> on behalf of Ismael Juma 
<ism...@juma.me.uk>
 Sent: Saturday, April 22, 2017 11:53:45 PM
 To: dev@kafka.apache.org
     Subject: Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface
 
 On Sat, Apr 22, 2017 at 6:16 PM, Matthias J. Sax <matth...@confluent.io>

 wrote:
 
 > I think Ismael's suggestion is a valid alternative.

 >
 > However, `timestamp` is an optional field and thus we should have at
 > least two constructors for this:
 >
 >  - ProducerRecord(String topic, K key, V value)
 
  - ProducerRecord(String topic, K key, V value, Long timestamp)

 >
 
 Yes, the other one already exists.
 
 Ismael

 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.
 





--
Signature
<http://www.openbet.com/> Michal Borowiecki
Senior Software Engineer L4
T:  +44 208 742 1600


+44 203 249 8448



E:  michal.borowie...@openbet.com
W:  www.openbet.com <http://www.openbet.com/>


OpenBet Ltd

Chiswick Park Building 9

566 Chiswick High Rd

London

W4 5XT

UK


<https://www.openbet.com/email_promo>

This message is confidential and intended only for the addressee. If you 
have received this message in error, please immediately notify the 
postmas...@openbet.com <mailto:postmas...@openbet.com> and delete it 
from your system as well as any copies. The content of e-mails as well 
as traffic data may be monitored by OpenBet for employment and security 
purposes. To protect the environment please do not print this e-mail 
unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building 
9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company 
registered in England and Wales. Registered no. 3134634. VAT no. 
GB927523612




Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-22 Thread Stephane Maarek
Good call.
That’s when I heavily miss Scala Case classes and options. You get clarity on 
optional vs mandatory fields, one constructor, and immutability. If losing 
Immutability is an issue, then the with pattern is a no-go and then I’ll just 
add a missing constructor the way Ismael described it. That’ll make the PR way 
simpler, with limited refactoring.

Regarding the ConsumerRecord, I’m happy to have a look, but it’s the first time 
I see it concretely. When would you manually construct such a record? Isn’t the 
client handling all that for you behind the scene?
 

On 23/4/17, 3:21 pm, "Michael Pearce" <michael.pea...@ig.com> wrote:

If moving to a wither pattern instead of a builder. How will this enforce 
immutability? Eg current PR it is now changing to allow possible change values 
once set.

Or are you proposing to change it to a mutable record? And move to a 
closable record similar to the closing of the headers on send.

How about also the consumer record, is this also being looked at so we 
don't have two very different styles.

Cheers
Mike



Sent using OWA for iPhone

From: isma...@gmail.com <isma...@gmail.com> on behalf of Ismael Juma 
<ism...@juma.me.uk>
Sent: Saturday, April 22, 2017 11:53:45 PM
To: dev@kafka.apache.org
    Subject: Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

On Sat, Apr 22, 2017 at 6:16 PM, Matthias J. Sax <matth...@confluent.io>
wrote:

> I think Ismael's suggestion is a valid alternative.
>
> However, `timestamp` is an optional field and thus we should have at
> least two constructors for this:
>
>  - ProducerRecord(String topic, K key, V value)

 - ProducerRecord(String topic, K key, V value, Long timestamp)
>

Yes, the other one already exists.

Ismael
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.





Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-22 Thread Michael Pearce
If moving to a wither pattern instead of a builder. How will this enforce 
immutability? Eg current PR it is now changing to allow possible change values 
once set.

Or are you proposing to change it to a mutable record? And move to a closable 
record similar to the closing of the headers on send.

How about also the consumer record, is this also being looked at so we don't 
have two very different styles.

Cheers
Mike



Sent using OWA for iPhone

From: isma...@gmail.com <isma...@gmail.com> on behalf of Ismael Juma 
<ism...@juma.me.uk>
Sent: Saturday, April 22, 2017 11:53:45 PM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

On Sat, Apr 22, 2017 at 6:16 PM, Matthias J. Sax <matth...@confluent.io>
wrote:

> I think Ismael's suggestion is a valid alternative.
>
> However, `timestamp` is an optional field and thus we should have at
> least two constructors for this:
>
>  - ProducerRecord(String topic, K key, V value)

 - ProducerRecord(String topic, K key, V value, Long timestamp)
>

Yes, the other one already exists.

Ismael
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.


Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-22 Thread Ismael Juma
On Sat, Apr 22, 2017 at 6:16 PM, Matthias J. Sax 
wrote:

> I think Ismael's suggestion is a valid alternative.
>
> However, `timestamp` is an optional field and thus we should have at
> least two constructors for this:
>
>  - ProducerRecord(String topic, K key, V value)

 - ProducerRecord(String topic, K key, V value, Long timestamp)
>

Yes, the other one already exists.

Ismael


Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-22 Thread Matthias J. Sax
I think Ismael's suggestion is a valid alternative.

However, `timestamp` is an optional field and thus we should have at
least two constructors for this:

 - ProducerRecord(String topic, K key, V value)
 - ProducerRecord(String topic, K key, V value, Long timestamp)


> If anything, I think the withForcedPartition method could be just removed, 
> and if users need to force partition, shouldn’t they just mandatorily use a 
> custom Partitioner?  It would achieve the same purpose, but in a much “safer” 
> way?
> It just sounds like there would be two ways to achieve the same purpose here. 
> Nonetheless I understand it could be a breaking change

There is an important difference here: If a Partitioner is used, you can
change the strategy via config, while if you specify the partition by
constructor it's hard coded. Thus, while the first gives you more
flexibility, the second allows you to guard against miss configuration.
Furthermore, the decision to which partition you want to send might
depend not just on the key and value (you might even want to send the
same record to multiple partitions). This would not be possible using
the Partitioner, that should only use the give parameters to compute the
partition.

It might sound like a subtle difference, but IMHO you cannot say one is
safer (or better) than the other. Both are valid alternatives and it
depends on the use case. But I agree, that a custom partitioner covers
the lion's share of use cases :)


-Matthias


On 4/21/17 5:50 PM, Stephane Maarek wrote:
> 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"  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"  wrote:
> >
> > Thanks for the KIP!
> >
> > While I agree, that the current 

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-21 Thread Stephane Maarek
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"  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"  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>
>  

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-21 Thread Ismael Juma
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"  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
> >>
> >
>
>
>
>
>


Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-21 Thread Stephane Maarek
Agreed! I’ll update the KIP shortly
I’m self-taught, so I guess I still have a lot to learn (

If anything, I think the withForcedPartition method could be just removed, and 
if users need to force partition, shouldn’t they just mandatorily use a custom 
Partitioner?  It would achieve the same purpose, but in a much “safer” way?
It just sounds like there would be two ways to achieve the same purpose here. 
Nonetheless I understand it could be a breaking change

- Stephane
 
 

On 22/4/17, 4:05 am, "Matthias J. Sax"  wrote:

Ismael's comment is quite reasonable.

And I actually like the idea about `withForcedPartition` -- it helps
newcomers to understand the API better. Even if I don't agree with some
of you reasoning:

> I was always told that the key is solely the determining factor

Bad teacher?

> I find it incredibly unintuitive and dangerous to provide the users the 
ability to force a partition

Disagree completely. This is a very useful feature for advanced use
cases. Btw: users can also specify a custom partitioner -- and this
custom partitioner could also do any computation to determine the
partitions (it also has access to key and value -- thus, it could also
use the value to compute the partition).

But I like `withForcedPartition` because it indicates that the
partitioner (default or custom) is not going to be used for this case.


Btw: if we plan to make `public ProducerRecord(String topic, Integer
partition, Long timestamp, K key, V value)` protected as some point, we
should deprecate it, too.


I also like the compromise you suggest

>> public ProducerRecordBuilder(String topic, K key, V value)
>> 
>> coming with withPartition and withTimestamp?  


The only thing, I still don't like is that we use a builder and are thus
forced to call .build() -- boilerplate.

Maybe we would just change ProducerRecord itself? Like:

new ProducerRecord(topic, key,
value).withTimestamp(ts).withForcedPartition(p);

WDYT?


-Matthias

On 4/20/17 4:57 PM, Stephane Maarek 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"  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,
> > 

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-21 Thread Matthias J. Sax
Ismael's comment is quite reasonable.

And I actually like the idea about `withForcedPartition` -- it helps
newcomers to understand the API better. Even if I don't agree with some
of you reasoning:

> I was always told that the key is solely the determining factor

Bad teacher?

> I find it incredibly unintuitive and dangerous to provide the users the 
> ability to force a partition

Disagree completely. This is a very useful feature for advanced use
cases. Btw: users can also specify a custom partitioner -- and this
custom partitioner could also do any computation to determine the
partitions (it also has access to key and value -- thus, it could also
use the value to compute the partition).

But I like `withForcedPartition` because it indicates that the
partitioner (default or custom) is not going to be used for this case.


Btw: if we plan to make `public ProducerRecord(String topic, Integer
partition, Long timestamp, K key, V value)` protected as some point, we
should deprecate it, too.


I also like the compromise you suggest

>> public ProducerRecordBuilder(String topic, K key, V value)
>> 
>> coming with withPartition and withTimestamp?  


The only thing, I still don't like is that we use a builder and are thus
forced to call .build() -- boilerplate.

Maybe we would just change ProducerRecord itself? Like:

new ProducerRecord(topic, key,
value).withTimestamp(ts).withForcedPartition(p);

WDYT?


-Matthias

On 4/20/17 4:57 PM, Stephane Maarek 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"  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 
> 
> > wrote:
> > 
> >> Hi all,
> >>
> >> My first KIP, let me know your thoughts!
> >>
> >> 
> https://cwiki.apache.org/confluence/display/KAFKA/KIP+141+-+ProducerRecordBuilder+Interface
> >>
> >>
> >> Cheers,
> >> Stephane
> >>
> > 
> 
> 
> 
> 



signature.asc
Description: OpenPGP digital signature


Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-20 Thread Stephane Maarek
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"  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 

> wrote:
> 
>> Hi all,
>>
>> My first KIP, let me know your thoughts!
>>
>> 
https://cwiki.apache.org/confluence/display/KAFKA/KIP+141+-+ProducerRecordBuilder+Interface
>>
>>
>> Cheers,
>> Stephane
>>
> 






Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-20 Thread Matthias J. Sax
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 
> wrote:
> 
>> Hi all,
>>
>> My first KIP, let me know your thoughts!
>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP+141+-+ProducerRecordBuilder+Interface
>>
>>
>> Cheers,
>> Stephane
>>
> 



signature.asc
Description: OpenPGP digital signature


Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-20 Thread Damian Guy
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 
wrote:

> Hi all,
>
> My first KIP, let me know your thoughts!
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP+141+-+ProducerRecordBuilder+Interface
>
>
> Cheers,
> Stephane
>


[DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-04-19 Thread Stephane Maarek
Hi all,

My first KIP, let me know your thoughts!
https://cwiki.apache.org/confluence/display/KAFKA/KIP+141+-+ProducerRecordBuilder+Interface


Cheers,
Stephane