Re: [DISCUSS] KIP-511: Collect and Expose Client's Name and Version in the Brokers

2019-09-03 Thread David Jacot
Hi Colin, Thanks for your input. Please, find my comments below: >> Currently, we don't parse the contents of ApiVersionsRequest at all, since it's an empty message. KIP-511 proposes adding some fields here, which will clearly change that situation. In the future, any changes to ApiVersionsRequ

Re: [VOTE] KIP-512:Adding headers to RecordMetaData

2019-09-03 Thread Maulin Vasavada
+1 (non-binding) On Tue, Sep 3, 2019 at 3:38 PM Renuka M wrote: > Hi All, > > After good discussion for KIP-512 > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-512%3AAdding+headers+to+RecordMetaData > , > am starting thread for voting. > > Thanks > Renuka M >

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

2019-09-03 Thread Apache Jenkins Server
See Changes: [rhauch] Changed for updatedTasks, avoids stopping and starting of unnecessary [rhauch] MINOR: Add unit test for KAFKA-8676 to guard against unrequired task

Re: [DISCUSS] KIP-486 Support for pluggable KeyStore and TrustStore

2019-09-03 Thread Maulin Vasavada
Hi all Please check https://github.com/maulin-vasavada/kafka/commit/44f86395b1ba3fe4bd87de89029d72da77995ff8 This is just the first cut obviously. There are few call outs I would like to make, 1. So far I kept the old SslEngineBuilder hence I had to name the interface with "I" (that can change

Build failed in Jenkins: kafka-2.3-jdk8 #99

