Re: [DISCUSS] KIP-739: Block Less on KafkaProducer#send

2021-06-02 Thread Colin McCabe
ve an opt-in way of returning an error, if people feel strongly. But honestly #2 is much less of an issue in practice. best, Colin > > Ryanne > > On Tue, Jun 1, 2021 at 12:31 PM Colin McCabe wrote: > > > On Tue, Jun 1, 2021, at 07:00, Nakamura wrote: > > > Hi Colin, &

[DISCUSS] KIP-746: Revise KRaft Metadata Records

2021-06-02 Thread Colin McCabe
Hi all, I have posted a KIP about updating the KRaft metadata records for 3.0. Check it out at : https://cwiki.apache.org/confluence/x/34zOCg best, Colin

[jira] [Created] (KAFKA-12877) Fix KRPC files with missing flexibleVersions annotation

2021-06-01 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-12877: Summary: Fix KRPC files with missing flexibleVersions annotation Key: KAFKA-12877 URL: https://issues.apache.org/jira/browse/KAFKA-12877 Project: Kafka

Re: [DISCUSS] KIP-739: Block Less on KafkaProducer#send

2021-06-01 Thread Colin McCabe
/O.) best, Colin > I think there's something I'm missing here. Would you mind helping me > figure out what it is? > > Best, > Moses > > On Sun, May 30, 2021 at 5:35 PM Colin McCabe wrote: > > > On Tue, May 25, 2021, at 11:26, Nakamura wrote: > > > Hey Colin,

Re: [DISCUSS] KIP-739: Block Less on KafkaProducer#send

2021-05-30 Thread Colin McCabe
o you describe of kafka clients producing too many messages, > as you said buffering is what should be done but I wouldn't classify this > as blocking. > > On Mon, May 24, 2021 at 7:54 PM Colin McCabe wrote: > > > Hi all, > > > > I agree that we should give use

Re: [DISCUSS] KIP-739: Block Less on KafkaProducer#send

2021-05-30 Thread Colin McCabe
ork communication in the producer and consumer. best, Colin > Best, > Moses > > On Mon, May 24, 2021 at 1:54 PM Colin McCabe wrote: > > > Hi all, > > > > I agree that we should give users the option of having a fully async API, > > but I don't think external thre

[jira] [Created] (KAFKA-12864) Move KafkaEventQueue and timeline data structures into server-common

2021-05-28 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-12864: Summary: Move KafkaEventQueue and timeline data structures into server-common Key: KAFKA-12864 URL: https://issues.apache.org/jira/browse/KAFKA-12864 Project: Kafka

[jira] [Created] (KAFKA-12853) Implement broker-side KRaft snapshots

2021-05-26 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-12853: Summary: Implement broker-side KRaft snapshots Key: KAFKA-12853 URL: https://issues.apache.org/jira/browse/KAFKA-12853 Project: Kafka Issue Type

Re: [DISCUSS] Apache Kafka 3.0.0 release

2021-05-26 Thread Colin McCabe
On Wed, May 26, 2021, at 07:18, Ismael Juma wrote: > Hi Konstantine, > > Looking at the schedule and some of the ongoing work in KIP-500 (one of the > KIPs that is important to land in 3.0), I think we'll need a bit more time. > We definitely do not want to cause a disruption to our time-based

Re: [DISCUSS] KIP-739: Block Less on KafkaProducer#send

2021-05-24 Thread Colin McCabe
Hi all, I agree that we should give users the option of having a fully async API, but I don't think external thread pools or queues are the right direction to go here. They add performance overheads and don't address the root causes of the problem. There are basically two scenarios where we

Re: [VOTE] KIP-720 Deprecate MirrorMaker v1

2021-05-24 Thread Colin McCabe
; On Fri, Apr 2, 2021 at 10:48 AM Colin McCabe wrote: > > > Hi Ryanne, > > > > Thanks for the response. It would be good to have a PR for KIP-382, I > > agree. > > > > Perhaps one possible compromise for KIP-712 would be to make the changes > > in MM2

[jira] [Created] (KAFKA-12803) Support reassigning partitions when in KRaft mode

