Re: [DISCUSS] Single broker failures causing offline partitions

2024-09-11 Thread Kamal Chandraprakash
Hi Haruki,

We are also interested in this issue.

> The problem is how to identify such "eligible" replicas.

Is the leader election automated to find the replica with the highest
offset and latest epoch?
If yes, could you please open a PR for it?

When a broker goes down, it might be serving leadership for 1000s of
partitions.
And, finding the eligible replica manually, will increase the outage
mitigation time
as the producers/consumers are blocked when there are offline partitions.

--
Kamal


On Wed, Sep 11, 2024 at 3:57 AM Haruki Okada  wrote:

> Hi Martin,
>
> Thank you for bringing up this issue.
>
> We suffer from this "single-broker failure causing unavailable partition"
> issue due to the disk failure for years too! Because we use HDDs and HDDs
> tend to cause high disk latency (tens~ of seconds) easily on disk glitch,
> which often blocks request-handler threads and making it unable to handle
> fetch-requests, then kicking followers out of ISRs.
>
> I believe solving the issue fundamentally is impossible unless we stop
> relying on external quorum (either KRaft or ZK) for failure
> detection/leader election and move to quorum-based data replication, which
> is not currently planned in Kafka.
>
> Let me share some of our experiences on how to address this problem.
>
> ## Proactive disk replacement / broker removal
> This is kind of dumb solution but we monitor disk health (e.g. Physical
> disk error counts under RAID) and replace disks or remove brokers
> proactively before it gets worse.
>
> ## Mitigate disk failure impact to the broker functionality
> In the first place, basically Kafka is page-cache intensive so disk-latency
> impacting the broker so much is unexpected.
> We found there are some call paths which disk-latency impact amplifies and
> we fixed them.
>
> - https://github.com/apache/kafka/pull/14289
> * This is a heuristic to mitigate KAFKA-7504, the network-thread
> blocking issue on catch-up reads which may impact many clients (including
> followers)
> * Not merged to upstream yet but we run this patch on production for
> years.
> - https://github.com/apache/kafka/pull/14242
> * This is a patch to stop calling fsync under Log#lock, which may cause
> all request handler threads to exhaust easily due to the lock contention
> when one thread is executing fsync. (disk latency directly impacts fsync
> latency)
>
> ## Prepare offline-partition-handling manual as the last resort
> Even with above efforts, unavailable-partition still may occur so we
> prepared (manual) runbook for such situations.
> Essentially, this is a procedure to do KIP-966 manually.
>
> We use acks=all and min.insync.replicas=2 on all partition, which means
> there should be one "eligible" (i.e. have all committed messages) replica
> even after a partition goes offline.
> The problem is how to identify such "eligible" replicas.
>
> If we can still login to the last leader, we can just check if the
> log-suffix matches. (by DumpLogSegments tool...)
> What about if the last leader completely fails and unable to login?
> In this case, we check the remaining two replicas' log segments and decide
> one has a longer log as the "eligible" replica, as long as they have the
> same leader epoch.
> (NOTE: Checking leader epoch is necessary, because in case leader is
> changing around the incidental time, "replica with higher offset" and
> "replica with all committed messages" may not match)
>
>
> Hope these help you.
>
> 2024年9月11日(水) 5:36 Justine Olshan :
>
> > Hey Calvin and Martin,
> >
> > Makes sense. So KIP-966 can help with 2, but 1 (some mechanism to
> identify
> > the issue) is still required.
> > If you haven't already filed a JIRA ticket for this, do you mind doing
> so?
> > I think it makes sense to close this gap.
> >
> > Justine
> >
> > On Tue, Sep 10, 2024 at 1:15 PM Calvin Liu 
> > wrote:
> >
> > > Hi Martin.
> > > Yes, the KIP-966 does not resolve your concern for the degraded leader.
> > To
> > > fill the gap,
> > > 1. As you have mentioned, we need a leader degradation detection
> > mechanism.
> > > So that the controller can promote another replica to the leader.
> > > 2. The controller needs to know which replica is a valid candidate to
> be
> > > the leader. ELR could be a good candidate(KIP-966 currently targets
> 4.0).
> > > It is a very interesting problem, probably the community can pick this
> up
> > > after 4.0.
> > >
> > > Calvin
> > >
> > > On Tue, Sep 10, 2024 at 9:37 AM Martin Dickson
> > >  wrote:
> > >
> > > > Hi Justine,
> > > >
> > > > Indeed we are very much interested in the improvements that will come
> > > from
> > > > KIP-966!
> > > >
> > > > However I think there is still a gap regarding the failure detection
> of
> > > the
> > > > leader. Please correct me if this is wrong but my understanding is
> that
> > > > with KIP-966 we'll stop advancing the HWM minISR isn't satisfied, and
> > > will
> > > > be able to fail-over leadership to any member of ELR following
> complete
> 

[VOTE] KIP-1034: Dead letter queue in Kafka Streams

2024-09-11 Thread Sebastien Viale
Hi all,

We would like to start a vote for KIP-1034: Dead letter queue in Kafka 
Streams

The KIP is available on 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1034%3A+Dead+letter+queue+in+Kafka+Streams

If you have any suggestions or feedback, feel free to participate to the 
discussion thread:
https://lists.apache.org/thread/1nhhsrogmmv15o7mk9nj4kvkb5k2bx9s

Best regards,

Damien Sebastien and Loic



Re: [VOTE] KIP-1064 - Upgrade slf4j to 2.x

2024-09-11 Thread Muralidhar Basani
Hi Chia, thank you for the vote.
I have updated kip motivation with it.

Thanks,
Murali

On Wed, Sep 11, 2024 at 8:15 AM Chia-Ping Tsai  wrote:

> +1 (binding)
>
> BTW, could you please add "compatibility issue" to the motivation?
>
> the version of slf4j provider must be equal to slf4j API [0]. Including
> both in kafka distribution can avoid compatibility issue;
>
> [0] https://www.slf4j.org/manual.html#compatibility
>
> Muralidhar Basani  於 2024年9月11日 週三
> 上午4:02寫道:
>
> > Hi all,
> >
> > I would like to call a vote on KIP-1064 - Upgrade slf4j to 2.x
> >
> > Target version : 4.0.0
> >
> > KIP -
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1064%3A+Upgrade+slf4j+to+2.x
> >
> > Discussion thread -
> > https://lists.apache.org/thread/z05rlzbjv89rpvzjmzsr8v4w17qn21r1
> >
> > Jira with some discussion :
> > https://issues.apache.org/jira/browse/KAFKA-16936
> >
> > Thanks,
> > Murali
> >
>


[jira] [Created] (KAFKA-17520) align the low bound of ducktape version

2024-09-11 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-17520:
--

 Summary: align the low bound of ducktape version 
 Key: KAFKA-17520
 URL: https://issues.apache.org/jira/browse/KAFKA-17520
 Project: Kafka
  Issue Type: Improvement
Reporter: Chia-Ping Tsai
Assignee: Chia-Ping Tsai


There was a discussion for it: 
https://github.com/apache/kafka/pull/14216#issuecomment-2117519600

and I noticed this issue when trying to upgrade base image for e2e



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [DISCUSS] Single broker failures causing offline partitions

2024-09-11 Thread Haruki Okada
Hi Kamal,

> Is the leader election automated to find the replica with the highest
offset and latest epoch?
> And, finding the eligible replica manually, will increase the outage
mitigation time

That's right, but this procedure is not yet automated in our operation.
Because, we found that "choosing highest epoch with latest epoch" is still
NOT safe and may lose committed messages in rare edge case if there are
simultaneous preferred leader switch (which might be the case in our
deployment, because we enable CruiseControl).

We used formal methods to prove this.
-
https://speakerdeck.com/line_developers/the-application-of-formal-methods-in-kafka-reliability-engineering
- https://github.com/ocadaruma/kafka-spec

So I believe we need KIP-966.

2024年9月11日(水) 15:55 Kamal Chandraprakash :

