Re: [DISCUSS] Apache Kafka 3.0.0 release plan with new updated dates

2021-06-14 Thread Konstantine Karantasis
Done. Moved it into the table of Adopted KIPs targeting 3.0.0 and to the release plan of course. Thanks for catching this Israel. Best, Konstantine On Mon, Jun 14, 2021 at 7:40 PM Israel Ekpo wrote: > Konstantine, > > One of mine is missing from this list > > KIP-633: Drop 24 hour default of

Re: [DISCUSS] KIP-747 Add support for basic aggregation APIs

2021-06-14 Thread Matthias J. Sax
Hi, I think extending min/max to non-numeric types makes sense. Wondering why we should require a `Comparator` or if we should require that the types implement `Comparable` instead? I also think, that min/max should not change the value type. Using `Long` for sum() make sense though, and also to

Re: Adding @NotNull annotation to public APIs, KIP needed or not?

2021-06-14 Thread Matthias J. Sax
Personally, I think it might make sense to use annotations. And I agree, that we should have proper null-checks in place anyway, so existing code should not break. But I don't feel strong about it either -- not sure if some people might have concerns? In the end, a KIP sound appropriate though.

Re: [DISCUSS] Apache Kafka 3.0.0 release plan with new updated dates

2021-06-14 Thread Israel Ekpo
Konstantine, One of mine is missing from this list KIP-633: Drop 24 hour default of grace period in Streams Please could you include it? Voting has already concluded a long time ago On Mon, Jun 14, 2021 at 6:08 PM Konstantine Karantasis wrote: > Hi all. > > KIP Freeze for the next major

Re: [VOTE] KIP-748: Add Broker Count Metrics

2021-06-14 Thread Israel Ekpo
Xavier, I just responded. On Mon, Jun 14, 2021 at 1:58 PM Xavier Léauté wrote: > I had one quick question in the discussion thread. Any chance we can > provide some thought there? > > On Thu, Jun 10, 2021 at 9:46 AM Ryan Dielhenn > wrote: > > > Hello, > > > > I would like to start a vote on

Re: [DISCUSS] KIP-748: Add Broker Count Metrics

2021-06-14 Thread Israel Ekpo
Ryan, I think you can add a section under Rejected Alternatives to elaborate on why you feel combining the metrics for KRaft mode and legacy mode is not a good idea. That could help clarify future questions such as the one raised by Xavier. On Mon, Jun 14, 2021 at 10:20 PM Israel Ekpo wrote:

Re: [DISCUSS] KIP-748: Add Broker Count Metrics

2021-06-14 Thread Israel Ekpo
Xavier, I think the reason for doing this is to make them independent so that it is easier to design and implement the tracking for legacy mode (with Zookeeper) and KRaft Mode (without ZK) That is my assessment. On Mon, Jun 14, 2021 at 1:57 PM Xavier Léauté wrote: > Any reason we need two

Re: [VOTE] New Kafka PMC member: Konstantine Karantasis

2021-06-14 Thread Israel Ekpo
I figured but still wanted to congratulate him. On Mon, Jun 14, 2021 at 5:12 PM Bill Bejeck wrote: > This email in the thread is not an announcement and was sent prematurely to > the dev list. > > Bill > > On Mon, Jun 14, 2021 at 1:36 PM Bill Bejeck wrote: > > > I'm a +1 as well > > > > Bill >

Re: [DISCUSS] KIP-714: Client metrics and observability

2021-06-14 Thread Ryanne Dolan
Magnus, I think such a substantial change requires more motivation than is currently provided. As I read it, the motivation boils down to this: you want your clients to phone-home unless they opt-out. As stated in the KIP, "there are plenty of existing solutions [...] to send metrics [...] to a

[jira] [Created] (KAFKA-12949) TestRaftServer's scala.MatchError: null on test-kraft-server-start.sh

2021-06-14 Thread Ignacio Acuna (Jira)
Ignacio Acuna created KAFKA-12949: - Summary: TestRaftServer's scala.MatchError: null on test-kraft-server-start.sh Key: KAFKA-12949 URL: https://issues.apache.org/jira/browse/KAFKA-12949 Project:

Re: [DISCUSS] Apache Kafka 3.0.0 release plan with new updated dates

