Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-18 Thread Lucas Wang
Hi Dong, Sure. Regarding the 2nd case you mentioned "- If the controller has not received response for R1 before it is disconnected, it will re-send R1 followed by R2 after it is re-connected to the broker." with the max inflight request set to 1, after the connection is re-established, the

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-18 Thread Dong Lin
Hey Lucas, I think for now we can probably discuss based on the existing Kafka's design where controller to a broker is hard coded to be 1. It looks like Becket has provided a good example in which requests from the same controller can be processed out of order. Thanks, Dong On Wed, Jul 18,

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-18 Thread Dong Lin
Hey Becket, Sorry I misunderstood your example. I thought you mean requests from different controller are re-ordered. I think you have provided a very good example and it should be safer to still use two queues. Let me clarify the example a bit more below. - If the controller has received

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-18 Thread Lucas Wang
@Becket and Dong, I think currently the ordering guarantee is achieved because the max inflight request from the controller to a broker is hard coded to be 1. If let's hypothetically say the max inflight requests is > 1, then I think Dong is right to say that even the separate queue cannot

Re: [Discuss] KIP-321: Add method to get TopicNameExtractor in TopologyDescription

2018-07-18 Thread Nishanth Pradeep
I have revised the KIP . Here is a summary of the changes. 1. Changed return type from String to Set for Source#topics(). Set Source#topics() 2. Added

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