2021-05-17 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-12803: Summary: Support reassigning partitions when in KRaft mode Key: KAFKA-12803 URL: https://issues.apache.org/jira/browse/KAFKA-12803 Project: Kafka Issue Type

[jira] [Resolved] (KAFKA-12788) Improve KRaft replica placement

2021-05-17 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12788?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-12788. -- Fix Version/s: 3.0.0 Reviewer: Jun Rao Resolution: Fixed > Improve KR

[jira] [Created] (KAFKA-12792) Fix metrics bug and introduce TimelineInteger

2021-05-16 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-12792: Summary: Fix metrics bug and introduce TimelineInteger Key: KAFKA-12792 URL: https://issues.apache.org/jira/browse/KAFKA-12792 Project: Kafka Issue Type

[jira] [Created] (KAFKA-12788) Improve KRaft replica placement

2021-05-14 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-12788: Summary: Improve KRaft replica placement Key: KAFKA-12788 URL: https://issues.apache.org/jira/browse/KAFKA-12788 Project: Kafka Issue Type: Improvement

[jira] [Resolved] (KAFKA-12471) Implement createPartitions in KIP-500 mode

2021-05-14 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12471?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-12471. -- Fix Version/s: 3.0.0 Resolution: Fixed > Implement createPartitions in KIP-500 m

[jira] [Resolved] (KAFKA-12334) Add the KIP-631 metadata shell

2021-05-14 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12334?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-12334. -- Fix Version/s: 2.8.0 Resolution: Fixed Added in 2.8 > Add the KIP-631 metadata sh

[jira] [Resolved] (KAFKA-12275) KIP-500: Remove controllerOnly restriction from the DecommissionBroker API

2021-05-14 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12275?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-12275. -- Fix Version/s: 2.8.0 Assignee: Colin McCabe Resolution: Fixed

[jira] [Created] (KAFKA-12783) Remove the deprecated ZK-based partition reassignment API

2021-05-13 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-12783: Summary: Remove the deprecated ZK-based partition reassignment API Key: KAFKA-12783 URL: https://issues.apache.org/jira/browse/KAFKA-12783 Project: Kafka

[jira] [Resolved] (KAFKA-1676) Ability to cancel replica reassignment in progress

2021-05-13 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-1676?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-1676. - Fix Version/s: 2.5.0 Assignee: Colin McCabe Resolution: Fixed > Abil

[jira] [Resolved] (KAFKA-9520) Deprecate ZooKeeper access for kafka-reassign-partitions.sh

2021-05-13 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9520?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-9520. - Fix Version/s: 2.5.0 Assignee: Colin McCabe Resolution: Fixed We deprecated ZK

[jira] [Resolved] (KAFKA-4712) kafka-reassign-partitions causes ISR to shrink

2021-05-13 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-4712?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-4712. - Resolution: Cannot Reproduce This seems to be related to overall load on the cluster as a result

[jira] [Created] (KAFKA-12778) QuorumController should honor request timeouts

2021-05-13 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-12778: Summary: QuorumController should honor request timeouts Key: KAFKA-12778 URL: https://issues.apache.org/jira/browse/KAFKA-12778 Project: Kafka Issue Type

Re: [VOTE] KIP-730: Producer ID generation in KRaft mode

2021-05-11 Thread Colin McCabe
consider this an > implementation detail and defer the decision? > > Thanks to all who have voted so far! > -David > > On Thu, May 6, 2021 at 4:01 PM Colin McCabe wrote: > > > Sorry, I meant to write "AllocateProducerIdsRecord" in the previous > > messag

Re: [VOTE] KIP-730: Producer ID generation in KRaft mode

2021-05-06 Thread Colin McCabe
Sorry, I meant to write "AllocateProducerIdsRecord" in the previous message. -C. On Thu, May 6, 2021, at 12:58, Colin McCabe wrote: > Hi David, > > Thanks for the KIP -- it looks good. > > It seems like we should be clear that the new RPC should be used for > bot

Re: [VOTE] KIP-730: Producer ID generation in KRaft mode

