Re: [VOTE] 3.9.0 RC2

2024-10-23 Thread Anton Agestam
Hi Colin,

I understand your perspective that this is a bug in the client, but I
disagree that this is something that's established, or inferrable from
existing specification and implementation.

The behavior of exposing the implicit defaults on the wire protocol really
is new. The only scenario that they have come into play prior to this
change is when parsing an entity version that is not the latest version
that the parsing code base supports -- and -- when that code base models
all versions of an API with the same entity. In this scenario the implicit
defaults must be there to fill in newly added fields with _some_ value in
lack of being able to read from the message. However, for an implementation
that doesn't reuse the same class for every version, there has never been
any need to use the implicit defaults, as when each version has a dedicated
class it doesn't need to define any fields for values that are added in
future versions. We have a comprehensive compatibility test suite in kio
that proves this is the case, we can parse and serialize the full protocol
up to version 3.8.0 (every version of every message). This would not have
been possible if the implicit defaults were semantically required, because
we do not use them.

The reason that the above hasn't become visible until now, as I mentioned
in the bug report, is that the newly introduced protocol message is the
first one (of all message versions in the whole protocol) to make use of
this construct: a nested entity field where not all nested fields have an
explicit default value.

Further, the new behavior is not in accordance with what is specified. The
implicit defaults are really only documented to be used in the above
mentioned scenario, with the following prerequisite
:
"Any fields in the message object that are not present in the version that
you are deserializing will be reset to default values". That prerequisite
is not fulfilled here, so this cannot be said to be in accordance with what
is specified. The behavior occurs even though I am parsing a message of the
_same_, latest, version as the schema of the model.

In summary, this is new behavior, and it is unspecified behavior.

On top of that, I think this is also bad API design, as it forces
cumbersome and strange semantics into client implementations. In the
context of parsing a model that has more fields than the data it is
parsing, it makes sense to have implicit defaults. However, when the model
has the same amount of fields as there are values in the data, this does
not make sense. Why should we force client implementations to induce non
defaults into their data structures? The empty string for a hostname, and
zero for a port are clearly not good defaults, both are really invalid
values.

The choice of implicitly introducing this behavior at the protocol level in
this way will most likely mean that this becomes a quirk of the protocol
forever. I think this situation is avoidable, and I think this is a
decision that should not be made silently without proper specification and
design process to make sure it is really the right decision.

BR,
Anton

Den mån 21 okt. 2024 kl 23:12 skrev Colin McCabe :

> Hi all,
>
> I have posted a new release candidate, RC3. See the RC3 thread.
>
> best,
> Colin
>
> On Mon, Oct 21, 2024, at 11:31, Colin McCabe wrote:
> > Hi Anton,
> >
> > I replied on the JIRA. I do not think this is a bug, you just failed to
> > account for implicit defaults in your protocol code. That is, 0 is the
> > default of numeric fields if no other default is specified, etc.
> >
> > best,
> > Colin
> >
> > On Mon, Oct 21, 2024, at 08:07, Anton Agestam wrote:
> >> Hi everyone,
> >>
> >> I have found a protocol serialization bug that surfaces only with one of
> >> the entities introduced for KIP-853 (UpdateRaftVoterResponse).
> >>
> >> Due to the irreversible implications this might have once merged, I'd
> argue
> >> that this needs to be considered a release blocker.
> >>
> >> https://issues.apache.org/jira/browse/KAFKA-17845
> >>
> >> BR,
> >> Anton
> >>
> >> Den tors 10 okt. 2024 kl 23:16 skrev Colin McCabe :
> >>
> >>> This is the second candidate for the release of Apache Kafka 3.9.0. I
> have
> >>> titled it rc2 since I had an rc1 which got very far, even to the point
> of
> >>> pushing tags and docker images, before I spotted an issue. So rather
> than
> >>> mutate the tags, I decided to skip over rc1.
> >>>
> >>> - This is a major release, the final one in the 3.x line. (There may of
> >>> course be other minor releases in this line, such as 3.9.1.)
> >>> - Tiered storage will be considered production-ready in this release.
> >>> - This will be the final major release to feature the deprecated
> ZooKeeper
> >>> mode.
> >>>
> >>> This release includes the following KIPs:
> >>> - KIP-853: Support dynamically changing KRaft controller membership
> >>> - KIP-1057: 