> Hi Haruki,
>
> We are also interested in this issue.
>
> > The problem is how to identify such "eligible" replicas.
>
> Is the leader election automated to find the replica with the highest
> offset and latest epoch?
> If yes, could you please open a PR for it?
>
> When a broker goes down, it might be serving leadership for 1000s of
> partitions.
> And, finding the eligible replica manually, will increase the outage
> mitigation time
> as the producers/consumers are blocked when there are offline partitions.
>
> --
> Kamal
>
>
> On Wed, Sep 11, 2024 at 3:57 AM Haruki Okada  wrote:
>
> > Hi Martin,
> >
> > Thank you for bringing up this issue.
> >
> > We suffer from this "single-broker failure causing unavailable partition"
> > issue due to the disk failure for years too! Because we use HDDs and HDDs
> > tend to cause high disk latency (tens~ of seconds) easily on disk glitch,
> > which often blocks request-handler threads and making it unable to handle
> > fetch-requests, then kicking followers out of ISRs.
> >
> > I believe solving the issue fundamentally is impossible unless we stop
> > relying on external quorum (either KRaft or ZK) for failure
> > detection/leader election and move to quorum-based data replication,
> which
> > is not currently planned in Kafka.
> >
> > Let me share some of our experiences on how to address this problem.
> >
> > ## Proactive disk replacement / broker removal
> > This is kind of dumb solution but we monitor disk health (e.g. Physical
> > disk error counts under RAID) and replace disks or remove brokers
> > proactively before it gets worse.
> >
> > ## Mitigate disk failure impact to the broker functionality
> > In the first place, basically Kafka is page-cache intensive so
> disk-latency
> > impacting the broker so much is unexpected.
> > We found there are some call paths which disk-latency impact amplifies
> and
> > we fixed them.
> >
> > - https://github.com/apache/kafka/pull/14289
> > * This is a heuristic to mitigate KAFKA-7504, the network-thread
> > blocking issue on catch-up reads which may impact many clients (including
> > followers)
> > * Not merged to upstream yet but we run this patch on production for
> > years.
> > - https://github.com/apache/kafka/pull/14242
> > * This is a patch to stop calling fsync under Log#lock, which may
> cause
> > all request handler threads to exhaust easily due to the lock contention
> > when one thread is executing fsync. (disk latency directly impacts fsync
> > latency)
> >
> > ## Prepare offline-partition-handling manual as the last resort
> > Even with above efforts, unavailable-partition still may occur so we
> > prepared (manual) runbook for such situations.
> > Essentially, this is a procedure to do KIP-966 manually.
> >
> > We use acks=all and min.insync.replicas=2 on all partition, which means
> > there should be one "eligible" (i.e. have all committed messages) replica
> > even after a partition goes offline.
> > The problem is how to identify such "eligible" replicas.
> >
> > If we can still login to the last leader, we can just check if the
> > log-suffix matches. (by DumpLogSegments tool...)
> > What about if the last leader completely fails and unable to login?
> > In this case, we check the remaining two replicas' log segments and
> decide
> > one has a longer log as the "eligible" replica, as long as they have the
> > same leader epoch.
> > (NOTE: Checking leader epoch is necessary, because in case leader is
> > changing around the incidental time, "replica with higher offset" and
> > "replica with all committed messages" may not match)
> >
> >
> > Hope these help you.
> >
> > 2024年9月11日(水) 5:36 Justine Olshan :
> >
> > > Hey Calvin and Martin,
> > >
> > > Makes sense. So KIP-966 can help with 2, but 1 (some mechanism to
> > identify
> > > the issue) is still required.
> > > If you haven't already filed a JIRA ticket for this, do you mind doing
> > so?
> > > I think it makes sense to close this gap.
> > >
> > > Justine
> > >
> > > On Tue, Sep 10, 2024 at 1:15 PM Calvin Liu  >
> > > wrote:
> > >
> > > > Hi Martin.
> > > > Yes, the KIP-966 does not resolve your concern for the degraded
> leader.
> > > To
> > > > fill the 

[jira] [Created] (KAFKA-17521) bootstrap-controller option buggy behavior

2024-09-11 Thread Michal Medvecky (Jira)
Michal Medvecky created KAFKA-17521:
---

 Summary: bootstrap-controller option buggy behavior
 Key: KAFKA-17521
 URL: https://issues.apache.org/jira/browse/KAFKA-17521
 Project: Kafka
  Issue Type: Bug
  Components: admin
Affects Versions: 3.7.1
Reporter: Michal Medvecky


Once running kafka admin tools with --bootstrap-controller, I am experiencing 
weird behavior. Let me show examples.
{code:java}
[appuser@e4bbc669d343 ~]$ kafka-configs --describe --bootstrap-controller 
kafka1:9093 --command-config /tmp/kafka-client.properties --entity-type brokers 
--entity-name 1
Dynamic configs for broker 1 are: {code}
That's "sort of" fine, but:
 * my set up consists of 3 controller nodes (1,2,3) and 3 broker nodes (4,5,6).
 * entity-type must be "brokers", even though I am connecting to a controller 
(9093/tcp is a controller listener)
 * node 1 is not a broker, but a controller instead ("for broker 1 are ...")

When trying to describe config for node 2:
{code:java}
[appuser@e4bbc669d343 ~]$ kafka-configs --describe --bootstrap-controller 
kafka1:9093 --command-config /tmp/kafka-client.properties --entity-type brokers 
--entity-name 2
Dynamic configs for broker 2 are:
Error while executing config command with args '--describe 
--bootstrap-controller kafka1:9093 --command-config 
/tmp/kafka-client.properties --entity-type brokers --entity-name 2'
java.util.concurrent.ExecutionException: 
org.apache.kafka.common.errors.InvalidRequestException: Unexpected broker id, 
expected 1 or empty string, but received 2
    at 
java.base/java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:396)
    at 
java.base/java.util.concurrent.CompletableFuture.get(CompletableFuture.java:2096)
    at 
org.apache.kafka.common.internals.KafkaFutureImpl.get(KafkaFutureImpl.java:180)
    at kafka.admin.ConfigCommand$.getResourceConfig(ConfigCommand.scala:610)
    at 
kafka.admin.ConfigCommand$.$anonfun$describeResourceConfig$5(ConfigCommand.scala:568)
    at 
kafka.admin.ConfigCommand$.$anonfun$describeResourceConfig$5$adapted(ConfigCommand.scala:560)
    at scala.collection.immutable.List.foreach(List.scala:333)
    at 
kafka.admin.ConfigCommand$.describeResourceConfig(ConfigCommand.scala:560)
    at kafka.admin.ConfigCommand$.describeConfig(ConfigCommand.scala:538)
    at kafka.admin.ConfigCommand$.processCommand(ConfigCommand.scala:343)
    at kafka.admin.ConfigCommand$.main(ConfigCommand.scala:97)
    at kafka.admin.ConfigCommand.main(ConfigCommand.scala)
Caused by: org.apache.kafka.common.errors.InvalidRequestException: Unexpected 
broker id, expected 1 or empty string, but received 2 {code}
Ehm, what? Expected 1? I need to describe configs for node 2, not 1. The same 
thing happens, once connecting to node 2 instead of node 1:
{code:java}
[appuser@e4bbc669d343 ~]$ kafka-configs --describe --bootstrap-controller 
kafka2:9093 --command-config /tmp/kafka-client.properties --entity-type brokers 
--entity-name 2
Dynamic configs for broker 2 are:
Error while executing config command with args '--describe 
--bootstrap-controller kafka2:9093 --command-config 
/tmp/kafka-client.properties --entity-type brokers --entity-name 2'
java.util.concurrent.ExecutionException: 
org.apache.kafka.common.errors.InvalidRequestException: Unexpected broker id, 
expected 1 or empty string, but received 2
    at 
java.base/java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:396)
    at 
java.base/java.util.concurrent.CompletableFuture.get(CompletableFuture.java:2096)
    at 
org.apache.kafka.common.internals.KafkaFutureImpl.get(KafkaFutureImpl.java:180)
    at kafka.admin.ConfigCommand$.getResourceConfig(ConfigCommand.scala:610)
    at 
kafka.admin.ConfigCommand$.$anonfun$describeResourceConfig$5(ConfigCommand.scala:568)
    at 
kafka.admin.ConfigCommand$.$anonfun$describeResourceConfig$5$adapted(ConfigCommand.scala:560)
    at scala.collection.immutable.List.foreach(List.scala:333)
    at 
kafka.admin.ConfigCommand$.describeResourceConfig(ConfigCommand.scala:560)
    at kafka.admin.ConfigCommand$.describeConfig(ConfigCommand.scala:538)
    at kafka.admin.ConfigCommand$.processCommand(ConfigCommand.scala:343)
    at kafka.admin.ConfigCommand$.main(ConfigCommand.scala:97)
    at kafka.admin.ConfigCommand.main(ConfigCommand.scala)
Caused by: org.apache.kafka.common.errors.InvalidRequestException: Unexpected 
broker id, expected 1 or empty string, but received 2 {code}
If i specify {{--all}} instead of {{{}entity-name{}}}, what I see is:
{code:java}
[appuser@e4bbc669d343 ~]$ kafka-configs --describe --bootstrap-controller 
kafka2:9093 --command-config /tmp/kafka-client.properties --entity-type brokers 
--all
All configs for broker 1 are:
  advertised.listeners=null sensitive=false synonyms={}

  zookeeper.ssl.truststore.type=null sensitive=false synonyms={}
All configs for broker 2 are:
Error while executi

Jenkins build is unstable: Kafka » Kafka Branch Builder » trunk #3292

2024-09-11 Thread Apache Jenkins Server
See 




[jira] [Created] (KAFKA-17522) Share partition acquire() need not return a future

2024-09-11 Thread Abhinav Dixit (Jira)
Abhinav Dixit created KAFKA-17522:
-

 Summary: Share partition acquire() need not return a future
 Key: KAFKA-17522
 URL: https://issues.apache.org/jira/browse/KAFKA-17522
 Project: Kafka
  Issue Type: Sub-task
Reporter: Abhinav Dixit
Assignee: Apoorv Mittal


