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
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
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 <
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
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
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
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
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
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
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
er 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
>
>> 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 I
eph...@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 wr
7 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’
ucerRecord 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 - ProducerRecordBuilde
.
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
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,
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
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 usin
7 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>
Se
+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] K
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:
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,
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
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
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
017 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.
>
g 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:1
nt: 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, `time
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)
-
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
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
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
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
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
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
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
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
Hi all,
My first KIP, let me know your thoughts!
https://cwiki.apache.org/confluence/display/KAFKA/KIP+141+-+ProducerRecordBuilder+Interface
Cheers,
Stephane
39 matches
Mail list logo