Re: [DISCUSS] KIP-519: Make SSL context/engine configuration extensible

2019-09-09 Thread Maulin Vasavada
Hi Clement/Rajini/Colin Please review our latest updates on the KIP and let me know your thoughts. Clement, please let me know if my understanding about the "custom configs" is correct based on what I wrote in the KIP. Thanks Maulin On Mon, Sep 9, 2019 at 3:28 PM Maulin Vasavada wrote: > Hi a

[jira] [Resolved] (KAFKA-8889) Root cause is lost for FetchSessionHandler.handleError

2019-09-09 Thread Guozhang Wang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8889?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-8889. -- Resolution: Fixed > Root cause is lost for FetchSessionHandler.handleError > --

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

2019-09-09 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-8816: Make offsets immutable to users of RecordCollector.offsets -- [...truncated 2.78 MB...] kafka.zk.KafkaZkClientTest > testSetAndGetConsume

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

2019-09-09 Thread Maulin Vasavada
Thanks Clement. I"ll start accommodating your suggestions and addressing comments on the KIP-519. However, can we please start discussing on the KIP-519 discussion thread? On Mon, Sep 9, 2019 at 9:23 PM Pellerin, Clement wrote: > Please specify in the KIP if the new config is reconfigurable or

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

2019-09-09 Thread Pellerin, Clement
Please specify in the KIP if the new config is reconfigurable or not. -Original Message- From: Pellerin, Clement Sent: Monday, September 9, 2019 8:28 PM To: dev@kafka.apache.org Subject: RE: [DISCUSS] KIP-486 Support for pluggable KeyStore and TrustStore This is a good start. Please doc

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

2019-09-09 Thread Apache Jenkins Server
See

Build failed in Jenkins: kafka-2.0-jdk8 #292

2019-09-09 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-8816: Make offsets immutable to users of RecordCollector.offsets -- [...truncated 442.92 KB...] kafka.controller.PartitionStateMachineTest > t

Re: [DISCUSS] KIP-515: Reorganize checkpoint system in log cleaner to be per partition

2019-09-09 Thread Richard Yu
Hi Jun, Thanks for chipping in. :) The description you provided is pretty apt in describing the motivation of the KIP, so I will add it. I've made some changes to the KIP and outlined the basic approaches of what we have so far (basically changing the checkpoint file organization or incorporating

Re: [DISCUSS] KIP-450: Sliding Window Aggregations in the DSL

2019-09-09 Thread Guozhang Wang
Hello Matthias, Thanks for your thoughts! On Mon, Sep 9, 2019 at 6:02 PM Matthias J. Sax wrote: > From my point of view, a Tumbling/Hopping window has different semantics > than a Sliding-Window, and hence, I am not convinced atm that it's a > good idea to use 1ms-hopping-windows. > > > > (1) I

Re: [DISCUSS] KIP-450: Sliding Window Aggregations in the DSL

2019-09-09 Thread Matthias J. Sax
From my point of view, a Tumbling/Hopping window has different semantics than a Sliding-Window, and hence, I am not convinced atm that it's a good idea to use 1ms-hopping-windows. (1) I think that the window bounds are different, ie, while a time-window hast the lower-start-time as an inclusive

Re: [DISCUSS] KIP-516: Topic Identifiers

2019-09-09 Thread Ryanne Dolan
Lucas, this would be great. I've run into issues with topics being resurrected accidentally, since a client cannot easily distinguish between a deleted topic and a new topic with the same name. I'd need the ID accessible from the client to solve that issue, but this is a good first step. Ryanne O

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

2019-09-09 Thread Pellerin, Clement
This is a good start. Please document the constants for the config you are creating. You will notice you need to make the name of your default implementation public. How do the configs get into your factory instance? Is it through the constructor or you will add method arguments? Please define y

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

2019-09-09 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-8222 & KIP-345 part 5: admin request to batch remove members -- [...truncated 2.62 MB...] org.apache.kafka.streams.tools.StreamsResetterTest

Re: [VOTE] KIP-444: Refactor and Augment Metrics for Kafka Streams

2019-09-09 Thread Matthias J. Sax
+1 (binding) -Matthias On 9/5/19 11:47 AM, Guozhang Wang wrote: > +1 from myself. > > I'm now officially closing this voting thread with the following tally: > > binding +1: 3 (Guozhang, Bill, Matthias voted on the DISCUSS thread). > non-binding +1: 2 (Bruno, John). > > > Guozhang > > > On

Re: [DISCUSS] KIP-444: Augment Metrics for Kafka Streams