2021-06-14 Thread Konstantine Karantasis
Hi all. KIP Freeze for the next major release of Apache Kafka was reached last week. As of now, 36 KIPs have concluded their voting process and have been adopted. These KIPs are targeting 3.0 (unless it's noted otherwise in the release plan) and their inclusion as new features will be finalized

Re: [VOTE] New Kafka PMC member: Konstantine Karantasis

2021-06-14 Thread Bill Bejeck
This email in the thread is not an announcement and was sent prematurely to the dev list. Bill On Mon, Jun 14, 2021 at 1:36 PM Bill Bejeck wrote: > I'm a +1 as well > > Bill > > >> >>

[jira] [Created] (KAFKA-12948) NetworkClient.close(node) with node in connecting state makes NetworkClient unusable

2021-06-14 Thread Rajini Sivaram (Jira)
Rajini Sivaram created KAFKA-12948: -- Summary: NetworkClient.close(node) with node in connecting state makes NetworkClient unusable Key: KAFKA-12948 URL: https://issues.apache.org/jira/browse/KAFKA-12948

Re: [VOTE] New Kafka PMC member: Konstantine Karantasis

2021-06-14 Thread Israel Ekpo
Congratulations Konstantine! Thank you for your service and contributions. If I could vote, I would be a +1 as well :) On Mon, Jun 14, 2021 at 1:36 PM Bill Bejeck wrote: > I'm a +1 as well > > Bill > > On Mon, Jun 14, 2021 at 1:33 PM Randall Hauch wrote: > > > +1. He's been a very valuable

Re: [DISCUSS] KIP-714: Client metrics and observability

2021-06-14 Thread Travis Bischel
Apologies for this duplicate reply, I did not notice the success confirmation on the first submission. On 2021/06/14 04:52:11, Travis Bischel wrote: > Hi! I have a few thoughts on this KIP. First, I'd like to thank you for your > work > and writeup, it's clear that a lot of thought went into

Re: [VOTE] KIP-748: Add Broker Count Metrics

2021-06-14 Thread Xavier Léauté
I had one quick question in the discussion thread. Any chance we can provide some thought there? On Thu, Jun 10, 2021 at 9:46 AM Ryan Dielhenn wrote: > Hello, > > I would like to start a vote on KIP-748: Add Broker Count Metrics. > > Here is the KIP: > >

Re: [DISCUSS] KIP-748: Add Broker Count Metrics

2021-06-14 Thread Xavier Léauté
Any reason we need two different metrics for ZK an Quorum based controllers? Wouldn't it make sense to have one metric that abstracts the implementation detail? On Mon, Jun 7, 2021 at 2:29 PM Ryan Dielhenn wrote: > Hey Colin and David, > > I added another table for the ZK version of

[jira] [Created] (KAFKA-12947) Replace EasyMock and PowerMock with Mockito for StreamsMetricsImplTest ...

2021-06-14 Thread YI-CHEN WANG (Jira)
YI-CHEN WANG created KAFKA-12947: Summary: Replace EasyMock and PowerMock with Mockito for StreamsMetricsImplTest ... Key: KAFKA-12947 URL: https://issues.apache.org/jira/browse/KAFKA-12947 Project:

Re: [VOTE] New Kafka PMC member: Konstantine Karantasis

2021-06-14 Thread Bill Bejeck
I'm a +1 as well Bill On Mon, Jun 14, 2021 at 1:33 PM Randall Hauch wrote: > +1. He's been a very valuable contributor and committer. > > Randall > > On Sun, Jun 13, 2021 at 10:40 PM Ismael Juma wrote: > > > I think it's worth pinging people once before closing the vote. I'm +1. > > > >

Re: [VOTE] KIP-716: Allow configuring the location of the offset-syncs topic with MirrorMaker2

2021-06-14 Thread Mickael Maison
I'm +1 (binding) too I'm closing the vote with: - 3 +1 (binding) votes from Tom, Konstantine and myself - 3 +1 (non-binding) votes from Ryanne, Igor and Omnia Thanks for the feedback and votes On Mon, Jun 14, 2021 at 5:57 PM Omnia Ibrahim wrote: > > +1 (non-binding) thanks > > On Mon, Jun 14,

Re: [idea] Kafka topic metadata

2021-06-14 Thread Israel Ekpo
Sorry Ivan and Garmes, I misunderstood the suggestion earlier. I think this will be a great idea for a KIP. https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals You were referring to metadata for the actual topic and not its contents. Sorry about that confusion. On

Re: [idea] Kafka topic metadata