2021-05-06 Thread Colin McCabe
Hi David, Thanks for the KIP -- it looks good. It seems like we should be clear that the new RPC should be used for both the ZK and KRaft cases. I think that is implied, but it would be good to spell it out just to be clear. As the KIP explains, this is needed for the bridge release. I

[jira] [Created] (KAFKA-12755) Add server-common, server-tools gradle modules

2021-05-05 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-12755: Summary: Add server-common, server-tools gradle modules Key: KAFKA-12755 URL: https://issues.apache.org/jira/browse/KAFKA-12755 Project: Kafka Issue Type

[jira] [Created] (KAFKA-12733) KRaft: always bump leader epoch when changing the ISR for a controlled shutdown

2021-04-29 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-12733: Summary: KRaft: always bump leader epoch when changing the ISR for a controlled shutdown Key: KAFKA-12733 URL: https://issues.apache.org/jira/browse/KAFKA-12733

Re: [VOTE] KIP-334 Include partitions in exceptions raised during consumer record deserialization/validation

2021-04-22 Thread Colin McCabe
d `FaultyRecordException` (don't like this name > >> too much > >> > > > >>> >> honestly, but don't have a better idea for it anyway). > >> > > > >>> >> > >> > > > >>> >> Side remark. If we introduce class >

Re: Requesting review of PR 10377

2021-04-22 Thread Colin McCabe
Hi feyman2009, Thanks for the PR. I pinged a few people about reviewing it. I don't know if anyone will be able to look at it this week, but probably next week we can. The good news is that this field is optional (tagged) so the protocol will still work with older clients. best, Colin On

Re: [VOTE] 2.8.0 RC2

2021-04-17 Thread Colin McCabe
Hi John, +1 (binding) from me. I performed the following: 1. Built the project from the source code tar file, kafka-2.8.0-src.tgz 2. Ran all of the junit tests locally via "./gradlew test -PignoreFailures=true". I had one failure in SocketServerTest#testIdleConnection, which I was not able

[jira] [Resolved] (KAFKA-9119) KIP-500: Replace ZooKeeper with a Metadata Quorum

2021-04-17 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9119?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-9119. - Fix Version/s: 2.8.0 Resolution: Done > KIP-500: Replace ZooKeeper with a Metadata Quo

[jira] [Resolved] (KAFKA-9125) GroupMetadataManager and TransactionStateManager should query the controller instead of zkClient