2019-09-09 Thread Matthias J. Sax
Works for me. (Btw: I did not really vote, just stated my overall support :P ) On 9/5/19 2:50 PM, John Roesler wrote: > Thanks, all. > > FWIW, the most recent formulation from Guozhang + Bruno's addendum would > have my support. > > Thanks, > -John > > On Thu, Sep 5, 2019 at 4:05 PM Bruno Cado

Re: [VOTE] KIP-520: Augment Consumer.committed(partition) to allow multiple partitions

2019-09-09 Thread Boyang Chen
Hey Guozhang, LGTM, +1 (non-binding) On Mon, Sep 9, 2019 at 5:07 PM Guozhang Wang wrote: > Hello folks, > > I've created a new KIP allowing consumer.committed to take a set of > partitions instead of just one partition to allow batching effects of such > requests (the protocol already allows us

Build failed in Jenkins: kafka-2.1-jdk8 #228

2019-09-09 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-8816: Make offsets immutable to users of RecordCollector.offsets -- Started by an SCM change [EnvInject] - Loading node environment variables.

Build failed in Jenkins: kafka-2.2-jdk8 #164

2019-09-09 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-8816: Make offsets immutable to users of RecordCollector.offsets -- [...truncated 69.88 KB...]

[VOTE] KIP-520: Augment Consumer.committed(partition) to allow multiple partitions

2019-09-09 Thread Guozhang Wang
Hello folks, I've created a new KIP allowing consumer.committed to take a set of partitions instead of just one partition to allow batching effects of such requests (the protocol already allows us to send multiple partitions in one round-trip): https://cwiki.apache.org/confluence/display/KAFKA/KI

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2019-09-09 Thread Boyang Chen
Thank you Jason! Addressed the comments. Thank you Guozhang for explaining. I will document the timeout setting reasoning in the design doc. On Mon, Sep 9, 2019 at 1:49 PM Guozhang Wang wrote: > On Fri, Sep 6, 2019 at 6:33 PM Boyang Chen > wrote: > > > Thanks Guozhang, I have polished the des

Re: [DISCUSS] KIP-507: Securing Internal Connect REST Endpoints

2019-09-09 Thread Randall Hauch
Thanks for putting this KIP together, Chris. It's thorough and well thought out, and you've done a great job responding to comments. It is indeed going to be nice to harden the REST API a bit more. I do have a few questions/concerns/comments, all of which I think can be incorporated relatively eas

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

2019-09-09 Thread Colin McCabe
Hi all, I agree that freezing the request header is not very appealing. We might want to add something there later. Having a separate request type is very clean, but as David mentioned, it does add an extra request/response cycle to the time required to get a connection into a usable state.

Re: [DISCUSS] KIP-491: Preferred Leader Deprioritized List (Temporary Blacklist)

2019-09-09 Thread Stanislav Kozlovski
I agree with Colin that the same result should be achievable through proper abstraction in a tool. Even if that might be "4xO(N)" operations, that is still not a lot - it is still classified as O(N) Let's say a healthy broker hosting 3000 partitions, and of which 1000 are > the preferred leaders (

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

2019-09-09 Thread Maulin Vasavada
Hi all I created a KIP-519 for pluggability of SSLContext/Engine object. Looking forward for a great discussion and feedback on the same. I'll keep KIP-486 in discussion state until we reach to some good conclusion on how to allow custom key/trust stores after KIP-519 work is done. Based on that,

[jira] [Created] (KAFKA-8890) KIP-519: Make SSL context/engine configuration extensible

2019-09-09 Thread Maulin Vasavada (Jira)
Maulin Vasavada created KAFKA-8890: -- Summary: KIP-519: Make SSL context/engine configuration extensible Key: KAFKA-8890 URL: https://issues.apache.org/jira/browse/KAFKA-8890 Project: Kafka I

[DISCUSS] KIP-519: Make SSL context/engine configuration extensible

2019-09-09 Thread Maulin Vasavada
Hi all Based on longer discussion on another KIP-486 we are opening this KIP-519 ( https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=128650952). Please help us review this and provide your suggestions. Thanks Maulin

Re: [DISCUSS] KIP-515: Reorganize checkpoint system in log cleaner to be per partition

2019-09-09 Thread Jun Rao
Hi, Richard, Thanks for drafting the KIP. A few comments below. 1. We need to provide a better motivation for the KIP. The goal of the KIP is not to reorganize the checkpoint for log cleaning. It's just an implementation detail. I was thinking that we could add sth like the following in the Motiv

[jira] [Created] (KAFKA-8889) Root cause is lost for FetchSessionHandler.handleError

2019-09-09 Thread Qinghui Xu (Jira)
Qinghui Xu created KAFKA-8889: - Summary: Root cause is lost for FetchSessionHandler.handleError Key: KAFKA-8889 URL: https://issues.apache.org/jira/browse/KAFKA-8889 Project: Kafka Issue Type: Bu

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