2018-07-18 Thread Apache Jenkins Server
See Changes: [rajinisivaram] MINOR: Remove references to version 1.2 in docs (#5386) [matthias] MINOR: improve docs version numbers (#5372) -- [...truncated 2.47 MB...]

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-18 Thread Dong Lin
Hey Becket, It seems that the requests from the old controller will be discarded due to old controller epoch. It is not clear whether this is a problem. And if this out-of-order processing of controller requests is a problem, it seems like an existing problem which also applies to the

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-18 Thread Becket Qin
Hi Mayuresh/Joel, Using the request channel as a dequeue was bright up some time ago when we initially thinking of prioritizing the request. The concern was that the controller requests are supposed to be processed in order. If we can ensure that there is one controller request in the request

Build failed in Jenkins: kafka-1.0-jdk7 #221

2018-07-18 Thread Apache Jenkins Server
See Changes: [github] MINOR: improve docs version numbers (#5372) -- [...truncated 379.25 KB...] kafka.admin.AdminTest > testPartitionReassignmentWithLeaderInNewReplicas

KAFKA-7137 - max time guarantees of compaction for GDPR et al

2018-07-18 Thread Brett Rann
re: https://issues.apache.org/jira/browse/KAFKA-7137 My team is investigating what would be involved in code changes to give some finer control over when compaction runs. Detail in the ticket but essentially the current way is that dirty.ratio is hijacked to be set to 0 to give guarantees that a

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

2018-07-18 Thread Apache Jenkins Server
See Changes: [github] MINOR: remove unused MeteredKeyValueStore (#5380) [github] MINOR: Remove references to version 1.2 in docs (#5386) [matthias] MINOR: improve docs version numbers (#5372)

Re: [VOTE] KIP-338 Support to exclude the internal topics in kafka-topics.sh command

2018-07-18 Thread Jason Gustafson
> > (just a double check) the option name used in kip-338 is > "--exclude-internal" rather than "--exclude-internal-topics". Do you prefer > "--exclude-internal-topics"? Nah, just a typo on my part. I think --exclude-internal is clear enough given the name of the tool. Sorry for the confusion.

Re: [DISCUSS] KIP-341: Update Sticky Assignor's User Data Protocol

2018-07-18 Thread Vahid S Hashemian
The KIP is updated to follow the suggestion of using consumer group generation. --Vahid From: "Vahid S Hashemian" To: dev@kafka.apache.org Date: 07/17/2018 02:32 PM Subject:Re: [DISCUSS] KIP-341: Update Sticky Assignor's User Data Protocol Hi Jason, Thanks for the

Re: [DISCUSS] KIP-342 Add Customizable SASL extensions to OAuthBearer authentication

2018-07-18 Thread Stanislav Kozlovski
Hey Ron, You brought up some great points. I did my best to address them and updated the KIP. I should mention that I used commas to separate extensions in the protocol, because we did not use the recommended Control-A character for separators in the OAuth message and I figured I would not

[jira] [Created] (KAFKA-7183) Add a trogdor test that creates many connections to brokers

2018-07-18 Thread Colin P. McCabe (JIRA)
Colin P. McCabe created KAFKA-7183: -- Summary: Add a trogdor test that creates many connections to brokers Key: KAFKA-7183 URL: https://issues.apache.org/jira/browse/KAFKA-7183 Project: Kafka

Build failed in Jenkins: kafka-trunk-jdk10 #289

2018-07-18 Thread Apache Jenkins Server
See Changes: [github] KAFKA-7168: Treat connection close during SSL handshake as retriable [github] MINOR: remove unused MeteredKeyValueStore (#5380) [github] MINOR: Remove references to version 1.2 in docs

[jira] [Created] (KAFKA-7182) SASL/OAUTHBEARER client response is missing %x01 separators

2018-07-18 Thread Ron Dagostino (JIRA)
Ron Dagostino created KAFKA-7182: Summary: SASL/OAUTHBEARER client response is missing %x01 separators Key: KAFKA-7182 URL: https://issues.apache.org/jira/browse/KAFKA-7182 Project: Kafka

[jira] [Created] (KAFKA-7181) Kafka Streams State stuck in rebalancing after one of the StreamThread encounters IllegalStateException

2018-07-18 Thread Romil Kumar Vasani (JIRA)
Romil Kumar Vasani created KAFKA-7181: - Summary: Kafka Streams State stuck in rebalancing after one of the StreamThread encounters IllegalStateException Key: KAFKA-7181 URL:

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-18 Thread Joel Koshy
@Mayuresh - I like your idea. It appears to be a simpler less invasive alternative and it should work. Jun/Becket/others, do you see any pitfalls with this approach? On Wed, Jul 18, 2018 at 12:03 PM, Lucas Wang wrote: > @Mayuresh, > That's a very interesting idea that I haven't thought before.

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-18 Thread Lucas Wang
@Mayuresh, That's a very interesting idea that I haven't thought before. It seems to solve our problem at hand pretty well, and also avoids the need to have a new size metric and capacity config for the controller request queue. In fact, if we were to adopt this design, there is no public

Re: [kafka-clients] Re: [VOTE] 1.1.1 RC3

2018-07-18 Thread Matthias J. Sax
Thanks Dong. I am a little late, but +1, too. - verified signatures - build from sources - run unit test suite - run streams quickstart Thanks for running the release! -Matthias On 7/18/18 10:24 AM, Dong Lin wrote: > Thank you all for taking time to certify and vote for the release! > >

Re: [kafka-clients] Re: [VOTE] 1.1.1 RC3

2018-07-18 Thread Dong Lin
Thank you all for taking time to certify and vote for the release! This vote has passed with 10 +1 votes (3 bindings) and no 0 or -1 votes. +1 vote from PMC Members: - Jason Gustafson - Rajini Sivaram - Ismael Juma +1 vote from Committers: - Sriharsha Chintalapani - Dong Lin +1 vote from

[jira] [Created] (KAFKA-7180) In testHWCheckpointWithFailuresSingleLogSegment, wait until server1 has joined the ISR before shutting down server2

2018-07-18 Thread Lucas Wang (JIRA)
Lucas Wang created KAFKA-7180: - Summary: In testHWCheckpointWithFailuresSingleLogSegment, wait until server1 has joined the ISR before shutting down server2 Key: KAFKA-7180 URL:

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-18 Thread Mayuresh Gharat
Hi Lucas, Seems like the main intent here is to prioritize the controller request over any other requests. In that case, we can change the request queue to a dequeue, where you always insert the normal requests (produce, consume,..etc) to the end of the dequeue, but if its a controller request,

Re: [DISCUSS] KIP-342 Add Customizable SASL extensions to OAuthBearer authentication

2018-07-18 Thread Ron Dagostino
Hi Stanislav. Could you add something to the KIP about the security implications related to the CSV name/value pairs sent in the extension? For example, the OAuth access token may have a digital signature, but the extensions generally will not (unless one of the values is a JWS compact

Re: KAFKA-6690 Priorities for Source Topics

2018-07-18 Thread Stanislav Kozlovski
Hey Nick, This seems like a non-trivial change and more importantly - a change in the public interface. As such, following our contributing guidelines , it warrants a KIP (Kafka

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

2018-07-18 Thread Apache Jenkins Server
See Changes: [harsha] MINOR: Updated Quickstart to mention log.dirs (#5361) [github] KAFKA-7168: Treat connection close during SSL handshake as retriable --

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

2018-07-18 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: web docs fixes on message header (#5381) [rajinisivaram] MINOR: Updated Quickstart to mention log.dirs (#5361) [rajinisivaram] KAFKA-7168: Treat connection close during SSL handshake

Re: [VOTE] 2.0.0 RC2

2018-07-18 Thread Ismael Juma
Thanks Rajini! A documentation issue that we must fix before the release (but does not require another RC), 1.2 (which became 2.0) is mentioned in the upgrade notes: http://kafka.apache.org/20/documentation.html#upgrade Ismael On Sun, Jul 15, 2018 at 9:25 AM Rajini Sivaram wrote: > Hi Ismael,

Re: [VOTE] 1.1.1 RC3

2018-07-18 Thread Ismael Juma
+1 (binding) Verified signature of source artifact, ran tests and verified quickstart on source artifact with Java 8, verified quickstart on binary artifact (Scala 2.12) with Java 8, sanity checked release notes and Maven staging repository. Thanks for managing the release Dong! Ismael On Sun,

[jira] [Created] (KAFKA-7179) No response to INIT_PRODUCER_ID request on handler exception

2018-07-18 Thread Bastien Bouclet (JIRA)
Bastien Bouclet created KAFKA-7179: -- Summary: No response to INIT_PRODUCER_ID request on handler exception Key: KAFKA-7179 URL: https://issues.apache.org/jira/browse/KAFKA-7179 Project: Kafka

KAFKA-6690 Priorities for Source Topics

2018-07-18 Thread Nick
Hi all, I picked up KAFKA-6690 Priorities for Source Topics. https://issues.apache.org/jira/browse/KAFKA-6690 In the ticket I made some notes about extending the API to incorporate topic priorities. For implementation, I was thinking of filtering out lower priority topics in method

[jira] [Resolved] (KAFKA-7168) Broker shutdown during SSL handshake may be handled as handshake failure

2018-07-18 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7168?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-7168. --- Resolution: Fixed Fix Version/s: 2.1.0 2.0.1 > Broker shutdown

Request for review of pull request

2018-07-18 Thread David Nies
Dear Kafka developers, my company is using Kafka for quite some time now. Lately we’ve come across an issue that caused some headache. I’ve tried to mitigate this problem with the changed in pull request #5365 which I’m kindly asking you to have a

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-18 Thread Becket Qin
Hey Joel, Thank for the detail explanation. I agree the current design makes sense. My confusion is about whether the new config for the controller queue capacity is necessary. I cannot think of a case in which users would change it. Thanks, Jiangjie (Becket) Qin On Wed, Jul 18, 2018 at 6:00

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-18 Thread Becket Qin
Hi Lucas, I guess my question can be rephrased to "do we expect user to ever change the controller request queue capacity"? If we agree that 20 is already a very generous default number and we do not expect user to change it, is it still necessary to expose this as a config? Thanks, Jiangjie

Build failed in Jenkins: kafka-trunk-jdk10 #288

2018-07-18 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Fix broken Javadoc on [AbstractIndex|OffsetIndex] (#5370) [github] MINOR: web docs fixes on message header (#5381) [harsha] MINOR: Updated Quickstart to mention log.dirs (#5361)

Re: [VOTE] 1.1.1 RC3

2018-07-18 Thread Rajini Sivaram
Hi Dong, +1 (binding) Verified binary using quick start, ran tests from source, checked release notes. Thanks for running the release! Regards, Rajini On Tue, Jul 17, 2018 at 9:19 PM, Jason Gustafson wrote: > +1 > > - Did basic quickstarts on the 2.11 and 2.12 artifacts > - Checked

[jira] [Created] (KAFKA-7178) Is kafka compatible with zookeeper 3.5.x

2018-07-18 Thread fwq (JIRA)
fwq created KAFKA-7178: -- Summary: Is kafka compatible with zookeeper 3.5.x Key: KAFKA-7178 URL: https://issues.apache.org/jira/browse/KAFKA-7178 Project: Kafka Issue Type: Improvement

Re: [DISCUSS] KIP-333 Consider a faster form of rebalancing

2018-07-18 Thread Richard Yu
Hi Becket, I made some changes and clarified the motivation for this KIP. :)It should be easier to understand now since I included a diagram. Thanks,Richard Yu On Tuesday, July 17, 2018, 4:38:11 PM GMT+8, Richard Yu wrote: Hi Becket, Thanks for reviewing this KIP. :) I probably did