Release Notes - Kafka - Version 4.0.0
Below is a summary of the JIRA issues addressed in the 4.0.0
release of Kafka. For full documentation of the release, a guide
to get started, and information about the project, see the
Kafka project site.
Note about upgrades: Please carefully review the
upgrade documentation for this release thoroughly before upgrading
your cluster. The upgrade notes discuss any critical information about
incompatibilities and breaking changes, performance changes, and any
other changes that might impact your production deployment of Kafka.
The documentation for the most recent release can be found at
https://kafka.apache.org/documentation.html.
New Feature
[KAFKA-15561] - Client support for new SubscriptionPattern based subscription
[KAFKA-16658] - Drop `offsets.commit.required.acks` config in 4.0 (deprecate in 3.8)
[KAFKA-17248] - Extend KIP-714 to allow for registering additional metrics with clients
[KAFKA-17693] - Remove testCheckEarliestLocalTimestampVersion and testCheckLatestTieredTimestampVersion
[KAFKA-17885] - Enable clients to rebootstrap based on timeout or error code
[KAFKA-18026] - Allow custom processor wrapping
Improvement
[KAFKA-6197] - Difficult to get to the Kafka Streams javadocs
[KAFKA-7302] - Remove Java7 examples from Streams Docs
[KAFKA-8666] - Improve Documentation on usage of Materialized config object
[KAFKA-8779] - Fix flaky tests introduced by dynamic log levels
[KAFKA-9738] - Add Generics Type Parameters to forwarded() in MockProcessorContext
[KAFKA-12359] - Update Jetty to 11
[KAFKA-12770] - Gradle build: allow the CheckStyle version to be specified via parameter
[KAFKA-12872] - KIP-724: Drop support for message formats v0 and v1
[KAFKA-12894] - KIP-750: Drop support for Java 8 in Kafka 4.0 (deprecate in 3.0)
[KAFKA-12895] - KIP-751: Drop support for Scala 2.12 in Kafka 4.0 (deprecate in 3.0)
[KAFKA-13588] - We should consolidate `changelogFor` methods to simplify the generation of internal topic names
[KAFKA-14048] - The Next Generation of the Consumer Rebalance Protocol
[KAFKA-14552] - Assume a baseline of 3.0 for server protocol versions
[KAFKA-14560] - Remove old client protocol API versions in Kafka 4.0 (KIP-896)
[KAFKA-14619] - KRaft validate snapshot id are at batch boundries
[KAFKA-14934] - KafkaClusterTestKit makes FaultHandler accessible
[KAFKA-14995] - Automate asf.yaml collaborators refresh
[KAFKA-15549] - Bump swagger dependency version
[KAFKA-16096] - Drop broker and tools support for Java 11 in Kafka 4.0 (deprecate in 3.7) (KIP-1013)
[KAFKA-16143] - New JMX metrics for AsyncKafkaConsumer
[KAFKA-16164] - Pre-Vote 4.0 changes
[KAFKA-16181] - Use incrementalAlterConfigs when updating broker configs by kafka-configs.sh
[KAFKA-16368] - Change constraints and default values for various configurations - part 1
[KAFKA-16437] - Upgrade to Jakarta and JavaEE 10 in Kafka 4.0 (KIP-1032)
[KAFKA-16508] - Infinite loop if output topic does not exisit
[KAFKA-16564] - Apply `Xlint` to java code in core module
[KAFKA-16617] - Add KRaft info for the `advertised.listeners` doc description
[KAFKA-16689] - Move LogValidatorTest to storage module
[KAFKA-16818] - Move event processing-related tests from ConsumerNetworkThreadTest to ApplicationEventProcessorTest
[KAFKA-16830] - Remove the scala version formatters support
[KAFKA-16863] - Consider removing `default.` prefix for exception handler config
[KAFKA-16899] - Rename MembershipManagerImpl's rebalanceTimeoutMs for clarity
[KAFKA-16908] - Refactor QuorumConfig with AbstractConfig
[KAFKA-17010] - Remove DescribeLogDirsResponse#ReplicaInfo
[KAFKA-17035] - Add debug log to retention cleanupLogs method to help troubleshoot issues
[KAFKA-17056] - Convert producer state metadata schemas to use generated protocol
[KAFKA-17057] - Add "retry" option to ProductionExceptionHandler
[KAFKA-17087] - Deprecate `delete-config` of TopicCommand
[KAFKA-17099] - Improve the process exception logs with the exact processor node name in which processing exceptions occur
[KAFKA-17109] - Reduce log message load for failed locking
[KAFKA-17128] - Make node.id immutable after removing zookeeper migration
[KAFKA-17137] - Ensure Admin APIs are properly tested
[KAFKA-17151] - Remove waitForCondition when detecting thread leak
[KAFKA-17173] - Move all quota getters from `KafkaConfig` to `QuotaConfigs`
[KAFKA-17175] - Remove interface `BrokerNode` and `ControllerNode`
[KAFKA-17180] - Upgrade gradle from 8.8 to 8.10
[KAFKA-17185] - Make sure a single logger instance is created
[KAFKA-17200] - Enable MM2 to replicate topics ending in "internal" suffix
[KAFKA-17222] - Remove the subclass of KafkaMetricsGroup
[KAFKA-17224] - Make ForeachProcessor internal
[KAFKA-17240] - Try to complete delyed ops for all purgatories even though one of delayed op throws exception
[KAFKA-17253] - Remove leaking getter methods in Joined helper class
[KAFKA-17275] - Move ReplicatedCounter to test scope
[KAFKA-17285] - Consider using `Utils.closeQuietly` to replace `CoreUtils.swallow` when handling Closeable objects
[KAFKA-17314] - Fix the typo: `maxlifeTimeMs`
[KAFKA-17322] - Document required request and response header versions in Protocol Guide
[KAFKA-17323] - Document UINT16 and COMPACT_RECORDS in Protocol Guide
[KAFKA-17327] - Add support of group in kafka-configs.sh
[KAFKA-17340] - Kafka Java sdk ConsumerConfig describe doc is incorrectly
[KAFKA-17359] - add tests and enhance the docs of Admin#describeConfigs for the case of nonexistent resource
[KAFKA-17370] - Move LeaderAndIsr from core module to metadata module
[KAFKA-17373] - add print.epoch to kafka-console-share-consumer.sh/kafka-console-consumer.sh
[KAFKA-17377] - Consider using defaultApiTimeoutMs in AsyncKafkaConsumer#unsubscribe
[KAFKA-17382] - cleanup out-of-date configs of config_property
[KAFKA-17384] - Remove deprecated options in tools
[KAFKA-17399] - Apply LambdaValidator to code base
[KAFKA-17409] - Remove deprecated constructor of org.apache.kafka.clients.producer.RecordMetadata
[KAFKA-17416] - Add a checkstyle rule to suppress all generated code
[KAFKA-17418] - Fix the incorrect markdown of junit.py caused by newline character
[KAFKA-17427] - Fix leaking *_DOC variables in StreamsConfig
[KAFKA-17428] - Add retry mechanism for cleaning up dangling remote segments
[KAFKA-17433] - Add a CI job to run a single test N times
[KAFKA-17435] - remove use.incremental.alter.configs
[KAFKA-17437] - Upgrade commons-validator from 1.7 to 1.9.0
[KAFKA-17461] - move the .github/readme to ./github/workflows/readme
[KAFKA-17465] - Refactor getMembersFromGroup to be non-blocking
[KAFKA-17494] - Document message structure in Protocol Guide
[KAFKA-17508] - Adding some guard for fallback deletion logic
[KAFKA-17520] - align the low bound of ducktape version
[KAFKA-17525] - Convert the UnknownServerException to InvalidRequestException when altering client-metrics config at runtime
[KAFKA-17534] - Allow disabling hearbeats topic replication in MirrorSourceConnector
[KAFKA-17536] - Ensure clear error message when "new" consumer used with incompatible cluster
[KAFKA-17561] - Operator Metrics for Kafka Streams
[KAFKA-17566] - What is the next about other.kafka
[KAFKA-17585] - `offsetResetStrategyTimestamp` should return `long` instead of `Long`
[KAFKA-17600] - Add nextOffsets to the ConsumerRecords
[KAFKA-17610] - Drop deprecated alterConfigs
[KAFKA-17638] - Add support for Java 23
[KAFKA-17655] - add example of changing the e2e image name
[KAFKA-17656] - Use parameterized logging for PartitionChangeBuilder
[KAFKA-17657] - Replace the "consumer-fetch-manager-metrics" by groupName
[KAFKA-17658] - Move BootstrapControllersIntegrationTest to kafka.server
[KAFKA-17663] - Add Metadata caching in admin.internals.PartitionLeaderStrategy
[KAFKA-17686] - AsyncKafkaConsumer.offsetsForTimes() fails with NullPointerException
[KAFKA-17695] - cleanup org.apache.kafka.common.test.TestUtils
[KAFKA-17709] - remove retry_zinc
[KAFKA-17717] - Remove ConfigUtils#translateDeprecatedConfigs
[KAFKA-17728] - Add missing config `replica-directory-id` to raft README
[KAFKA-17744] - Improve the State Updater logs when restoring state
[KAFKA-17748] - Remove scala-java8-compat
[KAFKA-17750] - Extend kafka-consumer-groups command line tool to support new consumer group
[KAFKA-17757] - cleanup code case under JDK 11
[KAFKA-17776] - Add connection disconnect listener in socket server
[KAFKA-17778] - Clean up client instance cache on connection close
[KAFKA-17785] - Kafka protocol documentation should include tagged field information
[KAFKA-17791] - Dockerfile should use `requirements.txt` to ensure dependencies
[KAFKA-17802] - Update bouncy-castle from 1.75 to 1.78.1
[KAFKA-17803] - Reconcile Differences in MockLog and KafkaMetadataLog `read` Implementation
[KAFKA-17813] - Create server connection id class encapsulating functionality for generation
[KAFKA-17816] - add `testRuntimeOnly runtimeTestLibs` to `test-common-api` module
[KAFKA-17818] - add log4j.properties to test-common and test-common-api
[KAFKA-17841] - Use placeholders instead of string concatenation in logs
[KAFKA-17847] - Avoid the extra bytes copy when compressing telemetry payload
[KAFKA-17864] - Ensure all fields in the protocol have a description
[KAFKA-17867] - Consider using zero-copy for PushTelemetryRequest
[KAFKA-17873] - Add description to all packages in the public API
[KAFKA-17874] - Fix the KRaft metric names in the documentation
[KAFKA-17875] - Align KRaft controller count recommendations
[KAFKA-17882] - improve the document about "default value" of tagged structure field
[KAFKA-17893] - Support record keys in the foreignKeyExtractor argument of KTable foreign join
[KAFKA-17903] - Remove KafkaFuture#Function and KafkaFuture#thenApply
[KAFKA-17905] - Remove the specified type of using lambda for BaseFunction
[KAFKA-17906] - remove redundant sourceSets from build.gradle
[KAFKA-17917] - Convert Kafka core system tests to use KRaft
[KAFKA-17919] - enable back the failing testShareGroups test
[KAFKA-17924] - Remove `bufferpool-wait-time-total`, `io-waittime-total`, and `iotime-total`
[KAFKA-17926] - Improve the documentation explaining why max.in.flight.requests.per.connection should not exceed 5.
[KAFKA-17927] - Disallow users to configure `max.in.flight.requests.per.connection` bigger than 5
[KAFKA-17931] - Revise the `specifying-test-retries` section in README
[KAFKA-17934] - Add duration based offset reset option for consumer clients
[KAFKA-17936] - Refactor SslTransportLayerTest#testDsaKeyPair to avoid skip test
[KAFKA-17947] - Update currentLag(), pause(), and resume() to update SubscriptionState in background thread
[KAFKA-17974] - Upgrade gradle from 8.10 to 8.10.2
[KAFKA-17975] - Remove ControllerQuorumVotersFutureManager
[KAFKA-17992] - Remove `getUnderlying` and `isKRaftTest` from ClusterInstance
[KAFKA-17997] - Remove deprecated config log.message.timestamp.difference.max.ms
[KAFKA-18037] - Remove deprecated methods from EmbeddedConnect and EmbeddedConnectCluster
[KAFKA-18046] - High CPU usage when using Log4j2
[KAFKA-18049] - Upgrade the caffeine version to 3.1.1
[KAFKA-18050] - Upgrade the checkstyle version to 10.20.2
[KAFKA-18059] - kafka-metadata-quorum.sh add-controller subcommand can't recognize argument --config
[KAFKA-18071] - Avoid unneeded background event to refresh regex if no subscription pattern in use
[KAFKA-18077] - Remove deprecated JmxReporter(String)
[KAFKA-18080] - Replace DelayedItem by Long type
[KAFKA-18093] - Remove deprecated DeleteTopicsResult#values
[KAFKA-18094] - Remove deprecated TopicListing(String, Boolean)
[KAFKA-18095] - Consider allowing member to join without subscription under new consumer protocol
[KAFKA-18096] - Allow new consumer to join group after subscribe to pattern if no matching topics
[KAFKA-18130] - Remove kafka.utils.VersionInfo
[KAFKA-18131] - Improve logs for voters
[KAFKA-18153] - remove jenkins settings from settings.gradle
[KAFKA-18159] - Remove onPartitionsRevoked and onPartitionsAssigned from SinkTask
[KAFKA-18181] - Refactor ShareConsumerTest
[KAFKA-18183] - ClusterInstance's helper should use byte array instead of Bytes in creating producer/consumer
[KAFKA-18184] - Remove the unnecessary project path check from build.gradle
[KAFKA-18195] - Enter "incompatible" instead of leaving incompatible entires blank in Kafka Streams broker compatibility matrix
[KAFKA-18203] - Add a section for Java version in intellij idea in README
[KAFKA-18219] - Use INFO level instead of ERROR after successfully performing an unclean leader election
[KAFKA-18230] - admin client will fail immediately when the active controller changed
[KAFKA-18237] - add 3.7.2 to code base
[KAFKA-18262] - Remove DefaultPartitioner and UniformStickyPartitioner
[KAFKA-18264] - Remove NotLeaderForPartitionException
[KAFKA-18289] - Remove deprecated methods of DescribeTopicsResult
[KAFKA-18290] - Remove deprecated methods of FeatureUpdate
[KAFKA-18291] - Remove deprecated methods of ListConsumerGroupOffsetsOptions
[KAFKA-18292] - Remove deprecated methods of UpdateFeaturesOptions
[KAFKA-18293] - Remove org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler and org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerValidatorCallbackHandler
[KAFKA-18294] - Remove deprecated SourceTask#commitRecord
[KAFKA-18295] - Remove deprecated function Partitioner#onNewBatch
[KAFKA-18296] - Remove deprecated KafkaBasedLog constructor
[KAFKA-18338] - add log4j.yaml to test-common-api and remove unsed log4j.properties from test-common
[KAFKA-18340] - Change Dockerfile to use log4j2 yaml instead log4j properties
[KAFKA-18343] - Use java_pids to implement pids
[KAFKA-18347] - Add tools-log4j2.yaml to config and remove unsed tools-log4j.properties from config
[KAFKA-18348] - Remove the deprecated MockConsumer#setException
[KAFKA-18354] - Use log4j2 APIs to refactor LogCaptureAppender
[KAFKA-18375] - Update the LICENSE-binary
[KAFKA-18388] - test-kraft-server-start.sh should use log4j2.yaml
[KAFKA-18396] - Migrate log4j1 configuration to log4j2 in KafkaDockerWrapper
[KAFKA-18422] - add Kafka client upgrade path
[KAFKA-18465] - Remove MetadataVersions older than 3.0-IV1
[KAFKA-18476] - KafkaStreams should swallow TransactionAbortedException
[KAFKA-18484] - More robust exception handling for new group coordinator unload
[KAFKA-18570] - Update documemtation for log loading metrics during Kafka broker startup
[KAFKA-18579] - Add TransactionAbortableException instruction in docs
[KAFKA-18601] - Assume a baseline of 3.3 for server protocol versions
[KAFKA-18839] - Drop support for eager rebalancing in Streams
[KAFKA-18849] - add "strict min ISR" to the docs of "min.insync.replicas"
[KAFKA-18850] - Fix the docs of org.apache.kafka.automatic.config.providers
[KAFKA-18864] - remove the Evolving tag from stable public interfaces
[KAFKA-18868] - add the "default value" explanation to the docs of num.replica.alter.log.dirs.threads
[KAFKA-18869] - add remote storage threads to "Updating Thread Configs" section
[KAFKA-18876] - 4.0 documentation improvement
Bug
[KAFKA-8116] - Add Kafka Streams archetype for Java11
[KAFKA-8366] - partitions of topics being deleted show up in the offline partitions metric
[KAFKA-8862] - Misleading exception message for non-existant partition
[KAFKA-9366] - Upgrade log4j to log4j2
[KAFKA-10812] - Unclean worker shutdown in MirrorConnectorsIntegrationTest
[KAFKA-12679] - Rebalancing a restoring or running task may cause directory livelocking with newly created task
[KAFKA-12989] - MockClient should respect the request matcher passed to prepareUnsupportedVersionResponse
[KAFKA-14460] - In-memory store iterators can return results with null values
[KAFKA-15072] - Flaky test MirrorConnectorsIntegrationExactlyOnceTest.testReplicationWithEmptyPartition
[KAFKA-15203] - Remove dependency on Reflections
[KAFKA-15344] - Kafka Streams should include the message leader epoch when committing offsets
[KAFKA-15524] - Flaky test org.apache.kafka.connect.integration.OffsetsApiIntegrationTest.testResetSinkConnectorOffsetsZombieSinkTasks
[KAFKA-15892] - Flaky test: testAlterSinkConnectorOffsets – org.apache.kafka.connect.integration.OffsetsApiIntegrationTest
[KAFKA-15933] - Flaky test: testRestartReplication() – org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationTransactionsTest
[KAFKA-15935] - Flaky test: testRestartReplication() – org.apache.kafka.connect.mirror.integration.IdentityReplicationIntegrationTest
[KAFKA-15945] - Flaky test - testSyncTopicConfigs() – org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest
[KAFKA-16022] - AsyncKafkaConsumer sometimes complains “No current assignment for partition {}”
[KAFKA-16248] - Kafka consumer should cache leader offset ranges
[KAFKA-16326] - Kafka Connect unable to find javax dependency on Quarkus update to 3.X
[KAFKA-16346] - Fix flaky MetricsTest.testMetrics
[KAFKA-16372] - max.block.ms behavior inconsistency with javadoc and the config description
[KAFKA-16460] - New consumer times out consuming records in multiple consumer_test.py system tests
[KAFKA-16576] - New consumer fails with assert in consumer_test.py’s test_consumer_failure system test
[KAFKA-16649] - Remove lock from DynamicBrokerConfig.removeReconfigurable
[KAFKA-16900] - kafka-producer-perf-test reports error when using transaction.
[KAFKA-16985] - Ensure consumer attempts to send leave request on close even if interrupted
[KAFKA-17040] - Unknown telemetry state: TERMINATED thrown when closing AsyncKafkaConsumer
[KAFKA-17064] - New consumer assign should update assignment in background thread
[KAFKA-17066] - New consumer updateFetchPositions should perform all operations in background thread
[KAFKA-17076] - logEndOffset could be lost due to log cleaning
[KAFKA-17094] - Make it possible to list registered KRaft nodes in order to know which nodes should be unregistered
[KAFKA-17112] - StreamThread shutdown calls completeShutdown only in CREATED state
[KAFKA-17116] - New consumer may not send effective leave group if member ID received after close
[KAFKA-17154] - New consumer subscribe may join group without a call to consumer.poll
[KAFKA-17181] - Fix possible partial write in writing snapshot (ProducerStateManager)
[KAFKA-17182] - Consumer fetch sessions are evicted too quickly with AsyncKafkaConsumer
[KAFKA-17188] - LoginManager ctor might throw an exception causing login and loginCallbackHandler not being closed properly
[KAFKA-17201] - SelectorTest.testInboundConnectionsCountInConnectionCreationMetric leaks sockets and threads
[KAFKA-17208] - replica_scale_test.py fails for new consumer
[KAFKA-17219] - Adjust system test framework for new protocol consumer
[KAFKA-17230] - Kafka consumer client doesn't report node request-latency metrics
[KAFKA-17233] - MirrorCheckpointConnector should use batched listConsumerGroupOffsets
[KAFKA-17234] - Partition.tryCompleteDelayedRequests should not throw
[KAFKA-17239] - Kafka admin client doesn't report node request-latency metrics
[KAFKA-17262] - kafka-topics.sh usage message is confusing
[KAFKA-17267] - New group coordinator can return REQUEST_TIMED_OUT for OFFSET_FETCHes
[KAFKA-17268] - Fix NPE when closing a non-started acceptor
[KAFKA-17286] - Flaky PlaintextConsumerSubscriptionTest#testSubscribeInvalidTopic
[KAFKA-17293] - New consumer HeartbeatRequestManager should rediscover disconnected coordinator
[KAFKA-17294] - Handle retriable errors when fetching offsets in new consumer
[KAFKA-17295] - New consumer fails with assert in consumer_test.py’s test_fencing_static_consumer system test
[KAFKA-17299] - Kafka Streams consumer stops consumption
[KAFKA-17347] - Add omitted --client-metrics option to kafka-configs.sh
[KAFKA-17349] - Make sure all IT have "quorum" input arguments
[KAFKA-17371] - Flaky test in DefaultTaskExecutorTest.shouldUnassignTaskWhenRequired
[KAFKA-17402] - Test failure: DefaultStateUpdaterTest.shouldGetTasksFromRestoredActiveTasks expected: <2> but was: <3>
[KAFKA-17403] - New consumer may not send leave group on race condition on network thread run & close
[KAFKA-17415] - Avoid overflow of expiried timestamp
[KAFKA-17432] - DefaultStateUpdater/DefaultTaskExecutor shutdown returns before threads have fully exited
[KAFKA-17439] - Make polling for new records an explicit action/event in the new consumer
[KAFKA-17448] - New consumer seek should update positions in background thread
[KAFKA-17455] - `TaskCorruptedException` After Client Quota Throttling
[KAFKA-17463] - Flaky test: kafka.api.PlaintextAdminIntegrationTest."testShareGroups(String).quorum=kraft+kip932"
[KAFKA-17470] - CommitRequestManager should record failed request only once even if multiple errors in response
[KAFKA-17478] - Wrong configuration of metric.reporters lead to NPE in KafkaProducer constructor
[KAFKA-17480] - New consumer commit all consumed should retrieve offsets in background thread
[KAFKA-17492] - skip features with minVersion of 0 instead of replacing 0 with 1 when BrokerRegistrationRequest < 4
[KAFKA-17505] - New consumer seekToBeginning/End should run in background thread
[KAFKA-17507] - WriteTxnMarkers API must not return until markers are written and materialized in group coordinator's cache
[KAFKA-17515] - Fix flaky RestoreIntegrationTest.shouldInvokeUserDefinedGlobalStateRestoreListener
[KAFKA-17518] - AsyncKafkaConsumer cannot reliably leave group when closed with small timeout
[KAFKA-17519] - Define and validate correctness of Consumer.close() and its timeout when thread is interrupted
[KAFKA-17553] - DefaultTaskManager.shutdown stalling on Github CI
[KAFKA-17574] - KafkaClusterTestKit produces error logs if shut down from a shutdown hook
[KAFKA-17575] - TestUtils.tempDirectory registers two shutdown hooks for the same thing
[KAFKA-17583] - kafka-config script cannot set `cleanup.policy=delete,compact`
[KAFKA-17622] - Kafka Streams Timeout During Partition Rebalance
[KAFKA-17623] - Flaky testSeekPositionAndPauseNewlyAssignedPartitionOnPartitionsAssignedCallback
[KAFKA-17632] - Custom `partitioner.class` with an even number of partitions always writes to even partitions if use RoundRobinPartitioner
[KAFKA-17635] - Lost events on internal repartition topic when excatly_once_v2 is set and producer is fenced
[KAFKA-17636] - The StorageTool does not create SCRAM credentials when formatting disk
[KAFKA-17646] - Fix flaky KafkaStreamsTest.testStateGlobalThreadClose
[KAFKA-17661] - Fix flaky BufferPoolTest.testBlockTimeout
[KAFKA-17674] - New consumer reset positions for newly added partitions before retrieving committed offsets
[KAFKA-17696] - New consumer background operations unaware of metadata errors
[KAFKA-17697] - Fix flaky DefaultStateUpdaterTest.shouldRestoreSingleActiveStatefulTask
[KAFKA-17706] - Allow all imports for test-common and test-common-api
[KAFKA-17713] - Ensure snapshots are aligned to batch boundaries
[KAFKA-17726] - New consumer subscribe/subscribeFromPattern in background thread
[KAFKA-17730] - ReplicaFetcherThreadBenchmark is broken
[KAFKA-17749] - Throttle metrics have changed name
[KAFKA-17751] - Contoller high CPU when formatted with --initial-controllers
[KAFKA-17753] - Update protobuf and commons-io dependencies
[KAFKA-17766] - TopicBasedRemoteLogMetadataManager stuck in close
[KAFKA-17769] - Fix flaky PlaintextConsumerSubscriptionTest.testSubscribeInvalidTopicCanUnsubscribe
[KAFKA-17781] - add `psutil` to e2e dockerfile and upgrade ducktape version
[KAFKA-17792] - header parsing ends up timing out and using large quantities of memory if the string looks like a number
[KAFKA-17793] - Improve kcontroller robustness against long delays
[KAFKA-17795] - Fix the file path of suppressions.xml to run build on windows
[KAFKA-17801] - RemoteLogManager may compute inaccurate upperBoundOffset for aborted txns
[KAFKA-17817] - Remove cache from FetchRequest#fetchData
[KAFKA-17833] - Convert DescribeAuthorizedOperationsTest to use kraft
[KAFKA-17838] - move the log4j.properties from resources/test to resources
[KAFKA-17846] - ClientTelemetryReporter does not log trace-level message
[KAFKA-17850] - Stop leaking internal exception to StreamsUncaughtExceptionHandler
[KAFKA-17852] - Add help message to the --ignore-formatted flag of StorageTool
[KAFKA-17870] - Disallow creating too many partitions before using ReplicaPlacer
[KAFKA-17872] - Kafka stream job does not commit offset when packets are getting ignored intentionally
[KAFKA-17877] - IllegalStateException: missing producer id from the WriteTxnMarkersResponse
[KAFKA-17881] - Apply the minJavaVersion to test code
[KAFKA-17908] - Tidy up log messages for consumer group in group metadata manager
[KAFKA-17911] - Unable to set kafka.metrics.reporter via environment variables in docker image
[KAFKA-17913] - Fix KRaft controller count recommendations
[KAFKA-17941] - TransactionStateManager.loadTransactionMetadata method may get stuck in an infinite loop
[KAFKA-17954] - Error getting oldest-iterator-open-since-ms from JMX
[KAFKA-17960] - PlaintextAdminIntegrationTest.testConsumerGroups fails with CONSUMER group protocol
[KAFKA-17988] - Fix flaky ReconfigurableQuorumIntegrationTest.testRemoveAndAddSameController
[KAFKA-17994] - Checked exceptions are not handled when deserializing kafka stream record
[KAFKA-17995] - Large value for `retention.ms` could prevent remote data cleanup in Tiered Storage
[KAFKA-18017] - New consumer should notify HB errors to group manager after errors have been handled/propagated
[KAFKA-18021] - Disabled MirrorCheckpointConnector throws RetriableException on task config generation
[KAFKA-18027] - MINOR: Correct DelayedOperationPurgatory code around adding of an already completed operation
[KAFKA-18028] - the effective kraft version of `--no-initial-controllers` should be 1 rather than 0
[KAFKA-18029] - Fix quorum_reconfiguration_test.py
[KAFKA-18034] - CommitRequestManager should fail pending requests on fatal coordinator errors
[KAFKA-18040] - PlaintextProducerSendTest.testSendToPartitionWithFollowerShutdownShouldNotTimeout fails with CONSUMER group protocol
[KAFKA-18051] - Disallow creating ACLs with principals that do not contain a colon
[KAFKA-18060] - new coordinator does not handle TxnOffsetCommitRequest with empty member id when using CONSUMER group
[KAFKA-18063] - SnapshotRegistry should not leak memory
[KAFKA-18073] - Data loss when exception is raised from Kafka Connect record conversion call
[KAFKA-18075] - Prevent ClusterInstance default producer and consumer initialization with empty configs
[KAFKA-18078] - Fix failed MetricsTest.testJMXFilter
[KAFKA-18083] - ClusterInstance custom controllerListener not work
[KAFKA-18085] - Abort inflight requests when rebootstrapping due to timeout or error
[KAFKA-18092] - TransactionsTest.testBumpTransactionalEpochWithTV2Enabled is flaky
[KAFKA-18100] - `Using` block suppresses all errors
[KAFKA-18109] - Fix failed SaslClientsWithInvalidCredentialsTest in tools module
[KAFKA-18118] - Fix the incorrect soft link of results/latest
[KAFKA-18123] - Fix flaky DynamicBrokerReconfigurationTest#testThreadPoolResize
[KAFKA-18127] - Handle subscription pattern used with v0 HB broker
[KAFKA-18189] - CoordinatorRequestManager log message can include incorrect coordinator disconnect time
[KAFKA-18194] - Flaky test_broker_rolling_bounce due to metadata update
[KAFKA-18199] - Incorrect size calculation for ConsumerGroupMemberMetadataValue.classicMemberMetadata
[KAFKA-18200] - New coordinator can flush an empty batch if append fails
[KAFKA-18206] - EmbeddedKafkaCluster must set features
[KAFKA-18211] - ClassGraph scanning does not correctly find isolated connect plugins
[KAFKA-18214] - TestUtils#waitForCondition does not honor the maxWaitMs
[KAFKA-18221] - Set explicit consumer group protocol in streams tests
[KAFKA-18228] - The MetricsDuringTopicCreationDeletionTest should delete topics to ensure that the metrics are recreated.
[KAFKA-18234] - DumpLogSegments cannot print ConsumerGroupRegularExpression record
[KAFKA-18235] - can't build release tar due to broken genConnectOpenAPIDocs task
[KAFKA-18240] - Remove nonexistent `LOG4J_CONFIG` from transactional_message_copier.py
[KAFKA-18242] - The java code in core module is NOT configured with suitable release version
[KAFKA-18259] - Documentation for consumer auto.offset.reset contains invalid HTML
[KAFKA-18263] - Group lock must be acquired when reverting static membership rejoin
[KAFKA-18281] - Kafka (3.9.0) is improperly validating non-advertised listeners for routable controller addresses
[KAFKA-18306] - TransactionsTest testFailureToFenceEpoch is flaky
[KAFKA-18326] - Cached stores may return deleted values
[KAFKA-18342] - Vagrantfile can't work with ruby 3.2
[KAFKA-18401] - Transaction version 2 does not support commit transaction without records
[KAFKA-18451] - Flaky RemoteLogManagerTest#testRLMOpsWhenMetadataIsNotReady
[KAFKA-18464] - Empty Abort Transaction can fence producer incorrectly with Transactions V2
[KAFKA-18466] - Remove log4j-1.2-api from runtime scope while keeping it in distribution package
[KAFKA-18469] - AsyncConsumer fails to retry ListOffsetRequest on ReplicaNotAvailable error without metadata update
[KAFKA-18478] - RocksDBTimeOrderedKeyValueBuffer does only use serdes from config
[KAFKA-18480] - Fix fail e2e `test_offset_truncate`
[KAFKA-18482] - Fix failing system test StreamsSmokeTest
[KAFKA-18497] - Remove config/kraft/server.properties
[KAFKA-18547] - Failing test ClientUtilsTest#testParseAndValidateAddressesWithReverseLookup in trunk
[KAFKA-18569] - New consumer close may wait on unneeded FindCoordinator
[KAFKA-18575] - Transaction Version 2 doesn't correctly handle race condition with completing and new transaction
[KAFKA-18619] - New consumer topic metadata events should set requireMetadata flag
[KAFKA-18634] - Fix ELR metadata version issues
[KAFKA-18635] - Re-enable the unclean shutdown detection when in ELR mode
[KAFKA-18641] - AsyncKafkaConsumer could lose records with auto offset commit
[KAFKA-18645] - New consumer should align close timeout handling with classic consumer
[KAFKA-18649] - Complete ClearElrRecord handling in topic delta
[KAFKA-18654] - Transaction Version 2 performance regression due to early return
[KAFKA-18660] - Transactions Version 2 doesn't handle epoch overflow correctly
[KAFKA-18668] - config/kraft/reconfig-server.properties no longer exists in quickstart
[KAFKA-18690] - New consumer doesn't updateLatestMetadata for topics matching new regex
[KAFKA-18691] - Flaky test testFencingOnTransactionExpiration
[KAFKA-18737] - KafkaDockerWrapper setup functions fails due to storage format command
[KAFKA-18743] - leader.imbalance.per.broker.percentage is not supported by Kraft
[KAFKA-18758] - NullPointerException in shutdown following InvalidConfigurationException
[KAFKA-18773] - Migrate the log4j1 config to log4j 2 for native image and README
[KAFKA-18784] - Fix ConsumerWithLegacyMessageFormatIntegrationTest
[KAFKA-18803] - The acls would appear at the wrong level of the metadata shell "tree"
[KAFKA-18807] - Group coordinator under-reports thread idle time
[KAFKA-18813] - ConsumerGroupHeartbeat API and ConsumerGroupDescribe API must check topic describe
[KAFKA-18831] - Migrating to log4j2 introduce behavior changes of adjusting level dynamically
[KAFKA-18844] - Stale features information in QuorumController#registerBroker
[KAFKA-18916] - Resolved regular expressions must update the group by topics data structure
[KAFKA-18917] - TransformValues with unnamed Materialized instance throws NPE
[KAFKA-18920] - The kcontrollers do not set kraft.version in ApiVersionsResponse
[KAFKA-18943] - Kafka Streams incorrectly commits TX during task revokation
[KAFKA-18979] - kraft.version wrongly upgraded to 1 after upgrading to v4.0
Task
[KAFKA-12689] - Remove deprecated EOS configs
[KAFKA-12690] - Remove deprecated Producer#sendOffsetsToTransaction
[KAFKA-12822] - Remove Deprecated APIs of Kafka Streams in 4.0
[KAFKA-14158] - KIP-830 cleanups for Kafka 4.0
[KAFKA-14262] - Delete MirrorMaker v1
[KAFKA-14417] - Producer doesn't handle REQUEST_TIMED_OUT for InitProducerIdRequest, treats as fatal error
[KAFKA-15387] - Deprecate and remove Connect's redundant task configurations retrieval endpoint
[KAFKA-15443] - Upgrade RocksDB dependency
[KAFKA-15737] - KRaft support in ConsumerBounceTest
[KAFKA-15859] - Introduce delayed remote list offsets to make LIST_OFFSETS async
[KAFKA-15875] - Snapshot class is package protected but returned in public methods
[KAFKA-15907] - Remove previously deprecated Consumer features from 4.0
[KAFKA-16041] - Replace Afterburn module with Blackbird
[KAFKA-16188] - Delete deprecated kafka.common.MessageReader
[KAFKA-16290] - Investigate propagating subscription state updates via queues
[KAFKA-16308] - Formatting and Updating Kafka Features
[KAFKA-16769] - Delete deprecated add.source.alias.to.metrics configuration
[KAFKA-16780] - Txn consumer exerts pressure on remote storage when collecting aborted transactions
[KAFKA-16816] - Remove unneeded FencedInstanceId support on commit path for new consumer
[KAFKA-16974] - KRaft support in SslAdminIntegrationTest
[KAFKA-17078] - Add SecurityManager reflective shim
[KAFKA-17100] - GlobalStreamThread#start should not busy-wait
[KAFKA-17170] - Add test to ensure new consumer acks reconciled assignment even if first HB with ack lost
[KAFKA-17193] - Pin external GitHub actions to specific git hash
[KAFKA-17324] - Set config group.protocol to classic in Streams
[KAFKA-17338] - ConsumerConfig should prevent using partition assignors with CONSUMER group protocol
[KAFKA-17355] - Update TestFeatureVersion 2 to always be based on MetadataVersion.latestTesting
[KAFKA-17369] - Remove org.reflections from logging and licenses
[KAFKA-17488] - Cleanup (test) code for Kafka Streams "metric version"
[KAFKA-17591] - Consumer protocol subscription regex support
[KAFKA-17609] - Convert Kafka Streams system tests to use KRaft
[KAFKA-17611] - Remove ZK from Kafka 4.0
[KAFKA-17618] - group consumer heartbeat interval should be less than session timeout
[KAFKA-17631] - Convert SaslApiVersionsRequestTest to kraft
[KAFKA-17648] - AsyncKafkaConsumer#unsubscribe swallow TopicAuthorizationException
[KAFKA-17671] - Create better documentation for transactions
[KAFKA-17805] - Deprecate named topologies
[KAFKA-17858] - Remove ZK-related configuration from the log4j configuration.
[KAFKA-17909] - Remove zkBroker from ConsumerGroupHeartbeatRequest and ConsumerGroupDescribeRequest
[KAFKA-17915] - Convert Kafka Client system tests to use KRaft
[KAFKA-17925] - Convert Kafka Client integration tests to use KRaft
[KAFKA-17928] - Make remote log manager thread-pool configs dynamic
[KAFKA-17980] - Add isReady API in RemoteLogMetadataManager
[KAFKA-18041] - Update key for global consumer in client instance ids
[KAFKA-18047] - Add org.apache.kafka.automatic.config.providers to System Properties section
[KAFKA-18052] - Decouple the dependency of feature stable version to the metadata version
[KAFKA-18074] - Add kafka client compatibility matrix
[KAFKA-18137] - Unloading transaction state incorrectly removes loading partitions
[KAFKA-18224] - StreamsResetter tool should set/validate group protocol
[KAFKA-18225] - ClientQuotaCallback#updateClusterMetadata is unsupported by kraft
[KAFKA-18305] - validate controller.listener.names is not in inter.broker.listener.name for kcontrollers
[KAFKA-18364] - migrating from zk to kraft document
[KAFKA-18436] - Revert KIP-891 from 4.0 release
[KAFKA-18583] - Improve/fix KRaftMetadataCache.getPartitionReplicaEndpoints
Wish
[KAFKA-17086] - Java 21 support in Kafka
Test
[KAFKA-14453] - Flaky test suite MirrorConnectorsWithCustomForwardingAdminIntegrationTest
[KAFKA-14572] - Migrate EmbeddedKafkaCluster used by Streams integration tests from EmbeddedZookeeper to KRaft
[KAFKA-15197] - Flaky test MirrorConnectorsIntegrationExactlyOnceTest.testOffsetTranslationBehindReplicationFlow()
[KAFKA-15292] - Flaky test IdentityReplicationIntegrationTest#testReplicateSourceDefault()
[KAFKA-16792] - Enable consumer unit tests that fail to fetch offsets only for new consumer with poll(0)
[KAFKA-16919] - Flaky test testNoCheckpointsIfNoRecordsAreMirrored() – org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest
[KAFKA-16993] - Flaky test RestoreIntegrationTest.shouldInvokeUserDefinedGlobalStateRestoreListener.shouldInvokeUserDefinedGlobalStateRestoreListener()
[KAFKA-17199] - add unit tests for TransactionLogConfig and TransactionStateManagerConfig
[KAFKA-17203] - StreamThread leaking producer instances
[KAFKA-17258] - Migrate AdminFenceProducersIntegrationTest to ClusterTestExtensions framework
[KAFKA-17269] - Fix ConcurrentModificationException caused by NioEchoServer.closeNewChannels
[KAFKA-17270] - Let test code use Exit.java rather than Exit.scala
[KAFKA-17297] - testHandleEndQuorumRequest fails, why?
[KAFKA-17303] - Enable testPutControllersInBootstrapBrokersConfig
[KAFKA-17407] - Fix flaky ShareGroupHeartbeatRequestTest.testPartitionAssignmentWithChangingTopics
[KAFKA-17459] - Stabilize reassign_partitions_test.py
[KAFKA-17460] - Remove downgrade_test.py as it is duplicate to kraft_upgrade_test.py
[KAFKA-17474] - Flaky tests in GlobalStreamThreadTest
[KAFKA-17477] - Topic command integration test migrate to new test infra
[KAFKA-17524] - StreamThreadTest shouldReturnErrorIfProducerInstanceIdNotInitialized hanging
[KAFKA-17526] - make ConfigCommandIntegrationTest.java test use correct arguments in testing alias
[KAFKA-17555] - uncomment all checks of testCommonNameLoggingTrustManagerMixValidAndInvalidCertificates
[KAFKA-17558] - Cleanup Kafka Streams integration tests
[KAFKA-17654] - Fix flaky ProducerIdManagerTest#testUnrecoverableErrors
[KAFKA-17772] - Remove inControlledShutdownBrokers(Set) and unfenceBrokers(Set) from ReplicationControlManagerTest
[KAFKA-17779] - Fix flaky RemoteLogManagerTest#testFetchOffsetByTimestampWithTieredStorageDoesNotFetchIndexWhenExistsLocally
[KAFKA-17837] - Rewrite DeleteTopicTest
[KAFKA-17839] - DescribeConsumerGroupTest doesn't really test different arguments
[KAFKA-17854] - Improve tests for ReadOnlyWindowStoreStub#fetch and #backwardFetch
[KAFKA-17899] - Add more unit tests for NetworkReceive
[KAFKA-17978] - StreamsUpgradeTest#test_rolling_upgrade_with_2_bounces system tests fail
[KAFKA-17979] - Change [pytest] to [tool:pytest] in setup.cfg file
[KAFKA-17998] - Fix flaky test OffloadAndTxnConsumeFromLeaderTest.executeTieredStorageTest
[KAFKA-18000] - Fix flaky ReplicaManagerTest#testSuccessfulBuildRemoteLogAuxStateMetrics
[KAFKA-18008] - Flaky testMultiConsumerSessionTimeoutOnStopPolling
[KAFKA-18036] - TransactionsWithTieredStoreTest testReadCommittedConsumerShouldNotSeeUndecidedData is flaky
[KAFKA-18038] - Flaky StreamThreadTest shouldLogAndRecordSkippedRecordsForInvalidTimestamps
[KAFKA-18104] - Add test for in Loggers#currentLoggers
[KAFKA-18158] - Add subscribePattern integration tests for subsequent subscription
[KAFKA-18218] - Trogdor system test can't find coordinator
[KAFKA-18277] - Convert network_degrade_test to Kraft mode
[KAFKA-18280] - fix e2e TestSecurityRollingUpgrade.test_rolling_upgrade_sasl_mechanism_phase_one
[KAFKA-18346] - Fix e2e TestKRaftUpgrade from v3.3.2
[KAFKA-18415] - Flaky ApplicationEventHandlerTest testRecordApplicationEventQueueSize
[KAFKA-18525] - Update the Test should run for async consumer
[KAFKA-18546] - Change ClientUtilsTest#testParseAndValidateAddressesWithReverseLookup to mock test
[KAFKA-18568] - Flaky test ClientIdQuotaTest
[KAFKA-18771] - Flaky test KRaftClusterTest .testDescribeQuorumRequestToControllers
[KAFKA-18790] - testCustomQuotaCallback time out build failure
Sub-task
[KAFKA-12601] - Remove deprecated `delegation.token.master.key`
[KAFKA-12823] - Remove Deprecated method KStream#through
[KAFKA-12824] - Remove Deprecated method KStream#branch
[KAFKA-12826] - Remove Deprecated Class Serdes (Streams)
[KAFKA-12827] - Remove Deprecated method KafkaStreams#setUncaughtExceptionHandler
[KAFKA-12828] - Remove Deprecated methods under KeyQueryMetadata
[KAFKA-12829] - Remove Deprecated methods can classes of old Processor API
[KAFKA-12830] - Remove Deprecated constructor in TimeWindowedDeserializer and TimeWindowedSerde
[KAFKA-12832] - Remove Deprecated methods under RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter
[KAFKA-12834] - Remove Deprecated method MockProcessorContext#setTimestamp
[KAFKA-12844] - KIP-740 follow up: clean up TaskId
[KAFKA-13093] - Log compaction should write new segments with record version v2 (KIP-724)
[KAFKA-14482] - Move LogLoader to storage module
[KAFKA-14483] - Move LocalLog to storage module
[KAFKA-14510] - Extend DescribeConfigs API to support group configs
[KAFKA-14511] - Extend AlterIncrementalConfigs API to support group config
[KAFKA-14562] - Implement epoch bump after every transaction
[KAFKA-14577] - Move ConsoleProducer to tools
[KAFKA-14587] - Move AclCommand to tools
[KAFKA-14705] - Remove deprecated options and redirections
[KAFKA-14772] - Add ConsumerGroupHeartbeat API to AuthorizerIntegrationTest
[KAFKA-15621] - Add histogram metrics to GroupCoordinatorRuntimeMetrics
[KAFKA-15908] - Remove deprecated Consumer API poll(long timeout)
[KAFKA-15909] - Throw error when consumer configured with empty/whitespace-only group.id for LegacyKafkaConsumer
[KAFKA-16106] - group size counters do not reflect the actual sizes when operations fail
[KAFKA-16324] - Move BrokerApiVersionsCommand to tools
[KAFKA-16327] - Remove Deprecated variable StreamsConfig#TOPOLOGY_OPTIMIZATION
[KAFKA-16328] - Remove Deprecated config from StreamsConfig
[KAFKA-16329] - Remove Deprecated Task/ThreadMetadata classes and related methods
[KAFKA-16330] - Remove Deprecated methods/variables from TaskId
[KAFKA-16331] - Remove Deprecated EOSv1
[KAFKA-16332] - Remove Deprecated builder methods for Time/Session/Join/SlidingWindows
[KAFKA-16333] - Removed Deprecated methods KTable#join
[KAFKA-16334] - Remove Deprecated command line option from reset tool
[KAFKA-16335] - Remove Deprecated method on StreamPartitioner
[KAFKA-16339] - Remove Deprecated "transformer" methods and classes
[KAFKA-16379] - Coordinator flush time and event purgatory time metrics
[KAFKA-16540] - Update partitions when the min isr config is updated.
[KAFKA-16607] - Have metrics implementation include the new state
[KAFKA-16681] - Rewrite MiniKDC by Java
[KAFKA-16682] - Rewrite JassTestUtils by Java
[KAFKA-16683] - Extract security-related helpers from scala.TestUtils to java class
[KAFKA-16726] - Add dynamic group configuration for offset reset
[KAFKA-16732] - Investigate missing values for share-coordinator-metrics and share-group-metrics in the broker
[KAFKA-16733] - Add support for formatting new records written to offsets topic in kafka-dump-log.sh
[KAFKA-16734] - Add support for formatting records written to share-group state topic in kafka-dump-log.sh
[KAFKA-16736] - Remove offsets.commit.required.acks in 4.0
[KAFKA-16745] - Add support for share fetch request in KafkaApis
[KAFKA-16746] - Add support for share acknowledgement request in KafkaApis
[KAFKA-16751] - Implement release acquired records in SharePartitionManager
[KAFKA-16754] - Implement release acquired records functionality in SharePartition
[KAFKA-16755] - Implement lock timeout functionality in SharePartition
[KAFKA-16813] - Add global timeout for "@Test" and "@TestTemplate"
[KAFKA-16845] - Migrate ReplicationQuotasTestRig to new test infra
[KAFKA-16896] - upgrade spotless version after we drop JDK8
[KAFKA-16964] - Integration tests for adding and removing voters
[KAFKA-16972] - Move `BrokerTopicStats` and `BrokerTopicMetrics` to `org.apache.kafka.storage.log.metrics` (storage module)
[KAFKA-17002] - Integrate partition leader epoch in Share Partition
[KAFKA-17003] - Implement SharePartitionManager close functionality
[KAFKA-17004] - MINOR: Remove extra synchronized blocks in SharePartitionManager
[KAFKA-17005] - Online protocol migration integration tests
[KAFKA-17026] - Implement updateCacheAndOffsets functionality on LSO movement
[KAFKA-17036] - KIP-919 supports for `createAcls`, `deleteAcls`, `describeAcls`
[KAFKA-17038] - KIP-919 supports for `alterPartitionReassignments` and `listPartitionReassignments`
[KAFKA-17071] - SharePartition - Add more unit tests
[KAFKA-17225] - Refactor consumer membership managers
[KAFKA-17228] - Static member using new protocol should always replace the one using the old protocol
[KAFKA-17231] - Share consumer node latency metrics
[KAFKA-17247] - Revised share group record schemas
[KAFKA-17256] - KRAFT should honor the listener name and security protocol from ClusterConfig
[KAFKA-17265] - Fix flaky MemoryRecordsBuilderTest#testBuffersDereferencedOnClose
[KAFKA-17272] - System tests for protocol migration
[KAFKA-17273] - Migrate BootstrapControllersIntegrationTest to use ClusterTestExtensions
[KAFKA-17288] - Remove tracking member partition epoch for share groups
[KAFKA-17289] - Add integration test for ShareGroupDescribe requests
[KAFKA-17298] - Update upgrade notes for 4.0
[KAFKA-17306] - Soften the validation when replaying tombstones
[KAFKA-17311] - testClientInstanceId, testClientInstanceIdInvalidTimeout, and testClientInstanceIdNoTelemetryReporterRegistered should include CONSUMER protocol
[KAFKA-17312] - Fix tests shouldForwardAllDbOptionsCalls and shouldOverwriteAllOptionsMethods in RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapterTest
[KAFKA-17317] - Validate and maybe trigger downgrade after static member replacement
[KAFKA-17318] - Introduce ConsumerRecord.deliveryCount() and remove deprecations
[KAFKA-17320] - Move SensorAccess to server-common module
[KAFKA-17329] - Implement DefaultShareStatePersister
[KAFKA-17341] - Refactor consumer heartbeat managers
[KAFKA-17342] - Move common coordinator code from group-coordinator into separate module coordinator-common
[KAFKA-17350] - Improve kafka-share-groups.sh --describe output for empty groups
[KAFKA-17356] - add integration test for KAFKA-17310
[KAFKA-17357] - Replace the deprecated `retry.maxRetries`/`retry.maxFailures` by `develocity.testRetry.maxRetries`/`develocity.testRetry.maxFailures`/
[KAFKA-17368] - Add delivery count to kafka-console-share-consumer.sh
[KAFKA-17372] - Move `ThrottledChannelExpirationTest#testThrottledChannelDelay` and `ThrottledChannel` to server module
[KAFKA-17374] - add bootstrap.controller to kafka-reassign-partitions.sh
[KAFKA-17376] - Use the new group coordinator by default in 4.0
[KAFKA-17378] - Initial performance testing fixes
[KAFKA-17383] - Update upgrade notes about removal of `offsets.commit.required.acks`
[KAFKA-17385] - Remove authorizer, authorizer-properties, zk-tls-config-file in AclCommand
[KAFKA-17386] - Remove broker-list, threads, num-fetch-threads in ConsumerPerformance
[KAFKA-17387] - Remove broker-list in VerifiableConsumer
[KAFKA-17388] - Remove broker-list in VerifiableProducer
[KAFKA-17390] - Remove broker-list in GetOffsetShell
[KAFKA-17392] - Remove whitelist in ConsoleConsumerOptions
[KAFKA-17393] - Remove message.format.version and in TopicConfig
[KAFKA-17395] - Flaky test testMissingOffsetNoResetPolicy for new consumer
[KAFKA-17400] - Introduce a purgatory to deal with share fetch requests that cannot be completed instantaneously
[KAFKA-17401] - Add config for purge interval for Share Fetch Purgatory
[KAFKA-17406] - Move ClientIdAndBroker to server-common module
[KAFKA-17413] - Re-introduce `group.version` feature flag
[KAFKA-17414] - Move RequestLocal to server-common module
[KAFKA-17425] - Improve coexistence of consumer groups and share groups
[KAFKA-17430] - Move RequestChannel.Metrics and RequestChannel.RequestMetrics to server module
[KAFKA-17449] - Move Quota classes to server-common module
[KAFKA-17450] - Optimise the handler methods in ShareConsumeRequestManager.
[KAFKA-17451] - Remove deprecated Consumer#committed
[KAFKA-17468] - Move kafka.log.remote.quota to storage module
[KAFKA-17472] - Speed Up DescribeConsumerGroupTest
[KAFKA-17473] - Speed Up ClientTelemetryTest
[KAFKA-17476] - Delete kafka.common.OffsetAndMetadata
[KAFKA-17491] - Move BrokerServerMetrics to server module
[KAFKA-17495] - Factor out common setup from server-side assignor benchmarks
[KAFKA-17496] - Add heterogeneous configuration to TargetAssignmentBuilderBenchmark
[KAFKA-17500] - NOT_LEADER_OR_FOLLOWER with metadata redirection in share group response
[KAFKA-17509] - Introduce a delayed action queue to complete purgatory actions outside purgatory code.
[KAFKA-17510] - Refactor code between SharePartitionManager and DelayedShareFetch for share partition initialization
[KAFKA-17511] - Move ElectLeadersRequestOps to ElectLeadersRequest
[KAFKA-17512] - Move LogSegmentTest to storage module
[KAFKA-17513] - Move LogSegmentsTest to storage module
[KAFKA-17528] - Remove whitelist/blacklist from JMXReporter
[KAFKA-17529] - Remove blacklist from MM2
[KAFKA-17530] - Remove blacklist/whitelist from ReplaceField
[KAFKA-17544] - Add a log message for KafkaShareConsumer early access
[KAFKA-17545] - Get rid of fetch queue in SharePartitionManager
[KAFKA-17546] - Create kafka-groups.sh tool
[KAFKA-17548] - Improve admin client integration tests
[KAFKA-17549] - Improve error messages for incorrect group type in kafka-share-groups.sh
[KAFKA-17550] - Exploit DescribeGroups v6 API in kafka-consumer-groups.sh
[KAFKA-17563] - Move RequestConvertToJson to server module
[KAFKA-17564] - Move BrokerFeatures to server module
[KAFKA-17567] - Remove TestTruncate
[KAFKA-17568] - Rewrite TestPurgatoryPerformance by Java
[KAFKA-17569] - Rewrite TestLinearWriteSpeed by JMH
[KAFKA-17570] - Rewrite StressTestLog by JMH
[KAFKA-17571] - Revert #17219
[KAFKA-17576] - Fix all references to kraft/server.properties to use reconfig-server.properties
[KAFKA-17578] - Remove partitionRacks from TopicMetadata
[KAFKA-17589] - Move JUnit extensions to test-common module
[KAFKA-17592] - Support for subscription regex in HB RPC
[KAFKA-17593] - Async regex resolution
[KAFKA-17598] - Command line validation tool for RE2J regex
[KAFKA-17602] - Remove `ConfigUtils.translateDeprecatedConfigs` from JmxReporter
[KAFKA-17612] - Remove some tests that only apply to ZK mode or migration
[KAFKA-17613] - Remove ZK migration code
[KAFKA-17614] - Remove AclAuthorizer
[KAFKA-17615] - Remove KafkaServer references in tests
[KAFKA-17616] - Remove KafkaServer
[KAFKA-17620] - Simplify share partition acquire API
[KAFKA-17621] - Reduce logging verbosity on ConsumerGroupHeartbeat path
[KAFKA-17624] - Remove the E2E uses of accessing ACLs from zk
[KAFKA-17634] - Tighten up wakeup handling for share consumer
[KAFKA-17639] - Add Java 23 to CI build matrix
[KAFKA-17640] - Document Java 23 support and include release note
[KAFKA-17641] - Update vote RPC with new pre-vote field
[KAFKA-17642] - Add ProspectiveState and ProspectiveStateTest
[KAFKA-17643] - Response handling for pre-vote set to True
[KAFKA-17653] - Update Scala version to 2.13.15 for support Java 23
[KAFKA-17668] - Rewrite LogCleaner#maxOverCleanerThreads and LogCleanerManager#maintainUncleanablePartitions
[KAFKA-17672] - Run quarantined tests separately
[KAFKA-17675] - Add tests to RaftEventSimulationTest
[KAFKA-17677] - Consider removing `atomicGetOrUpdate`
[KAFKA-17683] - Remove ZK from Kafka READMEs in 4.0
[KAFKA-17684] - Run tests on Java 11 and Java 21
[KAFKA-17692] - Remove KafkaServer references in streams tests
[KAFKA-17703] - Move DelayedActionsQueue outside DelayedShareFetch
[KAFKA-17707] - Remove zk from BaseConsumerTest
[KAFKA-17710] - Rework UniformHeterogeneousAssignor to improve performance
[KAFKA-17720] - Remove zookeeper_migration_test.py
[KAFKA-17721] - Enable to configure listener name and protocol for controller
[KAFKA-17722] - Fix "this-escape" compiler warnings (BrokerTopicMetrics) for JDK 23
[KAFKA-17723] - Fix "this-escape" compiler warnings (MultiThreadedEventProcessor and DistributedHerder) for JDK 23
[KAFKA-17724] - Clients - resolve race condition for SubscriptionState in share group consumer
[KAFKA-17729] - Remove ZooKeeper from jmh-benchmarks
[KAFKA-17733] - Protocol upgrade should allow empty member assignment in group conversion
[KAFKA-17737] - E2E tests need to drop Kafka versions prior to 1.0.0
[KAFKA-17738] - upgrade base image from jdk8 to jdk11
[KAFKA-17739] - Clean up build.gradle to adopt the minimum Java version as 11.
[KAFKA-17740] - Update Readme and documentation
[KAFKA-17741] - cleanup code base for JDK 11
[KAFKA-17742] - Move DelayedShareFetchPurgatory declaration to ReplicaManager
[KAFKA-17743] - Add minBytes implementation to DelayedShareFetch
[KAFKA-17746] - Replace JavaConverters with CollectionConverters
[KAFKA-17759] - Remove Utils.mkSet
[KAFKA-17763] - Remove Utils.covariantCast
[KAFKA-17764] - Remove unnecessary ignorable annotations from RPC schemas
[KAFKA-17765] - Remove TestUtils.toSet
[KAFKA-17773] - Upgrade spotbug to work under java 23
[KAFKA-17774] - Add capability for max fetch records in share fetch
[KAFKA-17783] - Remove sharePartition from SharePartitionManager if the partition is deleted or becomes a follower
[KAFKA-17787] - Remove --zookeeper option and logic from ConfigCommand
[KAFKA-17796] - Persist higher leaderEpoch in read state calls to the share coordinator.
[KAFKA-17810] - Update Jetty to 9.4.56 for trunk, 3.8 and 3.7
[KAFKA-17811] - Separate modules to use different JDKs
[KAFKA-17812] - upgrade base image of e2e from JDK 11 to JDK 17
[KAFKA-17814] - Use `final` declaration to replace the suppression `this-escape`
[KAFKA-17827] - cleanup the mockit version
[KAFKA-17832] - Remove all EnabledForJreRange
[KAFKA-17840] - Move ReplicationQuotaManager, ClientRequestQuotaManager and QuotaFactory to server module
[KAFKA-17848] - Resolve reliability issue on the broker post moving delayed share fetch purgatory to ReplicaManager
[KAFKA-17857] - Move AbstractResetIntegrationTest and subclasses to tools
[KAFKA-17860] - Remove log4j-appender module
[KAFKA-17878] - Move ActionQueue to server module
[KAFKA-17879] - test_performance_services.py should use DEV version to run kafka service
[KAFKA-17880] - Move integration test from streams module to streams/integration-tests module
[KAFKA-17883] - Fix jvm error caused by UseParNewGC when running old kafka client in e2e
[KAFKA-17888] - Upgrade ZooKeeper version from 3.4.9 to 3.5.7 to avoid ZOOKEEPER-3779, which can't run under JDK 11
[KAFKA-17890] - Move DelayedOperationPurgatory to server-common
[KAFKA-17896] - Create Admin.describeClassicGroups
[KAFKA-17910] - Create integration tests for Admin.listGroups and Admin.describeClassicGroups
[KAFKA-17912] - Align string conversion of SharePartitionKey
[KAFKA-17914] - Refactor ShareCoordinator.partitionFor to use SharePartitionKey argument
[KAFKA-17921] - Support SASL_PLAINTEXT protocol with java.security.auth.login.config
[KAFKA-17922] - add helper to ClusterInstance to create client component
[KAFKA-17923] - Remove old kafka version from e2e
[KAFKA-17933] - Add ShareRoundTripWorker for ShareConsumers.
[KAFKA-17943] - Error handling when calling replicaManager.getPartitionOrException
[KAFKA-17944] - Add error handling in the cases when replicaManager.fetchOffsetForTimestamp throws an error
[KAFKA-17945] - Use leader epoch while calling replicaManager.fetchOffsetForTimestamp in ShareFetchUtils
[KAFKA-17948] - Potential issue during tryComplete and onComplete simultaneous calls to access global variables
[KAFKA-17949] - Add GroupState and ListGroupsOptions.inStates
[KAFKA-17956] - Remove Admin.listShareGroups in favour of Admin.listGroups
[KAFKA-17970] - Move some purgatory classes from core to share
[KAFKA-17973] - Relax Restriction for Voters Set Change
[KAFKA-17977] - Remove new_consumer from E2E
[KAFKA-17985] - Set default value for share.auto.offset.reset in ShareRoundTripWorker.
[KAFKA-17987] - Remove assorted ZK-related files
[KAFKA-17991] - Timed calls to future.get in DefaultStatePersister and test improvements
[KAFKA-18003] - add test to make sure `Admin#deleteRecords` can handle the corrupted records
[KAFKA-18004] - Use version 3.8 to run the ZooKeeper service for end-to-end tests
[KAFKA-18009] - Remove spurious public constructor for KafkaShareConsumer
[KAFKA-18011] - Remove ZooKeeper from the docs
[KAFKA-18012] - Update the Scram configuration section for KRaft
[KAFKA-18013] - Add duration based offset reset option for Kafka consumer
[KAFKA-18014] - Add duration based offset reset option for ShareConsumer
[KAFKA-18015] - Add duration based offset reset option for Kafka Streams
[KAFKA-18016] - Fix bugs in handling piggyback acknowledgements in ShareFetch
[KAFKA-18019] - Convert INVALID_PRODUCER_ID_MAPPING from abortable error to fatal error
[KAFKA-18022] - fetchOffsetMetadata handling for minBytes estimation in both common/uncommon cases of share fetch
[KAFKA-18030] - Remove old upgrade-system-tests modules
[KAFKA-18045] - Add 0.11, 1.0, 1.1, and 2.0 back to streams_upgrade_test.py
[KAFKA-18056] - Fix a bug related to updating the callbacks after commitAsync.
[KAFKA-18058] - Integrate offset cleanup algorithm with share coordinator
[KAFKA-18076] - Remove `isZkMigrationTest` and related code
[KAFKA-18079] - `consumer-config` does not work with console-share-consumer
[KAFKA-18081] - Remove isKRaftTest from the kraft-only tests
[KAFKA-18084] - Null and leaked AcquisitionLockTimerTask causes hanging AcknowledgeRequest and corrupted state of batch
[KAFKA-18086] - Enable propagation of the error message when writing state
[KAFKA-18090] - Create specific classes for share group members and assignments
[KAFKA-18097] - Upgrade readme to include min JDK changes
[KAFKA-18098] - add kraft support to testReplicaPlacementAllServers and testReplicaPlacementPartialServers
[KAFKA-18116] - Admin LeaveGroup fails for static consumer group member
[KAFKA-18122] - Add ShareConsumeBenchWorker for running trogdor workloads.
[KAFKA-18124] - Remove zk migration from RaftManagerTest, BrokerLifecycleManagerTest, KafkaConfigTest, and ReplicaManagerTest
[KAFKA-18129] - `SharePartition#maybeInitialize` should complete the future outsize the write lock
[KAFKA-18132] - Remove "session.timeout.ms" from connect-distributed.properties to fix connect e2e
[KAFKA-18134] - Reject the upgrade if the member assignment has non empty userData
[KAFKA-18136] - Remove zk migration from code base
[KAFKA-18140] - Remove ZooKeeperMainWithTlsSupportForKafka
[KAFKA-18144] - Move storage exceptions out of core
[KAFKA-18145] - Fix failed e2e ConnectDistributedTest.test_dynamic_logging
[KAFKA-18146] - tests/kafkatest/tests/core/upgrade_test.py needs to be re-added as KRaft
[KAFKA-18148] - the persister methods deleteState, initializeState, and readSummary should not declare exceptions in their method signatures
[KAFKA-18150] - Maybe downgrade the consumer group if a consumer member leaves the group though admin client
[KAFKA-18152] - add 0.11, 1.0, 1.1, and 2.0 streams dependencies to dockerfile
[KAFKA-18155] - Fix bug in response handler for ShareAcknowledge in clients.
[KAFKA-18156] - VerifiableConsumer should ignore "--session-timeout" when using CONSUMER protocol
[KAFKA-18160] - Interrupting or waking up onPartitionsAssigned in AsyncConsumer can cause the ConsumerRebalanceListenerCallbackCompletedEvent to be skipped
[KAFKA-18162] - Move LocalLogTest to storage module
[KAFKA-18164] - Empty in memory acknowledgements when a shareAcknowledge request fails
[KAFKA-18165] - Update nodesWithPendingRequests only when a request successfully builds
[KAFKA-18169] - Mark KIP-848's API & Records as stable
[KAFKA-18174] - Handle hdr histogram exceptions gracefully
[KAFKA-18178] - Delete ZkSecurityMigrator
[KAFKA-18179] - Move AsyncOffsetReadFutureHolder to storage module
[KAFKA-18186] - add sourceCompatibility back to build.gradle to allow idea to configure suitable language level automatically
[KAFKA-18188] - Admin LeaveGroup should allow removing member using consumer protocol by member id
[KAFKA-18192] - generator module should run under java 11
[KAFKA-18197] - Check if nodeToSend is null before sending ShareAcknowledge.
[KAFKA-18226] - Disable CustomQuotaCallbackTest and remove `isKRaftTest`
[KAFKA-18229] - Move configs out of "kraft" directory
[KAFKA-18243] - Loggers breaks the use of root logger after migrating to log4j2
[KAFKA-18247] - Use log4j2 yaml to rewrite the fix of KAFKA-18145
[KAFKA-18269] - Remove deprecated protocol APIs support in 4.0 (KIP-724, KIP-896)
[KAFKA-18270] - SaslHandshake v0 and FindCoordinator v0 incorrectly tagged as deprecated
[KAFKA-18272] - Deprecated protocol api usage should be logged at info level
[KAFKA-18313] - Fix to Kraft or remove tests associate with Zk Broker config in SocketServerTest
[KAFKA-18314] - Fix to Kraft or remove tests associate with Zk Broker config in KafkaApisTest
[KAFKA-18315] - Fix to Kraft or remove tests associate with Zk Broker config in DynamicBrokerConfigTest, ReplicaManagerTest, DescribeTopicPartitionsRequestHandlerTest, KafkaConfigTest
[KAFKA-18316] - Fix to Kraft or remove tests associate with Zk Broker config in ConnectionQuotasTest
[KAFKA-18317] - Remove zookeeper.connect from RemoteLogManagerTest
[KAFKA-18318] - Add more logs to the migration path
[KAFKA-18320] - Ensure that assignors are at the right place
[KAFKA-18330] - Update documentation to remove controller deployment limitations
[KAFKA-18331] - Update process.roles to required configuration
[KAFKA-18334] - Produce v4-v6 should be undeprecated
[KAFKA-18339] - Remove raw unversioned direct SASL protocol (KIP-896)
[KAFKA-18341] - Remove KafkaConfig GroupType config check and warn log
[KAFKA-18352] - DeleteGroups v0 incorrectly tagged as deprecated
[KAFKA-18353] - Remove zk config `control.plane.listener.name`
[KAFKA-18359] - Set zkConnect to null in LocalLeaderEndPointTest, HighwatermarkPersistenceTest, IsrExpirationTest, ReplicaManagerQuotasTest, OffsetsForLeaderEpochTest
[KAFKA-18360] - Remove zookeeper configurations
[KAFKA-18361] - Remove PasswordEncoderConfigs
[KAFKA-18363] - Clean up Server config documents which contained zookeeper
[KAFKA-18365] - Remove zookeeper.connect in Test
[KAFKA-18366] - Remove KafkaConfig.interBrokerProtocolVersion
[KAFKA-18367] - Remove ZkConfigManager
[KAFKA-18368] - Remove TestUtils#MockZkConnect and remove zkConnect from TestUtils#createBrokerConfig
[KAFKA-18373] - Remove ZkMetadataCache
[KAFKA-18374] - Remove EncryptingPasswordEncoder
[KAFKA-18383] - Remove reserved.broker.max.id and broker.id.generation.enable
[KAFKA-18384] - Remove ZkAlterPartitionManager
[KAFKA-18399] - Remove ZooKeeper from KafkaApis
[KAFKA-18405] - Remove ZooKeeper logic from DynamicBrokerConfig
[KAFKA-18406] - Remove ZkBrokerEpochManager
[KAFKA-18407] - Remove ZkAdminManager, DelayedCreatePartitions, CreatePartitionsMetadata, ZkConfigRepository, DelayedDeleteTopics
[KAFKA-18411] - Remove ZkProducerIdManager
[KAFKA-18412] - Remove EmbeddedZookeeper
[KAFKA-18413] - Remove AdminZkClient
[KAFKA-18414] - Remove KRaftRegistrationResult
[KAFKA-18417] - Remove controlled.shutdown.max.retries and controlled.shutdown.retry.backoff.ms
[KAFKA-18423] - Remove ZkData and related unused references
[KAFKA-18425] - Remove OffsetTrackingListener
[KAFKA-18426] - Remove FinalizedFeatureChangeListener
[KAFKA-18427] - Remove ZooKeeperClient
[KAFKA-18429] - Remove ZkFinalizedFeatureCache and StateChangeFailedException
[KAFKA-18430] - Remove ZkNodeChangeNotificationListener
[KAFKA-18431] - Remove KafkaController
[KAFKA-18432] - Remove unused code from AutoTopicCreationManager
[KAFKA-18435] - Remove zookeeper dependencies in build.gradle
[KAFKA-18437] - Correct version of ShareUpdateValue record from v1 to v0
[KAFKA-18443] - Remove ZkFourLetterWords
[KAFKA-18445] - Remove LazyDownConversionRecords and LazyDownConversionRecordsSend
[KAFKA-18446] - Remove MetadataCacheControllerNodeProvider
[KAFKA-18449] - Add share-group state configs to reconfigure-server.properties
[KAFKA-18472] - Remove MetadataSupport
[KAFKA-18474] - Remove zkBroker listener
[KAFKA-18485] - Update log4j2.yaml
[KAFKA-18487] - Remove ReplicaManager#stopReplicas
[KAFKA-18491] - Remove zkClient & maybeUpdateMetadataCache from ReplicaManager
[KAFKA-18492] - Cleanup RequestHandlerHelper
[KAFKA-18499] - Clean up LogConfig
[KAFKA-18502] - Remove kafka.controller.Election
[KAFKA-18503] - Remove TopicDeletionManager
[KAFKA-18505] - Remove ControllerEventManager
[KAFKA-18506] - Remove ControllerState
[KAFKA-18510] - Remove ControllerChannelContext
[KAFKA-18511] - Remove ControllerChannelManager
[KAFKA-18512] - Remove ReplicaStateMachine
[KAFKA-18514] - Refactor share module to server and server-common
[KAFKA-18515] - Remove DelegationTokenManagerZk
[KAFKA-18516] - Remove RackAwareMode
[KAFKA-18517] - ConsumerBounceTest should run for async consumer
[KAFKA-18519] - Remove Json.scala , cleanup AclEntry.scala
[KAFKA-18520] - Remove ZooKeeper logic from JaasUtils
[KAFKA-18521] - Cleanup NodeApiVersions zkMigrationEnabled field
[KAFKA-18528] - MultipleListenersWithSameSecurityProtocolBaseTest and GssapiAuthenticationTest should run for async consumer
[KAFKA-18530] - Remove ZooKeeperInternals
[KAFKA-18532] - Clean Partition.scala zookeeper logic
[KAFKA-18533] - Remove KafkaConfig zookeeper related logic
[KAFKA-18539] - sharePartitionManager and clientMetricsManager should be required to KafkaAPIs
[KAFKA-18540] - Remove UpdataMetadataRequest in KafkaApisTest
[KAFKA-18542] - Cleanup AlterPartitionManager
[KAFKA-18553] - Update javadoc and comment of ConfigType
[KAFKA-18556] - Remove JaasModule#zkDigestModule, JaasTestUtils#zkSections and its usage
[KAFKA-18565] - Cleanup SaslSetup
[KAFKA-18578] - Remove UpdateMetadataRequest in MetadataCacheTest
[KAFKA-18588] - Remove TopicKey.scala
[KAFKA-18589] - Cleanup GroupMetadataManager
[KAFKA-18591] - Remove QuotaConfgHandler
[KAFKA-18595] - Remove AuthorizerUtils#sessionToRequestContext
[KAFKA-18599] - Remove optional ForwardingManager in ApiVersionManager
[KAFKA-18611] - Update MV to enable transaction version 2 by default for 4.0
[KAFKA-18622] - Enable ConsumerBounceTest testClose for new consumer
[KAFKA-18623] - Enable ConsumerBounceTest testCloseDuringRebalance for new consumer
[KAFKA-18631] - Remove ZkConfigs
[KAFKA-18646] - Null records breaks librdkafka
[KAFKA-18648] - kafka-python requires metadata request version 0
[KAFKA-18655] - Implement the consumer group size counter with scheduled task
[KAFKA-18659] - librdkafka compressed produce fails unless api versions returns produce v0
[KAFKA-18672] - CoordinatorRecordSerde must validate value versions
[KAFKA-18674] - document the incompatible changes of parsing bootstrap.servers
[KAFKA-18675] - add unit tests for KAFKA-18171 to verify the parser of bootstrap.servers
[KAFKA-18676] - Update Benchmark system tests
[KAFKA-18677] - Update ConsoleConsumerTest system test
[KAFKA-18678] - Update TestVerifiableProducer system test
[KAFKA-18741] - document the removal of `inter.broker.protocol.version`
[KAFKA-18829] - Can't use explicit ack when poll returns an empty batch
[KAFKA-18860] - Clarify the KRaft missing feature
[KAFKA-18886] - add behavior change of CreateTopicPolicy and AlterConfigPolicy to zk2kraft
[KAFKA-18908] - the size of appended value can't be larger than Short.MAX_VALUE
[KAFKA-18919] - Clarify that KafkaPrincipalBuilder classes must also implement KafkaPrincipalSerde