Re: [DISCUSS] KIP-378: Enable Dependency Injection for Kafka Streams handlers

2018-12-03 Thread Guozhang Wang
Hello Wladimir, I've thought about the two options and I think I'm sold on the second option and actually I think it is better generalize it to be potentially used for other clients (producer, consumer) as while since they also have similar dependency injection requests for metrics reporter, parti

KAFKA-6144(or other ticket to resolve)

2018-12-03 Thread Nikolay Izhikov
Hello, Kafka developers. I found the link to a KAFKA-6144 in youtube video [1] Gwen, Matthias, great video by the way :) I want to know - is this ticket actual for the Kafka product? Are commiters and other community members are interested in it's resolve? It unassigned for now and I want to wo

Re: [VOTE] KIP-345: Introduce static membership protocol to reduce consumer rebalances

2018-12-03 Thread Guozhang Wang
2. That's a good point. I think I'm convinced. Guozhang On Mon, Dec 3, 2018 at 5:43 PM Boyang Chen wrote: > Thanks Guozhang for the reply! > > 1. RemoveMemberFromGroupOptions seems not defined anywhere. > Added the definition. > 2. LeaveGroupRequest added a list of group instance id, but still

Re: [VOTE] KIP-345: Introduce static membership protocol to reduce consumer rebalances

2018-12-03 Thread Boyang Chen
Oops, sure thing Mayuresh :) I have only one open question for Guozhang. Will definitely move the discussion back. Boyang From: Mayuresh Gharat Sent: Tuesday, December 4, 2018 9:52 AM To: dev@kafka.apache.org Subject: Re: [VOTE] KIP-345: Introduce static members

Re: [VOTE] KIP-345: Introduce static membership protocol to reduce consumer rebalances

2018-12-03 Thread Mayuresh Gharat
Hi Folks, Would it be good to move this to the DISCUSS thread and keep this thread only for voting purposes, else it will be hard to coordinate responses between 2 threads. Thanks, Mayuresh On Mon, Dec 3, 2018 at 5:43 PM Boyang Chen wrote: > Thanks Guozhang for the reply! > > 1. RemoveMembe

[jira] [Created] (KAFKA-7705) Update javadoc for default value of delivery.timeout.ms

2018-12-03 Thread huxihx (JIRA)
huxihx created KAFKA-7705: - Summary: Update javadoc for default value of delivery.timeout.ms Key: KAFKA-7705 URL: https://issues.apache.org/jira/browse/KAFKA-7705 Project: Kafka Issue Type: Bug

Re: [VOTE] KIP-345: Introduce static membership protocol to reduce consumer rebalances

2018-12-03 Thread Boyang Chen
Thanks Guozhang for the reply! 1. RemoveMemberFromGroupOptions seems not defined anywhere. Added the definition. 2. LeaveGroupRequest added a list of group instance id, but still keep the member id as a singleton; is that intentional? I think to make the protocol consistent both member id and inst

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-12-03 Thread John Roesler
Hi Jan and Adam, Wow, thanks for doing that test, Adam. Those results are encouraging. Thanks for your performance experience as well, Jan. I agree that avoiding unnecessary join outputs is especially important when the fan-out is so high. I suppose this could also be built into the implementatio

[jira] [Created] (KAFKA-7704) kafka.server.ReplicaFetechManager.MaxLag.Replica metric is reported incorrectly

2018-12-03 Thread Yu Yang (JIRA)
Yu Yang created KAFKA-7704: -- Summary: kafka.server.ReplicaFetechManager.MaxLag.Replica metric is reported incorrectly Key: KAFKA-7704 URL: https://issues.apache.org/jira/browse/KAFKA-7704 Project: Kafka

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

2018-12-03 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-7703) KafkaConsumer.position may return a wrong offset after "seekToEnd" is called

2018-12-03 Thread Shixiong Zhu (JIRA)
Shixiong Zhu created KAFKA-7703: --- Summary: KafkaConsumer.position may return a wrong offset after "seekToEnd" is called Key: KAFKA-7703 URL: https://issues.apache.org/jira/browse/KAFKA-7703 Project: Kaf