2021-04-17 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9125?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-9125. - Fix Version/s: 2.8.0 Assignee: Ron Dagostino (was: Viktor Somogyi-Vass

[jira] [Resolved] (KAFKA-9166) Implement MetadataFetch API

2021-04-17 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9166?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-9166. - Fix Version/s: 2.8.0 Resolution: Duplicate Closing as duplicate of KAFKA-10435

[jira] [Resolved] (KAFKA-9124) KIP-497: ISR changes should be propagated via Kafka protocol

2021-04-17 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9124?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-9124. - Fix Version/s: 2.7.0 Resolution: Fixed > KIP-497: ISR changes should be propagated

[jira] [Resolved] (KAFKA-12345) KIP-500: AlterIsrManager crashes on broker idle-state

2021-04-17 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12345?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-12345. -- Fix Version/s: 2.8.0 Assignee: Boyang Chen Resolution: Fixed > KIP-

[jira] [Resolved] (KAFKA-12282) KIP-500: Reconcile configuration variables between trunk and the KIP-500 branch

2021-04-17 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12282?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-12282. -- Fix Version/s: 2.8.0 Resolution: Fixed > KIP-500: Reconcile configuration variab

[jira] [Resolved] (KAFKA-12276) Add KIP-500 controller code

2021-04-17 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12276?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-12276. -- Fix Version/s: 2.8.0 Resolution: Fixed > Add KIP-500 controller c

[jira] [Created] (KAFKA-12670) KRaft support for unclean.leader.election.enable

2021-04-14 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-12670: Summary: KRaft support for unclean.leader.election.enable Key: KAFKA-12670 URL: https://issues.apache.org/jira/browse/KAFKA-12670 Project: Kafka Issue Type

Re: [DISCUSS] KIP-729 Custom validation of records on the broker prior to log append

2021-04-06 Thread Colin McCabe
interfaces/classes. > > > > > > > > > > > https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/record/Records.java > > > and > > > > > > > > https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/a

Re: [DISCUSS] KIP-729 Custom validation of records on the broker prior to log append

2021-04-02 Thread Colin McCabe
Hi Soumyajit, I believe we've had discussions about proposals similar to this before, although I'm having trouble finding one right now. The issue here is that Record is a private class -- it is not part of any public API, and may change at any time. So we can't expose it in public APIs.

Re: [VOTE] KIP-720 Deprecate MirrorMaker v1

2021-04-02 Thread Colin McCabe
eality is that MM1 has been sort of unofficially deprecated > for a long time, so people are understandably disinterested in landing new > features there. But let's have that debate in the KIP-712 thread. I believe > we'd be having the same discussion there with or without KIP-720 passing. &

Re: [VOTE] KIP-707: The future of KafkaFuture

2021-04-01 Thread Colin McCabe
+1 (binding). Thanks for the KIP. Colin On Tue, Mar 30, 2021, at 20:36, Chia-Ping Tsai wrote: > Thanks for this KIP. +1 (binding) > > On 2021/03/29 15:34:55, Tom Bentley wrote: > > Hi, > > > > I'd like to start a vote on KIP-707, which proposes to add > > KafkaFuture.toCompletionStage(),

Re: [DISCUSS] KIP-712: Shallow Mirroring

2021-04-01 Thread Colin McCabe
On Thu, Apr 1, 2021, at 09:45, Jun Rao wrote: > Hi, Henry, > > Thanks for the response. > > 1. I agree with Tom that it's worth thinking about a separate class for > shallow iteration instead of trying to add more complexity into the > existing producer/consumer API. We could potentially make

Re: [VOTE] KIP-720 Deprecate MirrorMaker v1

2021-04-01 Thread Colin McCabe
Thanks for bringing this up, Ismael. I agree that we need to figure this out before we accept this KIP. If MM1 is deprecated, then that means we are telling users they need to migrate away from it as soon as they can. I think that rules out adding big new features to MM1, unless those

Re: [DISCUSS] KIP-723: Add socket.tcp.no.delay property to Kafka Config

2021-03-22 Thread Colin McCabe
Hi Andrei, Kafka intentionally sets TCP_NODELAY in order to disable Nagle's algorithm. The reason is because Nagle's algorithm can cause to long latencies in Kafka RPC. Wikipedia explains the problem: > [Nagle's] algorithm interacts badly with TCP delayed acknowledgments > (delayed ACK),

Re: [ANNOUNCE] New committer: Tom Bentley

2021-03-19 Thread Colin McCabe
Congratulations, Tom! Colin On Fri, Mar 19, 2021, at 08:09, Viktor Somogyi-Vass wrote: > Congrats Tom! :) > > On Wed, Mar 17, 2021 at 3:47 PM Chia-Ping Tsai wrote: > > > Congratulations!!! > > > > On 2021/03/15 17:59:56, Mickael Maison wrote: > > > Hi all, > > > > > > The PMC for Apache

[jira] [Resolved] (KAFKA-12376) Use scheduleAtomicAppend for records that need to be atomic

2021-03-18 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12376?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-12376. -- Fix Version/s: 2.8.0 Resolution: Fixed > Use scheduleAtomicAppend for reco

[jira] [Resolved] (KAFKA-12382) Create KIP-500 README for the 2.8 release

2021-03-18 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12382?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-12382. -- Fix Version/s: 2.8.0 Resolution: Fixed > Create KIP-500 README for the 2.8 rele

[jira] [Created] (KAFKA-12471) Implement createPartitions in KIP-500 mode

2021-03-15 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-12471: Summary: Implement createPartitions in KIP-500 mode Key: KAFKA-12471 URL: https://issues.apache.org/jira/browse/KAFKA-12471 Project: Kafka Issue Type: New

[jira] [Created] (KAFKA-12439) When in KIP-500 mode, we should be able to assign new partitions to nodes that are fenced

2021-03-08 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-12439: Summary: When in KIP-500 mode, we should be able to assign new partitions to nodes that are fenced Key: KAFKA-12439 URL: https://issues.apache.org/jira/browse/KAFKA-12439

[jira] [Created] (KAFKA-12432) AdminClient will not honor connection setup or request timeouts in some cases

2021-03-05 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-12432: Summary: AdminClient will not honor connection setup or request timeouts in some cases Key: KAFKA-12432 URL: https://issues.apache.org/jira/browse/KAFKA-12432

[jira] [Created] (KAFKA-12383) Get RaftClusterTest.java and other KIP-500 junit tests working

2021-02-26 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-12383: Summary: Get RaftClusterTest.java and other KIP-500 junit tests working Key: KAFKA-12383 URL: https://issues.apache.org/jira/browse/KAFKA-12383 Project: Kafka

[jira] [Created] (KAFKA-12382) Create KIP-500 README for the 2.8 rleease

2021-02-26 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-12382: Summary: Create KIP-500 README for the 2.8 rleease Key: KAFKA-12382 URL: https://issues.apache.org/jira/browse/KAFKA-12382 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-12381) Incompatible change in verifiable_producer.log in 2.8

2021-02-26 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-12381: Summary: Incompatible change in verifiable_producer.log in 2.8 Key: KAFKA-12381 URL: https://issues.apache.org/jira/browse/KAFKA-12381 Project: Kafka Issue

[jira] [Created] (KAFKA-12349) Follow up on PartitionEpoch in KIP-500

2021-02-19 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-12349: Summary: Follow up on PartitionEpoch in KIP-500 Key: KAFKA-12349 URL: https://issues.apache.org/jira/browse/KAFKA-12349 Project: Kafka Issue Type

[jira] [Created] (KAFKA-12334) Add the KIP-631 metadata shell

2021-02-17 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-12334: Summary: Add the KIP-631 metadata shell Key: KAFKA-12334 URL: https://issues.apache.org/jira/browse/KAFKA-12334 Project: Kafka Issue Type: Improvement

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2021-02-08 Thread Colin McCabe
PI that matches, > to avoid more confusion). > > Any objections? > > On Fri, Dec 18, 2020 at 10:08 AM Jun Rao wrote: > > > > Hi, Colin, > > > > Thanks for the reply. The KIP looks good to me now. Thanks for your > > diligence. > > > > Jun > >

