rondagostino commented on a change in pull request #10823: URL: https://github.com/apache/kafka/pull/10823#discussion_r646148894
########## File path: metadata/src/main/java/org/apache/kafka/controller/StripedReplicaPlacer.java ########## @@ -340,14 +340,14 @@ int numUnfencedBrokers() { } List<Integer> place(int replicationFactor) { - if (replicationFactor <= 0) { - throw new InvalidReplicationFactorException("Invalid replication factor " + - replicationFactor + ": the replication factor must be positive."); - } + throwInvalidReplicationFactorIfNonPositive(replicationFactor); + throwInvalidReplicationFactorIfTooFewBrokers(replicationFactor, numTotalBrokers()); + throwInvalidReplicationFactorIfZero(numUnfencedBrokers()); // If we have returned as many assignments as there are unfenced brokers in // the cluster, shuffle the rack list and broker lists to try to avoid // repeating the same assignments again. - if (epoch == numUnfencedBrokers) { + // But don't reset the iteration epoch for a single unfenced broker -- otherwise we would loop forever + if (epoch == numUnfencedBrokers && numUnfencedBrokers > 1) { Review comment: There was no test covering this case, but I added one: `testMultiPartitionTopicPlacementOnSingleUnfencedBroker()` will never finish without this fix. The `while (true)` loop in `RackList.place()` will never exit without this change when placing multiple partitions on a cluster with just a single unfenced broker. The issue is that the iteration epoch will start at 0 for the first partition but (without the change) will be reset back to 0 for the second partition; the `Rack` instance associated with the broker will see the same iteration epoch for the second partition and therefore says it has no more unfenced brokers available. The loop moves to the next rack, but there is no next rack -- there's only the one -- so we around we go again asking the same question, ad infinitum. One might wonder about the validity of resetting the iteration epoch backwards to zero at all -- if it is possible that a rack with a single broker could see some iteration epoch and then be asked to place another partition just at the moment when the epoch loops back to the same value. I think this is not possible because the racks are shuffled once every broker gets an assignment (and hence every rack gets at least one assignment); no rack will see the same iteration epoch again without it seeing a different iteration epoch in between. The degenerate case of just 1 broker is the one we are fixing here: we can't reset the epoch because shuffling has no effect. -- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org