Re: [ANNOUNCE] Apache Kafka 2.4.1

2020-03-12 Thread Guozhang Wang
Thanks Bill for driving this. And Many thanks to all who've contributed to this release! Guozhang On Thu, Mar 12, 2020 at 3:00 PM Matthias J. Sax wrote: > Thanks for driving the release Bill! > > -Matthias > > On 3/12/20 1:22 PM, Bill Bejeck wrote: > > The Apache Kafka community is pleased to

Create KIP permission request

2020-03-12 Thread 67
hi there I have a proposal which may change behavior of kafka-server-stop.sh, this script kill all instance on the host without any confirm, someone may just want to kill the single one, so maybe a confirm when find multy instance is more safer. already create a

Re: [kafka-clients] Re: [VOTE] 2.5.0 RC1

2020-03-12 Thread Konstantine Karantasis
Hi David, after some broader testing with Kafka Connect, we've discovered https://issues.apache.org/jira/browse/KAFKA-9712 and I'd like to raise this issue as a potential blocker for 2.5.0. It seems to be a regression after a recent essential upgrade of the reflections library. The jira contains

[jira] [Created] (KAFKA-9715) TransactionStateManager: Eliminate unused reference to interBrokerProtocolVersion

2020-03-12 Thread Kowshik Prakasam (Jira)
Kowshik Prakasam created KAFKA-9715: --- Summary: TransactionStateManager: Eliminate unused reference to interBrokerProtocolVersion Key: KAFKA-9715 URL: https://issues.apache.org/jira/browse/KAFKA-9715

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