Re: doc clarification about meesage format

2024-10-23 Thread Greg Harris
Hi Xiang,

Thanks for your question! That sentence is a justification for why the
partitionLeaderEpoch field is not included in the CRC.

If you mutate fields which are included in a CRC, you need to recompute the
CRC value. See [1] for mutating the maxTimestamp. Compare that with [2] for
setting the partitionLeaderEpoch.
This makes setting the partitionLeaderEpoch faster than setting the max
timestamp. And because setting the partitionLeaderEpoch happens on every
Produce request, it was optimized in the protocol design.
It does have the tradeoff that corruptions in the partitionLeaderEpoch are
not detected by the CRC, but someone decided this was worth the
optimization to the Produce flow.

I don't have more information on why this optimization was made for
partitionLeaderEpoch and not maxTimestamp.

Hope this helps,
Greg

[1]
https://github.com/apache/kafka/blob/2d896d9130f121e75ccba2d913bdffa358cf3867/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java#L371-L382
[2]
https://github.com/apache/kafka/blob/2d896d9130f121e75ccba2d913bdffa358cf3867/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java#L385-L387


On Tue, Oct 22, 2024 at 7:51 PM Xiang Zhang 
wrote:

> Hi all,
>
> I am reading official doc here:
> https://kafka.apache.org/documentation/#messageformat, and I could not
> fully understand it. If someone can clarify it for me, it would be much
> appreciated. The sentence is
>
> The partition leader epoch field is not included in the CRC computation to
> avoid the need to recompute the CRC when this field is assigned for every
> batch that is received by the broker.
>
> I just don’t really get what the highlight part is trying to say.
>
> Regards,
> Xiang Zhang
>


Re: [DISCUSS] KIP-1099: Extend kafka-consumer-groups command line tool to support new consumer group

2024-10-23 Thread Lucas Brutschy
Hi Frank,

thanks for the KIP!

1) For consistency, should we do the same for
kafka-share-groups.sh, ShareGroupDescription, etc. ? Even if we do not
implement it right now if the share group implementation may still be
incomplete, it may make sense to include it in the KIP.

2) Why call it CONSUMER-EPOCH, not MEMBER-EPOCH? That would seem more
consistent.

Cheers,
Lucas

On Wed, Oct 23, 2024 at 2:41 PM Frank Yang  wrote:

> Hi all,
>
> I would like to kick off the discussion of KIP-1099. This KIP enhances the
> kafka-consumer-groups tools to include state which is introduced by KIP-848.
>
> KIP-1099: Extend kafka-consumer-groups command line tool to support new
> consumer group - Apache Kafka - Apache Software Foundation
> 
> cwiki.apache.org
> 
> [image: favicon.ico]
> 
> 
>
> Thank you,
> PoAn
>


[jira] [Created] (KAFKA-17860) Remove log4j-appender module

2024-10-23 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-17860:
--

 Summary: Remove log4j-appender module
 Key: KAFKA-17860
 URL: https://issues.apache.org/jira/browse/KAFKA-17860
 Project: Kafka
  Issue Type: Sub-task
Reporter: Chia-Ping Tsai
Assignee: TengYao Chi


as title, we need to remove it first in order to upgrade log4j



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


[DISCUSS] KIP-1099: Extend kafka-consumer-groups command line tool to support new consumer group

2024-10-23 Thread Frank Yang
Hi all,