[jira] [Created] (KAFKA-7702) Prefixed ACLs don't work with single character prefix

2018-12-03 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-7702: - Summary: Prefixed ACLs don't work with single character prefix Key: KAFKA-7702 URL: https://issues.apache.org/jira/browse/KAFKA-7702 Project: Kafka Issue T

[jira] [Created] (KAFKA-7701) Revert kafka-trunk-jdk8 CI change to re-enable testAll-based builds

2018-12-03 Thread Colin Hicks (JIRA)
Colin Hicks created KAFKA-7701: -- Summary: Revert kafka-trunk-jdk8 CI change to re-enable testAll-based builds Key: KAFKA-7701 URL: https://issues.apache.org/jira/browse/KAFKA-7701 Project: Kafka

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

2018-12-03 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-7700) AbstractConfig does not honor Properties defaults

2018-12-03 Thread Tommy Becker (JIRA)
Tommy Becker created KAFKA-7700: --- Summary: AbstractConfig does not honor Properties defaults Key: KAFKA-7700 URL: https://issues.apache.org/jira/browse/KAFKA-7700 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-7699) Improve wall-clock time punctuations

2018-12-03 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-7699: -- Summary: Improve wall-clock time punctuations Key: KAFKA-7699 URL: https://issues.apache.org/jira/browse/KAFKA-7699 Project: Kafka Issue Type: New Featur

[jira] [Created] (KAFKA-7698) Kafka Broker fail to start: ProducerFencedException thrown from producerstatemanager.scala!checkProducerEpoch

2018-12-03 Thread Ming Liu (JIRA)
Ming Liu created KAFKA-7698: --- Summary: Kafka Broker fail to start: ProducerFencedException thrown from producerstatemanager.scala!checkProducerEpoch Key: KAFKA-7698 URL: https://issues.apache.org/jira/browse/KAFKA-7698

Re: [VOTE] KIP-345: Introduce static membership protocol to reduce consumer rebalances

2018-12-03 Thread Guozhang Wang
Hello Boyang, I've browsed through the new wiki and there are still a couple of minor things to notice: 1. RemoveMemberFromGroupOptions seems not defined anywhere. 2. LeaveGroupRequest added a list of group instance id, but still keep the member id as a singleton; is that intentional? I think to

[jira] [Created] (KAFKA-7697) Possible deadlock in kafka.cluster.Partition

2018-12-03 Thread Gian Merlino (JIRA)
Gian Merlino created KAFKA-7697: --- Summary: Possible deadlock in kafka.cluster.Partition Key: KAFKA-7697 URL: https://issues.apache.org/jira/browse/KAFKA-7697 Project: Kafka Issue Type: Bug

Re: [DISCUSS] KIP-392: Allow consumers to fetch from the closest replica

2018-12-03 Thread Eno Thereska
Hi Jason, This is an interesting KIP. This will have massive implications for consistency and serialization, since currently the leader for a partition serializes requests. A few questions for now: - before we deal with the complexity, it'd be great to see a crisp example in the motivation as to

Re: [VOTE] KIP-345: Introduce static membership protocol to reduce consumer rebalances

2018-12-03 Thread Boyang Chen
Hey community friends, after another month of polishing, KIP-345 design is ready for vote. Feel free to add your comment on the discussion thread or here. Thanks for

Re: [DISCUSS] KIP-394: Require member.id for initial join group request

2018-12-03 Thread Stanislav Kozlovski
Everything sounds good to me. On Sun, Dec 2, 2018 at 1:24 PM Boyang Chen wrote: > In fact, it's probably better to move KIP-394< > https://cwiki.apache.org/confluence/display/KAFKA/KIP-394%3A+Require+member.id+for+initial+join+group+request> > to the vote stage first, so that it's easier to fina

Re: [Discuss] KIP-389: Enforce group.max.size to cap member metadata growth