As per discussion 
[https://github.com/apache/kafka/pull/16274#discussion_r1700968453] and 
[https://github.com/apache/kafka/pull/16969#discussion_r1752362118] , we don't 
need acquire method to return a future since we are not persisting acquisitions.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-17523) Connectors should be able to override offset.flush.interval.ms

2024-09-11 Thread Viktor Somogyi-Vass (Jira)
Viktor Somogyi-Vass created KAFKA-17523:
---

 Summary: Connectors should be able to override 
offset.flush.interval.ms
 Key: KAFKA-17523
 URL: https://issues.apache.org/jira/browse/KAFKA-17523
 Project: Kafka
  Issue Type: Improvement
Reporter: Viktor Somogyi-Vass


The offset.flush.interval.ms config affects how frequently the tasks of a 
connector is flushed. This is a worker level config, affecting all connectors. 
In a prod cluster, it is possible to have multiple connectors with different 
desired flush rate which means that users may want to override this config on 
connector level too.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [DISCUSS] KIP-1043: Administration of groups

2024-09-11 Thread Lucas Brutschy
Hi Andrew,

thanks for the KIP!

It is looking good from my side! I like the simplification, and that
we added the new error but only of the Describe RPCs. It's a good
pragmatic improvement of the current state of things.

I only have very minor comments:
 - nit: In `GroupListing`, you seem to import `ShareGroupState` and
it's not clear why.
 - The documentation for `--consumer` in the table is not enough. We
should make sure that the comment below the table is also included in
the command-line help of the CLI tool -- I was confused by this at
first. Possibly just explain it in terms of the equivalent sequence of
commands.

>From my point of view, this is ready for a vote.

Cheers,
Lucas



On Tue, Sep 3, 2024 at 2:56 PM Andrew Schofield
 wrote:
>
> Hi,
> I’ve spent some time working with clusters containing groups of multiple
> types, fixing problems and improving error handling.
>
> I’ve simplified the KIP so that it just adds kafka-groups.sh and improves
> the error handling for describing groups of the wrong type. With the other
> improvements I’ve already made, it seems to me that this is sufficient to
> make working with groups of multiple types work nicely.
>
> I’d like to ask for another round of reviews before hopefully opening up
> a vote soon.
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1043%3A+Administration+of+groups
>
> Thanks,
> Andrew
>
> 
> From: Andrew Schofield 
> Sent: 02 August 2024 15:00
> To: dev@kafka.apache.org 
> Subject: Re: [DISCUSS] KIP-1043: Administration of groups
>
> Hi Lianet,
> Thanks for your comment.
>
> I’ve been digging more into the situation with describing groups in a
> broker with groups of multiple types. It’s a bit fiddly because of the
> introduction of the modern consumer groups by KIP-848 and the
> need for the admin client to cope with both kinds of consumer groups
> and older brokers.
>
> If you use `kafka-consumer-groups.sh --describe --group MYSHARE`
> the output is:
>
>   Error: Consumer group ‘MYSHARE’ does not exist.
>
> How does it get there? AdminClient.describeConsumerGroups
> is complicated.
>
> First, it uses the ConsumerGroupDescribe RPC which responds
> with GROUP_ID_NOT_FOUND (69) and an empty error message.
> The broker *could* fill in the error message to help with this situation
> but I don’t like that as a solution. Seems quite brittle.
>
> Then, it uses the DescribeGroups RPC in case it’s a classic consumer
> group. This responds with error code NONE (0) and makes the group
> look like a Dead consumer group. There is no error message field
> in that RPC at all, so we don’t have the option of using an error
> message to disambiguate.
>
> So, `kafka-consumer-groups.sh` thinks that it’s dealing with a dead
> consumer group and its output makes sense.
>
> My preferred course of action here is as you suggest to introduce
> the new error code, INVALID_GROUP_TYPE. If you use any of the following
> RPCs with the wrong type of group, you get this response:
>
> * ConsumerGroupDescribe
> * ShareGroupDescribe
> * ConsumerGroupHeartbeat
> * ShareGroupHeartbeat
>
> The remaining RPCs for consumer groups, such as ListOffsets and
> TxnOffsetCommit continue to use `GROUP_ID_NOT_FOUND`.
>
> Does that make sense? Any further comments?
>
> Thanks,
> Andrew
>
> > On 23 Jul 2024, at 17:26, Lianet M.  wrote:
> >
> > Hello Andrew,
> >
> > Bringing here the point I surfaced on the KIP-1071 thread:
> >
> > I wonder if at this point, where we're getting several new group types
> >> added, each with RPCs that are supposed to include groupId of a certain
> >> type, we should be more explicit about this situation. Maybe a kind of
> >> INVALID_GROUP_TYPE (group exists but not with a valid type for this RPC) vs
> >> a GROUP_ID_NOT_FOUND (group does not exist).  Those errors would be
> >> consistently used across consumer, share, and streams RPCs whenever the
> >> group id is not of the expected type.
> >
> >
> > I noticed it on KIP-1071 but totally agree with you that it would make more
> > sense to consider it here.
> >
> > LM9. Regarding the point of introducing a new INVALID_GROUP_TYPE vs reusing
> > the existing INCONSISTENT_PROTOCOL_TYPE. My concern with reusing
> > INCONSISTENT_GROUP_PROTOCOL for errors with the group ID is that it mixes
> > the concepts of group type and protocol. Even though they are closely
> > related, we have 2 separate concepts (internally and presented in output
> > for commands), and the relationship is not 1-1 in all cases. Also, the
> > INCONSISTENT_GROUP_PROTOCOL is already used not only for protocol but also
> > when validating the list of assignors provided by a consumer in a
> > JoinGroupRequest. Seems a bit confusing to me already, so maybe better not
> > to add more to it? Just first thoughts. What do you think?
> >
> > Thanks,
> > Lianet
> >
> > On Fri, Jul 19, 2024 at 5:00 AM Andrew Schofield 
> > wrote:
> >
> >> Hi Apoorv,
> >> Thanks for your comments.
> >>
> >> AM1: I chose 

[jira] [Created] (KAFKA-17524) StreamThreadTest shouldReturnErrorIfProducerInstanceIdNotInitialized hanging

2024-09-11 Thread David Arthur (Jira)
David Arthur created KAFKA-17524:


 Summary: StreamThreadTest 
shouldReturnErrorIfProducerInstanceIdNotInitialized hanging
 Key: KAFKA-17524
 URL: https://issues.apache.org/jira/browse/KAFKA-17524
 Project: Kafka
  Issue Type: Test
Reporter: David Arthur


A trunk build had a timeout and it appears that this test was the cause.

[https://github.com/apache/kafka/actions/runs/10798234851/job/29953919232]

In the Gradle log, we see 
{code:java}
2024-09-10T20:31:26.6830206Z Gradle Test Run :streams:test > Gradle Test 
Executor 47 > StreamThreadTest > 
shouldReturnErrorIfProducerInstanceIdNotInitialized(boolean, boolean) > 
"shouldReturnErrorIfProducerInstanceIdNotInitialized(boolean, 
boolean).stateUpdaterEnabled=true, processingThreadsEnabled=true" STARTED {code}
but no "FINISHED" or "FAILED" later in the log.

It seems that this test was running for around 50 minutes.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-17506) KRaftMigrationDriver initialization race condition

2024-09-11 Thread David Arthur (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-17506?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

David Arthur resolved KAFKA-17506.
--
Resolution: Fixed

> KRaftMigrationDriver initialization race condition
> --
>
> Key: KAFKA-17506
> URL: https://issues.apache.org/jira/browse/KAFKA-17506
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 3.8.0, 3.7.1, 3.9.0
>Reporter: David Arthur
>Assignee: David Arthur
>Priority: Blocker
> Fix For: 3.9.0, 3.7.2, 3.8.1
>
>
> There is a race condition between KRaftMigrationDriver running its first 
> poll() and being notified by Raft about a leader change. If 
> {{onControllerChange}} is called before RecoverMigrationStateFromZKEvent is 
> run, we will end up getting stuck in the INACTIVE state.
> This is likely a very narrow race condition that is not likely to be seen in 
> practice. It was found when debugging a flaky integration test.
> If encountered, this can be worked around by restarting the stuck controller. 
> Leadership will move to another controller which will allow the migration to 
> be started.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Jenkins build is still unstable: Kafka » Kafka Branch Builder » 3.9 #64

2024-09-11 Thread Apache Jenkins Server
See 




[jira] [Created] (KAFKA-17525) Convert the UnknownServerException to InvalidRequestException when altering client-metrics config at runtime

2024-09-11 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-17525:
--

 Summary: Convert the UnknownServerException to 
InvalidRequestException when altering client-metrics config at runtime
 Key: KAFKA-17525
 URL: https://issues.apache.org/jira/browse/KAFKA-17525
 Project: Kafka
  Issue Type: Improvement
Reporter: Chia-Ping Tsai
Assignee: Chia-Ping Tsai


see following test:

{code:bash}
chia7712@fedora:~/project/kafka$ bin/kafka-configs.sh --bootstrap-server 
192.168.1.149:2 --alter --entity-type client-metrics --entity-name test 
--add-config interval.ms=bbb
Error while executing config command with args '--bootstrap-server 
192.168.1.149:2 --alter --entity-type client-metrics --entity-name test 
--add-config interval.ms=bbb'
java.util.concurrent.ExecutionException: 
org.apache.kafka.common.errors.UnknownServerException: The server experienced 
an unexpected error when processing the request.
at 
java.base/java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:396)
at 
java.base/java.util.concurrent.CompletableFuture.get(CompletableFuture.java:2096)
at 
org.apache.kafka.common.internals.KafkaFutureImpl.get(KafkaFutureImpl.java:180)
at 
kafka.admin.ConfigCommand$.alterResourceConfig(ConfigCommand.scala:581)
at kafka.admin.ConfigCommand$.alterConfig(ConfigCommand.scala:449)
at kafka.admin.ConfigCommand$.processCommand(ConfigCommand.scala:351)
at kafka.admin.ConfigCommand$.main(ConfigCommand.scala:100)
at kafka.admin.ConfigCommand.main(ConfigCommand.scala)
Caused by: org.apache.kafka.common.errors.UnknownServerException: The server 
experienced an unexpected error when processing the request.

{code}