I would like to kick off the discussion of KIP-1099. This KIP enhances the 
kafka-consumer-groups tools to include state which is introduced by KIP-848.

https://cwiki.apache.org/confluence/display/KAFKA/KIP-1099%3A+Extend+kafka-consumer-groups+command+line+tool+to+support+new+consumer+group
Thank you,
PoAn

[jira] [Created] (KAFKA-17861) Serialize with ByteBuffer instead of byte[]

2024-10-23 Thread Sarah Hennenkamp (Jira)
Sarah Hennenkamp created KAFKA-17861:


 Summary: Serialize with ByteBuffer instead of byte[]
 Key: KAFKA-17861
 URL: https://issues.apache.org/jira/browse/KAFKA-17861
 Project: Kafka
  Issue Type: Wish
  Components: producer 
Affects Versions: 3.3.2
Reporter: Sarah Hennenkamp


This is a request to consider changing the return value of the 
[Serializer|https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/serialization/Serializer.java]
 from a byte[] to a ByteBuffer. 

 

Understandably folks may balk at this since it's a large lift. However, we've 
noticed a good chunk of memory allocation in our application comes from the 
[KafkaProducer 
serializing|https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java#L1045]
 the key and value pair. Using ByteBuffer could allow this to be off-heap and 
save on garbage collection time.



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


Re: [VOTE] KIP-1054: Support external schemas in JSONConverter

2024-10-23 Thread Greg Harris
Hi Priyanka,

This is definitely an improvement for single-schema Connectors, and simple
to configure.
I think mutable or multiple schemas are fair to consider out-of-scope, as
they are currently served by the existing schema management tooling and
would complicate this design.

I'm also interested to see where knowing the schema ahead-of-time could
improve other parts of the Connector development process, and this could be
the first step in that work.

+1 (binding)

Thanks!
Greg

On Tue, Oct 15, 2024 at 12:29 PM Chris Egerton 
wrote:

> Hi Priyanka,
>
> Sorry for the delay! +1 (binding), thanks for the KIP.
>
> Cheers,
>
> Chris
>
> On Wed, Sep 4, 2024, 01:35 Priyanka K U 
> wrote:
>
> > Hi Everyone,
> >
> > Please go through the proposal and requesting everyone to support with
> > your votes  :
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1054%3A+Support+external+schemas+in+JSONConverter
> > . If you need any further clarifications please post your queries in the
> > discussion thread :
> > https://lists.apache.org/thread/rwxkh1fnbxh5whobsyrt4gystyl9yhc5
> >
> > Thank you,
> >
> > Priyanka
> >
> >
> > From: Priyanka K U 
> > Date: Friday, 28 June 2024 at 3:02 PM
> > To: dev@kafka.apache.org 
> > Subject: [EXTERNAL] [VOTE] KIP-1054: Support external schemas in
> > JSONConverter
> > Hello Everyone,
> >
> > I'd like to start a vote on KIP-1054, which aims to Support external
> > schemas in JSONConverter to Kafka Connect:
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1054%3A+Support+external+schemas+in+JSONConverter
> >
> > Discussion thread -
> > https://lists.apache.org/thread/rwxkh1fnbxh5whobsyrt4gystyl9yhc5
> >
> > Thank you,
> >
> > Priyanka
> >
> >
> >
>


Re: [VOTE] KIP-891: Running multiple versions of Connector plugins

2024-10-23 Thread Mickael Maison
Hi,

+1 (binding), thanks for the KIP!

Mickael

On Tue, Oct 22, 2024 at 10:35 PM Greg Harris
 wrote:
>
> Hi all,
>
> Bumping this thread in advance of the November 20th KIP freeze for 4.0.
>
> Thanks!
> Greg
>
> On Tue, Oct 8, 2024 at 5:03 AM Snehashis  wrote:
>
> > Thanks Greg, Chris and Ashwin
> >
> > For the individual queries,
> >
> > 1. The example is incorrect and both should be treated as exact versions.
> > Thanks for pointing that out.
> > 2. I can see a converter/transformation having a version property but it's
> > probably not likely. That said I don't see why we should not use
> > plugin.version to make this more specific and avoid encountering this in
> > the first place.
> >
> > Will make the appropriate changes to the KIP to include these two points.
> > Thanks
> > Snehashis
> >
> > On Tue, Oct 8, 2024 at 10:33 AM Ashwin 
> > wrote:
> >
> > > Hi Snehasis,
> > >
> > > Thanks for the KIP - +1 (non-binding)
> > >
> > > I just had a question regarding the sample config in
> > >
> > >
> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=235834793#KIP891:RunningmultipleversionsofConnectorplugins-Configuration
> > >
> > > >  "connector.version": "3.8"
> > > > "transforms.flatten-old.version": "3.8.0"
> > >
> > > How will the feature be implemented so that first config treated as an
> > > exact version requirement, while the second one is a
> > > "use-only-if-you-have-it" version ?
> > >
> > > Thanks,
> > > Ashwin
> > >
> > >
> > > On Tue, Oct 8, 2024 at 12:19 AM Chris Egerton 
> > > wrote:
> > >
> > > > Hi Snehashis,
> > > >
> > > > Thanks for the KIP. I'm +1 (binding) but have one more non-blocking
> > > thought
> > > > I wanted to share.
> > > >
> > > > On the off chance that an existing plugin is designed to accept a
> > > "version"
> > > > property, could we either 1) keep passing that property to plugins
> > > instead
> > > > of stripping it, or 2) rename our new property to something like
> > > > "plugin.version"?
> > > >
> > > > Feel free to close the vote (if/when it gets a third binding+1) without
> > > > addressing this if you believe the tradeoffs with the existing design
> > are
> > > > superior.
> > > >
> > > > Cheers,
> > > >
> > > > Chris
> > > >
> > > > On Mon, Oct 7, 2024, 11:22 Greg Harris 
> > > > wrote:
> > > >
> > > > > Hey Snehashis,
> > > > >
> > > > > Thanks for the KIP! +1 (binding)
> > > > >
> > > > > Greg
> > > > >
> > > > > On Fri, Oct 4, 2024 at 10:14 PM Snehashis 
> > > > > wrote:
> > > > >
> > > > > > Hi everyone
> > > > > >
> > > > > > I would like to call a vote for KIP-891. Please take a moment to
> > > review
> > > > > the
> > > > > > proposal and submit your vote. Special thanks to Greg who helped to
> > > > > expand
> > > > > > this to make it much more broadly useful, and everyone else who
> > > > > > participated in both discussion threads.
> > > > > >
> > > > > > KIP
> > > > > > KIP-891: Running multiple versions of Connector plugins - Apache
> > > Kafka
> > > > -
> > > > > > Apache Software Foundation
> > > > > > <
> > > > > >
> > > > >
> > > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-891%3A+Running+multiple+versions+of+Connector+plugins
> > > > > > >
> > > > > >
> > > > > > Thanks
> > > > > > Snehashis
> > > > > >
> > > > >
> > > >
> > >
> >