2019-09-09 Thread Renuka M
Hi All, Could you please take a look and let us know what you think on KIP-512 -- adding record headers to RecordMetaData. Since headers also provides metadata about the record, adding these to RecordMetaData, will allow to link record to its acknowledgement in Interceptors. The message tracing s

Re: Request to be added as a contributor

2019-09-09 Thread Jun Rao
Hi, Greg, Thanks for your interest. Just gave you jira and wiki permissions. Jun On Mon, Sep 9, 2019 at 9:50 AM Greg Harris wrote: > Hi, > > I'd like to contribute to Kafka, and would like permissions for the > following accounts: > > ASF Jira: gharris1727 > Confluence: gharris1727 > > Thanks!

Re: Kafka/Zookeeper issues

2019-09-09 Thread Jill BV
According to the console, looking at Google, it seems to not be an error, but I don't know why Kafka close down. This is the console of Zookeeper after running Kafka: th:/config/users Error:KeeperErrorCode = NodeExists for /config/users 2019-09-09 16:25:59,054 [myid:] - INFO [ProcessThread(sid:0

Kafka/Zookeeper issues

2019-09-09 Thread Jill BV
Hello, Can you help me with an issue between local Kafka and Zookeeper? in random moments when I run Zookeeper, the server starts working, and then after running Kafka, Zookeeper connection falls out, even deleting logs it continues failing. A troubleshooting that we found is to open server.prop

Re: [DISCUSS] KIP-450: Sliding Window Aggregations in the DSL

