Skip to content

Commit

Permalink
KAFKA-16907: Rebase and fix conflicts
Browse files Browse the repository at this point in the history
  • Loading branch information
frankvicky committed Aug 9, 2024
1 parent d5f639c commit d4b745c
Show file tree
Hide file tree
Showing 4 changed files with 9 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -3082,7 +3082,7 @@ private long maybeSendFetchOrFetchSnapshot(FollowerState state, long currentTime
}

private UpdateRaftVoterRequestData buildUpdateVoterRequest() {
return RaftUtil.updateVoterRequest(
return VoteRpc.updateVoterRequest(
clusterId,
quorum.localReplicaKeyOrThrow(),
quorum.epoch(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -204,7 +204,7 @@ public CompletableFuture<UpdateRaftVoterResponseData> handleUpdateVoterRequest(

// Reply immediately and don't wait for the change to commit
return CompletableFuture.completedFuture(
RaftUtil.updateVoterResponse(
VoteRpc.updateVoterResponse(
Errors.NONE,
requestListenerName,
new LeaderAndEpoch(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,10 +16,12 @@
*/
package org.apache.kafka.raft.utils;

import org.apache.kafka.common.message.ApiVersionsResponseData;
import org.apache.kafka.common.message.BeginQuorumEpochResponseData;
import org.apache.kafka.common.message.EndQuorumEpochResponseData;
import org.apache.kafka.common.message.FetchResponseData;
import org.apache.kafka.common.message.FetchSnapshotResponseData;
import org.apache.kafka.common.message.UpdateRaftVoterResponseData;
import org.apache.kafka.common.message.VoteResponseData;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ApiMessage;
Expand All @@ -38,6 +40,10 @@ public static ApiMessage parseErrorResponse(ApiKeys apiKey, Errors error) {
return new FetchResponseData().setErrorCode(error.code());
case FETCH_SNAPSHOT:
return new FetchSnapshotResponseData().setErrorCode(error.code());
case API_VERSIONS:
return new ApiVersionsResponseData().setErrorCode(error.code());
case UPDATE_RAFT_VOTER:
return new UpdateRaftVoterResponseData().setErrorCode(error.code());
default:
throw new IllegalArgumentException("Received response for unexpected request type: " + apiKey);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1758,7 +1758,7 @@ UpdateRaftVoterResponseData updateVoterResponse(
Errors error,
LeaderAndEpoch leaderAndEpoch
) {
return RaftUtil.updateVoterResponse(
return VoteRpc.updateVoterResponse(
error,
channel.listenerName(),
leaderAndEpoch,
Expand Down

0 comments on commit d4b745c

Please sign in to comment.