Commit Graph

292 Commits

Author SHA1 Message Date
PoAn Yang 949617b0b2
KAFKA-17747: [7/N] Add consumer group integration test for rack aware assignment (#19856)
* Add `RackAwareAssignor`. It uses `racksForPartition` to check the rack
id of a partition and assign it to a member which has the same rack id.
* Add `ConsumerIntegrationTest#testRackAwareAssignment` to check
`racksForPartition` works correctly.

Reviewers: David Jacot <djacot@confluent.io>

---------

Signed-off-by: PoAn Yang <payang@apache.org>
2025-06-04 19:32:17 +02:00
Kirk True 1e917906ab
KAFKA-18573: Add support for OAuth jwt-bearer grant type (#19754)
CI / build (push) Waiting to run Details
Adding support for the `urn:ietf:params:oauth:grant-type:jwt-bearer`
grant type (AKA `jwt-bearer`). Includes further refactoring of the
existing OAuth layer and addition of generic JWT assertion layer that
can be leveraged in the future.

This constitutes the main piece of the JWT Bearer grant type support.

Forthcoming commits/PRs will include improvements for both the
`client_credentials` and `jwt-bearer` grant types in the following
areas:

* Integration test coverage (KAFKA-19153)
* Unit test coverage (KAFKA-19308)
* Top-level documentation (KAFKA-19152)
* Improvements to and documentation for `OAuthCompatibilityTool`
(KAFKA-19307)

Reviewers: Manikumar Reddy <manikumar@confluent.io>, Lianet Magrans
 <lmagrans@confluent.io>

---------

Co-authored-by: Zachary Hamilton <77027819+zacharydhamilton@users.noreply.github.com>
Co-authored-by: Lianet Magrans <98415067+lianetm@users.noreply.github.com>
2025-06-04 09:01:05 -04:00
Ken Huang bcda92b5b9
KAFKA-19080 The constraint on segment.ms is not enforced at topic level (#19371)
CI / build (push) Waiting to run Details
The main issue was that we forgot to set
`TopicConfig.SEGMENT_BYTES_CONFIG` to at least `1024 * 1024`, which
caused problems in tests with small segment sizes.

To address this, we introduced a new internal config:
`LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG`, allowing us to set smaller
segment bytes specifically for testing purposes.

We also updated the logic so that if a user configures the topic-level
segment bytes without explicitly setting the internal config, the
internal value will no longer be returned to the user.

In addition, we removed
`MetadataLogConfig#METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG` and added
three new internal configurations:
- `INTERNAL_MAX_BATCH_SIZE_IN_BYTES_CONFIG`
- `INTERNAL_MAX_FETCH_SIZE_IN_BYTES_CONFIG`
- `INTERNAL_DELETE_DELAY_MILLIS_CONFIG`

Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-05-25 20:57:22 +08:00
Alyssa Huang 042be5b9ac
MINOR: Fix some Request toString methods (#19655)
CI / build (push) Waiting to run Details
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2025-05-09 23:42:34 -07:00
Logan Zhu 50fb993ce0
KAFKA-19136 Move metadata-related configs from KRaftConfigs to MetadataLogConfig (#19465)
Separates metadata-related configurations from the `KRaftConfigs` into
the `MetadataLogConfig` class.

Previously, metadata-related configs were placed in `KRaftConfigs`,
which mixed server-related configs (like process.roles) with
metadata-specific ones (like metadata.log.*), leading to confusion and
tight coupling.

In this PR:
- Extract metadata-related config definitions and variables from
`KRaftConfig` into `MetadataLogConfig`.
- Move `node.id` out of `MetadataLogConfig` into `KafkaMetadataLog’s
constructor` to avoid redundant config references.
- Leave server-related configurations in `KRaftConfig`, consistent with
its role.

This separation makes `KafkaConfig` and `KRaftConfig` cleaner, and
aligns with the goal of having a dedicated MetadataLogConfig class for
managing metadata-specific configurations.

Reviewers: PoAn Yang <payang@apache.org>, Ken Huang
 <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-04-17 22:17:11 +08:00
David Jacot b6adec48c5
KAFKA-18616; Refactor Tools's ApiMessageFormatter (#18695)
This patch refactors the `ApiMessageFormatter` to follow what we have
done in https://github.com/apache/kafka/pull/18688.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-26 03:12:38 -07:00
Mickael Maison 121ec2a662
KAFKA-15599 Move MetadataLogConfig to raft module (#19246)
Rewrite the class in Java and move it to the raft module.

Reviewers: PoAn Yang <payang@apache.org>, TengYao Chi
<kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-21 13:44:20 +08:00
Ken Huang d5413fdb48
KAFKA-17856 Move ConfigCommandTest and ConfigCommandIntegrationTest to tool module (#17767)
Reviewers: TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-09 21:05:36 +08:00
Piotr P. Karwasz 666571216b
KAFKA-18483 Disable `Log4jController` and `Loggers` if Log4j Core absent (#18496)
If Log4j Core is absent, most calls to Log4jController and Loggers will end up with a NoClassDefFoundError.

This changeset:

- Profits from the major version bump to rename k.util.Log4jController to LoggingController.
- Removes o.a.l.l.Level from the signature of public methods of o.a.k.connect.runtime.Loggers and replaces it with String.
- Provides an additional no-op implementation of k.util.LoggingController and o.a.k.connect.runtime.Loggers: if Log4j Core is not present on the runtime classpath the no-op implementation will be used.

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-02-07 00:04:33 +08:00
Mickael Maison 71314739f9
KAFKA-15995: Initial API + make Producer/Consumer plugins Monitorable (#17511)
Reviewers: Greg Harris <gharris1727@gmail.com>, Luke Chen <showuon@gmail.com>
2025-01-31 10:40:10 +01:00
Chung, Ming-Yen a8f6fc9cc4
KAFKA-18631 Remove ZkConfigs (#18693)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-26 04:37:49 +08:00
David Arthur 8c0a0e07ce
KAFKA-17587 Refactor test infrastructure (#18602)
This patch reorganizes our test infrastructure into three Gradle modules:

":test-common:test-common-internal-api" is now a minimal dependency which exposes interfaces and annotations only. It has one project dependency on server-common to expose commonly used data classes (MetadataVersion, Feature, etc). Since this pulls in server-common, this module is Java 17+. It cannot be used by ":clients" or other Java 11 modules.

":test-common:test-common-util" includes the auto-quarantined JUnit extension. The @Flaky annotation has been moved here. Since this module has no project dependencies, we can add it to the Java 11 list so that ":clients" and others can utilize the @Flaky annotation

":test-common:test-common-runtime" now includes all of the test infrastructure code (TestKitNodes, etc). This module carries heavy dependencies (core, etc) and so it should not normally be included as a compile-time dependency.

In addition to this reorganization, this patch leverages JUnit SPI service discovery so that modules can utilize the integration test framework without depending on ":core". This will allow us to start moving integration tests out of core and into the appropriate sub-module. This is done by adding ":test-common:test-common-runtime" as a testRuntimeOnly dependency rather than as a testImplementation dependency. A trivial example was added to QuorumControllerTest to illustrate this.

Reviewers: Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-24 09:03:43 -05:00
Colin Patrick McCabe c28d9a3486
KAFKA-18435 Remove zookeeper dependencies in build.gradle (#18450)
Remove Apache ZooKeeper from the Apache Kafka build. Also remove commons IO, commons CLI, and netty, which were dependencies we took only because of ZooKeeper.

In order to keep the size of this PR manageable, I did not remove all classes which formerly interfaced with ZK. I just removed the ZK types. Fortunately, Kafka generally wrapped ZK data structures rather than using them directly.

Some classes were pretty entangled with ZK, so it was easier just to stub them out. For ZkNodeChangeNotificationListener.scala, PartitionStateMachine.scala, ReplicaStateMachine.scala, KafkaZkClient.scala, and ZookeeperClient.scala, I replaced all the functions with "throw new UnsupportedOperationException". Since the tests for these classes have been removed, as well as the ZK-based broker code, this should be OK as an incremental step.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-09 10:25:17 +08:00
TengYao Chi af255a0c37
KAFKA-18412: Remove EmbeddedZookeeper (#18399)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2025-01-07 17:08:00 +01:00
Mickael Maison 57eb5fd7dc
KAFKA-14587: Move AclCommand to tools (#17880)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-14 20:05:46 +01:00
TengYao Chi b37b89c668
KAFKA-9366 Upgrade log4j to log4j2 (#17373)
This pull request replaces Log4j with Log4j2 across the entire project, including dependencies, configurations, and code. The notable changes are listed below:

1. Introduce Log4j2 Instead of Log4j
2. Change Configuration File Format from Properties to YAML
3. Adds warnings to notify users if they are still using Log4j properties, encouraging them to transition to Log4j2 configurations

Co-authored-by: Lee Dongjin <dongjin@apache.org>

Reviewers: Luke Chen <showuon@gmail.com>, Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-14 01:14:31 +08:00
snehashisp f4fe6064cc
KAFKA-18215: KIP-891 Connect Multiversioning Support (Configs and Validation changes for Connectors and Converters) (#17741)
Reviewers: Greg Harris <greg.harris@aiven.io>
2024-12-11 15:34:21 -08:00
Christopher L. Shannon bd6d0fbf3d
KAFKA-16437 Upgrade to Jakarta and Jetty 12 (KIP-1032) (#16754)
This commit implements the changes for KIP-1032. This updates Kafka to Jakarta specs, JavaEE 10 and Jetty 12. The changes here primarily effect Kafka Connect and MM2.

Todo/Notes:

1) I bumped the connect modules to JDK 17 but I also had to bump a couple other things that had a dependency on conect. The tools project depends on connect so that had to be bumped, and streams depends on tools so that needed to be bumped. This means we may need to separate some things if we don't want to enforce JDK 17 on streams.

2) There is an issue with a test in DedicatedMirrorIntegrationTest that I had to change for now that involves escaping characters and not quite sure what to do about it yet. The cause is the Servlet 6 spec changing what is allowed in the path. See: Jetty 12: 400: Ambiguous URI path encoding for path <%=FOO%>~1 (encoded: %3C%25%3DFOO%25%3E%7E1) jetty/jetty.project#11890

3) I had to configure the idle timeout in Jetty requests to match our request timeout so tests didn't fail. This was needed to fix the ConnectWorkerIntegrationTest#testPollTimeoutExpiry() test

Testing is being done by just using the existing tests for Connect and MM2 which should be sufficient.

Reviewers: Greg Harris <greg.harris@aiven.io>, David Arthur <mumrah@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-11 23:24:14 +08:00
David Jacot c3506834e9
KAFKA-17598; Command line validation tool for RE2J regex (#18031)
This patch introduces the `--validate-regex` argument to the `kafka-consumer-group` command line tool as defined in KIP-848. The new argument allows the verification of RE2 regular expressions.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Lianet Magrans <lmagrans@confluent.io>
2024-12-04 23:40:32 -08:00
mannoopj be4ea8092b
MINOR: Add git support for schema compatibility checker (#17684)
Add git support for schema compatibility checker. Pulls in valid schema from remote git trunk branch to check with edited schema in local branch. Adds new option for command line verify-evolution-git which takes in a required file name.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2024-11-22 14:02:31 -08:00
Bill Bejeck 1c998f8ef3
KAFKA-17869: Adding tests to ensure KIP-1076 doesn't interfere with consumer metrics[1/3] (#17781)
Adding tests to ensure the KIP-1076 methods don't interfere with existing metrics in clients

Reviewers: Apoorv Mittal <amittal@confluent.io>, Matthias Sax <mjsax@apache.org>
2024-11-21 13:41:29 -05:00
ShivsundarR 6cf4081540
KAFKA-17985: Set share.auto.offset.reset to earliest in ShareRoundTripWorker (#17758)
After the share.auto.offset.reset dynamic config was added for share groups in this commit - 9db5ed0, we needed to update this config value to "earliest" in ShareRoundTripWorker when it creates the consumer.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Manikumar Reddy <manikumar.reddy@gmail.com>
2024-11-12 22:24:28 +05:30
Bill Bejeck 4ed0a958e5
KAFKA-17248 - KIP 1076 implementation (#17021)
Implementation of KIP-1076 to allow for adding client application metrics to the KIP-714 framework

Reviewers: Apoorv Mittal <amittal@confluent.io>, Andrew Schofield <aschofield@confluent.io>, Matthias Sax <mjsax@apache.org>
2024-11-05 11:29:54 -05:00
Kuan-Po Tseng e3f953483c
KAFKA-17857 Move AbstractResetIntegrationTest and subclasses to tools (#17594)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-05 04:16:19 +08:00
Dmitry Werner 1eb7644349
KAFKA-16845 Migrate ReplicationQuotasTestRig to new test infra (#17089)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-25 03:33:01 +08:00
TengYao Chi 553e6b4c6d
KAFKA-17860 Remove log4j-appender module (#17588)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-24 18:13:30 +08:00
Sushant Mahajan 5545d717c3
KAFKA-17633: Add share group record formatter and parser. (#17467)
As part of KIP-932, a new internal topic __share_group_state was introduced. There are 2 types of records which are currently being added in this topic - ShareSnapshotKey/Value and ShareUpdateKey/Value
In light of this, we must make the existing tooling like kafka-console-consumer and kafka-dump-log aware of these records for debugging and introspection purposes.
This PR introduces ShareGroupStateMessageFormatter to be used used with kafka-console-consumer and adds an internal class ShareGroupStateMessageParser in DumpLogSegments.scala.
Unit tests have been added to DumpLogSegmentsTest.scala


Reviewers:  Andrew Schofield <aschofield@confluent.io>,  Manikumar Reddy <manikumar.reddy@gmail.com>
2024-10-15 11:44:15 +05:30
Linsiyuan9 76a1af984b
KAFKA-17746 Replace JavaConverters with CollectionConverters (#17451)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-14 17:13:20 +08:00
Federico Valeri 8dbbf5ea76
KAFKA-14705: Move topic filters to tools module (#17387)
After MirrorMaker 1 removal, there are no other modules dependencies for these classes, so we can safely move them to tools module.

Signed-off-by: Federico Valeri <fedevaleri@gmail.com>
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2024-10-07 16:54:36 +02:00
Dmitry Werner 672c617233
KAFKA-14577: Move ConsoleProducer to tools module (#17157)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Federico Valeri <fedevaleri@gmail.com>
2024-10-07 11:19:59 +02:00
Colin Patrick McCabe dbd50ff847
KAFKA-16469: Metadata schema checker (#15995)
Create a schema checker that can validate that later versions of a KRPC schema are compatible with earlier ones.

Reviewers: David Arthur <mumrah@gmail.com>
2024-10-03 12:13:38 -07:00
Chia-Ping Tsai 979740b49d
KAFKA-17589 Move JUnit extensions to test-common module (#17318)
This patch completely removes the compile-time dependency on core for both test and main sources by introducing two new modules.

1) `test-common` include all the common test implementation code (including dependency on :core for BrokerServer, ControllerServer, etc)
2) `test-common:api` new sub-module that just includes interfaces including our junit extension

Reviewers: David Arthur <mumrah@gmail.com>
2024-10-03 10:28:37 +08:00
Omnia Ibrahim 1854d4b8a1
KAFKA-14572: Migrate EmbeddedKafkaCluster used by Streams integration tests from EmbeddedZookeeper to KRaft (#17016)
Migrate the EmbeddedKafkaCluster from the EmbeddedZookeeper to KRaft

Reviewers Bill Bejeck <bbejeck@apache.org>
2024-09-27 15:49:12 -04:00
Omnia Ibrahim f59d829381
KAFKA-15853 Move TransactionLogConfig and TransactionStateManagerConfig getters out of KafkaConfig (#16665)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-03 18:24:12 +08:00
Mickael Maison 1841c07d4a
KAFKA-17449 Move Quota classes to server-common module (#17060)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-31 12:41:34 +08:00
TengYao Chi d67c18b4ae
KAFKA-17331 Set correct version for EarliestLocalSpec and LatestTieredSpec (#16876)
Add the version check to client side when building ListOffsetRequest for the specific timestamp:
1) the version must be >=8 if timestamp=-4L (EARLIEST_LOCAL_TIMESTAMP)
2) the version must be >=9 if timestamp=-5L (LATEST_TIERED_TIMESTAMP)

Reviewers: PoAn Yang <payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2024-08-25 17:39:28 +08:00
Steven Xu e24354a21d
KAFKA-15203: Use Classgraph since org.reflections is no longer under maintenance (#16604)
Reviewers: Liam Miller-Cushon <cushon@google.com>, Greg Harris <greg.harris@aiven.io>
2024-08-19 10:46:24 -07:00
Colin Patrick McCabe 6a44fb154d
KAFKA-16523; kafka-metadata-quorum: support add-controller and remove-controller (#16774)
This PR adds support for add-controller and remove-controller in the kafka-metadata-quorum.sh
command-line tool. It also fixes some minor server-side bugs that blocked the tool from working.

In kafka-metadata-quorum.sh, the implementation of remove-controller is fairly straightforward. It
just takes some command-line flags and uses them to invoke AdminClient. The add-controller
implementation is a bit more complex because we have to look at the new controller's configuration
file. The parsing logic for the advertised.listeners and listeners server configurations that we
need was previously implemented in the :core module. However, the gradle module where
kafka-metadata-quorum.sh lives, :tools, cannot depend on :core. Therefore, I moved listener parsing
into SocketServerConfigs.listenerListToEndPoints. This will be a small step forward in our efforts
to move Kafka configuration out of :core.

I also made some minor changes in kafka-metadata-quorum.sh and Kafka-storage-tool.sh to handle
--help without displaying a backtrace on the screen, and give slightly better error messages on
stderr. Also, in DynamicVoter.toString, we now enclose the host in brackets if it contains a colon
(as IPV6 addresses can).

This PR fixes our handling of clusterId in addRaftVoter and removeRaftVoter, in two ways. Firstly,
it marks clusterId as nullable in the AddRaftVoterRequest.json and RemoveRaftVoterRequest.json
schemas, as it was always intended to be. Secondly, it allows AdminClient to optionally send
clusterId, by using AddRaftVoterOptions and RemoveRaftVoterOptions. We now also remember to
properly set timeoutMs in AddRaftVoterRequest. This PR adds unit tests for
KafkaAdminClient#addRaftVoter and KafkaAdminClient#removeRaftVoter, to make sure they are sending
the right things.

Finally, I fixed some minor server-side bugs that were blocking the handling of these RPCs.
Firstly, ApiKeys.ADD_RAFT_VOTER and ApiKeys.REMOVE_RAFT_VOTER are now marked as forwardable so that
forwarding from the broker to the active controller works correctly. Secondly,
org.apache.kafka.raft.KafkaNetworkChannel has now been updated to enable API_VERSIONS_REQUEST and
API_VERSIONS_RESPONSE.

Co-authored-by: Murali Basani muralidhar.basani@aiven.io
Reviewers: José Armando García Sancio <jsancio@apache.org>, Alyssa Huang <ahuang@confluent.io>
2024-08-08 15:54:12 -07:00
Mickael Maison 7c5d339d07
KAFKA-17227: Refactor compression code to only load codecs when used (#16782)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Josep Prat <josep.prat@aiven.io>
2024-08-06 11:01:21 +02:00
Kuan-Po Tseng 84add30ea5
KAFKA-16154: Broker returns offset for LATEST_TIERED_TIMESTAMP (#16783)
This pr support EarliestLocalSpec LatestTierSpec in GetOffsetShell, and add integration tests.

Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, PoAn Yang <payang@apache.org>
2024-08-05 10:41:14 +08:00
Ken Huang fa682623b9
KAFKA-16666 Migrate OffsetMessageFormatter to tools module (#16689)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-07-31 15:18:14 +08:00
Chris Egerton 61f61d6240
KAFKA-14569: Migrate Connect's integration test EmbeddedKafkaCluster from ZK to KRaft mode (#16599)
Reviewers: Omnia Ibrahim <o.g.h.ibrahim@gmail.com>, Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-07-29 10:43:55 -04:00
Kuan-Po Tseng 437b86cd20
KAFKA-17176 Move FileLock to server-common module (#16671)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-07-25 14:24:27 +08:00
Ken Huang ee68f3215f
KAFKA-16666 Migrate `TransactionLogMessageFormatter` to tools module (#16019)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-07-25 00:42:25 +08:00
PoAn Yang defcbb51ee
KAFKA-17082 replace kafka.utils.LogCaptureAppender with org.apache.kafka.common.utils.LogCaptureAppender (#16601)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-07-21 18:22:05 +08:00
PoAn Yang 3d5d1504f7
KAFKA-16878 Remove powermock and easymock from code base (#16236)
Reviewers: TaiJuWu <tjwu1217@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-06-09 00:17:43 +08:00
José Armando García Sancio 459da4795a
KAFKA-16525; Dynamic KRaft network manager and channel (#15986)
Allow KRaft replicas to send requests to any node (Node) not just the nodes configured in the
controller.quorum.voters property. This flexibility is needed so KRaft can implement the
controller.quorum.voters configuration, send request to the dynamically changing set of voters and
send request to the leader endpoint (Node) discovered through the KRaft RPCs (specially
BeginQuorumEpoch request and Fetch response).

This was achieved by changing the RequestManager API to accept Node instead of just the replica ID.
Internally, the request manager tracks connection state using the Node.idString method to match the
connection management used by NetworkClient.

The API for RequestManager is also changed so that the ConnectState class is not exposed in the
API. This allows the request manager to reclaim heap memory for any connection that is ready.

The NetworkChannel was updated to receive the endpoint information (Node) through the outbound raft
request (RaftRequent.Outbound). This makes the network channel more flexible as it doesn't need to
be configured with the list of all possible endpoints. RaftRequest.Outbound and
RaftResponse.Inbound were updated to include the remote node instead of just the remote id.

The follower state tracked by KRaft replicas was updated to include both the leader id and the
leader's endpoint (Node). In this comment the node value is computed from the set of voters. In
future commit this will be updated so that it is sent through KRaft RPCs. For example
BeginQuorumEpoch request and Fetch response.

Support for configuring controller.quorum.bootstrap.servers was added. This includes changes to
KafkaConfig, QuorumConfig, etc. All of the tests using QuorumTestHarness were changed to use the
controller.quorum.bootstrap.servers instead of the controller.quorum.voters for the broker
configuration. Finally, the node id for the bootstrap server will be decreasing negative numbers
starting with -2.

Reviewers: Jason Gustafson <jason@confluent.io>, Luke Chen <showuon@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
2024-06-03 14:24:48 -07:00
Frederik Rouleau 4eb60b5104
KAFKA-16507 Add KeyDeserializationException and ValueDeserializationException with record content (#15691)
Implements KIP-1036.

Add raw ConsumerRecord data to RecordDeserialisationException to make DLQ implementation easier.

Reviewers: Kirk True <ktrue@confluent.io>, Andrew Schofield <aschofield@confluent.io>, Matthias J. Sax <matthias@confluent.io>
2024-05-28 14:56:47 -07:00
Mickael Maison affe8da54c
KAFKA-7632: Support Compression Levels (KIP-390) (#15516)
Reviewers: Jun Rao <jun@confluent.io>,  Luke Chen <showuon@gmail.com>
Co-authored-by: Lee Dongjin <dongjin@apache.org>
2024-05-21 17:58:49 +02:00
David Jacot c1e7aa77de
MINOR: Move `group-coordinator` import control to its own file (#15934)
Follow the pattern used by the other modules.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-14 00:05:32 -07:00