Commit Graph

13475 Commits

Author SHA1 Message Date
Apoorv Mittal 126b25b51d
KAFKA-17288 Removed tracking partition member epoch (KIP-932) (#16828)
Partition epochs are tracked for consumer groups where epoch is the current assigned member epoch. As share groups have partitions shared hence maintaing the partition epochs is not required.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2024-08-12 07:17:46 +08:00
Ken Huang 9a85705b56
KAFKA-17297 stabalize testHandleEndQuorumRequest (#16855)
In old code leaderId and oldLeaderId all use randomReplicaId(), thus it will be flaky when leaderId is equals oldLeaderId. When KafkaRaftClient initialize it will find leaderId and oldLeaderId, so it will tansfer to candidate, and the leaderEpoch will be + 1

Reviewers: TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-08-12 07:08:47 +08:00
PoAn Yang 97cdaaa15f
KAFKA-17312 Fix tests shouldForwardAllDbOptionsCalls and shouldOverwriteAllOptionsMethods in RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapterTest (#16857)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-12 07:02:35 +08:00
Eric Chang 73fe45f43a
KAFKA-17137 Add integration tests for admin client (createPartitions and quota) (#16687)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-12 06:57:18 +08:00
PoAn Yang 49fc14f611
KAFKA-17273 Migrate BootstrapControllersIntegrationTest to use ClusterTestExtensions (#16822)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-11 19:48:13 +08:00
Kuan-Po Tseng 5d66f8d88c
KAFKA-17035 Add debug log to retention cleanupLogs method to help troubleshoot issues (#16790)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-11 01:10:29 +08:00
Alyssa Huang b4d5f163a9
KAFKA-17067; Fix KRaft transition to CandidateState (#16820)
Only voters should be able to transition to Candidate state. This removes VotedState as one of the EpochStates and moves voted information into UnattachedState.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2024-08-10 07:43:16 -04:00
Colin Patrick McCabe e1b2adea07
KAFKA-17190: AssignmentsManager gets stuck retrying on deleted topics (#16672)
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>
2024-08-10 12:31:45 +01:00
PoAn Yang 98cdf97170
KAFKA-17240 Try to complete delyed ops for all purgatories even though one of delayed op throws exception (#16777)
Make DelayedOperations#checkAndCompleteAll has chance to complete delayed ops even though there is a exception caused by one of delayed op.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-10 00:26:49 +08:00
TengYao Chi 6f18f6f335
KAFKA-16328 Remove Deprecated config from StreamsConfig (#16805)
- StreamsConfig#RETRIES_CONFIG was deprecated in AK 2.7 and is no longer in use.
- StreamsConfig#DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS and
- StreamsConfig#DEFAULT_WINDOWED_VALUE_SERDE_INNER_CLASS were deprecated in AK 3.0.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-10 00:16:52 +08:00
Andrew Schofield 0a4a12fbc4
KAFKA-17225: Refactor consumer membership managers (#16751)
The initial drop of ShareMembershipManager contained a lot of code duplicated from MembershipManagerImpl. The plan was always to share as much code as possible between the membership managers for consumer groups and share groups. This issue refactors the membership managers so that almost all of the code is in common.

Reviewers:  Lianet Magrans <lianetmr@gmail.com>,  Chia Chuan Yu <yujuan476@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
2024-08-09 16:18:07 +05:30
PoAn Yang 242114eab9
KAFKA-12830: Remove Deprecated constructor in TimeWindowedDeserializer and TimeWindowedSerde (#16809)
Signed-off-by: PoAn Yang <payang@apache.org>

Reviewers: Josep Prat <josep.prat@aiven.io>
2024-08-09 11:57:44 +02:00
Ken Huang 1ff7cf3d36
KAFKA-12832 remove deprecated methods in RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter (#16813)
Reviewers: Josep Prat <josep.prat@aiven.io>
2024-08-09 11:49:28 +02:00
Dongnuo Lyu ed0e1086a9
KAFKA-17228; Static member using new protocol should always replace the one using the old protocol (#16800)
This patch enables the static member using the old protocol to be replaced in ConsumerGroupHeartbeat even if it hasn't sent a request to leave the group.

This helps in cases where existing static member rejoins and triggers a group upgrade, because the classic static consumer doesn't send leave group request before shutting down.

Reviewers: TengYao Chi <kitingiao@gmail.com>, David Jacot <djacot@confluent.io>
2024-08-09 01:11:34 -07:00
Sean Quah 7a8edffad1
KAFKA-17267; Don't return REQUEST_TIMED_OUT for OFFSET_FETCHes (#16825)
When handling an OFFSET_FETCH request requiring stable offsets, the new
group coordinator may encounter a timeout under some circumstances, such
as a zombie coordinator or a lagging __consumer_offsets replica that has
not yet dropped out of the ISR. Existing and older clients do not expect
the REQUEST_TIMED_OUT error code won't retry, so remap it to
NOT_COORDINATOR to trigger a coordinator lookup and retry.

Reviewers: David Jacot <djacot@confluent.io>
2024-08-09 01:06:38 -07:00
José Armando García Sancio 8ce514a52e
KAFKA-16534; Implemeent update voter sending (#16837)
This change implements the KRaft voter sending UpdateVoter request. The
UpdateVoter RPC is used to update a voter's listeners and supported
kraft versions. The UpdateVoter RPC is sent if the replicated voter set
(VotersRecord in the log) doesn't match the local voter's supported
kraft versions and controller listeners.

To not starve the Fetch request, the UpdateVoter request is sent at most
every 3 fetch timeouts. This is required to make sure that replication
is making progress and eventually the voter set in the replicated log
matches the local voter configuration.

This change also modifies the semantic for UpdateVoter. Now the
UpdateVoter response is sent right after the leader has created the new
voter set. This is required so that updating voter can transition from
sending UpdateVoter request to sending Fetch request. If the leader
waits for the VotersRecord control record to commit before sending the
UpdateVoter response, it may never send the UpdateVoter response. This
can happen if the leader needs that voter's Fetch request to commit the
control record.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2024-08-08 16:16:09 -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
bboyleonp666 e7317b37bf
KAFKA-17269 Fix ConcurrentModificationException caused by NioEchoServer.closeNewChannels (#16817)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-09 03:24:59 +08:00
Josep Prat 4e862c0903
KAFKA-15875: Stops leak Snapshot in public methods (#16807)
* 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>
2024-08-08 20:05:47 +02:00
PoAn Yang 130af38481
KAFKA-17223 Retrying the call after encoutering UnsupportedVersionException will cause ConcurrentModificationException (#16753)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-09 01:07:54 +08:00
Alyssa Huang 3066019efa
KAFKA-16521: Have Raft endpoints printed as name://host:port (#16830)
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2024-08-08 09:22:23 -07:00
ShivsundarR 0cbc5e083a
KAFKA-17217: Batched acknowledge requests per node in ShareConsumeRequestManager (#16727)
In ShareConsumeRequestManager, currently every time we perform a commitSync/commitAsync/acknowledgeOnClose we create one AcknowledgeRequestState for each call. But this can be optimised further as we can batch up the acknowledgements to be sent to the same node before the next poll() is invoked.

This will ensure that between 2 polls, the acknowledgements are accumulated in one request per node and then sent during poll, resulting in lesser RPC calls.

To achieve this, we are storing a pair of acknowledge request states for every node, the first value denotes the requestState for commitAsync() and the second value denotes the requestState for commitSync() and acknowledgeOnClose(). All the acknowledgements to be sent to a particular node are stored in the corresponding acknowledgeRequestState based on whether it was synchronous or asynchronous.

Reviewers:  Andrew Schofield <aschofield@confluent.io>,  Manikumar Reddy <manikumar.reddy@gmail.com>
2024-08-08 20:43:37 +05:30
Dmitry Werner 4763d0fc30
KAFKA-17242: Do not log spurious timeout message for MirrorCheckpointTask sync store startup (#16773)
Reviewers: Chris Egerton <chrise@aiven.io>
2024-08-08 10:03:42 -04:00
TengYao Chi 0dc74c5556
KAFKA-17232: Do not generate task configs in MirrorCheckpointConnector if initial consumer group load times out (#16767)
Reviewers: Hongten <hongtenzone@foxmail.com>, Chris Egerton <chrise@aiven.io>
2024-08-08 09:58:11 -04:00
Dmitry Werner c9d415f361
KAFKA-17275: Move ReplicatedCounter to test scope (#16814)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2024-08-08 15:56:21 +02:00
Luke Chen 164f899605
KAFKA-17236: Handle local log deletion when remote.log.copy.disabled=true (#16765)
Handle local log deletion when remote.log.copy.disabled=true based on the KIP-950.

When tiered storage is disabled or becomes read-only on a topic, the local retention configuration becomes irrelevant, and all data expiration follows the topic-wide retention configuration exclusively.

- added remoteLogEnabledAndRemoteCopyEnabled method to check if this topic enables tiered storage and remote log copy is enabled. We should adopt local.retention.ms/bytes when remote.storage.enable=true,remote.log.copy.disable=false.
- Changed to use retention.bytes/retention.ms when remote copy disabled.
- Added validation to ask users to set local.retention.ms == retention.ms and local.retention.bytes == retention.bytes
- Added tests

Reviewers: Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Satish Duggana <satishd@apache.org>, Christo Lolov <lolovc@amazon.com>
2024-08-08 17:07:40 +05:30
Ken Huang 786d2c9975
KAFKA-17276; replicaDirectoryId for Fetch and FetchSnapshot should be ignorable (#16819)
The replicaDirectoryId field for FetchRequest and FetchSnapshotRequest should be ignorable. This allows data objects with the directory id to be serialized to any version of the requests.

Reviewers: José Armando García Sancio <jsancio@apache.org>, Chia-Ping Tsai <chia7712@apache.org>
2024-08-07 20:53:15 -04:00
Andrew Schofield b9099301fb
KAFKA-16716: Add admin list and describe share groups (#16827)
Adds the methods to the admin client for listing and describing share groups.
There are some unit tests, but integration tests will be in a follow-on PR.

Reviewers:  Manikumar Reddy <manikumar.reddy@gmail.com>
2024-08-08 00:46:25 +05:30
Tim Fox 837684a1b9
MINOR: Correct type in README.md from "boolean" to "bool" (#16706)
The actual type name in the JSON descriptor files is "bool" not "boolean"

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Andrew Schofield <andrew_schofield@live.com>
2024-08-07 17:34:53 +02:00
xijiu c2f53a9760
KAFKA-17268 Fix NPE when closing a non-started acceptor (#16818)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-07 18:50:42 +08:00
Apoorv Mittal 96816a3ed7
KAFKA-16742: Add share group describe in coordinator (KIP-932) (#16797)
Share group describe functionality for KIP-932

Reviewers:  Andrew Schofield <aschofield@confluent.io>,  Manikumar Reddy <manikumar.reddy@gmail.com>
2024-08-07 14:53:01 +05:30
abhi-ksolves 6b5d172dcf
KAFKA-12828: Removed Deprecated methods under KeyQueryMetadata (#16747)
Reviewers: Matthias J. Sax <matthias@confluent.io>
2024-08-06 18:21:23 -07:00
dujian0068 5596f9e1d5
KAFKA-16584: Make log processing summary configurable or debug--update upgrade-guide (#16709)
Updates Kafka Streams upgrade-guide for KIP-1049.

Reviewers: Bill Bejeck <bill@confluent.io>, Matthias J. Sax <matthias@confluent.io>
2024-08-06 12:11:33 -07:00
bboyleonp666 5fac905749
KAFKA-17222 Remove the subclass of KafkaMetricsGroup (#16752)
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>
2024-08-07 01:26:49 +08:00
xijiu 46f1f0268b
KAFKA-17137 Add integration tests for admin client (Transaction and UserScramCredentials related) (#16652)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-07 01:11:55 +08:00
Bill Bejeck 4c9795eddf
MINOR: Flaky RestoreIntegrationTest (#16721)
Increase some timeouts to work against race condition.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2024-08-06 10:05:10 -07:00
Chirag Wadhwa 1db84c1a11
KAFKA-16745: Implemented handleShareFetchRequest RPC including unit tests (#16456)
Implemented handleShareFetch request RPC in KafkaApis.scala. This method is called whenever the client sends a Share Fetch request to the broker. Although Share Fetch request support acknowledgements, since the logic for acknowledging records is not completely implemented in SharePartitionManager.java class, this method currently includes placeholder code for acknowledging, which will be replaced by the actual functionality in the upcoming PRs.

Reviewers:  Apoorv Mittal <apoorvmittal10@gmail.com>, Abhinav Dixit <adixit@confluent.io>, Jun Rao <junrao@gmail.com>
2024-08-06 07:59:04 -07:00
Andrew Schofield 3ddd8d0a0e
KAFKA-17262 Improve kafka-topics usage message (#16795)
There is a lot of historical cruft in the usage message for kafka-topics.sh. For example, the --bootstrap-server option is required nowadays, but the usage message is written in some cases as if it was still optional.

This PR just tightens up the usage message, removing out-of-date text and improving the grammar.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-08-06 19:31:39 +08:00
Igor Soarez 8c066e625c
MINOR: Update docs, remove JBOD from missing features (#16801)
Reviewers: Josep Prat <josep.prat@aiven.io>
2024-08-06 11:18:14 +01: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
Philip Nee c18463ec2d
MINOR: Rethrow caught exception instead of wrapping into LoginException (#16769)
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
2024-08-06 09:48:07 +01:00
Kuan-Po Tseng 5a602b2f86
KAFKA-17235 system test test_performance_service.py failed (#16789)
related to https://issues.apache.org/jira/browse/KAFKA-17235

The root cause of this issue is a change we introduced in KAFKA-16879, where we modified the PushHttpMetricsReporter constructor to use Time.System [1]. However, Time.System doesn't exist in Kafka versions 0.8.2 and 0.9.

In test_performance_services.py, we have system tests for Kafka versions 0.8.2 and 0.9 [2]. These tests always use the tools JAR from the trunk branch, regardless of the Kafka version being tested [3], while the client JAR aligns with the Kafka version specified in the test suite [4]. This discrepancy is what causes the issue to arise.

To resolve this issue, we have a few options:

1) Add Time.System to Kafka 0.8.2 and 0.9: This isn't practical, as we no longer maintain these versions.
2) Modify the PushHttpMetricsReporter constructor to use new SystemTime() instead of Time.System: This would contradict the intent of KAFKA-16879, which aims to make SystemTime a singleton.
3) Implement Time in PushHttpMetricsReporter use the time to get current time
4) Remove system tests for Kafka 0.8.2 and 0.9 from test_performance_services.py

Given that we no longer maintain Kafka 0.8.2 and 0.9, and altering the constructor goes against the design goals of KAFKA-16879, option 4 appears to be the most feasible solution. However, I'm not sure whether it's acceptable to remove these old version tests. Maybe someone else has a better solution

"We'll proceed with option 3 since support for versions 0.8 and 0.9 is still required, meaning we can't remove those Kafka versions from the system tests."

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-06 14:51:28 +08:00
Chris Egerton 8152ee6519
MINOR: Increase test producer max.block.ms in MirrorConnectorsIntegrationTransactionsTest (#16785)
Reviewers: Greg Harris <greg.harris@aiven.io>
2024-08-05 14:37:17 -04:00
Chris Egerton 3f3e376797
MINOR: Fix stage recording in DistributedHerder::addRequest (#16784)
Reviewers: Greg Harris <greg.harris@aiven.io>
2024-08-05 14:36:39 -04:00
José Armando García Sancio e49c8df1f7 KAFKA-16533; Update voter handling
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>
2024-08-05 11:32:21 -07:00
TengYao Chi 736bd3cd09
KAFKA-17201 SelectorTest.testInboundConnectionsCountInConnectionCreationMetric leaks sockets and threads (#16723)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-05 22:47:06 +08:00
Josep Prat a4277c6617
KAFKA-17227: Update zstd-jni lib (#16763)
* KAFKA-17227: Update zstd-jni lib
* Add note in upgrade docs
* Change zstd-jni version in docker native file and add warning in dependencies.gradle file
* Add reference to snappy in upgrade

Reviewers:  Chia-Ping Tsai <chia7712@gmail.com>,  Mickael Maison <mickael.maison@gmail.com>
2024-08-05 09:51:46 +02:00
brenden20 0b78d8459c
MINOR: Remove ConsumerTestBuilder.java (#16691)
The purpose of this PR is to remove ConsumerTestBuilder.java since it is no longer needed. The following PRs have eliminated the use of ConsumerTestBuilder:
#14930
#16140
#16200
#16312

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-05 11:05:53 +08: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
David Arthur b3fd9a5a95
KAFKA-17146 Include note to remove migration znode (#16770)
When reverting the ZK migration, we must also remove the /migration ZNode in order to allow the migration to be re-attempted in the future.

Reviewers: Colin P. McCabe <cmccabe@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2024-08-05 03:13:26 +08:00