Commit Graph

150 Commits

Author SHA1 Message Date
Luke Chen 3dab6159e8 test 2024-05-08 13:06:53 +08:00
Luke Chen 3c25e02524 test 2024-05-08 08:05:34 +08:00
PoAn Yang 4825c89d14
KAFKA-16588 broker shutdown hangs when log.segment.delete.delay.ms is zero (#15773)
Instead of entering pending forever, this PR invoke next schedule after 1ms. However, the side effect is busy-waiting. Hence, This PR also update the docs to remind users about that - the issue about smaller log.segment.delete.delay.ms

Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-05-01 17:11:20 +08:00
Omnia Ibrahim cfe5ab5cf2
KAFKA-15853 Move quota configs into server-common package (#15774)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-04-24 13:05:18 +08:00
Omnia Ibrahim 5e96e5c898
KAFKA-15853 Refactor KafkaConfig to use PasswordEncoderConfigs (#15770)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-22 00:47:57 +08:00
Kuan-Po (Cooper) Tseng ced79ee12f
KAFKA-16552 Create an internal config to control InitialTaskDelayMs in LogManager to speed up tests (#15719)
Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-04-20 20:34:02 +08:00
Omnia Ibrahim ecb2dd4cdc
KAFKA-15853 Move KafkaConfig log properties and docs out of core (#15569)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Nikolay <nizhikov@apache.org>, Federico Valeri <fvaleri@redhat.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-04-20 04:14:23 +08:00
Mickael Maison 2b9729ba77
MINOR: Various cleanups in server and server-common (#15710)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-16 15:20:49 +08:00
Igor Soarez 15c4ade06a
MINOR: Improve logging in AssignmentsManager (#15522)
At the moment it can be a bit difficult to troubleshoot issues related to the AssignmentsManager. Mainly because:

    Topic partitions are logged with topic ID and partition index but without the topic name.
    Directory IDs are logged without the directory path.
    Assignment reasons aren't tracked.

This patch addresses the three issues.

Reviewers: Luke Chen <showuon@gmail.com>
2024-04-12 14:13:40 +08:00
Kuan-Po (Cooper) Tseng 169ed60fe1
KAFKA-16477 Detect thread leaked client-metrics-reaper in tests (#15668)
After profiling the kafka tests, tons of client-metrics-reaper thread not cleanup after BrokerServer shutdown.
The thread client-metrics-reaper comes from ClientMetricsManager#expirationTimer, and BrokerServer#shudown doesn't close ClientMetricsManager which let the thread still runs in background.

Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-04-09 05:07:33 +08:00
Erik van Oosten 8e61f04228
MINOR: Fix usage of none in javadoc (#15674)
- Use `Empty` instead of 'none' when referring to `Optional` values.
- `Headers.lastHeader` returns `null` when no header is found.
- Fix minor spelling mistakes.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-08 08:43:05 +08:00
Greg Harris bf5e04e416
KAFKA-16349: Prevent race conditions in Exit class from stopping test JVM (#15484)
Signed-off-by: Greg Harris <greg.harris@aiven.io>
Reviewers: Chris Egerton <chrise@aiven.io>
2024-03-28 20:07:42 -07:00
Colin Patrick McCabe 8d914b543d
KAFKA-16411: Correctly migrate default client quota entities (#15584)
KAFKA-16222 fixed a bug whereby we didn't undo the name sanitization used on client quota entity names
stored in ZooKeeper. However, it incorrectly claimed to fix the handling of default client quota
entities. It also failed to correctly re-sanitize when syncronizing the data back to ZooKeeper.

This PR fixes ZkConfigMigrationClient to do the sanitization correctly on both the read and write
paths. We do de-sanitization before invoking the visitors, since after all it does not make sense to
do the same de-sanitization step in each and every visitor.

Additionally, this PR fixes a bug causing default entities to be converted incorrectly. For example,
ClientQuotaEntity(user -> null) is stored under the /config/users/<default> znode in ZooKeeper. In
KRaft it appears as a ClientQuotaRecord with EntityData(entityType=users, entityName=null).
Prior to this PR, this was being converted to a ClientQuotaRecord with EntityData(entityType=users,
entityName=""). That represents a quota on the user whose name is the empty string (yes, we allow
users to name themselves with the empty string, sadly.)

The confusion appears to have arisen because for TOPIC and BROKER configurations, the default
ConfigResource is indeed the one named with the empty (not null) string. For example, the default
topic configuration resource is ConfigResource(name="", type=TOPIC).  However, things are different
for client quotas. Default client quota entities in KRaft (and also in AdminClient) are represented
by maps with null values. For example, the default User entity is represented by Map("user" ->
null).  In retrospect, using a map with null values was a poor choice; a Map<String,
Optional<String>> would have made more sense. However, this is the way the API currently is and we
have to convert correctly.

There was an additional level of confusion present in KAFKA-16222 where someone thought that using
the ZooKeeper placeholder string "<default>" in the AdminClient API would yield a default client
quota entity. Thise seems to have been suggested by the ConfigEntityName class that was created
recently. In fact, <default> is not part of any public API in Kafka. Accordingly, this PR also
renames ConfigEntityName.DEFAULT to ZooKeeperInternals.DEFAULT_STRING, to make it clear that the
string <default> is just a detail of the ZooKeeper implementation.  It is not used in the Kafka API
to indicate defaults. Hopefully this will avoid confusion in the future.

Finally, the PR also creates KRaftClusterTest.testDefaultClientQuotas to get extra test coverage of
setting default client quotas.

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Igor Soarez <soarez@apple.com>
2024-03-26 16:49:38 -07:00
PoAn Yang 6f8d4fe26b
KAFKA-15949: Unify metadata.version format in log and error message (#15505)
There were different words for metadata.version like metadata version or metadataVersion. Unify format as metadata.version.

Reviewers: Luke Chen <showuon@gmail.com>
2024-03-26 20:09:29 +08:00
Igor Soarez f8ce7feebc
KAFKA-15950: Serialize heartbeat requests (#14903)
In between HeartbeatRequest being sent and the response being handled,
i.e. while a HeartbeatRequest is in flight, an extra request may be
immediately scheduled if propagateDirectoryFailure, setReadyToUnfence,
or beginControlledShutdown is called.

To prevent the extra request, we can avoid the extra requests by checking
whether a request is in flight, and delay the scheduling if necessary.

Some of the tests in BrokerLifecycleManagerTest are also improved to
remove race conditions and reduce flakiness.

Reviewers: Colin McCabe <colin@cmccabe.xyz>, Ron Dagostino <rdagostino@confluent.io>, Jun Rao <junrao@gmail.com>
2024-03-25 10:31:19 -07:00
Kuan-Po (Cooper) Tseng bf9a27fefd
KAFKA-16388 add production-ready test of 3.3 - 3.6 release to MetadataVersionTest.testFromVersionString (#15563)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-03-24 13:09:21 +08:00
Kuan-Po (Cooper) Tseng 12a1d85362
KAFKA-12187 replace assertTrue(obj instanceof X) with assertInstanceOf (#15512)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-03-20 10:36:25 +08:00
Chris Holland e878654e95
MINOR: Cleanup BoundedList to Make Constructors More Safe (#15507)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-03-15 21:18:24 +08:00
Kamal Chandraprakash e4c53d093e
KAFKA-15206: Fix the flaky RemoteIndexCacheTest.testClose test (#15523)
It is possible that due to resource constraint, ShutdownableThread#run might be called later than the ShutdownableThread#close method.

Reviewers: Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>
2024-03-15 10:33:40 +08:00
David Jacot f5c4d522fd
MINOR: Add read/write all operation (#15462)
There are a few cases in the group coordinator service where we want to read from or write to each of the known coordinators (each of __consumer_offsets partitions). The current implementation needs to get the list of the known coordinators then schedules the operation and finally aggregate the results. This patch is an attempt to streamline this by adding multi read/write to the runtime.

Reviewers: Omnia Ibrahim <o.g.h.ibrahim@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-03-07 07:51:04 -08:00
Nikolay eea369af94
KAFKA-14588 Log cleaner configuration move to CleanerConfig (#15387)
In order to move ConfigCommand to tools we must move all it's dependencies which includes KafkaConfig and other core classes to java. This PR moves log cleaner configuration to CleanerConfig class of storage module.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-03-05 18:11:56 +08:00
David Jacot 0472db2cd3
MINOR: Uniformize error handling/transformation in GroupCoordinatorService (#15196)
This patch uniformizes the error handling in the GroupCoordinatorService with the aim to reuse the same error translation for all operations. It also ensures that exceptions are unwrapped if needed.

Reviewers: Dongnuo Lyu <dlyu@confluent.io>, Jeff Kim <jeff.kim@confluent.io>, Justine Olshan <jolshan@confluent.io>
2024-01-30 23:23:58 -08:00
Mickael Maison 3e9ef70853
KAFKA-15853: Move PasswordEncoder to server-common (#15246)
Reviewers: Luke Chen <showuon@gmail.com>, Omnia Ibrahim <o.g.h.ibrahim@gmail.com>
2024-01-30 19:08:50 +01:00
Gaurav Narula 4c6f975ab3 KAFKA-16162: resend broker registration on metadata update to IBP 3.7-IV2
We update metadata update handler to resend broker registration when
metadata has been updated to >= 3.7IV2 so that the controller becomes
aware of the log directories in the broker.

We also update DirectoryId::isOnline to return true on an empty list of
log directories while the controller awaits broker registration.

Co-authored-by: Proven Provenzano <pprovenzano@confluent.io>

Reviewers: Omnia G H Ibrahim <o.g.h.ibrahim@gmail.com>, Luke Chen <showuon@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
2024-01-30 10:00:07 -08:00
Apoorv Mittal 208f9e7765
KAFKA-15813: Evict client instances from cache (KIP-714) (#15234)
KIP-714 requires client instance cache in broker which should also have a time-based eviction policy where client instances which are not actively sending metrics should be evicted. KIP mentions This client instance specific state is maintained in broker memory up to MAX(60*1000, PushIntervalMs * 3) milliseconds.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Jun Rao <junrao@gmail.com>
2024-01-23 15:06:02 -08:00
David Arthur 7bf7fd99a5
KAFKA-16078: Be more consistent about getting the latest MetadataVersion
This PR creates MetadataVersion.latestTesting to represent the highest metadata version (which may be unstable) and MetadataVersion.latestProduction to represent the latest version that should be used in production. It fixes a few cases where the broker was advertising that it supported the testing versions even when unstable metadata versions had not been configured.

Reviewers: Colin P. McCabe <cmccabe@apache.org>, Ismael Juma <ismael@juma.me.uk>
2024-01-17 14:59:22 -08:00
Nikolay da2aa68269
KAFKA-14588: Move ConfigEntityName to server-common (#14868)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
2024-01-08 12:41:43 +01:00
Nikolay 45bd19f2ef
KAFKA-14588: Move ConfigType to server-common (#14867)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2023-12-22 18:35:27 +01:00
Proven Provenzano b0e99b5593
KAFKA-15922: Bump MetadataVersion to support JBOD with KRaft (#14984)
Moves ELR from MetadataVersion IBP_3_7_IV3 into the new IBP_3_8_IV0 because the ELR feature was not completed before 3.7 reached feature freeze.  Leaves IBP_3_7_IV3 empty -- it is a no-op and is not reused for anything.  Adds the new MetadataVersion IBP_3_7_IV4 for the FETCH request changes from KIP-951, which were mistakenly never associated with a MetadataVersion.  Updates the LATEST_PRODUCTION MetadataVersion to IBP_3_7_IV4 to declare both KRaft JBOD and the KIP-951 changes ready for production use.

Reviewers: Omnia G H Ibrahim <o.g.h.ibrahim@gmail.com>, Ron Dagostino <rdagostino@confluent.io>, Ismael Juma <ismael@juma.me.uk>, José Armando García Sancio <jsancio@apache.org>, Justine Olshan <jolshan@confluent.io>
2023-12-14 10:08:54 -05:00
Bruno Cadonna 87e3cbe4da
MINOR: Add junit properties to display parameterized test names (#14983)
In many parameterized tests, the display name is broken. Example - testMetadataFetch appears as [1] true, [2] false link
This is because the constant in @ParameterizedTest

String DEFAULT_DISPLAY_NAME = "[{index}] {argumentsWithNames}";

This PR adds a new junit-platform.properties which overrides to add a {displayName} which shows the the display name of the method

For existing tests which override the name, should work as is. The precedence rules are explained

    name attribute in @ParameterizedTest, if present
    value of the junit.jupiter.params.displayname.default configuration parameter, if present
    DEFAULT_DISPLAY_NAME constant defined in @ParameterizedTest

Source: https://junit.org/junit5/docs/current/user-guide/#writing-tests-parameterized-tests-display-names

Sample test run output
Before: [1] true link
After: testMetadataExpiry(boolean).false link

This commit is an extension of bdf6d46b41 which needed to reverted due to introduces test failures.

Reviewers: David Jacot <djacot@confluent.io>, Lucas Brutschy <lbrutschy@confluent.io>
2023-12-13 09:42:18 +01:00
David Jacot b96ded9859
Revert "MINOR: Add junit properties to display parameterized test names (#14687)" (#14961)
This reverts commit bdf6d46b41. We found out that this commit introduced flakiness in Streams' tests. We will revise it.

Reviewers: Bruno Cadonna <cadonna@apache.org>
2023-12-07 23:20:03 -08:00
Omnia Ibrahim ec92410e59
KAFKA-15363: Broker log directory failure changes (#14790)
Part of JBOD KIP-858, https://cwiki.apache.org/confluence/display/KAFKA/KIP-858%3A+Handle+JBOD+broker+disk+failure+in+KRaft

Reviewers: Igor Soarez <i@soarez.me>, Colin P. McCabe <cmccabe@apache.org>, Ron Dagostino <rdagostino@confluent.io>
2023-12-07 20:44:56 -05:00
Igor Soarez c515bf51f8 KAFKA-15426: Process and persist directory assignments
Handle AssignReplicasToDirs requests, persist metadata changes
with new directory assignments and possible leader elections.

Reviewers: Proven Provenzano <pprovenzano@confluent.io>, Ron Dagostino <rndgstn@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
2023-12-07 11:44:45 -08:00
Alok Thatikunta bdf6d46b41
MINOR: Add junit properties to display parameterized test names (#14687)
In many parameterized tests, the display name is broken. Example - `testMetadataFetch` appears as `[1] true`, `[2] false`  [link](https://ci-builds.apache.org/job/Kafka/job/kafka-pr/job/PR-14607/9/testReport/junit/org.apache.kafka.clients.producer/KafkaProducerTest/) 
This is because the constant in `@ParameterizedTest`
```java
String DEFAULT_DISPLAY_NAME = "[{index}] {argumentsWithNames}";
```

This PR adds a new `junit-platform.properties` which overrides to add a `{displayName}` which shows the `the display name of the method`

For existing tests which override the name, should work as is. The precedence rules are explained

> 1. `name` attribute in `@ParameterizedTest`, if present
> 2. value of the `junit.jupiter.params.displayname.default` configuration parameter, if present
> 3. `DEFAULT_DISPLAY_NAME` constant defined in `@ParameterizedTest`

Source: https://junit.org/junit5/docs/current/user-guide/#writing-tests-parameterized-tests-display-names

Sample test run output 
Before: `[1] true` [link](https://ci-builds.apache.org/job/Kafka/job/kafka-pr/job/PR-14607/9/testReport/junit/org.apache.kafka.clients.producer/KafkaProducerTest/)
After: `testMetadataExpiry(boolean).false` [link](https://ci-builds.apache.org/job/Kafka/job/kafka-pr/job/PR-14687/1/testReport/junit/org.apache.kafka.clients.producer/KafkaProducerTest/)

Reviewers: Divij Vaidya <diviv@amazon.com>, Bruno Cadonna <cadonna@apache.org>, David Jacot <djacot@confluent.io>
2023-12-06 08:42:45 -08:00
Igor Soarez 6b87c85291 KAFKA-15886: Always specify directories for new partition registrations
When creating partition registrations directories must always be defined.

If creating a partition from a PartitionRecord or PartitionChangeRecord from an older version that
does not support directory assignments, then DirectoryId.MIGRATING is assumed.

If creating a new partition, or triggering a change in assignment, DirectoryId.UNASSIGNED should be
specified, unless the target broker has a single online directory registered, in which case the
replica should be assigned directly to that single directory.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2023-11-30 14:10:47 -08:00
Colin Patrick McCabe a94bc8d6d5
KAFKA-15922: Add a MetadataVersion for JBOD (#14860)
Assign MetadataVersion.IBP_3_7_IV2 to JBOD.

Move KIP-966 support to MetadataVersion.IBP_3_7_IV3.

Create MetadataVersion.LATEST_PRODUCTION as the latest metadata version that can be used when formatting a
new cluster, or upgrading a cluster using kafka-features.sh. This will allow us to clearly distinguish between stable
and unstable metadata versions for the first time.

Reviewers: Igor Soarez <soarez@apple.com>, Ron Dagostino <rndgstn@gmail.com>, Calvin Liu <caliu@confluent.io>, Proven Provenzano <pprovenzano@confluent.io>
2023-11-30 10:35:13 -08:00
Colin Patrick McCabe bd18551b32
MINOR: DirectoryId.MIGRATING should be all zeros (#14858)
DirectoryId.MIGRATING should be all zeros. All zeros is the default Uuid value in KPRC, and
MIGRATING is the default directory ID value.

Reviewers: Ron Dagostino <rdagostino@confluent.io>
2023-11-29 13:12:33 -08:00
Okada Haruki d71d0639d9
KAFKA-15046: Get rid of unnecessary fsyncs inside UnifiedLog.lock to stabilize performance (#14242)
While any blocking operation under holding the UnifiedLog.lock could lead to serious performance (even availability) issues, currently there are several paths that calls fsync(2) inside the lock
In the meantime the lock is held, all subsequent produces against the partition may block
This easily causes all request-handlers to be busy on bad disk performance
Even worse, when a disk experiences tens of seconds of glitch (it's not rare in spinning drives), it makes the broker to unable to process any requests with unfenced from the cluster (i.e. "zombie" like status)
This PR gets rid of 4 cases of essentially-unnecessary fsync(2) calls performed under the lock:
(1) ProducerStateManager.takeSnapshot at UnifiedLog.roll
I moved fsync(2) call to the scheduler thread as part of existing "flush-log" job (before incrementing recovery point)
Since it's still ensured that the snapshot is flushed before incrementing recovery point, this change shouldn't cause any problem
(2) ProducerStateManager.removeAndMarkSnapshotForDeletion as part of log segment deletion
This method calls Utils.atomicMoveWithFallback with needFlushParentDir = true internally, which calls fsync.
I changed it to call Utils.atomicMoveWithFallback with needFlushParentDir = false (which is consistent behavior with index files deletion. index files deletion also doesn't flush parent dir)
This change shouldn't cause problems neither.
(3) LeaderEpochFileCache.truncateFromStart when incrementing log-start-offset
This path is called from deleteRecords on request-handler threads.
Here, we don't need fsync(2) either actually.
On unclean shutdown, few leader epochs might be remained in the file but it will be handled by LogLoader on start-up so not a problem
(4) LeaderEpochFileCache.truncateFromEnd as part of log truncation
Likewise, we don't need fsync(2) here, since any epochs which are untruncated on unclean shutdown will be handled on log loading procedure

Reviewers: Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>, Justine Olshan <jolshan@confluent.io>, Jun Rao <junrao@gmail.com>
2023-11-29 09:43:44 -08:00
Igor Soarez a03a71d7b5 KAFKA-15357: Aggregate and propagate assignments
A new AssignmentsManager accumulates, batches, and sends KIP-858
assignment events to the Controller. Assignments are sent via
AssignReplicasToDirs requests.

Move QuorumTestHarness.formatDirectories into TestUtils so it can be
used in other test contexts.

Fix a bug in ControllerRegistration.java where the wrong version of the
record was being generated in ControllerRegistration.toRecord.

Reviewers: Colin P. McCabe <cmccabe@apache.org>, Proven Provenzano <pprovenzano@confluent.io>, Omnia G H Ibrahim <o.g.h.ibrahim@gmail.com>
2023-11-16 16:19:49 -08:00
Colin P. McCabe e3dd60ef3c HOTFIX: fix checkstyle 2023-11-02 11:35:44 -07:00
Colin P. McCabe a672a19e80 MINOR: small optimization for DirectoryId.random
DirectoryId.random doesn't need to instantiate the first 100 IDs to check if an ID is one of them.

Reviewers: Ismael Juma <ismael@juma.me.uk>, Justine Olshan <jolshan@confluent.io>, Proven Provenzano <93720617+pprovenzano@users.noreply.github.com>
2023-11-02 11:29:11 -07:00
Igor Soarez 0390d5b1a2
KAFKA-15355: Message schema changes (#14290)
Reviewers: Christo Lolov <lolovc@amazon.com>, Colin P. McCabe <cmccabe@apache.org>, Proven Provenzano <pprovenzano@confluent.io>, Ron Dagostino <rdagostino@confluent.io>
2023-11-02 09:46:05 -04:00
Crispin Bernier c8f687ac15
KAFKA-15661: KIP-951: protocol changes (#14627)
Separating out the protocol changes from #14444 in an effort to more quickly unblock the client side PR.

This is the protocol changes to populate the fields in KIP-951. On NOT_LEADER_OR_FOLLOWER errors in both FETCH and PRODUCE the new leader ID and epoch are included in the response. The endpoint for the new leader is retrieved from the metadata cache. The new fields are all optional (tagged) and an IBP bump is required.

https://cwiki.apache.org/confluence/display/KAFKA/KIP-951%3A+Leader+discovery+optimisations+for+the+client

Reviewers: Justine Olshan <jolshan@confluent.io>, Mayank Shekhar Narula <mayanks.narula@gmail.com>
2023-10-31 17:16:11 -07:00
Igor Soarez 9dbee599f1
MINOR: Rename log dir UUIDs (#14517)
After a late discussion in the voting thread for KIP-858 we
decided to improve the names for the designated reserved
log directory UUID values.

Reviewers: Christo Lolov <lolovc@amazon.com>, Ismael Juma <ismael@juma.me.uk>,  Ziming Deng <dengziming1993@gmail.com>.
2023-10-30 19:10:57 +08:00
Josep Prat eed5e68880
MINOR: Server-Commons cleanup (#14572)
MINOR: Server-Commons cleanup

Fixes Javadoc and minor issues in the Java files of Server-Commons modules.

Javadoc is now formatted as intended by the author of the doc itself.

Signed-off-by: Josep Prat <josep.prat@aiven.io>

Reviewers: Mickael Maison <mickael.maison@gmail.com>
2023-10-20 21:04:04 +02:00
Calvin Liu af747fbfed
KAFKA-15581: Introduce ELR (#14312)
This patch introduces preliminary changes for Eligible Leader Replicas (KIP-966)

* New MetadataVersion 16 (3.7-IV1)
* New record versions for PartitionRecord and PartitionChangeRecord
* New tagged fields on PartitionRecord and PartitionChangeRecord
* New static config "eligible.leader.replicas.enable" to gate the whole feature

Reviewers: Artem Livshits <alivshits@confluent.io>, David Arthur <mumrah@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
2023-10-19 14:05:15 -04:00
Ismael Juma 4cf86c5d2f
KAFKA-15492: Upgrade and enable spotbugs when building with Java 21 (#14533)
Spotbugs was temporarily disabled as part of KAFKA-15485 to support Kafka build with JDK 21. This PR upgrades the spotbugs version to 4.8.0 which adds support for JDK 21 and enables it's usage on build again.

Reviewers: Divij Vaidya <diviv@amazon.com>
2023-10-12 14:09:10 +02:00
Ritika Reddy bcfc9543d1
MINOR: Move TopicIdPartition class to server-common (#14418)
This patch moves the TopicIdPartition from the metadata module to the server-common module so it can be used by the group-coordinator module as well.

Reviewers: Sagar Rao <sagarmeansocean@gmail.com>, David Jacot <djacot@confluent.io>
2023-09-28 13:55:44 -07:00
Colin Patrick McCabe fcac880fd5
KAFKA-15466: Add KIP-919 support for some admin APIs (#14399)
Add support for --bootstrap-controller in the following command-line tools:
    - kafka-cluster.sh
    - kafka-configs.sh
    - kafka-features.sh
    - kafka-metadata-quorum.sh

To implement this, the following AdminClient APIs now support the new bootstrap.controllers
configuration:
    - Admin.alterConfigs
    - Admin.describeCluster
    - Admin.describeConfigs
    - Admin.describeFeatures
    - Admin.describeMetadataQuorum
    - Admin.incrementalAlterConfigs
    - Admin.updateFeatures

Command-line tool changes:
    - Add CommandLineUtils.initializeBootstrapProperties to handle parsing --bootstrap-controller
      in addition to --bootstrap-server.
    - Add --bootstrap-controller to ConfigCommand.scala, ClusterTool.java, FeatureCommand.java, and
      MetadataQuorumCommand.java.

KafkaAdminClient changes:
    - Add the AdminBootstrapAddresses class to handle extracting bootstrap.servers or
      bootstrap.controllers from the config map for KafkaAdminClient.
    - In AdminMetadataManager, store the new usingBootstrapControllers boolean. Generalize
      authException to encompass the concept of fatal exceptions in general. (For example, the
      fatal exception where we talked to the wrong node type.) Treat
      MismatchedEndpointTypeException and UnsupportedEndpointTypeException as fatal exceptions.
    - Extend NodeProvider to include information about whether bootstrap.controllers is supported.
    - Modify the APIs described above to support bootstrap.controllers.

Server-side changes:
    - Support DescribeConfigsRequest on kcontrollers.
    - Add KRaftMetadataCache to the kcontroller to simplify implemeting describeConfigs (and
      probably more APIs in the future). It's mainly a wrapper around MetadataImage, so there is
      essentially no extra resource consumption.
    - Split RuntimeLoggerManager out of ConfigAdminManager to handle the incrementalAlterConfigs
      support for BROKER_LOGGER. This is now supported on kcontrollers as well as brokers.
    - Fix bug in AuthHelper.computeDescribeClusterResponse that resulted in us always sending back
      BROKER as the endpoint type, even on the kcontroller.

Miscellaneous:
    - Fix a few places in exceptions and log messages where we wrote "broker" instead of "node".
      For example, an exception in NodeApiVersions.java, and a log message in NetworkClient.java.
    - Fix the slf4j log prefix used by KafkaRequestHandler logging so that request handlers on a
      controller don't look like they're on a broker.
    - Make the FinalizedVersionRange constructor public for the sake of a junit test.
    - Add unit and integration tests for the above.

Reviewers: David Arthur <mumrah@gmail.com>, Doguscan Namal <namal.doguscan@gmail.com>
2023-09-26 14:43:42 -07:00
Ismael Juma 98febb989a
KAFKA-15485: Fix "this-escape" compiler warnings introduced by JDK 21 (1/N) (#14427)
This is one of the steps required for kafka to compile with Java 21.

For each case, one of the following fixes were applied:
1. Suppress warning if fixing would potentially result in an incompatible change (for public classes)
2. Add final to one or more methods so that the escape is not possible
3. Replace method calls with direct field access.

In addition, we also fix a couple of compiler warnings related to deprecated references in the `core` module.

See the following for more details regarding the new lint warning:
https://www.oracle.com/java/technologies/javase/21-relnote-issues.html#JDK-8015831

Reviewers: Divij Vaidya <diviv@amazon.com>, Satish Duggana <satishd@apache.org>, Chris Egerton <chrise@aiven.io>
2023-09-24 05:59:29 -07:00