Re: Please help | KIP-500 | KAFKA-9119

2021-02-08 Thread Colin McCabe
Hi Gaurav, There will be a preview of KIP-500 in Apache Kafka 2.8, like we discussed earlier. Hope this helps. best, Colin On Wed, Jan 27, 2021, at 22:57, gaurav chhabra wrote: > Hi Team, > > Can you please help on the timelines of the below? > > Is it confirmed for Apache Kafka version

[jira] [Created] (KAFKA-12276) Add KIP-500 controller code

2021-02-02 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-12276: Summary: Add KIP-500 controller code Key: KAFKA-12276 URL: https://issues.apache.org/jira/browse/KAFKA-12276 Project: Kafka Issue Type: Improvement

Re: We want to remove Zookeeper from our Kafka cluster which version to use?

2021-01-20 Thread Colin McCabe
Hi Manojbvn, Thanks for the question. It's good to know that people are enthusiastic about this work! We're hoping to have an alpha version of ZK-free operation in the next Apache Kafka release. The alpha version will not be suitable for production, and will not implement all Kafka

[jira] [Created] (KAFKA-12214) Generated code does not include UUID or struct fields in its toString output

2021-01-15 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-12214: Summary: Generated code does not include UUID or struct fields in its toString output Key: KAFKA-12214 URL: https://issues.apache.org/jira/browse/KAFKA-12214 Project

[jira] [Resolved] (KAFKA-12208) Rename AdminManager to ZkAdminManager

2021-01-15 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12208?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-12208. -- Fix Version/s: 2.8.0 Resolution: Fixed > Rename AdminManager to ZkAdminMana

[jira] [Created] (KAFKA-12209) Add the timeline data structures for the KIP-631 controller

