> > 3. Did we consider the possibility of introducing a new interface which
> > extended both CompletionStage and Future? That would make it easier for
> > people to update their existing code, since the handling of the result (in
> > the case they weren't using the Callback version) would be source
> > compatible. Not that I particularly want to introduce a new
> > KafkaFuture-like type, but just thought it worthwhile to float the idea.
> > 4. What about the possibility of users doing
> > `send().toCompletableFuture().complete(...)`. In KIP-707 I explicitly
> > prevented that, and I can't think of any use cases why we'd want to allow
> > it here. It's easier to start off preventing that kind of accidental misuse
> > and later allowing it when people turn up with valid use cases. KIP-707
> > would provide an internal KafkaCompletableFuture which should make doing
> > this relatively simple, I think.

As we don't deprecate other two send methods, I feel returning CompletionStage 
is more acceptable. Users who want to call blocking method can call other send 
methods. Also, returning interface (CompletionStage) open a room to us to 
implement more powerful custom CompletionStage.


On 2021/04/02 01:38:12, Chia-Ping Tsai <chia7...@apache.org> wrote: 
> hi Tom,
> 
> thanks for all your suggestions!
> 
> > 2. I'm not sure that having separate Builder.topic() and .partition()
> > methods is better than forcing people to set the target via a single method
> > call. For example, `Builder.target(String topic)`, `Builder.target(String
> > topic, int partition)`, `Builder.target(TopicPartition)` and
> > `Builder.target(Uuid)` (or higher level equivalent) forces the targeting to
> > a single place. It's also compatible with the SendTarget idea being added
> > later on.
> 
> nice one. will update KIP.
> 
> > 3. Did we consider the possibility of introducing a new interface which
> > extended both CompletionStage and Future? That would make it easier for
> > people to update their existing code, since the handling of the result (in
> > the case they weren't using the Callback version) would be source
> > compatible. Not that I particularly want to introduce a new
> > KafkaFuture-like type, but just thought it worthwhile to float the idea.
> > 4. What about the possibility of users doing
> > `send().toCompletableFuture().complete(...)`. In KIP-707 I explicitly
> > prevented that, and I can't think of any use cases why we'd want to allow
> > it here. It's easier to start off preventing that kind of accidental misuse
> > and later allowing it when people turn up with valid use cases. KIP-707
> > would provide an internal KafkaCompletableFuture which should make doing
> > this relatively simple, I think.
> 
> Yep, returning ComopletableFuture can bring less change to users. Also, I can 
> file a follow-up to replace ComopletableFuture by KafkaCompletableFuture if 
> KafkaCompletableFuture can prevent such accident.
> 
> > 2. I'm not sure that having separate Builder.topic() and .partition()
> > methods is better than forcing people to set the target via a single method
> > call. For example, `Builder.target(String topic)`, `Builder.target(String
> > topic, int partition)`, `Builder.target(TopicPartition)` and
> > `Builder.target(Uuid)` (or higher level equivalent) forces the targeting to
> > a single place. It's also compatible with the SendTarget idea being added
> > later on.
> > If we're intending to add a send-to-topic-id feature it would affect
> > ProducerRecord.topic(). Although not currently documented to never return
> > null, it currently has that semantic, and the methods of ProducerRecord
> > which can return null are explicitly documented to do so. Making it return
> > null would be a backwards incompatible change, so perhaps we should change
> > its contract now to allow us to support topic ids in 3.x? If we went with
> > the SendTarget idea ProducerRecord would presumably gain a target() method
> > (and we could deprecate topic()), and if not I suppose ProducerRecord would
> > gain a topicId() method, and exactly one of topic() and topicId() would
> > return null, which isn't terribly nice.
> 
> SendTarget seems to be a good way to deal with both topic name and topic id. 
> I will re-think the interface mentioned by Jason (send(SendTarget target, 
> Record record)) and your suggestions. As the new interface `Record` don't 
> have the topic name/id, we don't need to make ProducerRecord extend the new 
> interface and so users have to call new `send` if they want to send data to 
> topic id.
> 
> On 2021/03/31 14:10:35, Tom Bentley <tbent...@redhat.com> wrote: 
> > Hi,
> > 
> > Starting with the KIP as written:
> > 
> > 1. I think the Builder.key() and Builder.value() methods in the KIP have
> > the wrong parameter type: Should be K and V I think.
> > 
> > 2. I'm not sure that having separate Builder.topic() and .partition()
> > methods is better than forcing people to set the target via a single method
> > call. For example, `Builder.target(String topic)`, `Builder.target(String
> > topic, int partition)`, `Builder.target(TopicPartition)` and
> > `Builder.target(Uuid)` (or higher level equivalent) forces the targeting to
> > a single place. It's also compatible with the SendTarget idea being added
> > later on.
> > 
> > 3. Did we consider the possibility of introducing a new interface which
> > extended both CompletionStage and Future? That would make it easier for
> > people to update their existing code, since the handling of the result (in
> > the case they weren't using the Callback version) would be source
> > compatible. Not that I particularly want to introduce a new
> > KafkaFuture-like type, but just thought it worthwhile to float the idea.
> > 
> > 4. What about the possibility of users doing
> > `send().toCompletableFuture().complete(...)`. In KIP-707 I explicitly
> > prevented that, and I can't think of any use cases why we'd want to allow
> > it here. It's easier to start off preventing that kind of accidental misuse
> > and later allowing it when people turn up with valid use cases. KIP-707
> > would provide an internal KafkaCompletableFuture which should make doing
> > this relatively simple, I think.
> > 
> > Some thoughts on Jason's SendTarget idea... There is some utility in being
> > able to construct a Record to be sent prior to deciding its destination.
> > The Builder gives us this too, but in a mutable way which allows the
> > destination to be overwritten. The Builder also allows people to easily
> > write a Builder-returning method in Java; in contrast it's awkward to
> > return a Record and SendTarget pair. Java's (current) lack of sealed
> > interfaces also reduces the type safety of the SendTarget a little.
> > 
> > If we're intending to add a send-to-topic-id feature it would affect
> > ProducerRecord.topic(). Although not currently documented to never return
> > null, it currently has that semantic, and the methods of ProducerRecord
> > which can return null are explicitly documented to do so. Making it return
> > null would be a backwards incompatible change, so perhaps we should change
> > its contract now to allow us to support topic ids in 3.x? If we went with
> > the SendTarget idea ProducerRecord would presumably gain a target() method
> > (and we could deprecate topic()), and if not I suppose ProducerRecord would
> > gain a topicId() method, and exactly one of topic() and topicId() would
> > return null, which isn't terribly nice.
> > 
> > Hopefully at least some of that is helpful.
> > 
> > Kind regards,
> > 
> > Tom
> > 
> > 
> > On Wed, Mar 31, 2021 at 4:55 AM Chia-Ping Tsai <chia7...@apache.org> wrote:
> > 
> > > hi,
> > >
> > > I have updated KIP according to my latest response. I will start vote
> > > thread next week if there is no more comments :)
> > >
> > > Best Regards,
> > > Chia-Ping
> > >
> > > On 2021/01/31 05:39:17, Chia-Ping Tsai <chia7...@apache.org> wrote:
> > > > It seems to me changing the input type might make complicate the
> > > migration from deprecated send method to new API.
> > > >
> > > > Personally, I prefer to introduce a interface called “SendRecord” to
> > > replace ProducerRecord. Hence, the new API/classes is shown below.
> > > >
> > > > 1. CompletionStage send(SendRecord)
> > > > 2. class ProducerRecord implement SendRecord
> > > > 3. Introduce builder pattern for SendRecord
> > > >
> > > > That includes following benefit.
> > > >
> > > > 1. Kafka users who don’t use both return type and callback do not need
> > > to change code even though we remove deprecated send methods. (of course,
> > > they still need to compile code with new Kafka)
> > > >
> > > > 2. Kafka users who need Future can easily migrate to new API by regex
> > > replacement. (cast ProduceRecord to SendCast and add toCompletableFuture)
> > > >
> > > > 3. It is easy to support topic id in the future. We can add new method
> > > to SendRecord builder. For example:
> > > >
> > > > Builder topicName(String)
> > > > Builder topicId(UUID)
> > > >
> > > > 4. builder pattern can make code more readable. Especially, Produce
> > > record has a lot of fields which can be defined by users.
> > > > —
> > > > Chia-Ping
> > > >
> > > > On 2021/01/30 22:50:36 Ismael Juma wrote:
> > > > > Another thing to think about: the consumer api currently has
> > > > > `subscribe(String|Pattern)` and a number of methods that accept
> > > > > `TopicPartition`. A similar approach could be used for the Consumer to
> > > work
> > > > > with topic ids or topic names. The consumer side also has to support
> > > > > regexes so it probably makes sense to have a separate interface.
> > > > >
> > > > > Ismael
> > > > >
> > > > > On Sat, Jan 30, 2021 at 2:40 PM Ismael Juma <ism...@juma.me.uk> wrote:
> > > > >
> > > > > > I think this is a promising idea. I'd personally avoid the overload
> > > and
> > > > > > simply have a `Topic` type that implements `SendTarget`. It's a mix
> > > of both
> > > > > > proposals: strongly typed, no overloads and general class names that
> > > > > > implement `SendTarget`.
> > > > > >
> > > > > > Ismael
> > > > > >
> > > > > > On Sat, Jan 30, 2021 at 2:22 PM Jason Gustafson <ja...@confluent.io>
> > > > > > wrote:
> > > > > >
> > > > > >> Giving this a little more thought, I imagine sending to a topic is
> > > the
> > > > > >> most
> > > > > >> common case, so maybe it's an overload worth having. Also, if
> > > `SendTarget`
> > > > > >> is just a marker interface, we could let `TopicPartition` implement
> > > it
> > > > > >> directly. Then we have:
> > > > > >>
> > > > > >> interface SendTarget;
> > > > > >> class TopicPartition implements SendTarget;
> > > > > >>
> > > > > >> CompletionStage<RecordMetadata> send(String topic, Record record);
> > > > > >> CompletionStage<RecordMetadata> send(SendTarget target, Record
> > > record);
> > > > > >>
> > > > > >> The `SendTarget` would give us a lot of flexibility in the future.
> > > It
> > > > > >> would
> > > > > >> give us a couple options for topic ids. We could either have an
> > > overload
> > > > > >> of
> > > > > >> `send` which accepts `Uuid`, or we could add a `TopicId` type which
> > > > > >> implements `SendTarget`.
> > > > > >>
> > > > > >> -Jason
> > > > > >>
> > > > > >>
> > > > > >> On Sat, Jan 30, 2021 at 1:11 PM Jason Gustafson <ja...@confluent.io
> > > >
> > > > > >> wrote:
> > > > > >>
> > > > > >> > Yeah, good question. I guess we always tend to regret using
> > > lower-level
> > > > > >> > types in these APIs. Perhaps there should be some kind of
> > > interface:
> > > > > >> >
> > > > > >> > interface SendTarget
> > > > > >> > class TopicIdTarget implements SendTarget
> > > > > >> > class TopicTarget implements SendTarget
> > > > > >> > class TopicPartitionTarget implements SendTarget
> > > > > >> >
> > > > > >> > Then we just have:
> > > > > >> >
> > > > > >> > CompletionStage<RecordMetadata> send(SendTarget target, Record
> > > record);
> > > > > >> >
> > > > > >> > Not sure if we could reuse `Record` in the consumer though. We do
> > > have
> > > > > >> > some state in `ConsumerRecord` which is not present in
> > > `ProducerRecord`
> > > > > >> > (e.g. offset). Perhaps we could provide a `Record` view from
> > > > > >> > `ConsumerRecord` for convenience. That would be useful for use
> > > cases
> > > > > >> which
> > > > > >> > involve reading from one topic and writing to another.
> > > > > >> >
> > > > > >> > -Jason
> > > > > >> >
> > > > > >> > On Sat, Jan 30, 2021 at 12:29 PM Ismael Juma <ism...@juma.me.uk>
> > > wrote:
> > > > > >> >
> > > > > >> >> Interesting idea. A couple of things to consider:
> > > > > >> >>
> > > > > >> >> 1. Would we introduce the Message concept to the Consumer too? I
> > > think
> > > > > >> >> that's what .NET does.
> > > > > >> >> 2. If we eventually allow a send to a topic id instead of topic
> > > name,
> > > > > >> >> would
> > > > > >> >> that result in two additional overloads?
> > > > > >> >>
> > > > > >> >> Ismael
> > > > > >> >>
> > > > > >> >> On Sat, Jan 30, 2021 at 11:38 AM Jason Gustafson <
> > > ja...@confluent.io>
> > > > > >> >> wrote:
> > > > > >> >>
> > > > > >> >> > For the sake of having another option to shoot down, we could
> > > take a
> > > > > >> >> page
> > > > > >> >> > from the .net client and separate the message data from the
> > > > > >> destination
> > > > > >> >> > (i.e. topic or partition). This would get around the need to
> > > use a
> > > > > >> new
> > > > > >> >> > verb. For example:
> > > > > >> >> >
> > > > > >> >> > CompletionStage<RecordMetadata> send(String topic, Message
> > > message);
> > > > > >> >> > CompletionStage<RecordMetadata> send(TopicPartition
> > > topicPartition,
> > > > > >> >> Message
> > > > > >> >> > message);
> > > > > >> >> >
> > > > > >> >> > -Jason
> > > > > >> >> >
> > > > > >> >> >
> > > > > >> >> >
> > > > > >> >> > On Sat, Jan 30, 2021 at 11:30 AM Jason Gustafson <
> > > ja...@confluent.io
> > > > > >> >
> > > > > >> >> > wrote:
> > > > > >> >> >
> > > > > >> >> > > I think this still makes sense as a separate KIP. For
> > > KIP-691, we
> > > > > >> are
> > > > > >> >> > just
> > > > > >> >> > > looking to help define the error contract for the new API.
> > > > > >> >> > >
> > > > > >> >> > > -Jason
> > > > > >> >> > >
> > > > > >> >> > > On Sat, Jan 30, 2021 at 8:39 AM Ismael Juma <
> > > ism...@juma.me.uk>
> > > > > >> >> wrote:
> > > > > >> >> > >
> > > > > >> >> > >> Are we saying that we won't pursue this KIP in favor of the
> > > other
> > > > > >> >> one?
> > > > > >> >> > >>
> > > > > >> >> > >> Ismael
> > > > > >> >> > >>
> > > > > >> >> > >> On Sat, Jan 30, 2021, 4:15 AM Chia-Ping Tsai <
> > > chia7...@apache.org
> > > > > >> >
> > > > > >> >> > wrote:
> > > > > >> >> > >>
> > > > > >> >> > >> > hi Jason
> > > > > >> >> > >> >
> > > > > >> >> > >> > Thanks for your response. "transmit" is good to me.
> > > > > >> >> > >> >
> > > > > >> >> > >> > As we discussed by email, KIP-706 is going to be merged 
> > > > > >> >> > >> > to
> > > > > >> KIP-691(
> > > > > >> >> > >> > https://cwiki.apache.org/confluence/x/PSfZCQ). Hence,
> > > please
> > > > > >> feel
> > > > > >> >> > free
> > > > > >> >> > >> to
> > > > > >> >> > >> > replace "produce" by "transmit" in KIP-691.
> > > > > >> >> > >> >
> > > > > >> >> > >> > Best,
> > > > > >> >> > >> > Chia-Ping
> > > > > >> >> > >> >
> > > > > >> >> > >> > On 2021/01/30 00:48:38, Jason Gustafson <
> > > ja...@confluent.io>
> > > > > >> >> wrote:
> > > > > >> >> > >> > > Hi Chia-Ping,
> > > > > >> >> > >> > >
> > > > > >> >> > >> > > I think this is a great idea. It is a pity that we
> > > cannot
> > > > > >> >> continue
> > > > > >> >> > to
> > > > > >> >> > >> use
> > > > > >> >> > >> > > the `send` verb, but I don't see how we can. I know we
> > > > > >> considered
> > > > > >> >> > >> > > `transmit` as another option which is closer to `send`.
> > > That
> > > > > >> >> would
> > > > > >> >> > >> avoid
> > > > > >> >> > >> > > the redundancy when people choose the common "producer"
> > > > > >> variable
> > > > > >> >> > name.
> > > > > >> >> > >> > >
> > > > > >> >> > >> > > producer.transmit
> > > > > >> >> > >> > >
> > > > > >> >> > >> > > instead of
> > > > > >> >> > >> > >
> > > > > >> >> > >> > > producer.produce
> > > > > >> >> > >> > >
> > > > > >> >> > >> > > A couple alternatives might be `write` or `append`. I'm
> > > happy
> > > > > >> >> with
> > > > > >> >> > >> > > `produce` as well, but curious if others have thoughts.
> > > > > >> >> > >> > >
> > > > > >> >> > >> > > -Jason
> > > > > >> >> > >> > >
> > > > > >> >> > >> > >
> > > > > >> >> > >> > > On Wed, Jan 20, 2021 at 9:37 AM Chia-Ping Tsai <
> > > > > >> >> chia7...@apache.org
> > > > > >> >> > >
> > > > > >> >> > >> > wrote:
> > > > > >> >> > >> > >
> > > > > >> >> > >> > > > Dear all,
> > > > > >> >> > >> > > >
> > > > > >> >> > >> > > > I'd like to start the discussion thread for KIP-706:
> > > > > >> >> > >> > > >
> > > > > >> >> > >> >
> > > > > >> >> > >>
> > > > > >> >> >
> > > > > >> >>
> > > > > >>
> > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=100829459
> > > > > >> >> > >> > > >
> > > > > >> >> > >> > > > KIP-706 is proposing to introduce new API
> > > "CompletionStage
> > > > > >> >> > >> > > > produce(record)" to Producer. Kafka users can 
> > > > > >> >> > >> > > > leverage
> > > > > >> >> > >> CompletionStage
> > > > > >> >> > >> > to
> > > > > >> >> > >> > > > write asynchronous non-blocking code. CompletionStage
> > > is
> > > > > >> more
> > > > > >> >> > >> powerful
> > > > > >> >> > >> > than
> > > > > >> >> > >> > > > Future and callback. Also, the code using Future and
> > > > > >> callback
> > > > > >> >> can
> > > > > >> >> > be
> > > > > >> >> > >> > easily
> > > > > >> >> > >> > > > re-written by CompletionStage.
> > > > > >> >> > >> > > >
> > > > > >> >> > >> > > > Cheers,
> > > > > >> >> > >> > > > Chia-Ping
> > > > > >> >> > >> > > >
> > > > > >> >> > >> > > >
> > > > > >> >> > >> > >
> > > > > >> >> > >> >
> > > > > >> >> > >>
> > > > > >> >> > >
> > > > > >> >> >
> > > > > >> >>
> > > > > >> >
> > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> > >
> > 
> 

Reply via email to