junrao commented on a change in pull request #10823:
URL: https://github.com/apache/kafka/pull/10823#discussion_r645905158



##########
File path: 
metadata/src/main/java/org/apache/kafka/controller/StripedReplicaPlacer.java
##########
@@ -412,14 +433,9 @@ public StripedReplicaPlacer(Random random) {
                                      short replicationFactor,
                                      Iterator<UsableBroker> iterator) {
         RackList rackList = new RackList(random, iterator);
-        if (rackList.numUnfencedBrokers() == 0) {
-            throw new InvalidReplicationFactorException("All brokers are 
currently fenced.");
-        }
-        if (replicationFactor > rackList.numTotalBrokers()) {
-            throw new InvalidReplicationFactorException("The target 
replication factor " +
-                "of " + replicationFactor + " cannot be reached because only " 
+
-                rackList.numTotalBrokers() + " broker(s) are registered.");
-        }
+        throwInvalidReplicationFactorIfNonPositive(replicationFactor);
+        throwInvalidReplicationFactorIfZero(rackList.numUnfencedBrokers());
+        throwInvalidReplicationFactorIfTooFewBrokers(replicationFactor, 
rackList.numTotalBrokers());

Review comment:
       It seems that we are doing the same tests in rackList.place(). Should we 
just do the tests in one place?

##########
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:
       Which loop loops forever because of this? Also, is there an existing 
test covering this? 




-- 
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


Reply via email to