-
Notifications
You must be signed in to change notification settings - Fork 14.1k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
KAFKA-12897: KRaft multi-partition placement on single broker #10823
Conversation
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@rondagostino : Thanks for the PR. Just a couple of comments.
} | ||
throwInvalidReplicationFactorIfNonPositive(replicationFactor); | ||
throwInvalidReplicationFactorIfZero(rackList.numUnfencedBrokers()); | ||
throwInvalidReplicationFactorIfTooFewBrokers(replicationFactor, rackList.numTotalBrokers()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It seems that we are doing the same tests in rackList.place(). Should we just do the tests in one place?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We have three separate argument checks to perform, each with its separate error message that the test check for:
a) replication factory can't be non-positive
b) unfenced broker count can't be 0
c) unfenced broker count must must not be less than replication factor
StripedReplicaPlacer.place()
was performing checks (b) and (c).
RackList.place()
was performing check (a)
Given that RackList
is publicly accessible within the package, I felt it was important to perform all the sanity checks there. But StripedReplicaPlacer.place()
also has a loop and allocates an array, so while we could allow the first iteration of the loop to raise the exception, I felt it was clearer to perform the checks there as well.
// 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) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Which loop loops forever because of this? Also, is there an existing test covering this?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
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 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.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@rondagostino : Thanks for the explanation. LGTM.
The JDK 11 test seems to have failed with 20 failures. Are they related to this PR?
Thanks for the review, @junrao. Those failures are unrelated -- they are due to https://issues.apache.org/jira/browse/KAFKA-12892. |
For example, here's one of the 20 failures:
|
#10494 introduced a bug in the KRaft controller where the controller will loop forever in
StripedReplicaPlacer
trying to identify the racks on which to place partition replicas if there is a single unfenced broker in the cluster and the number of requested partitions in a CREATE_TOPICS request is greater than 1.This patch refactors out some argument sanity checks and invokes those checks in both
RackList
andStripedReplicaPlacer
, and it adds tests for this as well as the single broker placement issue.Committer Checklist (excluded from commit message)