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

2019-08-02 Thread George Li
Hi Colin, Thanks for looking into this KIP.  Sorry for the late response. been busy.  If a cluster has MAMY topic partitions, moving this "blacklist" broker to the end of replica list is still a rather "big" operation, involving submitting reassignments.  The KIP-491 way of blacklist is much

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

2019-08-02 Thread Apache Jenkins Server
See Changes: [gwen] KAFKA-7800; Dynamic log levels admin API (KIP-412) -- [...truncated 2.59 MB...] org.apache.kafka.connect.json.JsonConverterTest > timestampToConnect

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

2019-08-02 Thread Colin McCabe
On Fri, Aug 2, 2019, at 16:33, Jose Armando Garcia Sancio wrote: > Thanks Colin for the detail KIP. I have a few comments and questions. > > In the KIP's Motivation and Overview you mentioned the LeaderAndIsr and > UpdateMetadata RPC. For example, "updates which the controller pushes, such > as

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

2019-08-02 Thread Guozhang Wang
For the existing releases, yes (with KIP-447 we are already going to do that anyways), for future release maybe not --- hopefully we only do such metrics refactoring once. Guozhang On Fri, Aug 2, 2019 at 3:23 PM Matthias J. Sax wrote: > Would this imply that we need to update the config in

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

2019-08-02 Thread Colin McCabe
Hi Ryanne, Good idea. I added some of this discussion to the KIP-- in particular, more about controller failover. cheers, Colin On Fri, Aug 2, 2019, at 13:28, Ryanne Dolan wrote: > Thanks Colin, that helps. Can we add some of this to the KIP? > > Ryanne > > On Fri, Aug 2, 2019 at 12:23 PM

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

2019-08-02 Thread Guozhang Wang
Thanks Boyang, I've made another pass on KIP-447 as well as https://github.com/apache/kafka/pull/7078, and have some minor comments about the proposed API: 1. it seems instead of needing the whole KafkaConsumer object, you'd only need the "ConsumerGroupMetadata", in that case can we just pass in

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

2019-08-02 Thread Jose Armando Garcia Sancio
Thanks Colin for the detail KIP. I have a few comments and questions. In the KIP's Motivation and Overview you mentioned the LeaderAndIsr and UpdateMetadata RPC. For example, "updates which the controller pushes, such as LeaderAndIsr and UpdateMetadata messages". Is your thinking that we will use

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

2019-08-02 Thread Apache Jenkins Server
See Changes: [gwen] KAFKA-7800; Dynamic log levels admin API (KIP-412) -- [...truncated 2.58 MB...] kafka.coordinator.group.GroupMetadataManagerTest > testStoreEmptyGroup

[jira] [Created] (KAFKA-8747) Flaky Test ControllerEventManagerTest#testEventQueueTime

2019-08-02 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-8747: -- Summary: Flaky Test ControllerEventManagerTest#testEventQueueTime Key: KAFKA-8747 URL: https://issues.apache.org/jira/browse/KAFKA-8747 Project: Kafka

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

2019-08-02 Thread Matthias J. Sax
Would this imply that we need to update the config in each release to add a new accepted value? -Matthias On 8/2/19 1:07 PM, Guozhang Wang wrote: > Hello Matthias, > > That's a good question. Thinking about a bit more, I'd like to propose that > we just list all the version numbers since 0.10

Re: Add to the contributor list

2019-08-02 Thread Guozhang Wang
Hi Vinoth, You've been added to the list, cheers! Guozhang On Fri, Aug 2, 2019 at 2:50 PM Vinoth Chandar wrote: > I am interested in picking up KAFKA-7149 > Can I be added to the list? My jira id : vinoth > -- -- Guozhang

Add to the contributor list

2019-08-02 Thread Vinoth Chandar
I am interested in picking up KAFKA-7149 Can I be added to the list? My jira id : vinoth

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