2020-03-12 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9659: Add more log4j when updating static member mappings (#8269) -- [...truncated 5.89 MB...]

[jira] [Created] (KAFKA-9714) Flaky Test SslTransportLayerTest#testTLSDefaults

2020-03-12 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-9714: -- Summary: Flaky Test SslTransportLayerTest#testTLSDefaults Key: KAFKA-9714 URL: https://issues.apache.org/jira/browse/KAFKA-9714 Project: Kafka Issue

[jira] [Resolved] (KAFKA-9644) incrementalAlterConfigs OpType.APPEND on unset property fails with NullPointerException

2020-03-12 Thread Manikumar (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9644?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-9644. -- Fix Version/s: 2.6.0 Resolution: Fixed Issue resolved by pull request 8216

Re: Requesting Access to Submit a KIP

2020-03-12 Thread Matthias J. Sax
Done. On 3/12/20 5:06 PM, Sanjana Kaundinya wrote: > Hi, > > I’d like to submit a KIP and am requesting access to the wiki page. My wiki > ID is skaundinya. Please let me know if anything else is needed from my side > - thanks! > > Sincerely, > Sanjana > > > signature.asc Description:

Re: Add a customized logo for Kafka Streams

2020-03-12 Thread Matthias J. Sax
I personally love it! -Matthias On 3/12/20 11:31 AM, Sophie Blee-Goldman wrote: > How reasonable of it. Let's try this again: > Streams Logo option 2 > > > On Thu, Mar 12, 2020 at 9:34 AM Guozhang

Re: 回复:回复:回复:[Vote] KIP-571: Add option to force remove members in StreamsResetter

2020-03-12 Thread Guozhang Wang
Hi Matthias, About the AdminClient param API: that's a great point here. I think overall if users want to just "remove all members" they should not need to first get all the member.ids themselves, but instead internally the admin client can first issue a describe-group request to get all the

Requesting Access to Submit a KIP

2020-03-12 Thread Sanjana Kaundinya
Hi, I’d like to submit a KIP and am requesting access to the wiki page. My wiki ID is skaundinya. Please let me know if anything else is needed from my side - thanks! Sincerely, Sanjana

[DISCUSS] KIP-577: Allow HTTP Response Headers Configured for Kafka Connect

2020-03-12 Thread Zhiguo Huang

Re: [VOTE] KIP-518: Allow listing consumer groups per state

2020-03-12 Thread Colin McCabe
Thanks, Mickael. +1 (binding) best, Colin On Fri, Mar 6, 2020, at 02:05, Mickael Maison wrote: > Thanks David and Gwen for the votes > Colin, I believe I've answered all your questions, can you take another look? > > So far we have 1 binding and 5 non binding votes. > > On Mon, Mar 2, 2020 at

Re: [ANNOUNCE] Apache Kafka 2.4.1

2020-03-12 Thread Matthias J. Sax
Thanks for driving the release Bill! -Matthias On 3/12/20 1:22 PM, Bill Bejeck wrote: > The Apache Kafka community is pleased to announce the release for Apache > Kafka 2.4.1 > > This is a bug fix release and it includes fixes and improvements from 39 > JIRAs, including a few critical bugs. >

Re: 回复:回复:回复:[Vote] KIP-571: Add option to force remove members in StreamsResetter

2020-03-12 Thread Matthias J. Sax
Feyman, some more comments/questions: The description of `LeaveGroupRequest` is clear but it's unclear how `MemberToRemove` should behave. Which parameter is required? Which is optional? What is the relationship between both. The `LeaveGroupRequest` description clearly states that specifying a

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

2020-03-12 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9675: Fix bug that prevents RocksDB metrics to be updated (#8256) -- [...truncated 2.91 MB...]

Re: jeffhuang0126 --- Need permission to create KIP, discuss thread etc. for working on new KIP

2020-03-12 Thread John Roesler
Hi Jeff, I've granted you access to edit wiki pages. For discussions, you just send a message to this list (similar to the ones you can see from other KIPs). Thanks in advance for your contribution! -John On Thu, Mar 12, 2020, at 15:25, Zhiguo Huang wrote: > Hi, > > My wikid jeffhuang0126 > I

jeffhuang0126 --- Need permission to create KIP, discuss thread etc. for working on new KIP

2020-03-12 Thread Zhiguo Huang
Hi, My wikid jeffhuang0126 I need permission to create KIP, discuss thread etc. for working on new KIP. Jeff.

[ANNOUNCE] Apache Kafka 2.4.1

2020-03-12 Thread Bill Bejeck
The Apache Kafka community is pleased to announce the release for Apache Kafka 2.4.1 This is a bug fix release and it includes fixes and improvements from 39 JIRAs, including a few critical bugs. All of the changes in this release can be found in the release notes:

[jira] [Created] (KAFKA-9713) Remove BufferExhausedException

2020-03-12 Thread Brian Byrne (Jira)
Brian Byrne created KAFKA-9713: -- Summary: Remove BufferExhausedException Key: KAFKA-9713 URL: https://issues.apache.org/jira/browse/KAFKA-9713 Project: Kafka Issue Type: Task

[jira] [Resolved] (KAFKA-9364) Fix misleading consumer logs on throttling

2020-03-12 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9364?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-9364. - Fix Version/s: 2.5.0 Resolution: Fixed > Fix misleading consumer logs on throttling >

Re: Add a customized logo for Kafka Streams

2020-03-12 Thread Sophie Blee-Goldman
How reasonable of it. Let's try this again: Streams Logo option 2 On Thu, Mar 12, 2020 at 9:34 AM Guozhang Wang wrote: > Hi Sophie, > > I cannot find the attachment from your previous email --- in

[jira] [Created] (KAFKA-9712) Reflections library 0.9.12 introduced in 2.5 causes regression scanning for plugins on plugin_path

2020-03-12 Thread Nigel Liang (Jira)
Nigel Liang created KAFKA-9712: -- Summary: Reflections library 0.9.12 introduced in 2.5 causes regression scanning for plugins on plugin_path Key: KAFKA-9712 URL: https://issues.apache.org/jira/browse/KAFKA-9712

Re: Add a customized logo for Kafka Streams

2020-03-12 Thread Guozhang Wang
Hi Sophie, I cannot find the attachment from your previous email --- in fact, ASF mailing list usually blocks all attachments for security reasons. If you can share a link to the image (google drawings etc) in your email that would be great. Guozhang On Wed, Mar 11, 2020 at 1:02 PM Sophie

Re: Timestamp check in Kafka TransactionMetadata

2020-03-12 Thread Guozhang Wang
Hello, We have one PR which I believe is addressing exactly your observed problem: https://github.com/apache/kafka/pull/8278 Guozhang On Wed, Mar 11, 2020 at 11:57 PM Guru C G wrote: > We have come across an issue where in FATAL messages are logged in the > broker. > > FATAL

Re: Create KIP permissions

2020-03-12 Thread Bill Bejeck
Hi Paolo, You're all set now. Thanks, Bill On Thu, Mar 12, 2020 at 9:54 AM Paolo Moriello wrote: > Hi, > > My name is Paolo Moriello, I'd like to get permissions to create KIPs. My > wiki id is: paolomoriello. > > Thanks, > Paolo >

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

2020-03-12 Thread Apache Jenkins Server
See

Re: Create KIP permissions

2020-03-12 Thread Jun Rao
Hi, Paolo, Thanks for your interest. Just gave you the wiki permissions. Jun On Thu, Mar 12, 2020 at 6:54 AM Paolo Moriello wrote: > Hi, > > My name is Paolo Moriello, I'd like to get permissions to create KIPs. My > wiki id is: paolomoriello. > > Thanks, > Paolo >

Re: [kafka-clients] Re: [VOTE] 2.5.0 RC1

2020-03-12 Thread Bill Bejeck
Hi David, 1. Scanned the Javadoc, looks good 2. Downloaded kafka_2.12-2.5.0 and ran the quickstart and streams quickstart 3. Verified the signatures +1 (non-binding) Thanks for running the release David! -Bill On Tue, Mar 10, 2020 at 4:01 PM David Arthur wrote: > Thanks for the

[jira] [Created] (KAFKA-9711) The authentication failure caused by SSLEngine#beginHandshake is not properly caught and handled

2020-03-12 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-9711: - Summary: The authentication failure caused by SSLEngine#beginHandshake is not properly caught and handled Key: KAFKA-9711 URL: https://issues.apache.org/jira/browse/KAFKA-9711

Create KIP permissions

2020-03-12 Thread Paolo Moriello
Hi, My name is Paolo Moriello, I'd like to get permissions to create KIPs. My wiki id is: paolomoriello. Thanks, Paolo

[jira] [Resolved] (KAFKA-9695) AdminClient allows null topic configs, but broker throws NPE

2020-03-12 Thread Rajini Sivaram (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9695?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-9695. --- Reviewer: Manikumar Resolution: Fixed Updated broker to throw InvalidRequestException

Re: [DISCUSS] KIP-552: Add interface to handle unused config

2020-03-12 Thread Patrik Kleindl
+1 to Matthias' proposal. First reduce the false positives, then improve the various clients. best regards Patrik On Thu, 12 Mar 2020 at 07:20, Matthias J. Sax wrote: > Personally, I think that adding a specify prefix for user configs would > be the simplest fix. Kafka Streams and Kafka Connect

[jira] [Created] (KAFKA-9710) Windows kafka-server-start.bat script fails when Kafka is located in path with whitespace

2020-03-12 Thread Tomas Kovan (Jira)
Tomas Kovan created KAFKA-9710: -- Summary: Windows kafka-server-start.bat script fails when Kafka is located in path with whitespace Key: KAFKA-9710 URL: https://issues.apache.org/jira/browse/KAFKA-9710

Timestamp check in Kafka TransactionMetadata

2020-03-12 Thread Guru C G
We have come across an issue where in FATAL messages are logged in the broker. FATAL kafka.coordinator.transaction.TransactionMetadata: TransactionMetadata(transactionalId=tx-id-1, producerId=96011, producerEpoch=51, txnTimeoutMs=6, state=CompleteCommit, pendingState=Some(Ongoing),

Re: [DISCUSS] KIP-552: Add interface to handle unused config

2020-03-12 Thread Matthias J. Sax
Personally, I think that adding a specify prefix for user configs would be the simplest fix. Kafka Streams and Kafka Connect could just use this prefix to avoid the false positive WARN logs. Hence, the KIP does not need to fix every false positive warning, it just changes the AbstractConfig about

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

2020-03-12 Thread Apache Jenkins Server
See Changes: [github] KAFKA-9701: Add more debug log on client to reproduce the issue (#8272) -- [...truncated 5.89 MB...]