2019-09-03 Thread Apache Jenkins Server
See Changes: [rhauch] Changed for updatedTasks, avoids stopping and starting of unnecessary [rhauch] MINOR: Add unit test for KAFKA-8676 to guard against unrequired task -- [

Re: [VOTE] KIP-401: TransformerSupplier/ProcessorSupplier StateStore connecting

2019-09-03 Thread Guozhang Wang
Hi Paul, thanks for the confirmation! Since we have three binding votes now I think you can proceed and mark it as accepted. On Tue, Sep 3, 2019 at 3:17 PM Paul Whalen wrote: > Yeah, agreed on it being the same reference. That’s the way I have it in > the working PR and I’ll update the KIP for

[jira] [Created] (KAFKA-8864) Kafka Producer deadlocked on flush call

2019-09-03 Thread Shaan Appel (Jira)
Shaan Appel created KAFKA-8864: -- Summary: Kafka Producer deadlocked on flush call Key: KAFKA-8864 URL: https://issues.apache.org/jira/browse/KAFKA-8864 Project: Kafka Issue Type: Bug C

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

2019-09-03 Thread Apache Jenkins Server
See Changes: [bill] MINOR: Use new `Admin` interface instead of `KafkaAdminClient` class -- [...truncated 5.99 MB...] org.apache.kafka.connect.runtime.rest.resources.Conne

Re: Request for permission to create KIP

2019-09-03 Thread Bill Bejeck
Lucas, You're all set now. -Bill On Tue, Sep 3, 2019 at 7:00 PM Lucas Bradstreet wrote: > Hi, > > Could I please be given permission to add a KIP to > > https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals > ? > My username is lucasbradstreet. > > Thanks >

Request for permission to create KIP

2019-09-03 Thread Lucas Bradstreet
Hi, Could I please be given permission to add a KIP to https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals? My username is lucasbradstreet. Thanks

Re: [DISCUSS] KIP-504 - Add new Java Authorizer Interface

2019-09-03 Thread Don Bosco Durai
Hi Rajini Help me understand this a bit more. 1. For all practical purpose, without authorization you can't go to the next step. The calling code needs to block anyway. So should we just let the implementation code do the async part? 2. If you feel management calls need to be async, then we sho

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

2019-09-03 Thread Apache Jenkins Server
See

[VOTE] KIP-512:Adding headers to RecordMetaData

2019-09-03 Thread Renuka M
Hi All, After good discussion for KIP-512 https://cwiki.apache.org/confluence/display/KAFKA/KIP-512%3AAdding+headers+to+RecordMetaData, am starting thread for voting. Thanks Renuka M

Re: [VOTE] KIP-401: TransformerSupplier/ProcessorSupplier StateStore connecting

2019-09-03 Thread Paul Whalen
Yeah, agreed on it being the same reference. That’s the way I have it in the working PR and I’ll update the KIP for clarity. > On Sep 3, 2019, at 5:04 PM, Matthias J. Sax wrote: > > I am strongly in favor of "must be the same reference". > > > -Matthias > >> On 9/3/19 2:09 PM, Guozhang Wang

Re: [DISCUSS] KIP-511: Collect and Expose Client's Name and Version in the Brokers

2019-09-03 Thread Colin McCabe
Hi David, Thanks again for the KIP. Currently, we don't parse the contents of ApiVersionsRequest at all, since it's an empty message. KIP-511 proposes adding some fields here, which will clearly change that situation. In the future, any changes to ApiVersionsRequest will have to only add stu

Re: [VOTE] KIP-482: The Kafka Protocol should Support Optional Tagged Fields

2019-09-03 Thread Jose Armando Garcia Sancio
+1 (non-binding) Looking forward to this improvement. On Tue, Sep 3, 2019 at 12:49 PM David Jacot wrote: > +1 (non-binding) > > Thank for the KIP. Great addition to the Kafka protocol! > > Best, > David > > Le mar. 3 sept. 2019 à 19:17, Colin McCabe a écrit : > > > Hi all, > > > > I'd like to

Re: request for permission to create KIP

2019-09-03 Thread Matthias J. Sax
What is your wiki ID (ie, account name) -Matthias On 8/26/19 11:43 AM, KUN DU wrote: > Hey, > > I want to create a KIP for > https://issues.apache.org/jira/browse/KAFKA-7711 > to initiate discussion. > > Can someone grant me permission? > > Thanks, > Kun > signature.asc Description: OpenP

Re: [VOTE] KIP-401: TransformerSupplier/ProcessorSupplier StateStore connecting

2019-09-03 Thread Matthias J. Sax
I am strongly in favor of "must be the same reference". -Matthias On 9/3/19 2:09 PM, Guozhang Wang wrote: > Hi Paul, > > Thanks for the KIP! +1 (binding). > > One minor comment about the following: > > "In order to solve the problem of addStateStore potentially being called > twice for the sa

Re: [VOTE] KIP-401: TransformerSupplier/ProcessorSupplier StateStore connecting

2019-09-03 Thread Guozhang Wang
Hi Paul, Thanks for the KIP! +1 (binding). One minor comment about the following: "In order to solve the problem of addStateStore potentially being called twice for the same store (because more than one Supplier specifies it), the check for duplicate stores in addStateStores will be relaxed to *

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

2019-09-03 Thread Ron Dagostino
Thanks, Colin. That all makes sense, especially the part about the onerous testing requirements associated with supporting both Zookeeper and the new metadata quorum simultaneously. Given that, I now buy into the idea that the transition to the new metadata quorum becomes the main path forward on

Re: [VOTE] KIP-482: The Kafka Protocol should Support Optional Tagged Fields

2019-09-03 Thread David Jacot
+1 (non-binding) Thank for the KIP. Great addition to the Kafka protocol! Best, David Le mar. 3 sept. 2019 à 19:17, Colin McCabe a écrit : > Hi all, > > I'd like to start the vote for KIP-482: The Kafka Protocol should Support > Optional Tagged Fields. > > KIP: > https://cwiki.apache.org/confl

Re: [DISCUSS] KIP-504 - Add new Java Authorizer Interface

2019-09-03 Thread Colin McCabe
Hi Rajini, That's an interesting point. I guess I assumed that we would always cache metadata locally, so that a synchronous operation would be OK here. But, I suppose as time goes on, we will eventually want paged authorization metadata. If the operations are done asynchronously, then that i

[VOTE] KIP-482: The Kafka Protocol should Support Optional Tagged Fields

2019-09-03 Thread Colin McCabe
Hi all, I'd like to start the vote for KIP-482: The Kafka Protocol should Support Optional Tagged Fields. KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-482%3A+The+Kafka+Protocol+should+Support+Optional+Tagged+Fields Discussion thread here: https://lists.apache.org/thread.html/cdc

Re: [DISCUSS] KIP-486 Support for pluggable KeyStore and TrustStore

2019-09-03 Thread Colin McCabe
On Mon, Sep 2, 2019, at 03:33, Rajini Sivaram wrote: > I would expect SslEngineBuilder interface to look something like this, > perhaps with some tweaking: > > public interface SslEngineBuilder extends Configurable, Closeable { > > Set reconfigurableConfigs(); > > boolean shouldBeRebuilt

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

2019-09-03 Thread Colin McCabe
On Mon, Sep 2, 2019, at 07:51, Ron Dagostino wrote: > Hi Colin. It is not unusual for customers to wait before upgrading — > to avoid so-called “point-zero” releases — to avoid as many of the > inevitable bugs that ride along with new functionality as possible. > Removal of Zookeeper is going

[DISCUSS] KIP-514: Add a bounded flush() API to Kafka Producer

2019-09-03 Thread KUN DU
Hi, I would like to start discussion on KIP-514 that proposes we add a bounded flush() API to producer. Link to the KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-514%3A+Add+a+bounded+flush%28%29+API+to+Kafka+Producer Suggestions and feedback are welcome! Thanks, Kun

[jira] [Created] (KAFKA-8863) Add InsertHeader and DropHeaders connect transforms KIP-145

2019-09-03 Thread Albert Lozano (Jira)
Albert Lozano created KAFKA-8863: Summary: Add InsertHeader and DropHeaders connect transforms KIP-145 Key: KAFKA-8863 URL: https://issues.apache.org/jira/browse/KAFKA-8863 Project: Kafka Is

[jira] [Created] (KAFKA-8862) Misleading exception message for non-existant partition

2019-09-03 Thread Tom Bentley (Jira)
Tom Bentley created KAFKA-8862: -- Summary: Misleading exception message for non-existant partition Key: KAFKA-8862 URL: https://issues.apache.org/jira/browse/KAFKA-8862 Project: Kafka Issue Type:

Re: [DISCUSS] KIP-515: Enable ZK client to use the new TLS supported authentication

2019-09-03 Thread Ismael Juma
Hi Pere, Thanks for the KIP. With regards to the CLI tools, most of them support direct access to ZK for compatibility reasons and we encourage usage of the Kafka protocol instead. I am not sure we should be extending them as described in the KIP. What are your thoughts on that? Ismael On Thu, A

Re: [DISCUSS] KIP-504 - Add new Java Authorizer Interface

2019-09-03 Thread Rajini Sivaram
Hi all, Ismael brought up a point that it will be good to make the Authorizer interface asynchronous to avoid blocking request threads during remote operations. 1) Since we want to support different backends for authorization metadata, making createAcls() and deleteAcls() asynchronous makes sense

Re: [DISCUSS] KIP-373: Allow users to create delegation tokens for other users

2019-09-03 Thread Gabor Somogyi
+1 (non-binding) I've had a deeper look and this would be a good addition to Spark. On Thu, Aug 15, 2019 at 6:19 PM Viktor Somogyi-Vass wrote: > Started to implement my proposition and thought about it a little bit more > and it seems like I overthought the problem and we'd actually be better o

Re: [DISCUSS] KIP-515: Enable ZK client to use the new TLS supported authentication

2019-09-03 Thread Jorge Esteban Quilcate Otoya
Hi Pere, Have you add your KIP to the list here https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals? I found the KIP number assigned to another. On Mon, Sep 2, 2019 at 2:23 PM Pere Urbón Bayes wrote: > Thanks for your time Harsha, >anyone else with comments? loo

[jira] [Created] (KAFKA-8861) Fix flaky RegexSourceIntegrationTest.testMultipleConsumersCanReadFromPartitionedTopic

2019-09-03 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-8861: - Summary: Fix flaky RegexSourceIntegrationTest.testMultipleConsumersCanReadFromPartitionedTopic Key: KAFKA-8861 URL: https://issues.apache.org/jira/browse/KAFKA-8861

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

2019-09-03 Thread Apache Jenkins Server
See Changes: [manikumar] KAFKA-8860: Let SslPrincipalMapper split SSL principal mapping rules -- [...truncated 2.57 MB...] org.apache.kafka.streams.integration.TableTableJ

Re: [DISCUSS] KIP-511: Collect and Expose Client's Name and Version in the Brokers

2019-09-03 Thread David Jacot
Hi all, I have updated the KIP to address the various comments. I have also added a section about the handling of the ApiVersionsRequest/Response in the broker. Please, let me know what you think. I would like to make it for the next release if possible. Best, David On Fri, Aug 30, 2019 at 10:3