(kafka) branch trunk updated: KAFKA-16520: Support KIP-853 in DescribeQuorum (#16106)

2024-06-11 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new aecaf444756 KAFKA-16520: Support KIP-853 in DescribeQuorum (#16106)
aecaf444756 is described below

commit aecaf4447561edd8da9f06e3abdf46f382dc9d89
Author: Nikolay 
AuthorDate: Tue Jun 11 20:01:35 2024 +0300

KAFKA-16520: Support KIP-853 in DescribeQuorum (#16106)

Add support for KIP-953 KRaft Quorum reconfiguration in the DescribeQuorum 
request and response.
Also add support to AdminClient.describeQuorum, so that users will be able 
to find the current set of
quorum nodes, as well as their directories, via these RPCs.

Reviewers: Luke Chen , Colin P. McCabe 
, Andrew Schofield 
---
 .../kafka/clients/admin/KafkaAdminClient.java  |  20 +-
 .../org/apache/kafka/clients/admin/QuorumInfo.java |  76 ++-
 .../common/requests/DescribeQuorumRequest.java |   4 +-
 .../common/requests/DescribeQuorumResponse.java|  13 +-
 .../common/message/DescribeQuorumRequest.json  |   4 +-
 .../common/message/DescribeQuorumResponse.json |  24 +-
 .../kafka/clients/admin/KafkaAdminClientTest.java  |  23 +-
 .../unit/kafka/server/ApiVersionsRequestTest.scala |   4 +-
 .../kafka/server/DescribeQuorumRequestTest.scala   |   2 +
 .../scala/unit/kafka/server/KafkaApisTest.scala|   3 +-
 .../org/apache/kafka/raft/KafkaRaftClient.java |  10 +-
 .../apache/kafka/raft/KafkaRaftClientDriver.java   |   1 +
 .../java/org/apache/kafka/raft/LeaderState.java|  84 +--
 .../java/org/apache/kafka/raft/QuorumState.java|   3 +-
 .../java/org/apache/kafka/raft/RaftMessage.java|   3 +-
 .../java/org/apache/kafka/raft/RaftRequest.java|  17 +-
 .../java/org/apache/kafka/raft/RaftResponse.java   |   5 +
 .../kafka/raft/internals/BlockingMessageQueue.java |   5 +
 .../org/apache/kafka/raft/internals/VoterSet.java  |   6 +-
 .../apache/kafka/raft/KafkaNetworkChannelTest.java |   1 +
 .../org/apache/kafka/raft/KafkaRaftClientTest.java |   7 +
 .../org/apache/kafka/raft/LeaderStateTest.java | 243 -
 .../apache/kafka/raft/RaftClientTestContext.java   |  19 +-
 .../apache/kafka/raft/RaftEventSimulationTest.java |   2 +-
 .../kafka/raft/internals/KafkaRaftMetricsTest.java |   4 +-
 25 files changed, 418 insertions(+), 165 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java 
b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
index c59cccf67c4..d7d525e4431 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
@@ -4414,12 +4414,13 @@ public class KafkaAdminClient extends AdminClient {
 private QuorumInfo.ReplicaState 
translateReplicaState(DescribeQuorumResponseData.ReplicaState replica) {
 return new QuorumInfo.ReplicaState(
 replica.replicaId(),
+replica.replicaDirectoryId() == null ? Uuid.ZERO_UUID 
: replica.replicaDirectoryId(),
 replica.logEndOffset(),
 replica.lastFetchTimestamp() == -1 ? 
OptionalLong.empty() : OptionalLong.of(replica.lastFetchTimestamp()),
 replica.lastCaughtUpTimestamp() == -1 ? 
OptionalLong.empty() : OptionalLong.of(replica.lastCaughtUpTimestamp()));
 }
 
-private QuorumInfo createQuorumResult(final 
DescribeQuorumResponseData.PartitionData partition) {
+private QuorumInfo createQuorumResult(final 
DescribeQuorumResponseData.PartitionData partition, 
DescribeQuorumResponseData.NodeCollection nodeCollection) {
 List voters = 
partition.currentVoters().stream()
 .map(this::translateReplicaState)
 .collect(Collectors.toList());
@@ -4428,12 +4429,21 @@ public class KafkaAdminClient extends AdminClient {
 .map(this::translateReplicaState)
 .collect(Collectors.toList());
 
+Map nodes = 
nodeCollection.stream().map(n -> {
+List endpoints = n.listeners().stream()
+.map(l -> new RaftVoterEndpoint(l.name(), l.host(), 
l.port()))
+.collect(Collectors.toList());
+
+return new QuorumInfo.Node(n.nodeId(), endpoints);
+}).collect(Collectors.toMap(QuorumInfo.Node::nodeId, 
Function.identity()));
+
 return new QuorumInfo(
 partition.leaderId(),
 partition.leaderEpoch(),
 partition.highWatermark(),
 voters,
-observers
+observers,
+   

(kafka) branch 3.8 updated: KAFKA-16535: Implement AddVoter, RemoveVoter, UpdateVoter RPCs

2024-06-07 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.8
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.8 by this push:
 new 7879f1c0131 KAFKA-16535: Implement AddVoter, RemoveVoter, UpdateVoter 
RPCs
7879f1c0131 is described below

commit 7879f1c01311912be5a055045111f1a628c47c60
Author: Colin P. McCabe 
AuthorDate: Tue Jun 4 14:04:59 2024 -0700

KAFKA-16535: Implement AddVoter, RemoveVoter, UpdateVoter RPCs

Implement the add voter, remove voter, and update voter RPCs for
KIP-853. This is just adding the RPC handling; the current
implementation in RaftManager just throws UnsupportedVersionException.

Reviewers: Andrew Schofield , José Armando García 
Sancio 

Conflicts: Fix some conflicts caused by the lack of KIP-932 RPCs in 3.8.
---
 .../kafka/clients/admin/AddRaftVoterOptions.java   |  26 ++
 .../kafka/clients/admin/AddRaftVoterResult.java|  42 +
 .../java/org/apache/kafka/clients/admin/Admin.java |  56 
 .../kafka/clients/admin/ForwardingAdmin.java   |  10 +++
 .../kafka/clients/admin/KafkaAdminClient.java  | 100 +
 .../kafka/clients/admin/RaftVoterEndpoint.java | 100 +
 .../clients/admin/RemoveRaftVoterOptions.java  |  26 ++
 .../kafka/clients/admin/RemoveRaftVoterResult.java |  42 +
 .../org/apache/kafka/common/protocol/ApiKeys.java  |   6 +-
 .../kafka/common/requests/AbstractRequest.java |   6 ++
 .../kafka/common/requests/AbstractResponse.java|   6 ++
 .../kafka/common/requests/AddRaftVoterRequest.java |  75 
 .../common/requests/AddRaftVoterResponse.java  |  65 ++
 .../common/requests/RemoveRaftVoterRequest.java|  75 
 .../common/requests/RemoveRaftVoterResponse.java   |  65 ++
 .../common/requests/UpdateRaftVoterRequest.java|  73 +++
 .../common/requests/UpdateRaftVoterResponse.java   |  65 ++
 .../common/message/AddRaftVoterRequest.json|  40 +
 .../common/message/AddRaftVoterResponse.json   |  30 +++
 .../common/message/RemoveRaftVoterRequest.json |  30 +++
 .../common/message/RemoveRaftVoterResponse.json|  30 +++
 .../common/message/UpdateRaftVoterRequest.json |  46 ++
 .../common/message/UpdateRaftVoterResponse.json|  28 ++
 .../kafka/clients/admin/MockAdminClient.java   |  10 +++
 .../kafka/common/requests/RequestResponseTest.java |  64 +
 .../scala/kafka/network/RequestConvertToJson.scala |   6 ++
 core/src/main/scala/kafka/raft/RaftManager.scala   |   5 +-
 .../main/scala/kafka/server/ControllerApis.scala   |  18 
 core/src/main/scala/kafka/server/KafkaApis.scala   |   2 +
 .../scala/unit/kafka/server/RequestQuotaTest.scala |   9 ++
 .../scala/unit/kafka/tools/StorageToolTest.scala   |  10 ++-
 31 files changed, 1161 insertions(+), 5 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/clients/admin/AddRaftVoterOptions.java 
b/clients/src/main/java/org/apache/kafka/clients/admin/AddRaftVoterOptions.java
new file mode 100644
index 000..d917c09fa1e
--- /dev/null
+++ 
b/clients/src/main/java/org/apache/kafka/clients/admin/AddRaftVoterOptions.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin;
+
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+/**
+ * Options for {@link Admin#addRaftVoter}.
+ */
+@InterfaceStability.Stable
+public class AddRaftVoterOptions extends AbstractOptions {
+}
diff --git 
a/clients/src/main/java/org/apache/kafka/clients/admin/AddRaftVoterResult.java 
b/clients/src/main/java/org/apache/kafka/clients/admin/AddRaftVoterResult.java
new file mode 100644
index 000..d42204c5e4e
--- /dev/null
+++ 
b/clients/src/main/java/org/apache/kafka/clients/admin/AddRaftVoterResult.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional i

(kafka) branch trunk updated (8b3c77c6711 -> 9ceed8f18f4)

2024-06-04 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


from 8b3c77c6711 KAFKA-15305 The background thread should try to process 
the remaining task until the shutdown timer is expired. (#16156)
 add 9ceed8f18f4 KAFKA-16535: Implement AddVoter, RemoveVoter, UpdateVoter 
RPCs

No new revisions were added by this update.

Summary of changes:
 .../admin/AddRaftVoterOptions.java}|  12 +--
 ...ionTokenResult.java => AddRaftVoterResult.java} |  20 ++---
 .../java/org/apache/kafka/clients/admin/Admin.java |  56 
 .../kafka/clients/admin/ForwardingAdmin.java   |  10 +++
 .../kafka/clients/admin/KafkaAdminClient.java  | 100 +
 .../kafka/clients/admin/RaftVoterEndpoint.java | 100 +
 .../admin/RemoveRaftVoterOptions.java} |  12 +--
 ...TokenResult.java => RemoveRaftVoterResult.java} |  20 ++---
 .../org/apache/kafka/common/protocol/ApiKeys.java  |   5 +-
 .../kafka/common/requests/AbstractRequest.java |   6 ++
 .../kafka/common/requests/AbstractResponse.java|   6 ++
 ...nTokenRequest.java => AddRaftVoterRequest.java} |  65 +++---
 ...DirsResponse.java => AddRaftVoterResponse.java} |  37 
 ...irsRequest.java => RemoveRaftVoterRequest.java} |  48 +-
 ...nResponse.java => RemoveRaftVoterResponse.java} |  26 +++---
 ...irsRequest.java => UpdateRaftVoterRequest.java} |  46 +-
 ...nResponse.java => UpdateRaftVoterResponse.java} |  26 +++---
 .../common/message/AddRaftVoterRequest.json|  40 +
 ...rIdsResponse.json => AddRaftVoterResponse.json} |  12 ++-
 ...onsRequest.json => RemoveRaftVoterRequest.json} |  17 ++--
 ...sResponse.json => RemoveRaftVoterResponse.json} |  12 ++-
 ...ionRequest.json => UpdateRaftVoterRequest.json} |  37 
 ...yResponse.json => UpdateRaftVoterResponse.json} |  18 ++--
 .../kafka/clients/admin/MockAdminClient.java   |  10 +++
 .../kafka/common/requests/RequestResponseTest.java |  64 +
 .../scala/kafka/network/RequestConvertToJson.scala |   6 ++
 core/src/main/scala/kafka/raft/RaftManager.scala   |   5 +-
 .../main/scala/kafka/server/ControllerApis.scala   |  18 
 core/src/main/scala/kafka/server/KafkaApis.scala   |   2 +
 .../scala/unit/kafka/server/RequestQuotaTest.scala |   9 ++
 .../scala/unit/kafka/tools/StorageToolTest.scala   |  10 ++-
 31 files changed, 648 insertions(+), 207 deletions(-)
 copy 
clients/src/main/java/org/apache/kafka/{server/authorizer/AuthorizationResult.java
 => clients/admin/AddRaftVoterOptions.java} (81%)
 copy 
clients/src/main/java/org/apache/kafka/clients/admin/{ExpireDelegationTokenResult.java
 => AddRaftVoterResult.java} (68%)
 create mode 100644 
clients/src/main/java/org/apache/kafka/clients/admin/RaftVoterEndpoint.java
 copy 
clients/src/main/java/org/apache/kafka/{server/authorizer/AuthorizationResult.java
 => clients/admin/RemoveRaftVoterOptions.java} (80%)
 copy 
clients/src/main/java/org/apache/kafka/clients/admin/{ExpireDelegationTokenResult.java
 => RemoveRaftVoterResult.java} (68%)
 copy 
clients/src/main/java/org/apache/kafka/common/requests/{CreateDelegationTokenRequest.java
 => AddRaftVoterRequest.java} (55%)
 copy 
clients/src/main/java/org/apache/kafka/common/requests/{AssignReplicasToDirsResponse.java
 => AddRaftVoterResponse.java} (64%)
 copy 
clients/src/main/java/org/apache/kafka/common/requests/{DescribeLogDirsRequest.java
 => RemoveRaftVoterRequest.java} (55%)
 copy 
clients/src/main/java/org/apache/kafka/common/requests/{ControllerRegistrationResponse.java
 => RemoveRaftVoterResponse.java} (64%)
 copy 
clients/src/main/java/org/apache/kafka/common/requests/{DescribeLogDirsRequest.java
 => UpdateRaftVoterRequest.java} (56%)
 copy 
clients/src/main/java/org/apache/kafka/common/requests/{ControllerRegistrationResponse.java
 => UpdateRaftVoterResponse.java} (64%)
 create mode 100644 
clients/src/main/resources/common/message/AddRaftVoterRequest.json
 copy 
clients/src/main/resources/common/message/{AllocateProducerIdsResponse.json => 
AddRaftVoterResponse.json} (74%)
 copy 
clients/src/main/resources/common/message/{GetTelemetrySubscriptionsRequest.json
 => RemoveRaftVoterRequest.json} (65%)
 copy 
clients/src/main/resources/common/message/{AllocateProducerIdsResponse.json => 
RemoveRaftVoterResponse.json} (74%)
 copy 
clients/src/main/resources/common/message/{ControllerRegistrationRequest.json 
=> UpdateRaftVoterRequest.json} (51%)
 copy clients/src/main/resources/common/message/{PushTelemetryResponse.json => 
UpdateRaftVoterResponse.json} (77%)



(kafka) branch KAFKA-16649 deleted (was 32eb8c3d68d)

2024-05-29 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a change to branch KAFKA-16649
in repository https://gitbox.apache.org/repos/asf/kafka.git


 was 32eb8c3d68d KAFKA-16649: Remove lock from 
DynamicBrokerConfig.removeReconfigurable

The revisions that were on this branch are still contained in
other references; therefore, this change does not discard any commits
from the repository.



(kafka) branch KAFKA-16649 updated (f435287ac63 -> 32eb8c3d68d)

2024-05-29 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a change to branch KAFKA-16649
in repository https://gitbox.apache.org/repos/asf/kafka.git


 discard f435287ac63 Fixes