2019-09-09 Thread Guozhang Wang
Hello John, I like your idea of adding a new Combinator interface better! In addition to your arguments, we can also leverage on each overloaded function that users supplies for different aggregation implementation (i.e. if combinator is provided we can do window-slicing, otherwise we follow the c

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2019-09-09 Thread Guozhang Wang
On Fri, Sep 6, 2019 at 6:33 PM Boyang Chen wrote: > Thanks Guozhang, I have polished the design doc to make it sync with > current KIP. As for overriding default timeout values, I guess it's already > stated in the KIP to set txn timeout to 10s, are you suggesting we should > also put down this r

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2019-09-09 Thread Jason Gustafson
> I agree group.instance.id should be nullable, for member.id, are you suggesting that the purpose is to be able to set it to null when we deserialize older version txn offset request? We will still support the old sendOffsetsToTransaction API, right? In that case, we won't have a memberId. On a

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

2019-09-09 Thread Apache Jenkins Server
See Changes: [cmccabe] MINOR. implement --expose-ports option in ducker-ak (#7269) -- [...truncated 2.62 MB...] org.apache.kafka.trogdor.common.WorkerUtilsTest > testExi

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2019-09-09 Thread Boyang Chen
Thanks Jason for the vote! On Mon, Sep 9, 2019 at 12:07 PM Jason Gustafson wrote: > +1 Thanks for the KIP. Just a couple comments below: > > 1. Kafka APIs traditionally leave off `get` from API names. How about > `groupMetadata` instead of `getMetadata`? > That sounds good to me. > 2. I am gues

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

2019-09-09 Thread Apache Jenkins Server
See Changes: [gwen] KAFKA-8853; Create sustained connections test for Trogdor -- [...truncated 8.28 MB...] kafka.controller.ReplicaStateMachineTest > testInvalidNonexiste

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2019-09-09 Thread Jason Gustafson
+1 Thanks for the KIP. Just a couple comments below: 1. Kafka APIs traditionally leave off `get` from API names. How about `groupMetadata` instead of `getMetadata`? 2. I am guessing memberId and groupInstanceId should be nullable in the TxnOffsetCommit schema? 3. Just to clarify on the upgrade pro

Request to be added as a contributor

2019-09-09 Thread Greg Harris
Hi, I'd like to contribute to Kafka, and would like permissions for the following accounts: ASF Jira: gharris1727 Confluence: gharris1727 Thanks! Greg Harris

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

2019-09-09 Thread Mickael Maison
Hi, I have created a KIP to allow listing groups per state: https://cwiki.apache.org/confluence/display/KAFKA/KIP-518%3A+Allow+listing+consumer+groups+per+state Have a look and let me know what you think. Thank you

[jira] [Created] (KAFKA-8888) Possible to get null IQ value if partition migrates

2019-09-09 Thread Chris Pettitt (Jira)
Chris Pettitt created KAFKA-: Summary: Possible to get null IQ value if partition migrates Key: KAFKA- URL: https://issues.apache.org/jira/browse/KAFKA- Project: Kafka Issue Type:

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

2019-09-09 Thread Apache Jenkins Server
See Changes: [gwen] KAFKA-8853; Create sustained connections test for Trogdor -- [...truncated 3.01 MB...] org.apache.kafka.streams.internals.ApiUtilsTest > shouldThrowAr

Custom callback handler getting unexpected callback types

2019-09-09 Thread David McNelis
Hello. I’d tried posting this to the user list but had no response, so I’m x-posting here (since its probably more appropriate as a dev question). *Kafka_2.12-2.3.0 is the version I’m running and compiling for. I’m a bit confused as to some of the behavior of building a custom AuthenticateCall

[DISCUSS] KIP-399: Extend ProductionExceptionHandler to cover serialization exceptions

2019-09-09 Thread alaa
Hi Matthias, I have checked the KIP-399 and the discussion and also KIP-210. So the question we need to answer is whether it's okay to also skip writing the record in the internal topics, the current implementation of 'ProductionExceptionHandler' is applied for all topics and if we decided to

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

2019-09-09 Thread Rajini Sivaram
Hi Jun, Thanks for the response. If we use the existing purgatory implementation, we should get additional purgatory metrics for ACL updates with the new purgatory name as tag, consistent with what we have for other delayed operations. I will add these to the KIP. We also have request metrics whic

[jira] [Resolved] (KAFKA-8853) Create sustained connections test for Trogdor

2019-09-09 Thread Gwen Shapira (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8853?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira resolved KAFKA-8853. - Fix Version/s: 2.4.0 Reviewer: Stanislav Kozlovski Resolution: Fixed > Create sus

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

2019-09-09 Thread Jun Rao
Hi, Rajini, Ismael, Yes, I can see the argument for making CreateAcls/DeleteAcls async. I am ok with that if you feel the implementation is not too complicated. Should we consider adding some additional metric to reflect the portion of the time spent in waiting for the async operation to complete?

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

2019-09-09 Thread Colin McCabe
Hi all, I'd like to start the vote for KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum. The DISCUSS thread from the mailing list is here: https://lists.apache.org/thread.html/cce5313ebe72bde34bf0da3af5a1723db3ee871667b1fd8edf2ee7ab@%3Cdev.kafka.apache.org%3E The KIP is here: http

Re: [DISCUSS] KIP-491: Preferred Leader Deprioritized List (Temporary Blacklist)

2019-09-09 Thread Colin McCabe
On Sat, Sep 7, 2019, at 09:21, Harsha Chintalapani wrote: > Hi Colin, > Can you give us more details on why you don't want this to be > part of the Kafka core. You are proposing KIP-500 which will take away > zookeeper and writing this interim tools to change the zookeeper > metadata doe

Custom callback handler getting unexpected callback types

2019-09-09 Thread David McNelis
Hello. I’d tried posting this to the user list but had no response, so I’m x-posting here (since its probably more appropriate as a dev question). *Kafka_2.12-2.3.0 is the version I’m running and compiling for. I’m a bit confused as to some of the behavior of building a custom AuthenticateCall

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

2019-09-09 Thread David Jacot
Hi Gwen, The reasoning behind having the information before the Sasl authentication was to have the information for troubleshooting purposes. For instance, when there are errors in the handshake, it would be great to know if it comes from a specific buggy clients. I think we could live without tho

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

2019-09-09 Thread David Jacot
Hey, I have made a couple ot scenarios to better understand the situation, the options and their tradeoffs. Please, find (raw) my notes below. Assumptions - - AK 2.4 is released with RequestHeader v1 (KIP-482) and ApiVersionsRequest v3 (KIP-511); - Broker fails back to the latest

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

2019-09-09 Thread Ismael Juma
Hi Jun, As you say, even though the average number of operations may be low, the request rate can be high in bursts. This can overwhelm the request queue and cause an outage for no good reason. Or the backing storage system may be slow for a period of time, causing a similar issue. I've seen seve

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

2019-09-09 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Clean up partition assignment logic (#7249) -- [...truncated 8.60 MB...] org.apache.kafka.connect.converters.ByteArrayConverterTest > test

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

2019-09-09 Thread Rajini Sivaram
Hi Jun, Thanks for your note. Yes, agree that sync authorize() will be sufficient in most cases based on the numbers we expect to see in the foreseeable future. For async CreateAcls/DeleteAcls, we will need to add a callback in KafkaApis and add to purgatory if future is not complete. Since we do

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

2019-09-09 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Clean up partition assignment logic (#7249) -- [...truncated 3.08 MB...] org.apache.kafka.streams.integration.TableTableJoinIntegrationTest

Re: [DISCUSS] KIP-467: Augment ProduceResponse error messaging

2019-09-09 Thread Magnus Edenhill
And a question regarding transactions: what is the effect of a failing record on a transaction, do we rely on the application to abort the transaction as needed? This is a behavioural change since today an application can rely on the underlying idempotent producer to be sure that all messages in a