2018-12-03 Thread Stanislav Kozlovski
Hi Jason, > 2. Do you think we should make this a dynamic config? I'm not sure. Looking at the config from the perspective of a prescriptive config, we may get away with not updating it dynamically. But in my opinion, it always makes sense to have a config be dynamically configurable. As long as w

Re: [DISCUSS] KIP-392: Allow consumers to fetch from the closest replica

2018-12-03 Thread Mickael Maison
Hi Jason, Very cool KIP! A couple of questions: - I'm guessing the selector will be invoke after each rebalance so every time the consumer is assigned a partition it will be able to select it. Is that true? - From the selector API, I'm not sure how the consumer will be able to address some of the

Re: [DISCUSS] KIP-392: Allow consumers to fetch from the closest replica

2018-12-03 Thread Stanislav Kozlovski
Hey Jason, This is certainly a very exciting KIP. I assume that no changes will be made to the offset commits and they will continue to be sent to the group coordinator? I also wanted to address metrics - have we considered any changes there? I imagine that it would be valuable for users to be ab

[jira] [Created] (KAFKA-7696) kafka-delegation-tokens.sh using a config file that contains security.protocol=SASL_PLAINTEXT throws OutOfMemoryError if it tries to connect to an SSL-enabled secured bro

2018-12-03 Thread Attila Sasvari (JIRA)
Attila Sasvari created KAFKA-7696: - Summary: kafka-delegation-tokens.sh using a config file that contains security.protocol=SASL_PLAINTEXT throws OutOfMemoryError if it tries to connect to an SSL-enabled secured broker Key: KAFKA

Re: [DISCUSS] KIP-201: Rationalising Policy interfaces

2018-12-03 Thread Mickael Maison
Hi Tom, This is a very interesting KIP. If you are not going to continue working on it, would it be ok for us to grab it and complete it? Thanks On Thu, Jun 14, 2018 at 7:06 PM Anna Povzner wrote: > > Hi Tom, > > Just wanted to check what you think about the comments I made in my last > message.

[jira] [Created] (KAFKA-7695) Cannot override StreamsPartitionAssignor in configuration

2018-12-03 Thread Dmitry Buykin (JIRA)
Dmitry Buykin created KAFKA-7695: Summary: Cannot override StreamsPartitionAssignor in configuration Key: KAFKA-7695 URL: https://issues.apache.org/jira/browse/KAFKA-7695 Project: Kafka Issu

Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted.

2018-12-03 Thread Jan Filipiak
On 29.11.2018 15:14, John Roesler wrote: > Hi all, > > Sorry that this discussion petered out... I think the 2.1 release caused an > extended distraction that pushed it off everyone's radar (which was > precisely Adam's concern). Personally, I've also had some extend > distractions of my own that

Jenkins build is back to normal : kafka-2.0-jdk8 #195

2018-12-03 Thread Apache Jenkins Server
See

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

2018-12-03 Thread Apache Jenkins Server
See Changes: [manikumar.reddy] MINOR: hygene cleanup in TransactionManagerTest (#5951) [manikumar.reddy] KAFKA-4544: Add system tests for delegation token based authentication [junrao] KAFKA-7235: Detect outda

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

2018-12-03 Thread Apache Jenkins Server
See Changes: [manikumar.reddy] MINOR: hygene cleanup in TransactionManagerTest (#5951) [manikumar.reddy] KAFKA-4544: Add system tests for delegation token based authentication [junrao] KAFKA-7235: Detect outda

Build failed in Jenkins: kafka-0.10.2-jdk7 #240

2018-12-03 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: improve QueryableStateIntegrationTest (#5988) -- [...truncated 326.13 KB...] kafka.server.ClientQuotaManagerTest > testExpireThrottleTimeSe

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

2018-12-03 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: improve QueryableStateIntegrationTest (#5987) -- [...truncated 459.40 KB...] kafka.server.KafkaConfigTest > testListenerAndAdvertisedListenerNa