omit 5a7a9b7f213 KAFKA-16649: Remove lock from 
DynamicBrokerConfig.removeReconfigurable
 add 31355ef8f94  KAFKA-16475: add more tests to TopicImageNodeTest (#15735)
 add 4825c89d14e KAFKA-16588 broker shutdown hangs when 
log.segment.delete.delay.ms is zero (#15773)
 add d9c36299db7 KAFKA-16614 Disallow @ClusterTemplate("") (#15800)
 add 89d8045a15b KAFKA-16647 Remove setMetadataDirectory from 
BrokerNode/ControllerNode (#15833)
 add cdc4caa5787 KAFKA-14588 UserScramCredentialsCommandTest rewritten in 
Java (#15832)
 add a3f24149905 KAFKA-16624: Don't generate useless PartitionChangeRecord 
on older MV (#15810)
 add 87390f961f3 KAFKA-16572 allow defining number of disks per broker in 
ClusterTest (#15745)
 add 240243b91d6 KAFKA-10199: Accept only one task per element in output 
queue for failed tasks (#15849)
 add 2c0b8b69207 MINOR: ConsumerGroup#getOrMaybeCreateMember should not add 
the member to the group (#15847)
 add 1fd39150aa3 KAFKA-16655: Deflake 
ZKMigrationIntegrationTest.testDualWrite #15845
 add 9b8aac22ec7 KAFKA-16427 KafkaConsumer#position() does not respect 
timeout when group protocol is CONSUMER (#15843)
 add bfe81d62297 KAFKA-16207; KRaft's internal log listener to update voter 
set (#15671)
 add 25118cec145 MINOR: remove redundant check in KafkaClusterTestKit 
(#15858)
 add 970ac078812 KAFKA-16659 KafkaConsumer#position() does not respect 
wakup when group protocol is CONSUMER (#15853)
 add 41f5bf844df  KAFKA-16223 Replace EasyMock/PowerMock with Mockito for 
KafkaConfigBackingStoreTest (2/3) (#15841)
 add 55a00be4e97 MINOR: Replaced Utils.join() with JDK API. (#15823)
 add 366aeab488c KAFKA-10199: Add remove operation with future to state 
updater (#15852)
 add 6a8977e2125 KAFKA-14588 [3/N] ConfigCommandTest rewritten in java 
(#15850)
 add 5c96ad61d95 KAFKA-16393 read/write sequence of buffers correctly 
(#15571)
 add 42754336e1f MINOR: Remove `ConsumerGroupPartitionMetadataValue.Epoch` 
field (#15854)
 add 0b4eaefd863 KAFKA-16646: KAFKA-16646Don't run cve scan job on forks 
(#15831)
 add aeca384641b KAFKA-16356: Remove class-name dispatch in 
RemoteLogMetadataSerde (#15620)
 add 4c4ae6e39c4 KAFKA-16608 Honour interrupted thread state on 
KafkaConsumer.poll (#15803)
 add fe8ccbc92c2 KAFKA-16539 Fix IncrementalAlterConfigs during ZK 
migration (#15744)
 add 0df340d64d3 KAFKA-16470 kafka-dump-log --offsets-decoder should 
support new records (#15652)
 add 0de3b7c40b0 KAFKA-16593 Rewrite DeleteConsumerGroupsTest by 
ClusterTestExtensions (#15766)
 add d76352e2151 MINOR: log newly created processId (#15851)
 add 459eaec666c KAFKA-16615; JoinGroup API for upgrading ConsumerGroup 
(#15798)
 add ea485a70611 KAFKA-16665: Allow to initialize newly assigned 
partition's positions without allowing fetching while callback runs (#15856)
 add cb35ddc5ca2 KAFKA-10199: Remove lost tasks in state updater with new 
remove (#15870)
 add 21bf715622e KAFKA-16307; Fix coordinator thread idle ratio (#15835)
 add 05df10449eb KAFKA-13328, KAFKA-13329 (2): Add custom preflight 
validation support for connector header, key, and value converters (#14309)
 add a4c6cefd10f KAFKA-14226: Introduce FieldPath abstraction and nested 
path support for ExtractField SMT (#15379)
 add 525b9b1d768 KAFKA-15018: Write connector tombstone offsets to 
secondary store before primary store (#13801)
 add a0f1658bb13 KAFKA-16678 Remove variable "unimplementedquorum" (#15879)
 add 5f933ac8403 MINOR: Correct connector scheduled rebalance logs (#15875)
 add 8655094e6c9 KAFKA-16511: Fix the leaking tiered segments during 
segment deletion (#15817)
 add 3b43edd7a1c MINOR: Remove dev_version parameter from streams tests 
(#15874)
 add f7b242f94e8 KAFKA-10199: Revoke tasks from state updater with new 
remove  (#15871)
 add f74f596bc7d KAFKA-16640 Replace TestUtils#resource by scala.util.Using 
(#15881)
 add 2a5efe4a334 KAFKA-16685: Add parent exception to RLMTask warning logs 
(#15880)
 add c64a315fd55 MINOR: Made the supportedOperation variable name more 
verbose (#15892)
 add 29f3260a9c0 MINOR: Fix streams javadoc links (#15900)
 add 397d58b14c9 MINOR: use classic consumer with ZK mode for 
DeleteOffsetsConsumerGroupCommandIntegrationTest (#15872)
 add f4fdaa702a2 MINOR: Add missing RPCs to security.html (#15878)
 add 5a9ccb6b773 KAFKA-16445: Add PATCH method for connector config (#6934)
 add 8fd6596454e KAFKA-15045: (KIP-924) New interfaces and stubbed utility 
classes for pluggable TaskAssignors. (#15887)
 add 7e9ab4b2c68 KAFKA-16484 Support to define per broker/controller 
property by ClusterConfigProperty (#15715)

(kafka) 01/01: KAFKA-16649: Remove lock from DynamicBrokerConfig.removeReconfigurable

2024-05-29 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch KAFKA-16649
in repository https://gitbox.apache.org/repos/asf/kafka.git

commit 32eb8c3d68d4d6fc30705a55371cfe2ee57b8f77
Author: Colin P. McCabe 
AuthorDate: Tue Apr 30 14:05:32 2024 -0700

KAFKA-16649: Remove lock from DynamicBrokerConfig.removeReconfigurable

Do not acquire the DynamicBrokerConfig lock in 
DynamicBrokerConfig.removeReconfigurable. It's not
necessary, because the list that these functions are modifying is a 
thread-safe
CopyOnWriteArrayList.  In 
DynamicBrokerConfig.reloadUpdatedFilesWithoutConfigChange, I changed the
code to use a simple Java forEach rather than a Scala conversion, in order 
to feel more confident
that concurrent modifications to the List would not have any bad effects 
here. (forEach is always
safe on CopyOnWriteArrayList.)
---
 .../scala/kafka/server/DynamicBrokerConfig.scala   | 26 +
 .../kafka/server/KRaftClusterTest.scala| 34 --
 2 files changed, 46 insertions(+), 14 deletions(-)

diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala 
b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala
index e2879d4378e..156d2d02b09 100755
--- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala
+++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala
@@ -303,17 +303,17 @@ class DynamicBrokerConfig(private val kafkaConfig: 
KafkaConfig) extends Logging
 addBrokerReconfigurable(controller.socketServer)
   }
 
-  def addReconfigurable(reconfigurable: Reconfigurable): Unit = 
CoreUtils.inWriteLock(lock) {
+  def addReconfigurable(reconfigurable: Reconfigurable): Unit = {
 verifyReconfigurableConfigs(reconfigurable.reconfigurableConfigs.asScala)
 reconfigurables.add(reconfigurable)
   }
 
-  def addBrokerReconfigurable(reconfigurable: BrokerReconfigurable): Unit = 
CoreUtils.inWriteLock(lock) {
+  def addBrokerReconfigurable(reconfigurable: BrokerReconfigurable): Unit = {
 verifyReconfigurableConfigs(reconfigurable.reconfigurableConfigs)
 brokerReconfigurables.add(reconfigurable)
   }
 
-  def removeReconfigurable(reconfigurable: Reconfigurable): Unit = 
CoreUtils.inWriteLock(lock) {
+  def removeReconfigurable(reconfigurable: Reconfigurable): Unit = {
 reconfigurables.remove(reconfigurable)
   }
 
@@ -370,16 +370,18 @@ class DynamicBrokerConfig(private val kafkaConfig: 
KafkaConfig) extends Logging
* changes are processed. At the moment, only listener configs are 
considered for reloading.
*/
   private[server] def reloadUpdatedFilesWithoutConfigChange(newProps: 
Properties): Unit = CoreUtils.inWriteLock(lock) {
-reconfigurables.asScala
-  .filter(reconfigurable => 
ReloadableFileConfigs.exists(reconfigurable.reconfigurableConfigs.contains))
-  .foreach {
-case reconfigurable: ListenerReconfigurable =>
-  val kafkaProps = validatedKafkaProps(newProps, perBrokerConfig = 
true)
-  val newConfig = new KafkaConfig(kafkaProps.asJava, false, None)
-  processListenerReconfigurable(reconfigurable, newConfig, 
Collections.emptyMap(), validateOnly = false, reloadOnly = true)
-case reconfigurable =>
-  trace(s"Files will not be reloaded without config change for 
$reconfigurable")
+reconfigurables.forEach(r => {
+  if (ReloadableFileConfigs.exists(r.reconfigurableConfigs.contains)) {
+r match {
+  case reconfigurable: ListenerReconfigurable =>
+val kafkaProps = validatedKafkaProps(newProps, perBrokerConfig = 
true)
+val newConfig = new KafkaConfig(kafkaProps.asJava, false, None)
+processListenerReconfigurable(reconfigurable, newConfig, 
Collections.emptyMap(), validateOnly = false, reloadOnly = true)
+  case reconfigurable =>
+trace(s"Files will not be reloaded without config change for 
$reconfigurable")
+}
   }
+})
   }
 
   private def maybeCreatePasswordEncoder(secret: Option[Password]): 
Option[PasswordEncoder] = {
diff --git 
a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala 
b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala
index f5db6bd1a95..f54f4e8b3fd 100644
--- a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala
+++ b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala
@@ -46,7 +46,7 @@ import org.apache.kafka.metadata.bootstrap.BootstrapMetadata
 import org.apache.kafka.network.SocketServerConfigs
 import org.apache.kafka.server.authorizer._
 import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion}
-import org.apache.kafka.server.config.KRaftConfigs
+import org.apache.kafka.server.config.{KRaftConfigs, ServerConfigs}
 import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
 import org.apache.kafka.server.quota
 import org.apache.k

(kafka) branch trunk updated: MINOR: fix typo in KAFKA-16515

2024-05-27 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new bac8df56ffd MINOR: fix typo in KAFKA-16515
bac8df56ffd is described below

commit bac8df56ffdf8a64ecfb78ec0779bcbc8e9f7c10
Author: Colin P. McCabe 
AuthorDate: Mon May 27 08:53:53 2024 -0700

MINOR: fix typo in KAFKA-16515
---
 core/src/main/scala/kafka/server/KafkaServer.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala 
b/core/src/main/scala/kafka/server/KafkaServer.scala
index 5531ab1dbc3..4702b7e5a97 100755
--- a/core/src/main/scala/kafka/server/KafkaServer.scala
+++ b/core/src/main/scala/kafka/server/KafkaServer.scala
@@ -441,7 +441,7 @@ class KafkaServer(
 CompletableFuture.completedFuture(quorumVoters),
 fatalFaultHandler = new LoggingFaultHandler("raftManager", () => 
shutdown())
   )
-  val quorumControllerNodeProvider = 
RaftControllerNodeProvider(raftManager, config)
+  quorumControllerNodeProvider = 
RaftControllerNodeProvider(raftManager, config)
   val brokerToQuorumChannelManager = new 
NodeToControllerChannelManagerImpl(
 controllerNodeProvider = quorumControllerNodeProvider,
 time = time,



(kafka) branch trunk updated: KAFKA-16515: Fix the ZK Metadata cache confusion between brokers and controllers

2024-05-24 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 4f55786a8a8 KAFKA-16515: Fix the ZK Metadata cache confusion between 
brokers and controllers
4f55786a8a8 is described below

commit 4f55786a8a86fe228a0b10a2f28529f5128e5d6f
Author: Colin P. McCabe 
AuthorDate: Mon May 20 15:41:52 2024 -0700

KAFKA-16515: Fix the ZK Metadata cache confusion between brokers and 
controllers

ZkMetadataCache could theoretically return KRaft controller information 
from a call to
ZkMetadataCache.getAliveBrokerNode, which doesn't make sense. KRaft 
controllers are not part of the
set of brokers. The only use-case for this functionality was in 
MetadataCacheControllerNodeProvider
during ZK migration, where it allowed ZK brokers in migration mode to 
forward requests to
kcontrollers when appropriate. This PR changes 
MetadataCacheControllerNodeProvider to simply
delegate to quorumControllerNodeProvider in this case.

Reviewers: José Armando García Sancio 
---
 core/src/main/scala/kafka/server/KafkaServer.scala  | 17 +++--
 .../src/main/scala/kafka/server/MetadataCache.scala |  3 +--
 .../server/NodeToControllerChannelManager.scala | 21 +
 .../kafka/server/metadata/ZkMetadataCache.scala |  9 +
 .../jmh/fetcher/ReplicaFetcherThreadBenchmark.java  |  2 +-
 .../jmh/metadata/MetadataRequestBenchmark.java  |  2 +-
 .../apache/kafka/jmh/server/CheckpointBench.java|  2 +-
 .../kafka/jmh/server/PartitionCreationBench.java|  2 +-
 8 files changed, 18 insertions(+), 40 deletions(-)

diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala 
b/core/src/main/scala/kafka/server/KafkaServer.scala
index 106fd442259..5531ab1dbc3 100755
--- a/core/src/main/scala/kafka/server/KafkaServer.scala
+++ b/core/src/main/scala/kafka/server/KafkaServer.scala
@@ -165,8 +165,10 @@ class KafkaServer(
 
   var kafkaScheduler: KafkaScheduler = _
 
-  var kraftControllerNodes: Seq[Node] = _
   @volatile var metadataCache: ZkMetadataCache = _
+
+  @volatile var quorumControllerNodeProvider: RaftControllerNodeProvider = _
+
   var quotaManagers: QuotaFactory.QuotaManagers = _
 
   val zkClientConfig: ZKClientConfig = 
KafkaServer.zkClientConfigFromKafkaConfig(config)
@@ -324,20 +326,13 @@ class KafkaServer(
 
 remoteLogManagerOpt = createRemoteLogManager()
 
-if (config.migrationEnabled) {
-  kraftControllerNodes = QuorumConfig.voterConnectionsToNodes(
-QuorumConfig.parseVoterConnections(config.quorumVoters)
-  ).asScala
-} else {
-  kraftControllerNodes = Seq.empty
-}
 metadataCache = MetadataCache.zkMetadataCache(
   config.brokerId,
   config.interBrokerProtocolVersion,
   brokerFeatures,
-  kraftControllerNodes,
   config.migrationEnabled)
-val controllerNodeProvider = new 
MetadataCacheControllerNodeProvider(metadataCache, config)
+val controllerNodeProvider = new 
MetadataCacheControllerNodeProvider(metadataCache, config,
+  () => 
Option(quorumControllerNodeProvider).map(_.getControllerInfo()))
 
 /* initialize feature change listener */
 _featureChangeListener = new 
FinalizedFeatureChangeListener(metadataCache, _zkClient)
@@ -1075,6 +1070,8 @@ class KafkaServer(
 }
 _brokerState = BrokerState.NOT_RUNNING
 
+quorumControllerNodeProvider = null
+
 startupComplete.set(false)
 isShuttingDown.set(false)
 
CoreUtils.swallow(AppInfoParser.unregisterAppInfo(Server.MetricsPrefix, 
config.brokerId.toString, metrics), this)
diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala 
b/core/src/main/scala/kafka/server/MetadataCache.scala
index 015e46a7652..b8eda3fe4dc 100755
--- a/core/src/main/scala/kafka/server/MetadataCache.scala
+++ b/core/src/main/scala/kafka/server/MetadataCache.scala
@@ -116,10 +116,9 @@ object MetadataCache {
   def zkMetadataCache(brokerId: Int,
   metadataVersion: MetadataVersion,
   brokerFeatures: BrokerFeatures = 
BrokerFeatures.createEmpty(),
-  kraftControllerNodes: collection.Seq[Node] = 
collection.Seq.empty[Node],
   zkMigrationEnabled: Boolean = false)
   : ZkMetadataCache = {
-new ZkMetadataCache(brokerId, metadataVersion, brokerFeatures, 
kraftControllerNodes, zkMigrationEnabled)
+new ZkMetadataCache(brokerId, metadataVersion, brokerFeatures, 
zkMigrationEnabled)
   }
 
   def kRaftMetadataCache(brokerId: Int): KRaftMetadataCache = {
diff --git 
a/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala 
b/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala
index 36997a4ea49..0017a5876af 100644
--- a/c

(kafka) branch trunk updated (2432a1866e7 -> 90892ae99fb)

2024-05-24 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


from 2432a1866e7 KAFKA-16373: KIP-1028:  Adding code to support Apache 
Kafka Docker Official Images (#16027)
 add 90892ae99fb KAFKA-16516: Fix the controller node provider for broker 
to control channel

No new revisions were added by this update.

Summary of changes:
 core/src/main/scala/kafka/raft/RaftManager.scala   | 13 +---
 .../src/main/scala/kafka/server/BrokerServer.scala |  6 ++--
 .../main/scala/kafka/server/ControllerServer.scala |  2 +-
 core/src/main/scala/kafka/server/KafkaServer.scala |  3 +-
 .../server/NodeToControllerChannelManager.scala|  8 ++---
 .../kafka/server/KRaftClusterTest.scala| 38 ++
 .../org/apache/kafka/raft/KafkaRaftClient.java |  5 +++
 .../java/org/apache/kafka/raft/QuorumState.java|  1 +
 .../org/apache/kafka/raft/internals/VoterSet.java  | 14 
 9 files changed, 74 insertions(+), 16 deletions(-)



(kafka) branch trunk updated (2c0b8b69207 -> 1fd39150aa3)

2024-05-03 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


from 2c0b8b69207 MINOR: ConsumerGroup#getOrMaybeCreateMember should not add 
the member to the group (#15847)
 add 1fd39150aa3 KAFKA-16655: Deflake 
ZKMigrationIntegrationTest.testDualWrite #15845

No new revisions were added by this update.

Summary of changes:
 .../kafka/zk/ZkMigrationIntegrationTest.scala  | 22 ++
 1 file changed, 18 insertions(+), 4 deletions(-)



(kafka) branch KAFKA-16624 deleted (was bdc07910b00)

2024-05-02 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a change to branch KAFKA-16624
in repository https://gitbox.apache.org/repos/asf/kafka.git


 was bdc07910b00 New test, etc.

The revisions that were on this branch are still contained in
other references; therefore, this change does not discard any commits
from the repository.



(kafka) branch trunk updated: KAFKA-16624: Don't generate useless PartitionChangeRecord on older MV (#15810)

2024-05-02 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new a3f24149905 KAFKA-16624: Don't generate useless PartitionChangeRecord 
on older MV (#15810)
a3f24149905 is described below

commit a3f24149905886ac719901eeab8dc6d7b3c79614
Author: Colin Patrick McCabe 
AuthorDate: Thu May 2 09:23:25 2024 -0700

KAFKA-16624: Don't generate useless PartitionChangeRecord on older MV 
(#15810)

Fix a case where we could generate useless PartitionChangeRecords on 
metadata versions older than
3.6-IV0. This could happen in the case where we had an ISR with only one 
broker in it, and we were
trying to go down to a fully empty ISR. In this case, 
PartitionChangeBuilder would block the record
to going down to a fully empty ISR (since that is not valid in these 
pre-KIP-966 metadata
versions), but it would still emit the record, even though it had no effect.

Reviewers: Igor Soarez 
---
 .../kafka/controller/PartitionChangeBuilder.java   |  89 +
 .../controller/PartitionChangeBuilderTest.java | 213 -
 2 files changed, 181 insertions(+), 121 deletions(-)

diff --git 
a/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java
 
b/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java
index 7f1b2cb6d17..0d2c1bd6a9d 100644
--- 
a/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java
+++ 
b/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java
@@ -214,6 +214,10 @@ public class PartitionChangeBuilder {
 }
 }
 
+public List targetIsr() {
+return targetIsr;
+}
+
 // VisibleForTesting
 /**
  * Perform leader election based on the partition state and leader 
election type.
@@ -365,44 +369,61 @@ public class PartitionChangeBuilder {
 }
 
 /**
- * Trigger a leader epoch bump if one is needed.
- *
- * We need to bump the leader epoch if:
- * 1. The leader changed, or
- * 2. The new replica list does not contain all the nodes that the old 
replica list did.
+ * Trigger a leader epoch bump if one is needed because of replica 
reassignment.
  *
- * Changes that do NOT fall in any of these categories will increase the 
partition epoch, but
- * not the leader epoch. Note that if the leader epoch increases, the 
partition epoch will
- * always increase as well; there is no case where the partition epoch 
increases more slowly
- * than the leader epoch.
- *
- * If the PartitionChangeRecord sets the leader field to something other 
than
- * NO_LEADER_CHANGE, a leader epoch bump will automatically occur. That 
takes care of
- * case 1. In this function, we check for cases 2 and 3, and handle them 
by manually
- * setting record.leader to the current leader.
- *
- * In MV before 3.6 there was a bug (KAFKA-15021) in the brokers' replica 
manager
- * that required that the leader epoch be bump whenever the ISR shrank. In 
MV 3.6 this leader
- * bump is not required when the ISR shrinks. Note, that the leader epoch 
is never increased if
- * the ISR expanded.
+ * Note that if the leader epoch increases, the partition epoch will 
always increase as well; there is no
+ * case where the partition epoch increases more slowly than the leader 
epoch.
+ */
+void 
triggerLeaderEpochBumpForReplicaReassignmentIfNeeded(PartitionChangeRecord 
record) {
+if (record.leader() != NO_LEADER_CHANGE) {
+// The leader is already changing, so there will already be a 
leader epoch bump.
+return;
+}
+if (!Replicas.contains(targetReplicas, partition.replicas)) {
+// If the new replica list does not contain all the brokers that 
the old one did,
+// ensure that there will be a leader epoch bump by setting the 
leader field.
+record.setLeader(partition.leader);
+}
+}
+
+/**
+ * Trigger a leader epoch bump if one is needed because of an ISR shrink.
  *
- * In MV 3.6 and beyond, if the controller is in ZK migration mode, the 
leader epoch must
- * be bumped during ISR shrink for compatability with ZK brokers.
+ * Note that it's important to call this function only after we have set 
the ISR field in
+ * the PartitionChangeRecord.
  */
-void triggerLeaderEpochBumpIfNeeded(PartitionChangeRecord record) {
-if (record.leader() == NO_LEADER_CHANGE) {
-boolean bumpLeaderEpochOnIsrShrink = 
metadataVersion.isLeaderEpochBumpRequiredOnIsrShrink() || zkMigrationEnabled;
-
-if (!Replicas.contains(targetReplicas, partition.replicas)) {
-// Reassignment
-record.setLeader(partition.leader

(kafka) branch KAFKA-16624 updated (65d96aeaf50 -> bdc07910b00)

2024-05-01 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a change to branch KAFKA-16624
in repository https://gitbox.apache.org/repos/asf/kafka.git


from 65d96aeaf50 KAFKA-16624: Don't generate useless PartitionChangeRecord 
on older MV
 add bdc07910b00 New test, etc.

No new revisions were added by this update.

Summary of changes:
 .../kafka/controller/PartitionChangeBuilder.java   |  5 ++--
 .../controller/PartitionChangeBuilderTest.java | 33 ++
 2 files changed, 35 insertions(+), 3 deletions(-)



(kafka) 01/01: Fixes

2024-05-01 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch KAFKA-16649
in repository https://gitbox.apache.org/repos/asf/kafka.git

commit f435287ac6315f9bacd41c7862559b42cbd8f395
Author: Colin P. McCabe 
AuthorDate: Wed May 1 14:25:18 2024 -0700

Fixes
---
 core/src/main/scala/kafka/server/DynamicBrokerConfig.scala| 8 +++-
 .../test/scala/integration/kafka/server/KRaftClusterTest.scala| 4 ++--
 2 files changed, 9 insertions(+), 3 deletions(-)

diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala 
b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala
index 924bb0f660e..d2d518f1dd2 100755
--- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala
+++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala
@@ -221,7 +221,13 @@ class DynamicBrokerConfig(private val kafkaConfig: 
KafkaConfig) extends Logging
   // collections, while another thread is iterating over them.
   private[server] val reconfigurables = new 
CopyOnWriteArrayList[Reconfigurable]()
   private val brokerReconfigurables = new 
CopyOnWriteArrayList[BrokerReconfigurable]()
+
+  /**
+   * The DynamicBrokerConfig lock which prevents concurrent changes to dynamic 
configuration.
+   * (It does not prevent new reconfigurables from being registered or 
unregistered, though.)
+   */
   private val lock = new ReentrantReadWriteLock
+
   private var metricsReceiverPluginOpt: Option[ClientMetricsReceiverPlugin] = _
   private var currentConfig: KafkaConfig = _
   private val dynamicConfigPasswordEncoder = if 
(kafkaConfig.processRoles.isEmpty) {
@@ -317,7 +323,7 @@ class DynamicBrokerConfig(private val kafkaConfig: 
KafkaConfig) extends Logging
 reconfigurables.remove(reconfigurable)
   }
 
-  private def verifyReconfigurableConfigs(configNames: Set[String]): Unit = 
CoreUtils.inWriteLock(lock) {
+  private def verifyReconfigurableConfigs(configNames: Set[String]): Unit = {
 val nonDynamic = 
configNames.filter(DynamicConfig.Broker.nonDynamicProps.contains)
 require(nonDynamic.isEmpty, s"Reconfigurable contains non-dynamic configs 
$nonDynamic")
   }
diff --git 
a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala 
b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala
index 83182d54443..d59d8859562 100644
--- a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala
+++ b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala
@@ -1548,7 +1548,7 @@ class KRaftClusterTest {
   new TestKitNodes.Builder().
 setNumBrokerNodes(1).
 setNumControllerNodes(1).build()).
-  setConfigProp(KafkaConfig.NumNetworkThreadsProp, "4").
+  setConfigProp(KafkaConfig.NumNetworkThreadsProp, "2").
   build()
 try {
   cluster.format()
@@ -1559,7 +1559,7 @@ class KRaftClusterTest {
 admin.incrementalAlterConfigs(
   Collections.singletonMap(new ConfigResource(Type.BROKER, ""),
 Collections.singletonList(new AlterConfigOp(
-  new ConfigEntry(KafkaConfig.NumNetworkThreadsProp, "2"), 
OpType.SET.all().get()
+  new ConfigEntry(KafkaConfig.NumNetworkThreadsProp, "3"), 
OpType.SET.all().get()
 val newTopic = Collections.singletonList(new NewTopic("test-topic", 1, 
1.toShort))
 val createTopicResult = admin.createTopics(newTopic)
 createTopicResult.all().get()



(kafka) branch KAFKA-16649 created (now f435287ac63)

2024-05-01 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a change to branch KAFKA-16649
in repository https://gitbox.apache.org/repos/asf/kafka.git


  at f435287ac63 Fixes

This branch includes the following new commits:

 new f435287ac63 Fixes

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.




(kafka) branch trunk updated: KAFKA-16475: add more tests to TopicImageNodeTest (#15735)

2024-04-30 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 31355ef8f94  KAFKA-16475: add more tests to TopicImageNodeTest (#15735)
31355ef8f94 is described below

commit 31355ef8f948f369e240ebc203f889f187116d75
Author: mannoopj <139923522+manno...@users.noreply.github.com>
AuthorDate: Tue Apr 30 17:59:00 2024 -0400

 KAFKA-16475: add more tests to TopicImageNodeTest (#15735)

Add more test cases to TopicImageNodeTest.java.

Reviewers: Colin P. McCabe 
---
 .../kafka/image/node/TopicImageNodeTest.java   | 43 +++---
 1 file changed, 38 insertions(+), 5 deletions(-)

diff --git 
a/metadata/src/test/java/org/apache/kafka/image/node/TopicImageNodeTest.java 
b/metadata/src/test/java/org/apache/kafka/image/node/TopicImageNodeTest.java
index 83855d94b8c..8ecd4be28c9 100644
--- a/metadata/src/test/java/org/apache/kafka/image/node/TopicImageNodeTest.java
+++ b/metadata/src/test/java/org/apache/kafka/image/node/TopicImageNodeTest.java
@@ -17,14 +17,17 @@
 
 package org.apache.kafka.image.node;
 
+import org.apache.kafka.common.DirectoryId;
 import org.apache.kafka.common.Uuid;
 import org.apache.kafka.image.TopicImage;
+import org.apache.kafka.image.node.printer.NodeStringifier;
+import org.apache.kafka.metadata.LeaderRecoveryState;
+import org.apache.kafka.metadata.PartitionRegistration;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.Timeout;
-
 import java.util.Arrays;
-import java.util.Collections;
-
+import java.util.HashMap;
+import java.util.Map;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertNull;
@@ -32,11 +35,21 @@ import static org.junit.jupiter.api.Assertions.assertNull;
 
 @Timeout(value = 40)
 public class TopicImageNodeTest {
-private final static TopicImageNode NODE = new TopicImageNode(new 
TopicImage("topic-image-node-test-topic", Uuid.ZERO_UUID, 
Collections.emptyMap()));
+private final static TopicImageNode NODE = new 
TopicImageNode(newTopicImage("topic-image-node-test-topic", Uuid.ZERO_UUID, new 
PartitionRegistration.Builder().setReplicas(new int[] {2, 3, 4}).
+setDirectories(DirectoryId.migratingArray(3)).
+setIsr(new int[] {2, 
3}).setLeader(2).setLeaderRecoveryState(LeaderRecoveryState.RECOVERED).setLeaderEpoch(1).setPartitionEpoch(345).build()));
 
+private static TopicImage newTopicImage(String name, Uuid id, 
PartitionRegistration... partitions) {
+Map partitionMap = new HashMap<>();
+int i = 0;
+for (PartitionRegistration partition : partitions) {
+partitionMap.put(i++, partition);
+}
+return new TopicImage(name, id, partitionMap);
+}
 @Test
 public void testChildNames() {
-assertEquals(Arrays.asList("name", "id"), NODE.childNames());
+assertEquals(Arrays.asList("name", "id", "0"), NODE.childNames());
 }
 
 @Test
@@ -57,4 +70,24 @@ public class TopicImageNodeTest {
 public void testUnknownChild() {
 assertNull(NODE.child("unknown"));
 }
+
+@Test
+public void testChildPartitionId() {
+MetadataNode child = NODE.child("0");
+assertNotNull(child);
+NodeStringifier stringifier = new NodeStringifier();
+child.print(stringifier);
+assertEquals("PartitionRegistration(replicas=[2, 3, 4], " +
+"directories=[AA, AA, 
AA], " +
+"isr=[2, 3], removingReplicas=[], addingReplicas=[], elr=[], 
lastKnownElr=[], leader=2, " +
+"leaderRecoveryState=RECOVERED, leaderEpoch=1, 
partitionEpoch=345)", stringifier.toString());
+}
+
+@Test
+public void testChildPartitionIdNull() {
+MetadataNode child1 = NODE.child("1");
+MetadataNode child2 = NODE.child("a");
+assertNull(child1);
+assertNull(child2);
+}
 }



(kafka) branch trunk updated: MINOR: Change the documentation of the Brokers field. (#15809)

2024-04-26 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new ba6a73776b6 MINOR: Change the documentation of the Brokers field. 
(#15809)
ba6a73776b6 is described below

commit ba6a73776b65603b4c866f8d2db966d826031e74
Author: Emanuele Sabellico 
AuthorDate: Sat Apr 27 00:53:18 2024 +0200

MINOR: Change the documentation of the Brokers field. (#15809)

Change the documentation of the Brokers field to make it clear that it 
doesn't always have all the
brokers that are listed as replicas.

Reviewer: Colin P. McCabe 
---
 clients/src/main/resources/common/message/MetadataResponse.json | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/clients/src/main/resources/common/message/MetadataResponse.json 
b/clients/src/main/resources/common/message/MetadataResponse.json
index bb0b6a802c3..408cdc7940a 100644
--- a/clients/src/main/resources/common/message/MetadataResponse.json
+++ b/clients/src/main/resources/common/message/MetadataResponse.json
@@ -48,7 +48,7 @@
 { "name": "ThrottleTimeMs", "type": "int32", "versions": "3+", 
"ignorable": true,
   "about": "The duration in milliseconds for which the request was 
throttled due to a quota violation, or zero if the request did not violate any 
quota." },
 { "name": "Brokers", "type": "[]MetadataResponseBroker", "versions": "0+",
-  "about": "Each broker in the response.", "fields": [
+  "about": "A list of brokers present in the cluster.", "fields": [
   { "name": "NodeId", "type": "int32", "versions": "0+", "mapKey": true, 
"entityType": "brokerId",
 "about": "The broker ID." },
   { "name": "Host", "type": "string", "versions": "0+",



(kafka) branch KAFKA-16624 updated: KAFKA-16624: Don't generate useless PartitionChangeRecord on older MV

2024-04-25 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch KAFKA-16624
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/KAFKA-16624 by this push:
 new 65d96aeaf50 KAFKA-16624: Don't generate useless PartitionChangeRecord 
on older MV
65d96aeaf50 is described below

commit 65d96aeaf50b6076b982b4b2e69fec0803c4be83
Author: Colin P. McCabe 
AuthorDate: Thu Apr 25 14:08:25 2024 -0700

KAFKA-16624: Don't generate useless PartitionChangeRecord on older MV

Fix a case where we could generate useless PartitionChangeRecords on 
metadata versions older than
3.6-IV0. This could happen in the case where we had an ISR with only one 
broker in it, and we were
trying to go down to a fully empty ISR. In this case, 
PartitionChangeBuilder would block the record
to going down to a fully empty ISR (since that is not valid in these 
pre-KIP-966 metadata
versions), but it would still emit the record, even though it had no effect.
---
 .../kafka/controller/PartitionChangeBuilder.java   |  94 +++
 .../controller/PartitionChangeBuilderTest.java | 180 +++--
 2 files changed, 151 insertions(+), 123 deletions(-)

diff --git 
a/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java
 
b/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java
index 7f1b2cb6d17..59733445695 100644
--- 
a/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java
+++ 
b/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java
@@ -214,6 +214,10 @@ public class PartitionChangeBuilder {
 }
 }
 
+public List targetIsr() {
+return targetIsr;
+}
+
 // VisibleForTesting
 /**
  * Perform leader election based on the partition state and leader 
election type.
@@ -365,45 +369,62 @@ public class PartitionChangeBuilder {
 }
 
 /**
- * Trigger a leader epoch bump if one is needed.
- *
- * We need to bump the leader epoch if:
- * 1. The leader changed, or
- * 2. The new replica list does not contain all the nodes that the old 
replica list did.
- *
- * Changes that do NOT fall in any of these categories will increase the 
partition epoch, but
- * not the leader epoch. Note that if the leader epoch increases, the 
partition epoch will
- * always increase as well; there is no case where the partition epoch 
increases more slowly
- * than the leader epoch.
+ * Trigger a leader epoch bump if one is needed because of replica 
reassignment.
  *
- * If the PartitionChangeRecord sets the leader field to something other 
than
- * NO_LEADER_CHANGE, a leader epoch bump will automatically occur. That 
takes care of
- * case 1. In this function, we check for cases 2 and 3, and handle them 
by manually
- * setting record.leader to the current leader.
- *
- * In MV before 3.6 there was a bug (KAFKA-15021) in the brokers' replica 
manager
- * that required that the leader epoch be bump whenever the ISR shrank. In 
MV 3.6 this leader
- * bump is not required when the ISR shrinks. Note, that the leader epoch 
is never increased if
- * the ISR expanded.
+ * Note that if the leader epoch increases, the partition epoch will 
always increase as well; there is no
+ * case where the partition epoch increases more slowly than the leader 
epoch.
+ */
+void 
triggerLeaderEpochBumpForReplicaReassignmentIfNeeded(PartitionChangeRecord 
record) {
+if (record.leader() != NO_LEADER_CHANGE) {
+// The leader is already changing, so there will already be a 
leader epoch bump.
+return;
+}
+if (!Replicas.contains(targetReplicas, partition.replicas)) {
+// If the new replica list does not contain all the brokers that 
the old one did,
+// ensure that there will be a leader epoch bump by setting the 
leader field.
+record.setLeader(partition.leader);
+}
+}
+
+/**
+ * Trigger a leader epoch bump if one is needed because of an ISR shrink.
  *
- * In MV 3.6 and beyond, if the controller is in ZK migration mode, the 
leader epoch must
- * be bumped during ISR shrink for compatability with ZK brokers.
+ * Note that it's important to call this function only after we have set 
the ISR field in
+ * the PartitionChangeRecord.
  */
-void triggerLeaderEpochBumpIfNeeded(PartitionChangeRecord record) {
-if (record.leader() == NO_LEADER_CHANGE) {
-boolean bumpLeaderEpochOnIsrShrink = 
metadataVersion.isLeaderEpochBumpRequiredOnIsrShrink() || zkMigrationEnabled;
-
-if (!Replicas.contains(targetReplicas, partition.replicas)) {
-// Reassignment
-record.setLeader(partition.leader);
-} else if (bumpLeaderEpochOnIsrShrink

(kafka) branch KAFKA-16624 created (now 6feae817d25)

2024-04-25 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a change to branch KAFKA-16624
in repository https://gitbox.apache.org/repos/asf/kafka.git


  at 6feae817d25 MINOR: Rename RaftConfig to QuorumConfig (#15797)

No new revisions were added by this update.



(kafka) branch 3.7 updated: KAFKA-16003: Always create the /config/topics ZNode even for topics without configs (#15022)

2024-04-17 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.7
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.7 by this push:
 new f6bf85edbfb KAFKA-16003: Always create the /config/topics ZNode even 
for topics without configs (#15022)
f6bf85edbfb is described below

commit f6bf85edbfb0885ce192fae19b3b7866e687c886
Author: Mickael Maison 
AuthorDate: Thu Jan 25 15:46:24 2024 +0100

KAFKA-16003: Always create the /config/topics ZNode even for topics without 
configs (#15022)

Reviewers: Luke Chen 
Conflicts: Handle the fact that the ConfigType refactor is missing from 3.7.
---
 .../zk/migration/ZkTopicMigrationClient.scala  |  11 ++-
 .../kafka/zk/ZkMigrationIntegrationTest.scala  | 104 +++--
 2 files changed, 87 insertions(+), 28 deletions(-)

diff --git 
a/core/src/main/scala/kafka/zk/migration/ZkTopicMigrationClient.scala 
b/core/src/main/scala/kafka/zk/migration/ZkTopicMigrationClient.scala
index dd042ff96a7..6cd752d2d21 100644
--- a/core/src/main/scala/kafka/zk/migration/ZkTopicMigrationClient.scala
+++ b/core/src/main/scala/kafka/zk/migration/ZkTopicMigrationClient.scala
@@ -34,6 +34,7 @@ import org.apache.zookeeper.CreateMode
 import org.apache.zookeeper.KeeperException.Code
 
 import java.util
+import java.util.Properties
 import scala.collection.Seq
 import scala.collection.mutable.ArrayBuffer
 import scala.jdk.CollectionConverters._
@@ -122,9 +123,17 @@ class ZkTopicMigrationClient(zkClient: KafkaZkClient) 
extends TopicMigrationClie
 zkClient.defaultAcls(path),
 CreateMode.PERSISTENT)
 }
+val topicConfigZNode = {
+  val path = ConfigEntityZNode.path(ConfigType.Topic, topicName)
+  CreateRequest(
+path,
+ConfigEntityZNode.encode(new Properties()),
+zkClient.defaultAcls(path),
+CreateMode.PERSISTENT)
+}
 val createPartitionZNodeReqs = 
createTopicPartitionZNodesRequests(topicName, partitions, state)
 
-val requests = Seq(createTopicZNode) ++ createPartitionZNodeReqs
+val requests = Seq(createTopicZNode, topicConfigZNode) ++ 
createPartitionZNodeReqs
 val (migrationZkVersion, responses) = 
zkClient.retryMigrationRequestsUntilConnected(requests, state)
 val resultCodes = responses.map { response => response.path -> 
response.resultCode }.toMap
 if (resultCodes(TopicZNode.path(topicName)).equals(Code.NODEEXISTS)) {
diff --git 
a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala 
b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala
index 15bb563ad1c..75eab542031 100644
--- a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala
@@ -50,7 +50,7 @@ import org.apache.kafka.server.common.{ApiMessageAndVersion, 
MetadataVersion, Pr
 import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, 
assertNotEquals, assertNotNull, assertTrue, fail}
 import org.junit.jupiter.api.{Assumptions, Timeout}
 import org.junit.jupiter.api.extension.ExtendWith
-import org.slf4j.LoggerFactory
+import org.slf4j.{Logger, LoggerFactory}
 
 import java.util
 import java.util.concurrent.{CompletableFuture, ExecutionException, TimeUnit}
@@ -91,7 +91,7 @@ object ZkMigrationIntegrationTest {
 @Timeout(300)
 class ZkMigrationIntegrationTest {
 
-  val log = LoggerFactory.getLogger(classOf[ZkMigrationIntegrationTest])
+  val log: Logger = 
LoggerFactory.getLogger(classOf[ZkMigrationIntegrationTest])
 
   class MetadataDeltaVerifier {
 val metadataDelta = new MetadataDelta(MetadataImage.EMPTY)
@@ -184,7 +184,7 @@ class ZkMigrationIntegrationTest {
   // Enable migration configs and restart brokers
   log.info("Restart brokers in migration mode")
   
zkCluster.config().serverProperties().put(KafkaConfig.MigrationEnabledProp, 
"true")
-  
zkCluster.config().serverProperties().put(RaftConfig.QUORUM_VOTERS_CONFIG, 
kraftCluster.quorumVotersConfig());
+  
zkCluster.config().serverProperties().put(RaftConfig.QUORUM_VOTERS_CONFIG, 
kraftCluster.quorumVotersConfig())
   
zkCluster.config().serverProperties().put(KafkaConfig.ControllerListenerNamesProp,
 "CONTROLLER")
   
zkCluster.config().serverProperties().put(KafkaConfig.ListenerSecurityProtocolMapProp,
 "CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT,PLAINTEXT:PLAINTEXT")
   zkCluster.rollingBrokerRestart() // This would throw if authorizers 
weren't allowed
@@ -401,8 +401,8 @@ class ZkMigrationIntegrationTest {
 assertEquals(10, topicDescriptions("test-topic-3").partitions().size())
 topicDescriptions.foreach { case (topic, description) =>
   description.partitions().forEach(partition => {
-assertEquals(3, partition.replicas().size(), s"Unexpected number 
of replica

(kafka) branch 3.7 updated: MINOR: Fix typo in javadoc (#15031)

2024-04-17 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.7
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.7 by this push:
 new 2c3674dbf83 MINOR: Fix typo in javadoc (#15031)
2c3674dbf83 is described below

commit 2c3674dbf83cc85b6935b99398943ce72e6158b8
Author: jiangyuan 
AuthorDate: Tue Dec 19 00:35:13 2023 +0800

MINOR: Fix typo in javadoc (#15031)

Reviewers: Divij Vaidya 
---
 .../src/main/java/org/apache/kafka/message/MessageClassGenerator.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git 
a/generator/src/main/java/org/apache/kafka/message/MessageClassGenerator.java 
b/generator/src/main/java/org/apache/kafka/message/MessageClassGenerator.java
index 83117560a80..9b5f730b8ff 100644
--- 
a/generator/src/main/java/org/apache/kafka/message/MessageClassGenerator.java
+++ 
b/generator/src/main/java/org/apache/kafka/message/MessageClassGenerator.java
@@ -27,7 +27,7 @@ public interface MessageClassGenerator {
 String outputName(MessageSpec spec);
 
 /**
- * Generate the convertere, and then write it out.
+ * Generate the converter, and then write it out.
  *
  * @param spec  The message to generate a converter for.
  * @param writerThe writer to write out the state to.



(kafka) branch 3.7 updated: MINOR: Fixed typo in quickstart documentation (#15037)

2024-04-17 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.7
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.7 by this push:
 new 5b64c599119 MINOR: Fixed typo in quickstart documentation (#15037)
5b64c599119 is described below

commit 5b64c5991199db380ead1f6d149c4f48098f8082
Author: Lokesh Kumar 
AuthorDate: Mon Dec 18 13:51:36 2023 +0100

MINOR: Fixed typo in quickstart documentation (#15037)

Reviewers: Divij Vaidya 
---
 docs/quickstart.html | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/docs/quickstart.html b/docs/quickstart.html
index 94f278b2b86..e0c84279347 100644
--- a/docs/quickstart.html
+++ b/docs/quickstart.html
@@ -47,7 +47,7 @@ $ cd kafka_{{scalaVersion}}-{{fullDotVersion}}
 
 
 
-Apache Kafka can be started using ZooKeeper or KRaft. To get 
started with either configuration follow one the sections below but not both.
+Apache Kafka can be started using ZooKeeper or KRaft. To get 
started with either configuration follow one of the sections below but not both.
 
 
 



(kafka) branch trunk updated: KAFKA-16475: add test for TopicImageNodeTest (#15720)

2024-04-16 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 5193eb93237 KAFKA-16475: add test for TopicImageNodeTest (#15720)
5193eb93237 is described below

commit 5193eb93237ba9093ae444d73a1eaa2d6abcc9c1
Author: Johnny Hsu <44309740+johnnych...@users.noreply.github.com>
AuthorDate: Wed Apr 17 01:20:34 2024 +0800

KAFKA-16475: add test for TopicImageNodeTest (#15720)

Add a unit test for TopicImageNodeTest.

Co-authored-by: Johnny Hsu 
Reviewers: Colin P. McCabe 
---
 .../kafka/image/node/TopicImageNodeTest.java   | 60 ++
 1 file changed, 60 insertions(+)

diff --git 
a/metadata/src/test/java/org/apache/kafka/image/node/TopicImageNodeTest.java 
b/metadata/src/test/java/org/apache/kafka/image/node/TopicImageNodeTest.java
new file mode 100644
index 000..83855d94b8c
--- /dev/null
+++ b/metadata/src/test/java/org/apache/kafka/image/node/TopicImageNodeTest.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.image.node;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.image.TopicImage;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import java.util.Arrays;
+import java.util.Collections;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+
+
+@Timeout(value = 40)
+public class TopicImageNodeTest {
+private final static TopicImageNode NODE = new TopicImageNode(new 
TopicImage("topic-image-node-test-topic", Uuid.ZERO_UUID, 
Collections.emptyMap()));
+
+@Test
+public void testChildNames() {
+assertEquals(Arrays.asList("name", "id"), NODE.childNames());
+}
+
+@Test
+public void testNameChild() {
+MetadataNode child = NODE.child("name");
+assertNotNull(child);
+assertEquals(MetadataLeafNode.class, child.getClass());
+}
+
+@Test
+public void testIdChild() {
+MetadataNode child = NODE.child("id");
+assertNotNull(child);
+assertEquals(MetadataLeafNode.class, child.getClass());
+}
+
+@Test
+public void testUnknownChild() {
+assertNull(NODE.child("unknown"));
+}
+}



(kafka) branch 3.7 updated: KAFKA-16286; Notify listener of latest leader and epoch (#15397)

2024-04-15 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.7
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.7 by this push:
 new 736fd76a12b KAFKA-16286; Notify listener of latest leader and epoch 
(#15397)
736fd76a12b is described below

commit 736fd76a12b173f952794356791d0b06ed15c127
Author: José Armando García Sancio 
AuthorDate: Fri Feb 23 12:56:25 2024 -0800

KAFKA-16286; Notify listener of latest leader and epoch (#15397)

KRaft was only notifying listeners of the latest leader and epoch when the 
replica transition to a new state. This can result in the listener never 
getting notified if the registration happened after it had become a follower.

This problem doesn't exists for the active leader because the KRaft 
implementation attempts to notified the listener of the latest leader and epoch 
when the replica is the active leader.

This issue is fixed by notifying the listeners of the latest leader and 
epoch after processing the listener registration request.

Reviewers: Colin P. McCabe 
---
 .../org/apache/kafka/raft/KafkaRaftClient.java |  8 +++
 .../org/apache/kafka/raft/KafkaRaftClientTest.java | 60 ++
 .../apache/kafka/raft/RaftClientTestContext.java   |  6 +++
 3 files changed, 74 insertions(+)

diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java 
b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java
index e9fca1d6b2b..8790de92405 100644
--- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java
+++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java
@@ -2202,6 +2202,14 @@ public class KafkaRaftClient implements RaftClient 
{
 quorum.highWatermark().ifPresent(highWatermarkMetadata -> {
 updateListenersProgress(highWatermarkMetadata.offset);
 });
+
+// Notify the new listeners of the latest leader and epoch
+Optional> leaderState = quorum.maybeLeaderState();
+if (leaderState.isPresent()) {
+maybeFireLeaderChange(leaderState.get());
+} else {
+maybeFireLeaderChange();
+}
 }
 
 private void processRegistration(Registration registration) {
diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java 
b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java
index 51a5938220f..f18a0ff2e88 100644
--- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java
+++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java
@@ -2958,6 +2958,66 @@ public class KafkaRaftClientTest {
 assertEquals(OptionalInt.empty(), 
secondListener.currentClaimedEpoch());
 }
 
+@Test
+public void testHandleLeaderChangeFiresAfterUnattachedRegistration() 
throws Exception {
+// When registering a listener while the replica is unattached, it 
should get notified
+// with the current epoch
+// When transitioning to follower, expect another notification with 
the leader and epoch
+
+int localId = 0;
+int otherNodeId = 1;
+int epoch = 7;
+Set voters = Utils.mkSet(localId, otherNodeId);
+
+RaftClientTestContext context = new 
RaftClientTestContext.Builder(localId, voters)
+.withUnknownLeader(epoch)
+.build();
+
+// Register another listener and verify that it is notified of latest 
epoch
+RaftClientTestContext.MockListener secondListener = new 
RaftClientTestContext.MockListener(
+OptionalInt.of(localId)
+);
+context.client.register(secondListener);
+context.client.poll();
+
+// Expected leader change notification
+LeaderAndEpoch expectedLeaderAndEpoch = new 
LeaderAndEpoch(OptionalInt.empty(), epoch);
+assertEquals(expectedLeaderAndEpoch, 
secondListener.currentLeaderAndEpoch());
+
+// Transition to follower and the expect a leader changed notification
+context.deliverRequest(context.beginEpochRequest(epoch, otherNodeId));
+context.pollUntilResponse();
+
+// Expected leader change notification
+expectedLeaderAndEpoch = new 
LeaderAndEpoch(OptionalInt.of(otherNodeId), epoch);
+assertEquals(expectedLeaderAndEpoch, 
secondListener.currentLeaderAndEpoch());
+}
+
+@Test
+public void testHandleLeaderChangeFiresAfterFollowerRegistration() throws 
Exception {
+// When registering a listener while the replica is a follower, it 
should get notified with
+// the current leader and epoch
+
+int localId = 0;
+int otherNodeId = 1;
+int epoch = 7;
+Set voters = Utils.mkSet(localId, otherNodeId);
+
+RaftClientTestContext context = new 
RaftClientTestContext.Builder(localId, voters)
+.withElectedLeader(epoch, otherNodeId)
+

(kafka) branch 3.7 updated: KAFKA-16509: CurrentControllerId metric is unreliable in ZK mode (#15695)

2024-04-12 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.7
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.7 by this push:
 new 6baa6deea5b KAFKA-16509: CurrentControllerId metric is unreliable in 
ZK mode (#15695)
6baa6deea5b is described below

commit 6baa6deea5bdd6aafd83131c6ed9c5c968b09c29
Author: Colin Patrick McCabe 
AuthorDate: Thu Apr 11 09:34:27 2024 -0700

KAFKA-16509: CurrentControllerId metric is unreliable in ZK mode (#15695)

The CurrentControllerId metric added by KIP-1001 is unreliable in ZK
mode. Sometimes when there is no active ZK-based controller, it still
shows the previous controller ID. Instead, it should show -1 in that
situation.

This PR fixes that by using the controller ID from the
KafkaController.scala, which is obtained directly from the controller
znode. It also adds a new test, ControllerIdMetricTest.scala.

Reviewers: David Arthur 
---
 .../scala/kafka/controller/KafkaController.scala   |  2 +-
 core/src/main/scala/kafka/server/KafkaServer.scala | 24 +-
 .../unit/kafka/server/ControllerIdMetricTest.scala | 52 ++
 3 files changed, 67 insertions(+), 11 deletions(-)

diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala 
b/core/src/main/scala/kafka/controller/KafkaController.scala
index b989837f891..15a22acae6e 100644
--- a/core/src/main/scala/kafka/controller/KafkaController.scala
+++ b/core/src/main/scala/kafka/controller/KafkaController.scala
@@ -161,7 +161,7 @@ class KafkaController(val config: KafkaConfig,
   private val isrChangeNotificationHandler = new 
IsrChangeNotificationHandler(eventManager)
   private val logDirEventNotificationHandler = new 
LogDirEventNotificationHandler(eventManager)
 
-  @volatile private var activeControllerId = -1
+  @volatile var activeControllerId = -1
   @volatile private var offlinePartitionCount = 0
   @volatile private var preferredReplicaImbalanceCount = 0
   @volatile private var globalTopicCount = 0
diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala 
b/core/src/main/scala/kafka/server/KafkaServer.scala
index 203d2fffd51..874022d3534 100755
--- a/core/src/main/scala/kafka/server/KafkaServer.scala
+++ b/core/src/main/scala/kafka/server/KafkaServer.scala
@@ -195,7 +195,7 @@ class KafkaServer(
 
   override def logManager: LogManager = _logManager
 
-  def kafkaController: KafkaController = _kafkaController
+  @volatile def kafkaController: KafkaController = _kafkaController
 
   var lifecycleManager: BrokerLifecycleManager = _
   private var raftManager: KafkaRaftManager[ApiMessageAndVersion] = _
@@ -657,15 +657,19 @@ class KafkaServer(
   }
 
   def createCurrentControllerIdMetric(): Unit = {
-
KafkaYammerMetrics.defaultRegistry().newGauge(MetadataLoaderMetrics.CURRENT_CONTROLLER_ID,
 () => {
-  Option(metadataCache) match {
-case None => -1
-case Some(cache) => cache.getControllerId match {
-  case None => -1
-  case Some(id) => id.id
-}
-  }
-})
+
KafkaYammerMetrics.defaultRegistry().newGauge(MetadataLoaderMetrics.CURRENT_CONTROLLER_ID,
+  () => getCurrentControllerIdFromOldController())
+  }
+
+  /**
+   * Get the current controller ID from the old controller code.
+   * This is the most up-to-date controller ID we can get when in ZK mode.
+   */
+  def getCurrentControllerIdFromOldController(): Int = {
+Option(_kafkaController) match {
+  case None => -1
+  case Some(controller) => controller.activeControllerId
+}
   }
 
   def unregisterCurrentControllerIdMetric(): Unit = {
diff --git a/core/src/test/scala/unit/kafka/server/ControllerIdMetricTest.scala 
b/core/src/test/scala/unit/kafka/server/ControllerIdMetricTest.scala
new file mode 100755
index 000..8d0328f02a4
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/server/ControllerIdMetricTest.scala
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package kafka.server
+
+import kafka.integration.KafkaServerTestHarness
+import kafka.utils.

(kafka) branch 3.7 updated: KAFKA-16463 Delete metadata log on ZK broker startup (#15648)

2024-04-12 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.7
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.7 by this push:
 new 7bb8324121c KAFKA-16463 Delete metadata log on ZK broker startup 
(#15648)
7bb8324121c is described below

commit 7bb8324121c2036282a383b11a50abb2426cc939
Author: David Arthur 
AuthorDate: Fri Apr 12 13:21:30 2024 -0400

KAFKA-16463 Delete metadata log on ZK broker startup (#15648)

This patch changes the behavior of the migrating ZK broker to always delete 
the local metadata log
during startup. This deletion is done immediately before creating the 
RaftManager which will
re-create the log directory and let the broker re-replicate the log from 
the active controller.

This new behavior is only present for ZK brokers that having migrations 
enabled. KRaft brokers,
even those with migrations enabled, will not delete their local metadata 
log. KRaft controllers are
not impacted by this change.

The rationale for this change is to make it easier for operators to 
re-attempt a ZK to KRaft
migration after having reverted back to ZK mode. If an operator has 
reverted back to ZK mode, there
will be an invalid metadata log on the disk of each broker. In order to 
re-attempt the migration in
the future, this log needs to be deleted. This can be pretty burdensome to 
the operator for large
clusters, especially since the log deletion must be done while the broker 
is offline.

Reviewers: Colin P. McCabe , Igor Soarez 
, Chia-Ping Tsai 
Conflicts: Fix some conflicts in ZkMigrationIntegrationTest.scala
related to refactorings that happened in master but not 3.7, such as the
ProcessRoles class getting split out.
---
 core/src/main/scala/kafka/raft/RaftManager.scala   |  54 ++-
 core/src/main/scala/kafka/server/KafkaServer.scala |   7 +-
 .../server/NodeToControllerChannelManager.scala|   7 +-
 .../kafka/zk/ZkMigrationIntegrationTest.scala  |  83 +-
 .../scala/unit/kafka/raft/RaftManagerTest.scala| 179 +++--
 5 files changed, 270 insertions(+), 60 deletions(-)

diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala 
b/core/src/main/scala/kafka/raft/RaftManager.scala
index cd8a9739654..9c56dc35416 100644
--- a/core/src/main/scala/kafka/raft/RaftManager.scala
+++ b/core/src/main/scala/kafka/raft/RaftManager.scala
@@ -34,13 +34,14 @@ import org.apache.kafka.clients.{ApiVersions, 
ManualMetadataUpdater, NetworkClie
 import org.apache.kafka.common.KafkaException
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.Uuid
+import org.apache.kafka.common.internals.Topic
 import org.apache.kafka.common.metrics.Metrics
 import org.apache.kafka.common.network.{ChannelBuilders, ListenerName, 
NetworkReceive, Selectable, Selector}
 import org.apache.kafka.common.protocol.ApiMessage
 import org.apache.kafka.common.requests.RequestHeader
 import org.apache.kafka.common.security.JaasContext
 import org.apache.kafka.common.security.auth.SecurityProtocol
-import org.apache.kafka.common.utils.{LogContext, Time}
+import org.apache.kafka.common.utils.{LogContext, Time, Utils}
 import org.apache.kafka.raft.RaftConfig.{AddressSpec, InetAddressSpec, 
NON_ROUTABLE_ADDRESS, UnknownAddressSpec}
 import org.apache.kafka.raft.{FileBasedStateStore, KafkaNetworkChannel, 
KafkaRaftClient, LeaderAndEpoch, RaftClient, RaftConfig, RaftRequest, 
ReplicatedLog}
 import org.apache.kafka.server.common.serialization.RecordSerde
@@ -107,6 +108,51 @@ object KafkaRaftManager {
 
 lock
   }
+
+  /**
+   * Test if the configured metadata log dir is one of the data log dirs.
+   */
+  def hasDifferentLogDir(config: KafkaConfig): Boolean = {
+!config
+  .logDirs
+  .map(Paths.get(_).toAbsolutePath)
+  .contains(Paths.get(config.metadataLogDir).toAbsolutePath)
+  }
+
+  /**
+   * Obtain the file lock and delete the metadata log directory completely.
+   *
+   * This is only used by ZK brokers that are in pre-migration or hybrid mode 
of the ZK to KRaft migration.
+   * The rationale for deleting the metadata log in these cases is that it is 
safe to do on brokers and it
+   * it makes recovery from a failed migration much easier. See KAFKA-16463.
+   *
+   * @param config  The broker config
+   */
+  def maybeDeleteMetadataLogDir(config: KafkaConfig): Unit = {
+// These constraints are enforced in KafkaServer, but repeating them here 
to guard against future callers
+if (config.processRoles.nonEmpty) {
+  throw new RuntimeException("Not deleting metadata log dir since this 
node is in KRaft mode.")
+} else if (!config.migrationEnabled) {
+  throw new RuntimeException("Not deleting metadata log dir since 
migrations are not enabled.")
+} else {
+  val metadataDir = new File(config.metadataLogDir)
+

(kafka) branch trunk updated: KAFKA-16463 Delete metadata log on ZK broker startup (#15648)

2024-04-12 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new e02ffd852fa KAFKA-16463 Delete metadata log on ZK broker startup 
(#15648)
e02ffd852fa is described below

commit e02ffd852fae7c1d2681621be7eb888e3805e027
Author: David Arthur 
AuthorDate: Fri Apr 12 13:21:30 2024 -0400

KAFKA-16463 Delete metadata log on ZK broker startup (#15648)

This patch changes the behavior of the migrating ZK broker to always delete 
the local metadata log
during startup. This deletion is done immediately before creating the 
RaftManager which will
re-create the log directory and let the broker re-replicate the log from 
the active controller.

This new behavior is only present for ZK brokers that having migrations 
enabled. KRaft brokers,
even those with migrations enabled, will not delete their local metadata 
log. KRaft controllers are
not impacted by this change.

The rationale for this change is to make it easier for operators to 
re-attempt a ZK to KRaft
migration after having reverted back to ZK mode. If an operator has 
reverted back to ZK mode, there
will be an invalid metadata log on the disk of each broker. In order to 
re-attempt the migration in
the future, this log needs to be deleted. This can be pretty burdensome to 
the operator for large
clusters, especially since the log deletion must be done while the broker 
is offline.

Reviewers: Colin P. McCabe , Igor Soarez 
, Chia-Ping Tsai 
---
 core/src/main/scala/kafka/raft/RaftManager.scala   |  54 +++-
 core/src/main/scala/kafka/server/KafkaServer.scala |   7 +-
 .../server/NodeToControllerChannelManager.scala|   7 +-
 .../kafka/zk/ZkMigrationIntegrationTest.scala  |  83 +++-
 .../scala/unit/kafka/raft/RaftManagerTest.scala| 144 +++--
 5 files changed, 277 insertions(+), 18 deletions(-)

diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala 
b/core/src/main/scala/kafka/raft/RaftManager.scala
index a9e64fb967b..a16e528d797 100644
--- a/core/src/main/scala/kafka/raft/RaftManager.scala
+++ b/core/src/main/scala/kafka/raft/RaftManager.scala
@@ -32,13 +32,14 @@ import org.apache.kafka.clients.{ApiVersions, 
ManualMetadataUpdater, NetworkClie
 import org.apache.kafka.common.KafkaException
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.Uuid
+import org.apache.kafka.common.internals.Topic
 import org.apache.kafka.common.metrics.Metrics
 import org.apache.kafka.common.network.{ChannelBuilders, ListenerName, 
NetworkReceive, Selectable, Selector}
 import org.apache.kafka.common.protocol.ApiMessage
 import org.apache.kafka.common.requests.RequestHeader
 import org.apache.kafka.common.security.JaasContext
 import org.apache.kafka.common.security.auth.SecurityProtocol
-import org.apache.kafka.common.utils.{LogContext, Time}
+import org.apache.kafka.common.utils.{LogContext, Time, Utils}
 import org.apache.kafka.raft.RaftConfig.{AddressSpec, InetAddressSpec, 
NON_ROUTABLE_ADDRESS, UnknownAddressSpec}
 import org.apache.kafka.raft.{FileBasedStateStore, KafkaNetworkChannel, 
KafkaRaftClient, KafkaRaftClientDriver, LeaderAndEpoch, RaftClient, RaftConfig, 
ReplicatedLog}
 import org.apache.kafka.server.ProcessRole
@@ -69,6 +70,51 @@ object KafkaRaftManager {
 
 lock
   }
+
+  /**
+   * Test if the configured metadata log dir is one of the data log dirs.
+   */
+  def hasDifferentLogDir(config: KafkaConfig): Boolean = {
+!config
+  .logDirs
+  .map(Paths.get(_).toAbsolutePath)
+  .contains(Paths.get(config.metadataLogDir).toAbsolutePath)
+  }
+
+  /**
+   * Obtain the file lock and delete the metadata log directory completely.
+   *
+   * This is only used by ZK brokers that are in pre-migration or hybrid mode 
of the ZK to KRaft migration.
+   * The rationale for deleting the metadata log in these cases is that it is 
safe to do on brokers and it
+   * it makes recovery from a failed migration much easier. See KAFKA-16463.
+   *
+   * @param config  The broker config
+   */
+  def maybeDeleteMetadataLogDir(config: KafkaConfig): Unit = {
+// These constraints are enforced in KafkaServer, but repeating them here 
to guard against future callers
+if (config.processRoles.nonEmpty) {
+  throw new RuntimeException("Not deleting metadata log dir since this 
node is in KRaft mode.")
+} else if (!config.migrationEnabled) {
+  throw new RuntimeException("Not deleting metadata log dir since 
migrations are not enabled.")
+} else {
+  val metadataDir = new File(config.metadataLogDir)
+  val logDirName = 
UnifiedLog.logDirName(Topic.CLUSTER_METADATA_TOPIC_PARTITION)
+  val metadataPartitionDir = 
KafkaRaftManager.createLogDirectory(metadataDir, logDirName)
+  

(kafka) branch trunk updated: KAFKA-16509: CurrentControllerId metric is unreliable in ZK mode (#15695)

2024-04-11 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new b67a3fa79da KAFKA-16509: CurrentControllerId metric is unreliable in 
ZK mode (#15695)
b67a3fa79da is described below

commit b67a3fa79dab8bb253be353fb35d9e9f8536d750
Author: Colin Patrick McCabe 
AuthorDate: Thu Apr 11 09:34:27 2024 -0700

KAFKA-16509: CurrentControllerId metric is unreliable in ZK mode (#15695)

The CurrentControllerId metric added by KIP-1001 is unreliable in ZK
mode. Sometimes when there is no active ZK-based controller, it still
shows the previous controller ID. Instead, it should show -1 in that
situation.

This PR fixes that by using the controller ID from the
KafkaController.scala, which is obtained directly from the controller
znode. It also adds a new test, ControllerIdMetricTest.scala.

Reviewers: David Arthur 
---
 .../scala/kafka/controller/KafkaController.scala   |  2 +-
 core/src/main/scala/kafka/server/KafkaServer.scala | 24 +-
 .../unit/kafka/server/ControllerIdMetricTest.scala | 52 ++
 3 files changed, 67 insertions(+), 11 deletions(-)

diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala 
b/core/src/main/scala/kafka/controller/KafkaController.scala
index 5b893c8e851..62887395e32 100644
--- a/core/src/main/scala/kafka/controller/KafkaController.scala
+++ b/core/src/main/scala/kafka/controller/KafkaController.scala
@@ -161,7 +161,7 @@ class KafkaController(val config: KafkaConfig,
   private val isrChangeNotificationHandler = new 
IsrChangeNotificationHandler(eventManager)
   private val logDirEventNotificationHandler = new 
LogDirEventNotificationHandler(eventManager)
 
-  @volatile private var activeControllerId = -1
+  @volatile var activeControllerId = -1
   @volatile private var offlinePartitionCount = 0
   @volatile private var preferredReplicaImbalanceCount = 0
   @volatile private var globalTopicCount = 0
diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala 
b/core/src/main/scala/kafka/server/KafkaServer.scala
index 369bae7e4cb..b8da5f964de 100755
--- a/core/src/main/scala/kafka/server/KafkaServer.scala
+++ b/core/src/main/scala/kafka/server/KafkaServer.scala
@@ -197,7 +197,7 @@ class KafkaServer(
 
   override def logManager: LogManager = _logManager
 
-  def kafkaController: KafkaController = _kafkaController
+  @volatile def kafkaController: KafkaController = _kafkaController
 
   var lifecycleManager: BrokerLifecycleManager = _
   private var raftManager: KafkaRaftManager[ApiMessageAndVersion] = _
@@ -657,15 +657,19 @@ class KafkaServer(
   }
 
   private def createCurrentControllerIdMetric(): Unit = {
-
KafkaYammerMetrics.defaultRegistry().newGauge(MetadataLoaderMetrics.CURRENT_CONTROLLER_ID,
 () => {
-  Option(metadataCache) match {
-case None => -1
-case Some(cache) => cache.getControllerId match {
-  case None => -1
-  case Some(id) => id.id
-}
-  }
-})
+
KafkaYammerMetrics.defaultRegistry().newGauge(MetadataLoaderMetrics.CURRENT_CONTROLLER_ID,
+  () => getCurrentControllerIdFromOldController())
+  }
+
+  /**
+   * Get the current controller ID from the old controller code.
+   * This is the most up-to-date controller ID we can get when in ZK mode.
+   */
+  def getCurrentControllerIdFromOldController(): Int = {
+Option(_kafkaController) match {
+  case None => -1
+  case Some(controller) => controller.activeControllerId
+}
   }
 
   private def unregisterCurrentControllerIdMetric(): Unit = {
diff --git a/core/src/test/scala/unit/kafka/server/ControllerIdMetricTest.scala 
b/core/src/test/scala/unit/kafka/server/ControllerIdMetricTest.scala
new file mode 100755
index 000..8d0328f02a4
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/server/ControllerIdMetricTest.scala
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package kafka.server
+
+import kafka.integration.KafkaServerTestHarness
+import kafk

(kafka) branch 3.6 updated: KAFKA-16428: Fix bug where config change notification znode may not get created during migration (#15608)

2024-03-27 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.6
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.6 by this push:
 new 7303b6063bc KAFKA-16428: Fix bug where config change notification 
znode may not get created during migration (#15608)
7303b6063bc is described below

commit 7303b6063bc16a4bc8a75083a55460be08c5c9c0
Author: Colin Patrick McCabe 
AuthorDate: Wed Mar 27 12:26:36 2024 -0700

KAFKA-16428: Fix bug where config change notification znode may not get 
created during migration (#15608)

Reviewers: Chia-Ping Tsai , Luke Chen 

---
 core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git 
a/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala 
b/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala
index 844c1aabc4c..fbf6d7c3434 100644
--- a/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala
+++ b/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala
@@ -244,7 +244,7 @@ class ZkConfigMigrationClient(
 state
   } else if (responses.head.resultCode.equals(Code.OK)) {
 // Write the notification znode if our update was successful
-zkClient.createConfigChangeNotification(s"$configType/$configName")
+
zkClient.createConfigChangeNotification(s"${configType.get}/$configName")
 state.withMigrationZkVersion(migrationZkVersion)
   } else {
 throw KeeperException.create(responses.head.resultCode, path)



(kafka) branch 3.7 updated: KAFKA-16428: Fix bug where config change notification znode may not get created during migration (#15608)

2024-03-27 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.7
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.7 by this push:
 new 313f3f22b38 KAFKA-16428: Fix bug where config change notification 
znode may not get created during migration (#15608)
313f3f22b38 is described below

commit 313f3f22b38edafaf87f1e68b16b22fd8c41808c
Author: Colin Patrick McCabe 
AuthorDate: Wed Mar 27 12:26:36 2024 -0700

KAFKA-16428: Fix bug where config change notification znode may not get 
created during migration (#15608)

Reviewers: Chia-Ping Tsai , Luke Chen 

---
 core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git 
a/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala 
b/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala
index 844c1aabc4c..fbf6d7c3434 100644
--- a/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala
+++ b/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala
@@ -244,7 +244,7 @@ class ZkConfigMigrationClient(
 state
   } else if (responses.head.resultCode.equals(Code.OK)) {
 // Write the notification znode if our update was successful
-zkClient.createConfigChangeNotification(s"$configType/$configName")
+
zkClient.createConfigChangeNotification(s"${configType.get}/$configName")
 state.withMigrationZkVersion(migrationZkVersion)
   } else {
 throw KeeperException.create(responses.head.resultCode, path)



(kafka) branch trunk updated: KAFKA-16428: Fix bug where config change notification znode may not get created during migration (#15608)

2024-03-27 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new f40c06690bd KAFKA-16428: Fix bug where config change notification 
znode may not get created during migration (#15608)
f40c06690bd is described below

commit f40c06690bd3d982ed704411b952e5f9645d5f76
Author: Colin Patrick McCabe 
AuthorDate: Wed Mar 27 12:26:36 2024 -0700

KAFKA-16428: Fix bug where config change notification znode may not get 
created during migration (#15608)

Reviewers: Chia-Ping Tsai , Luke Chen 

---
 core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git 
a/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala 
b/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala
index fda6e93c249..a1f6e1a112f 100644
--- a/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala
+++ b/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala
@@ -244,7 +244,7 @@ class ZkConfigMigrationClient(
 state
   } else if (responses.head.resultCode.equals(Code.OK)) {
 // Write the notification znode if our update was successful
-zkClient.createConfigChangeNotification(s"$configType/$configName")
+
zkClient.createConfigChangeNotification(s"${configType.get}/$configName")
 state.withMigrationZkVersion(migrationZkVersion)
   } else {
 throw KeeperException.create(responses.head.resultCode, path)



(kafka) branch 3.6 updated: KAFKA-16411: Correctly migrate default client quota entities (#15584)

2024-03-27 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.6
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.6 by this push:
 new fbff947d259 KAFKA-16411: Correctly migrate default client quota 
entities (#15584)
fbff947d259 is described below

commit fbff947d25956100b1189fa4ee4ca64ea775df11
Author: Colin P. McCabe 
AuthorDate: Wed Mar 27 09:38:11 2024 -0700

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/ 
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> 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 "" 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,  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  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 , Igor Soarez 


Conflicts: Do not backport the changes to create 
ZooKeeperInternals.DEFAULT_STRING to this branch,
to make the cherry-pick smaller.
---
 .../main/scala/kafka/zk/ZkMigrationClient.scala|  4 -
 .../zk/migration/ZkConfigMigrationClient.scala | 89 --
 .../kafka/server/KRaftClusterTest.scala| 63 +++
 .../kafka/zk/ZkMigrationIntegrationTest.scala  | 45 ++-
 .../zk/migration/ZkConfigMigrationClientTest.scala | 22 +++---
 5 files changed, 167 insertions(+), 56 deletions(-)

diff --git a/core/src/main/scala/kafka/zk/ZkMigrationClient.scala 
b/core/src/main/scala/kafka/zk/ZkMigrationClient.scala
index 76e0b47aee8..a11a84c017b 100644
--- a/core/src/main/scala/kafka/zk/ZkMigrationClient.scala
+++ b/core/src/main/scala/kafka/zk/ZkMigrationClient.scala
@@ -27,7 +27,6 @@ import org.apache.kafka.common.errors.ControllerMovedException
 import org.apache.kafka.common.metadata._
 import org.apache.kafka.common.resource.ResourcePattern
 import org.apache.kafka.common.security.scram.ScramCredential
-import org.apache.kafka.common.utils.Sanitizer
 import org.apache.kafka.common.{TopicIdPartition, Uuid}
 import org.apache.kafka.metadata.DelegationTokenData
 import org.apache.kafka.metadata.PartitionRegistration
@@ -226,9 +225,6 @@ class ZkMigrationClient(
 entityDataList: util.List[ClientQuotaRecord.EntityData],
 quotas: util.Map[String, lang.Double]
   ): Unit = {
-entityDataList.forEach(entityData => {
-  
entityData.setEntityName(Sanitizer.desanitize(entityData.entityName()))
-})
 val batch = new util.ArrayList[

(kafka) branch 3.7 updated: KAFKA-16411: Correctly migrate default client quota entities (#15584)

2024-03-27 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.7
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.7 by this push:
 new fedf8cfa7a2 KAFKA-16411: Correctly migrate default client quota 
entities (#15584)
fedf8cfa7a2 is described below

commit fedf8cfa7a2cc11dfb0c7c35e6eb9a9935a1e146
Author: Colin P. McCabe 
AuthorDate: Wed Mar 27 09:38:11 2024 -0700

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/ 
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> 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 "" 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,  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  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 , Igor Soarez 


Conflicts: Do not backport the changes to create 
ZooKeeperInternals.DEFAULT_STRING to this branch,
to make the cherry-pick smaller.
---
 .../main/scala/kafka/zk/ZkMigrationClient.scala|  4 -
 .../zk/migration/ZkConfigMigrationClient.scala | 89 --
 .../kafka/server/KRaftClusterTest.scala| 63 +++
 .../kafka/zk/ZkMigrationIntegrationTest.scala  | 45 ++-
 .../zk/migration/ZkConfigMigrationClientTest.scala | 22 +++---
 5 files changed, 167 insertions(+), 56 deletions(-)

diff --git a/core/src/main/scala/kafka/zk/ZkMigrationClient.scala 
b/core/src/main/scala/kafka/zk/ZkMigrationClient.scala
index 76e0b47aee8..a11a84c017b 100644
--- a/core/src/main/scala/kafka/zk/ZkMigrationClient.scala
+++ b/core/src/main/scala/kafka/zk/ZkMigrationClient.scala
@@ -27,7 +27,6 @@ import org.apache.kafka.common.errors.ControllerMovedException
 import org.apache.kafka.common.metadata._
 import org.apache.kafka.common.resource.ResourcePattern
 import org.apache.kafka.common.security.scram.ScramCredential
-import org.apache.kafka.common.utils.Sanitizer
 import org.apache.kafka.common.{TopicIdPartition, Uuid}
 import org.apache.kafka.metadata.DelegationTokenData
 import org.apache.kafka.metadata.PartitionRegistration
@@ -226,9 +225,6 @@ class ZkMigrationClient(
 entityDataList: util.List[ClientQuotaRecord.EntityData],
 quotas: util.Map[String, lang.Double]
   ): Unit = {
-entityDataList.forEach(entityData => {
-  
entityData.setEntityName(Sanitizer.desanitize(entityData.entityName()))
-})
 val batch = new util.ArrayList[

(kafka) branch trunk updated: KAFKA-16411: Correctly migrate default client quota entities (#15584)

2024-03-26 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 8d914b543d5 KAFKA-16411: Correctly migrate default client quota 
entities (#15584)
8d914b543d5 is described below

commit 8d914b543d5d23031fe178d424f45789eaa8d1fc
Author: Colin Patrick McCabe 
AuthorDate: Tue Mar 26 16:49:38 2024 -0700

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/ 
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> 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 "" 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,  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  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 , Igor Soarez 

---
 .../src/main/scala/kafka/admin/ConfigCommand.scala | 12 +--
 .../scala/kafka/server/ClientQuotaManager.scala| 13 ++-
 .../main/scala/kafka/server/ConfigHandler.scala|  8 +-
 .../scala/kafka/server/DynamicBrokerConfig.scala   |  4 +-
 .../main/scala/kafka/server/DynamicConfig.scala|  4 +-
 .../main/scala/kafka/server/ZkAdminManager.scala   |  8 +-
 .../metadata/ClientQuotaMetadataManager.scala  | 14 +--
 .../server/metadata/DynamicConfigPublisher.scala   |  4 +-
 .../kafka/server/metadata/ZkConfigRepository.scala |  4 +-
 core/src/main/scala/kafka/zk/AdminZkClient.scala   |  8 +-
 .../main/scala/kafka/zk/ZkMigrationClient.scala|  4 -
 .../zk/migration/ZkConfigMigrationClient.scala | 99 +++---
 .../kafka/admin/ConfigCommandIntegrationTest.scala |  4 +-
 .../kafka/server/KRaftClusterTest.scala| 65 +-
 .../kafka/zk/ZkMigrationIntegrationTest.scala  | 46 +-
 .../scala/unit/kafka/admin/ConfigCommandTest.scala | 12 +--
 .../unit/kafka/server/ClientQuotaManagerTest.scala | 42 -
 .../kafka/server/ClientQuotasRequestTest.scala |  5 +-
 .../kafka/server/DynamicConfigChangeTest.scala |  4 +-
 .../zk/migration/ZkConfigMigrationClientTest.scala | 22 ++---
 ...nfigEntityName.java => ZooKeeperInternals.java} | 10 ++-
 21 files changed, 251 insertions(+), 141 deletions(-)

diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala 
b/core/src/main/scala/kafka/admin/ConfigCommand.scala
index bcbeedc468e..d03d7dfb9b5 100644
--- a/core/src/main/scala/kafka/admin/ConfigCommand.scala
+++ b/core/src/main/scala/kafk

(kafka) branch KAFKA-16411 created (now bf1900e9971)

2024-03-25 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a change to branch KAFKA-16411
in repository https://gitbox.apache.org/repos/asf/kafka.git


  at bf1900e9971 Improve tests

This branch includes the following new commits:

 new cbe5ada027c Rename 'mangling' to 'sanitization' to be more consistent
 new bf1900e9971 Improve tests

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.




(kafka) 02/02: Improve tests

2024-03-25 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch KAFKA-16411
in repository https://gitbox.apache.org/repos/asf/kafka.git

commit bf1900e9971cddd6c08e38baf40c9ae04dfc84cc
Author: Colin P. McCabe 
AuthorDate: Mon Mar 25 17:01:37 2024 -0700

Improve tests
---
 .../kafka/zk/ZkMigrationIntegrationTest.scala  | 22 +-
 1 file changed, 13 insertions(+), 9 deletions(-)

diff --git 
a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala 
b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala
index 16d68b3ae38..a3e5ecc0814 100644
--- a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala
@@ -40,7 +40,7 @@ import org.apache.kafka.common.resource.ResourcePattern
 import org.apache.kafka.common.resource.ResourceType.TOPIC
 import org.apache.kafka.common.security.auth.KafkaPrincipal
 import org.apache.kafka.common.security.scram.internals.ScramCredentialUtils
-import org.apache.kafka.common.utils.SecurityUtils
+import org.apache.kafka.common.utils.{Sanitizer, SecurityUtils}
 import org.apache.kafka.image.{MetadataDelta, MetadataImage, 
MetadataProvenance}
 import org.apache.kafka.metadata.authorizer.StandardAcl
 import org.apache.kafka.metadata.migration.ZkMigrationLeadershipState
@@ -881,11 +881,14 @@ class ZkMigrationIntegrationTest {
   def alterClientQuotas(admin: Admin): AlterClientQuotasResult = {
 val quotas = new util.ArrayList[ClientQuotaAlteration]()
 quotas.add(new ClientQuotaAlteration(
-  new ClientQuotaEntity(Map("user" -> "user1").asJava),
+  new ClientQuotaEntity(Map("user" -> "user@1").asJava),
   List(new ClientQuotaAlteration.Op("consumer_byte_rate", 1000.0)).asJava))
 quotas.add(new ClientQuotaAlteration(
-  new ClientQuotaEntity(Map("user" -> "user1", "client-id" -> 
"clientA").asJava),
+  new ClientQuotaEntity(Map("user" -> "user@1", "client-id" -> 
"clientA").asJava),
   List(new ClientQuotaAlteration.Op("consumer_byte_rate", 800.0), new 
ClientQuotaAlteration.Op("producer_byte_rate", 100.0)).asJava))
+quotas.add(new ClientQuotaAlteration(
+  new ClientQuotaEntity(Collections.singletonMap("user", null)),
+  List(new ClientQuotaAlteration.Op("consumer_byte_rate", 900.0), new 
ClientQuotaAlteration.Op("producer_byte_rate", 100.0)).asJava))
 quotas.add(new ClientQuotaAlteration(
   new ClientQuotaEntity(Map("ip" -> "8.8.8.8").asJava),
   List(new ClientQuotaAlteration.Op("connection_creation_rate", 
10.0)).asJava))
@@ -903,7 +906,7 @@ class ZkMigrationIntegrationTest {
 val alterations = new util.ArrayList[UserScramCredentialAlteration]()
 alterations.add(new UserScramCredentialUpsertion("user1",
 new ScramCredentialInfo(ScramMechanism.SCRAM_SHA_256, 8191), 
"password1"))
-alterations.add(new UserScramCredentialUpsertion("user2",
+alterations.add(new UserScramCredentialUpsertion("user@2",
 new ScramCredentialInfo(ScramMechanism.SCRAM_SHA_256, 8192), 
"password2"))
 admin.alterUserScramCredentials(alterations)
   }
@@ -918,20 +921,21 @@ class ZkMigrationIntegrationTest {
 
   def verifyClientQuotas(zkClient: KafkaZkClient): Unit = {
 TestUtils.retry(1) {
-  assertEquals("1000", zkClient.getEntityConfigs(ConfigType.USER, 
"user1").getProperty("consumer_byte_rate"))
-  assertEquals("800", zkClient.getEntityConfigs("users/user1/clients", 
"clientA").getProperty("consumer_byte_rate"))
-  assertEquals("100", zkClient.getEntityConfigs("users/user1/clients", 
"clientA").getProperty("producer_byte_rate"))
+  assertEquals("1000", zkClient.getEntityConfigs(ConfigType.USER, 
Sanitizer.sanitize("user@1")).getProperty("consumer_byte_rate"))
+  assertEquals("900", zkClient.getEntityConfigs(ConfigType.USER, 
"").getProperty("consumer_byte_rate"))
+  assertEquals("800", zkClient.getEntityConfigs("users/" + 
Sanitizer.sanitize("user@1") + "/clients", 
"clientA").getProperty("consumer_byte_rate"))
+  assertEquals("100", zkClient.getEntityConfigs("users/" + 
Sanitizer.sanitize("user@1") + "/clients", 
"clientA").getProperty("producer_byte_rate"))
   assertEquals("10", zkClient.getEntityConfigs(ConfigType.IP, 
"8.8.8.8").getProperty("connection_creation_rate"))
 }

(kafka) 01/02: Rename 'mangling' to 'sanitization' to be more consistent

2024-03-25 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch KAFKA-16411
in repository https://gitbox.apache.org/repos/asf/kafka.git

commit cbe5ada027c1882b74ea55e998325f62be2f611b
Author: Colin P. McCabe 
AuthorDate: Mon Mar 25 12:23:08 2024 -0700

Rename 'mangling' to 'sanitization' to be more consistent
---
 .../kafka/zk/migration/ZkConfigMigrationClient.scala   | 18 +-
 1 file changed, 9 insertions(+), 9 deletions(-)

diff --git 
a/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala 
b/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala
index 04a58894ae7..fda6e93c249 100644
--- a/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala
+++ b/core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala
@@ -21,7 +21,7 @@ import kafka.server.{DynamicBrokerConfig, DynamicConfig, 
ZkAdminManager}
 import kafka.utils.Logging
 import kafka.zk.ZkMigrationClient.{logAndRethrow, wrapZkException}
 import kafka.zk._
-import 
kafka.zk.migration.ZkConfigMigrationClient.getMangledClientQuotaZNodeName
+import 
kafka.zk.migration.ZkConfigMigrationClient.getSanitizedClientQuotaZNodeName
 import kafka.zookeeper.{CreateRequest, DeleteRequest, SetDataRequest}
 import org.apache.kafka.clients.admin.ScramMechanism
 import org.apache.kafka.common.config.types.Password
@@ -81,8 +81,8 @@ class ZkConfigMigrationClient(
   // which have their names set to the empty string instead.
   result.setEntityName(null)
 } else {
-  // ZNode names are mangled before being stored in ZooKeeper.
-  // For example, @ is turned into %40. Undo the mangling here.
+  // ZNode names are sanitized before being stored in ZooKeeper.
+  // For example, @ is turned into %40. Undo the sanitization here.
   result.setEntityName(Sanitizer.desanitize(znodeName))
 }
 result
@@ -261,9 +261,9 @@ class ZkConfigMigrationClient(
 scram: util.Map[String, String],
 state: ZkMigrationLeadershipState
   ): ZkMigrationLeadershipState = wrapZkException {
-val user: Option[String] = getMangledClientQuotaZNodeName(entity, 
ClientQuotaEntity.USER)
-val client: Option[String] = getMangledClientQuotaZNodeName(entity, 
ClientQuotaEntity.CLIENT_ID)
-val ip: Option[String] = getMangledClientQuotaZNodeName(entity, 
ClientQuotaEntity.IP)
+val user: Option[String] = getSanitizedClientQuotaZNodeName(entity, 
ClientQuotaEntity.USER)
+val client: Option[String] = getSanitizedClientQuotaZNodeName(entity, 
ClientQuotaEntity.CLIENT_ID)
+val ip: Option[String] = getSanitizedClientQuotaZNodeName(entity, 
ClientQuotaEntity.IP)
 val props = new Properties()
 
 val (configType, path, configKeys) = if (user.isDefined && client.isEmpty) 
{
@@ -369,7 +369,7 @@ object ZkConfigMigrationClient {
* @param component   The component that we want a znode name for.
* @returnSome(znodeName) if there is a znode path; None 
otherwise.
*/
-  def getMangledClientQuotaZNodeName(
+  def getSanitizedClientQuotaZNodeName(
 entity: util.Map[String, String],
 component: String
   ): Option[String] = {
@@ -385,10 +385,10 @@ object ZkConfigMigrationClient {
 // "not present." This is an unfortunate API that should be revisited 
at some point.
 Some(ZooKeeperInternals.DEFAULT_STRING)
   } else {
-// We found a non-null value, and now we need to mangle it. For 
example, "c@@ldude" will
+// We found a non-null value, and now we need to sanitize it. For 
example, "c@@ldude" will
 // turn into c%40%40ldude, so that we can use it as a znode name in 
ZooKeeper.
 Some(Sanitizer.sanitize(rawValue))
   }
 }
   }
-}
\ No newline at end of file
+}



(kafka) branch cmccabe_2024_03_22_fix_closeables updated (bf79133bf02 -> ad3876dfdfe)

2024-03-22 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a change to branch cmccabe_2024_03_22_fix_closeables
in repository https://gitbox.apache.org/repos/asf/kafka.git


 discard bf79133bf02 MINOR: ensure LocalLogManagerTestEnv is closed in 
QuorumControllerTest
 add ad3876dfdfe MINOR: ensure LocalLogManagerTestEnv is closed in 
QuorumControllerTest

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (bf79133bf02)
\
 N -- N -- N   refs/heads/cmccabe_2024_03_22_fix_closeables 
(ad3876dfdfe)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

No new revisions were added by this update.

Summary of changes:
 .../java/org/apache/kafka/controller/QuorumControllerTest.java | 10 +-
 1 file changed, 5 insertions(+), 5 deletions(-)



(kafka) 01/01: MINOR: ensure LocalLogManagerTestEnv is closed in QuorumControllerTest

2024-03-22 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch cmccabe_2024_03_22_fix_closeables
in repository https://gitbox.apache.org/repos/asf/kafka.git

commit bf79133bf02c7e6425ac8ea61cf3f13a7d3e39ca
Author: Colin P. McCabe 
AuthorDate: Fri Mar 22 13:18:59 2024 -0700

MINOR: ensure LocalLogManagerTestEnv is closed in QuorumControllerTest
---
 .../kafka/controller/QuorumControllerTest.java | 37 ++
 1 file changed, 17 insertions(+), 20 deletions(-)

diff --git 
a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java 
b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java
index bcdb3ed7919..462cd726857 100644
--- 
a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java
+++ 
b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java
@@ -1540,15 +1540,14 @@ public class QuorumControllerTest {
 @Test
 public void testMigrationsEnabledForOldBootstrapMetadataVersion() throws 
Exception {
 try (
-LocalLogManagerTestEnv logEnv = new 
LocalLogManagerTestEnv.Builder(1).build()
+LocalLogManagerTestEnv logEnv = new 
LocalLogManagerTestEnv.Builder(1).build();
+QuorumControllerTestEnv controlEnv = new 
QuorumControllerTestEnv.Builder(logEnv).
+setControllerBuilderInitializer(controllerBuilder -> {
+controllerBuilder.setZkMigrationEnabled(true);
+}).
+
setBootstrapMetadata(BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_3_IV0, 
"test")).
+build();
 ) {
-QuorumControllerTestEnv.Builder controlEnvBuilder = new 
QuorumControllerTestEnv.Builder(logEnv).
-setControllerBuilderInitializer(controllerBuilder -> {
-controllerBuilder.setZkMigrationEnabled(true);
-}).
-
setBootstrapMetadata(BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_3_IV0, 
"test"));
-
-QuorumControllerTestEnv controlEnv = controlEnvBuilder.build();
 QuorumController active = controlEnv.activeController();
 assertEquals(ZkMigrationState.NONE, active.appendReadEvent("read 
migration state", OptionalLong.empty(),
 () -> active.featureControl().zkMigrationState()).get(30, 
TimeUnit.SECONDS));
@@ -1658,12 +1657,12 @@ public class QuorumControllerTest {
 @Test
 public void testFailoverDuringMigrationTransaction() throws Exception {
 try (
-LocalLogManagerTestEnv logEnv = new 
LocalLogManagerTestEnv.Builder(3).build()
-) {
-QuorumControllerTestEnv.Builder controlEnvBuilder = new 
QuorumControllerTestEnv.Builder(logEnv).
+LocalLogManagerTestEnv logEnv = new 
LocalLogManagerTestEnv.Builder(3).build();
+QuorumControllerTestEnv controlEnv = new 
QuorumControllerTestEnv.Builder(logEnv).
 setControllerBuilderInitializer(controllerBuilder -> 
controllerBuilder.setZkMigrationEnabled(true)).
-
setBootstrapMetadata(BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV1, 
"test"));
-QuorumControllerTestEnv controlEnv = controlEnvBuilder.build();
+
setBootstrapMetadata(BootstrapMetadata.fromVersion(MetadataVersion.IBP_3_6_IV1, 
"test")).
+build();
+) {
 QuorumController active = controlEnv.activeController(true);
 ZkRecordConsumer migrationConsumer = active.zkRecordConsumer();
 migrationConsumer.beginMigration().get(30, TimeUnit.SECONDS);
@@ -1703,18 +1702,17 @@ public class QuorumControllerTest {
 @EnumSource(value = MetadataVersion.class, names = {"IBP_3_4_IV0", 
"IBP_3_5_IV0", "IBP_3_6_IV0", "IBP_3_6_IV1"})
 public void testBrokerHeartbeatDuringMigration(MetadataVersion 
metadataVersion) throws Exception {
 try (
-LocalLogManagerTestEnv logEnv = new 
LocalLogManagerTestEnv.Builder(1).build()
-) {
-QuorumControllerTestEnv.Builder controlEnvBuilder = new 
QuorumControllerTestEnv.Builder(logEnv).
+LocalLogManagerTestEnv logEnv = new 
LocalLogManagerTestEnv.Builder(1).build();
+QuorumControllerTestEnv controlEnv = new 
QuorumControllerTestEnv.Builder(logEnv).
 setControllerBuilderInitializer(controllerBuilder ->
 controllerBuilder
 .setZkMigrationEnabled(true)
 
.setMaxIdleIntervalNs(OptionalLong.of(TimeUnit.MILLISECONDS.toNanos(100)))
 ).
-
setBootstrapMetadata(BootstrapMetadata.fromVersion(metadataVersion, "test"));
-QuorumControllerTestEnv controlEnv = controlEnvBuilder.build();
+
setBootstrapMetadata(BootstrapMetadata

(kafka) branch cmccabe_2024_03_22_fix_closeables created (now bf79133bf02)

2024-03-22 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a change to branch cmccabe_2024_03_22_fix_closeables
in repository https://gitbox.apache.org/repos/asf/kafka.git


  at bf79133bf02 MINOR: ensure LocalLogManagerTestEnv is closed in 
QuorumControllerTest

This branch includes the following new commits:

 new bf79133bf02 MINOR: ensure LocalLogManagerTestEnv is closed in 
QuorumControllerTest

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.




(kafka) branch 3.6 updated: KAFKA-16180: Fix UMR and LAIR handling during ZK migration (#15293)

2024-03-13 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.6
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.6 by this push:
 new 4cec48f86ec KAFKA-16180: Fix UMR and LAIR handling during ZK migration 
(#15293)
4cec48f86ec is described below

commit 4cec48f86ec95848b2e3893f04dc90d1d8415a47
Author: Colin Patrick McCabe 
AuthorDate: Fri Feb 2 15:49:10 2024 -0800

KAFKA-16180: Fix UMR and LAIR handling during ZK migration (#15293)

While migrating from ZK mode to KRaft mode, the broker passes through a 
"hybrid" phase, in which it
receives LeaderAndIsrRequest and UpdateMetadataRequest RPCs from the KRaft 
controller. For the most
part, these RPCs can be handled just like their traditional equivalents 
from a ZK-based controller.
However, there is one thing that is different: the way topic deletions are 
handled.

In ZK mode, there is a "deleting" state which topics enter prior to being 
completely removed.
Partitions stay in this state until they are removed from the disks of all 
replicas. And partitions
associated with these deleting topics show up in the UMR and LAIR as having 
a leader of -2 (which
is not a valid broker ID, of course, because it's negative). When brokers 
receive these RPCs, they
know to remove the associated partitions from their metadata caches, and 
disks. When a full UMR or
ISR is sent, deleting partitions are included as well.

In hybrid mode, in contrast, there is no "deleting" state. Topic deletion 
happens immediately. We
can do this because we know that we have topic IDs that are never reused. 
This means that we can
always tell the difference between a broker that had an old version of some 
topic, and a broker
that has a new version that was re-created with the same name. To make this 
work, when handling a
full UMR or LAIR, hybrid brokers must compare the full state that was sent 
over the wire to their
own local state, and adjust accordingly.

Prior to this PR, the code for handling those adjustments had several major 
flaws. The biggest flaw
is that it did not correctly handle the "re-creation" case where a topic 
named FOO appears in the
RPC, but with a different ID than the broker's local FOO. Another flaw is 
that a problem with a
single partition would prevent handling the whole request.

In ZkMetadataCache.scala, we handle full UMR requests from KRaft 
controllers by rewriting the UMR
so that it contains the implied deletions. I fixed this code so that 
deletions always appear at the
start of the list of topic states. This is important for the re-creation 
case since it means that a
single request can both delete the old FOO and add a new FOO to the cache. 
Also, rather than
modifying the requesst in-place, as the previous code did, I build a whole 
new request with the
desired list of topic states. This is much safer because it avoids 
unforseen interactions with
other parts of the code that deal with requests (like request logging). 
While this new copy may
sound expensive, it should actually not be. We are doing a "shallow copy" 
which references the
previous list topic state entries.

I also reworked ZkMetadataCache.updateMetadata so that if a partition is 
re-created, it does not
appear in the returned set of deleted TopicPartitions. Since this set is 
used only by the group
manager, this seemed appropriate. (If I was in the consumer group for the 
previous iteration of
FOO, I should still be in the consumer group for the new iteration.)

On the ReplicaManager.scala side, we handle full LAIR requests by treating 
anything which does not
appear in them as a "stray replica." (But we do not rewrite the request 
objects as we do with UMR.)
I moved the logic for finding stray replicas from ReplicaManager into 
LogManager. It makes more
sense there, since the information about what is on-disk is managed in 
LogManager. Also, the stray
replica detection logic for KRaft mode is there, so it makes sense to put 
the stray replica
detection logic for hybrid mode there as well.

Since the stray replica detection is now in LogManager, I moved the unit 
tests there as well.
Previously some of those tests had been in BrokerMetadataPublisherTest for 
historical reasons.

The main advantage of the new LAIR logic is that it takes topic ID into 
account. A replica can be a
stray even if the LAIR contains a topic of the given name, but a different 
ID. I also moved the
stray replica handling earlier in the becomeLeaderOrFollower function, so 
that we could correctly
handle the "delete and re-create FOO" case.

Reviewers: David Arthur 

Conflicts: For this cherry-pic

(kafka) branch 3.6 updated: HOTFIX for KAFKA-16126 test in 3.6

2024-03-12 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.6
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.6 by this push:
 new 05a36eb10bf HOTFIX for KAFKA-16126 test in 3.6
05a36eb10bf is described below

commit 05a36eb10bf23a7515066926449b057e970d0428
Author: Colin P. McCabe 
AuthorDate: Tue Mar 12 16:59:41 2024 -0700

HOTFIX for KAFKA-16126 test in 3.6
---
 core/src/test/java/kafka/testkit/KafkaClusterTestKit.java | 4 +---
 core/src/test/java/kafka/testkit/TestKitNodes.java| 4 
 2 files changed, 5 insertions(+), 3 deletions(-)

diff --git a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java 
b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java
index e09ee49402f..f6e4566b479 100644
--- a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java
+++ b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java
@@ -221,8 +221,6 @@ public class KafkaClusterTestKit implements AutoCloseable {
 
ThreadUtils.createThreadFactory("kafka-cluster-test-kit-executor-%d", false));
 for (ControllerNode node : nodes.controllerNodes().values()) {
 setupNodeDirectories(baseDirectory, 
node.metadataDirectory(), Collections.emptyList());
-BootstrapMetadata bootstrapMetadata = BootstrapMetadata.
-fromVersion(nodes.bootstrapMetadataVersion(), 
"testkit");
 SharedServer sharedServer = new 
SharedServer(createNodeConfig(node),
 MetaProperties.apply(nodes.clusterId().toString(), 
node.id()),
 Time.SYSTEM,
@@ -234,7 +232,7 @@ public class KafkaClusterTestKit implements AutoCloseable {
 controller = new ControllerServer(
 sharedServer,
 KafkaRaftServer.configSchema(),
-bootstrapMetadata);
+nodes.bootstrapMetadata());
 } catch (Throwable e) {
 log.error("Error creating controller {}", node.id(), 
e);
 Utils.swallow(log, Level.WARN, 
"sharedServer.stopForController error", () -> sharedServer.stopForController());
diff --git a/core/src/test/java/kafka/testkit/TestKitNodes.java 
b/core/src/test/java/kafka/testkit/TestKitNodes.java
index da4e10a4431..90d095f70c7 100644
--- a/core/src/test/java/kafka/testkit/TestKitNodes.java
+++ b/core/src/test/java/kafka/testkit/TestKitNodes.java
@@ -165,6 +165,10 @@ public class TestKitNodes {
 return bootstrapMetadata.metadataVersion();
 }
 
+public BootstrapMetadata bootstrapMetadata() {
+return bootstrapMetadata;
+}
+
 public Map controllerNodes() {
 return controllerNodes;
 }



(kafka) branch 3.6 updated: KAFKA-16171: Fix ZK migration controller race #15238

2024-03-12 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.6
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.6 by this push:
 new 5d3e691e47b KAFKA-16171: Fix ZK migration controller race #15238
5d3e691e47b is described below

commit 5d3e691e47ba35b1cac481bcf158890413f19e3c
Author: David Arthur 
AuthorDate: Fri Jan 19 13:38:37 2024 -0500

KAFKA-16171: Fix ZK migration controller race #15238

This patch causes the active KRaftMigrationDriver to reload the /migration 
ZK state after electing
itself as the leader in ZK. This closes a race condition where the previous 
active controller could
make an update to /migration after the new leader was elected. The update 
race was not actually a
problem regarding the data since both controllers would be syncing the same 
state from KRaft to ZK,
but the change to the znode causes the new controller to fail on the zk 
version check on
/migration.

This patch also fixes a as-yet-unseen bug where the active controllers 
failing to elect itself via
claimControllerLeadership would not retry.

Reviewers: Colin P. McCabe 
---
 .../kafka/zk/ZkMigrationFailoverTest.scala | 284 +
 .../metadata/migration/KRaftMigrationDriver.java   |  14 +-
 .../migration/ZkMigrationLeadershipState.java  |   9 +-
 .../migration/CapturingMigrationClient.java|  12 +-
 4 files changed, 312 insertions(+), 7 deletions(-)

diff --git 
a/core/src/test/scala/integration/kafka/zk/ZkMigrationFailoverTest.scala 
b/core/src/test/scala/integration/kafka/zk/ZkMigrationFailoverTest.scala
new file mode 100644
index 000..1b297f4fc9b
--- /dev/null
+++ b/core/src/test/scala/integration/kafka/zk/ZkMigrationFailoverTest.scala
@@ -0,0 +1,284 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.zk
+
+import kafka.utils.{Logging, PasswordEncoder, TestUtils}
+import org.apache.kafka.clients.ApiVersions
+import org.apache.kafka.common.{Node, Uuid}
+import org.apache.kafka.common.metadata.{FeatureLevelRecord, TopicRecord}
+import org.apache.kafka.common.utils.{Time, Utils}
+import org.apache.kafka.controller.QuorumFeatures
+import org.apache.kafka.controller.metrics.QuorumControllerMetrics
+import org.apache.kafka.image.loader.LogDeltaManifest
+import org.apache.kafka.image.publisher.MetadataPublisher
+import org.apache.kafka.image.{MetadataDelta, MetadataImage, 
MetadataProvenance}
+import org.apache.kafka.metadata.KafkaConfigSchema
+import org.apache.kafka.metadata.migration._
+import org.apache.kafka.raft.{LeaderAndEpoch, OffsetAndEpoch}
+import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion}
+import org.apache.kafka.server.fault.FaultHandler
+import org.apache.zookeeper.client.ZKClientConfig
+import org.junit.jupiter.api.Assertions.{assertTrue, fail}
+import org.junit.jupiter.api.Test
+
+import java.util
+import java.util.concurrent.{CompletableFuture, TimeUnit}
+import java.util.{Optional, OptionalInt}
+import scala.collection.mutable
+
+class ZkMigrationFailoverTest extends Logging {
+
+  class CapturingFaultHandler(nodeId: Int) extends FaultHandler {
+val faults = mutable.Buffer[Throwable]()
+var future: CompletableFuture[Throwable] = 
CompletableFuture.completedFuture(new RuntimeException())
+var waitingForMsg = ""
+
+override def handleFault(failureMessage: String, cause: Throwable): 
RuntimeException = {
+  error(s"Fault handled on node $nodeId", cause)
+  faults.append(cause)
+  if (!future.isDone && cause.getMessage.contains(waitingForMsg)) {
+future.complete(cause)
+  }
+  new RuntimeException(cause)
+}
+
+def checkAndClear(verifier: (Seq[Throwable]) => Unit): Unit = {
+  val faultsSoFar = faults.toSeq
+  try {
+verifier.apply(faultsSoFar)
+  } catch {
+case ae: AssertionError => fail(s"Assertion failed. Faults on $nodeId 
were: $faultsSoFar", ae)
+  }
+}
+
+def waitForError(message: String): CompletableFuture[Throwable] = {
+  future = n

(kafka) branch 3.6 updated: KAFKA-16216: Reduce batch size for initial metadata load during ZK migration

2024-03-09 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.6
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.6 by this push:
 new f734f32b85f KAFKA-16216: Reduce batch size for initial metadata load 
during ZK migration
f734f32b85f is described below

commit f734f32b85f3846bf6ca017483b31d99a2ed5b72
Author: David Arthur 
AuthorDate: Thu Feb 1 15:29:07 2024 -0800

KAFKA-16216: Reduce batch size for initial metadata load during ZK migration

During migration from ZK mode to KRaft mode, there is a step where the 
kcontrollers load all of the
data from ZK into the metadata log. Previously, we were using a batch size 
of 1000 for this, but
200 seems better. This PR also adds an internal configuration to control 
this batch size, for
testing purposes.

Reviewers: Colin P. McCabe 
---
 .../main/scala/kafka/server/ControllerServer.scala |  1 +
 core/src/main/scala/kafka/server/KafkaConfig.scala |  5 +
 .../metadata/migration/KRaftMigrationDriver.java   | 17 +++---
 .../migration/KRaftMigrationDriverTest.java| 26 +-
 4 files changed, 36 insertions(+), 13 deletions(-)

diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala 
b/core/src/main/scala/kafka/server/ControllerServer.scala
index 5ffc382cf8d..4ba7f10d2a0 100644
--- a/core/src/main/scala/kafka/server/ControllerServer.scala
+++ b/core/src/main/scala/kafka/server/ControllerServer.scala
@@ -287,6 +287,7 @@ class ControllerServer(
   .setQuorumFeatures(quorumFeatures)
   .setConfigSchema(configSchema)
   .setControllerMetrics(quorumControllerMetrics)
+  .setMinMigrationBatchSize(config.migrationMetadataMinBatchSize)
   .setTime(time)
   .build()
 migrationDriver.start()
diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala 
b/core/src/main/scala/kafka/server/KafkaConfig.scala
index 618faeee790..4a11c70c6a5 100755
--- a/core/src/main/scala/kafka/server/KafkaConfig.scala
+++ b/core/src/main/scala/kafka/server/KafkaConfig.scala
@@ -91,6 +91,7 @@ object Defaults {
   /** KRaft mode configs */
   val EmptyNodeId: Int = -1
   val ServerMaxStartupTimeMs = Long.MaxValue
+  val MigrationMetadataMinBatchSize: Int = 200
 
   /* Authorizer Configuration ***/
   val AuthorizerClassName = ""
@@ -398,6 +399,7 @@ object KafkaConfig {
 
   /** ZK to KRaft Migration configs */
   val MigrationEnabledProp = "zookeeper.metadata.migration.enable"
+  val MigrationMetadataMinBatchSizeProp = 
"zookeeper.metadata.migration.min.batch.size"
 
   /* Authorizer Configuration ***/
   val AuthorizerClassNameProp = "authorizer.class.name"
@@ -1236,6 +1238,8 @@ object KafkaConfig {
   .define(MetadataMaxIdleIntervalMsProp, INT, 
Defaults.MetadataMaxIdleIntervalMs, atLeast(0), LOW, 
MetadataMaxIdleIntervalMsDoc)
   .defineInternal(ServerMaxStartupTimeMsProp, LONG, 
Defaults.ServerMaxStartupTimeMs, atLeast(0), MEDIUM, ServerMaxStartupTimeMsDoc)
   .define(MigrationEnabledProp, BOOLEAN, false, HIGH, "Enable ZK to KRaft 
migration")
+  .defineInternal(MigrationMetadataMinBatchSizeProp, INT, 
Defaults.MigrationMetadataMinBatchSize, atLeast(1),
+MEDIUM, "Soft minimum batch size to use when migrating metadata from 
ZooKeeper to KRaft")
 
   /* Authorizer Configuration ***/
   .define(AuthorizerClassNameProp, STRING, Defaults.AuthorizerClassName, 
new ConfigDef.NonNullValidator(), LOW, AuthorizerClassNameDoc)
@@ -1735,6 +1739,7 @@ class KafkaConfig private(doLog: Boolean, val props: 
java.util.Map[_, _], dynami
   def usesSelfManagedQuorum: Boolean = processRoles.nonEmpty
 
   val migrationEnabled: Boolean = getBoolean(KafkaConfig.MigrationEnabledProp)
+  val migrationMetadataMinBatchSize: Int = 
getInt(KafkaConfig.MigrationMetadataMinBatchSizeProp)
 
   private def parseProcessRoles(): Set[ProcessRole] = {
 val roles = getList(KafkaConfig.ProcessRolesProp).asScala.map {
diff --git 
a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java
 
b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java
index 52a9d74b23f..172e5f2f601 100644
--- 
a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java
+++ 
b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java
@@ -90,8 +90,6 @@ public class KRaftMigrationDriver implements 
MetadataPublisher {
  */
 final static int METADATA_COMMIT_MAX_WAIT_MS = 300_000;
 
-final static int MIGRATION_MIN_BATCH_SIZE = 1_000;
-
 private final Time time;
 private final Logger log;
 private final int nodeId;
@@ -110,6 +108,7 @@ public class KRaftMigrationDriver implements 
MetadataPublisher {
  * MetadataPublish

(kafka) branch cmccabe_2024_02_26_fix deleted (was fc13cd04d7f)

2024-03-05 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a change to branch cmccabe_2024_02_26_fix
in repository https://gitbox.apache.org/repos/asf/kafka.git


 was fc13cd04d7f MINOR: remove test constructor for PartitionAssignment

The revisions that were on this branch are still contained in
other references; therefore, this change does not discard any commits
from the repository.



(kafka) branch trunk updated: MINOR: remove test constructor for PartitionAssignment (#15435)

2024-03-05 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 2c1943d836d MINOR: remove test constructor for PartitionAssignment 
(#15435)
2c1943d836d is described below

commit 2c1943d836d828755af382b9d7996ff092854fe2
Author: Colin Patrick McCabe 
AuthorDate: Tue Mar 5 12:02:19 2024 -0800

MINOR: remove test constructor for PartitionAssignment (#15435)

Remove the test constructor for PartitionAssignment and remove the TODO.
Also add KRaftClusterTest.testCreatePartitions to get more coverage for
createPartitions.

Reviewers: David Arthur , Chia-Ping Tsai 

---
 .../kafka/server/KRaftClusterTest.scala| 35 +-
 .../metadata/placement/PartitionAssignment.java|  6 
 .../controller/PartitionChangeBuilderTest.java |  6 ++--
 .../PartitionReassignmentReplicasTest.java | 22 +++---
 .../controller/ReplicationControlManagerTest.java  | 18 +--
 .../placement/PartitionAssignmentTest.java | 12 +---
 .../placement/StripedReplicaPlacerTest.java| 19 ++--
 .../metadata/placement/TopicAssignmentTest.java|  9 +++---
 8 files changed, 80 insertions(+), 47 deletions(-)

diff --git 
a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala 
b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala
index 130d0e5642e..3be1b400ab5 100644
--- a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala
+++ b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala
@@ -26,7 +26,7 @@ import org.apache.kafka.clients.admin._
 import org.apache.kafka.common.acl.{AclBinding, AclBindingFilter}
 import org.apache.kafka.common.config.{ConfigException, ConfigResource}
 import org.apache.kafka.common.config.ConfigResource.Type
-import org.apache.kafka.common.errors.{PolicyViolationException, 
UnsupportedVersionException}
+import 
org.apache.kafka.common.errors.{InvalidPartitionsException,PolicyViolationException,
 UnsupportedVersionException}
 import org.apache.kafka.common.message.DescribeClusterRequestData
 import org.apache.kafka.common.metadata.{ConfigRecord, FeatureLevelRecord}
 import org.apache.kafka.common.metrics.Metrics
@@ -792,6 +792,39 @@ class KRaftClusterTest {
 }
   }
 
+  @ParameterizedTest
+  @ValueSource(strings = Array("3.7-IV0", "3.7-IV2"))
+  def testCreatePartitions(metadataVersionString: String): Unit = {
+val cluster = new KafkaClusterTestKit.Builder(
+  new TestKitNodes.Builder().
+setNumBrokerNodes(4).
+
setBootstrapMetadataVersion(MetadataVersion.fromVersionString(metadataVersionString)).
+setNumControllerNodes(3).build()).
+  build()
+try {
+  cluster.format()
+  cluster.startup()
+  cluster.waitForReadyBrokers()
+  val admin = Admin.create(cluster.clientProperties())
+  try {
+val createResults = admin.createTopics(Arrays.asList(
+  new NewTopic("foo", 1, 3.toShort),
+  new NewTopic("bar", 2, 3.toShort))).values()
+createResults.get("foo").get()
+createResults.get("bar").get()
+val increaseResults = admin.createPartitions(Map(
+  "foo" -> NewPartitions.increaseTo(3),
+  "bar" -> NewPartitions.increaseTo(2)).asJava).values()
+increaseResults.get("foo").get()
+assertEquals(classOf[InvalidPartitionsException], assertThrows(
+  classOf[ExecutionException], () => 
increaseResults.get("bar").get()).getCause.getClass)
+  } finally {
+admin.close()
+  }
+} finally {
+  cluster.close()
+}
+  }
   private def clusterImage(
 cluster: KafkaClusterTestKit,
 brokerId: Int
diff --git 
a/metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java
 
b/metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java
index 177d5311afd..a7012d1505c 100644
--- 
a/metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java
+++ 
b/metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java
@@ -17,7 +17,6 @@
 
 package org.apache.kafka.metadata.placement;
 
-import org.apache.kafka.common.DirectoryId;
 import org.apache.kafka.common.Uuid;
 
 import java.util.ArrayList;
@@ -39,11 +38,6 @@ public class PartitionAssignment {
 private final List replicas;
 private final List directories;
 
-// TODO remove -- just here for testing
-public PartitionAssignment(List replicas) {
-this(replicas, brokerId -> DirectoryId.UNASSIGNED);
-}
-
 public PartitionAssignment(List replicas, DefaultDirProvider 
defaultDirProvider) {
 this.replicas = Collections.unmodifiableList(new 

(kafka) branch cmccabe_2024_02_26_fix updated (925c61a99ce -> fc13cd04d7f)

2024-03-05 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a change to branch cmccabe_2024_02_26_fix
in repository https://gitbox.apache.org/repos/asf/kafka.git


 discard 925c61a99ce MINOR: remove test constructor for PartitionAssignment
 add 027fad4b2a0 KAFKA-16277: AbstractStickyAssignor - Sort owned 
TopicPartitions by partition when reassigning (#15416)
 add ddfcc333f8d KAFKA-16226 Add test for concurrently updatingMetadata and 
fetching snapshot/cluster (#15385)
 add 1c9f360f4af KAFKA-15215: migrate StreamedJoinTest to Mockito (#15424)
 add 3075a3f144c MINOR: Add 3.7.0 to system tests (#15436)
 add 09793275208 KAFKA-16306: fix GroupCoordinatorService logger (#15433)
 add 5d6936a4992  KAFKA-16305: Avoid optimisation in handshakeUnwrap 
(#15434)
 add 53c41aca7ba KAFKA-16116: Rebalance Metrics for AsyncKafkaConsumer 
(#15339)
 add 52289c92be4 MINOR: Optimize EventAccumulator (#15430)
 add 1bb9a851744 MINOR: Remove the space between two words (#15439)
 add 55a6d30ccbe KAFKA-16154: Broker returns offset for 
LATEST_TIERED_TIMESTAMP (#15213)
 add 96c68096a26 KAFKA-15462: Add Group Type Filter for List Group to the 
Admin Client  (#15150)
 add f8eb4294d67 KAFKA-16191: Clean up of consumer client internal events 
(#15438)
 add d066b94c810 MINOR: Fix UpdatedImage and HighWatermarkUpdated events' 
logs (#15432)
 add c8843f06841 KAFKA-16167: Disable wakeups during autocommit on close 
(#15445)
 add 8e1516f88b8 KAFKA-16261: updateSubscription fails if already empty 
subscription (#15440)
 add 52a3fa07446 KAFKA-15878: KIP-768 - Extend support for opaque (i.e. 
non-JWT) tokens in SASL/OAUTHBEARER (#14818)
 add 21a5bbd84c3 MINOR: Upgrade jqwik to version 1.8.3 (#14365)
 add 44af72fd773 MINOR: fix link for 
ListTransactionsOptions#filterOnDuration (#15459)
 add 907e945c0b7 MINOR: fix SessionStore java doc (#15412)
 add 2c0cab39aed MINOR: Remove unnecessary easymock/powermock dependencies 
(#15460)
 add 7dbdc15c668 KAFKA-15625: Do not flush global state store at each 
commit (#15361)
 add 21a888c4ca0 MINOR: Updating comments to match the code (#15388)
 add aa0443eb607 KAFKA-16285: Make group metadata available when a new 
assignment is set (#15426)
 add c254b22a487 MINOR: simplify ensure topic exists condition (#15458)
 add 4f92a3f0afd KAFKA-14747: record discarded FK join subscription 
responses (#15395)
 add 99e511c706b KAFKA-16288, KAFKA-16289: Fix Values convertToDecimal 
exception and parseString corruption (#15399)
 add 47792770a28 KAFKA-16169: FencedException in commitAsync not propagated 
without callback (#15437)
 add eea369af947 KAFKA-14588 Log cleaner configuration move to 
CleanerConfig (#15387)
 add 2f401ff4c85 MINOR: parameterize group-id in 
GroupMetadataManagerTestContext (#15467)
 add fc13cd04d7f MINOR: remove test constructor for PartitionAssignment

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (925c61a99ce)
\
 N -- N -- N   refs/heads/cmccabe_2024_02_26_fix (fc13cd04d7f)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

No new revisions were added by this update.

Summary of changes:
 build.gradle   |  11 +-
 checkstyle/import-control-core.xml |   1 +
 checkstyle/import-control.xml  |   2 +
 checkstyle/suppressions.xml|   1 +
 .../java/org/apache/kafka/clients/admin/Admin.java |   2 +-
 .../kafka/clients/admin/ConsumerGroupListing.java  |  64 ++-
 .../kafka/clients/admin/KafkaAdminClient.java  |  20 +-
 .../clients/admin/ListConsumerGroupsOptions.java   |  25 +-
 .../consumer/internals/AbstractStickyAssignor.java |   1 +
 .../consumer/internals/AsyncKafkaConsumer.java | 230 +-
 .../consumer/internals/CommitRequestManager.java   |   2 +-
 .../internals/CoordinatorRequestManager.java   |   6 +-
 .../internals/HeartbeatRequestManager.java |  25 +-
 .../consumer/internals/MembershipManager.java  |   7 +-
 .../consumer/internals/MembershipManagerImpl.java  | 109 +++--
 .../internals/OffsetCommitCallbackInvoker.java |  10 -
 .../consumer/internals/OffsetsRequestManager.java  |   4 +-
 .../consumer/internals/RequestManagers.java|   7 +-
 .../clients/consumer/internals/RequestState.java   |   2 +-
 .../clients/consumer/internals/WakeupTrigger.java  |  

(kafka) 01/01: MINOR: remove test constructor for PartitionAssignment

2024-02-26 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch cmccabe_2024_02_26_fix
in repository https://gitbox.apache.org/repos/asf/kafka.git

commit 925c61a99ce51103af1ab3188997def5ad1f767b
Author: Colin P. McCabe 
AuthorDate: Mon Feb 26 13:24:16 2024 -0800

MINOR: remove test constructor for PartitionAssignment

Remove the test constructor for PartitionAssignment and remove the TODO.
---
 .../kafka/server/KRaftClusterTest.scala| 39 +-
 .../metadata/placement/PartitionAssignment.java|  6 
 .../controller/PartitionChangeBuilderTest.java |  6 ++--
 .../PartitionReassignmentReplicasTest.java | 22 ++--
 .../controller/ReplicationControlManagerTest.java  | 18 +-
 .../placement/PartitionAssignmentTest.java | 12 ---
 .../placement/StripedReplicaPlacerTest.java| 17 +-
 .../metadata/placement/TopicAssignmentTest.java|  9 ++---
 8 files changed, 83 insertions(+), 46 deletions(-)

diff --git 
a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala 
b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala
index 130d0e5642e..379a7dd24aa 100644
--- a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala
+++ b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala
@@ -26,7 +26,7 @@ import org.apache.kafka.clients.admin._
 import org.apache.kafka.common.acl.{AclBinding, AclBindingFilter}
 import org.apache.kafka.common.config.{ConfigException, ConfigResource}
 import org.apache.kafka.common.config.ConfigResource.Type
-import org.apache.kafka.common.errors.{PolicyViolationException, 
UnsupportedVersionException}
+import 
org.apache.kafka.common.errors.{InvalidPartitionsException,PolicyViolationException,
 UnsupportedVersionException}
 import org.apache.kafka.common.message.DescribeClusterRequestData
 import org.apache.kafka.common.metadata.{ConfigRecord, FeatureLevelRecord}
 import org.apache.kafka.common.metrics.Metrics
@@ -792,6 +792,43 @@ class KRaftClusterTest {
 }
   }
 
+  /**
+   * Test that setting the Confluent-specific configuration
+   * confluent.apply.create.topic.policy.to.create.partitions has the expected 
effect.
+   */
+  @ParameterizedTest
+  @ValueSource(strings = Array("3.7-IV0", "3.7-IV2"))
+  def testCreatePartitions(metadataVersionString: String): Unit = {
+val cluster = new KafkaClusterTestKit.Builder(
+  new TestKitNodes.Builder().
+setNumBrokerNodes(4).
+
setBootstrapMetadataVersion(MetadataVersion.fromVersionString(metadataVersionString)).
+setNumControllerNodes(3).build()).
+  build()
+try {
+  cluster.format()
+  cluster.startup()
+  cluster.waitForReadyBrokers()
+  val admin = Admin.create(cluster.clientProperties())
+  try {
+val createResults = admin.createTopics(Arrays.asList(
+  new NewTopic("foo", 1, 3.toShort),
+  new NewTopic("bar", 2, 3.toShort))).values()
+createResults.get("foo").get()
+createResults.get("bar").get()
+val increaseResults = admin.createPartitions(Map(
+  "foo" -> NewPartitions.increaseTo(3),
+  "bar" -> NewPartitions.increaseTo(2)).asJava).values()
+increaseResults.get("foo").get()
+assertEquals(classOf[InvalidPartitionsException], assertThrows(
+  classOf[ExecutionException], () => 
increaseResults.get("bar").get()).getCause.getClass)
+  } finally {
+admin.close()
+  }
+} finally {
+  cluster.close()
+}
+  }
   private def clusterImage(
 cluster: KafkaClusterTestKit,
 brokerId: Int
diff --git 
a/metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java
 
b/metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java
index 177d5311afd..a7012d1505c 100644
--- 
a/metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java
+++ 
b/metadata/src/main/java/org/apache/kafka/metadata/placement/PartitionAssignment.java
@@ -17,7 +17,6 @@
 
 package org.apache.kafka.metadata.placement;
 
-import org.apache.kafka.common.DirectoryId;
 import org.apache.kafka.common.Uuid;
 
 import java.util.ArrayList;
@@ -39,11 +38,6 @@ public class PartitionAssignment {
 private final List replicas;
 private final List directories;
 
-// TODO remove -- just here for testing
-public PartitionAssignment(List replicas) {
-this(replicas, brokerId -> DirectoryId.UNASSIGNED);
-}
-
 public PartitionAssignment(List replicas, DefaultDirProvider 
defaultDirProvider) {
 this.replicas = Collections.unmodifiableList(new 
ArrayList<>(replicas));
 Uuid[] directories = new Uuid[replicas.size()];
diff --git 
a/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBui

(kafka) branch cmccabe_2024_02_26_fix created (now 925c61a99ce)

2024-02-26 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a change to branch cmccabe_2024_02_26_fix
in repository https://gitbox.apache.org/repos/asf/kafka.git


  at 925c61a99ce MINOR: remove test constructor for PartitionAssignment

This branch includes the following new commits:

 new 925c61a99ce MINOR: remove test constructor for PartitionAssignment

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.




(kafka) 01/01: MINOR: update leaderAndEpoch before initializing metadata publishers

2024-02-13 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch init_metadata_publishers
in repository https://gitbox.apache.org/repos/asf/kafka.git

commit b31840f8d7bd6a88da4a7a1f190b0ab438b817ef
Author: Colin P. McCabe 
AuthorDate: Tue Feb 13 16:47:43 2024 -0800

MINOR: update leaderAndEpoch before initializing metadata publishers
---
 .../src/main/scala/kafka/server/SharedServer.scala |  1 +
 .../apache/kafka/image/loader/MetadataLoader.java  | 23 ++
 .../kafka/image/loader/MetadataLoaderTest.java | 10 ++
 3 files changed, 30 insertions(+), 4 deletions(-)

diff --git a/core/src/main/scala/kafka/server/SharedServer.scala 
b/core/src/main/scala/kafka/server/SharedServer.scala
index 31b4957e3f5..4dac53ed292 100644
--- a/core/src/main/scala/kafka/server/SharedServer.scala
+++ b/core/src/main/scala/kafka/server/SharedServer.scala
@@ -285,6 +285,7 @@ class SharedServer(
   setThreadNamePrefix(s"kafka-${sharedServerConfig.nodeId}-").
   setFaultHandler(metadataLoaderFaultHandler).
   setHighWaterMarkAccessor(() => _raftManager.client.highWatermark()).
+  setLeaderAndEpochAccessor(() => 
_raftManager.client.leaderAndEpoch()).
   setMetrics(metadataLoaderMetrics)
 loader = loaderBuilder.build()
 snapshotEmitter = new SnapshotEmitter.Builder().
diff --git 
a/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java 
b/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java
index c6194f4e6dd..e967228a59e 100644
--- a/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java
+++ b/metadata/src/main/java/org/apache/kafka/image/loader/MetadataLoader.java
@@ -75,6 +75,7 @@ public class MetadataLoader implements 
RaftClient.Listener
 private FaultHandler faultHandler = (m, e) -> new 
FaultHandlerException(m, e);
 private MetadataLoaderMetrics metrics = null;
 private Supplier highWaterMarkAccessor = null;
+private Supplier leaderAndEpochAccessor = () -> 
LeaderAndEpoch.UNKNOWN;
 
 public Builder setNodeId(int nodeId) {
 this.nodeId = nodeId;
@@ -101,6 +102,11 @@ public class MetadataLoader implements 
RaftClient.Listener
 return this;
 }
 
+public Builder setLeaderAndEpochAccessor(Supplier 
leaderAndEpochAccessor) {
+this.leaderAndEpochAccessor = leaderAndEpochAccessor;
+return this;
+}
+
 public Builder setMetrics(MetadataLoaderMetrics metrics) {
 this.metrics = metrics;
 return this;
@@ -126,7 +132,8 @@ public class MetadataLoader implements 
RaftClient.Listener
 threadNamePrefix,
 faultHandler,
 metrics,
-highWaterMarkAccessor);
+highWaterMarkAccessor,
+leaderAndEpochAccessor);
 }
 }
 
@@ -157,6 +164,11 @@ public class MetadataLoader implements 
RaftClient.Listener
  */
 private final Supplier highWaterMarkAccessor;
 
+/**
+ * A function which supplies the current leader and epoch.
+ */
+private final Supplier leaderAndEpochAccessor;
+
 /**
  * Publishers which haven't received any metadata yet.
  */
@@ -197,13 +209,15 @@ public class MetadataLoader implements 
RaftClient.Listener
 String threadNamePrefix,
 FaultHandler faultHandler,
 MetadataLoaderMetrics metrics,
-Supplier highWaterMarkAccessor
+Supplier highWaterMarkAccessor,
+Supplier leaderAndEpochAccessor
 ) {
 this.log = logContext.logger(MetadataLoader.class);
 this.time = time;
 this.faultHandler = faultHandler;
 this.metrics = metrics;
 this.highWaterMarkAccessor = highWaterMarkAccessor;
+this.leaderAndEpochAccessor = leaderAndEpochAccessor;
 this.uninitializedPublishers = new LinkedHashMap<>();
 this.publishers = new LinkedHashMap<>();
 this.image = MetadataImage.EMPTY;
@@ -245,8 +259,9 @@ public class MetadataLoader implements 
RaftClient.Listener
 offset + " and high water mark is {}", where, 
highWaterMark.getAsLong());
 return true;
 }
-log.info("{}: The loader finished catching up to the current high 
water mark of {}",
-where, highWaterMark.getAsLong());
+currentLeaderAndEpoch = leaderAndEpochAccessor.get();
+log.info("{}: The loader finished catching up to the current high 
water mark of {}, {}",
+where, highWaterMark.getAsLong(), currentLeaderAndEpoch);
 catchingUp = false;
 return false;
 }
diff --git 
a/metadata/src/test/java/org/apache/kafka/image/loader/MetadataLoaderTest.java 
b/metadata/src/test/java/org/apache/kafka/image/loader/MetadataLoaderTest.java
index 921c241a09a..ad771c7

(kafka) branch init_metadata_publishers created (now b31840f8d7b)

2024-02-13 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a change to branch init_metadata_publishers
in repository https://gitbox.apache.org/repos/asf/kafka.git


  at b31840f8d7b MINOR: update leaderAndEpoch before initializing metadata 
publishers

This branch includes the following new commits:

 new b31840f8d7b MINOR: update leaderAndEpoch before initializing metadata 
publishers

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.




(kafka) branch 3.5 updated: MINOR: Fix some MetadataDelta handling issues during ZK migration (#15327)

2024-02-08 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.5
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.5 by this push:
 new 5e4ca35f539 MINOR: Fix some MetadataDelta handling issues during ZK 
migration (#15327)
5e4ca35f539 is described below

commit 5e4ca35f5393d17ae9a845892f8d3bc78c76a40f
Author: David Arthur 
AuthorDate: Wed Feb 7 15:54:59 2024 -0500

MINOR: Fix some MetadataDelta handling issues during ZK migration (#15327)

Reviewers: Colin P. McCabe 

Conflicts: skip adding the error logger callback in 
KRaftMigrationZkWriter.handleAclsDelta to
simplify the cherry-pick and avoid conflicts.
---
 .../zk/migration/ZkAclMigrationClientTest.scala| 150 -
 .../java/org/apache/kafka/image/AclsDelta.java |  13 --
 .../metadata/migration/KRaftMigrationZkWriter.java |  62 -
 3 files changed, 180 insertions(+), 45 deletions(-)

diff --git 
a/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala 
b/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala
index 0107b9fa954..57960e26687 100644
--- a/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala
+++ b/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala
@@ -21,7 +21,7 @@ import kafka.security.authorizer.AclEntry.{WildcardHost, 
WildcardPrincipalString
 import kafka.utils.TestUtils
 import org.apache.kafka.common.Uuid
 import org.apache.kafka.common.acl._
-import org.apache.kafka.common.metadata.AccessControlEntryRecord
+import org.apache.kafka.common.metadata.{AccessControlEntryRecord, 
RemoveAccessControlEntryRecord}
 import org.apache.kafka.common.resource.{PatternType, ResourcePattern, 
ResourcePatternFilter, ResourceType}
 import org.apache.kafka.common.security.auth.KafkaPrincipal
 import org.apache.kafka.common.utils.SecurityUtils
@@ -189,4 +189,152 @@ class ZkAclMigrationClientTest extends 
ZkMigrationTestHarness {
 AclPermissionType.fromCode(acl1Resource3.permissionType())),
   resource3AclsInZk.head.ace)
   }
+
+  def user(user: String): String = {
+new KafkaPrincipal(KafkaPrincipal.USER_TYPE, user).toString
+  }
+
+  def acl(resourceName: String,
+  resourceType: ResourceType,
+  resourcePattern: PatternType,
+  principal: String,
+  host: String = "*",
+  operation: AclOperation = AclOperation.READ,
+  permissionType: AclPermissionType = AclPermissionType.ALLOW
+  ): AccessControlEntryRecord = {
+new AccessControlEntryRecord()
+  .setId(Uuid.randomUuid())
+  .setHost(host)
+  .setOperation(operation.code())
+  .setPrincipal(principal)
+  .setPermissionType(permissionType.code())
+  .setPatternType(resourcePattern.code())
+  .setResourceName(resourceName)
+  .setResourceType(resourceType.code())
+  }
+
+  @Test
+  def testDeleteOneAclOfMany(): Unit = {
+zkClient.createAclPaths()
+val topicName = "topic-" + Uuid.randomUuid()
+val resource = new ResourcePattern(ResourceType.TOPIC, topicName, 
PatternType.LITERAL)
+
+// Create a delta with some ACLs
+val delta = new MetadataDelta(MetadataImage.EMPTY)
+val acl1 = acl(topicName, ResourceType.TOPIC, PatternType.LITERAL, 
user("alice"))
+val acl2 = acl(topicName, ResourceType.TOPIC, PatternType.LITERAL, 
user("bob"))
+val acl3 = acl(topicName, ResourceType.TOPIC, PatternType.LITERAL, 
user("carol"))
+delta.replay(acl1)
+delta.replay(acl2)
+delta.replay(acl3)
+val image = delta.apply(MetadataProvenance.EMPTY)
+
+// Sync image to ZK
+val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient)
+kraftMigrationZkWriter.handleSnapshot(image, (_, _, operation) => {
+  migrationState = operation.apply(migrationState)
+})
+
+// verify 3 ACLs in ZK
+val aclsInZk = zkClient.getVersionedAclsForResource(resource).acls
+assertEquals(3, aclsInZk.size)
+
+// Delete one of the ACLs
+val delta2 = new MetadataDelta.Builder()
+  .setImage(image)
+  .build()
+delta2.replay(new RemoveAccessControlEntryRecord().setId(acl3.id()))
+val image2 = delta2.apply(MetadataProvenance.EMPTY)
+kraftMigrationZkWriter.handleDelta(image, image2, delta2, (_, _, 
operation) => {
+  migrationState = operation.apply(migrationState)
+})
+
+// verify the other 2 ACLs are still in ZK
+val aclsInZk2 = zkClient.getVersionedAclsForResource(resource).acls
+assertEquals(2, aclsInZk2.size)
+
+// Add another ACL
+val acl4 = acl(topicName, ResourceType.TOPIC, PatternType.LITERAL, 
user("carol"))
+delta2.replay(acl4)
+val image3 = delta2.apply(MetadataProvenance.EMPTY)
+
+// This is a contrived error case. In practice, we will never pass the 
same image as prev and current.

(kafka) branch 3.6 updated: MINOR: fix scala compile issue (#15343)

2024-02-08 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.6
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.6 by this push:
 new c4fa14a7d95 MINOR: fix scala compile issue (#15343)
c4fa14a7d95 is described below

commit c4fa14a7d95dbfc5db4c5476aa14cf013d33f2ca
Author: David Arthur 
AuthorDate: Thu Feb 8 18:44:42 2024 -0500

MINOR: fix scala compile issue (#15343)

Reviewers: David Jacot 
---
 .../test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git 
a/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala 
b/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala
index bb4d3e646c7..a4045f57948 100644
--- a/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala
+++ b/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala
@@ -231,7 +231,7 @@ class ZkAclMigrationClientTest extends 
ZkMigrationTestHarness {
 
 // Sync image to ZK
 val errorLogs = mutable.Buffer[String]()
-val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, 
errorLogs.append)
+val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, 
msg => errorLogs.append(msg))
 kraftMigrationZkWriter.handleSnapshot(image, (_, _, operation) => {
   migrationState = operation.apply(migrationState)
 })
@@ -276,7 +276,7 @@ class ZkAclMigrationClientTest extends 
ZkMigrationTestHarness {
   def testAclUpdateAndDelete(): Unit = {
 zkClient.createAclPaths()
 val errorLogs = mutable.Buffer[String]()
-val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, 
errorLogs.append)
+val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, 
msg => errorLogs.append(msg))
 
 val topicName = "topic-" + Uuid.randomUuid()
 val otherName = "other-" + Uuid.randomUuid()



(kafka) branch 3.7 updated: MINOR: fix scala compile issue (#15343)

2024-02-08 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.7
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.7 by this push:
 new 38bac936343 MINOR: fix scala compile issue (#15343)
38bac936343 is described below

commit 38bac936343cc6a03d0f558b7ae8917cdb2707b7
Author: David Arthur 
AuthorDate: Thu Feb 8 18:44:42 2024 -0500

MINOR: fix scala compile issue (#15343)

Reviewers: David Jacot 
---
 .../test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git 
a/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala 
b/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala
index bb4d3e646c7..a4045f57948 100644
--- a/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala
+++ b/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala
@@ -231,7 +231,7 @@ class ZkAclMigrationClientTest extends 
ZkMigrationTestHarness {
 
 // Sync image to ZK
 val errorLogs = mutable.Buffer[String]()
-val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, 
errorLogs.append)
+val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, 
msg => errorLogs.append(msg))
 kraftMigrationZkWriter.handleSnapshot(image, (_, _, operation) => {
   migrationState = operation.apply(migrationState)
 })
@@ -276,7 +276,7 @@ class ZkAclMigrationClientTest extends 
ZkMigrationTestHarness {
   def testAclUpdateAndDelete(): Unit = {
 zkClient.createAclPaths()
 val errorLogs = mutable.Buffer[String]()
-val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, 
errorLogs.append)
+val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, 
msg => errorLogs.append(msg))
 
 val topicName = "topic-" + Uuid.randomUuid()
 val otherName = "other-" + Uuid.randomUuid()



(kafka) branch trunk updated: MINOR: fix scala compile issue (#15343)

2024-02-08 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 116bc000c8c MINOR: fix scala compile issue (#15343)
116bc000c8c is described below

commit 116bc000c8c6533552321fe1d395629c2aa00bd9
Author: David Arthur 
AuthorDate: Thu Feb 8 18:44:42 2024 -0500

MINOR: fix scala compile issue (#15343)

Reviewers: David Jacot 
---
 .../test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git 
a/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala 
b/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala
index bb4d3e646c7..a4045f57948 100644
--- a/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala
+++ b/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala
@@ -231,7 +231,7 @@ class ZkAclMigrationClientTest extends 
ZkMigrationTestHarness {
 
 // Sync image to ZK
 val errorLogs = mutable.Buffer[String]()
-val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, 
errorLogs.append)
+val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, 
msg => errorLogs.append(msg))
 kraftMigrationZkWriter.handleSnapshot(image, (_, _, operation) => {
   migrationState = operation.apply(migrationState)
 })
@@ -276,7 +276,7 @@ class ZkAclMigrationClientTest extends 
ZkMigrationTestHarness {
   def testAclUpdateAndDelete(): Unit = {
 zkClient.createAclPaths()
 val errorLogs = mutable.Buffer[String]()
-val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, 
errorLogs.append)
+val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, 
msg => errorLogs.append(msg))
 
 val topicName = "topic-" + Uuid.randomUuid()
 val otherName = "other-" + Uuid.randomUuid()



(kafka) branch 3.6 updated: MINOR: Fix some MetadataDelta handling issues during ZK migration (#15327)

2024-02-07 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.6
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.6 by this push:
 new 3c623b4fb50 MINOR: Fix some MetadataDelta handling issues during ZK 
migration (#15327)
3c623b4fb50 is described below

commit 3c623b4fb50144cba782c0704f345e6473e66f02
Author: David Arthur 
AuthorDate: Wed Feb 7 15:54:59 2024 -0500

MINOR: Fix some MetadataDelta handling issues during ZK migration (#15327)

Reviewers: Colin P. McCabe 
---
 .../zk/migration/ZkAclMigrationClientTest.scala| 162 -
 .../zk/migration/ZkConfigMigrationClientTest.scala |   4 +-
 .../kafka/zk/migration/ZkMigrationClientTest.scala |   2 +-
 .../java/org/apache/kafka/image/AclsDelta.java |  13 --
 .../metadata/migration/KRaftMigrationDriver.java   |   5 +-
 .../metadata/migration/KRaftMigrationZkWriter.java |  70 +
 .../migration/KRaftMigrationZkWriterTest.java  |   6 +-
 7 files changed, 206 insertions(+), 56 deletions(-)

diff --git 
a/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala 
b/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala
index 0107b9fa954..bb4d3e646c7 100644
--- a/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala
+++ b/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala
@@ -21,14 +21,14 @@ import kafka.security.authorizer.AclEntry.{WildcardHost, 
WildcardPrincipalString
 import kafka.utils.TestUtils
 import org.apache.kafka.common.Uuid
 import org.apache.kafka.common.acl._
-import org.apache.kafka.common.metadata.AccessControlEntryRecord
+import org.apache.kafka.common.metadata.{AccessControlEntryRecord, 
RemoveAccessControlEntryRecord}
 import org.apache.kafka.common.resource.{PatternType, ResourcePattern, 
ResourcePatternFilter, ResourceType}
 import org.apache.kafka.common.security.auth.KafkaPrincipal
 import org.apache.kafka.common.utils.SecurityUtils
 import org.apache.kafka.image.{MetadataDelta, MetadataImage, 
MetadataProvenance}
 import org.apache.kafka.metadata.migration.KRaftMigrationZkWriter
 import org.apache.kafka.server.common.ApiMessageAndVersion
-import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
+import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue, fail}
 import org.junit.jupiter.api.Test
 
 import scala.collection.mutable
@@ -169,7 +169,7 @@ class ZkAclMigrationClientTest extends 
ZkMigrationTestHarness {
 val image = delta.apply(MetadataProvenance.EMPTY)
 
 // load snapshot to Zookeeper.
-val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient)
+val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, 
fail(_))
 kraftMigrationZkWriter.handleSnapshot(image, (_, _, operation) => { 
migrationState = operation.apply(migrationState) })
 
 // Verify the new ACLs in Zookeeper.
@@ -189,4 +189,160 @@ class ZkAclMigrationClientTest extends 
ZkMigrationTestHarness {
 AclPermissionType.fromCode(acl1Resource3.permissionType())),
   resource3AclsInZk.head.ace)
   }
+
+  def user(user: String): String = {
+new KafkaPrincipal(KafkaPrincipal.USER_TYPE, user).toString
+  }
+
+  def acl(resourceName: String,
+  resourceType: ResourceType,
+  resourcePattern: PatternType,
+  principal: String,
+  host: String = "*",
+  operation: AclOperation = AclOperation.READ,
+  permissionType: AclPermissionType = AclPermissionType.ALLOW
+  ): AccessControlEntryRecord = {
+new AccessControlEntryRecord()
+  .setId(Uuid.randomUuid())
+  .setHost(host)
+  .setOperation(operation.code())
+  .setPrincipal(principal)
+  .setPermissionType(permissionType.code())
+  .setPatternType(resourcePattern.code())
+  .setResourceName(resourceName)
+  .setResourceType(resourceType.code())
+  }
+
+  @Test
+  def testDeleteOneAclOfMany(): Unit = {
+zkClient.createAclPaths()
+val topicName = "topic-" + Uuid.randomUuid()
+val resource = new ResourcePattern(ResourceType.TOPIC, topicName, 
PatternType.LITERAL)
+
+// Create a delta with some ACLs
+val delta = new MetadataDelta(MetadataImage.EMPTY)
+val acl1 = acl(topicName, ResourceType.TOPIC, PatternType.LITERAL, 
user("alice"))
+val acl2 = acl(topicName, ResourceType.TOPIC, PatternType.LITERAL, 
user("bob"))
+val acl3 = acl(topicName, ResourceType.TOPIC, PatternType.LITERAL, 
user("carol"))
+delta.replay(acl1)
+delta.replay(acl2)
+delta.replay(acl3)
+val image = delta.apply(MetadataProvenance.EMPTY)
+
+// Sync image to ZK
+val errorLogs = mutable.Buffer[String]()
+val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, 
errorLogs.append)
+kraftMigrationZkWriter.handleSnapshot(image, (_, _, op

(kafka) branch 3.7 updated: MINOR: Fix some MetadataDelta handling issues during ZK migration (#15327)

2024-02-07 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.7
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.7 by this push:
 new b0721296b25 MINOR: Fix some MetadataDelta handling issues during ZK 
migration (#15327)
b0721296b25 is described below

commit b0721296b259aebbbd80bbd10597a46e55b32655
Author: David Arthur 
AuthorDate: Wed Feb 7 15:54:59 2024 -0500

MINOR: Fix some MetadataDelta handling issues during ZK migration (#15327)

Reviewers: Colin P. McCabe 
---
 .../zk/migration/ZkAclMigrationClientTest.scala| 162 -
 .../zk/migration/ZkConfigMigrationClientTest.scala |   4 +-
 .../kafka/zk/migration/ZkMigrationClientTest.scala |   2 +-
 .../java/org/apache/kafka/image/AclsDelta.java |  13 --
 .../metadata/migration/KRaftMigrationDriver.java   |   5 +-
 .../metadata/migration/KRaftMigrationZkWriter.java |  70 +
 .../migration/KRaftMigrationZkWriterTest.java  |   6 +-
 7 files changed, 206 insertions(+), 56 deletions(-)

diff --git 
a/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala 
b/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala
index 0107b9fa954..bb4d3e646c7 100644
--- a/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala
+++ b/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala
@@ -21,14 +21,14 @@ import kafka.security.authorizer.AclEntry.{WildcardHost, 
WildcardPrincipalString
 import kafka.utils.TestUtils
 import org.apache.kafka.common.Uuid
 import org.apache.kafka.common.acl._
-import org.apache.kafka.common.metadata.AccessControlEntryRecord
+import org.apache.kafka.common.metadata.{AccessControlEntryRecord, 
RemoveAccessControlEntryRecord}
 import org.apache.kafka.common.resource.{PatternType, ResourcePattern, 
ResourcePatternFilter, ResourceType}
 import org.apache.kafka.common.security.auth.KafkaPrincipal
 import org.apache.kafka.common.utils.SecurityUtils
 import org.apache.kafka.image.{MetadataDelta, MetadataImage, 
MetadataProvenance}
 import org.apache.kafka.metadata.migration.KRaftMigrationZkWriter
 import org.apache.kafka.server.common.ApiMessageAndVersion
-import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
+import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue, fail}
 import org.junit.jupiter.api.Test
 
 import scala.collection.mutable
@@ -169,7 +169,7 @@ class ZkAclMigrationClientTest extends 
ZkMigrationTestHarness {
 val image = delta.apply(MetadataProvenance.EMPTY)
 
 // load snapshot to Zookeeper.
-val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient)
+val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, 
fail(_))
 kraftMigrationZkWriter.handleSnapshot(image, (_, _, operation) => { 
migrationState = operation.apply(migrationState) })
 
 // Verify the new ACLs in Zookeeper.
@@ -189,4 +189,160 @@ class ZkAclMigrationClientTest extends 
ZkMigrationTestHarness {
 AclPermissionType.fromCode(acl1Resource3.permissionType())),
   resource3AclsInZk.head.ace)
   }
+
+  def user(user: String): String = {
+new KafkaPrincipal(KafkaPrincipal.USER_TYPE, user).toString
+  }
+
+  def acl(resourceName: String,
+  resourceType: ResourceType,
+  resourcePattern: PatternType,
+  principal: String,
+  host: String = "*",
+  operation: AclOperation = AclOperation.READ,
+  permissionType: AclPermissionType = AclPermissionType.ALLOW
+  ): AccessControlEntryRecord = {
+new AccessControlEntryRecord()
+  .setId(Uuid.randomUuid())
+  .setHost(host)
+  .setOperation(operation.code())
+  .setPrincipal(principal)
+  .setPermissionType(permissionType.code())
+  .setPatternType(resourcePattern.code())
+  .setResourceName(resourceName)
+  .setResourceType(resourceType.code())
+  }
+
+  @Test
+  def testDeleteOneAclOfMany(): Unit = {
+zkClient.createAclPaths()
+val topicName = "topic-" + Uuid.randomUuid()
+val resource = new ResourcePattern(ResourceType.TOPIC, topicName, 
PatternType.LITERAL)
+
+// Create a delta with some ACLs
+val delta = new MetadataDelta(MetadataImage.EMPTY)
+val acl1 = acl(topicName, ResourceType.TOPIC, PatternType.LITERAL, 
user("alice"))
+val acl2 = acl(topicName, ResourceType.TOPIC, PatternType.LITERAL, 
user("bob"))
+val acl3 = acl(topicName, ResourceType.TOPIC, PatternType.LITERAL, 
user("carol"))
+delta.replay(acl1)
+delta.replay(acl2)
+delta.replay(acl3)
+val image = delta.apply(MetadataProvenance.EMPTY)
+
+// Sync image to ZK
+val errorLogs = mutable.Buffer[String]()
+val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, 
errorLogs.append)
+kraftMigrationZkWriter.handleSnapshot(image, (_, _, op

(kafka) branch trunk updated: MINOR: Fix some MetadataDelta handling issues during ZK migration (#15327)

2024-02-07 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new c000b1fae2b MINOR: Fix some MetadataDelta handling issues during ZK 
migration (#15327)
c000b1fae2b is described below

commit c000b1fae2bd7d4b76713a53508f128a13431ab6
Author: David Arthur 
AuthorDate: Wed Feb 7 15:54:59 2024 -0500

MINOR: Fix some MetadataDelta handling issues during ZK migration (#15327)

Reviewers: Colin P. McCabe 
---
 .../zk/migration/ZkAclMigrationClientTest.scala| 162 -
 .../zk/migration/ZkConfigMigrationClientTest.scala |   4 +-
 .../kafka/zk/migration/ZkMigrationClientTest.scala |   2 +-
 .../java/org/apache/kafka/image/AclsDelta.java |  13 --
 .../metadata/migration/KRaftMigrationDriver.java   |   5 +-
 .../metadata/migration/KRaftMigrationZkWriter.java |  70 +
 .../migration/KRaftMigrationZkWriterTest.java  |   6 +-
 7 files changed, 206 insertions(+), 56 deletions(-)

diff --git 
a/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala 
b/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala
index 0107b9fa954..bb4d3e646c7 100644
--- a/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala
+++ b/core/src/test/scala/unit/kafka/zk/migration/ZkAclMigrationClientTest.scala
@@ -21,14 +21,14 @@ import kafka.security.authorizer.AclEntry.{WildcardHost, 
WildcardPrincipalString
 import kafka.utils.TestUtils
 import org.apache.kafka.common.Uuid
 import org.apache.kafka.common.acl._
-import org.apache.kafka.common.metadata.AccessControlEntryRecord
+import org.apache.kafka.common.metadata.{AccessControlEntryRecord, 
RemoveAccessControlEntryRecord}
 import org.apache.kafka.common.resource.{PatternType, ResourcePattern, 
ResourcePatternFilter, ResourceType}
 import org.apache.kafka.common.security.auth.KafkaPrincipal
 import org.apache.kafka.common.utils.SecurityUtils
 import org.apache.kafka.image.{MetadataDelta, MetadataImage, 
MetadataProvenance}
 import org.apache.kafka.metadata.migration.KRaftMigrationZkWriter
 import org.apache.kafka.server.common.ApiMessageAndVersion
-import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
+import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue, fail}
 import org.junit.jupiter.api.Test
 
 import scala.collection.mutable
@@ -169,7 +169,7 @@ class ZkAclMigrationClientTest extends 
ZkMigrationTestHarness {
 val image = delta.apply(MetadataProvenance.EMPTY)
 
 // load snapshot to Zookeeper.
-val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient)
+val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, 
fail(_))
 kraftMigrationZkWriter.handleSnapshot(image, (_, _, operation) => { 
migrationState = operation.apply(migrationState) })
 
 // Verify the new ACLs in Zookeeper.
@@ -189,4 +189,160 @@ class ZkAclMigrationClientTest extends 
ZkMigrationTestHarness {
 AclPermissionType.fromCode(acl1Resource3.permissionType())),
   resource3AclsInZk.head.ace)
   }
+
+  def user(user: String): String = {
+new KafkaPrincipal(KafkaPrincipal.USER_TYPE, user).toString
+  }
+
+  def acl(resourceName: String,
+  resourceType: ResourceType,
+  resourcePattern: PatternType,
+  principal: String,
+  host: String = "*",
+  operation: AclOperation = AclOperation.READ,
+  permissionType: AclPermissionType = AclPermissionType.ALLOW
+  ): AccessControlEntryRecord = {
+new AccessControlEntryRecord()
+  .setId(Uuid.randomUuid())
+  .setHost(host)
+  .setOperation(operation.code())
+  .setPrincipal(principal)
+  .setPermissionType(permissionType.code())
+  .setPatternType(resourcePattern.code())
+  .setResourceName(resourceName)
+  .setResourceType(resourceType.code())
+  }
+
+  @Test
+  def testDeleteOneAclOfMany(): Unit = {
+zkClient.createAclPaths()
+val topicName = "topic-" + Uuid.randomUuid()
+val resource = new ResourcePattern(ResourceType.TOPIC, topicName, 
PatternType.LITERAL)
+
+// Create a delta with some ACLs
+val delta = new MetadataDelta(MetadataImage.EMPTY)
+val acl1 = acl(topicName, ResourceType.TOPIC, PatternType.LITERAL, 
user("alice"))
+val acl2 = acl(topicName, ResourceType.TOPIC, PatternType.LITERAL, 
user("bob"))
+val acl3 = acl(topicName, ResourceType.TOPIC, PatternType.LITERAL, 
user("carol"))
+delta.replay(acl1)
+delta.replay(acl2)
+delta.replay(acl3)
+val image = delta.apply(MetadataProvenance.EMPTY)
+
+// Sync image to ZK
+val errorLogs = mutable.Buffer[String]()
+val kraftMigrationZkWriter = new KRaftMigrationZkWriter(migrationClient, 
errorLogs.append)
+kraftMigrationZkWriter.handleSnapshot(image, (_, _, op

(kafka) branch 3.7 updated: KAFKA-16180: Fix UMR and LAIR handling during ZK migration (#15293)

2024-02-02 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.7
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.7 by this push:
 new 5e9c61d79e7 KAFKA-16180: Fix UMR and LAIR handling during ZK migration 
(#15293)
5e9c61d79e7 is described below

commit 5e9c61d79e79960f138b6020450f888947217382
Author: Colin Patrick McCabe 
AuthorDate: Fri Feb 2 15:49:10 2024 -0800

KAFKA-16180: Fix UMR and LAIR handling during ZK migration (#15293)

While migrating from ZK mode to KRaft mode, the broker passes through a 
"hybrid" phase, in which it
receives LeaderAndIsrRequest and UpdateMetadataRequest RPCs from the KRaft 
controller. For the most
part, these RPCs can be handled just like their traditional equivalents 
from a ZK-based controller.
However, there is one thing that is different: the way topic deletions are 
handled.

In ZK mode, there is a "deleting" state which topics enter prior to being 
completely removed.
Partitions stay in this state until they are removed from the disks of all 
replicas. And partitions
associated with these deleting topics show up in the UMR and LAIR as having 
a leader of -2 (which
is not a valid broker ID, of course, because it's negative). When brokers 
receive these RPCs, they
know to remove the associated partitions from their metadata caches, and 
disks. When a full UMR or
ISR is sent, deleting partitions are included as well.

In hybrid mode, in contrast, there is no "deleting" state. Topic deletion 
happens immediately. We
can do this because we know that we have topic IDs that are never reused. 
This means that we can
always tell the difference between a broker that had an old version of some 
topic, and a broker
that has a new version that was re-created with the same name. To make this 
work, when handling a
full UMR or LAIR, hybrid brokers must compare the full state that was sent 
over the wire to their
own local state, and adjust accordingly.

Prior to this PR, the code for handling those adjustments had several major 
flaws. The biggest flaw
is that it did not correctly handle the "re-creation" case where a topic 
named FOO appears in the
RPC, but with a different ID than the broker's local FOO. Another flaw is 
that a problem with a
single partition would prevent handling the whole request.

In ZkMetadataCache.scala, we handle full UMR requests from KRaft 
controllers by rewriting the UMR
so that it contains the implied deletions. I fixed this code so that 
deletions always appear at the
start of the list of topic states. This is important for the re-creation 
case since it means that a
single request can both delete the old FOO and add a new FOO to the cache. 
Also, rather than
modifying the requesst in-place, as the previous code did, I build a whole 
new request with the
desired list of topic states. This is much safer because it avoids 
unforseen interactions with
other parts of the code that deal with requests (like request logging). 
While this new copy may
sound expensive, it should actually not be. We are doing a "shallow copy" 
which references the
previous list topic state entries.

I also reworked ZkMetadataCache.updateMetadata so that if a partition is 
re-created, it does not
appear in the returned set of deleted TopicPartitions. Since this set is 
used only by the group
manager, this seemed appropriate. (If I was in the consumer group for the 
previous iteration of
FOO, I should still be in the consumer group for the new iteration.)

On the ReplicaManager.scala side, we handle full LAIR requests by treating 
anything which does not
appear in them as a "stray replica." (But we do not rewrite the request 
objects as we do with UMR.)
I moved the logic for finding stray replicas from ReplicaManager into 
LogManager. It makes more
sense there, since the information about what is on-disk is managed in 
LogManager. Also, the stray
replica detection logic for KRaft mode is there, so it makes sense to put 
the stray replica
detection logic for hybrid mode there as well.

Since the stray replica detection is now in LogManager, I moved the unit 
tests there as well.
Previously some of those tests had been in BrokerMetadataPublisherTest for 
historical reasons.

The main advantage of the new LAIR logic is that it takes topic ID into 
account. A replica can be a
stray even if the LAIR contains a topic of the given name, but a different 
ID. I also moved the
stray replica handling earlier in the becomeLeaderOrFollower function, so 
that we could correctly
handle the "delete and re-create FOO" case.

Reviewers: David Arthur 
---
 .../kafka/common/requests/LeaderAndIs

(kafka) branch trunk updated: KAFKA-16180: Fix UMR and LAIR handling during ZK migration (#15293)

2024-02-02 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 4169ac9f5dc KAFKA-16180: Fix UMR and LAIR handling during ZK migration 
(#15293)
4169ac9f5dc is described below

commit 4169ac9f5dc819c0495247c2230e96581800b1c7
Author: Colin Patrick McCabe 
AuthorDate: Fri Feb 2 15:49:10 2024 -0800

KAFKA-16180: Fix UMR and LAIR handling during ZK migration (#15293)

While migrating from ZK mode to KRaft mode, the broker passes through a 
"hybrid" phase, in which it
receives LeaderAndIsrRequest and UpdateMetadataRequest RPCs from the KRaft 
controller. For the most
part, these RPCs can be handled just like their traditional equivalents 
from a ZK-based controller.
However, there is one thing that is different: the way topic deletions are 
handled.

In ZK mode, there is a "deleting" state which topics enter prior to being 
completely removed.
Partitions stay in this state until they are removed from the disks of all 
replicas. And partitions
associated with these deleting topics show up in the UMR and LAIR as having 
a leader of -2 (which
is not a valid broker ID, of course, because it's negative). When brokers 
receive these RPCs, they
know to remove the associated partitions from their metadata caches, and 
disks. When a full UMR or
ISR is sent, deleting partitions are included as well.

In hybrid mode, in contrast, there is no "deleting" state. Topic deletion 
happens immediately. We
can do this because we know that we have topic IDs that are never reused. 
This means that we can
always tell the difference between a broker that had an old version of some 
topic, and a broker
that has a new version that was re-created with the same name. To make this 
work, when handling a
full UMR or LAIR, hybrid brokers must compare the full state that was sent 
over the wire to their
own local state, and adjust accordingly.

Prior to this PR, the code for handling those adjustments had several major 
flaws. The biggest flaw
is that it did not correctly handle the "re-creation" case where a topic 
named FOO appears in the
RPC, but with a different ID than the broker's local FOO. Another flaw is 
that a problem with a
single partition would prevent handling the whole request.

In ZkMetadataCache.scala, we handle full UMR requests from KRaft 
controllers by rewriting the UMR
so that it contains the implied deletions. I fixed this code so that 
deletions always appear at the
start of the list of topic states. This is important for the re-creation 
case since it means that a
single request can both delete the old FOO and add a new FOO to the cache. 
Also, rather than
modifying the requesst in-place, as the previous code did, I build a whole 
new request with the
desired list of topic states. This is much safer because it avoids 
unforseen interactions with
other parts of the code that deal with requests (like request logging). 
While this new copy may
sound expensive, it should actually not be. We are doing a "shallow copy" 
which references the
previous list topic state entries.

I also reworked ZkMetadataCache.updateMetadata so that if a partition is 
re-created, it does not
appear in the returned set of deleted TopicPartitions. Since this set is 
used only by the group
manager, this seemed appropriate. (If I was in the consumer group for the 
previous iteration of
FOO, I should still be in the consumer group for the new iteration.)

On the ReplicaManager.scala side, we handle full LAIR requests by treating 
anything which does not
appear in them as a "stray replica." (But we do not rewrite the request 
objects as we do with UMR.)
I moved the logic for finding stray replicas from ReplicaManager into 
LogManager. It makes more
sense there, since the information about what is on-disk is managed in 
LogManager. Also, the stray
replica detection logic for KRaft mode is there, so it makes sense to put 
the stray replica
detection logic for hybrid mode there as well.

Since the stray replica detection is now in LogManager, I moved the unit 
tests there as well.
Previously some of those tests had been in BrokerMetadataPublisherTest for 
historical reasons.

The main advantage of the new LAIR logic is that it takes topic ID into 
account. A replica can be a
stray even if the LAIR contains a topic of the given name, but a different 
ID. I also moved the
stray replica handling earlier in the becomeLeaderOrFollower function, so 
that we could correctly
handle the "delete and re-create FOO" case.

Reviewers: David Arthur 
---
 .../kafka/

(kafka) branch 3.7 updated: KAFKA-16195: ignore metadata.log.dir failure in ZK mode (#15262)

2024-02-02 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.7
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.7 by this push:
 new 73fb4de9aa0 KAFKA-16195: ignore metadata.log.dir failure in ZK mode 
(#15262)
73fb4de9aa0 is described below

commit 73fb4de9aa03b2c39b11d7c296e55154a3b341de
Author: Gaurav Narula 
AuthorDate: Fri Feb 2 17:47:14 2024 +

KAFKA-16195: ignore metadata.log.dir failure in ZK mode (#15262)

In KRaft mode, or on ZK brokers that are migrating to KRaft, we have a 
local __cluster_metadata
log. This log is stored in a single log directory which is configured via 
metadata.log.dir. If
there is no metadata.log.dir given, it defaults to the first entry in 
log.dirs. In the future we
may support multiple metadata log directories, but we don't yet. For now, 
we must abort the
process when this log directory fails.

In ZK mode, it is not necessary to abort the process when this directory 
fails, since there is no
__cluster_metadata log there. This PR changes the logic so that we check 
for whether we're in ZK
mode and do not abort in that scenario (unless we lost the final remaining 
log directory. of
course.)

Reviewers: Luke Chen , Colin P. McCabe 
, Omnia G H Ibrahim , Proven 
Provenzano 
---
 .../main/scala/kafka/server/ReplicaManager.scala   |  2 +-
 .../unit/kafka/server/ReplicaManagerTest.scala | 64 +-
 2 files changed, 64 insertions(+), 2 deletions(-)

diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala 
b/core/src/main/scala/kafka/server/ReplicaManager.scala
index e445fb7215a..2c6958cc34a 100644
--- a/core/src/main/scala/kafka/server/ReplicaManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
@@ -2598,7 +2598,7 @@ class ReplicaManager(val config: KafkaConfig,
 // retrieve the UUID here because logManager.handleLogDirFailure handler 
removes it
 val uuid = logManager.directoryId(dir)
 logManager.handleLogDirFailure(dir)
-if (dir == config.metadataLogDir) {
+if (dir == new File(config.metadataLogDir).getAbsolutePath && 
(config.processRoles.nonEmpty || config.migrationEnabled)) {
   fatal(s"Shutdown broker because the metadata log dir $dir has failed")
   Exit.halt(1)
 }
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala 
b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
index 9cc283eecad..4eab4df73c0 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
@@ -33,7 +33,7 @@ import kafka.log._
 import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota}
 import kafka.server.checkpoints.{LazyOffsetCheckpoints, OffsetCheckpointFile}
 import kafka.server.epoch.util.MockBlockingSender
-import kafka.utils.{Pool, TestInfoUtils, TestUtils}
+import kafka.utils.{Exit, Pool, TestInfoUtils, TestUtils}
 import org.apache.kafka.clients.FetchSessionHandler
 import org.apache.kafka.common.errors.{InvalidPidMappingException, 
KafkaStorageException}
 import org.apache.kafka.common.message.LeaderAndIsrRequestData
@@ -71,6 +71,7 @@ import org.junit.jupiter.params.ParameterizedTest
 import org.junit.jupiter.params.provider.{EnumSource, ValueSource}
 import com.yammer.metrics.core.{Gauge, Meter}
 import kafka.log.remote.RemoteLogManager
+import kafka.zk.KafkaZkClient
 import org.apache.kafka.common.config.{AbstractConfig, TopicConfig}
 import org.apache.kafka.metadata.properties.{MetaProperties, 
MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils}
 import org.apache.kafka.raft.RaftConfig
@@ -6369,6 +6370,67 @@ class ReplicaManagerTest {
   replicaManager.shutdown(checkpointHW = false)
 }
   }
+
+  @Test
+  def testMetadataLogDirFailureInZkShouldNotHaltBroker(): Unit = {
+// Given
+val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect, 
logDirCount = 2)
+val config = KafkaConfig.fromProps(props)
+val logDirFiles = config.logDirs.map(new File(_))
+val logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size)
+val logManager = TestUtils.createLogManager(logDirFiles, defaultConfig = 
new LogConfig(new Properties()), time = time)
+val mockZkClient = mock(classOf[KafkaZkClient])
+val replicaManager = new ReplicaManager(
+  metrics = metrics,
+  config = config,
+  time = time,
+  scheduler = time.scheduler,
+  logManager = logManager,
+  quotaManagers = quotaManager,
+  metadataCache = MetadataCache.zkMetadataCache(config.brokerId, 
config.interBrokerProtocolVersion),
+  logDirFailureChannel = logDirFailureChannel,
+  alterPartitionManager = alterPartitionManager,
+  threadNamePrefix = Option(this.getClass.getName),
+  zkClient = Option(mockZkClient),
+)
+
+try {
+  

(kafka) branch trunk updated: KAFKA-16195: ignore metadata.log.dir failure in ZK mode (#15262)

2024-02-02 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 3d95a69a28c KAFKA-16195: ignore metadata.log.dir failure in ZK mode 
(#15262)
3d95a69a28c is described below

commit 3d95a69a28c2d16e96618cfa9a1eb69180fb66ea
Author: Gaurav Narula 
AuthorDate: Fri Feb 2 17:47:14 2024 +

KAFKA-16195: ignore metadata.log.dir failure in ZK mode (#15262)

In KRaft mode, or on ZK brokers that are migrating to KRaft, we have a 
local __cluster_metadata
log. This log is stored in a single log directory which is configured via 
metadata.log.dir. If
there is no metadata.log.dir given, it defaults to the first entry in 
log.dirs. In the future we
may support multiple metadata log directories, but we don't yet. For now, 
we must abort the
process when this log directory fails.

In ZK mode, it is not necessary to abort the process when this directory 
fails, since there is no
__cluster_metadata log there. This PR changes the logic so that we check 
for whether we're in ZK
mode and do not abort in that scenario (unless we lost the final remaining 
log directory. of
course.)

Reviewers: Luke Chen , Colin P. McCabe 
, Omnia G H Ibrahim , Proven 
Provenzano 
---
 .../main/scala/kafka/server/ReplicaManager.scala   |  2 +-
 .../unit/kafka/server/ReplicaManagerTest.scala | 64 +-
 2 files changed, 64 insertions(+), 2 deletions(-)

diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala 
b/core/src/main/scala/kafka/server/ReplicaManager.scala
index 66ac417b5b1..5f34ade6021 100644
--- a/core/src/main/scala/kafka/server/ReplicaManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
@@ -2453,7 +2453,7 @@ class ReplicaManager(val config: KafkaConfig,
 // retrieve the UUID here because logManager.handleLogDirFailure handler 
removes it
 val uuid = logManager.directoryId(dir)
 logManager.handleLogDirFailure(dir)
-if (dir == config.metadataLogDir) {
+if (dir == new File(config.metadataLogDir).getAbsolutePath && 
(config.processRoles.nonEmpty || config.migrationEnabled)) {
   fatal(s"Shutdown broker because the metadata log dir $dir has failed")
   Exit.halt(1)
 }
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala 
b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
index 174d4192fe4..4ec1a392c1f 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
@@ -33,7 +33,7 @@ import kafka.log._
 import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota}
 import kafka.server.checkpoints.{LazyOffsetCheckpoints, OffsetCheckpointFile}
 import kafka.server.epoch.util.MockBlockingSender
-import kafka.utils.{Pool, TestInfoUtils, TestUtils}
+import kafka.utils.{Exit, Pool, TestInfoUtils, TestUtils}
 import org.apache.kafka.clients.FetchSessionHandler
 import org.apache.kafka.common.errors.{InvalidPidMappingException, 
KafkaStorageException}
 import org.apache.kafka.common.message.LeaderAndIsrRequestData
@@ -71,6 +71,7 @@ import org.junit.jupiter.params.ParameterizedTest
 import org.junit.jupiter.params.provider.{EnumSource, ValueSource}
 import com.yammer.metrics.core.{Gauge, Meter}
 import kafka.log.remote.RemoteLogManager
+import kafka.zk.KafkaZkClient
 import org.apache.kafka.common.config.{AbstractConfig, TopicConfig}
 import org.apache.kafka.metadata.properties.{MetaProperties, 
MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils}
 import org.apache.kafka.raft.RaftConfig
@@ -6361,6 +6362,67 @@ class ReplicaManagerTest {
   replicaManager.shutdown(checkpointHW = false)
 }
   }
+
+  @Test
+  def testMetadataLogDirFailureInZkShouldNotHaltBroker(): Unit = {
+// Given
+val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect, 
logDirCount = 2)
+val config = KafkaConfig.fromProps(props)
+val logDirFiles = config.logDirs.map(new File(_))
+val logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size)
+val logManager = TestUtils.createLogManager(logDirFiles, defaultConfig = 
new LogConfig(new Properties()), time = time)
+val mockZkClient = mock(classOf[KafkaZkClient])
+val replicaManager = new ReplicaManager(
+  metrics = metrics,
+  config = config,
+  time = time,
+  scheduler = time.scheduler,
+  logManager = logManager,
+  quotaManagers = quotaManager,
+  metadataCache = MetadataCache.zkMetadataCache(config.brokerId, 
config.interBrokerProtocolVersion),
+  logDirFailureChannel = logDirFailureChannel,
+  alterPartitionManager = alterPartitionManager,
+  threadNamePrefix = Option(this.getClass.getName),
+  zkClient = Option(mockZkClient),
+)
+
+try {
+  

(kafka) branch trunk updated: Revert "KAFKA-16101: Additional fixes on KRaft migration documentation (#15287)"

2024-02-01 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new b8ad609541b Revert "KAFKA-16101: Additional fixes on KRaft migration 
documentation (#15287)"
b8ad609541b is described below

commit b8ad609541bb8b224d16ce832a08f51017002bc1
Author: Colin P. McCabe 
AuthorDate: Thu Feb 1 16:17:18 2024 -0800

Revert "KAFKA-16101: Additional fixes on KRaft migration documentation 
(#15287)"

This reverts commit 46d76269fa72020d3d56d9e41d6f337563152837.
---
 docs/ops.html | 11 +--
 1 file changed, 5 insertions(+), 6 deletions(-)

diff --git a/docs/ops.html b/docs/ops.html
index eb8c7431665..8cb65549db6 100644
--- a/docs/ops.html
+++ b/docs/ops.html
@@ -4045,7 +4045,7 @@ listeners=CONTROLLER://:9093
 
   On each broker, remove the 
zookeeper.metadata.migration.enable,
   controller.listener.names, and 
controller.quorum.voters
-  configurations.
+  configurations, and replace node.id with 
broker.id.
   Then perform a rolling restart of all brokers.
 
 
@@ -4054,7 +4054,7 @@ listeners=CONTROLLER://:9093
   
 
 
-  It is important to perform the zookeeper-shell.sh step 
quickly together with rolling at least one broker (into pure ZooKeeper mode), 
to minimize the amount of
+  It is important to perform the zookeeper-shell.sh step 
quickly, to minimize the amount of
   time that the cluster lacks a controller.
 
   
@@ -4063,8 +4063,7 @@ listeners=CONTROLLER://:9093
 
   
 
-  On each broker, remove the process.roles 
configuration,
-  replace the node.id with broker.id and
+  On each broker, remove the process.roles 
configuration, and
   restore the zookeeper.connect configuration to its 
previous value.
   If your cluster requires other ZooKeeper configurations for 
brokers, such as
   zookeeper.ssl.protocol, re-add those configurations 
as well.
@@ -4080,7 +4079,7 @@ listeners=CONTROLLER://:9093
 
   On each broker, remove the 
zookeeper.metadata.migration.enable,
   controller.listener.names, and 
controller.quorum.voters
-  configurations.
+  configurations. Replace node.id with 
broker.id.
   Then perform a second rolling restart of all brokers.
 
 
@@ -4091,7 +4090,7 @@ listeners=CONTROLLER://:9093
 
   
 
-  It is important to perform the zookeeper-shell.sh 
step quickly together with rolling at least one broker (into pure 
ZooKeeper mode), to minimize the amount of
+  It is important to perform the zookeeper-shell.sh 
step quickly, to minimize the amount of
   time that the cluster lacks a controller.
 
 



(kafka) branch 3.7 updated: Revert "KAFKA-16101: Additional fixes on KRaft migration documentation (#15287)"

2024-02-01 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.7
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.7 by this push:
 new a1addb5668b Revert "KAFKA-16101: Additional fixes on KRaft migration 
documentation (#15287)"
a1addb5668b is described below

commit a1addb5668b146f3cc77b61bba9e8b6ad4e5d9a3
Author: Colin P. McCabe 
AuthorDate: Thu Feb 1 16:16:43 2024 -0800

Revert "KAFKA-16101: Additional fixes on KRaft migration documentation 
(#15287)"

This reverts commit f7882a2cda5117ddc7f3650c9f3655bcc89b2671.
---
 docs/ops.html | 11 +--
 1 file changed, 5 insertions(+), 6 deletions(-)

diff --git a/docs/ops.html b/docs/ops.html
index eb8c7431665..8cb65549db6 100644
--- a/docs/ops.html
+++ b/docs/ops.html
@@ -4045,7 +4045,7 @@ listeners=CONTROLLER://:9093
 
   On each broker, remove the 
zookeeper.metadata.migration.enable,
   controller.listener.names, and 
controller.quorum.voters
-  configurations.
+  configurations, and replace node.id with 
broker.id.
   Then perform a rolling restart of all brokers.
 
 
@@ -4054,7 +4054,7 @@ listeners=CONTROLLER://:9093
   
 
 
-  It is important to perform the zookeeper-shell.sh step 
quickly together with rolling at least one broker (into pure ZooKeeper mode), 
to minimize the amount of
+  It is important to perform the zookeeper-shell.sh step 
quickly, to minimize the amount of
   time that the cluster lacks a controller.
 
   
@@ -4063,8 +4063,7 @@ listeners=CONTROLLER://:9093
 
   
 
-  On each broker, remove the process.roles 
configuration,
-  replace the node.id with broker.id and
+  On each broker, remove the process.roles 
configuration, and
   restore the zookeeper.connect configuration to its 
previous value.
   If your cluster requires other ZooKeeper configurations for 
brokers, such as
   zookeeper.ssl.protocol, re-add those configurations 
as well.
@@ -4080,7 +4079,7 @@ listeners=CONTROLLER://:9093
 
   On each broker, remove the 
zookeeper.metadata.migration.enable,
   controller.listener.names, and 
controller.quorum.voters
-  configurations.
+  configurations. Replace node.id with 
broker.id.
   Then perform a second rolling restart of all brokers.
 
 
@@ -4091,7 +4090,7 @@ listeners=CONTROLLER://:9093
 
   
 
-  It is important to perform the zookeeper-shell.sh 
step quickly together with rolling at least one broker (into pure 
ZooKeeper mode), to minimize the amount of
+  It is important to perform the zookeeper-shell.sh 
step quickly, to minimize the amount of
   time that the cluster lacks a controller.
 
 



(kafka) branch 3.7 updated: KAFKA-16216: Reduce batch size for initial metadata load during ZK migration

2024-02-01 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.7
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.7 by this push:
 new 945f4b91dfc KAFKA-16216: Reduce batch size for initial metadata load 
during ZK migration
945f4b91dfc is described below

commit 945f4b91dfc2dd8ebfa49f04fba6230d01879d05
Author: David Arthur 
AuthorDate: Thu Feb 1 15:29:07 2024 -0800

KAFKA-16216: Reduce batch size for initial metadata load during ZK migration

During migration from ZK mode to KRaft mode, there is a step where the 
kcontrollers load all of the
data from ZK into the metadata log. Previously, we were using a batch size 
of 1000 for this, but
200 seems better. This PR also adds an internal configuration to control 
this batch size, for
testing purposes.

Reviewers: Colin P. McCabe 
---
 .../main/scala/kafka/server/ControllerServer.scala |  1 +
 core/src/main/scala/kafka/server/KafkaConfig.scala |  5 +
 .../metadata/migration/KRaftMigrationDriver.java   | 17 +++---
 .../migration/KRaftMigrationDriverTest.java| 26 +-
 4 files changed, 36 insertions(+), 13 deletions(-)

diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala 
b/core/src/main/scala/kafka/server/ControllerServer.scala
index fb7439d8e56..f229b1008aa 100644
--- a/core/src/main/scala/kafka/server/ControllerServer.scala
+++ b/core/src/main/scala/kafka/server/ControllerServer.scala
@@ -302,6 +302,7 @@ class ControllerServer(
   .setQuorumFeatures(quorumFeatures)
   .setConfigSchema(configSchema)
   .setControllerMetrics(quorumControllerMetrics)
+  .setMinMigrationBatchSize(config.migrationMetadataMinBatchSize)
   .setTime(time)
   .build()
 migrationDriver.start()
diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala 
b/core/src/main/scala/kafka/server/KafkaConfig.scala
index fbd27114c61..5ae8b236d04 100755
--- a/core/src/main/scala/kafka/server/KafkaConfig.scala
+++ b/core/src/main/scala/kafka/server/KafkaConfig.scala
@@ -92,6 +92,7 @@ object Defaults {
   /** KRaft mode configs */
   val EmptyNodeId: Int = -1
   val ServerMaxStartupTimeMs = Long.MaxValue
+  val MigrationMetadataMinBatchSize = 200
 
   /* Authorizer Configuration ***/
   val AuthorizerClassName = ""
@@ -403,6 +404,7 @@ object KafkaConfig {
 
   /** ZK to KRaft Migration configs */
   val MigrationEnabledProp = "zookeeper.metadata.migration.enable"
+  val MigrationMetadataMinBatchSizeProp = 
"zookeeper.metadata.migration.min.batch.size"
 
   /** Enable eligible leader replicas configs */
   val ElrEnabledProp = "eligible.leader.replicas.enable"
@@ -1260,6 +1262,8 @@ object KafkaConfig {
   .defineInternal(ServerMaxStartupTimeMsProp, LONG, 
Defaults.ServerMaxStartupTimeMs, atLeast(0), MEDIUM, ServerMaxStartupTimeMsDoc)
   .define(MigrationEnabledProp, BOOLEAN, false, HIGH, "Enable ZK to KRaft 
migration")
   .define(ElrEnabledProp, BOOLEAN, false, HIGH, "Enable the Eligible 
leader replicas")
+  .defineInternal(MigrationMetadataMinBatchSizeProp, INT, 
Defaults.MigrationMetadataMinBatchSize, atLeast(1),
+MEDIUM, "Soft minimum batch size to use when migrating metadata from 
ZooKeeper to KRaft")
 
   /* Authorizer Configuration ***/
   .define(AuthorizerClassNameProp, STRING, Defaults.AuthorizerClassName, 
new ConfigDef.NonNullValidator(), LOW, AuthorizerClassNameDoc)
@@ -1766,6 +1770,7 @@ class KafkaConfig private(doLog: Boolean, val props: 
java.util.Map[_, _], dynami
   def usesSelfManagedQuorum: Boolean = processRoles.nonEmpty
 
   val migrationEnabled: Boolean = getBoolean(KafkaConfig.MigrationEnabledProp)
+  val migrationMetadataMinBatchSize: Int = 
getInt(KafkaConfig.MigrationMetadataMinBatchSizeProp)
 
   val elrEnabled: Boolean = getBoolean(KafkaConfig.ElrEnabledProp)
 
diff --git 
a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java
 
b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java
index 534d0940195..f33f0790e0a 100644
--- 
a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java
+++ 
b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java
@@ -90,8 +90,6 @@ public class KRaftMigrationDriver implements 
MetadataPublisher {
  */
 final static int METADATA_COMMIT_MAX_WAIT_MS = 300_000;
 
-final static int MIGRATION_MIN_BATCH_SIZE = 1_000;
-
 private final Time time;
 private final Logger log;
 private final int nodeId;
@@ -110,6 +108,7 @@ public class KRaftMigrationDriver implements 
MetadataPublisher {
  * MetadataPublisher with MetadataLoader.
  */
 private final Consumer initialZkLoadHandler;
+private final int m

(kafka) branch trunk updated: KAFKA-16216: Reduce batch size for initial metadata load during ZK migration

2024-02-01 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 12ce9c7f98c KAFKA-16216: Reduce batch size for initial metadata load 
during ZK migration
12ce9c7f98c is described below

commit 12ce9c7f98c1617824d7bd86f9cc1f4560646e26
Author: David Arthur 
AuthorDate: Thu Feb 1 15:29:07 2024 -0800

KAFKA-16216: Reduce batch size for initial metadata load during ZK migration

During migration from ZK mode to KRaft mode, there is a step where the 
kcontrollers load all of the
data from ZK into the metadata log. Previously, we were using a batch size 
of 1000 for this, but
200 seems better. This PR also adds an internal configuration to control 
this batch size, for
testing purposes.

Reviewers: Colin P. McCabe 
---
 .../main/scala/kafka/server/ControllerServer.scala |  1 +
 core/src/main/scala/kafka/server/KafkaConfig.scala |  4 
 .../metadata/migration/KRaftMigrationDriver.java   | 17 +++---
 .../migration/KRaftMigrationDriverTest.java| 26 +-
 .../org/apache/kafka/server/config/Defaults.java   |  1 +
 5 files changed, 36 insertions(+), 13 deletions(-)

diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala 
b/core/src/main/scala/kafka/server/ControllerServer.scala
index 8f14d814b63..a882e54e52d 100644
--- a/core/src/main/scala/kafka/server/ControllerServer.scala
+++ b/core/src/main/scala/kafka/server/ControllerServer.scala
@@ -304,6 +304,7 @@ class ControllerServer(
   .setQuorumFeatures(quorumFeatures)
   .setConfigSchema(configSchema)
   .setControllerMetrics(quorumControllerMetrics)
+  .setMinMigrationBatchSize(config.migrationMetadataMinBatchSize)
   .setTime(time)
   .build()
 migrationDriver.start()
diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala 
b/core/src/main/scala/kafka/server/KafkaConfig.scala
index f7967ed8457..c6f51a000e2 100755
--- a/core/src/main/scala/kafka/server/KafkaConfig.scala
+++ b/core/src/main/scala/kafka/server/KafkaConfig.scala
@@ -166,6 +166,7 @@ object KafkaConfig {
 
   /** ZK to KRaft Migration configs */
   val MigrationEnabledProp = "zookeeper.metadata.migration.enable"
+  val MigrationMetadataMinBatchSizeProp = 
"zookeeper.metadata.migration.min.batch.size"
 
   /** Enable eligible leader replicas configs */
   val ElrEnabledProp = "eligible.leader.replicas.enable"
@@ -1029,6 +1030,8 @@ object KafkaConfig {
   .defineInternal(ServerMaxStartupTimeMsProp, LONG, 
Defaults.SERVER_MAX_STARTUP_TIME_MS, atLeast(0), MEDIUM, 
ServerMaxStartupTimeMsDoc)
   .define(MigrationEnabledProp, BOOLEAN, false, HIGH, "Enable ZK to KRaft 
migration")
   .define(ElrEnabledProp, BOOLEAN, false, HIGH, "Enable the Eligible 
leader replicas")
+  .defineInternal(MigrationMetadataMinBatchSizeProp, INT, 
Defaults.MIGRATION_METADATA_MIN_BATCH_SIZE, atLeast(1),
+MEDIUM, "Soft minimum batch size to use when migrating metadata from 
ZooKeeper to KRaft")
 
   /* Authorizer Configuration ***/
   .define(AuthorizerClassNameProp, STRING, Defaults.AUTHORIZER_CLASS_NAME, 
new ConfigDef.NonNullValidator(), LOW, AuthorizerClassNameDoc)
@@ -1538,6 +1541,7 @@ class KafkaConfig private(doLog: Boolean, val props: 
java.util.Map[_, _], dynami
   def usesSelfManagedQuorum: Boolean = processRoles.nonEmpty
 
   val migrationEnabled: Boolean = getBoolean(KafkaConfig.MigrationEnabledProp)
+  val migrationMetadataMinBatchSize: Int = 
getInt(KafkaConfig.MigrationMetadataMinBatchSizeProp)
 
   val elrEnabled: Boolean = getBoolean(KafkaConfig.ElrEnabledProp)
 
diff --git 
a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java
 
b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java
index 534d0940195..f33f0790e0a 100644
--- 
a/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java
+++ 
b/metadata/src/main/java/org/apache/kafka/metadata/migration/KRaftMigrationDriver.java
@@ -90,8 +90,6 @@ public class KRaftMigrationDriver implements 
MetadataPublisher {
  */
 final static int METADATA_COMMIT_MAX_WAIT_MS = 300_000;
 
-final static int MIGRATION_MIN_BATCH_SIZE = 1_000;
-
 private final Time time;
 private final Logger log;
 private final int nodeId;
@@ -110,6 +108,7 @@ public class KRaftMigrationDriver implements 
MetadataPublisher {
  * MetadataPublisher with MetadataLoader.
  */
 private final Consumer initialZkLoadHandler;
+private final int minBatchSize;
 private volatile MigrationDriverState migrationState;
 private volatile ZkMigrationLeadershipState migrationLeadershipState;
 private volatile MetadataImage image;
@@ -

(kafka) branch 3.7 updated: KAFKA-16162: resend broker registration on metadata update to IBP 3.7-IV2

2024-01-30 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.7
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.7 by this push:
 new cf6defb8b53 KAFKA-16162: resend broker registration on metadata update 
to IBP 3.7-IV2
cf6defb8b53 is described below

commit cf6defb8b5327ae76d2e53467a8570261bad15b2
Author: Gaurav Narula 
AuthorDate: Tue Jan 30 09:59:11 2024 -0800

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 

Reviewers: Omnia G H Ibrahim , Luke Chen 
, Colin P. McCabe 
---
 .../kafka/server/BrokerLifecycleManager.scala  |  13 +++
 .../src/main/scala/kafka/server/BrokerServer.scala |   4 +-
 .../server/metadata/BrokerMetadataPublisher.scala  |  15 ++-
 .../kafka/server/BrokerLifecycleManagerTest.scala  |  32 ++
 .../metadata/BrokerMetadataPublisherTest.scala | 111 -
 .../java/org/apache/kafka/common/DirectoryId.java  |  10 ++
 .../org/apache/kafka/common/DirectoryIdTest.java   |   6 ++
 7 files changed, 184 insertions(+), 7 deletions(-)

diff --git a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala 
b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala
index c3da9dcb7e6..bd80364274d 100644
--- a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala
+++ b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala
@@ -244,6 +244,19 @@ class BrokerLifecycleManager(
 eventQueue.append(new OfflineDirEvent(directory))
   }
 
+  def handleKraftJBODMetadataVersionUpdate(): Unit = {
+eventQueue.append(new KraftJBODMetadataVersionUpdateEvent())
+  }
+
+  private class KraftJBODMetadataVersionUpdateEvent extends EventQueue.Event {
+override def run(): Unit = {
+  if (!isZkBroker) {
+registered = false
+scheduleNextCommunicationImmediately()
+  }
+}
+  }
+
   def brokerEpoch: Long = _brokerEpoch
 
   def state: BrokerState = _state
diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala 
b/core/src/main/scala/kafka/server/BrokerServer.scala
index 593d28ac7ab..462b147a162 100644
--- a/core/src/main/scala/kafka/server/BrokerServer.scala
+++ b/core/src/main/scala/kafka/server/BrokerServer.scala
@@ -452,7 +452,9 @@ class BrokerServer(
   authorizer
 ),
 sharedServer.initialBrokerMetadataLoadFaultHandler,
-sharedServer.metadataPublishingFaultHandler)
+sharedServer.metadataPublishingFaultHandler,
+lifecycleManager
+  )
   metadataPublishers.add(brokerMetadataPublisher)
 
   // Register parts of the broker that can be reconfigured via dynamic 
configs.  This needs to
diff --git 
a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala 
b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala
index 21bb95d9ecb..e49b910d246 100644
--- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala
+++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala
@@ -20,7 +20,7 @@ package kafka.server.metadata
 import java.util.{OptionalInt, Properties}
 import kafka.coordinator.transaction.TransactionCoordinator
 import kafka.log.LogManager
-import kafka.server.{KafkaConfig, ReplicaManager, RequestLocal}
+import kafka.server.{BrokerLifecycleManager, KafkaConfig, ReplicaManager, 
RequestLocal}
 import kafka.utils.Logging
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.errors.TimeoutException
@@ -29,6 +29,7 @@ import org.apache.kafka.coordinator.group.GroupCoordinator
 import org.apache.kafka.image.loader.LoaderManifest
 import org.apache.kafka.image.publisher.MetadataPublisher
 import org.apache.kafka.image.{MetadataDelta, MetadataImage, TopicDelta}
+import org.apache.kafka.server.common.MetadataVersion
 import org.apache.kafka.server.fault.FaultHandler
 
 import java.util.concurrent.CompletableFuture
@@ -72,7 +73,8 @@ class BrokerMetadataPublisher(
   delegationTokenPublisher: DelegationTokenPublisher,
   aclPublisher: AclPublisher,
   fatalFaultHandler: FaultHandler,
-  metadataPublishingFaultHandler: FaultHandler
+  metadataPublishingFaultHandler: FaultHandler,
+  brokerLifecycleManager: BrokerLifecycleManager,
 ) extends MetadataPublisher with Logging {
   logIdent = s"[BrokerMetadataPublisher id=${config.nodeId}] "
 
@@ -130,6 +132,15 @@ class BrokerMetadataPublisher(
   Option(delta.featuresDelta()).foreach { featuresDelta =>
 featuresDelta.metadataVersionChange().ifPresent{ metadataVersion =>

(kafka) branch trunk updated: KAFKA-16162: resend broker registration on metadata update to IBP 3.7-IV2

2024-01-30 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 4c6f975ab3d KAFKA-16162: resend broker registration on metadata update 
to IBP 3.7-IV2
4c6f975ab3d is described below

commit 4c6f975ab3deba517c6621c4a051e9beb1240bbc
Author: Gaurav Narula 
AuthorDate: Tue Jan 30 09:59:11 2024 -0800

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 

Reviewers: Omnia G H Ibrahim , Luke Chen 
, Colin P. McCabe 
---
 .../kafka/server/BrokerLifecycleManager.scala  |  13 +++
 .../src/main/scala/kafka/server/BrokerServer.scala |   4 +-
 .../server/metadata/BrokerMetadataPublisher.scala  |  15 ++-
 .../kafka/server/BrokerLifecycleManagerTest.scala  |  32 ++
 .../metadata/BrokerMetadataPublisherTest.scala | 111 -
 .../java/org/apache/kafka/common/DirectoryId.java  |  10 ++
 .../org/apache/kafka/common/DirectoryIdTest.java   |   6 ++
 7 files changed, 184 insertions(+), 7 deletions(-)

diff --git a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala 
b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala
index c3da9dcb7e6..bd80364274d 100644
--- a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala
+++ b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala
@@ -244,6 +244,19 @@ class BrokerLifecycleManager(
 eventQueue.append(new OfflineDirEvent(directory))
   }
 
+  def handleKraftJBODMetadataVersionUpdate(): Unit = {
+eventQueue.append(new KraftJBODMetadataVersionUpdateEvent())
+  }
+
+  private class KraftJBODMetadataVersionUpdateEvent extends EventQueue.Event {
+override def run(): Unit = {
+  if (!isZkBroker) {
+registered = false
+scheduleNextCommunicationImmediately()
+  }
+}
+  }
+
   def brokerEpoch: Long = _brokerEpoch
 
   def state: BrokerState = _state
diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala 
b/core/src/main/scala/kafka/server/BrokerServer.scala
index 60b6c554f16..6ee44e3070a 100644
--- a/core/src/main/scala/kafka/server/BrokerServer.scala
+++ b/core/src/main/scala/kafka/server/BrokerServer.scala
@@ -452,7 +452,9 @@ class BrokerServer(
   authorizer
 ),
 sharedServer.initialBrokerMetadataLoadFaultHandler,
-sharedServer.metadataPublishingFaultHandler)
+sharedServer.metadataPublishingFaultHandler,
+lifecycleManager
+  )
   metadataPublishers.add(brokerMetadataPublisher)
 
   // Register parts of the broker that can be reconfigured via dynamic 
configs.  This needs to
diff --git 
a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala 
b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala
index 663afd226de..8edb2a1aa1a 100644
--- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala
+++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala
@@ -20,7 +20,7 @@ package kafka.server.metadata
 import java.util.{OptionalInt, Properties}
 import kafka.coordinator.transaction.TransactionCoordinator
 import kafka.log.LogManager
-import kafka.server.{KafkaConfig, ReplicaManager, RequestLocal}
+import kafka.server.{BrokerLifecycleManager, KafkaConfig, ReplicaManager, 
RequestLocal}
 import kafka.utils.Logging
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.errors.TimeoutException
@@ -29,6 +29,7 @@ import org.apache.kafka.coordinator.group.GroupCoordinator
 import org.apache.kafka.image.loader.LoaderManifest
 import org.apache.kafka.image.publisher.MetadataPublisher
 import org.apache.kafka.image.{MetadataDelta, MetadataImage, TopicDelta}
+import org.apache.kafka.server.common.MetadataVersion
 import org.apache.kafka.server.fault.FaultHandler
 
 import java.util.concurrent.CompletableFuture
@@ -72,7 +73,8 @@ class BrokerMetadataPublisher(
   delegationTokenPublisher: DelegationTokenPublisher,
   aclPublisher: AclPublisher,
   fatalFaultHandler: FaultHandler,
-  metadataPublishingFaultHandler: FaultHandler
+  metadataPublishingFaultHandler: FaultHandler,
+  brokerLifecycleManager: BrokerLifecycleManager,
 ) extends MetadataPublisher with Logging {
   logIdent = s"[BrokerMetadataPublisher id=${config.nodeId}] "
 
@@ -130,6 +132,15 @@ class BrokerMetadataPublisher(
   Option(delta.featuresDelta()).foreach { featuresDelta =>
 featuresDelta.metadataVersionChange().ifPresent{ metadataVersion =>

(kafka) branch 3.7 updated: KAFKA-14616: Fix stray replica of recreated topics in KRaft mode

2024-01-29 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.7
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.7 by this push:
 new 5a861075bdd KAFKA-14616: Fix stray replica of recreated topics in 
KRaft mode
5a861075bdd is described below

commit 5a861075bdde71f786cbb71949c213449d62af27
Author: Colin P. McCabe 
AuthorDate: Mon Jan 29 22:36:09 2024 -0800

KAFKA-14616: Fix stray replica of recreated topics in KRaft mode

When a broker is down, and a topic is deleted, this will result in that 
broker seeing "stray
replicas" the next time it starts up. These replicas contain data that used 
to be important, but
which now needs to be deleted. Stray replica deletion is handled during the 
initial metadata
publishing step on the broker.

Previously, we deleted these stray replicas after starting up BOTH 
LogManager and ReplicaManager.
However, this wasn't quite correct. The presence of the stray replicas 
confused ReplicaManager.
Instead, we should delete the stray replicas BEFORE starting ReplicaManager.

This bug triggered when a topic was deleted and re-created while a broker 
was down, and some of the
replicas of the re-created topic landed on that broker. The impact was that 
the stray replicas were
deleted, but the new replicas for the next iteration of the topic never got 
created. This, in turn,
led to persistent under-replication until the next time the broker was 
restarted.

Reviewers: Luke Chen , Omnia G H Ibrahim 
, Gaurav Narula 
---
 core/src/main/scala/kafka/log/LogManager.scala | 49 
 .../main/scala/kafka/server/ReplicaManager.scala   | 12 
 .../server/metadata/BrokerMetadataPublisher.scala  | 66 --
 .../kafka/server/KRaftClusterTest.scala| 57 ++-
 .../metadata/BrokerMetadataPublisherTest.scala |  6 +-
 5 files changed, 120 insertions(+), 70 deletions(-)

diff --git a/core/src/main/scala/kafka/log/LogManager.scala 
b/core/src/main/scala/kafka/log/LogManager.scala
index f84177d9717..48e739f6847 100755
--- a/core/src/main/scala/kafka/log/LogManager.scala
+++ b/core/src/main/scala/kafka/log/LogManager.scala
@@ -24,6 +24,7 @@ import java.util.concurrent.atomic.AtomicInteger
 import kafka.server.checkpoints.OffsetCheckpointFile
 import kafka.server.metadata.ConfigRepository
 import kafka.server._
+import kafka.server.metadata.BrokerMetadataPublisher.info
 import kafka.utils._
 import org.apache.kafka.common.{DirectoryId, KafkaException, TopicPartition, 
Uuid}
 import org.apache.kafka.common.utils.{KafkaThread, Time, Utils}
@@ -35,6 +36,7 @@ import scala.collection.mutable.ArrayBuffer
 import scala.util.{Failure, Success, Try}
 import kafka.utils.Implicits._
 import org.apache.kafka.common.config.TopicConfig
+import org.apache.kafka.image.TopicsImage
 import org.apache.kafka.metadata.properties.{MetaProperties, 
MetaPropertiesEnsemble, PropertiesUtils}
 
 import java.util.{OptionalLong, Properties}
@@ -563,6 +565,16 @@ class LogManager(logDirs: Seq[File],
 startupWithConfigOverrides(defaultConfig, 
fetchTopicConfigOverrides(defaultConfig, topicNames))
   }
 
+  def deleteStrayKRaftReplicas(
+brokerId: Int,
+image: TopicsImage
+  ): Unit = {
+val strayPartitions = findStrayReplicas(brokerId, image, allLogs)
+strayPartitions.foreach(topicPartition => {
+  asyncDelete(topicPartition, false, false, true)
+})
+  }
+
   // visible for testing
   @nowarn("cat=deprecation")
   private[log] def fetchTopicConfigOverrides(defaultConfig: LogConfig, 
topicNames: Set[String]): Map[String, LogConfig] = {
@@ -1522,4 +1534,41 @@ object LogManager {
   remoteStorageSystemEnable = 
config.remoteLogManagerConfig.enableRemoteStorageSystem())
   }
 
+  /**
+   * Find logs which should not be on the current broker, according to the 
metadata image.
+   *
+   * @param brokerIdThe ID of the current broker.
+   * @param newTopicsImage  The new topics image after broker has been reloaded
+   * @param logsA collection of Log objects.
+   *
+   * @return  The topic partitions which are no longer needed on this 
broker.
+   */
+  def findStrayReplicas(
+brokerId: Int,
+newTopicsImage: TopicsImage,
+logs: Iterable[UnifiedLog]
+  ): Iterable[TopicPartition] = {
+logs.flatMap { log =>
+  val topicId = log.topicId.getOrElse {
+throw new RuntimeException(s"The log dir $log does not have a topic 
ID, " +
+  "which is not allowed when running in KRaft mode.")
+  }
+
+  val partitionId = log.topicPartition.partition()
+  Option(newTopicsImage.getPartition(topicId, partitionId)) match {
+case Some(partition) =>
+  if (!partition.replicas.contains(brokerId)) {
+info(s"Found stray log dir $log

(kafka) branch trunk updated: KAFKA-14616: Fix stray replica of recreated topics in KRaft mode

2024-01-29 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new f7feb43af35 KAFKA-14616: Fix stray replica of recreated topics in 
KRaft mode
f7feb43af35 is described below

commit f7feb43af351efa25be37cb78814c19bd28a22a2
Author: Colin P. McCabe 
AuthorDate: Mon Jan 29 22:36:09 2024 -0800

KAFKA-14616: Fix stray replica of recreated topics in KRaft mode

When a broker is down, and a topic is deleted, this will result in that 
broker seeing "stray
replicas" the next time it starts up. These replicas contain data that used 
to be important, but
which now needs to be deleted. Stray replica deletion is handled during the 
initial metadata
publishing step on the broker.

Previously, we deleted these stray replicas after starting up BOTH 
LogManager and ReplicaManager.
However, this wasn't quite correct. The presence of the stray replicas 
confused ReplicaManager.
Instead, we should delete the stray replicas BEFORE starting ReplicaManager.

This bug triggered when a topic was deleted and re-created while a broker 
was down, and some of the
replicas of the re-created topic landed on that broker. The impact was that 
the stray replicas were
deleted, but the new replicas for the next iteration of the topic never got 
created. This, in turn,
led to persistent under-replication until the next time the broker was 
restarted.

Reviewers: Luke Chen , Omnia G H Ibrahim 
, Gaurav Narula 
---
 core/src/main/scala/kafka/log/LogManager.scala | 49 
 .../main/scala/kafka/server/ReplicaManager.scala   | 13 -
 .../server/metadata/BrokerMetadataPublisher.scala  | 66 --
 .../kafka/server/KRaftClusterTest.scala| 57 ++-
 .../metadata/BrokerMetadataPublisherTest.scala |  6 +-
 5 files changed, 120 insertions(+), 71 deletions(-)

diff --git a/core/src/main/scala/kafka/log/LogManager.scala 
b/core/src/main/scala/kafka/log/LogManager.scala
index deb77c62d75..5e657ccf379 100755
--- a/core/src/main/scala/kafka/log/LogManager.scala
+++ b/core/src/main/scala/kafka/log/LogManager.scala
@@ -24,6 +24,7 @@ import java.util.concurrent.atomic.AtomicInteger
 import kafka.server.checkpoints.OffsetCheckpointFile
 import kafka.server.metadata.ConfigRepository
 import kafka.server._
+import kafka.server.metadata.BrokerMetadataPublisher.info
 import kafka.utils._
 import org.apache.kafka.common.{DirectoryId, KafkaException, TopicPartition, 
Uuid}
 import org.apache.kafka.common.utils.{KafkaThread, Time, Utils}
@@ -35,6 +36,7 @@ import scala.collection.mutable.ArrayBuffer
 import scala.util.{Failure, Success, Try}
 import kafka.utils.Implicits._
 import org.apache.kafka.common.config.TopicConfig
+import org.apache.kafka.image.TopicsImage
 import org.apache.kafka.metadata.properties.{MetaProperties, 
MetaPropertiesEnsemble, PropertiesUtils}
 
 import java.util.{OptionalLong, Properties}
@@ -563,6 +565,16 @@ class LogManager(logDirs: Seq[File],
 startupWithConfigOverrides(defaultConfig, 
fetchTopicConfigOverrides(defaultConfig, topicNames))
   }
 
+  def deleteStrayKRaftReplicas(
+brokerId: Int,
+image: TopicsImage
+  ): Unit = {
+val strayPartitions = findStrayReplicas(brokerId, image, allLogs)
+strayPartitions.foreach(topicPartition => {
+  asyncDelete(topicPartition, false, false, true)
+})
+  }
+
   // visible for testing
   @nowarn("cat=deprecation")
   private[log] def fetchTopicConfigOverrides(defaultConfig: LogConfig, 
topicNames: Set[String]): Map[String, LogConfig] = {
@@ -1522,4 +1534,41 @@ object LogManager {
   remoteStorageSystemEnable = 
config.remoteLogManagerConfig.enableRemoteStorageSystem())
   }
 
+  /**
+   * Find logs which should not be on the current broker, according to the 
metadata image.
+   *
+   * @param brokerIdThe ID of the current broker.
+   * @param newTopicsImage  The new topics image after broker has been reloaded
+   * @param logsA collection of Log objects.
+   *
+   * @return  The topic partitions which are no longer needed on this 
broker.
+   */
+  def findStrayReplicas(
+brokerId: Int,
+newTopicsImage: TopicsImage,
+logs: Iterable[UnifiedLog]
+  ): Iterable[TopicPartition] = {
+logs.flatMap { log =>
+  val topicId = log.topicId.getOrElse {
+throw new RuntimeException(s"The log dir $log does not have a topic 
ID, " +
+  "which is not allowed when running in KRaft mode.")
+  }
+
+  val partitionId = log.topicPartition.partition()
+  Option(newTopicsImage.getPartition(topicId, partitionId)) match {
+case Some(partition) =>
+  if (!partition.replicas.contains(brokerId)) {
+info(s"Found stray log dir $log

(kafka) branch 3.7 updated: KAFKA-16101: Fix KRaft migration documentation (#15193)

2024-01-29 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.7
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.7 by this push:
 new e71992d3bea KAFKA-16101: Fix KRaft migration documentation (#15193)
e71992d3bea is described below

commit e71992d3bea1913969e3750f884fae5483d39eef
Author: Colin Patrick McCabe 
AuthorDate: Mon Jan 29 14:34:07 2024 -0800

KAFKA-16101: Fix KRaft migration documentation (#15193)

This PR fixes some bugs in the KRaft migration documentation and 
reorganizes it to be easier to read. (Specifically, there were some steps that 
were previously out of order.)

In order to keep it all straight, the revert documentation is now in the 
form of a table which maps the latest migration state to the actions which the 
system administrator should perform.

Reviewers: Luke Chen , David Arthur , 
Liu Zeyu , Paolo Patierno 
---
 docs/ops.html | 160 ++
 1 file changed, 139 insertions(+), 21 deletions(-)

diff --git a/docs/ops.html b/docs/ops.html
index f943dbed630..8cb65549db6 100644
--- a/docs/ops.html
+++ b/docs/ops.html
@@ -3874,7 +3874,7 @@ zookeeper.connect=localhost:2181
   Note: The KRaft cluster node.id values must be different 
from any existing ZK broker broker.id.
   In KRaft-mode, the brokers and controllers share the same Node ID 
namespace.
 
-  Enabling the migration on the brokers
+  Enter Migration Mode on the Brokers
   
 Once the KRaft controller quorum has been started, the brokers will need 
to be reconfigured and restarted. Brokers
 may be restarted in a rolling fashion to avoid impacting cluster 
availability. Each broker requires the
@@ -3919,9 +3919,10 @@ controller.listener.names=CONTROLLER
 
   Migrating brokers to KRaft
   
-Once the KRaft controller completes the metadata migration, the brokers 
will still be running in ZK mode. While the
-KRaft controller is in migration mode, it will continue sending controller 
RPCs to the ZK mode brokers. This includes
-RPCs like UpdateMetadata and LeaderAndIsr.
+Once the KRaft controller completes the metadata migration, the brokers 
will still be running
+in ZooKeeper mode. While the KRaft controller is in migration mode, it 
will continue sending
+controller RPCs to the ZooKeeper mode brokers. This includes RPCs like 
UpdateMetadata and
+LeaderAndIsr.
   
 
   
@@ -3956,29 +3957,16 @@ controller.listener.names=CONTROLLER
 Each broker is restarted with a KRaft configuration until the entire 
cluster is running in KRaft mode.
   
 
-  Reverting to ZooKeeper mode During the Migration
-While the cluster is still in migration mode, it is possible to revert to 
ZK mode. In order to do this:
-
-  
-For each KRaft broker:
-
-  Stop the broker.
-  Remove the __cluster_metadata directory on the broker.
-  Remove the zookeeper.metadata.migration.enable 
configuration and the KRaft controllers related configurations like 
controller.quorum.voters
-and controller.listener.names from the broker 
configuration properties file.
-  Restart the broker in ZooKeeper mode.
-
-  
-  Take down the KRaft quorum.
-  Using ZooKeeper shell, delete the controller node using rmr 
/controller, so that a ZooKeeper-based broker can become the next 
controller.
-
-
   Finalizing the migration
   
 Once all brokers have been restarted in KRaft mode, the last step to 
finalize the migration is to take the
 KRaft controllers out of migration mode. This is done by removing the 
"zookeeper.metadata.migration.enable"
 property from each of their configs and restarting them one at a time.
   
+  
+Once the migration has been finalized, you can safely deprovision your 
ZooKeeper cluster, assuming you are
+not using it for anything else. After this point, it is no longer possible 
to revert to ZooKeeper mode.
+  
 
   
 # Sample KRaft cluster controller.properties listening on 9093
@@ -3996,6 +3984,136 @@ listeners=CONTROLLER://:9093
 
 # Other configs ...
 
+  Reverting to ZooKeeper mode During the Migration
+  
+While the cluster is still in migration mode, it is possible to revert to 
ZooKeeper mode.  The process
+to follow depends on how far the migration has progressed. In order to 
find out how to revert,
+select the final migration step that you have completed in 
this table.
+  
+  
+Note that the directions given here assume that each step was fully 
completed, and they were
+done in order. So, for example, we assume that if "Enter Migration Mode on 
the Brokers" was
+completed, "Provisioning the KRaft controller quorum" was also fully 
completed previously.
+  
+  
+If you did not fully complete any step, back out whatever you have done 
and then

(kafka) branch 3.7 updated: KAFKA-16171: Fix ZK migration controller race #15238

2024-01-29 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.7
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.7 by this push:
 new b7098722994 KAFKA-16171: Fix ZK migration controller race #15238
b7098722994 is described below

commit b7098722994c7fc14ccc369b030123a8a6dc156b
Author: David Arthur 
AuthorDate: Fri Jan 19 13:38:37 2024 -0500

KAFKA-16171: Fix ZK migration controller race #15238

This patch causes the active KRaftMigrationDriver to reload the /migration 
ZK state after electing
itself as the leader in ZK. This closes a race condition where the previous 
active controller could
make an update to /migration after the new leader was elected. The update 
race was not actually a
problem regarding the data since both controllers would be syncing the same 
state from KRaft to ZK,
but the change to the znode causes the new controller to fail on the zk 
version check on
/migration.

This patch also fixes a as-yet-unseen bug where the active controllers 
failing to elect itself via
claimControllerLeadership would not retry.

Reviewers: Colin P. McCabe 
---
 .../kafka/zk/ZkMigrationFailoverTest.scala | 276 +
 .../metadata/migration/KRaftMigrationDriver.java   |  14 +-
 .../migration/ZkMigrationLeadershipState.java  |   9 +-
 .../migration/CapturingMigrationClient.java|  12 +-
 4 files changed, 304 insertions(+), 7 deletions(-)

diff --git 
a/core/src/test/scala/integration/kafka/zk/ZkMigrationFailoverTest.scala 
b/core/src/test/scala/integration/kafka/zk/ZkMigrationFailoverTest.scala
new file mode 100644
index 000..20dd5d91a83
--- /dev/null
+++ b/core/src/test/scala/integration/kafka/zk/ZkMigrationFailoverTest.scala
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.zk
+
+import kafka.utils.{Logging, PasswordEncoder, TestUtils}
+import org.apache.kafka.common.Uuid
+import org.apache.kafka.common.metadata.{FeatureLevelRecord, TopicRecord}
+import org.apache.kafka.common.utils.{Time, Utils}
+import org.apache.kafka.controller.QuorumFeatures
+import org.apache.kafka.controller.metrics.QuorumControllerMetrics
+import org.apache.kafka.image.loader.LogDeltaManifest
+import org.apache.kafka.image.publisher.MetadataPublisher
+import org.apache.kafka.image.{MetadataDelta, MetadataImage, 
MetadataProvenance}
+import org.apache.kafka.metadata.KafkaConfigSchema
+import org.apache.kafka.metadata.migration._
+import org.apache.kafka.raft.{LeaderAndEpoch, OffsetAndEpoch}
+import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion}
+import org.apache.kafka.server.fault.FaultHandler
+import org.apache.zookeeper.client.ZKClientConfig
+import org.junit.jupiter.api.Assertions.{assertTrue, fail}
+import org.junit.jupiter.api.Test
+
+import java.util
+import java.util.concurrent.{CompletableFuture, TimeUnit}
+import java.util.{Optional, OptionalInt}
+import scala.collection.mutable
+
+class ZkMigrationFailoverTest extends Logging {
+
+  class CapturingFaultHandler(nodeId: Int) extends FaultHandler {
+val faults = mutable.Buffer[Throwable]()
+var future: CompletableFuture[Throwable] = 
CompletableFuture.completedFuture(new RuntimeException())
+var waitingForMsg = ""
+
+override def handleFault(failureMessage: String, cause: Throwable): 
RuntimeException = {
+  error(s"Fault handled on node $nodeId", cause)
+  faults.append(cause)
+  if (!future.isDone && cause.getMessage.contains(waitingForMsg)) {
+future.complete(cause)
+  }
+  new RuntimeException(cause)
+}
+
+def checkAndClear(verifier: (Seq[Throwable]) => Unit): Unit = {
+  val faultsSoFar = faults.toSeq
+  try {
+verifier.apply(faultsSoFar)
+  } catch {
+case ae: AssertionError => fail(s"Assertion failed. Faults on $nodeId 
were: $faultsSoFar", ae)
+  }
+}
+
+def waitForError(message: String): CompletableFuture[Throwable] = {
+  future = new CompletableFuture[Throwable]()
+  waitingForMsg = messag

(kafka) branch trunk updated: KAFKA-16101: Fix KRaft migration documentation (#15193)

2024-01-29 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 5b422613a66 KAFKA-16101: Fix KRaft migration documentation (#15193)
5b422613a66 is described below

commit 5b422613a664740282440a5c65c7589095149d35
Author: Colin Patrick McCabe 
AuthorDate: Mon Jan 29 14:34:07 2024 -0800

KAFKA-16101: Fix KRaft migration documentation (#15193)

This PR fixes some bugs in the KRaft migration documentation and 
reorganizes it to be easier to read. (Specifically, there were some steps that 
were previously out of order.)

In order to keep it all straight, the revert documentation is now in the 
form of a table which maps the latest migration state to the actions which the 
system administrator should perform.

Reviewers: Luke Chen , David Arthur , 
Liu Zeyu , Paolo Patierno 
---
 docs/ops.html | 160 ++
 1 file changed, 139 insertions(+), 21 deletions(-)

diff --git a/docs/ops.html b/docs/ops.html
index f943dbed630..8cb65549db6 100644
--- a/docs/ops.html
+++ b/docs/ops.html
@@ -3874,7 +3874,7 @@ zookeeper.connect=localhost:2181
   Note: The KRaft cluster node.id values must be different 
from any existing ZK broker broker.id.
   In KRaft-mode, the brokers and controllers share the same Node ID 
namespace.
 
-  Enabling the migration on the brokers
+  Enter Migration Mode on the Brokers
   
 Once the KRaft controller quorum has been started, the brokers will need 
to be reconfigured and restarted. Brokers
 may be restarted in a rolling fashion to avoid impacting cluster 
availability. Each broker requires the
@@ -3919,9 +3919,10 @@ controller.listener.names=CONTROLLER
 
   Migrating brokers to KRaft
   
-Once the KRaft controller completes the metadata migration, the brokers 
will still be running in ZK mode. While the
-KRaft controller is in migration mode, it will continue sending controller 
RPCs to the ZK mode brokers. This includes
-RPCs like UpdateMetadata and LeaderAndIsr.
+Once the KRaft controller completes the metadata migration, the brokers 
will still be running
+in ZooKeeper mode. While the KRaft controller is in migration mode, it 
will continue sending
+controller RPCs to the ZooKeeper mode brokers. This includes RPCs like 
UpdateMetadata and
+LeaderAndIsr.
   
 
   
@@ -3956,29 +3957,16 @@ controller.listener.names=CONTROLLER
 Each broker is restarted with a KRaft configuration until the entire 
cluster is running in KRaft mode.
   
 
-  Reverting to ZooKeeper mode During the Migration
-While the cluster is still in migration mode, it is possible to revert to 
ZK mode. In order to do this:
-
-  
-For each KRaft broker:
-
-  Stop the broker.
-  Remove the __cluster_metadata directory on the broker.
-  Remove the zookeeper.metadata.migration.enable 
configuration and the KRaft controllers related configurations like 
controller.quorum.voters
-and controller.listener.names from the broker 
configuration properties file.
-  Restart the broker in ZooKeeper mode.
-
-  
-  Take down the KRaft quorum.
-  Using ZooKeeper shell, delete the controller node using rmr 
/controller, so that a ZooKeeper-based broker can become the next 
controller.
-
-
   Finalizing the migration
   
 Once all brokers have been restarted in KRaft mode, the last step to 
finalize the migration is to take the
 KRaft controllers out of migration mode. This is done by removing the 
"zookeeper.metadata.migration.enable"
 property from each of their configs and restarting them one at a time.
   
+  
+Once the migration has been finalized, you can safely deprovision your 
ZooKeeper cluster, assuming you are
+not using it for anything else. After this point, it is no longer possible 
to revert to ZooKeeper mode.
+  
 
   
 # Sample KRaft cluster controller.properties listening on 9093
@@ -3996,6 +3984,136 @@ listeners=CONTROLLER://:9093
 
 # Other configs ...
 
+  Reverting to ZooKeeper mode During the Migration
+  
+While the cluster is still in migration mode, it is possible to revert to 
ZooKeeper mode.  The process
+to follow depends on how far the migration has progressed. In order to 
find out how to revert,
+select the final migration step that you have completed in 
this table.
+  
+  
+Note that the directions given here assume that each step was fully 
completed, and they were
+done in order. So, for example, we assume that if "Enter Migration Mode on 
the Brokers" was
+completed, "Provisioning the KRaft controller quorum" was also fully 
completed previously.
+  
+  
+If you did not fully complete any step, back out whatever you have done 
and then

(kafka) branch KAFKA-16101 deleted (was 52fbe9bd5c6)

2024-01-29 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a change to branch KAFKA-16101
in repository https://gitbox.apache.org/repos/asf/kafka.git


 was 52fbe9bd5c6 KAFKA-16101: KRaft migration documentation is incorrect

The revisions that were on this branch are still contained in
other references; therefore, this change does not discard any commits
from the repository.



(kafka) branch KAFKA-16101 updated (30cbfd1609f -> 52fbe9bd5c6)

2024-01-29 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a change to branch KAFKA-16101
in repository https://gitbox.apache.org/repos/asf/kafka.git


 discard 30cbfd1609f fix
 discard 7690f2abed5 address review comments
 discard 6233d844c4a add table
 discard 6ec8ad0ad54 KAFKA-16101: KRaft migration documentation is incorrect
 add 52fbe9bd5c6 KAFKA-16101: KRaft migration documentation is incorrect

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (30cbfd1609f)
\
 N -- N -- N   refs/heads/KAFKA-16101 (52fbe9bd5c6)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

No new revisions were added by this update.

Summary of changes:
 docs/ops.html | 32 
 1 file changed, 16 insertions(+), 16 deletions(-)



(kafka) branch trunk updated: KAFKA-16171: Fix ZK migration controller race #15238

2024-01-29 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 16ed7357b18 KAFKA-16171: Fix ZK migration controller race #15238
16ed7357b18 is described below

commit 16ed7357b1899d4711c93c1fa094369f8b92480b
Author: David Arthur 
AuthorDate: Fri Jan 19 13:38:37 2024 -0500

KAFKA-16171: Fix ZK migration controller race #15238

This patch causes the active KRaftMigrationDriver to reload the /migration 
ZK state after electing
itself as the leader in ZK. This closes a race condition where the previous 
active controller could
make an update to /migration after the new leader was elected. The update 
race was not actually a
problem regarding the data since both controllers would be syncing the same 
state from KRaft to ZK,
but the change to the znode causes the new controller to fail on the zk 
version check on
/migration.

This patch also fixes a as-yet-unseen bug where the active controllers 
failing to elect itself via
claimControllerLeadership would not retry.

Reviewers: Colin P. McCabe 
---
 .../kafka/zk/ZkMigrationFailoverTest.scala | 276 +
 .../metadata/migration/KRaftMigrationDriver.java   |  14 +-
 .../migration/ZkMigrationLeadershipState.java  |   9 +-
 .../migration/CapturingMigrationClient.java|  12 +-
 4 files changed, 304 insertions(+), 7 deletions(-)

diff --git 
a/core/src/test/scala/integration/kafka/zk/ZkMigrationFailoverTest.scala 
b/core/src/test/scala/integration/kafka/zk/ZkMigrationFailoverTest.scala
new file mode 100644
index 000..20dd5d91a83
--- /dev/null
+++ b/core/src/test/scala/integration/kafka/zk/ZkMigrationFailoverTest.scala
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package kafka.zk
+
+import kafka.utils.{Logging, PasswordEncoder, TestUtils}
+import org.apache.kafka.common.Uuid
+import org.apache.kafka.common.metadata.{FeatureLevelRecord, TopicRecord}
+import org.apache.kafka.common.utils.{Time, Utils}
+import org.apache.kafka.controller.QuorumFeatures
+import org.apache.kafka.controller.metrics.QuorumControllerMetrics
+import org.apache.kafka.image.loader.LogDeltaManifest
+import org.apache.kafka.image.publisher.MetadataPublisher
+import org.apache.kafka.image.{MetadataDelta, MetadataImage, 
MetadataProvenance}
+import org.apache.kafka.metadata.KafkaConfigSchema
+import org.apache.kafka.metadata.migration._
+import org.apache.kafka.raft.{LeaderAndEpoch, OffsetAndEpoch}
+import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion}
+import org.apache.kafka.server.fault.FaultHandler
+import org.apache.zookeeper.client.ZKClientConfig
+import org.junit.jupiter.api.Assertions.{assertTrue, fail}
+import org.junit.jupiter.api.Test
+
+import java.util
+import java.util.concurrent.{CompletableFuture, TimeUnit}
+import java.util.{Optional, OptionalInt}
+import scala.collection.mutable
+
+class ZkMigrationFailoverTest extends Logging {
+
+  class CapturingFaultHandler(nodeId: Int) extends FaultHandler {
+val faults = mutable.Buffer[Throwable]()
+var future: CompletableFuture[Throwable] = 
CompletableFuture.completedFuture(new RuntimeException())
+var waitingForMsg = ""
+
+override def handleFault(failureMessage: String, cause: Throwable): 
RuntimeException = {
+  error(s"Fault handled on node $nodeId", cause)
+  faults.append(cause)
+  if (!future.isDone && cause.getMessage.contains(waitingForMsg)) {
+future.complete(cause)
+  }
+  new RuntimeException(cause)
+}
+
+def checkAndClear(verifier: (Seq[Throwable]) => Unit): Unit = {
+  val faultsSoFar = faults.toSeq
+  try {
+verifier.apply(faultsSoFar)
+  } catch {
+case ae: AssertionError => fail(s"Assertion failed. Faults on $nodeId 
were: $faultsSoFar", ae)
+  }
+}
+
+def waitForError(message: String): CompletableFuture[Throwable] = {
+  future = new CompletableFuture[Throwable]()
+  waitingForMsg =

(kafka) branch KAFKA-16101 updated (7690f2abed5 -> 30cbfd1609f)

2024-01-29 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a change to branch KAFKA-16101
in repository https://gitbox.apache.org/repos/asf/kafka.git


from 7690f2abed5 address review comments
 add 30cbfd1609f fix

No new revisions were added by this update.

Summary of changes:
 docs/ops.html | 5 -
 1 file changed, 4 insertions(+), 1 deletion(-)



(kafka) branch KAFKA-16101 updated (6233d844c4a -> 7690f2abed5)

2024-01-19 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a change to branch KAFKA-16101
in repository https://gitbox.apache.org/repos/asf/kafka.git


from 6233d844c4a add table
 add 7690f2abed5 address review comments

No new revisions were added by this update.

Summary of changes:
 docs/ops.html | 117 +++---
 1 file changed, 88 insertions(+), 29 deletions(-)



(kafka) branch KAFKA-16101 updated (6ec8ad0ad54 -> 6233d844c4a)

2024-01-18 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a change to branch KAFKA-16101
in repository https://gitbox.apache.org/repos/asf/kafka.git


from 6ec8ad0ad54 KAFKA-16101: KRaft migration documentation is incorrect
 add 6233d844c4a add table

No new revisions were added by this update.

Summary of changes:
 docs/ops.html | 90 ++-
 1 file changed, 70 insertions(+), 20 deletions(-)



(kafka) branch 3.7 updated: KAFKA-16078: Be more consistent about getting the latest MetadataVersion

2024-01-17 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.7
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.7 by this push:
 new 3c92330274a KAFKA-16078: Be more consistent about getting the latest 
MetadataVersion
3c92330274a is described below

commit 3c92330274a6d5ac7b2923b27f15b8d1c426da15
Author: David Arthur 
AuthorDate: Wed Jan 17 17:59:22 2024 -0500

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 , Ismael Juma 

---
 .../kafka/server/builders/LogManagerBuilder.java   |  2 +-
 core/src/main/scala/kafka/log/UnifiedLog.scala |  4 +-
 .../scala/kafka/server/ApiVersionManager.scala |  2 +-
 .../main/scala/kafka/server/BrokerFeatures.scala   | 15 ++--
 .../src/main/scala/kafka/server/BrokerServer.scala |  2 +-
 core/src/main/scala/kafka/server/KafkaConfig.scala |  2 +-
 core/src/test/java/kafka/test/ClusterConfig.java   |  2 +-
 .../java/kafka/testkit/KafkaClusterTestKit.java|  1 +
 core/src/test/java/kafka/testkit/TestKitNodes.java |  2 +-
 .../kafka/admin/ConfigCommandIntegrationTest.scala |  2 +-
 .../kafka/server/KRaftClusterTest.scala|  4 +-
 .../MetadataRequestBetweenDifferentIbpTest.scala   |  4 +-
 .../server/MetadataVersionIntegrationTest.scala|  4 +-
 .../kafka/server/QuorumTestHarness.scala   |  2 +-
 .../unit/kafka/cluster/AbstractPartitionTest.scala |  2 +-
 .../unit/kafka/cluster/PartitionLockTest.scala |  2 +-
 .../scala/unit/kafka/cluster/PartitionTest.scala   | 26 +++
 .../controller/ControllerChannelManagerTest.scala  |  8 +--
 .../controller/ControllerIntegrationTest.scala |  4 +-
 .../group/GroupMetadataManagerTest.scala   | 12 ++--
 .../test/scala/unit/kafka/log/LogLoaderTest.scala  |  2 +-
 .../scala/unit/kafka/log/LogValidatorTest.scala| 80 +++---
 .../unit/kafka/network/SocketServerTest.scala  |  2 +-
 .../kafka/security/auth/ZkAuthorizationTest.scala  |  2 +-
 .../server/AbstractApiVersionsRequestTest.scala|  6 +-
 .../AlterUserScramCredentialsRequestTest.scala |  2 +-
 .../unit/kafka/server/ApiVersionManagerTest.scala  |  4 +-
 .../unit/kafka/server/ApiVersionsRequestTest.scala | 16 -
 .../unit/kafka/server/BrokerFeaturesTest.scala | 10 +--
 .../unit/kafka/server/ControllerApisTest.scala |  2 +-
 .../kafka/server/FinalizedFeatureCacheTest.scala   | 10 +--
 .../FinalizedFeatureChangeListenerTest.scala   |  2 +-
 .../scala/unit/kafka/server/KafkaApisTest.scala|  8 +--
 .../scala/unit/kafka/server/KafkaConfigTest.scala  |  4 +-
 .../unit/kafka/server/KafkaRaftServerTest.scala|  6 +-
 .../scala/unit/kafka/server/KafkaServerTest.scala  |  2 +-
 .../unit/kafka/server/MetadataCacheTest.scala  |  6 +-
 .../server/ReplicaAlterLogDirsThreadTest.scala |  2 +-
 .../kafka/server/ReplicaFetcherThreadTest.scala|  4 +-
 .../server/ReplicaManagerConcurrencyTest.scala |  2 +-
 .../unit/kafka/server/ReplicaManagerTest.scala |  2 +-
 ...chDrivenReplicationProtocolAcceptanceTest.scala |  2 +-
 .../scala/unit/kafka/tools/StorageToolTest.scala   | 16 ++---
 .../test/scala/unit/kafka/utils/TestUtils.scala|  4 +-
 .../scala/unit/kafka/zk/KafkaZkClientTest.scala|  2 +-
 .../group/GroupMetadataManagerTest.java| 54 +++
 .../group/OffsetMetadataManagerTest.java   |  2 +-
 .../jmh/fetcher/ReplicaFetcherThreadBenchmark.java |  4 +-
 .../metadata/KRaftMetadataRequestBenchmark.java|  2 +-
 .../jmh/metadata/MetadataRequestBenchmark.java |  4 +-
 .../partition/PartitionMakeFollowerBenchmark.java  |  4 +-
 .../UpdateFollowerFetchStateBenchmark.java |  4 +-
 .../CompressedRecordBatchValidationBenchmark.java  |  2 +-
 ...UncompressedRecordBatchValidationBenchmark.java |  2 +-
 .../apache/kafka/jmh/server/CheckpointBench.java   |  2 +-
 .../kafka/jmh/server/PartitionCreationBench.java   |  2 +-
 .../kafka/controller/FeatureControlManager.java|  4 +-
 .../apache/kafka/controller/QuorumFeatures.java|  4 +-
 .../kafka/image/writer/ImageWriterOptions.java |  2 +-
 .../kafka/metadata/ControllerRegistration.java |  2 +-
 .../metadata/bootstrap/BootstrapDirectory.java |  2 +-
 .../controller/ClusterControlManagerTest.java  |  2 +-
 .../controller/FeatureControlManagerTest.java  |  2 +-
 .../controller/PartitionChangeBuilderTest.java |  2 +-
 .../QuorumControllerIntegrationTestUtils.java  |  4 +-
 .../kafka/controller/QuorumControllerTestEnv.java

(kafka) branch trunk updated: KAFKA-16078: Be more consistent about getting the latest MetadataVersion

2024-01-17 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 7bf7fd99a50 KAFKA-16078: Be more consistent about getting the latest 
MetadataVersion
7bf7fd99a50 is described below

commit 7bf7fd99a50ba2dd71c41cd4c3623a3adce092b6
Author: David Arthur 
AuthorDate: Wed Jan 17 17:59:22 2024 -0500

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 , Ismael Juma 

---
 .../kafka/server/builders/LogManagerBuilder.java   |  2 +-
 core/src/main/scala/kafka/log/UnifiedLog.scala |  4 +-
 .../scala/kafka/server/ApiVersionManager.scala |  2 +-
 .../main/scala/kafka/server/BrokerFeatures.scala   | 15 ++--
 .../src/main/scala/kafka/server/BrokerServer.scala |  2 +-
 core/src/main/scala/kafka/server/KafkaConfig.scala |  2 +-
 core/src/test/java/kafka/test/ClusterConfig.java   |  2 +-
 .../java/kafka/testkit/KafkaClusterTestKit.java|  1 +
 core/src/test/java/kafka/testkit/TestKitNodes.java |  2 +-
 .../kafka/admin/ConfigCommandIntegrationTest.scala |  2 +-
 .../kafka/server/KRaftClusterTest.scala|  4 +-
 .../MetadataRequestBetweenDifferentIbpTest.scala   |  4 +-
 .../server/MetadataVersionIntegrationTest.scala|  4 +-
 .../kafka/server/QuorumTestHarness.scala   |  2 +-
 .../unit/kafka/cluster/AbstractPartitionTest.scala |  2 +-
 .../unit/kafka/cluster/PartitionLockTest.scala |  2 +-
 .../scala/unit/kafka/cluster/PartitionTest.scala   | 26 +++
 .../controller/ControllerChannelManagerTest.scala  |  8 +--
 .../controller/ControllerIntegrationTest.scala |  4 +-
 .../group/GroupMetadataManagerTest.scala   | 12 ++--
 .../test/scala/unit/kafka/log/LogLoaderTest.scala  |  2 +-
 .../scala/unit/kafka/log/LogValidatorTest.scala| 80 +++---
 .../unit/kafka/network/SocketServerTest.scala  |  2 +-
 .../kafka/security/auth/ZkAuthorizationTest.scala  |  2 +-
 .../server/AbstractApiVersionsRequestTest.scala|  6 +-
 .../AlterUserScramCredentialsRequestTest.scala |  2 +-
 .../unit/kafka/server/ApiVersionManagerTest.scala  |  4 +-
 .../unit/kafka/server/ApiVersionsRequestTest.scala | 16 -
 .../unit/kafka/server/BrokerFeaturesTest.scala | 10 +--
 .../unit/kafka/server/ControllerApisTest.scala |  2 +-
 .../kafka/server/FinalizedFeatureCacheTest.scala   | 10 +--
 .../FinalizedFeatureChangeListenerTest.scala   |  2 +-
 .../scala/unit/kafka/server/KafkaApisTest.scala|  8 +--
 .../scala/unit/kafka/server/KafkaConfigTest.scala  |  4 +-
 .../unit/kafka/server/KafkaRaftServerTest.scala|  6 +-
 .../scala/unit/kafka/server/KafkaServerTest.scala  |  2 +-
 .../unit/kafka/server/MetadataCacheTest.scala  |  6 +-
 .../server/ReplicaAlterLogDirsThreadTest.scala |  2 +-
 .../kafka/server/ReplicaFetcherThreadTest.scala|  4 +-
 .../server/ReplicaManagerConcurrencyTest.scala |  2 +-
 ...chDrivenReplicationProtocolAcceptanceTest.scala |  2 +-
 .../scala/unit/kafka/tools/StorageToolTest.scala   | 16 ++---
 .../test/scala/unit/kafka/utils/TestUtils.scala|  4 +-
 .../scala/unit/kafka/zk/KafkaZkClientTest.scala|  2 +-
 .../group/GroupMetadataManagerTest.java| 54 +++
 .../group/OffsetMetadataManagerTest.java   |  2 +-
 .../jmh/fetcher/ReplicaFetcherThreadBenchmark.java |  4 +-
 .../metadata/KRaftMetadataRequestBenchmark.java|  2 +-
 .../jmh/metadata/MetadataRequestBenchmark.java |  4 +-
 .../partition/PartitionMakeFollowerBenchmark.java  |  4 +-
 .../UpdateFollowerFetchStateBenchmark.java |  4 +-
 .../CompressedRecordBatchValidationBenchmark.java  |  2 +-
 ...UncompressedRecordBatchValidationBenchmark.java |  2 +-
 .../apache/kafka/jmh/server/CheckpointBench.java   |  2 +-
 .../kafka/jmh/server/PartitionCreationBench.java   |  2 +-
 .../kafka/controller/FeatureControlManager.java|  4 +-
 .../apache/kafka/controller/QuorumFeatures.java|  4 +-
 .../kafka/image/writer/ImageWriterOptions.java |  2 +-
 .../kafka/metadata/ControllerRegistration.java |  2 +-
 .../metadata/bootstrap/BootstrapDirectory.java |  2 +-
 .../controller/ClusterControlManagerTest.java  |  2 +-
 .../controller/FeatureControlManagerTest.java  |  2 +-
 .../controller/PartitionChangeBuilderTest.java |  2 +-
 .../QuorumControllerIntegrationTestUtils.java  |  4 +-
 .../kafka/controller/QuorumControllerTestEnv.java  |  2 +-
 .../kafka/controller/QuorumFeaturesTest.java

(kafka) branch 3.7 updated: KAFKA-16131: Only update directoryIds if the metadata version supports DirectoryAssignment (#15197)

2024-01-17 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.7
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.7 by this push:
 new 2ff3ae5bed4 KAFKA-16131: Only update directoryIds if the metadata 
version supports DirectoryAssignment (#15197)
2ff3ae5bed4 is described below

commit 2ff3ae5bed423f0f8212a949df28a421f97f2ab9
Author: Proven Provenzano <93720617+pprovenz...@users.noreply.github.com>
AuthorDate: Wed Jan 17 17:06:38 2024 -0500

KAFKA-16131: Only update directoryIds if the metadata version supports 
DirectoryAssignment (#15197)

We only want to send directory assignments if the metadata version supports 
it.

Reviewers: Colin P. McCabe 
---
 core/src/main/scala/kafka/server/ReplicaManager.scala  | 6 +-
 core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala | 8 +++-
 2 files changed, 12 insertions(+), 2 deletions(-)

diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala 
b/core/src/main/scala/kafka/server/ReplicaManager.scala
index fe3fad216c2..4b33a288b89 100644
--- a/core/src/main/scala/kafka/server/ReplicaManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
@@ -2810,6 +2810,7 @@ class ReplicaManager(val config: KafkaConfig,
   def applyDelta(delta: TopicsDelta, newImage: MetadataImage): Unit = {
 // Before taking the lock, compute the local changes
 val localChanges = delta.localChanges(config.nodeId)
+val metadataVersion = newImage.features().metadataVersion()
 
 replicaStateChangeLock.synchronized {
   // Handle deleted partitions. We need to do this first because we might 
subsequently
@@ -2857,7 +2858,10 @@ class ReplicaManager(val config: KafkaConfig,
 remoteLogManager.foreach(rlm => 
rlm.onLeadershipChange(leaderChangedPartitions.asJava, 
followerChangedPartitions.asJava, localChanges.topicIds()))
   }
 
-  localChanges.directoryIds.forEach(maybeUpdateTopicAssignment)
+  if (metadataVersion.isDirectoryAssignmentSupported()) {
+// We only want to update the directoryIds if DirectoryAssignment is 
supported!
+localChanges.directoryIds.forEach(maybeUpdateTopicAssignment)
+  }
 }
   }
 
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala 
b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
index 9201db0765e..7ef92a97371 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
@@ -57,7 +57,9 @@ import org.apache.kafka.common.{DirectoryId, IsolationLevel, 
Node, TopicIdPartit
 import org.apache.kafka.image._
 import org.apache.kafka.metadata.LeaderConstants.NO_LEADER
 import org.apache.kafka.metadata.LeaderRecoveryState
+import org.apache.kafka.metadata.migration.ZkMigrationState
 import org.apache.kafka.server.common.{DirectoryEventHandler, OffsetAndEpoch}
+import org.apache.kafka.server.common.MetadataVersion
 import org.apache.kafka.server.common.MetadataVersion.IBP_2_6_IV0
 import 
org.apache.kafka.server.log.remote.storage.{NoOpRemoteLogMetadataManager, 
NoOpRemoteStorageManager, RemoteLogManagerConfig, RemoteLogSegmentMetadata, 
RemoteStorageException, RemoteStorageManager}
 import org.apache.kafka.server.metrics.{KafkaMetricsGroup, KafkaYammerMetrics}
@@ -6171,9 +6173,13 @@ class ReplicaManagerTest {
   }
 
   private def imageFromTopics(topicsImage: TopicsImage): MetadataImage = {
+val featuresImageLatest = new FeaturesImage(
+  Collections.emptyMap(),
+  MetadataVersion.latest(),
+  ZkMigrationState.NONE)
 new MetadataImage(
   new MetadataProvenance(100L, 10, 1000L),
-  FeaturesImage.EMPTY,
+  featuresImageLatest,
   ClusterImageTest.IMAGE1,
   topicsImage,
   ConfigurationsImage.EMPTY,



(kafka) branch trunk updated: KAFKA-16131: Only update directoryIds if the metadata version supports DirectoryAssignment (#15197)

2024-01-17 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 6c14d779989 KAFKA-16131: Only update directoryIds if the metadata 
version supports DirectoryAssignment (#15197)
6c14d779989 is described below

commit 6c14d779989dc7fec5f03c18830f1f3b80ad693e
Author: Proven Provenzano <93720617+pprovenz...@users.noreply.github.com>
AuthorDate: Wed Jan 17 17:06:38 2024 -0500

KAFKA-16131: Only update directoryIds if the metadata version supports 
DirectoryAssignment (#15197)

We only want to send directory assignments if the metadata version supports 
it.

Reviewers: Colin P. McCabe 
---
 core/src/main/scala/kafka/server/ReplicaManager.scala  | 6 +-
 core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala | 8 +++-
 2 files changed, 12 insertions(+), 2 deletions(-)

diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala 
b/core/src/main/scala/kafka/server/ReplicaManager.scala
index bcc01b675ed..a987ce0f7fa 100644
--- a/core/src/main/scala/kafka/server/ReplicaManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
@@ -2798,6 +2798,7 @@ class ReplicaManager(val config: KafkaConfig,
   def applyDelta(delta: TopicsDelta, newImage: MetadataImage): Unit = {
 // Before taking the lock, compute the local changes
 val localChanges = delta.localChanges(config.nodeId)
+val metadataVersion = newImage.features().metadataVersion()
 
 replicaStateChangeLock.synchronized {
   // Handle deleted partitions. We need to do this first because we might 
subsequently
@@ -2845,7 +2846,10 @@ class ReplicaManager(val config: KafkaConfig,
 remoteLogManager.foreach(rlm => 
rlm.onLeadershipChange(leaderChangedPartitions.asJava, 
followerChangedPartitions.asJava, localChanges.topicIds()))
   }
 
-  localChanges.directoryIds.forEach(maybeUpdateTopicAssignment)
+  if (metadataVersion.isDirectoryAssignmentSupported()) {
+// We only want to update the directoryIds if DirectoryAssignment is 
supported!
+localChanges.directoryIds.forEach(maybeUpdateTopicAssignment)
+  }
 }
   }
 
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala 
b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
index 9201db0765e..7ef92a97371 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
@@ -57,7 +57,9 @@ import org.apache.kafka.common.{DirectoryId, IsolationLevel, 
Node, TopicIdPartit
 import org.apache.kafka.image._
 import org.apache.kafka.metadata.LeaderConstants.NO_LEADER
 import org.apache.kafka.metadata.LeaderRecoveryState
+import org.apache.kafka.metadata.migration.ZkMigrationState
 import org.apache.kafka.server.common.{DirectoryEventHandler, OffsetAndEpoch}
+import org.apache.kafka.server.common.MetadataVersion
 import org.apache.kafka.server.common.MetadataVersion.IBP_2_6_IV0
 import 
org.apache.kafka.server.log.remote.storage.{NoOpRemoteLogMetadataManager, 
NoOpRemoteStorageManager, RemoteLogManagerConfig, RemoteLogSegmentMetadata, 
RemoteStorageException, RemoteStorageManager}
 import org.apache.kafka.server.metrics.{KafkaMetricsGroup, KafkaYammerMetrics}
@@ -6171,9 +6173,13 @@ class ReplicaManagerTest {
   }
 
   private def imageFromTopics(topicsImage: TopicsImage): MetadataImage = {
+val featuresImageLatest = new FeaturesImage(
+  Collections.emptyMap(),
+  MetadataVersion.latest(),
+  ZkMigrationState.NONE)
 new MetadataImage(
   new MetadataProvenance(100L, 10, 1000L),
-  FeaturesImage.EMPTY,
+  featuresImageLatest,
   ClusterImageTest.IMAGE1,
   topicsImage,
   ConfigurationsImage.EMPTY,



(kafka) branch 3.6 updated: KAFKA-16126: Kcontroller dynamic configurations may fail to apply at startup

2024-01-16 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.6
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.6 by this push:
 new b743f6fd884 KAFKA-16126: Kcontroller dynamic configurations may fail 
to apply at startup
b743f6fd884 is described below

commit b743f6fd884132c7a5c4e9d96ed62e3aec29007f
Author: Colin P. McCabe 
AuthorDate: Tue Jan 16 16:02:26 2024 -0800

KAFKA-16126: Kcontroller dynamic configurations may fail to apply at startup

Some kcontroller dynamic configurations may fail to apply at startup. This 
happens because there is
a race between registering the reconfigurables to the DynamicBrokerConfig 
class, and receiving the
first update from the metadata publisher. We can fix this by registering 
the reconfigurables first.
This seems to have been introduced by the "MINOR: Install ControllerServer 
metadata publishers
sooner" change.

Reviewers: Ron Dagostino  
---
 .../main/scala/kafka/server/ControllerServer.scala | 11 ---
 .../scala/kafka/server/KafkaRequestHandler.scala   |  2 +-
 core/src/test/java/kafka/testkit/TestKitNodes.java | 26 +---
 .../kafka/server/KRaftClusterTest.scala| 35 +-
 4 files changed, 57 insertions(+), 17 deletions(-)

diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala 
b/core/src/main/scala/kafka/server/ControllerServer.scala
index 9a626ab84b5..5ffc382cf8d 100644
--- a/core/src/main/scala/kafka/server/ControllerServer.scala
+++ b/core/src/main/scala/kafka/server/ControllerServer.scala
@@ -330,6 +330,13 @@ class ControllerServer(
 ),
 "controller"))
 
+  // Register this instance for dynamic config changes to the KafkaConfig. 
This must be called
+  // after the authorizer and quotaManagers are initialized, since it 
references those objects.
+  // It must be called before DynamicClientQuotaPublisher is installed, 
since otherwise we may
+  // miss the initial update which establishes the dynamic configurations 
that are in effect on
+  // startup.
+  config.dynamicConfig.addReconfigurables(this)
+
   // Set up the client quotas publisher. This will enable controller 
mutation quotas and any
   // other quotas which are applicable.
   metadataPublishers.add(new DynamicClientQuotaPublisher(
@@ -346,7 +353,6 @@ class ControllerServer(
 credentialProvider
   ))
 
-
   // Set up the DelegationToken publisher.
   // We need a tokenManager for the Publisher
   // The tokenCache in the tokenManager is the same used in 
DelegationTokenControlManager
@@ -397,9 +403,6 @@ class ControllerServer(
   FutureUtils.waitWithLogging(logger.underlying, logIdent,
 "all of the SocketServer Acceptors to be started",
 socketServerFuture, startupDeadline, time)
-
-  // register this instance for dynamic config changes to the KafkaConfig
-  config.dynamicConfig.addReconfigurables(this)
 } catch {
   case e: Throwable =>
 maybeChangeStatus(STARTING, STARTED)
diff --git a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala 
b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala
index 7885b436c59..c1c82f07e57 100755
--- a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala
+++ b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala
@@ -196,7 +196,7 @@ class KafkaRequestHandlerPool(val brokerId: Int,
   logAndThreadNamePrefix : String) extends Logging 
{
   private val metricsGroup = new KafkaMetricsGroup(this.getClass)
 
-  private val threadPoolSize: AtomicInteger = new AtomicInteger(numThreads)
+  val threadPoolSize: AtomicInteger = new AtomicInteger(numThreads)
   /* a meter to track the average free capacity of the request handlers */
   private val aggregateIdleMeter = 
metricsGroup.newMeter(requestHandlerAvgIdleMetricName, "percent", 
TimeUnit.NANOSECONDS)
 
diff --git a/core/src/test/java/kafka/testkit/TestKitNodes.java 
b/core/src/test/java/kafka/testkit/TestKitNodes.java
index 5bc36854494..da4e10a4431 100644
--- a/core/src/test/java/kafka/testkit/TestKitNodes.java
+++ b/core/src/test/java/kafka/testkit/TestKitNodes.java
@@ -20,6 +20,7 @@ package kafka.testkit;
 import kafka.server.MetaProperties;
 import org.apache.kafka.common.Uuid;
 import org.apache.kafka.common.network.ListenerName;
+import org.apache.kafka.metadata.bootstrap.BootstrapMetadata;
 import org.apache.kafka.server.common.MetadataVersion;
 
 import java.nio.file.Paths;
@@ -35,7 +36,8 @@ public class TestKitNodes {
 public static class Builder {
 private boolean combined = false;
 private Uuid clusterId = null;
-private MetadataVersion bootstrapMetadataVersion = null;
+private BootstrapMetadata bootstrapMetadata = BootstrapMetadata.
+fro

(kafka) branch 3.7 updated: KAFKA-16126: Kcontroller dynamic configurations may fail to apply at startup

2024-01-16 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.7
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.7 by this push:
 new b4036833081 KAFKA-16126: Kcontroller dynamic configurations may fail 
to apply at startup
b4036833081 is described below

commit b40368330814888d7f7f2fda3f5b7ecfa1eabeb2
Author: Colin P. McCabe 
AuthorDate: Tue Jan 16 16:02:26 2024 -0800

KAFKA-16126: Kcontroller dynamic configurations may fail to apply at startup

Some kcontroller dynamic configurations may fail to apply at startup. This 
happens because there is
a race between registering the reconfigurables to the DynamicBrokerConfig 
class, and receiving the
first update from the metadata publisher. We can fix this by registering 
the reconfigurables first.
This seems to have been introduced by the "MINOR: Install ControllerServer 
metadata publishers
sooner" change.

Reviewers: Ron Dagostino  
---
 .../main/scala/kafka/server/ControllerServer.scala | 11 ---
 .../scala/kafka/server/KafkaRequestHandler.scala   |  2 +-
 core/src/test/java/kafka/testkit/TestKitNodes.java | 25 +---
 .../kafka/server/KRaftClusterTest.scala| 34 +-
 4 files changed, 55 insertions(+), 17 deletions(-)

diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala 
b/core/src/main/scala/kafka/server/ControllerServer.scala
index db386e4a92e..fb7439d8e56 100644
--- a/core/src/main/scala/kafka/server/ControllerServer.scala
+++ b/core/src/main/scala/kafka/server/ControllerServer.scala
@@ -367,6 +367,13 @@ class ControllerServer(
 ),
 "controller"))
 
+  // Register this instance for dynamic config changes to the KafkaConfig. 
This must be called
+  // after the authorizer and quotaManagers are initialized, since it 
references those objects.
+  // It must be called before DynamicClientQuotaPublisher is installed, 
since otherwise we may
+  // miss the initial update which establishes the dynamic configurations 
that are in effect on
+  // startup.
+  config.dynamicConfig.addReconfigurables(this)
+
   // Set up the client quotas publisher. This will enable controller 
mutation quotas and any
   // other quotas which are applicable.
   metadataPublishers.add(new DynamicClientQuotaPublisher(
@@ -383,7 +390,6 @@ class ControllerServer(
 credentialProvider
   ))
 
-
   // Set up the DelegationToken publisher.
   // We need a tokenManager for the Publisher
   // The tokenCache in the tokenManager is the same used in 
DelegationTokenControlManager
@@ -449,9 +455,6 @@ class ControllerServer(
   FutureUtils.waitWithLogging(logger.underlying, logIdent,
 "all of the SocketServer Acceptors to be started",
 socketServerFuture, startupDeadline, time)
-
-  // register this instance for dynamic config changes to the KafkaConfig
-  config.dynamicConfig.addReconfigurables(this)
 } catch {
   case e: Throwable =>
 maybeChangeStatus(STARTING, STARTED)
diff --git a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala 
b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala
index 645a33ac931..895b048ca3e 100755
--- a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala
+++ b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala
@@ -206,7 +206,7 @@ class KafkaRequestHandlerPool(
 ) extends Logging {
   private val metricsGroup = new KafkaMetricsGroup(this.getClass)
 
-  private val threadPoolSize: AtomicInteger = new AtomicInteger(numThreads)
+  val threadPoolSize: AtomicInteger = new AtomicInteger(numThreads)
   /* a meter to track the average free capacity of the request handlers */
   private val aggregateIdleMeter = 
metricsGroup.newMeter(requestHandlerAvgIdleMetricName, "percent", 
TimeUnit.NANOSECONDS)
 
diff --git a/core/src/test/java/kafka/testkit/TestKitNodes.java 
b/core/src/test/java/kafka/testkit/TestKitNodes.java
index 0f0d8939fe9..90317c56224 100644
--- a/core/src/test/java/kafka/testkit/TestKitNodes.java
+++ b/core/src/test/java/kafka/testkit/TestKitNodes.java
@@ -33,7 +33,8 @@ public class TestKitNodes {
 public static class Builder {
 private boolean combined = false;
 private Uuid clusterId = null;
-private MetadataVersion bootstrapMetadataVersion = null;
+private BootstrapMetadata bootstrapMetadata = BootstrapMetadata.
+fromVersion(MetadataVersion.latest(), "testkit");
 private final NavigableMap 
controllerNodeBuilders = new TreeMap<>();
 private final NavigableMap 
brokerNodeBuilders = new TreeMap<>();
 
@@ -43,7 +44,12 @@ public class TestKitNodes {
 }
 
 public Builder setBootstrapMetadataVersion(MetadataVersion 
metadataVersion) {
-this.bootstrapMetad

(kafka) branch trunk updated: KAFKA-16126: Kcontroller dynamic configurations may fail to apply at startup

2024-01-16 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 0015d0f01b1 KAFKA-16126: Kcontroller dynamic configurations may fail 
to apply at startup
0015d0f01b1 is described below

commit 0015d0f01b130992acc37da85da6ee2088186a1f
Author: Colin P. McCabe 
AuthorDate: Tue Jan 16 16:02:26 2024 -0800

KAFKA-16126: Kcontroller dynamic configurations may fail to apply at startup

Some kcontroller dynamic configurations may fail to apply at startup. This 
happens because there is
a race between registering the reconfigurables to the DynamicBrokerConfig 
class, and receiving the
first update from the metadata publisher. We can fix this by registering 
the reconfigurables first.
This seems to have been introduced by the "MINOR: Install ControllerServer 
metadata publishers
sooner" change.

Reviewers: Ron Dagostino  
---
 .../main/scala/kafka/server/ControllerServer.scala | 11 ---
 .../scala/kafka/server/KafkaRequestHandler.scala   |  2 +-
 core/src/test/java/kafka/testkit/TestKitNodes.java | 25 +---
 .../kafka/server/KRaftClusterTest.scala| 34 +-
 4 files changed, 55 insertions(+), 17 deletions(-)

diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala 
b/core/src/main/scala/kafka/server/ControllerServer.scala
index c757d6d6eea..f74bb3de7a4 100644
--- a/core/src/main/scala/kafka/server/ControllerServer.scala
+++ b/core/src/main/scala/kafka/server/ControllerServer.scala
@@ -368,6 +368,13 @@ class ControllerServer(
 ),
 "controller"))
 
+  // Register this instance for dynamic config changes to the KafkaConfig. 
This must be called
+  // after the authorizer and quotaManagers are initialized, since it 
references those objects.
+  // It must be called before DynamicClientQuotaPublisher is installed, 
since otherwise we may
+  // miss the initial update which establishes the dynamic configurations 
that are in effect on
+  // startup.
+  config.dynamicConfig.addReconfigurables(this)
+
   // Set up the client quotas publisher. This will enable controller 
mutation quotas and any
   // other quotas which are applicable.
   metadataPublishers.add(new DynamicClientQuotaPublisher(
@@ -384,7 +391,6 @@ class ControllerServer(
 credentialProvider
   ))
 
-
   // Set up the DelegationToken publisher.
   // We need a tokenManager for the Publisher
   // The tokenCache in the tokenManager is the same used in 
DelegationTokenControlManager
@@ -450,9 +456,6 @@ class ControllerServer(
   FutureUtils.waitWithLogging(logger.underlying, logIdent,
 "all of the SocketServer Acceptors to be started",
 socketServerFuture, startupDeadline, time)
-
-  // register this instance for dynamic config changes to the KafkaConfig
-  config.dynamicConfig.addReconfigurables(this)
 } catch {
   case e: Throwable =>
 maybeChangeStatus(STARTING, STARTED)
diff --git a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala 
b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala
index 645a33ac931..895b048ca3e 100755
--- a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala
+++ b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala
@@ -206,7 +206,7 @@ class KafkaRequestHandlerPool(
 ) extends Logging {
   private val metricsGroup = new KafkaMetricsGroup(this.getClass)
 
-  private val threadPoolSize: AtomicInteger = new AtomicInteger(numThreads)
+  val threadPoolSize: AtomicInteger = new AtomicInteger(numThreads)
   /* a meter to track the average free capacity of the request handlers */
   private val aggregateIdleMeter = 
metricsGroup.newMeter(requestHandlerAvgIdleMetricName, "percent", 
TimeUnit.NANOSECONDS)
 
diff --git a/core/src/test/java/kafka/testkit/TestKitNodes.java 
b/core/src/test/java/kafka/testkit/TestKitNodes.java
index 0f0d8939fe9..90317c56224 100644
--- a/core/src/test/java/kafka/testkit/TestKitNodes.java
+++ b/core/src/test/java/kafka/testkit/TestKitNodes.java
@@ -33,7 +33,8 @@ public class TestKitNodes {
 public static class Builder {
 private boolean combined = false;
 private Uuid clusterId = null;
-private MetadataVersion bootstrapMetadataVersion = null;
+private BootstrapMetadata bootstrapMetadata = BootstrapMetadata.
+fromVersion(MetadataVersion.latest(), "testkit");
 private final NavigableMap 
controllerNodeBuilders = new TreeMap<>();
 private final NavigableMap 
brokerNodeBuilders = new TreeMap<>();
 
@@ -43,7 +44,12 @@ public class TestKitNodes {
 }
 
 public Builder setBootstrapMetadataVersion(MetadataVersion 
metadataVersion) {
-this.bootstrapMetad

(kafka) branch 3.6 updated: KAFKA-16120: Fix partition reassignment during ZK migration

2024-01-16 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.6
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.6 by this push:
 new 7486223d9e3 KAFKA-16120: Fix partition reassignment during ZK migration
7486223d9e3 is described below

commit 7486223d9e39a48700bfb6dfdad93e06e4ab550f
Author: David Mao 
AuthorDate: Fri Jan 12 09:12:33 2024 -0800

KAFKA-16120: Fix partition reassignment during ZK migration

When we are migrating from ZK mode to KRaft mode, the brokers pass through 
a phase where they are
running in ZK mode, but the controller is in KRaft mode (aka a 
kcontroller). This is called "hybrid
mode." In hybrid mode, the KRaft controllers send old-style controller RPCs 
to the remaining ZK
mode brokers. (StopReplicaRequest, LeaderAndIsrRequest, 
UpdateMetadataRequest, etc.)

To complete partition reassignment, the kcontroller must send a 
StopReplicaRequest to any brokers
that no longer host the partition in question. Previously, it was sending 
this StopReplicaRequest
with delete = false. This led to stray partitions, because the partition 
data was never removed as
it should have been. This PR fixes it to set delete = true. This fixes 
KAFKA-16120.

There is one additional problem with partition reassignment in hybrid mode, 
tracked as KAFKA-16121.
The issue is that in ZK mode, brokers ignore any LeaderAndIsr request where 
the partition leader
epoch is less than or equal to the current partition leader epoch. However, 
when in hybrid mode,
just as in KRaft mode, we do not bump the leader epoch when starting a new 
reassignment, see:
`triggerLeaderEpochBumpIfNeeded`. This PR resolves this problem by adding a 
special case on the
broker side when isKRaftController = true.

Reviewers: Akhilesh Chaganti , Colin 
P. McCabe 
---
 .../kafka/migration/MigrationPropagator.scala  |  2 +-
 .../main/scala/kafka/server/ReplicaManager.scala   |  2 +-
 .../kafka/zk/ZkMigrationIntegrationTest.scala  | 84 +-
 3 files changed, 85 insertions(+), 3 deletions(-)

diff --git a/core/src/main/scala/kafka/migration/MigrationPropagator.scala 
b/core/src/main/scala/kafka/migration/MigrationPropagator.scala
index 2a02f5891ec..7bf0fc3ff56 100644
--- a/core/src/main/scala/kafka/migration/MigrationPropagator.scala
+++ b/core/src/main/scala/kafka/migration/MigrationPropagator.scala
@@ -207,7 +207,7 @@ class MigrationPropagator(
 val newReplicas = partitionRegistration.replicas.toSet
 val removedReplicas = oldReplicas -- newReplicas
 if (removedReplicas.nonEmpty) {
-  requestBatch.addStopReplicaRequestForBrokers(removedReplicas.toSeq, 
tp, deletePartition = false)
+  requestBatch.addStopReplicaRequestForBrokers(removedReplicas.toSeq, 
tp, deletePartition = true)
 }
   }
 }
diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala 
b/core/src/main/scala/kafka/server/ReplicaManager.scala
index 469321e14d5..63a6f049247 100644
--- a/core/src/main/scala/kafka/server/ReplicaManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
@@ -1818,7 +1818,7 @@ class ReplicaManager(val config: KafkaConfig,
   s" match the topic ID for partition $topicPartition 
received: " +
   s"${requestTopicId.get}.")
 responseMap.put(topicPartition, Errors.INCONSISTENT_TOPIC_ID)
-  } else if (requestLeaderEpoch > currentLeaderEpoch) {
+  } else if (requestLeaderEpoch > currentLeaderEpoch || 
(requestLeaderEpoch == currentLeaderEpoch && 
leaderAndIsrRequest.isKRaftController)) {
 // If the leader epoch is valid record the epoch of the 
controller that made the leadership decision.
 // This is useful while updating the isr to maintain the 
decision maker controller's epoch in the zookeeper path
 if (partitionState.replicas.contains(localBrokerId)) {
diff --git 
a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala 
b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala
index ec28484172a..4c3ddd8b80d 100644
--- a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala
@@ -53,7 +53,7 @@ import org.slf4j.LoggerFactory
 
 import java.util
 import java.util.concurrent.{CompletableFuture, ExecutionException, TimeUnit}
-import java.util.{Properties, UUID}
+import java.util.{Collections, Optional, Properties, UUID}
 import scala.collection.Seq
 import scala.jdk.CollectionConverters._
 
@@ -678,6 +678,88 @@ class ZkMigrationIntegrationTest {
 }
   }
 
+  @ClusterTest(clusterType = Type.ZK, brokers = 4, metadataVersion = 
MetadataVersion.IBP_3_6_IV2, serverProperties

(kafka) 01/01: KAFKA-16126: Kcontroller dynamic configurations may fail to apply at startup

2024-01-15 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch KAFKA-16126
in repository https://gitbox.apache.org/repos/asf/kafka.git

commit d7f158e9d4ad3965d8e721c80c59f5e99ab9a2e7
Author: Colin P. McCabe 
AuthorDate: Sun Jan 14 13:50:10 2024 -0800

KAFKA-16126: Kcontroller dynamic configurations may fail to apply at startup

Some kcontroller dynamic configurations may fail to apply at startup. This 
happens because there is a race between registering the reconfigurables to the 
DynamicBrokerConfig class, and receiving the first update from the metadata 
publisher. We can fix this by registering the reconfigurables first. This seems 
to have been introduced by the "MINOR: Install ControllerServer metadata 
publishers sooner" change.
---
 .../main/scala/kafka/server/ControllerServer.scala | 11 ---
 .../scala/kafka/server/KafkaRequestHandler.scala   |  2 +-
 core/src/test/java/kafka/testkit/TestKitNodes.java | 25 +---
 .../kafka/server/KRaftClusterTest.scala| 34 +-
 4 files changed, 55 insertions(+), 17 deletions(-)

diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala 
b/core/src/main/scala/kafka/server/ControllerServer.scala
index c757d6d6eea..f74bb3de7a4 100644
--- a/core/src/main/scala/kafka/server/ControllerServer.scala
+++ b/core/src/main/scala/kafka/server/ControllerServer.scala
@@ -368,6 +368,13 @@ class ControllerServer(
 ),
 "controller"))
 
+  // Register this instance for dynamic config changes to the KafkaConfig. 
This must be called
+  // after the authorizer and quotaManagers are initialized, since it 
references those objects.
+  // It must be called before DynamicClientQuotaPublisher is installed, 
since otherwise we may
+  // miss the initial update which establishes the dynamic configurations 
that are in effect on
+  // startup.
+  config.dynamicConfig.addReconfigurables(this)
+
   // Set up the client quotas publisher. This will enable controller 
mutation quotas and any
   // other quotas which are applicable.
   metadataPublishers.add(new DynamicClientQuotaPublisher(
@@ -384,7 +391,6 @@ class ControllerServer(
 credentialProvider
   ))
 
-
   // Set up the DelegationToken publisher.
   // We need a tokenManager for the Publisher
   // The tokenCache in the tokenManager is the same used in 
DelegationTokenControlManager
@@ -450,9 +456,6 @@ class ControllerServer(
   FutureUtils.waitWithLogging(logger.underlying, logIdent,
 "all of the SocketServer Acceptors to be started",
 socketServerFuture, startupDeadline, time)
-
-  // register this instance for dynamic config changes to the KafkaConfig
-  config.dynamicConfig.addReconfigurables(this)
 } catch {
   case e: Throwable =>
 maybeChangeStatus(STARTING, STARTED)
diff --git a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala 
b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala
index 645a33ac931..895b048ca3e 100755
--- a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala
+++ b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala
@@ -206,7 +206,7 @@ class KafkaRequestHandlerPool(
 ) extends Logging {
   private val metricsGroup = new KafkaMetricsGroup(this.getClass)
 
-  private val threadPoolSize: AtomicInteger = new AtomicInteger(numThreads)
+  val threadPoolSize: AtomicInteger = new AtomicInteger(numThreads)
   /* a meter to track the average free capacity of the request handlers */
   private val aggregateIdleMeter = 
metricsGroup.newMeter(requestHandlerAvgIdleMetricName, "percent", 
TimeUnit.NANOSECONDS)
 
diff --git a/core/src/test/java/kafka/testkit/TestKitNodes.java 
b/core/src/test/java/kafka/testkit/TestKitNodes.java
index 0f0d8939fe9..90317c56224 100644
--- a/core/src/test/java/kafka/testkit/TestKitNodes.java
+++ b/core/src/test/java/kafka/testkit/TestKitNodes.java
@@ -33,7 +33,8 @@ public class TestKitNodes {
 public static class Builder {
 private boolean combined = false;
 private Uuid clusterId = null;
-private MetadataVersion bootstrapMetadataVersion = null;
+private BootstrapMetadata bootstrapMetadata = BootstrapMetadata.
+fromVersion(MetadataVersion.latest(), "testkit");
 private final NavigableMap 
controllerNodeBuilders = new TreeMap<>();
 private final NavigableMap 
brokerNodeBuilders = new TreeMap<>();
 
@@ -43,7 +44,12 @@ public class TestKitNodes {
 }
 
 public Builder setBootstrapMetadataVersion(MetadataVersion 
metadataVersion) {
-this.bootstrapMetadataVersion = metadataVersion;
+this.bootstrapMetadata = 
BootstrapMetadata.fromVersion(metadataVersion, "testkit");
+return this;
+}
+
+public Builder setBootstrapMetadata(BootstrapMetadata 
b

(kafka) branch KAFKA-16126 created (now d7f158e9d4a)

2024-01-15 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a change to branch KAFKA-16126
in repository https://gitbox.apache.org/repos/asf/kafka.git


  at d7f158e9d4a KAFKA-16126: Kcontroller dynamic configurations may fail 
to apply at startup

This branch includes the following new commits:

 new d7f158e9d4a KAFKA-16126: Kcontroller dynamic configurations may fail 
to apply at startup

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.




(kafka) branch KAFKA-16101 created (now 6ec8ad0ad54)

2024-01-15 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a change to branch KAFKA-16101
in repository https://gitbox.apache.org/repos/asf/kafka.git


  at 6ec8ad0ad54 KAFKA-16101: KRaft migration documentation is incorrect

This branch includes the following new commits:

 new 6ec8ad0ad54 KAFKA-16101: KRaft migration documentation is incorrect

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.




(kafka) 01/01: KAFKA-16101: KRaft migration documentation is incorrect

2024-01-15 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch KAFKA-16101
in repository https://gitbox.apache.org/repos/asf/kafka.git

commit 6ec8ad0ad541dfe81072121d88ee75461ebeeab2
Author: Colin P. McCabe 
AuthorDate: Mon Jan 15 00:06:34 2024 -0800

KAFKA-16101: KRaft migration documentation is incorrect

The documentation about reverting from KRaft mode to ZK mode previously 
said that the admin should roll each broker to remove the migration 
configuration, then take down the kcontrollers. However, this is not correct. 
Instead, the admin should roll each broker to get it into ZK mode (but not 
remove the migration configuration), then tke down the kcontrollers, then 
remove the controller znode in ZK.

This PR fixes this part of the documentation. It also adds a note stating 
that ZK can be deprovisioned after migration is finalized.
---
 docs/ops.html | 16 +++-
 1 file changed, 11 insertions(+), 5 deletions(-)

diff --git a/docs/ops.html b/docs/ops.html
index eb0cbd78828..62919ceab9a 100644
--- a/docs/ops.html
+++ b/docs/ops.html
@@ -3956,18 +3956,20 @@ controller.listener.names=CONTROLLER
 While the cluster is still in migration mode, it is possible to revert to 
ZK mode. In order to do this:
 
   
-For each KRaft broker:
+If you have brokers in KRaft mode, for each one:
 
   Stop the broker.
   Remove the __cluster_metadata directory on the broker.
-  Remove the zookeeper.metadata.migration.enable 
configuration and the KRaft controllers related configurations like 
controller.quorum.voters
-and controller.listener.names from the broker 
configuration properties file.
-  Restart the broker in ZooKeeper mode.
+  Reconfigure the broker so that it will restart in ZK mode, by 
removing process.roles
+  and re-adding zookeeper.connect, and so forth.
+  Restart the broker.
 
   
-  Take down the KRaft quorum.
+  Once all brokers are in ZK mode, take down the KRaft quorum.
   Using ZooKeeper shell, delete the controller node using rmr 
/controller, so that a ZooKeeper-based broker can become the next 
controller.
+  If desired (this step is optional) roll the brokers again, 
reconfiguring each without zookeeper.metadata.migration.enable.
 
+Note that you must not remove the zookeeper.metadata.migration.enable 
configuration on each broker until the KRaft quorum has been completely 
removed.
 
   Finalizing the migration
   
@@ -3975,6 +3977,10 @@ controller.listener.names=CONTROLLER
 KRaft controllers out of migration mode. This is done by removing the 
"zookeeper.metadata.migration.enable"
 property from each of their configs and restarting them one at a time.
   
+  
+Once the migration has been finalized, you can safely deprovision your 
ZooKeeper cluster, assuming you are
+not using it for anything else. After this point, it is no longer possible 
to revert to ZooKeeper mode.
+  
 
   
 # Sample KRaft cluster controller.properties listening on 9093



(kafka) branch 3.7 updated: KAFKA-16120: Fix partition reassignment during ZK migration

2024-01-14 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.7
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.7 by this push:
 new 4280df48161 KAFKA-16120: Fix partition reassignment during ZK migration
4280df48161 is described below

commit 4280df4816195204ac2b60c7f8cdc3173bdaa375
Author: David Mao 
AuthorDate: Fri Jan 12 09:12:33 2024 -0800

KAFKA-16120: Fix partition reassignment during ZK migration

When we are migrating from ZK mode to KRaft mode, the brokers pass through 
a phase where they are
running in ZK mode, but the controller is in KRaft mode (aka a 
kcontroller). This is called "hybrid
mode." In hybrid mode, the KRaft controllers send old-style controller RPCs 
to the remaining ZK
mode brokers. (StopReplicaRequest, LeaderAndIsrRequest, 
UpdateMetadataRequest, etc.)

To complete partition reassignment, the kcontroller must send a 
StopReplicaRequest to any brokers
that no longer host the partition in question. Previously, it was sending 
this StopReplicaRequest
with delete = false. This led to stray partitions, because the partition 
data was never removed as
it should have been. This PR fixes it to set delete = true. This fixes 
KAFKA-16120.

There is one additional problem with partition reassignment in hybrid mode, 
tracked as KAFKA-16121.
The issue is that in ZK mode, brokers ignore any LeaderAndIsr request where 
the partition leader
epoch is less than or equal to the current partition leader epoch. However, 
when in hybrid mode,
just as in KRaft mode, we do not bump the leader epoch when starting a new 
reassignment, see:
`triggerLeaderEpochBumpIfNeeded`. This PR resolves this problem by adding a 
special case on the
broker side when isKRaftController = true.

Reviewers: Akhilesh Chaganti , Colin 
P. McCabe 
---
 .../kafka/migration/MigrationPropagator.scala  |  2 +-
 .../main/scala/kafka/server/ReplicaManager.scala   |  2 +-
 .../kafka/zk/ZkMigrationIntegrationTest.scala  | 82 +-
 3 files changed, 83 insertions(+), 3 deletions(-)

diff --git a/core/src/main/scala/kafka/migration/MigrationPropagator.scala 
b/core/src/main/scala/kafka/migration/MigrationPropagator.scala
index 2a02f5891ec..7bf0fc3ff56 100644
--- a/core/src/main/scala/kafka/migration/MigrationPropagator.scala
+++ b/core/src/main/scala/kafka/migration/MigrationPropagator.scala
@@ -207,7 +207,7 @@ class MigrationPropagator(
 val newReplicas = partitionRegistration.replicas.toSet
 val removedReplicas = oldReplicas -- newReplicas
 if (removedReplicas.nonEmpty) {
-  requestBatch.addStopReplicaRequestForBrokers(removedReplicas.toSeq, 
tp, deletePartition = false)
+  requestBatch.addStopReplicaRequestForBrokers(removedReplicas.toSeq, 
tp, deletePartition = true)
 }
   }
 }
diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala 
b/core/src/main/scala/kafka/server/ReplicaManager.scala
index b9e0e1360e9..fe3fad216c2 100644
--- a/core/src/main/scala/kafka/server/ReplicaManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
@@ -2072,7 +2072,7 @@ class ReplicaManager(val config: KafkaConfig,
   s" match the topic ID for partition $topicPartition 
received: " +
   s"${requestTopicId.get}.")
 responseMap.put(topicPartition, Errors.INCONSISTENT_TOPIC_ID)
-  } else if (requestLeaderEpoch > currentLeaderEpoch) {
+  } else if (requestLeaderEpoch > currentLeaderEpoch || 
(requestLeaderEpoch == currentLeaderEpoch && 
leaderAndIsrRequest.isKRaftController)) {
 // If the leader epoch is valid record the epoch of the 
controller that made the leadership decision.
 // This is useful while updating the isr to maintain the 
decision maker controller's epoch in the zookeeper path
 if (partitionState.replicas.contains(localBrokerId)) {
diff --git 
a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala 
b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala
index 5bd1988201d..794f9b02181 100644
--- a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala
@@ -54,7 +54,7 @@ import org.slf4j.LoggerFactory
 
 import java.util
 import java.util.concurrent.{CompletableFuture, ExecutionException, TimeUnit}
-import java.util.{Properties, UUID}
+import java.util.{Collections, Optional, Properties, UUID}
 import scala.collection.Seq
 import scala.jdk.CollectionConverters._
 
@@ -676,6 +676,86 @@ class ZkMigrationIntegrationTest {
 }
   }
 
+  @ClusterTest(clusterType = Type.ZK, brokers = 4, metadataVersion = 
MetadataVersion.IBP_3_7_IV0, serverProperties

(kafka) branch trunk updated: KAFKA-16120: Fix partition reassignment during ZK migration

2024-01-14 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new d0f845a5e1a KAFKA-16120: Fix partition reassignment during ZK migration
d0f845a5e1a is described below

commit d0f845a5e1a2ecceadca3978cdc6adb1eeed27e3
Author: David Mao 
AuthorDate: Fri Jan 12 09:12:33 2024 -0800

KAFKA-16120: Fix partition reassignment during ZK migration

When we are migrating from ZK mode to KRaft mode, the brokers pass through 
a phase where they are
running in ZK mode, but the controller is in KRaft mode (aka a 
kcontroller). This is called "hybrid
mode." In hybrid mode, the KRaft controllers send old-style controller RPCs 
to the remaining ZK
mode brokers. (StopReplicaRequest, LeaderAndIsrRequest, 
UpdateMetadataRequest, etc.)

To complete partition reassignment, the kcontroller must send a 
StopReplicaRequest to any brokers
that no longer host the partition in question. Previously, it was sending 
this StopReplicaRequest
with delete = false. This led to stray partitions, because the partition 
data was never removed as
it should have been. This PR fixes it to set delete = true. This fixes 
KAFKA-16120.

There is one additional problem with partition reassignment in hybrid mode, 
tracked as KAFKA-16121.
The issue is that in ZK mode, brokers ignore any LeaderAndIsr request where 
the partition leader
epoch is less than or equal to the current partition leader epoch. However, 
when in hybrid mode,
just as in KRaft mode, we do not bump the leader epoch when starting a new 
reassignment, see:
`triggerLeaderEpochBumpIfNeeded`. This PR resolves this problem by adding a 
special case on the
broker side when isKRaftController = true.

Reviewers: Akhilesh Chaganti , Colin 
P. McCabe 
---
 .../kafka/migration/MigrationPropagator.scala  |  2 +-
 .../main/scala/kafka/server/ReplicaManager.scala   |  2 +-
 .../kafka/zk/ZkMigrationIntegrationTest.scala  | 82 +-
 3 files changed, 83 insertions(+), 3 deletions(-)

diff --git a/core/src/main/scala/kafka/migration/MigrationPropagator.scala 
b/core/src/main/scala/kafka/migration/MigrationPropagator.scala
index 2a02f5891ec..7bf0fc3ff56 100644
--- a/core/src/main/scala/kafka/migration/MigrationPropagator.scala
+++ b/core/src/main/scala/kafka/migration/MigrationPropagator.scala
@@ -207,7 +207,7 @@ class MigrationPropagator(
 val newReplicas = partitionRegistration.replicas.toSet
 val removedReplicas = oldReplicas -- newReplicas
 if (removedReplicas.nonEmpty) {
-  requestBatch.addStopReplicaRequestForBrokers(removedReplicas.toSeq, 
tp, deletePartition = false)
+  requestBatch.addStopReplicaRequestForBrokers(removedReplicas.toSeq, 
tp, deletePartition = true)
 }
   }
 }
diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala 
b/core/src/main/scala/kafka/server/ReplicaManager.scala
index 48ec0d0fe18..bcc01b675ed 100644
--- a/core/src/main/scala/kafka/server/ReplicaManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
@@ -2060,7 +2060,7 @@ class ReplicaManager(val config: KafkaConfig,
   s" match the topic ID for partition $topicPartition 
received: " +
   s"${requestTopicId.get}.")
 responseMap.put(topicPartition, Errors.INCONSISTENT_TOPIC_ID)
-  } else if (requestLeaderEpoch > currentLeaderEpoch) {
+  } else if (requestLeaderEpoch > currentLeaderEpoch || 
(requestLeaderEpoch == currentLeaderEpoch && 
leaderAndIsrRequest.isKRaftController)) {
 // If the leader epoch is valid record the epoch of the 
controller that made the leadership decision.
 // This is useful while updating the isr to maintain the 
decision maker controller's epoch in the zookeeper path
 if (partitionState.replicas.contains(localBrokerId)) {
diff --git 
a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala 
b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala
index d1afedd78c5..1fec4d65f09 100644
--- a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala
@@ -55,7 +55,7 @@ import org.slf4j.LoggerFactory
 
 import java.util
 import java.util.concurrent.{CompletableFuture, ExecutionException, TimeUnit}
-import java.util.{Properties, UUID}
+import java.util.{Collections, Optional, Properties, UUID}
 import scala.collection.Seq
 import scala.jdk.CollectionConverters._
 
@@ -677,6 +677,86 @@ class ZkMigrationIntegrationTest {
 }
   }
 
+  @ClusterTest(clusterType = Type.ZK, brokers = 4, metadataVersion = 
MetadataVersion.IBP_3_7_IV0, serverProper

(kafka) branch 3.7 updated: KAFKA-16094: BrokerRegistrationRequest.logDirs field must be ignorable (#15153)

2024-01-09 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch 3.7
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/3.7 by this push:
 new bdb4895f88f KAFKA-16094: BrokerRegistrationRequest.logDirs field must 
be ignorable (#15153)
bdb4895f88f is described below

commit bdb4895f88fc0753d8f2727225fd8ef72453e8c6
Author: Colin Patrick McCabe 
AuthorDate: Tue Jan 9 15:04:27 2024 -0800

KAFKA-16094: BrokerRegistrationRequest.logDirs field must be ignorable 
(#15153)

3.7 brokers must be able to register with 3.6 and earlier controllers. 
Currently, this is broken
because we will unconditionally try to set logDirs, but this field cannot 
be sent with
BrokerRegistrationRequest versions older than v2. This PR marks the logDirs 
field as "ignorable."
Marking the field as "ignorable" means that we will still be able to send 
the
BrokerRegistrationRequest even if the schema doesn't support logDirs.

Reviewers: Ron Dagostino  
---
 .../src/main/resources/common/message/BrokerRegistrationRequest.json  | 2 +-
 .../java/org/apache/kafka/common/requests/RequestResponseTest.java| 4 +++-
 2 files changed, 4 insertions(+), 2 deletions(-)

diff --git 
a/clients/src/main/resources/common/message/BrokerRegistrationRequest.json 
b/clients/src/main/resources/common/message/BrokerRegistrationRequest.json
index ace268db77a..9343465ec82 100644
--- a/clients/src/main/resources/common/message/BrokerRegistrationRequest.json
+++ b/clients/src/main/resources/common/message/BrokerRegistrationRequest.json
@@ -59,7 +59,7 @@
 { "name": "IsMigratingZkBroker", "type": "bool", "versions": "1+", 
"default": "false",
   "about": "If the required configurations for ZK migration are present, 
this value is set to true" },
 { "name": "LogDirs", "type":  "[]uuid", "versions":  "2+",
-  "about": "Log directories configured in this broker which are 
available." },
+  "about": "Log directories configured in this broker which are 
available.", "ignorable": true },
 { "name": "PreviousBrokerEpoch", "type": "int64", "versions": "3+", 
"default": "-1", "ignorable": true,
   "about": "The epoch before a clean shutdown." }
   ]
diff --git 
a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
 
b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
index b1ecb6aa4b4..4a19a3a1683 100644
--- 
a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
@@ -3502,7 +3502,9 @@ public class RequestResponseTest {
 new 
BrokerRegistrationRequestData.Feature()).iterator()))
 .setListeners(new 
BrokerRegistrationRequestData.ListenerCollection(singletonList(
 new 
BrokerRegistrationRequestData.Listener()).iterator()))
-.setIncarnationId(Uuid.randomUuid());
+.setIncarnationId(Uuid.randomUuid())
+
.setLogDirs(Arrays.asList(Uuid.fromString("qaJjNJ05Q36kEgeTBDcj0Q")))
+.setPreviousBrokerEpoch(123L);
 return new BrokerRegistrationRequest.Builder(data).build(v);
 }
 



(kafka) branch KAFKA-16094 deleted (was 584db5ede7b)

2024-01-09 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a change to branch KAFKA-16094
in repository https://gitbox.apache.org/repos/asf/kafka.git


 was 584db5ede7b KAFKA-16094: BrokerRegistrationRequest.logDirs field must 
be ignorable

The revisions that were on this branch are still contained in
other references; therefore, this change does not discard any commits
from the repository.



(kafka) branch trunk updated: KAFKA-16094: BrokerRegistrationRequest.logDirs field must be ignorable (#15153)

2024-01-09 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new aa7ba7bd5af KAFKA-16094: BrokerRegistrationRequest.logDirs field must 
be ignorable (#15153)
aa7ba7bd5af is described below

commit aa7ba7bd5afd570c6dcf0db7d180d5a04ca084b4
Author: Colin Patrick McCabe 
AuthorDate: Tue Jan 9 15:04:27 2024 -0800

KAFKA-16094: BrokerRegistrationRequest.logDirs field must be ignorable 
(#15153)

3.7 brokers must be able to register with 3.6 and earlier controllers. 
Currently, this is broken
because we will unconditionally try to set logDirs, but this field cannot 
be sent with
BrokerRegistrationRequest versions older than v2. This PR marks the logDirs 
field as "ignorable."
Marking the field as "ignorable" means that we will still be able to send 
the
BrokerRegistrationRequest even if the schema doesn't support logDirs.

Reviewers: Ron Dagostino  
---
 .../src/main/resources/common/message/BrokerRegistrationRequest.json  | 2 +-
 .../java/org/apache/kafka/common/requests/RequestResponseTest.java| 4 +++-
 2 files changed, 4 insertions(+), 2 deletions(-)

diff --git 
a/clients/src/main/resources/common/message/BrokerRegistrationRequest.json 
b/clients/src/main/resources/common/message/BrokerRegistrationRequest.json
index ace268db77a..9343465ec82 100644
--- a/clients/src/main/resources/common/message/BrokerRegistrationRequest.json
+++ b/clients/src/main/resources/common/message/BrokerRegistrationRequest.json
@@ -59,7 +59,7 @@
 { "name": "IsMigratingZkBroker", "type": "bool", "versions": "1+", 
"default": "false",
   "about": "If the required configurations for ZK migration are present, 
this value is set to true" },
 { "name": "LogDirs", "type":  "[]uuid", "versions":  "2+",
-  "about": "Log directories configured in this broker which are 
available." },
+  "about": "Log directories configured in this broker which are 
available.", "ignorable": true },
 { "name": "PreviousBrokerEpoch", "type": "int64", "versions": "3+", 
"default": "-1", "ignorable": true,
   "about": "The epoch before a clean shutdown." }
   ]
diff --git 
a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
 
b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
index b1ecb6aa4b4..4a19a3a1683 100644
--- 
a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
@@ -3502,7 +3502,9 @@ public class RequestResponseTest {
 new 
BrokerRegistrationRequestData.Feature()).iterator()))
 .setListeners(new 
BrokerRegistrationRequestData.ListenerCollection(singletonList(
 new 
BrokerRegistrationRequestData.Listener()).iterator()))
-.setIncarnationId(Uuid.randomUuid());
+.setIncarnationId(Uuid.randomUuid())
+
.setLogDirs(Arrays.asList(Uuid.fromString("qaJjNJ05Q36kEgeTBDcj0Q")))
+.setPreviousBrokerEpoch(123L);
 return new BrokerRegistrationRequest.Builder(data).build(v);
 }
 



(kafka) branch KAFKA-16094 created (now 584db5ede7b)

2024-01-08 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a change to branch KAFKA-16094
in repository https://gitbox.apache.org/repos/asf/kafka.git


  at 584db5ede7b KAFKA-16094: BrokerRegistrationRequest.logDirs field must 
be ignorable

This branch includes the following new commits:

 new 584db5ede7b KAFKA-16094: BrokerRegistrationRequest.logDirs field must 
be ignorable

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.




(kafka) 01/01: KAFKA-16094: BrokerRegistrationRequest.logDirs field must be ignorable

2024-01-08 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch KAFKA-16094
in repository https://gitbox.apache.org/repos/asf/kafka.git

commit 584db5ede7b2d8e8345623d971a5e153eb8221b8
Author: Colin P. McCabe 
AuthorDate: Mon Jan 8 14:32:53 2024 -0800

KAFKA-16094: BrokerRegistrationRequest.logDirs field must be ignorable

3.7 brokers must be able to register with 3.6 and earlier controllers. 
Currently, this is broken
because we will unconditionally try to set logDirs, but this field cannot 
be sent with
BrokerRegistrationRequest versions older than v2. This PR marks the logDirs 
field as "ignorable."
Marking the field as "ignorable" means that we will still be able to send 
the
BrokerRegistrationRequest even if the schema doesn't support logDirs.
---
 .../src/main/resources/common/message/BrokerRegistrationRequest.json  | 2 +-
 .../java/org/apache/kafka/common/requests/RequestResponseTest.java| 4 +++-
 2 files changed, 4 insertions(+), 2 deletions(-)

diff --git 
a/clients/src/main/resources/common/message/BrokerRegistrationRequest.json 
b/clients/src/main/resources/common/message/BrokerRegistrationRequest.json
index ace268db77a..9343465ec82 100644
--- a/clients/src/main/resources/common/message/BrokerRegistrationRequest.json
+++ b/clients/src/main/resources/common/message/BrokerRegistrationRequest.json
@@ -59,7 +59,7 @@
 { "name": "IsMigratingZkBroker", "type": "bool", "versions": "1+", 
"default": "false",
   "about": "If the required configurations for ZK migration are present, 
this value is set to true" },
 { "name": "LogDirs", "type":  "[]uuid", "versions":  "2+",
-  "about": "Log directories configured in this broker which are 
available." },
+  "about": "Log directories configured in this broker which are 
available.", "ignorable": true },
 { "name": "PreviousBrokerEpoch", "type": "int64", "versions": "3+", 
"default": "-1", "ignorable": true,
   "about": "The epoch before a clean shutdown." }
   ]
diff --git 
a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
 
b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
index b1ecb6aa4b4..4a19a3a1683 100644
--- 
a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
@@ -3502,7 +3502,9 @@ public class RequestResponseTest {
 new 
BrokerRegistrationRequestData.Feature()).iterator()))
 .setListeners(new 
BrokerRegistrationRequestData.ListenerCollection(singletonList(
 new 
BrokerRegistrationRequestData.Listener()).iterator()))
-.setIncarnationId(Uuid.randomUuid());
+.setIncarnationId(Uuid.randomUuid())
+
.setLogDirs(Arrays.asList(Uuid.fromString("qaJjNJ05Q36kEgeTBDcj0Q")))
+.setPreviousBrokerEpoch(123L);
 return new BrokerRegistrationRequest.Builder(data).build(v);
 }
 



(kafka) branch trunk updated: KAFKA-15956: MetadataShell must take the log directory lock when reading (#14899)

2023-12-10 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 32fdb8d173d KAFKA-15956: MetadataShell must take the log directory 
lock when reading (#14899)
32fdb8d173d is described below

commit 32fdb8d173dfc1e92f4f0d3b3cf5b0b9d294b4dd
Author: Colin Patrick McCabe 
AuthorDate: Sun Dec 10 19:18:34 2023 -0800

KAFKA-15956: MetadataShell must take the log directory lock when reading 
(#14899)

MetadataShell should take an advisory lock on the .lock file of the 
directory it is reading from.
Add an integration test of this functionality in 
MetadataShellIntegrationTest.java.

Note: in build.gradle, I had to add some dependencies on server-common's 
test files in order to use
MockFaultHandler, etc.

MetadataBatchLoader.java: fix a case where a log message was incorrect.  
The intention was to print
the number equivalent to (offset + index).  Instead it was printing the 
offset, followed by the
index. So if the offset was 100 and the index was 1, 1001 would be printed 
rather than 101.

Co-authored-by: Igor Soarez 
Reviewers: David Arthur , José Armando García Sancio 

---
 build.gradle   |   5 +
 checkstyle/import-control.xml  |   1 +
 .../kafka/image/loader/MetadataBatchLoader.java|   2 +-
 .../java/org/apache/kafka/shell/MetadataShell.java |  63 +++
 .../kafka/shell/MetadataShellIntegrationTest.java  | 123 +
 5 files changed, 193 insertions(+), 1 deletion(-)

diff --git a/build.gradle b/build.gradle
index f5e8283cdc6..24b299a023d 100644
--- a/build.gradle
+++ b/build.gradle
@@ -2091,6 +2091,11 @@ project(':shell') {
 implementation libs.jacksonJaxrsJsonProvider
 
 testImplementation project(':clients')
+testImplementation project(':clients').sourceSets.test.output
+testImplementation project(':core')
+testImplementation project(':core').sourceSets.test.output
+testImplementation project(':server-common')
+testImplementation project(':server-common').sourceSets.test.output
 testImplementation libs.junitJupiter
 
 testRuntimeOnly libs.slf4jlog4j
diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
index 4379136bacc..1569494b455 100644
--- a/checkstyle/import-control.xml
+++ b/checkstyle/import-control.xml
@@ -275,6 +275,7 @@
 
 
 
+
 
 
 
diff --git 
a/metadata/src/main/java/org/apache/kafka/image/loader/MetadataBatchLoader.java 
b/metadata/src/main/java/org/apache/kafka/image/loader/MetadataBatchLoader.java
index 70648bef11a..cdf95723d42 100644
--- 
a/metadata/src/main/java/org/apache/kafka/image/loader/MetadataBatchLoader.java
+++ 
b/metadata/src/main/java/org/apache/kafka/image/loader/MetadataBatchLoader.java
@@ -135,7 +135,7 @@ public class MetadataBatchLoader {
 replay(record);
 } catch (Throwable e) {
 faultHandler.handleFault("Error loading metadata log record 
from offset " +
-batch.baseOffset() + indexWithinBatch, e);
+(batch.baseOffset() + indexWithinBatch), e);
 }
 
 // Emit the accumulated delta if a new transaction has been 
started and one of the following is true
diff --git a/shell/src/main/java/org/apache/kafka/shell/MetadataShell.java 
b/shell/src/main/java/org/apache/kafka/shell/MetadataShell.java
index f99695d3f12..094eabef8b3 100644
--- a/shell/src/main/java/org/apache/kafka/shell/MetadataShell.java
+++ b/shell/src/main/java/org/apache/kafka/shell/MetadataShell.java
@@ -19,6 +19,7 @@ package org.apache.kafka.shell;
 
 import kafka.raft.KafkaRaftManager;
 import kafka.tools.TerseFailure;
+import kafka.utils.FileLock;
 import net.sourceforge.argparse4j.ArgumentParsers;
 import net.sourceforge.argparse4j.inf.ArgumentParser;
 import net.sourceforge.argparse4j.inf.Namespace;
@@ -36,6 +37,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.BufferedWriter;
+import java.io.File;
 import java.io.OutputStreamWriter;
 import java.io.PrintWriter;
 import java.nio.charset.StandardCharsets;
@@ -59,6 +61,8 @@ public final class MetadataShell {
 private String snapshotPath = null;
 private FaultHandler faultHandler = new LoggingFaultHandler("shell", 
() -> { });
 
+// Note: we assume that we have already taken the lock on the log 
directory before calling
+// this method.
 public Builder setRaftManager(KafkaRaftManager 
raftManager) {
 this.raftManager = raftManager;
 return this;
@@ -81,6 +85,52 @@ public final class MetadataShell {
 }
 }
 
+/**
+ * Return the parent directory of a file. This works around Java's quirky 
API,
+ * which does not ho

(kafka) branch trunk updated (93b6df6173d -> 8c184b47430)

2023-12-08 Thread cmccabe
This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


from 93b6df6173d KAFKA-15364: Handle log directory failure in the 
Controller (#14902)
 add 8c184b47430 MINOR: Fix some AssignmentsManager bugs (#14954)

No new revisions were added by this update.

Summary of changes:
 .../main/scala/kafka/server/ReplicaManager.scala   |   3 +-
 .../apache/kafka/server/AssignmentsManager.java|  60 +++-
 .../kafka/server/AssignmentsManagerTest.java   | 161 -
 3 files changed, 180 insertions(+), 44 deletions(-)



  1   2   3   4   5   6   7   8   9   >