By contrast, `topic` resource can handle the such error and then return 
`InvalidRequestException`
```bash
chia7712@fedora:~/project/kafka$ bin/kafka-configs.sh --bootstrap-server 
192.168.1.149:2 --alter --topic chia --add-config flush.ms=aaa
Error while executing config command with args '--bootstrap-server 
192.168.1.149:2 --alter --topic chia --add-config flush.ms=aaa'
java.util.concurrent.ExecutionException: 
org.apache.kafka.common.errors.InvalidConfigurationException: Invalid value aaa 
for configuration flush.ms: Not a number of type LONG
at 
java.base/java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:396)
at 
java.base/java.util.concurrent.CompletableFuture.get(CompletableFuture.java:2096)
at 
org.apache.kafka.common.internals.KafkaFutureImpl.get(KafkaFutureImpl.java:180)
at 
kafka.admin.ConfigCommand$.alterResourceConfig(ConfigCommand.scala:581)
at kafka.admin.ConfigCommand$.alterConfig(ConfigCommand.scala:371)
at kafka.admin.ConfigCommand$.processCommand(ConfigCommand.scala:351)
at kafka.admin.ConfigCommand$.main(ConfigCommand.scala:100)
at kafka.admin.ConfigCommand.main(ConfigCommand.scala)
Caused by: org.apache.kafka.common.errors.InvalidConfigurationException: 
Invalid value aaa for configuration flush.ms: Not a number of type LONG
```




--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [DISCUSS] Single broker failures causing offline partitions

2024-09-11 Thread Martin Dickson
Hi Justine and Calvin,

Thank you both for the feedback, I agree that KIP-966 should solve #2. I'll
file a JIRA ticket for #1 (currently in the process of registering my
account).

Best,
Martin

On Tue, Sep 10, 2024 at 9:36 PM Justine Olshan 
wrote:

> Hey Calvin and Martin,
>
> Makes sense. So KIP-966 can help with 2, but 1 (some mechanism to identify
> the issue) is still required.
> If you haven't already filed a JIRA ticket for this, do you mind doing so?
> I think it makes sense to close this gap.
>
> Justine
>
> On Tue, Sep 10, 2024 at 1:15 PM Calvin Liu 
> wrote:
>
> > Hi Martin.
> > Yes, the KIP-966 does not resolve your concern for the degraded leader.
> To
> > fill the gap,
> > 1. As you have mentioned, we need a leader degradation detection
> mechanism.
> > So that the controller can promote another replica to the leader.
> > 2. The controller needs to know which replica is a valid candidate to be
> > the leader. ELR could be a good candidate(KIP-966 currently targets 4.0).
> > It is a very interesting problem, probably the community can pick this up
> > after 4.0.
> >
> > Calvin
> >
> > On Tue, Sep 10, 2024 at 9:37 AM Martin Dickson
> >  wrote:
> >
> > > Hi Justine,
> > >
> > > Indeed we are very much interested in the improvements that will come
> > from
> > > KIP-966!
> > >
> > > However I think there is still a gap regarding the failure detection of
> > the
> > > leader. Please correct me if this is wrong but my understanding is that
> > > with KIP-966 we'll stop advancing the HWM minISR isn't satisfied, and
> > will
> > > be able to fail-over leadership to any member of ELR following complete
> > > failure of the leader. This gives good options for recovery following a
> > > complete failure, however, if the leader remains degraded but doesn't
> > > completely fail then the partition will stay online but will still be
> > > unavailable for writes. (So probably a better subject for my email
> would
> > > have been "Single degraded brokers causing unavailable partitions",
> which
> > > is an issue which I think remains post KIP-966?)
> > >
> > > Best,
> > > Martin
> > >
> > >
> > > On Tue, Sep 10, 2024 at 5:14 PM Justine Olshan
> > > 
> > > wrote:
> > >
> > > > Hey Martin.
> > > >
> > > > I recommend you take a look at KIP-966. I think can help the use case
> > you
> > > > are describing.
> > > > The KIP talks about failure scenarios, but I believe it will also
> help
> > > when
> > > > the leader has issues and kicks its followers out of ISR.
> > > > The goal is to better handle the "last replica standing" issue
> > > >
> > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-966%3A+Eligible+Leader+Replicas
> > > >
> > > > Let me know if it helps,
> > > >
> > > > Justine
> > > >
> > > > On Tue, Sep 10, 2024 at 9:00 AM Martin Dickson
> > > >  wrote:
> > > >
> > > > > Hi all,
> > > > >
> > > > > We have a recurring issue with single broker failures causing
> offline
> > > > > partitions. The issue is that when a leader is degraded, follower
> > > fetches
> > > > > can fail to happen in a timely manner, and all followers can fall
> out
> > > of
> > > > > sync. If that leader then later fails then the partition will go
> > > offline,
> > > > > but even if it remains only partially failed then applications
> might
> > > > still
> > > > > be impacted (for example, if the producer is using acks=all and
> > > > > min.insync.replicas=2). This can all happen because of a problem
> > solely
> > > > > with the leader, and hence a single broker failure can cause
> > > > > unavailability, even if RF=3 or higher.
> > > > >
> > > > > We’ve seen the issue with various kinds of failures, mostly related
> > to
> > > > > failing disks, e.g. due to pressure on request handler threads as a
> > > > result
> > > > > of produce requests waiting on a slow disk. But the easiest way for
> > us
> > > to
> > > > > reproduce it is at the outgoing network level: Setting up a cluster
> > > with
> > > > > moderate levels of ingoing throughput then injecting 50% outgoing
> > > packet
> > > > > drop on a single broker is enough to cause the partitions to cause
> > > > follower
> > > > > requests to be slow and replication to lag, but not enough for that
> > > > broker
> > > > > to lose its connection to ZK. This triggers the degraded broker to
> > > become
> > > > > the only member of ISR.
> > > > >
> > > > > We have replica.lag.time.max.ms=1 and
> > zookeeper.session.timeout.ms
> > > > > =6000
> > > > > (the pre-KIP-537 values, 1/3 of the current defaults, to control
> > > produce
> > > > > latency when a follower is failing). We are also able to reproduce
> > the
> > > > > issue in the same way on a KRaft cluster with the KRaft defaults.
> > (Note
> > > > > that we are not very experienced with operating KRaft as we aren’t
> > > > running
> > > > > it in production yet.)
> > > > >
> > > > > The last KIP I saw regarding this was KIP-501
> > > > > <
> > > > >
> > > >
> > >
> >
> https://cwiki.apach

Re: [DISCUSS] Single broker failures causing offline partitions

2024-09-11 Thread Martin Dickson
Hi Haruki and Kamal,

It's very interesting to see how you're handling this Haruki.
- For proactive broker replacement, we are doing something similar. We
monitor certain metrics, mostly related to slow disks, and automatically
replace the machines if those alerts fire but Kafka state is still safe. We
are currently doing broker replacements rather than leadership demotions,
so we are quite conservative. However it's a constant challenge to tune
these alerts, and in many cases the latency (or additional failure modes)
induced by relying on an external monitoring system means we cannot act
before the situation becomes critical (ISR=1)
- Thank you for sharing the patches you're running as well, I'm looking
through our history to see if we've also been impacted by KAFKA-7504
, we aren't using HDDs so
potentially not

The edge case for selecting the follower which should have full HWM is
interesting. We were considering implementing something similar as a
temporary measure until KIP-966. We hadn't looked into that properly yet,
but it's very good to know that "latest offset and epoch" is not 100%
correct.

I also believe there can be a fool-proof system for degraded brokers even
when KRaft is doing failure detection. However I am hopeful that if some
degradation detection and safe handling were built into Kafka the number of
issues could be reduced. We've been exploring adding some degradation
detection logic in controllers, another option is mentioned in the Kora
paper  (section 4.5.3)
based on per-broker health-checks.

Best,
Martin

On Wed, Sep 11, 2024 at 10:59 AM Haruki Okada  wrote:

> Hi Kamal,
>
> > Is the leader election automated to find the replica with the highest
> offset and latest epoch?
> > And, finding the eligible replica manually, will increase the outage
> mitigation time
>
> That's right, but this procedure is not yet automated in our operation.
> Because, we found that "choosing highest epoch with latest epoch" is still
> NOT safe and may lose committed messages in rare edge case if there are
> simultaneous preferred leader switch (which might be the case in our
> deployment, because we enable CruiseControl).
>
> We used formal methods to prove this.
> -
>
> https://speakerdeck.com/line_developers/the-application-of-formal-methods-in-kafka-reliability-engineering
> - https://github.com/ocadaruma/kafka-spec
>
> So I believe we need KIP-966.
>
> 2024年9月11日(水) 15:55 Kamal Chandraprakash :
>
> > Hi Haruki,
> >
> > We are also interested in this issue.
> >
> > > The problem is how to identify such "eligible" replicas.
> >
> > Is the leader election automated to find the replica with the highest
> > offset and latest epoch?
> > If yes, could you please open a PR for it?
> >
> > When a broker goes down, it might be serving leadership for 1000s of
> > partitions.
> > And, finding the eligible replica manually, will increase the outage
> > mitigation time
> > as the producers/consumers are blocked when there are offline partitions.
> >
> > --
> > Kamal
> >
> >
> > On Wed, Sep 11, 2024 at 3:57 AM Haruki Okada 
> wrote:
> >
> > > Hi Martin,
> > >
> > > Thank you for bringing up this issue.
> > >
> > > We suffer from this "single-broker failure causing unavailable
> partition"
> > > issue due to the disk failure for years too! Because we use HDDs and
> HDDs
> > > tend to cause high disk latency (tens~ of seconds) easily on disk
> glitch,
> > > which often blocks request-handler threads and making it unable to
> handle
> > > fetch-requests, then kicking followers out of ISRs.
> > >
> > > I believe solving the issue fundamentally is impossible unless we stop
> > > relying on external quorum (either KRaft or ZK) for failure
> > > detection/leader election and move to quorum-based data replication,
> > which
> > > is not currently planned in Kafka.
> > >
> > > Let me share some of our experiences on how to address this problem.
> > >
> > > ## Proactive disk replacement / broker removal
> > > This is kind of dumb solution but we monitor disk health (e.g. Physical
> > > disk error counts under RAID) and replace disks or remove brokers
> > > proactively before it gets worse.
> > >
> > > ## Mitigate disk failure impact to the broker functionality
> > > In the first place, basically Kafka is page-cache intensive so
> > disk-latency
> > > impacting the broker so much is unexpected.
> > > We found there are some call paths which disk-latency impact amplifies
> > and
> > > we fixed them.
> > >
> > > - https://github.com/apache/kafka/pull/14289
> > > * This is a heuristic to mitigate KAFKA-7504, the network-thread
> > > blocking issue on catch-up reads which may impact many clients
> (including
> > > followers)
> > > * Not merged to upstream yet but we run this patch on production
> for
> > > years.
> > > - https://github.com/apache/kafka/pull/14242
> > > * This is a patch to stop calling fsync under 