2021-01-14 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-12209: Summary: Add the timeline data structures for the KIP-631 controller Key: KAFKA-12209 URL: https://issues.apache.org/jira/browse/KAFKA-12209 Project: Kafka

[jira] [Resolved] (KAFKA-12183) Add the KIP-631 metadata record definitions

2021-01-14 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12183?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-12183. -- Fix Version/s: 2.8.0 Resolution: Fixed > Add the KIP-631 metadata record definiti

[jira] [Created] (KAFKA-12208) Rename AdminManager to ZkAdminManager

2021-01-14 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-12208: Summary: Rename AdminManager to ZkAdminManager Key: KAFKA-12208 URL: https://issues.apache.org/jira/browse/KAFKA-12208 Project: Kafka Issue Type

[jira] [Created] (KAFKA-12206) o.a.k.common.Uuid should implement Comparable

2021-01-14 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-12206: Summary: o.a.k.common.Uuid should implement Comparable Key: KAFKA-12206 URL: https://issues.apache.org/jira/browse/KAFKA-12206 Project: Kafka Issue Type

[jira] [Created] (KAFKA-12183) Add the KIP-631 metadata record definitions

2021-01-12 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-12183: Summary: Add the KIP-631 metadata record definitions Key: KAFKA-12183 URL: https://issues.apache.org/jira/browse/KAFKA-12183 Project: Kafka Issue Type

[jira] [Resolved] (KAFKA-12180) Implement the KIP-631 message generator changes

2021-01-12 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12180?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-12180. -- Fix Version/s: 2.8.0 Resolution: Fixed > Implement the KIP-631 message genera

Re: [VOTE] voting on KIP-631: the quorum-based Kafka controller

2021-01-11 Thread Colin McCabe
, Colin On Tue, Jan 5, 2021, at 17:03, Colin McCabe wrote: > Hi all, > > Addendum: some of the port types in this KIP were specified as int16 in > the wire protocol. But this does not gracefully handle ports like > 33,000, which shows up as negative when using a signed 16 bit numb

[jira] [Created] (KAFKA-12180) Implement the KIP-631 message generator changes

2021-01-11 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-12180: Summary: Implement the KIP-631 message generator changes Key: KAFKA-12180 URL: https://issues.apache.org/jira/browse/KAFKA-12180 Project: Kafka Issue Type

Re: [VOTE] voting on KIP-631: the quorum-based Kafka controller

2021-01-05 Thread Colin McCabe
int32. This is consistent with what we do in MetadataResponse and a few other places. best, Colin On Mon, Dec 21, 2020, at 14:42, Colin McCabe wrote: > Hi all, > > With non-binding +1 votes from Ron Dagostino, Tom Bentley and Unmesh > Joshi, and binding +1 votes from David Arthur,

Re: [VOTE] voting on KIP-631: the quorum-based Kafka controller

2020-12-21 Thread Colin McCabe
Hi all, With non-binding +1 votes from Ron Dagostino, Tom Bentley and Unmesh Joshi, and binding +1 votes from David Arthur, Boyang Chen, Jason Gusafson, Ismael Juma, David Jacot, Jun Rao, the KIP passes. thanks, all! cheers, Colin On Fri, Dec 18, 2020, at 12:42, Colin McCabe wrote: > Hi

Re: [DISCUSS] KIP-700: Add Describe Cluster API

2020-12-18 Thread Colin McCabe
e new API and to > > add new information with separate KIPs. There might be more information > > that we want to add as part of KIP-500. > > > > I will be happy to hear what other members of the community think about > > this. > > > > Best, >

Re: [VOTE] voting on KIP-631: the quorum-based Kafka controller

