Build failed in Jenkins: kafka-trunk-jdk11 #1593

2020-06-23 Thread Apache Jenkins Server
See Changes: [github] KAFKA-10126:Add a warning message for ConsumerPerformance (#8845) [github] KAFKA-10169: swallow non-fatal KafkaException and don't abort --

Re: [DISCUSS] KIP-487: Automatic Topic Creation on Producer

2020-06-23 Thread Boyang Chen
Hey Justin and Jiamei, I read the KIP and skimmed over the discussion. One thing I'm not fully convinced of is why we need to deprecate the server side auto topic creation logic, which seems orthogonal towards whether a client wants to create the topic or not. Won't it be more natural to assume

Build failed in Jenkins: kafka-trunk-jdk8 #4664

2020-06-23 Thread Apache Jenkins Server
See Changes: [github] KAFKA-10126:Add a warning message for ConsumerPerformance (#8845) -- [...truncated 6.31 MB...] org.apache.kafka.streams.TopologyTestDriverTest >

[jira] [Resolved] (KAFKA-9678) Introduce bounded exponential backoff in clients

2020-06-23 Thread Sanjana Kaundinya (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9678?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sanjana Kaundinya resolved KAFKA-9678. -- Resolution: Duplicate > Introduce bounded exponential backoff in clients >

Build failed in Jenkins: kafka-trunk-jdk14 #241

2020-06-23 Thread Apache Jenkins Server
See Changes: [github] KAFKA-10126:Add a warning message for ConsumerPerformance (#8845) [github] KAFKA-10169: swallow non-fatal KafkaException and don't abort --

Build failed in Jenkins: kafka-2.6-jdk8 #59

2020-06-23 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-10135: Extract Task#executeAndMaybeSwallow to be a general utility [wangguoz] KAFKA-10169: swallow non-fatal KafkaException and don't abort

Re: [DISCUSS] KIP-578: Add configuration to limit number of partitions

2020-06-23 Thread Ismael Juma
Thanks for the KIP. A couple of questions: 1. Have we considered reusing the existing PolicyViolation error code and renaming it? This would make it simpler to handle on the client. 2. What version was used for the perf section? I think master should do better than what's described there.

[jira] [Created] (KAFKA-10196) Add missing '--version' option to Kafka command producer-performance

2020-06-23 Thread jiamei xie (Jira)
jiamei xie created KAFKA-10196: -- Summary: Add missing '--version' option to Kafka command producer-performance Key: KAFKA-10196 URL: https://issues.apache.org/jira/browse/KAFKA-10196 Project: Kafka

Re: [DISCUSS] KIP-578: Add configuration to limit number of partitions

2020-06-23 Thread Boyang Chen
Hi Gokul, Thanks for the excellent KIP. I was recently driving the rollout of KIP-590 and proposed to fix the hole of the bypassing of topic creation policy when applying

Jenkins build is back to normal : kafka-trunk-jdk14 #240

2020-06-23 Thread Apache Jenkins Server
See

Re: [ANNOUNCE] New committer: Boyang Chen

2020-06-23 Thread Adam Bellemare
Just adding my congratulations, Boyang! Thank you for all your contributions and effort! On Tue, Jun 23, 2020 at 9:14 PM Kowshik Prakasam wrote: > Congrats, Boyang! :) > > > Cheers, > Kowshik > > On Tue, Jun 23, 2020 at 8:43 AM Aparnesh Gaurav > wrote: > > > Congrats Boyang. > > > > On Tue, 23

Re: [ANNOUNCE] New committer: Boyang Chen

2020-06-23 Thread Kowshik Prakasam
Congrats, Boyang! :) Cheers, Kowshik On Tue, Jun 23, 2020 at 8:43 AM Aparnesh Gaurav wrote: > Congrats Boyang. > > On Tue, 23 Jun, 2020, 9:07 PM Vahid Hashemian, > wrote: > > > Congrats Boyang! > > > > --Vahid > > > > On Tue, Jun 23, 2020 at 6:41 AM Wang (Leonard) Ge > > wrote: > > > > >

Re: [VOTE] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-06-23 Thread Boyang Chen
Thanks for the clarification, Colin and Ismael. Personally I also feel Option A is better to prioritize fixing the gap. Just to be clear, the proposed solution would be: 1. Bump the Metadata RPC version to return POLICY_VIOLATION. In the application level, we should swap the error message with

Jenkins build is back to normal : kafka-trunk-jdk11 #1591

2020-06-23 Thread Apache Jenkins Server
See

Jenkins build is back to normal : kafka-trunk-jdk8 #4662

2020-06-23 Thread Apache Jenkins Server
See

Build failed in Jenkins: kafka-trunk-jdk14 #239

2020-06-23 Thread Apache Jenkins Server
See Changes: [github] MINOR: correct the doc of transaction.timeout.ms (#8901) -- [...truncated 3.17 MB...] org.apache.kafka.streams.test.OutputVerifierTest >

Re: [VOTE] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-06-23 Thread Ismael Juma
Option A is basically what I was thinking. But with a slight adjustment: New versions of MetadataResponse return POLICY_VIOLATION, old versions return AUTHORIZATION_FAILED. The latter works correctly with old Java clients (i.e. the client fails fast and propagates the error), I've tested it.

Re: [ANNOUNCE] New committer: Boyang Chen

2020-06-23 Thread Aparnesh Gaurav
Congrats Boyang. On Tue, 23 Jun, 2020, 9:07 PM Vahid Hashemian, wrote: > Congrats Boyang! > > --Vahid > > On Tue, Jun 23, 2020 at 6:41 AM Wang (Leonard) Ge > wrote: > > > Congrats Boyang! This is a great achievement. > > > > On Tue, Jun 23, 2020 at 10:33 AM Mickael Maison < >

Re: [ANNOUNCE] New committer: Boyang Chen

2020-06-23 Thread Vahid Hashemian
Congrats Boyang! --Vahid On Tue, Jun 23, 2020 at 6:41 AM Wang (Leonard) Ge wrote: > Congrats Boyang! This is a great achievement. > > On Tue, Jun 23, 2020 at 10:33 AM Mickael Maison > wrote: > > > Congrats Boyang! Well deserved > > > > On Tue, Jun 23, 2020 at 8:20 AM Tom Bentley wrote: > > >

Re: [ANNOUNCE] New committer: Boyang Chen

2020-06-23 Thread Wang (Leonard) Ge
Congrats Boyang! This is a great achievement. On Tue, Jun 23, 2020 at 10:33 AM Mickael Maison wrote: > Congrats Boyang! Well deserved > > On Tue, Jun 23, 2020 at 8:20 AM Tom Bentley wrote: > > > > Congratulations Boyang! > > > > On Tue, Jun 23, 2020 at 8:11 AM Bruno Cadonna > wrote: > > > > >

[jira] [Created] (KAFKA-10195) Move offset management codes from ConsumerCoordinator to a new class

2020-06-23 Thread dengziming (Jira)
dengziming created KAFKA-10195: -- Summary: Move offset management codes from ConsumerCoordinator to a new class Key: KAFKA-10195 URL: https://issues.apache.org/jira/browse/KAFKA-10195 Project: Kafka

Re: [ANNOUNCE] New committer: Boyang Chen

2020-06-23 Thread Mickael Maison
Congrats Boyang! Well deserved On Tue, Jun 23, 2020 at 8:20 AM Tom Bentley wrote: > > Congratulations Boyang! > > On Tue, Jun 23, 2020 at 8:11 AM Bruno Cadonna wrote: > > > Congrats, Boyang! > > > > Best, > > Bruno > > > > On Tue, Jun 23, 2020 at 7:50 AM Konstantine Karantasis > > wrote: > > >

Re: [DISCUSS] KIP-629: Use racially neutral terms in our codebase

2020-06-23 Thread Bruno Cadonna
Hi Xavier, Thank you very much for starting this initiative! Not only for the changes to the code base but also for showing me where and how we can use more appropriate terms in general. Best, Bruno On Tue, Jun 23, 2020 at 4:17 AM John Roesler wrote: > > Hi Xavier, > > I think your approach

Build failed in Jenkins: kafka-trunk-jdk11 #1590

2020-06-23 Thread Apache Jenkins Server
See Changes: [github] KAFKA-10135: Extract Task#executeAndMaybeSwallow to be a general utility -- [...truncated 1.90 MB...] kafka.api.SaslSslAdminIntegrationTest >

Re: [ANNOUNCE] New committer: Boyang Chen

2020-06-23 Thread Tom Bentley
Congratulations Boyang! On Tue, Jun 23, 2020 at 8:11 AM Bruno Cadonna wrote: > Congrats, Boyang! > > Best, > Bruno > > On Tue, Jun 23, 2020 at 7:50 AM Konstantine Karantasis > wrote: > > > > Congrats, Boyang! > > > > -Konstantine > > > > On Mon, Jun 22, 2020 at 9:19 PM Navinder Brar > >

Re: [ANNOUNCE] New committer: Boyang Chen

2020-06-23 Thread Bruno Cadonna
Congrats, Boyang! Best, Bruno On Tue, Jun 23, 2020 at 7:50 AM Konstantine Karantasis wrote: > > Congrats, Boyang! > > -Konstantine > > On Mon, Jun 22, 2020 at 9:19 PM Navinder Brar > wrote: > > > Many Congratulations Boyang. Very well deserved. > > > > Regards,Navinder > > > > On Tuesday,

[jira] [Created] (KAFKA-10194) run the reset tool between stopping StreamsOptimizedTest and starting the new one

2020-06-23 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-10194: -- Summary: run the reset tool between stopping StreamsOptimizedTest and starting the new one Key: KAFKA-10194 URL: https://issues.apache.org/jira/browse/KAFKA-10194

Re: [VOTE] KIP-590: Redirect Zookeeper Mutation Protocols to The Controller

2020-06-23 Thread Colin McCabe
> > > On Fri, Jun 19, 2020 at 3:18 PM Ismael Juma wrote: > > > > > > > Hi Colin, > > > > > > > > The KIP states in the Compatibility section (not Future work): > > > > > > > > "To support the proxy of requests, we need to build a channel for > > > > brokers to talk directly to the controller.