-
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-16533; Update voter handling #16735
Changes from 5 commits
436271f
35c6562
55483a3
1c6f426
392f5a0
e28c537
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -23,6 +23,7 @@ | |
import org.apache.kafka.common.compress.Compression; | ||
import org.apache.kafka.common.errors.ClusterAuthorizationException; | ||
import org.apache.kafka.common.errors.NotLeaderOrFollowerException; | ||
import org.apache.kafka.common.feature.SupportedVersionRange; | ||
import org.apache.kafka.common.memory.MemoryPool; | ||
import org.apache.kafka.common.message.AddRaftVoterRequestData; | ||
import org.apache.kafka.common.message.AddRaftVoterResponseData; | ||
|
@@ -39,6 +40,8 @@ | |
import org.apache.kafka.common.message.FetchSnapshotResponseData; | ||
import org.apache.kafka.common.message.RemoveRaftVoterRequestData; | ||
import org.apache.kafka.common.message.RemoveRaftVoterResponseData; | ||
import org.apache.kafka.common.message.UpdateRaftVoterRequestData; | ||
import org.apache.kafka.common.message.UpdateRaftVoterResponseData; | ||
import org.apache.kafka.common.message.VoteRequestData; | ||
import org.apache.kafka.common.message.VoteResponseData; | ||
import org.apache.kafka.common.metrics.Metrics; | ||
|
@@ -76,6 +79,7 @@ | |
import org.apache.kafka.raft.internals.RemoveVoterHandler; | ||
import org.apache.kafka.raft.internals.ReplicaKey; | ||
import org.apache.kafka.raft.internals.ThresholdPurgatory; | ||
import org.apache.kafka.raft.internals.UpdateVoterHandler; | ||
import org.apache.kafka.raft.internals.VoterSet; | ||
import org.apache.kafka.server.common.KRaftVersion; | ||
import org.apache.kafka.server.common.serialization.RecordSerde; | ||
|
@@ -171,6 +175,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> { | |
private final int fetchMaxWaitMs; | ||
private final String clusterId; | ||
private final Endpoints localListeners; | ||
private final SupportedVersionRange localSupportedKRaftVersion; | ||
private final NetworkChannel channel; | ||
private final ReplicatedLog log; | ||
private final Random random; | ||
|
@@ -207,6 +212,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> { | |
// Specialized handlers | ||
private volatile AddVoterHandler addVoterHandler; | ||
private volatile RemoveVoterHandler removeVoterHandler; | ||
private volatile UpdateVoterHandler updateVoterHandler; | ||
|
||
/** | ||
* Create a new instance. | ||
|
@@ -226,6 +232,7 @@ public KafkaRaftClient( | |
String clusterId, | ||
Collection<InetSocketAddress> bootstrapServers, | ||
Endpoints localListeners, | ||
SupportedVersionRange localSupportedKRaftVersion, | ||
QuorumConfig quorumConfig | ||
) { | ||
this( | ||
|
@@ -242,6 +249,7 @@ public KafkaRaftClient( | |
clusterId, | ||
bootstrapServers, | ||
localListeners, | ||
localSupportedKRaftVersion, | ||
logContext, | ||
new Random(), | ||
quorumConfig | ||
|
@@ -262,6 +270,7 @@ public KafkaRaftClient( | |
String clusterId, | ||
Collection<InetSocketAddress> bootstrapServers, | ||
Endpoints localListeners, | ||
SupportedVersionRange localSupportedKRaftVersion, | ||
LogContext logContext, | ||
Random random, | ||
QuorumConfig quorumConfig | ||
|
@@ -279,6 +288,7 @@ public KafkaRaftClient( | |
this.time = time; | ||
this.clusterId = clusterId; | ||
this.localListeners = localListeners; | ||
this.localSupportedKRaftVersion = localSupportedKRaftVersion; | ||
this.fetchMaxWaitMs = fetchMaxWaitMs; | ||
this.logger = logContext.logger(KafkaRaftClient.class); | ||
this.random = random; | ||
|
@@ -349,6 +359,7 @@ private void onUpdateLeaderHighWatermark( | |
// add or remove voter request that need to be completed | ||
addVoterHandler.highWatermarkUpdated(state); | ||
removeVoterHandler.highWatermarkUpdated(state); | ||
updateVoterHandler.highWatermarkUpdated(state); | ||
|
||
// After updating the high watermark, we first clear the append | ||
// purgatory so that we have an opportunity to route the pending | ||
|
@@ -492,6 +503,7 @@ public void initialize( | |
nodeDirectoryId, | ||
partitionState, | ||
localListeners, | ||
localSupportedKRaftVersion, | ||
quorumConfig.electionTimeoutMs(), | ||
quorumConfig.fetchTimeoutMs(), | ||
quorumStateStore, | ||
|
@@ -543,6 +555,15 @@ public void initialize( | |
quorumConfig.requestTimeoutMs(), | ||
logContext | ||
); | ||
|
||
// Specialized remove voter handler | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: specialized update voter handler There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Done |
||
this.updateVoterHandler = new UpdateVoterHandler( | ||
nodeId, | ||
partitionState, | ||
channel.listenerName(), | ||
time, | ||
quorumConfig.requestTimeoutMs() | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. does this need to be passed in vs imported in UpdateVoterHandler? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Not sure I understand the question. In general inputs to a type or method should be limited to what they require. This avoid passing a configuration object with many methods where only one is used or required. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. sorry, you can ignore this comment. I had mistakenly thought it would be possible to import the underlying config in UpdateVoterHandler |
||
); | ||
} | ||
|
||
@Override | ||
|
@@ -2071,7 +2092,8 @@ private CompletableFuture<AddRaftVoterResponseData> handleAddVoterRequest( | |
.setErrorCode(Errors.INVALID_REQUEST.code()) | ||
.setErrorMessage( | ||
String.format( | ||
"Add voter request didn't include the default listener: %s", | ||
"Add voter request didn't include the endpoint (%s) for the default listener %s", | ||
newVoterEndpoints, | ||
channel.listenerName() | ||
) | ||
) | ||
|
@@ -2153,6 +2175,84 @@ private CompletableFuture<RemoveRaftVoterResponseData> handleRemoveVoterRequest( | |
); | ||
} | ||
|
||
private CompletableFuture<UpdateRaftVoterResponseData> handleUpdateVoterRequest( | ||
RaftRequest.Inbound requestMetadata, | ||
long currentTimeMs | ||
) { | ||
UpdateRaftVoterRequestData data = (UpdateRaftVoterRequestData) requestMetadata.data(); | ||
|
||
if (!hasValidClusterId(data.clusterId())) { | ||
return completedFuture( | ||
RaftUtil.updateVoterResponse( | ||
Errors.INCONSISTENT_CLUSTER_ID, | ||
requestMetadata.listenerName(), | ||
quorum.leaderAndEpoch(), | ||
quorum.leaderEndpoints() | ||
) | ||
); | ||
} | ||
|
||
Optional<Errors> leaderValidation = validateLeaderOnlyRequest(data.currentLeaderEpoch()); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: I know this is the convention handleFetchSnapshotRequest uses too, but could we rename to There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Are you referring to the variable name? Replace "leaderValidation" with "leaderValidationError"? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Done |
||
if (leaderValidation.isPresent()) { | ||
return completedFuture( | ||
RaftUtil.updateVoterResponse( | ||
leaderValidation.get(), | ||
requestMetadata.listenerName(), | ||
quorum.leaderAndEpoch(), | ||
quorum.leaderEndpoints() | ||
) | ||
); | ||
} | ||
|
||
Optional<ReplicaKey> voter = RaftUtil.updateVoterRequestVoterKey(data); | ||
if (!voter.isPresent() || !voter.get().directoryId().isPresent()) { | ||
return completedFuture( | ||
RaftUtil.updateVoterResponse( | ||
Errors.INVALID_REQUEST, | ||
requestMetadata.listenerName(), | ||
quorum.leaderAndEpoch(), | ||
quorum.leaderEndpoints() | ||
) | ||
); | ||
} | ||
|
||
Endpoints voterEndpoints = Endpoints.fromUpdateVoterRequest(data.listeners()); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. do want to catch any illegalArgumentExceptions that might be thrown from this method? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hmm. Catching IllegalArgumentExceptions is not enough. Let me file an issue to audit all of the validation of the new RPCs: https://issues.apache.org/jira/browse/KAFKA-17264 |
||
if (!voterEndpoints.address(channel.listenerName()).isPresent()) { | ||
return completedFuture( | ||
RaftUtil.updateVoterResponse( | ||
Errors.INVALID_REQUEST, | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. wonder if we might benefit from the update voter response schema including an error message field. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I thought about this before. The ErrorMessage field is a human readable message meant to be displayed to the end user. This is why the admin RPCs (AddVoter and RemoveVoter) have the additional ErrorMessage field. UpdateVoter is an internal RPC. Internal RPCs don't tend to have a human readable error message. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. what about some logging in the error cases? although the RPC is internal, an invalid update could stem from operator action (and it could be useful for them to dig into logs to understand why an update failed) |
||
requestMetadata.listenerName(), | ||
quorum.leaderAndEpoch(), | ||
quorum.leaderEndpoints() | ||
) | ||
); | ||
} | ||
|
||
UpdateRaftVoterRequestData.KRaftVersionFeature supportedKraftVersions = data.kRaftVersionFeature(); | ||
if (supportedKraftVersions.minSupportedVersion() < 0 || | ||
supportedKraftVersions.maxSupportedVersion() < 0 || | ||
supportedKraftVersions.maxSupportedVersion() < supportedKraftVersions.minSupportedVersion() | ||
) { | ||
return completedFuture( | ||
RaftUtil.updateVoterResponse( | ||
Errors.INVALID_REQUEST, | ||
requestMetadata.listenerName(), | ||
quorum.leaderAndEpoch(), | ||
quorum.leaderEndpoints() | ||
) | ||
); | ||
} | ||
|
||
return updateVoterHandler.handleUpdateVoterRequest( | ||
quorum.leaderStateOrThrow(), | ||
requestMetadata.listenerName(), | ||
voter.get(), | ||
voterEndpoints, | ||
supportedKraftVersions, | ||
currentTimeMs | ||
); | ||
} | ||
|
||
private boolean hasConsistentLeader(int epoch, OptionalInt leaderId) { | ||
// Only elected leaders are sent in the request/response header, so if we have an elected | ||
// leaderId, it should be consistent with what is in the message. | ||
|
@@ -2419,6 +2519,10 @@ private void handleRequest(RaftRequest.Inbound request, long currentTimeMs) { | |
responseFuture = handleRemoveVoterRequest(request, currentTimeMs); | ||
break; | ||
|
||
case UPDATE_RAFT_VOTER: | ||
responseFuture = handleUpdateVoterRequest(request, currentTimeMs); | ||
break; | ||
|
||
default: | ||
throw new IllegalArgumentException("Unexpected request type " + apiKey); | ||
} | ||
|
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 like
RequestResponseTest.createUpdateRaftVoterResponse
has not been updated to set these new fields.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.
Fixed.