2019-08-02 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-8578: Add basic functionality to expose RocksDB metrics (#6979) -- [...truncated 2.58 MB...] org.apache.kafka.trogdor.workload.ThrottleTest

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

2019-08-02 Thread Ryanne Dolan
Thanks Colin, that helps. Can we add some of this to the KIP? Ryanne On Fri, Aug 2, 2019 at 12:23 PM Colin McCabe wrote: > On Fri, Aug 2, 2019, at 07:50, Ryanne Dolan wrote: > > Thanks Colin, interesting KIP. > > > > I'm concerned that the KIP does not actually address its stated > >

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

2019-08-02 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-8578: Add basic functionality to expose RocksDB metrics (#6979) -- [...truncated 2.58 MB...]

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

2019-08-02 Thread Guozhang Wang
Hello Matthias, That's a good question. Thinking about a bit more, I'd like to propose that we just list all the version numbers since 0.10 to 2.4 as accepted values, and let Streams to decide if old / new set of metrics can be used internally (implementation wise we can reuse the const values

Re: [VOTE] KIP-479 Add Materialized to Join

2019-08-02 Thread Bill Bejeck
+1 (binding) from myself. This vote has been open for 7 days now. so I'm closing this vote thread. KIP-479 had the following votes: binding +1s: 3 (Guozhang, Matthias, and Bill) -1 votes: none Thanks to everyone who voted and participated in the discussion for this KIP! -Bill On Mon, Jul

[jira] [Created] (KAFKA-8746) Kibosh must handle an empty JSON string from Trogdor

2019-08-02 Thread Colin P. McCabe (JIRA)
Colin P. McCabe created KAFKA-8746: -- Summary: Kibosh must handle an empty JSON string from Trogdor Key: KAFKA-8746 URL: https://issues.apache.org/jira/browse/KAFKA-8746 Project: Kafka Issue

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

2019-08-02 Thread Colin McCabe
On Fri, Aug 2, 2019, at 07:50, Ryanne Dolan wrote: > Thanks Colin, interesting KIP. > > I'm concerned that the KIP does not actually address its stated > motivations. In particular, "Simpler Deployment and Configuration" are not > really achieved, given that: 1) the proposal still requires

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

2019-08-02 Thread Guozhang Wang
Hello folks, Just a quick update on this KIP that, after we've started implementing it, we realized it would be helpful to also add a few more broker-side metrics for user visibility of different types of invalid record errors on log validator. The wiki page has been updated accordingly with the

Re: [VOTE] KIP-455: Create an Administrative API for Replica Reassignment

2019-08-02 Thread Jun Rao
Hi, Colin, First, since we are changing the format of LeaderAndIsrRequest, which is an inter broker request, it seems that we will need IBP during rolling upgrade. Could we add that to the compatibility section? Regarding UnsupportedVersionException, even without ZK node version bump, we

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

2019-08-02 Thread Apache Jenkins Server
See Changes: [github] MINOR: Update docs to reflect the ZK 3.5.5 upgrade (#7149) -- [...truncated 2.58 MB...]

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

2019-08-02 Thread Apache Jenkins Server
See Changes: [github] MINOR: Update docs to reflect the ZK 3.5.5 upgrade (#7149) -- [...truncated 2.57 MB...] org.apache.kafka.streams.scala.kstream.SuppressedTest >

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

2019-08-02 Thread Ryanne Dolan
Thanks Colin, interesting KIP. I'm concerned that the KIP does not actually address its stated motivations. In particular, "Simpler Deployment and Configuration" are not really achieved, given that: 1) the proposal still requires quorums (now of controllers, instead of ZK nodes), with the same

Re: [DISCUSS] KIP-499 - Unify connection name flag for command line tool

2019-08-02 Thread Dongjin Lee
Hello Mitchel, Thanks for the KIP. Sure, This inconsistency is really annoying and causing lots of confusions. Here are some comments: First, there is already a Jira issue about this problem, created by Ismael. https://issues.apache.org/jira/browse/KAFKA-8507 I added the link to the KIP. Add to

Re: Dynamic configuration of interbroker SSL certificates

2019-08-02 Thread Rajini Sivaram
Hi Michael, Thanks for the logs. When keystore is reconfigured in a stable cluster, we wouldn't expect to see any failures even if there is an issue with the certs since existing connections continue to work without re-authenticating. When a broker is restarted, there could be controller