[jira] [Created] (KAFKA-17862) [buffer pool] corruption during buffer reuse from the pool

2024-10-23 Thread Bharath Vissapragada (Jira)
Bharath Vissapragada created KAFKA-17862:


 Summary: [buffer pool] corruption during buffer reuse from the pool
 Key: KAFKA-17862
 URL: https://issues.apache.org/jira/browse/KAFKA-17862
 Project: Kafka
  Issue Type: Bug
  Components: core
Affects Versions: 3.7.1
Reporter: Bharath Vissapragada
 Attachments: client-config.txt

We noticed malformed batches from the Kafka Java client + Redpanda under 
certain conditions that caused excessive client retries and we narrowed it down 
to a client bug related to corruption of buffers reused from the buffer pool. 
We were able to reproduce it with Kafka brokers too, so we are fairly certain 
the bug is on the client.

(Attached the full client config, fwiw)

We narrowed it down to a race condition between produce requests and failed 
batch expiration. If the network flush of produce request races with the 
expiration, the produce batch that the request uses is corrupted, so a 
malformed batch is sent to the broker.

The expiration is triggered by a timeout 
[https://github.com/apache/kafka/blob/2c6fb6c54472e90ae17439e62540ef3cb0426fe3/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java#L392C13-L392C22]

that eventually deallocates the batch
[https://github.com/apache/kafka/blob/2c6fb6c54472e90ae17439e62540ef3cb0426fe3/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java#L773]

adding it back to the buffer pool

[https://github.com/apache/kafka/blob/661bed242e8d7269f134ea2f6a24272ce9b720e9/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java#L1054]

Now it is probably all zeroed out or there is a competing producer that 
requests a new append that reuses this freed up buffer and starts writing to it 
corrupting it's contents.

If there is racing network flush of a produce batch backed with this buffer, a 
corrupt batch is sent to the broker resulting in a CRC mismatch. 

This issue can be easily reproduced in a simulated environment that triggers 
frequent timeouts (eg: lower timeouts) and then use a producer with high-ish 
throughput.



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


[jira] [Resolved] (KAFKA-17568) Rewrite TestPurgatoryPerformance by Java

2024-10-23 Thread Chia-Ping Tsai (Jira)


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

Chia-Ping Tsai resolved KAFKA-17568.

Fix Version/s: 4.0.0
   Resolution: Fixed

> Rewrite TestPurgatoryPerformance by Java
> 
>
> Key: KAFKA-17568
> URL: https://issues.apache.org/jira/browse/KAFKA-17568
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Chia-Ping Tsai
>Assignee: 黃竣陽
>Priority: Major
> Fix For: 4.0.0
>
>




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


[jira] [Resolved] (KAFKA-17774) Add capability for max fetch records in share fetch

2024-10-23 Thread Jun Rao (Jira)


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

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

merged the PR to trunk

> Add capability for max fetch records in share fetch
> ---
>
> Key: KAFKA-17774
> URL: https://issues.apache.org/jira/browse/KAFKA-17774
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Apoorv Mittal
>Assignee: Apoorv Mittal
>Priority: Major
> Fix For: 4.0.0
>
>




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


Re: doc clarification about meesage format

2024-10-23 Thread Xiang Zhang
Thank you Greg for all the knowledge, some follow up questions.

Does partitionLeaderEpoch always reflect the latest leader election or an
old epoch can be allowed ? If it is the first case, then I agree
partitionLeaderEpoch should not be included in CRC computation. But it
raises some new questions for me, which is which roles will check the
checksum and under what circumstances? I am asking this because after the
producing process, any record in the broker log can have an outdated leader
epoch field once leader election happens, right ? Do they get updated ?

Sorry for all the questions, I have been using Kafka for several years and
want to dive deep into it a little bit. I have become more interested and
ready to find out on my own. But still look forward to your thoughts on
this if the questions above do make some sense.


Thanks,
XIang

Greg Harris  于2024年10月24日周四 00:25写道:

> Hi Xiang,
>
> Thanks for your question! That sentence is a justification for why the
> partitionLeaderEpoch field is not included in the CRC.
>
> If you mutate fields which are included in a CRC, you need to recompute the
> CRC value. See [1] for mutating the maxTimestamp. Compare that with [2] for
> setting the partitionLeaderEpoch.
> This makes setting the partitionLeaderEpoch faster than setting the max
> timestamp. And because setting the partitionLeaderEpoch happens on every
> Produce request, it was optimized in the protocol design.
> It does have the tradeoff that corruptions in the partitionLeaderEpoch are
> not detected by the CRC, but someone decided this was worth the
> optimization to the Produce flow.
>
> I don't have more information on why this optimization was made for
> partitionLeaderEpoch and not maxTimestamp.
>
> Hope this helps,
> Greg
>
> [1]
>
> https://github.com/apache/kafka/blob/2d896d9130f121e75ccba2d913bdffa358cf3867/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java#L371-L382
> [2]
>
> https://github.com/apache/kafka/blob/2d896d9130f121e75ccba2d913bdffa358cf3867/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java#L385-L387
>
>
> On Tue, Oct 22, 2024 at 7:51 PM Xiang Zhang 
> wrote:
>
> > Hi all,
> >
> > I am reading official doc here:
> > https://kafka.apache.org/documentation/#messageformat, and I could not
> > fully understand it. If someone can clarify it for me, it would be much
> > appreciated. The sentence is
> >
> > The partition leader epoch field is not included in the CRC computation
> to
> > avoid the need to recompute the CRC when this field is assigned for every
> > batch that is received by the broker.
> >
> > I just don’t really get what the highlight part is trying to say.
> >
> > Regards,
> > Xiang Zhang
> >
>


Re: Running Kafka from IDE

2024-10-23 Thread Pradyumna K
Thank you.

Looks like I too accidentally discovered this approach minutes before your
reply.

I wasn't aware of the DEBUG_SUSPEND_FLAG though. It is useful.

Regards
Pradyumna


Re: Running Kafka from IDE

2024-10-23 Thread Andrew Schofield
Hi Pradyumna,
Yes, you can do this. What I tend to do is to start my broker from a terminal 
window with the
DEBUG_SUSPEND_FLAG and KAFKA_DEBUG environment variables set. This means the 
broker
suspends as it starts and waits for a connection from a debugger. Then I attach 
IntelliJ to the
running broker with my breakpoints set and I can use the debugger.

So, my command line would be something like:

DEBUG_SUSPEND_FLAG=y KAFKA_DEBUG=y bin/kafka-server-start.sh --bootstrap-server 
.

Hope this helps.
Andrew


From: Pradyumna K 
Sent: 24 October 2024 05:59
To: dev@kafka.apache.org 
Subject: Running Kafka from IDE
 
Hello everyone,

Is there a way I could run Kafka from within IntelliJ, in debug mode, and
let producers and consumers connect to it?

(I want to run producers and consumers as separate programs on my computer
and let them connect to this Kafka which is running under debugger, so that
I can put breakpoints in Kafka code in IntelliJ)

Thank you
Pradyumna

Re: [VOTE] 3.9.0 RC3

2024-10-23 Thread Justine Olshan
Hey Colin,

Thanks for running the release.

I checked the keys and scanned the docs
I built from source, used kraft quickstart, ran a transactions workload,
played around with a few other things including running some of the feature
version commands.

I noticed that some of the KIP-1022 work that was meant to make it in this
release was removed.
Without any specifications, the cluster is meant to take the latest version
of all available features. The code to do that was removed. Perhaps there
is a reason, but we should either change the KIP or remedy this.
Not only that, but the ability to configure a feature via the --feature
flag in the storage tool also seems to be removed but the text remains when
running with no arguments. I don't think it is a blocker, but this should
be fixed for 4.0.

I looked at two runs of the system tests and they looked fine to me.

+1 (binding) from me,

Justine

On Tue, Oct 22, 2024 at 1:21 PM Bill Bejeck  wrote:

> Hi Colin,
>
> Thanks for running the release.
>
> I did the following verification steps:
>
>1. Built from source
>2. Validated all checksums
>3. Ran all the unit tests
>4. Pulled both docker images and started up Kafka
>5. Ran the KRaft quick start
>6. Ran the ZK quick start
>7. Ran the Kafka Streams quick start
>8. Spot checked the documentation
>9. Spot checked the javadocs
>
> +1(binding)
>
> Thanks,
> Bill
>
>
>
> On Tue, Oct 22, 2024 at 9:08 AM Federico Valeri 
> wrote:
>
> > Hi, I did the following tests:
> >
> > - Ran all unit and integration tests
> > - Spot checked the documentation
> > - Tried out new Maven dependencies
> > - Tested KIP-853, KIP-950, KIP-956 and KIP-1057
> >
> > +1 (non binding)
> >
> > Thanks
> > Fede
> >
> >
> > On Mon, Oct 21, 2024 at 11:12 PM Colin McCabe 
> wrote:
> > >
> > > This is the third candidate for the release of Apache Kafka 3.9.0. I
> > have titled it rc3 since I had an rc1 which got very far, even to the
> point
> > of pushing tags and docker images, before I spotted an issue. So rather
> > than mutate the tags, I decided to skip over rc1.
> > >
> > > - This is a major release, the final one in the 3.x line. (There may of
> > course be other minor releases in this line, such as 3.9.1.)
> > > - Tiered storage will be considered production-ready in this release.
> > > - This will be the final major release to feature the deprecated
> > ZooKeeper mode.
> > >
> > > This release includes the following KIPs:
> > > - KIP-853: Support dynamically changing KRaft controller membership
> > > - KIP-1057: Add remote log metadata flag to the dump log tool
> > > - KIP-1049: Add config log.summary.interval.ms to Kafka Streams
> > > - KIP-1040: Improve handling of nullable values in InsertField,
> > ExtractField, and other transformations
> > > - KIP-1031: Control offset translation in MirrorSourceConnector
> > > - KIP-1033: Add Kafka Streams exception handler for exceptions
> occurring
> > during processing
> > > - KIP-1017: Health check endpoint for Kafka Connect
> > > - KIP-1025: Optionally URL-encode clientID and clientSecret in
> > authorization header
> > > - KIP-1005: Expose EarliestLocalOffset and TieredOffset
> > > - KIP-950: Tiered Storage Disablement
> > > - KIP-956: Tiered Storage Quotas
> > >
> > > Release notes for the 3.9.0 release:
> > >
> >
> https://dist.apache.org/repos/dist/dev/kafka/3.9.0-rc3/RELEASE_NOTES.html
> > >
> > > *** Please download, test and vote by October 24, 2024.
> > >
> > > Kafka's KEYS file containing PGP keys we use to sign the release:
> > > https://kafka.apache.org/KEYS
> > >
> > > * Release artifacts to be voted upon (source and binary):
> > > https://dist.apache.org/repos/dist/dev/kafka/3.9.0-rc3/
> > >
> > > * Docker release artifacts to be voted upon:
> > > apache/kafka:3.9.0-rc3
> > > apache/kafka-native:3.9.0-rc3
> > >
> > > * Maven artifacts to be voted upon:
> > > https://repository.apache.org/content/groups/staging/org/apache/kafka/
> > >
> > > * Javadoc:
> > > https://dist.apache.org/repos/dist/dev/kafka/3.9.0-rc3/javadoc/
> > >
> > > * Documentation:
> > > https://kafka.apache.org/39/documentation.html
> > >
> > > * Protocol:
> > > https://kafka.apache.org/39/protocol.html
> > >
> > > * Tag to be voted upon (off 3.9 branch) is the 3.9.0-rc3 tag:
> > > https://github.com/apache/kafka/releases/tag/3.9.0-rc3
> > >
> > > * Successful Docker Image Github Actions Pipeline for 3.9 branch:
> > > Docker Build Test Pipeline (JVM):
> > https://github.com/apache/kafka/actions/runs/11448325059
> > > Docker Build Test Pipeline (Native):
> > https://github.com/apache/kafka/actions/runs/11448338981
> > >
> > > Thanks to everyone who helped with this release candidate, either by
> > contributing code, testing, or documentation.
> > >
> > > Regards,
> > > Colin
> >
>