2021-06-14 Thread Ivan Yurchenko
Hi, Having metadata for topics seems pretty useful. Currently, one has to use external storage for this (e.g. a database) and the question of keeping topic and metadata in sync exists: A topic is deleted, how to delete its metadata? How to deal with delete-then-recreate scenarios (well, we have

Re: [VOTE] KIP-716: Allow configuring the location of the offset-syncs topic with MirrorMaker2

2021-06-14 Thread Omnia Ibrahim
+1 (non-binding) thanks On Mon, Jun 14, 2021 at 2:35 PM Igor Soarez wrote: > Thanks for the KIP Mickael. > > Makes sense. +1 (non-binding) > > -- > Igor > >

Re: [idea] Kafka topic metadata

2021-06-14 Thread Israel Ekpo
Garmes, I had similar questions in the past but @Matthias J. Sax pointed me to this https://cwiki.apache.org/confluence/display/KAFKA/KIP-244%3A+Add+Record+Header+support+to+Kafka+Streams+Processor+API With the headers, you can filter based on the header content and not just the contents of

Re: [DISCUSS] KIP-690 Add additional configuration to control MirrorMaker 2 internal topics naming convention

2021-06-14 Thread Omnia Ibrahim
Hi folks, let me try to clarify some of your concerns and questions. Mickael: Have you considered making names changeable via configurations? > Gwen: may be missing something, but we are looking at 3 new configs (one > for each topic). And this rejected alternative is basically identical to >

Re: [VOTE] KIP-752: Support --bootstrap-server in ReplicaVerificationTool

2021-06-14 Thread Guozhang Wang
If we believe this tool does not work anymore and there's other ways to achieve the intended function, then we should remove it in the next release; otherwise, I think this KIP still is worthy. In any ways, we should not left a cmd tool not maintained but not removed either. Guozhang On Thu, Jun

Re: Request for contributor permission

2021-06-14 Thread Guozhang Wang
Hello Nicolas, I've added you to the contributors list. You should be able to assign tickets to yourself now. Guozhang On Mon, Jun 14, 2021 at 7:24 AM Nicolas Guignard < nicolas.guignar...@gmail.com> wrote: > Hi, > > I am sending this email to ask to have the contributor's permission in >

Re: [DISCUSS] KIP-714: Client metrics and observability

2021-06-14 Thread Travis Bischel
Hi! I have a few thoughts on this KIP. First, I'd like to thank you for the writeup, clearly a lot of thought has gone into it and it is very thorough. However, I'm not convinced it's the right approach from a fundamental level. Fundamentally, this KIP seems like somewhat of a solution to an

Re: [DISCUSS] KIP-714: Client metrics and observability

2021-06-14 Thread Travis Bischel
Hi! I have a few thoughts on this KIP. First, I'd like to thank you for your work and writeup, it's clear that a lot of thought went into this and it's very thorough! However, I'm not convinced it's the right approach from a fundamental level. Fundamentally, this KIP seems like somewhat of a

Request for contributor permission

2021-06-14 Thread Nicolas Guignard
Hi, I am sending this email to ask to have the contributor's permission in order to be able to assign Jiras to me. My Jira username is: Nicolas Guignard. Is this all you need or do you need something else? Thanks in advance. Have a good day, Cheers, Nicolas Guignard -- Software engineer

Re: MM2 taking into consideration automatic topic creation property from original cluster

2021-06-14 Thread David Beech
Hi Omnia Maybe - I'll look forward to reading the KIP when it's submitted. When I researched the feature request from our client I found KIP-158 which added a bunch of topic auto-creation controls to the Kafka Connect framework in version 2.6. It seemed to me like a convenient solution would be

Re: [VOTE] KIP-724: Drop support for message formats v0 and v1

2021-06-14 Thread Ismael Juma
Thanks for the votes and discussion. The KIP vote passes with: 5 binding +1s: * Jason G * Colin M * Gwen S * David J * me 1 non-binding +1: * Dongjin Ismael On Wed, Jun 9, 2021 at 11:28 AM Ismael Juma wrote: > Hi all, > > Consensus was reached in the discussion thread and part of what is >

Re: [VOTE] KIP-716: Allow configuring the location of the offset-syncs topic with MirrorMaker2

2021-06-14 Thread Igor Soarez
Thanks for the KIP Mickael. Makes sense. +1 (non-binding) -- Igor

Re: MM2 taking into consideration automatic topic creation property from original cluster

2021-06-14 Thread Omnia Ibrahim
Hi, I am in the middle of writing a new KIP to introduced a new interface for topic updates/creation/description that will allow MM2 to either uses the default behaviour which uses Kafka AdminClient to create/update topics or to use a customised one that integrates with the user's ecosystem where

Re: MM2 taking into consideration automatic topic creation property from original cluster

2021-06-14 Thread Igor Soarez
Maybe it would be nice if it was possible to hook into or extend Admin client interactions, to allow for custom logic supporting use cases such as this. Scenarios where topic/resource management is centralized are probably not uncommon. -- Igor On Sat, Jun 12, 2021, at 9:42 AM, Matthew de

[jira] [Resolved] (KAFKA-12847) Dockerfile needed for kafka system tests needs changes

2021-06-14 Thread Abhijit Mane (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12847?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Abhijit Mane resolved KAFKA-12847. -- Reviewer: Chia-Ping Tsai Resolution: Won't Fix This issue is valid if SysTests are run

[jira] [Created] (KAFKA-12946) __consumer_offsets topic with very big partitions

2021-06-14 Thread Emi (Jira)
Emi created KAFKA-12946: --- Summary: __consumer_offsets topic with very big partitions Key: KAFKA-12946 URL: https://issues.apache.org/jira/browse/KAFKA-12946 Project: Kafka Issue Type: Bug

Re: [VOTE] KIP-719: Add Log4J2 Appender

2021-06-14 Thread Dongjin Lee
> VerifiableLog4jAppender is used for system tests, it's not a user facing tool. We don't need it to support log4j 2. I mean, as long as there is VerifiableLog4jAppender which uses log4j-appender, we can't entirely remove log4j 1.x artifact from the classpath, regardless of if it is a user-facing