2020-12-18 Thread Colin McCabe
this KIP, Colin. The KIP is really well written. This is > > so exciting! > > > > +1 (binding) > > > > Best, > > David > > > > On Wed, Dec 16, 2020 at 11:51 PM Colin McCabe wrote: > > > > > On Wed, Dec 16, 2020, at 13:08, Ismael Ju

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-12-17 Thread Colin McCabe
t; the sasl mechanism for the communication between the broker and the > controller. > Hi Jun, Yeah... sounds like we could use a new configuration key for this. I added sasl.mechanism.controller.protocol for this. regards, Colin > Jun > > On Thu, Dec 17, 2020 at 2:29 PM Colin

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-12-17 Thread Colin McCabe
d port, the client would have no way to connect since it doesn't know what security settings to use. regards, Colin > Jun > > On Wed, Dec 16, 2020 at 9:13 PM Colin McCabe wrote: > > > On Wed, Dec 16, 2020, at 18:13, Jun Rao wrote: > > > Hi, Colin, > > > &g

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-12-16 Thread Colin McCabe
on't appear in the MetadataResponses returned to clients, since clients can't access them. I should have been more clear about that in the KIP-- I added a sentence to "Networking" describing this. best, Colin > > Thanks, > > Jun > > On Wed, Dec 16, 2020 at 4:23 PM Col

Re: [DISCUSS] KIP-700: Add Describe Cluster API

2020-12-16 Thread Colin McCabe
Hi David, This seems reasonable. It would be nice to have an API specifically for describeCluster, so that we could extend this API without adding more fields to the already large MetadataRequest. As you mention in the KIP, KIP-700 would allow us to deprecate

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-12-16 Thread Colin McCabe
end to a controller that has just recently become inactive. In that case NOT_CONTROLLER would be returned. (A standby controller returns NOT_CONTROLLER for most APIs). > 221. Could we add the required ACL for the new requests? > Good point. I added the required ACL for each new RPC. be

Re: [VOTE] voting on KIP-631: the quorum-based Kafka controller

2020-12-16 Thread Colin McCabe
. > 11. Broker metrics typically have a PerSec suffix, should we stick with > that for the `MetadataCommitRate`? Added. > 12. For the lag metrics, would it be clearer if we included "Offset" in the > name? In theory, we could have time based lag metrics too. Having said > t

Re: [VOTE] voting on KIP-631: the quorum-based Kafka controller

2020-12-16 Thread Colin McCabe
t; > > > > -David > > > > > > > > > > On Sat, Dec 12, 2020 at 7:46 AM Ron Dagostino > > > wrote: > > > > > > > > > > > Thanks for shepherding this KIP through the extended discussion, > > > Co

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-12-16 Thread Colin McCabe
tells you partition X is on nodes A, B, and C. If you don't have the ZK registration for one or more of A, B, or C then you don't know whether we are following the policy of "two replicas on one rack, one replica on another." Or any other more complex rack placement policy that you might

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-12-16 Thread Colin McCabe
tup process. After the startup process is finished, it ignores this boolean. The broker uses fence / unfence records in the metadata log to determine which brokers should appear in its MetadataResponse. best, Colin > Jun > > On Tue, Dec 15, 2020 at 8:51 AM Colin McCabe wrote:

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-12-15 Thread Colin McCabe
is a good fit. So it would be good to understand about how > you're making this backward compatible for clients. > > Many thanks, > > Tom > > On Tue, Dec 15, 2020 at 1:42 AM Colin McCabe wrote: > > > On Fri, Dec 11, 2020, at 17:07, Jun Rao wrote: > >

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-12-14 Thread Colin McCabe
Heartbeat still references currentState and targetState. > Thanks. I've made these sections clearer and removed the obsolete references to sending states. best, Colin > Jun > > On Fri, Dec 11, 2020 at 1:33 PM Colin McCabe wrote: > > > On Wed, Dec 9, 2020, at 10:10, Jun Rao wro

[VOTE] voting on KIP-631: the quorum-based Kafka controller

2020-12-11 Thread Colin McCabe
Hi all, I'd like to restart the vote on KIP-631: the quorum-based Kafka Controller. The KIP is here: https://cwiki.apache.org/confluence/x/4RV4CQ The original DISCUSS thread is here:

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-12-11 Thread Colin McCabe
, but this hasn't been implemented yet. best, Colin > > Jun > > > > > On Tue, Dec 8, 2020 at 5:27 PM Colin McCabe wrote: > > > On Thu, Dec 3, 2020, at 16:37, Jun Rao wrote: > > > Hi, Colin, > > > > > > Thanks for the updated KIP.

