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.
>>> 
>>> -Jay
>>> 
>>> 
>>> 
>>> On Mon, Apr 24, 2017 at 9:40 AM, Matthias J. Sax <matth...@confluent.io>
>>> wrote:
>>> 
>>>> Hey Jay,
>>>> 
>>>> I understand your concern, and for sure, we will need to keep the
>>>> current constructors deprecated for a long time (ie, many years).
>>>> 
>>>> But if we don't make the move, we will not be able to improve. And I
>>>> think warnings about using deprecated APIs is an acceptable price to
>>>> pay. And the API improvements will help new people who adopt Kafka to
>>>> get started more easily.
>>>> 
>>>> Otherwise Kafka might end up as many other enterprise software with a
>>>> lots of old stuff that is kept forever because nobody has the guts to
>>>> improve/change it.
>>>> 
>>>> Of course, we can still improve the docs of the deprecated constructors,
>>>> too.
>>>> 
>>>> Just my two cents.
>>>> 
>>>> 
>>>> -Matthias
>>>> 
>>>>> On 4/23/17 3:37 PM, Jay Kreps wrote:
>>>>> Hey guys,
>>>>> 
>>>>> I definitely think that the constructors could have been better designed,
>>>>> but I think given that they're in heavy use I don't think this proposal
>>>>> will improve things. Deprecating constructors just leaves everyone with
>>>>> lots of warnings and crossed out things. We can't actually delete the
>>>>> methods because lots of code needs to be usable across multiple Kafka
>>>>> versions, right? So we aren't picking between the original approach
>>>> (worse)
>>>>> and the new approach (better); what we are proposing is a perpetual
>>>>> mingling of the original style and the new style with a bunch of
>>>> deprecated
>>>>> stuff, which I think is worst of all.
>>>>> 
>>>>> I'd vote for just documenting the meaning of null in the ProducerRecord
>>>>> constructor.
>>>>> 
>>>>> -Jay
>>>>> 
>>>>> On Wed, Apr 19, 2017 at 3:34 PM, Stephane Maarek <
>>>>> steph...@simplemachines.com.au> wrote:
>>>>> 
>>>>>> Hi all,
>>>>>> 
>>>>>> My first KIP, let me know your thoughts!
>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP+
>>>>>> 141+-+ProducerRecordBuilder+Interface
>>>>>> 
>>>>>> 
>>>>>> Cheers,
>>>>>> Stephane
>>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>    The information contained in this email is strictly confidential and for 
>> the use of the addressee only, unless otherwise indicated. If you are not 
>> the intended recipient, please do not read, copy, use or disclose to others 
>> this message or any attachment. Please also notify the sender by replying to 
>> this email or by telephone (+44(020 7896 0011) and then delete the email and 
>> any copies of it. Opinions, conclusion (etc) that do not relate to the 
>> official business of this company shall be understood as neither given nor 
>> endorsed by it. IG is a trading name of IG Markets Limited (a company 
>> registered in England and Wales, company number 04008957) and IG Index 
>> Limited (a company registered in England and Wales, company number 
>> 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill, 
>> London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG 
>> Index Limited (register number 114059) are authorised and regulated by the 
>> Financial Conduct Authority.
>> 
>> 
>> 
> 

Reply via email to