chia7712 commented on code in PR #21523:
URL: https://github.com/apache/kafka/pull/21523#discussion_r2853700891


##########
docs/operations/basic-kafka-operations.md:
##########
@@ -427,9 +427,61 @@ The --verify option can be used with the tool to check the 
status of the partiti
     Reassignment of partition [foo1,0] is completed
     Reassignment of partition [foo2,1] is completed
 
-## Decommissioning brokers
+## Decommissioning brokers and log directories
 
-The partition reassignment tool does not have the ability to automatically 
generate a reassignment plan for decommissioning brokers yet. As such, the 
admin has to come up with a reassignment plan to move the replica for all 
partitions hosted on the broker to be decommissioned, to the rest of the 
brokers. This can be relatively tedious as the reassignment needs to ensure 
that all the replicas are not moved from the decommissioned broker to only one 
other broker. To make this process effortless, we plan to add tooling support 
for decommissioning brokers in the future. 
+### Decommissioning brokers
+
+The first step to decommission brokers is to mark them as cordoned via the 
Admin API.
+
+For example to cordon broker 1:
+
+    $ bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter 
--add-config cordoned.log.dirs=all --entity-type brokers --entity-name 1

Review Comment:
   is ``cordoned.log.dirs=all` correct? I assume the correct symbol should be 
`*` instead



##########
server/src/test/java/org/apache/kafka/server/CordonedLogDirsIntegrationTest.java:
##########
@@ -223,30 +228,119 @@ public void testReassignWithCordonedLogDirs() throws 
Exception {
             assertInstanceOf(InvalidReplicaAssignmentException.class, 
ee.getCause());
 
             // After uncordoning the log dir, we can move the replica on it
-            setCordonedLogDirs(admin, List.of());
+            setCordonedLogDirs(admin, List.of(), BROKER_0);
             admin.alterReplicaLogDirs(Map.of(replica, 
otherLogDir)).all().get();
         }
     }
 
-    private Map<ConfigResource, Collection<AlterConfigOp>> 
cordonedDirsConfig(String value) {
+    @ClusterTest(
+            brokers = 2,
+            controllers = 1
+    )
+    public void testDecommissionBroker() throws ExecutionException, 
InterruptedException {

Review Comment:
   ah, my bad. I assumed the `testDecommissionBroker` wouldn't pass, but the 
case you wrote runs perfectly using `ClusterTest`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to