Re: Re: [DISCUSS] KIP-706: Add method "Producer#produce" to return CompletionStage instead of Future

2021-01-30 Thread Chia-Ping Tsai
Fair enough. I will isolate KIP-706 later. On 2021/01/30 20:18:42 Ismael Juma wrote: > I agree with Jason and would rather tackle this as one KIP and the error > handling in another. The error handling can follow this one once we agree > on the method signatures. Furthermore, we should align with

Re: Re: [DISCUSS] KIP-706: Add method "Producer#produce" to return CompletionStage instead of Future

2021-01-30 Thread Chia-Ping Tsai
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.

Build failed in Jenkins: Kafka » kafka-trunk-jdk15 #476

2021-01-30 Thread Apache Jenkins Server
See Changes: [github] KAFKA-12236; New meta.properties logic for KIP-500 (#9967) -- [...truncated 3.58 MB...] ZooKeeperClientTest > testPipelinedGetData()

Build failed in Jenkins: Kafka » kafka-trunk-jdk8 #427

2021-01-30 Thread Apache Jenkins Server
See Changes: [github] KAFKA-12236; New meta.properties logic for KIP-500 (#9967) -- [...truncated 3.57 MB...] LogValidatorTest >

Build failed in Jenkins: Kafka » kafka-trunk-jdk11 #460

2021-01-30 Thread Apache Jenkins Server
See Changes: [github] KAFKA-12236; New meta.properties logic for KIP-500 (#9967) -- [...truncated 3.60 MB...] KafkaZkClientTest >

[jira] [Resolved] (KAFKA-12236) Add version 1 of meta.properties for KIP-500

2021-01-30 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12236?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-12236. - Resolution: Fixed > Add version 1 of meta.properties for KIP-500 >

Re: [DISCUSS] KIP-706: Add method "Producer#produce" to return CompletionStage instead of Future

2021-01-30 Thread Ismael Juma
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

Re: [DISCUSS] KIP-706: Add method "Producer#produce" to return CompletionStage instead of Future

2021-01-30 Thread Ismael Juma
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

Re: [DISCUSS] KIP-706: Add method "Producer#produce" to return CompletionStage instead of Future

2021-01-30 Thread Jason Gustafson
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

Re: [DISCUSS] KIP-706: Add method "Producer#produce" to return CompletionStage instead of Future

2021-01-30 Thread Jason Gustafson
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

[jira] [Created] (KAFKA-12260) PartitionsFor should not return null value

2021-01-30 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-12260: --- Summary: PartitionsFor should not return null value Key: KAFKA-12260 URL: https://issues.apache.org/jira/browse/KAFKA-12260 Project: Kafka Issue Type:

Re: [DISCUSS] KIP-706: Add method "Producer#produce" to return CompletionStage instead of Future

2021-01-30 Thread Ismael Juma
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

Re: [DISCUSS] KIP-706: Add method "Producer#produce" to return CompletionStage instead of Future

2021-01-30 Thread Ismael Juma
I agree with Jason and would rather tackle this as one KIP and the error handling in another. The error handling can follow this one once we agree on the method signatures. Furthermore, we should align with the longer term plan for the AdminClient that returns KafkaFuture today. Ismael On Sat,

Re: [DISCUSS] KIP-691: Transactional Producer Exception Handling

2021-01-30 Thread Chia-Ping Tsai
> a richer return type. Let me know if you are good with this, and whether > Chia-Ping is also happy here :) sure. I'd like to join this bigger party :) On 2021/01/30 01:03:57, Boyang Chen wrote: > This is a great proposal Jason, I already integrated KIP-691 with KIP-706 > template to provide

Re: [DISCUSS] KIP-706: Add method "Producer#produce" to return CompletionStage instead of Future

2021-01-30 Thread Chia-Ping Tsai
I'd like to merge KIP-706 to KIP-691 as it can bring a comprehensive design for both new API and exception. The new exception should be included by the new API also. On 2021/01/30 19:30:40, Jason Gustafson wrote: > I think this still makes sense as a separate KIP. For KIP-691, we are just >

Re: [DISCUSS] KIP-706: Add method "Producer#produce" to return CompletionStage instead of Future

2021-01-30 Thread Jason Gustafson
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 send(String topic, Message message);

Re: [DISCUSS] KIP-706: Add method "Producer#produce" to return CompletionStage instead of Future

2021-01-30 Thread Jason Gustafson
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 wrote: > Are we saying that we won't pursue this KIP in favor of the other one? > > Ismael > > On Sat, Jan

Re: [DISCUSS] KIP-706: Add method "Producer#produce" to return CompletionStage instead of Future

2021-01-30 Thread Ismael Juma
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 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( >

Re: [DISCUSS] KIP-706: Add method "Producer#produce" to return CompletionStage instead of Future

2021-01-30 Thread Jorge Esteban Quilcate Otoya
Hi all, My 2c: Pros of "Produce": - Same as Kafka Protocol "ProduceRequest" and simple enough "producer#produce()"; though consumer also differ between "FetchRequest" and "consumer#poll". - As reference, other libraries also use "produce", e.g.: librdkafka-derived libraries:

Jenkins build is back to normal : Kafka » kafka-trunk-jdk8 #426

2021-01-30 Thread Apache Jenkins Server
See

Build failed in Jenkins: Kafka » kafka-trunk-jdk15 #475

2021-01-30 Thread Apache Jenkins Server
See Changes: [github] MINOR: AbstractCoordinatorTest should close coordinator explicitly (#10001) -- [...truncated 3.53 MB...] ZooKeeperClientTest >

Build failed in Jenkins: Kafka » kafka-trunk-jdk11 #459

2021-01-30 Thread Apache Jenkins Server
See Changes: [github] MINOR: AbstractCoordinatorTest should close coordinator explicitly (#10001) -- [...truncated 3.57 MB...] ControllerEventManagerTest

Re: [DISCUSS] KIP-706: Add method "Producer#produce" to return CompletionStage instead of Future

2021-01-30 Thread Govinda Sakhare
Hi, Indeed `transmit(msg)` looks good. Another option could be `publish(msg)` Best Regards, Govinda On Sat, Jan 30, 2021, 17:45 Chia-Ping Tsai 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( >

Re: [DISCUSS] KIP-706: Add method "Producer#produce" to return CompletionStage instead of Future

2021-01-30 Thread Chia-Ping Tsai
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