Re: [DISCUSS] KIP-660: Pluggable ReplicaAssignor

2020-12-08 Thread Colin McCabe
ges or > streams. > Therefore I'd expect them to use these plug-in point implementations > with due care, or else they risk extinction by survival of the fittest ... > :-) > > cheers > Edoardo > > On Wed, 2 Dec 2020 at 18:46, Colin McCabe wrote: > > > Hi Mickae

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-12-08 Thread Colin McCabe
ast step, that scrubs all metadata about the topic. In order to get to that last step, the topic data needs to removed from all brokers (after each broker notices that the topic is being deleted). best, Colin > Jun > > On Wed, Dec 2, 2020 at 2:50 PM Colin McCabe wrote: > > >

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-12-02 Thread Colin McCabe
ry. Similarly with the > controller.id value on controllers -- it would allow the same type of > sanity check for quorum controllers. > That's a good point. I will add broker.id back, and also add controller.id as a possibility. cheers, Colin > > On Mon, Nov 30, 2020 at 7:41 PM C

Re: [DISCUSS] KIP-660: Pluggable ReplicaAssignor

2020-12-02 Thread Colin McCabe
Hi Mickael, To be honest, I think it would be better not to make replica placement pluggable. When I worked on the Hadoop Distributed Filesystem, we supported pluggable replica placement policies, and it never worked out well. Users would write plugin code that ran in a very sensitive

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-11-30 Thread Colin McCabe
on every log append. > Given that the new broker epoch won't be visible until the commit has happened, I have changed this to "the next available offset in the log" > 104. REGISTERING(1) : It says "Otherwise, the broker moves into the FENCED > state.". It seems

[jira] [Created] (KAFKA-10703) Document that default configs are not supported for TOPIC entities

2020-11-09 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-10703: Summary: Document that default configs are not supported for TOPIC entities Key: KAFKA-10703 URL: https://issues.apache.org/jira/browse/KAFKA-10703 Project: Kafka

Re: Is AdminClient of Kafka thread-safe?

2020-10-29 Thread Colin McCabe
Yes, KafkaAdminClient is thread-safe. best, Colin On Wed, Oct 7, 2020, at 12:38, Efe Gencer wrote: > Hi All, > > Other than a Stack Overflow comment (see > https://stackoverflow.com/a/61738065) by Colin Patrick McCabe (CC'd), > there is no source that verifies the thread-safety of

[jira] [Created] (KAFKA-10656) NetworkClient.java: print out the feature flags received at DEBUG level, as well as the other version information

2020-10-28 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-10656: Summary: NetworkClient.java: print out the feature flags received at DEBUG level, as well as the other version information Key: KAFKA-10656 URL: https://issues.apache.org/jira

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-10-23 Thread Colin McCabe
On Wed, Oct 21, 2020, at 05:51, Tom Bentley wrote: > Hi Colin, > > On Mon, Oct 19, 2020, at 08:59, Ron Dagostino wrote: > > > Hi Colin. Thanks for the hard work on this KIP. > > > > > > I have some questions about what happens to a broker when it becomes > > > fenced (e.g. because it can't send

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-10-23 Thread Colin McCabe
current plan, might it be better to > > include a Boolean value within FencedBrokerRecord and specify true > > when the broker becomes fenced and false when it is no longer fenced? > > > > Also, is a fenced broker considered Offline for partition metadata > > purposes? I

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-10-21 Thread Colin McCabe
ded a rejected alternatives section about sharing IDs between multiple nodes. Colin > Thanks, > > Jun > > On Mon, Oct 12, 2020 at 11:14 AM Colin McCabe wrote: > > > On Tue, Oct 6, 2020, at 16:09, Jun Rao wrote: > > > Hi, Colin, > > > > >

Re: [DISCUSS] KIP-631: The Quorum-based Kafka Controller

2020-10-21 Thread Colin McCabe
following > > are some potential reasons. (a) We can guard duplicated brokerID, but it's > > hard to guard against duplicated controllerId. (b) brokerID can be auto > > assigned in the future, but controllerId is hard to be generated > > automatically. > > > >

<    1   2   3   4   5   6   7   8   9   10   >