[jira] [Created] (KAFKA-17526) make ConfigCommandIntegrationTest.java test use correct arguments in testing alias

2024-09-11 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-17526:
--

 Summary: make ConfigCommandIntegrationTest.java test use correct 
arguments in testing alias
 Key: KAFKA-17526
 URL: https://issues.apache.org/jira/browse/KAFKA-17526
 Project: Kafka
  Issue Type: Test
Reporter: Chia-Ping Tsai
Assignee: Chia-Ping Tsai


see https://github.com/apache/kafka/pull/17046#discussion_r1755095302



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-17527) Kafka Streams fails with NPE for missing RecordContext

2024-09-11 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-17527:
---

 Summary: Kafka Streams fails with NPE for missing RecordContext
 Key: KAFKA-17527
 URL: https://issues.apache.org/jira/browse/KAFKA-17527
 Project: Kafka
  Issue Type: Bug
  Components: streams
Affects Versions: 3.9.0
Reporter: Matthias J. Sax


He did observe a crash of Kafka Streams with the following stack trace:
{code:java}
2024-09-10 10:59:12,301] ERROR [kafka-producer-network-thread | 
i-0197827b22f4d4e4c-StreamThread-1-producer] Error executing user-provided 
callback on message for topic-partition 
'stream-soak-test-KSTREAM-AGGREGATE-STATE-STORE-47-changelog-1' 
(org.apache.kafka.clients.producer.internals.ProducerBatch)
java.lang.NullPointerException: Cannot invoke 
"org.apache.kafka.streams.processor.internals.InternalProcessorContext.recordContext()"
 because "context" is null
    at 
org.apache.kafka.streams.processor.internals.RecordCollectorImpl.recordSendError(RecordCollectorImpl.java:405)
    at 
org.apache.kafka.streams.processor.internals.RecordCollectorImpl.lambda$send$1(RecordCollectorImpl.java:285)
    at 
org.apache.kafka.clients.producer.KafkaProducer$AppendCallbacks.onCompletion(KafkaProducer.java:1574)
    at 
org.apache.kafka.clients.producer.internals.ProducerBatch.completeFutureAndFireCallbacks(ProducerBatch.java:312)
    at 
org.apache.kafka.clients.producer.internals.ProducerBatch.abort(ProducerBatch.java:200)
    at 
org.apache.kafka.clients.producer.internals.RecordAccumulator.abortUndrainedBatches(RecordAccumulator.java:1166)
    at 
org.apache.kafka.clients.producer.internals.Sender.maybeSendAndPollTransactionalRequest(Sender.java:474)
    at 
org.apache.kafka.clients.producer.internals.Sender.runOnce(Sender.java:337)
    at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:251)
    at java.base/java.lang.Thread.run(Thread.java:840) {code}
It seems to be a bug introduced via KIP-1033, coming from the changelogging 
layer which does pass a `null` context into `RecordCollector.send(...)`.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: New release branch 3.9

2024-09-11 Thread Matthias J. Sax

Hi Colin,

we found another blocker bug: 
https://issues.apache.org/jira/browse/KAFKA-17527


Working on a fix.


-Matthias

On 9/9/24 6:51 PM, David Arthur wrote:

Colin,

I found a race condition in the migration while investigating a flaky test.
If encountered, the migration driver will get stuck initializing.

JIRA: https://issues.apache.org/jira/browse/KAFKA-17506
PR: https://github.com/apache/kafka/pull/17147

I wouldn't normally consider this a blocker, since it's quite rare and easy
to work around, but I wanted to raise it here since this is our last chance
for migration fixes in 3.x

-David



On Mon, Sep 9, 2024 at 3:50 AM Lucas Brutschy
 wrote:


Hi Colin,

about KAFKA-17489, the bug Bruno mentioned. This needs to be fixed in
3.8 as well. The fix is probably small but tricky to get right - Bruno
has attempted to fix it, but soaking revealed that the fix is not
complete. Bruno is now out with Covid, so I will look into it. I'll
update this thread once I know more.

Cheers,
Lucas

On Mon, Sep 9, 2024 at 4:47 AM Colin McCabe  wrote:


Hi Chia-Ping Tsai,

Thanks for the bug report. Let’s follow up on this this week.

Best,
Colin

On Fri, Sep 6, 2024, at 06:19, Chia-Ping Tsai wrote:

hi Colin

