Hi all.

KIP Freeze for the next major release of Apache Kafka was reached last
week.

As of now, 36 KIPs have concluded their voting process and have been
adopted.
These KIPs are targeting 3.0 (unless it's noted otherwise in the release
plan) and their inclusion as new features will be finalized right after
Feature Freeze.

At the high level, out of these 36 KIPs, 11 have been implemented already
and 25 are open or in progress.
Here is the full list of adopted KIPs:

KIP-751: Drop support for Scala 2.12 in Kafka 4.0 (deprecate in 3.0)
KIP-750: Drop support for Java 8 in Kafka 4.0 (deprecate in 3.0)
KIP-746: Revise KRaft Metadata Records
KIP-745: Connect API to restart connector and tasks
KIP-744: Migrate TaskMetadata and ThreadMetadata to an interface with
internal implementation
KIP-743: Remove config value 0.10.0-2.4 of Streams built-in metrics version
config
KIP-741: Change default serde to be null
KIP-740: Clean up public API in TaskId and fix TaskMetadata#taskId()
KIP-738: Removal of Connect's internal converter properties
KIP-734: Improve AdminClient.listOffsets to return timestamp and offset for
the record with the largest timestamp
KIP-733: Change Kafka Streams default replication factor config
KIP-732: Deprecate eos-alpha and replace eos-beta with eos-v2
KIP-730: Producer ID generation in KRaft mode
KIP-726: Make the "cooperative-sticky, range" as the default assignor
KIP-725: Streamlining configurations for WindowedSerializer and
WindowedDeserializer.
KIP-724: Drop support for message formats v0 and v1
KIP-722: Enable connector client overrides by default
KIP-721: Enable connector log contexts in Connect Log4j configuration
KIP-720: Deprecate MirrorMaker v1
KIP-716: Allow configuring the location of the offset-syncs topic with
MirrorMaker2
KIP-715: Expose Committed offset in streams
KIP-709: Extend OffsetFetch requests to accept multiple group ids.
KIP-708: Rack awareness for Kafka Streams
KIP-707: The future of KafkaFuture
KIP-699: Update FindCoordinator to resolve multiple Coordinators at a time
KIP-698: Add Explicit User Initialization of Broker-side State to Kafka
Streams
KIP-695: Further Improve Kafka Streams Timestamp Synchronization
KIP-691: Enhance Transactional Producer Exception Handling
KIP-679: Producer will enable the strongest delivery guarantee by default
KIP-666: Add Instant-based methods to ReadOnlySessionStore
KIP-653: Upgrade log4j to log4j2
KIP-623: Add "internal-topics" option to streams application reset tool
KIP-622: Add currentSystemTimeMs and currentStreamTimeMs to ProcessorContext
KIP-466: Add support for List<T> serialization and deserialization
KIP-405: Kafka Tiered Storage
KIP-390: Support Compression Level

If you notice that a KIP is missing from the list above and should be part
of the release plan for 3.0, please reply below.
The single source of truth remains the official release plan for 3.0, which
you may read at any time here:

https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+3.0.0

Kind reminder that for all the adopted KIPs any required changes to the
documentation are also part of their respective feature.

For the KIPs that are still in progress, please work closely with your
reviewers to make sure that the features are stable and land on time for
Feature Freeze.

The next milestone for the Apache Kafka 3.0 release is Feature Freeze on
June 30th, 2021.

Best regards,
Konstantine

On Fri, Jun 4, 2021 at 3:47 PM Konstantine Karantasis <
konstant...@confluent.io> wrote:

>
> Hi all,
>
> Just a quick reminder that KIP Freeze is next Wednesday, June 9th.
> A vote thread needs to be open for at least 72 hours, so to everyone that
> is working hard on proposals targeting 3.0.0, please make sure that your
> [VOTE] threads are started on time.
>
> Best,
> Konstantine
>
>
> On Wed, May 26, 2021 at 8:10 PM Israel Ekpo <israele...@gmail.com> wrote:
>
>> +1 on the new schedule.
>>
>> On Wed, May 26, 2021 at 8:14 PM Sophie Blee-Goldman
>> <sop...@confluent.io.invalid> wrote:
>>
>> > Ah ok, thanks Konstantine. I won't bug you about every new KIP that
>> comes
>> > in between now and KIP Freeze :P
>> >
>> > +1 on the scheduling changes as well
>> >
>> > On Wed, May 26, 2021 at 4:00 PM David Arthur <mum...@gmail.com> wrote:
>> >
>> > > The new schedule looks good to me, +1
>> > >
>> > > On Wed, May 26, 2021 at 6:29 PM Ismael Juma <ism...@juma.me.uk>
>> wrote:
>> > >
>> > > > Thanks Konstantine, +1 from me.
>> > > >
>> > > > Ismael
>> > > >
>> > > > On Wed, May 26, 2021 at 2:48 PM Konstantine Karantasis
>> > > > <konstant...@confluent.io.invalid> wrote:
>> > > >
>> > > > > Hi all,
>> > > > >
>> > > > > Please find below the updated release plan for the Apache Kafka
>> 3.0.0
>> > > > > release.
>> > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=177046466
>> > > > >
>> > > > > New suggested dates for the release are as follows:
>> > > > >
>> > > > > KIP Freeze is 09 June 2021 (same date as in the initial plan)
>> > > > > Feature Freeze is 30 June 2021 (new date, extended by two weeks)
>> > > > > Code Freeze is 14 July 2021 (new date, extended by two weeks)
>> > > > >
>> > > > > At least two weeks of stabilization will follow Code Freeze.
>> > > > >
>> > > > > The release plan is up to date and currently includes all the
>> > approved
>> > > > KIPs
>> > > > > that are targeting 3.0.0.
>> > > > >
>> > > > > Please let me know if you have any objections with the recent
>> > extension
>> > > > of
>> > > > > Feature Freeze and Code Freeze or any other concerns.
>> > > > >
>> > > > > Regards,
>> > > > > Konstantine
>> > > > >
>> > > >
>> > > --
>> > > David Arthur
>> > >
>> >
>>
>

Reply via email to