Declare the delayed share fetch purgatory inside ReplicaManager along with the existing purgatories.
Check the share fetch purgatory when a replica becomes the follower or a replica is deleted from a broker through ReplicaManager.
Perform a checkAndComplete for share fetch when HWM is updated.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>, Jun Rao <junrao@gmail.com>
As of now, there isn't any way that resources can be cleaned in Kafka when client disconnects the connections. Generally time bound clean up happens which keep the resources occupied.
This PR provides capability to attach listeners in Socket Server which can react to client connection disconnections.
Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
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>
This change includes:
1. Dependency checking when updating the feature (all request versions)
2. Returning top level error and no feature level errors if any feature failed to update and using this error for all the features in the response. (all request versions)
3. Returning only top level none error for v2 and beyond
Reviewers: Jun Rao <jun@confluent.io>
Implement server side changes for epoch bump but keep EndTxn as an unstable API until the client side changes are implemented. EndTxnResponse will return the producer ID and epoch for the transaction. Introduces new tagged fields to the TransactionLogValue to persist the clientTransactionVersion, previousProducerId, and nextProducerId to the log so that the state can be reloaded. See KIP-890 for more details.
Small updates to naming of lastProducerId -> PreviousProducerId. Also cleans up the many TransactionMetadata constructors.
Reviewers: Artem Livshits <alivshits@confluent.io>, David Jacot <djacot@confluent.io>
This is part-3 of the KIP-1075.
Added a `timeoutMs` field to the ListOffsets request. This timeout is applicable only for the topic/partitions that are enabled with remote storage.
When the timeout is defined in the request, then we use it to define the delay timeout for `DelayedRemoteListOffsets` request. When the timeout is not defined (requests from older client), then we take the dynamic `remote.list.offsets.request.timeout.ms` server config as the timeout.
Consumer and Admin client behavior are different. Consumer retries the LIST_OFFSETS request in-case of an error but not the AdminClient. And, consumer timeouts the request, if the response exceeds `request.timeout.ms`, whereas, AdminClient timeouts the request when it exceeds the `default.api.timeout.ms`.
To retain the same behavior, we are passing the `requestTimeoutMs` as timeout from the consumer and defaultApiTimeout / overwritten ListOffsetsOption timeout from the admin.
Reviewers: Satish Duggana <satishd@apache.org>, Luke Chen <showuon@gmail.com>
This patch re-introduces the `group.version` feature flag and gates the new consumer rebalance protocol with it. The `group.version` feature flag is attached to the metadata version `4.0-IV0` and it is marked as production ready. This allows system tests to pick it up directly by default without requiring to set `unstable.feature.versions.enable` in all of them. This is fine because we don't plan to do any incompatible changes before 4.0.
Reviewers: Justine Olshan <jolshan@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
Introduce ShareCoordinator interface and related classes.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>, David Arthur <mumrah@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
- due to the server config UNSTABLE_API_VERSIONS_ENABLE_CONFIG is true, so we can't test the scenario of ListOffsetsRequest is unstable version. We want to test this case in this PR
- get the MV from metadataCache.metadataVersion() instead of config.interBrokerProtocolVersion since MV can be set dynamically.
Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
After some discussion on: https://github.com/apache/kafka/pull/16841#discussion_r1714322320
We decided it is best for test version to always map to MetadataVersion.latestTesting. We should always have one unstable MV (either because there is a feature being worked on OR when we mark the latest version stable we create a new one).
Reviewers: Jun Rao <junrao@apache.org>
- Mark 3.9-IV0 as stable. Metadata version 3.9-IV0 should return Fetch version 17.
- Move ELR to 4.0-IV0. Remove 3.9-IV1 since it's no longer needed.
- Create a new 4.0-IV1 MV for KIP-848.
Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Justine Olshan <jolshan@confluent.io>
This patch add resources to store and handle consumer group's config.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Manikumar Reddy <manikumar.reddy@gmail.com>, David Jacot <djacot@confluent.io>
In MetadataVersion 3.7-IV2 and above, the broker's AssignmentsManager sends an RPC to the
controller informing it about which directory we have chosen to place each new replica on.
Unfortunately, the code does not check to see if the topic still exists in the MetadataImage before
sending the RPC. It will also retry infinitely. Therefore, after a topic is created and deleted in
rapid succession, we can get stuck including the now-defunct replica in our subsequent
AssignReplicasToDirsRequests forever.
In order to prevent this problem, the AssignmentsManager should check if a topic still exists (and
is still present on the broker in question) before sending the RPC. In order to prevent log spam,
we should not log any error messages until several minutes have gone past without success.
Finally, rather than creating a new EventQueue event for each assignment request, we should simply
modify a shared data structure and schedule a deferred event to send the accumulated RPCs. This
will improve efficiency.
Reviewers: Igor Soarez <i@soarez.me>, Ron Dagostino <rndgstn@gmail.com>
* KAFKA-15875: Stops leak Snapshot in public methods
The Snapshot class is package protected but it's returned in
several public methods in SnapshotRegistry.
To prevent this accidental leakage, these methods are made
package protected as well. For getOrCreateSnapshot a new
method called IdempotentCreateSnapshot is created that returns void.
* Make builer package protected, replace <br> with <p>
Reviewers: Greg Harris <greg.harris@aiven.io>
The method overrides of metricName in KafkaMetricsGroup are no longer required since there's a new constructor that implement this with the same behavior.
Reviewers: PoAn Yang <payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
Add support for handling the update voter RPC. The update voter RPC is used to automatically update
the voters supported kraft versions and available endpoints as the operator upgrades and
reconfigures the KRaft controllers.
The add voter RPC is handled as follow:
1. Check that the leader has fenced the previous leader(s) by checking that the HWM is known;
otherwise, return the REQUEST_TIMED_OUT error.
2. Check that the cluster supports kraft.version 1; otherwise, return the UNSUPPORTED_VERSION error.
3. Check that there are no uncommitted voter changes, otherwise return the REQUEST_TIMED_OUT error.
4. Check that the updated voter still supports the currently finalized kraft.version; otherwise
return the INVALID_REQUEST error.
5. Check that the updated voter is still listening on the default listener.
6. Append the updated VotersRecord to the log. The KRaft internal listener will read this
uncommitted record from the log and update the voter in the set of voters.
7. Wait for the VotersRecord to commit using the majority of the voters. Return a REQUEST_TIMED_OUT
error if it doesn't commit in time.
8. Send the UpdateVoter successful response to the voter.
This change also implements the ability for the leader to update its own entry in the voter
set when it becomes leader for an epoch. This is done by updating the voter set and writing a
control batch as the first batch in a new leader epoch.
Finally, fix a bug in KafkaAdminClient's handling of removeRaftVoterResponse where we tried to cast
the response to the wrong type.
Reviewers: Alyssa Huang <ahuang@confluent.io>, Colin P. McCabe <cmccabe@apache.org>
As part of KIP-853, storage-tool.sh now has two new flags: --standalone, and --initial-voters. This PR implements these two flags in storage-tool.sh.
There are currently two valid ways to format a cluster:
The pre-KIP-853 way, where you use a statically configured controller quorum. In this case, neither --standalone nor --initial-voters may be specified, and kraft.version must be set to 0.
The KIP-853 way, where one of --standalone and --initial-voters must be specified with the initial value of the dynamic controller quorum. In this case, kraft.version must be set to 1.
This PR moves the formatting logic out of StorageTool.scala and into Formatter.java. The tool file was never intended to get so huge, or to implement complex logic like generating metadata records. Those things should be done by code in the metadata or raft gradle modules. This is also useful for junit tests, which often need to do formatting. (The 'info' and 'random-uuid' commands remain in StorageTool.scala, for now.)
Reviewers: José Armando García Sancio <jsancio@apache.org>
As discussed in #16657 (comment) , we should make logger as static to avoid creating multiple logger instances.
I use the regex private.*Logger.*LoggerFactory to search and check all the results if certain logs need to be static.
There are some exceptions that loggers don't need to be static:
1) The logger in the inner class. Since java8 doesn't support static field in the inner class.
https://github.com/apache/kafka/blob/trunk/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java#L3676
2) Custom loggers for each instance (non-static + non-final). In this case, multiple logger instances is actually really needed.
https://github.com/apache/kafka/blob/trunk/storage/src/test/java/org/apache/kafka/server/log/remote/storage/LocalTieredStorage.java#L166
3) The logger is initialized in constructor by LogContext. Many non-static but with final modifier loggers are in this category, that's why I use .*LoggerFactory to only check the loggers that are assigned initial value when declaration.
4) protected final Logger log = Logger.getLogger(getClass())
This is for subclass can do logging with subclass name instead of superclass name.
But in this case, if the log access modifier is private, the purpose cannot be achieved since subclass cannot access the log defined in superclass. So if access modifier is private, we can replace getClass() with <className>.class
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This change includes adding transaction.version (part of KIP-1022)
New transaction version 1 is introduced to support writing flexible fields in transaction state log messages.
Transaction version 2 is created in anticipation for further KIP-890 changes.
Neither are made production ready. Tests for the new transaction version and new MV are created.
Also include change to not report a feature as supported if the range is 0-0.
Reviewers: Jun Rao <junrao@apache.org>, David Jacot <djacot@confluent.io>, Artem Livshits <alivshits@confluent.io>, Colin P. McCabe <cmccabe@apache.org>
Introduce the KRaftVersion enum to describe the current value of kraft.version. Change a bunch of places in the code that were using raw shorts over to using this new enum.
In BrokerServer.scala, fix a bug that could cause null pointer exceptions during shutdown if we tried to shut down before fully coming up.
Do not send finalized features that are finalized as level 0, since it is a no-op.
Reviewers: dengziming <dengziming1993@gmail.com>, José Armando García Sancio <jsancio@apache.org>
LATEST_PRODUCTION version in MetadataVersion.java was updated in
both #16347 and #16400, but it was left unchanged in the system
tests.
Reviewers: Josep Prat <josep.prat@aiven.io>
This patch partially reverts `group.version` in trunk. I kept the `GroupVersion` class but removed it from `Features` so it is not advertised. I also kept all the changes in the test framework. I removed the logic to require `group.version=1` to enable the new consumer rebalance protocol. The new protocol is enabled based on the static configuration.
For the context, I prefer to revert it in trunk now so we don't forget to revert it in the 3.9 release. I will bring it back for the 4.0 release.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Create 3 new metadata versions:
- 3.8-IV0, for the upcoming 3.8 release.
- 3.9-IV0, to add support for KIP-1005.
- 3.9-IV1, as the new release vehicle for KIP-966.
Create ListOffsetRequest v9, which will be used in 3.9-IV0 to support KIP-1005. v9 is currently an unstable API version.
Reviewers: Jun Rao <junrao@gmail.com>, Justine Olshan <jolshan@confluent.io>
Implement request handling for the updated versions of the KRaft RPCs (Fetch, FetchSnapshot, Vote,
BeginQuorumEpoch and EndQuorumEpoch). This doesn't add support for KRaft replicas to send the new
version of the KRaft RPCs. That will be implemented in KAFKA-16529.
All of the RPCs responses were extended to include the leader's endpoint for the listener of the
channel used in the request. EpochState was extended to include the leader's endpoint information
but only the FollowerState and LeaderState know the leader id and its endpoint(s).
For the Fetch request, the replica directory id was added. The leader now tracks the follower's log
end offset using both the replica id and replica directory id.
For the FetchSnapshot request, the replica directory id was added. This is not used by the KRaft
leader and it is there for consistency with Fetch and for help debugging.
For the Vote request, the replica key for both the voter (destination) and the candidate (source)
were added. The voter key is checked for consistency. The candidate key is persisted when the vote
is granted.
For the BeginQuorumEpoch request, all of the leader's endpoints are included. This is needed so
that the voters can return the leader's endpoint for all of the supported listeners.
For the EndQuorumEpoch request, all of the leader's endpoints are included. This is needed so that
the voters can return the leader's endpoint for all of the supported listeners. The successor list
has been extended to include the directory id. Receiving voters can use the entire replica key when
searching their position in the successor list.
Updated the existing test in KafkaRaftClientTest and KafkaRaftClientSnapshotTest to execute using
both the old version and new version of the RPCs.
Reviewers: Luke Chen <showuon@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
The share-partition leader keeps track of the state and delivery attempts for in-flight records. However, delivery attempts tracking follows atleast-once semantics.
The consumer processes the records and acknowledges them upon successful consumption. This successful attempt triggers a transition to the "Acknowledged" state.
The code implements the functionality to acknowledge the offset/batches in the request to in-memory cached data.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Manikumar Reddy <manikumar.reddy@gmail.com>
Define the interfaces and RPCs for share-group persistence. (KIP-932). This PR is just RPCs and interfaces to allow building of the broker components which depend upon them. The implementation will follow in subsequent PRs.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Apoorv Mittal <apoorvmittal10@gmail.com>
This patch is the continuation of https://github.com/apache/kafka/pull/15964. It introduces the records coalescing to the CoordinatorRuntime. It also introduces a new configuration `group.coordinator.append.linger.ms` which allows administrators to chose the linger time or disable it with zero. The new configuration defaults to 10ms.
Reviewers: Jeff Kim <jeff.kim@confluent.io>, Justine Olshan <jolshan@confluent.io>
A patch for KAFKA-15046 got rid of fsync on LeaderEpochFileCache#truncateFromStart/End for performance reason, but it turned out this could cause corrupted leader-epoch checkpoint file on ungraceful OS shutdown, i.e. OS shuts down in the middle when kernel is writing dirty pages back to the device.
To address this problem, this PR makes below changes: (1) Revert LeaderEpochCheckpoint#write to always fsync
(2) truncateFromStart/End now call LeaderEpochCheckpoint#write asynchronously on scheduler thread
(3) UnifiedLog#maybeCreateLeaderEpochCache now loads epoch entries from checkpoint file only when current cache is absent
Reviewers: Jun Rao <junrao@gmail.com>
This patch fixes a typo in MetadataVersion.IBP_4_0_IV0. It should be 0 not O.
Reviewers: Justine Olshan <jolshan@confluent.io>, Jun Rao <junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
This patch introduces the `group.version` feature flag with one version:
1) Version 1 enables the new consumer group rebalance protocol (KIP-848).
Reviewers: Justine Olshan <jolshan@confluent.io>
As per KIP-1022, we will rename the unstable metadata versions enabled config to support all feature versions.
Features is also updated to return latest production and latest testing versions of each feature.
A feature is production ready when the corresponding metadata version (bootstrapMetadataVersion) is production ready.
Adds tests for the feature usage of the unstableFeatureVersionsEnabled config
Reviewers: David Jacot <djacot@confluent.io>, Jun Rao <junrao@gmail.com>
As part of KIP-1022, I have created an interface for all the new features to be used when parsing the command line arguments, doing validations, getting default versions, etc.
I've also added the --feature flag to the storage tool to show how it will be used.
Created a TestFeatureVersion to show an implementation of the interface (besides MetadataVersion which is unique) and added tests using this new test feature.
I will add the unstable config and tests in a followup.
Reviewers: David Mao <dmao@confluent.io>, David Jacot <djacot@confluent.io>, Artem Livshits <alivshits@confluent.io>, Jun Rao <junrao@apache.org>
A broker that is unable to communicate with the controller will shut down
after the configurable log.dir.failure.timeout.ms.
The implementation adds a new event to the Kafka EventQueue. This event
is deferred by the configured timeout and will execute the shutdown
if the heartbeat communication containing the failed log dir is still
pending with the controller.
Reviewers: Igor Soarez <soarez@apple.com>
Write events create and add a TimerTask to schedule the timeout operation. The issue is that we pile up the number of timer tasks which are essentially no-ops if replication was successful. They stay in memory for 15 seconds (default write timeout) and as the rate of write increases, the impact on memory usage increases.
Instead, cancel the corresponding write timeout task when the write event is committed to the log. This also applies to complete transaction events.
Reviewers: David Jacot <djacot@confluent.io>
We observe some thread leaks in CI which point to the executor service
thread. This change tries to shutdown the executor service using the
helper method in `ThreadUtils`.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Igor Soarez <soarez@apple.com>
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>
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>
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>
- 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>
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>
There were different words for metadata.version like metadata version or metadataVersion. Unify format as metadata.version.
Reviewers: Luke Chen <showuon@gmail.com>
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>
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>
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>
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>
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>
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>
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>
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>
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>
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>
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>
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>
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>
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>
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>