I raise a issue (https://issues.apache.org/jira/browse/KAFKA-17492) as
3.9 blocker, since the bug obstructs us from registering 3.9 broker to
3.8 controller

Best,
Chia-Ping

On 2024/08/09 01:46:35 Colin McCabe wrote:

Hi all,

I think it's time to transition the 3.9 branch to feature freeze.

Please note that this is 2 weeks past the original planned date. I
apologize for this; our planning wasn't perfect.


A few updates on features that are in or out of Apache Kafka 3.9:

- We have completed all the feature work for KIP-853: KRaft

controller membership changes. (Please note that there are still a few bug
fixes remaining, and the docs JIRA, KAFKA-17048, but those are all
post-feature-freeze tasks.)


- After speaking with Calvin, I moved KIP-966 out of 3.9 and into

4.0. I do feel bad about this but I just don't think we'll be able to get
it into this short release.


There are a few other KIPs that probably need to be moved to 4.0,

such as KIP-1023 and KIP-1025. I will do a more thorough review tomorrow
and reach out to people if there are gray areas. If you have a feature that
you have questions about, please reach out.


I would like to extend code freeze for Kafka 3.9 to August 29th. My

reasoning is that code freeze was 2 weeks later than expected, and also, I
will be out of the office most of next week.


I have also created a PR to mark 3.9-IV0 as stable, so look for that

shortly in trunk and 3.9.


As previously mentioned, please refrain from changing the Java

version in trunk (kafka 4.0) until we get an RC out. (However, please do
continue working on your 4.0 features and other refactors in trunk, even if
they don't apply to 3.9!)


Thanks to everyone who has worked on this release, and thanks for

your patience. I do believe we will deliver on the promise of a
much-shorter-than-usual release cycle, even with the 2 week delay.


best,
Colin








[jira] [Resolved] (KAFKA-17347) Add omitted --client-metrics option to kafka-configs.sh

2024-09-11 Thread Chia-Ping Tsai (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-17347?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Chia-Ping Tsai resolved KAFKA-17347.

Resolution: Fixed

> Add omitted --client-metrics option to kafka-configs.sh
> ---
>
> Key: KAFKA-17347
> URL: https://issues.apache.org/jira/browse/KAFKA-17347
> Project: Kafka
>  Issue Type: Bug
>  Components: tools
>Affects Versions: 3.7.0
>Reporter: Andrew Schofield
>Assignee: Andrew Schofield
>Priority: Major
> Fix For: 4.0.0
>
>
> KIP-714 introduced client metrics resources to kafka-configs.sh. The option 
> --entity-type client-metrics was added, and a shorthand of "client-metrics" 
> was also included in the comments. However, the "-client-metrics" option 
> whose syntax matches all of the other entity types was omitted. This corrects 
> that omission.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [DISCUSS] KIP-1043: Administration of groups

2024-09-11 Thread Lianet M.
Hi Andrew,

Took another full pass here after the latest changes (thanks for addressing
all the feedback!).

Only minor comment left is to add to the reject alternatives the
INCONSISTENT_GROUP_PROTOCOL error that we discussed and discarded (see
LM9). Also agree with the 2 minor points made by Lucas.

With that, I have no other comments and agree we could proceed to voting.

Thanks!

Lianet

On Wed, Sep 11, 2024 at 10:17 AM Lucas Brutschy
 wrote:

> Hi Andrew,
>
> thanks for the KIP!
>
> It is looking good from my side! I like the simplification, and that
> we added the new error but only of the Describe RPCs. It's a good
> pragmatic improvement of the current state of things.
>
> I only have very minor comments:
>  - nit: In `GroupListing`, you seem to import `ShareGroupState` and
> it's not clear why.
>  - The documentation for `--consumer` in the table is not enough. We
> should make sure that the comment below the table is also included in
> the command-line help of the CLI tool -- I was confused by this at
> first. Possibly just explain it in terms of the equivalent sequence of
> commands.
>
> From my point of view, this is ready for a vote.
>
> Cheers,
> Lucas
>
>
>
> On Tue, Sep 3, 2024 at 2:56 PM Andrew Schofield
>  wrote:
> >
> > Hi,
> > I’ve spent some time working with clusters containing groups of multiple
> > types, fixing problems and improving error handling.
> >
> > I’ve simplified the KIP so that it just adds kafka-groups.sh and improves
> > the error handling for describing groups of the wrong type. With the
> other
> > improvements I’ve already made, it seems to me that this is sufficient to
> > make working with groups of multiple types work nicely.
> >
> > I’d like to ask for another round of reviews before hopefully opening up
> > a vote soon.
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1043%3A+Administration+of+groups
> >
> > Thanks,
> > Andrew
> >
> > 
> > From: Andrew Schofield 
> > Sent: 02 August 2024 15:00
> > To: dev@kafka.apache.org 
> > Subject: Re: [DISCUSS] KIP-1043: Administration of groups
> >
> > Hi Lianet,
> > Thanks for your comment.
> >
> > I’ve been digging more into the situation with describing groups in a
> > broker with groups of multiple types. It’s a bit fiddly because of the
> > introduction of the modern consumer groups by KIP-848 and the
> > need for the admin client to cope with both kinds of consumer groups
> > and older brokers.
> >
> > If you use `kafka-consumer-groups.sh --describe --group MYSHARE`
> > the output is:
> >
> >   Error: Consumer group ‘MYSHARE’ does not exist.
> >
> > How does it get there? AdminClient.describeConsumerGroups
> > is complicated.
> >
> > First, it uses the ConsumerGroupDescribe RPC which responds
> > with GROUP_ID_NOT_FOUND (69) and an empty error message.
> > The broker *could* fill in the error message to help with this situation
> > but I don’t like that as a solution. Seems quite brittle.
> >
> > Then, it uses the DescribeGroups RPC in case it’s a classic consumer
> > group. This responds with error code NONE (0) and makes the group
> > look like a Dead consumer group. There is no error message field
> > in that RPC at all, so we don’t have the option of using an error
> > message to disambiguate.
> >
> > So, `kafka-consumer-groups.sh` thinks that it’s dealing with a dead
> > consumer group and its output makes sense.
> >
> > My preferred course of action here is as you suggest to introduce
> > the new error code, INVALID_GROUP_TYPE. If you use any of the following
> > RPCs with the wrong type of group, you get this response:
> >
> > * ConsumerGroupDescribe
> > * ShareGroupDescribe
> > * ConsumerGroupHeartbeat
> > * ShareGroupHeartbeat
> >
> > The remaining RPCs for consumer groups, such as ListOffsets and
> > TxnOffsetCommit continue to use `GROUP_ID_NOT_FOUND`.
> >
> > Does that make sense? Any further comments?
> >
> > Thanks,
> > Andrew
> >
> > > On 23 Jul 2024, at 17:26, Lianet M.  wrote:
> > >
> > > Hello Andrew,
> > >
> > > Bringing here the point I surfaced on the KIP-1071 thread:
> > >
> > > I wonder if at this point, where we're getting several new group types
> > >> added, each with RPCs that are supposed to include groupId of a
> certain
> > >> type, we should be more explicit about this situation. Maybe a kind of
> > >> INVALID_GROUP_TYPE (group exists but not with a valid type for this
> RPC) vs
> > >> a GROUP_ID_NOT_FOUND (group does not exist).  Those errors would be
> > >> consistently used across consumer, share, and streams RPCs whenever
> the
> > >> group id is not of the expected type.
> > >
> > >
> > > I noticed it on KIP-1071 but totally agree with you that it would make
> more
> > > sense to consider it here.
> > >
> > > LM9. Regarding the point of introducing a new INVALID_GROUP_TYPE vs
> reusing
> > > the existing INCONSISTENT_PROTOCOL_TYPE. My concern with reusing
> > > INCONSISTENT_GROUP_PROTOCOL for errors with the group ID is that it

[jira] [Created] (KAFKA-17528) Remove whitelist/blacklist from JMXReporter

2024-09-11 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-17528:
--

 Summary: Remove whitelist/blacklist from JMXReporter
 Key: KAFKA-17528
 URL: https://issues.apache.org/jira/browse/KAFKA-17528
 Project: Kafka
  Issue Type: Sub-task
Reporter: Chia-Ping Tsai
Assignee: Chia-Ping Tsai






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-17529) Remove blacklist from MM2

2024-09-11 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-17529:
--

 Summary: Remove blacklist from MM2
 Key: KAFKA-17529
 URL: https://issues.apache.org/jira/browse/KAFKA-17529
 Project: Kafka
  Issue Type: Sub-task
Reporter: Chia-Ping Tsai
Assignee: Chia-Ping Tsai






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-17530) Remove blacklist/whitelist from ReplaceField

2024-09-11 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-17530:
--

 Summary: Remove blacklist/whitelist from ReplaceField
 Key: KAFKA-17530
 URL: https://issues.apache.org/jira/browse/KAFKA-17530
 Project: Kafka
  Issue Type: Sub-task
Reporter: Chia-Ping Tsai
Assignee: Chia-Ping Tsai






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-17392) Remove whitelist in ConsoleConsumerOptions

2024-09-11 Thread Chia-Ping Tsai (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-17392?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Chia-Ping Tsai resolved KAFKA-17392.

Fix Version/s: 4.0.0
   Resolution: Fixed

> Remove whitelist in ConsoleConsumerOptions
> --
>
> Key: KAFKA-17392
> URL: https://issues.apache.org/jira/browse/KAFKA-17392
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: PoAn Yang
>Assignee: Ming-Yen Chung
>Priority: Minor
> Fix For: 4.0.0
>
>
> The 
> [whitelist|https://github.com/apache/kafka/blob/944c1353a925858ea9bd9024a713cd7301f55133/tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptions.java#L79-L83]
>  is deprecated option in ConsoleConsumerOptions. We can consider to remove it 
> in 4.0.
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-629%3A+Use+racially+neutral+terms+in+our+codebase



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [DISCUSS] Regarding Old PRs

2024-09-11 Thread David Arthur
Hey folks, I wanted to revive this old thread.

I'd like to do the following:

* Change our stale workflow to start with the oldest PRs and move forward
* Enable closing of stale PRs (over 120 days)

Here's a patch with these changes:
https://github.com/apache/kafka/pull/17166
Docs for actions/stale: https://github.com/actions/stale

Cheers,
David A

On Sat, Jun 10, 2023 at 2:53 AM David Jacot  wrote:

> Thanks, David. I left a few comments in the PR.
>
> -David
>
> Le ven. 9 juin 2023 à 15:31, David Arthur  .invalid>
> a écrit :
>
> > Hey all, I just wanted to bump this one more time before I merge this PR
> > (thanks for the review, Josep!). I'll merge it at the end of the day
> today
> > unless anyone has more feedback.
> >
> > Thanks!
> > David
> >
> > On Wed, Jun 7, 2023 at 8:50 PM David Arthur  wrote:
> >
> > > I filed KAFKA-15073 for this. Here is a patch
> > > https://github.com/apache/kafka/pull/13827. This simply adds a "stale"
> > > label to PRs with no activity in the last 90 days. I figure that's a
> good
> > > starting point.
> > >
> > > As for developer workflow, the "stale" action is quite flexible in how
> it
> > > finds candidate PRs to mark as stale. For example, we can exclude PRs
> > that
> > > have an Assignee, or a particular set of labels. Docs are here
> > > https://github.com/actions/stale
> > >
> > > -David
> > >
> > >
> > > On Wed, Jun 7, 2023 at 2:36 PM Josep Prat  >
> > > wrote:
> > >
> > > > Thanks David!
> > > >
> > > > ———
> > > > Josep Prat
> > > >
> > > > Aiven Deutschland GmbH
> > > >
> > > > Alexanderufer 3-7, 10117 Berlin
> > > >
> > > > Amtsgericht Charlottenburg, HRB 209739 B
> > > >
> > > > Geschäftsführer: Oskari Saarenmaa & Hannu Valtonen
> > > >
> > > > m: +491715557497
> > > >
> > > > w: aiven.io
> > > >
> > > > e: josep.p...@aiven.io
> > > >
> > > > On Wed, Jun 7, 2023, 20:28 David Arthur  > > > .invalid>
> > > > wrote:
> > > >
> > > > > Hey all, I started poking around at Github actions on my fork.
> > > > >
> > > > > https://github.com/mumrah/kafka/actions
> > > > >
> > > > > I'll post a PR if I get it working and we can discuss what kind of
> > > > settings
> > > > > we want (or if we want it all)
> > > > >
> > > > > -David
> > > > >
> > > > > On Tue, Jun 6, 2023 at 1:18 PM Chris Egerton
>  > >
> > > > > wrote:
> > > > >
> > > > > > Hi Josep,
> > > > > >
> > > > > > Thanks for bringing this up! Will try to keep things brief.
> > > > > >
> > > > > > I'm generally in favor of this initiative. A couple ideas that I
> > > really
> > > > > > liked: requiring a component label (producer, consumer, connect,
> > > > streams,
> > > > > > etc.) before closing, and disabling auto-close (i.e.,
> automatically
> > > > > tagging
> > > > > > PRs as stale, but leaving it to a human being to actually close
> > > them).
> > > > > >
> > > > > > We might replace the "stale" label with a "close-by-" label
> > so
> > > > that
> > > > > > it becomes even easier for us to find the PRs that are ready to
> be
> > > > closed
> > > > > > (as opposed to the ones that have just been labeled as stale
> > without
> > > > > giving
> > > > > > the contributor enough time to respond).
> > > > > >
> > > > > > I've also gone ahead and closed some of my stale PRs. Others I've
> > > > > > downgraded to draft to signal that I'd like to continue to pursue
> > > them,
> > > > > but
> > > > > > have to iron out merge conflicts first. For the last ones, I'll
> > ping
> > > > for
> > > > > > review.
> > > > > >
> > > > > > One question that came to mind--do we want to distinguish between
> > > > regular
> > > > > > and draft PRs? I'm guessing not, since they still add up to the
> > total
> > > > PR
> > > > > > count against the project, but since they do also implicitly
> signal
> > > > that
> > > > > > they're not intended for review (yet) it may be friendlier to
> leave
> > > > them
> > > > > > up.
> > > > > >
> > > > > > Cheers,
> > > > > >
> > > > > > Chris
> > > > > >
> > > > > > On Tue, Jun 6, 2023 at 10:18 AM Mickael Maison <
> > > > mickael.mai...@gmail.com
> > > > > >
> > > > > > wrote:
> > > > > >
> > > > > > > Hi Josep,
> > > > > > >
> > > > > > > Thanks for looking into this. This is clearly one aspect where
> we
> > > > need
> > > > > > > to improve.
> > > > > > >
> > > > > > > We had a similar initiative last year
> > > > > > > (
> > https://lists.apache.org/thread/66yj9m6tcyz8zqb3lqlbnr386bqwsopt)
> > > > and
> > > > > > > we closed many PRs. Unfortunately we did not follow up with a
> > > process
> > > > > > > or automation and we are back to the same situation.
> > > > > > >
> > > > > > > Manually reviewing all these PRs is a huge task, so I think we
> > > should
> > > > > > > at least partially automate it. I'm not sure if we should
> > manually
> > > > > > > review the oldest PRs (pre 2020). There's surely many
> interesting
> > > > > > > things but I wonder if we should instead focus on the more
> recent
> > > > ones
> > > > > > > as they have a higher chance of 1) still making sens

[jira] [Resolved] (KAFKA-17231) Share consumer node latency metrics

2024-09-11 Thread Chia-Ping Tsai (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-17231?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Chia-Ping Tsai resolved KAFKA-17231.

Fix Version/s: 4.0.0
   Resolution: Fixed

> Share consumer node latency metrics
> ---
>
> Key: KAFKA-17231
> URL: https://issues.apache.org/jira/browse/KAFKA-17231
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Andrew Schofield
>Assignee: Andrew Schofield
>Priority: Major
> Fix For: 4.0.0
>
>
> This is the share consumer equivalent of 
> https://github.com/apache/kafka/pull/16755.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-17400) Introduce a purgatory to deal with share fetch requests that cannot be completed instantaneously

2024-09-11 Thread Jun Rao (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-17400?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jun Rao resolved KAFKA-17400.
-
Fix Version/s: 4.0.0
   Resolution: Fixed

merged the PR to trunk.

> Introduce a purgatory to deal with share fetch requests that cannot be 
> completed instantaneously
> 
>
> Key: KAFKA-17400
> URL: https://issues.apache.org/jira/browse/KAFKA-17400
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Abhinav Dixit
>Assignee: Abhinav Dixit
>Priority: Major
> Fix For: 4.0.0
>
>
> h1. When record lock partition limit is reached, the ShareFetch should wait 
> for up to MaxWaitMs for records to be released



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [DISCUSS] Regarding Old PRs

2024-09-11 Thread Josep Prat
Hi David,

I checked the PR, looks good! And I think the times set are adequate.

Best,

--
Josep Prat
Open Source Engineering Director, Aiven
josep.p...@aiven.io   |   +491715557497 | aiven.io
Aiven Deutschland GmbH
Alexanderufer 3-7, 10117 Berlin
Geschäftsführer: Oskari Saarenmaa, Hannu Valtonen,
Anna Richardson, Kenneth Chen
Amtsgericht Charlottenburg, HRB 209739 B

On Wed, Sep 11, 2024, 19:51 David Arthur  wrote:

> Hey folks, I wanted to revive this old thread.
>
> I'd like to do the following:
>
> * Change our stale workflow to start with the oldest PRs and move forward
> * Enable closing of stale PRs (over 120 days)
>
> Here's a patch with these changes:
> https://github.com/apache/kafka/pull/17166
> Docs for actions/stale: https://github.com/actions/stale
>
> Cheers,
> David A
>
> On Sat, Jun 10, 2023 at 2:53 AM David Jacot  wrote:
>
> > Thanks, David. I left a few comments in the PR.
> >
> > -David
> >
> > Le ven. 9 juin 2023 à 15:31, David Arthur  > .invalid>
> > a écrit :
> >
> > > Hey all, I just wanted to bump this one more time before I merge this
> PR
> > > (thanks for the review, Josep!). I'll merge it at the end of the day
> > today
> > > unless anyone has more feedback.
> > >
> > > Thanks!
> > > David
> > >
> > > On Wed, Jun 7, 2023 at 8:50 PM David Arthur  wrote:
> > >
> > > > I filed KAFKA-15073 for this. Here is a patch
> > > > https://github.com/apache/kafka/pull/13827. This simply adds a
> "stale"
> > > > label to PRs with no activity in the last 90 days. I figure that's a
> > good
> > > > starting point.
> > > >
> > > > As for developer workflow, the "stale" action is quite flexible in
> how
> > it
> > > > finds candidate PRs to mark as stale. For example, we can exclude PRs
> > > that
> > > > have an Assignee, or a particular set of labels. Docs are here
> > > > https://github.com/actions/stale
> > > >
> > > > -David
> > > >
> > > >
> > > > On Wed, Jun 7, 2023 at 2:36 PM Josep Prat
> 
>  > >
> > > > wrote:
> > > >
> > > > > Thanks David!
> > > > >
> > > > > ———
> > > > > Josep Prat
> > > > >
> > > > > Aiven Deutschland GmbH
> > > > >
> > > > > Alexanderufer 3-7, 10117 Berlin
> > > > >
> > > > > Amtsgericht Charlottenburg, HRB 209739 B
> > > > >
> > > > > Geschäftsführer: Oskari Saarenmaa & Hannu Valtonen
> > > > >
> > > > > m: +491715557497
> > > > >
> > > > > w: aiven.io
> > > > >
> > > > > e: josep.p...@aiven.io
> > > > >
> > > > > On Wed, Jun 7, 2023, 20:28 David Arthur  > > > > .invalid>
> > > > > wrote:
> > > > >
> > > > > > Hey all, I started poking around at Github actions on my fork.
> > > > > >
> > > > > > https://github.com/mumrah/kafka/actions
> > > > > >
> > > > > > I'll post a PR if I get it working and we can discuss what kind
> of
> > > > > settings
> > > > > > we want (or if we want it all)
> > > > > >
> > > > > > -David
> > > > > >
> > > > > > On Tue, Jun 6, 2023 at 1:18 PM Chris Egerton
> >  > > >
> > > > > > wrote:
> > > > > >
> > > > > > > Hi Josep,
> > > > > > >
> > > > > > > Thanks for bringing this up! Will try to keep things brief.
> > > > > > >
> > > > > > > I'm generally in favor of this initiative. A couple ideas that
> I
> > > > really
> > > > > > > liked: requiring a component label (producer, consumer,
> connect,
> > > > > streams,
> > > > > > > etc.) before closing, and disabling auto-close (i.e.,
> > automatically
> > > > > > tagging
> > > > > > > PRs as stale, but leaving it to a human being to actually close
> > > > them).
> > > > > > >
> > > > > > > We might replace the "stale" label with a "close-by-"
> label
> > > so
> > > > > that
> > > > > > > it becomes even easier for us to find the PRs that are ready to
> > be
> > > > > closed
> > > > > > > (as opposed to the ones that have just been labeled as stale
> > > without
> > > > > > giving
> > > > > > > the contributor enough time to respond).
> > > > > > >
> > > > > > > I've also gone ahead and closed some of my stale PRs. Others
> I've
> > > > > > > downgraded to draft to signal that I'd like to continue to
> pursue
> > > > them,
> > > > > > but
> > > > > > > have to iron out merge conflicts first. For the last ones, I'll
> > > ping
> > > > > for
> > > > > > > review.
> > > > > > >
> > > > > > > One question that came to mind--do we want to distinguish
> between
> > > > > regular
> > > > > > > and draft PRs? I'm guessing not, since they still add up to the
> > > total
> > > > > PR
> > > > > > > count against the project, but since they do also implicitly
> > signal
> > > > > that
> > > > > > > they're not intended for review (yet) it may be friendlier to
> > leave
> > > > > them
> > > > > > > up.
> > > > > > >
> > > > > > > Cheers,
> > > > > > >
> > > > > > > Chris
> > > > > > >
> > > > > > > On Tue, Jun 6, 2023 at 10:18 AM Mickael Maison <
> > > > > mickael.mai...@gmail.com
> > > > > > >
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Hi Josep,
> > > > > > > >
> > > > > > > > Thanks for looking into this. Th

[jira] [Resolved] (KAFKA-17401) Add config for purge interval for Share Fetch Purgatory

2024-09-11 Thread Abhinav Dixit (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-17401?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Abhinav Dixit resolved KAFKA-17401.
---
Fix Version/s: 4.0.0
   Resolution: Fixed

> Add config for purge interval for Share Fetch Purgatory
> ---
>
> Key: KAFKA-17401
> URL: https://issues.apache.org/jira/browse/KAFKA-17401
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Abhinav Dixit
>Assignee: Abhinav Dixit
>Priority: Major
> Fix For: 4.0.0
>
>




--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-17290) Add integration test for ShareGroupFetch/Acknowledge requests

2024-09-11 Thread Chirag Wadhwa (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-17290?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Chirag Wadhwa resolved KAFKA-17290.
---
Resolution: Fixed

The task has been completed and the PR has been merged

> Add integration test for ShareGroupFetch/Acknowledge requests
> -
>
> Key: KAFKA-17290
> URL: https://issues.apache.org/jira/browse/KAFKA-17290
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Andrew Schofield
>Assignee: Chirag Wadhwa
>Priority: Major
>




--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Jenkins build is still unstable: Kafka » Kafka Branch Builder » 3.8 #88

2024-09-11 Thread Apache Jenkins Server
See 




[jira] [Created] (KAFKA-17531) Remove deprecates methods JoinWindows#of and #grace

2024-09-11 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-17531:
---

 Summary: Remove deprecates methods JoinWindows#of and #grace
 Key: KAFKA-17531
 URL: https://issues.apache.org/jira/browse/KAFKA-17531
 Project: Kafka
  Issue Type: Sub-task
  Components: streams
Affects Versions: 5.0.0
Reporter: Matthias J. Sax


Deprecated in 3.0: 
[https://cwiki.apache.org/confluence/display/KAFKA/KIP-633%3A+Deprecate+24-hour+Default+Grace+Period+for+Windowed+Operations+in+Streams]
 
 * JoinWindows#of
 * JoinWindows#grace

While we did remove the other methods deprecated via KIP-633 in 4.0 via 
KAFKA-16332 already, we could not remove these two because of 
https://issues.apache.org/jira/browse/KAFKA-13813. We an only pickup this 
ticket for 5.0 if KAFKA-13813 is resolved.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Jenkins build is still unstable: Kafka » Kafka Branch Builder » 3.9 #65

2024-09-11 Thread Apache Jenkins Server
See 




[jira] [Resolved] (KAFKA-16332) Remove Deprecated builder methods for Time/Session/Join/SlidingWindows

2024-09-11 Thread Matthias J. Sax (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-16332?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Matthias J. Sax resolved KAFKA-16332.
-
Resolution: Fixed

> Remove Deprecated builder methods for Time/Session/Join/SlidingWindows
> --
>
> Key: KAFKA-16332
> URL: https://issues.apache.org/jira/browse/KAFKA-16332
> Project: Kafka
>  Issue Type: Sub-task
>  Components: streams
>Reporter: Matthias J. Sax
>Assignee: Kuan Po Tseng
>Priority: Blocker
> Fix For: 4.0.0
>
>
> Deprecated in 3.0: 
> [https://cwiki.apache.org/confluence/display/KAFKA/KIP-633%3A+Deprecate+24-hour+Default+Grace+Period+for+Windowed+Operations+in+Streams]
>  
>  * TimeWindows#of
>  * TimeWindows#grace
>  * SessionWindows#with
>  * SessionWindows#grace
>  * -JoinWindows#of- (cf https://issues.apache.org/jira/browse/KAFKA-17531)
>  * -JoinWindows#grace- (cf https://issues.apache.org/jira/browse/KAFKA-17531)
>  * SlidingWindows#withTimeDifferencAndGrace
> Me might want to hold-off to cleanup JoinWindows due to 
> https://issues.apache.org/jira/browse/KAFKA-13813 (open for discussion)



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-15073) Automation for old/inactive PRs

2024-09-11 Thread David Arthur (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-15073?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

David Arthur resolved KAFKA-15073.
--
Resolution: Fixed

Resolving this since we've merged the change to actually close old PRs

> Automation for old/inactive PRs
> ---
>
> Key: KAFKA-15073
> URL: https://issues.apache.org/jira/browse/KAFKA-15073
> Project: Kafka
>  Issue Type: Improvement
>  Components: build
>Reporter: David Arthur
>Priority: Minor
>
> Following from a discussion on the mailing list. It would be nice to 
> automatically triage inactive PRs. There are currently over 1000 open PRs. 
> Most likely a majority of these will not ever be merged and should be closed.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [VOTE] KIP-1052: Enable warmup in producer performance test

2024-09-11 Thread Justine Olshan
+1 (binding) from me.

Thanks,
Justine

On Wed, Sep 4, 2024 at 3:22 PM Welch, Matt  wrote:

> Hi Kafka devs,
>
> Bumping this VOTE thread again for visibility.
>
> Thanks,
> Matt
>
> -Original Message-
> From: Welch, Matt 
> Sent: Friday, August 23, 2024 4:26 PM
> To: dev@kafka.apache.org
> Subject: RE: [VOTE] KIP-1052: Enable warmup in producer performance test
>
> Hi Kafka devs,
>
> Bumping this VOTE thread for visibility.
>
> Thanks,
> Matt
>
> -Original Message-
> From: Federico Valeri 
> Sent: Monday, August 19, 2024 12:38 AM
> To: dev@kafka.apache.org
> Subject: Re: [VOTE] KIP-1052: Enable warmup in producer performance test
>
> Hi Matt, +1 (non binding) from me. Thanks!
>
> Just a suggestion: I think that the following output line does not add
> much value and could be removed.
>
> "Warmup first 10 records. Steady-state results will print after the
> complete-test summary."
>
> On Wed, Aug 14, 2024 at 8:06 PM Welch, Matt  wrote:
> >
> >
> > Hi all,
> >
> > It seems discussion has been quiet for a couple of weeks so I'd like
> > to call a vote on KIP-1052
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-1052%3A+Enable+w
> > armup+in+producer+performance+test
> >
> > Thanks,
> > Matt Welch
> >
>


Re: [VOTE] KIP-1085: Fix leaking *_DOC variables in StreamsConfig

2024-09-11 Thread 黃竣陽
Hi everyone,

The vote is now closed.
The KIP has been accepted with 4 binding votes from Lucas, Matthias,
Luke, and Chia-Ping.
Thanks to everyone for participating!

Regards,
Jiunn-Yang

> Lucas Brutschy  於 2024年9月10日 晚上10:14 寫道:
> 
> +1 (binding). Thanks for the PR!
> 
> On Fri, Sep 6, 2024 at 4:20 AM Luke Chen  wrote:
>> 
>> +1 from me. Thanks.
>> 
>> On Fri, Sep 6, 2024 at 1:04 AM Matthias J. Sax  wrote:
>> 
>>> +1 (binding)
>>> 
>>> On 9/5/24 06:34, Chia-Ping Tsai wrote:
 +1 (binding)
 
 黃竣陽  於 2024年9月5日 週四 下午9:13寫道:
 
> Hello everyone,
> 
> I would like to call for a vote on KIP-1085: <
> 
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1085%3A+Fix+leaking+*_DOC+variables+in+StreamsConfig
>> 
> 
> The discussion can be found here:
> 
> 
> Best regards,
> Jiunn-Yang
 
>>> 



Jenkins build is still unstable: Kafka » Kafka Branch Builder » trunk #3295

2024-09-11 Thread Apache Jenkins Server
See 




[jira] [Created] (KAFKA-17532) Move ShareGroupConfig to share module

2024-09-11 Thread Chirag Wadhwa (Jira)
Chirag Wadhwa created KAFKA-17532:
-

 Summary: Move ShareGroupConfig to share module
 Key: KAFKA-17532
 URL: https://issues.apache.org/jira/browse/KAFKA-17532
 Project: Kafka
  Issue Type: Sub-task
Reporter: Chirag Wadhwa


Move ShareGroupConfig to share module



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Problematic new HWM increment behaviour introduced by KIP-207 and KIP-966

2024-09-11 Thread Francois Visconte
We identified a bug/new behaviour that would lead to consumer lagging for a
long time and ListOffsets requests failing during that time frame.

While the ListOffsets requests failure is expected and has been introduced
by KIP-207
,
the problematic behavior is more about the inability to increment
highWatermark and the consequence of having lagging consumers.


Here is the situation


   -

   We have a topic with min.isr=2


   -

   We have a partition on broker 16, 17 and 18
   -

   Leader for this partition is broker 17




   1.

   Broker 18 failed. Partition has 2 ISRs
   2.

   Broker 16 failed. Partition has 1 ISR (17)
   3.

   Broker 7 has LEO higher than HWM:

[Broker id=17] Leader topic-86 with topic id Some(yFhPOnPsRDiYHgfF2bR2aQ)
starts at leader epoch 7 from offset 3067193660 with partition epoch 11,
high watermark 3067191497, ISR [10017], adding replicas [] and removing
replicas [] (under-min-isr). Previous leader Some(10017) and previous
leader epoch was 6.

At this point producers cannot produce to topic-86 partition because there
is only one ISR, which is expected behavior.

But it seems that KIP-207 prevent answering to ListOffsets requests here



// Only consider throwing an error if we get a client request
(isolationLevel is defined) and the high watermark

// is lagging behind the start offset

val maybeOffsetsError: Option[ApiException] = leaderEpochStartOffsetOpt

  .filter(epochStart => isolationLevel.isDefined && epochStart >
localLog.highWatermark)

  .map(epochStart => Errors.OFFSET_NOT_AVAILABLE.exception(s"Failed to
fetch offsets for " +

s"partition $topicPartition with leader $epochLogString as this
partition's " +

s"high watermark (${localLog.highWatermark}) is lagging behind the " +

s"start offset from the beginning of this epoch ($epochStart)."))


It seems that the path to get to the HWM being stuck for so long was
introduced in preparation of KIP-966
,
see this ticket  and PR
.

As a result:

   -

   The stuck HWM in the above scenario can also mean that a small part of
   messages isn't readable by consumers even though it was in the past.
   -

   In case of truncation, the HWM might still go backwards. This is still
   possible even with min.ISR, although it should be rare.



Regards, F.