diff --git a/raft/src/main/java/org/apache/kafka/raft/utils/ApiMessageUtils.java b/raft/src/main/java/org/apache/kafka/raft/utils/ApiMessageUtils.java index e4bac676ca80b..7807ce3c6918c 100644 --- a/raft/src/main/java/org/apache/kafka/raft/utils/ApiMessageUtils.java +++ b/raft/src/main/java/org/apache/kafka/raft/utils/ApiMessageUtils.java @@ -29,23 +29,15 @@ public class ApiMessageUtils { public static ApiMessage parseErrorResponse(ApiKeys apiKey, Errors error) { - switch (apiKey) { - case VOTE: - return new VoteResponseData().setErrorCode(error.code()); - case BEGIN_QUORUM_EPOCH: - return new BeginQuorumEpochResponseData().setErrorCode(error.code()); - case END_QUORUM_EPOCH: - return new EndQuorumEpochResponseData().setErrorCode(error.code()); - case FETCH: - 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); - } + return switch (apiKey) { + case VOTE -> new VoteResponseData().setErrorCode(error.code()); + case BEGIN_QUORUM_EPOCH -> new BeginQuorumEpochResponseData().setErrorCode(error.code()); + case END_QUORUM_EPOCH -> new EndQuorumEpochResponseData().setErrorCode(error.code()); + case FETCH -> new FetchResponseData().setErrorCode(error.code()); + case FETCH_SNAPSHOT -> new FetchSnapshotResponseData().setErrorCode(error.code()); + case API_VERSIONS -> new ApiVersionsResponseData().setErrorCode(error.code()); + case UPDATE_RAFT_VOTER -> new UpdateRaftVoterResponseData().setErrorCode(error.code()); + default -> throw new IllegalArgumentException("Received response for unexpected request type: " + apiKey); + }; } } diff --git a/raft/src/main/java/org/apache/kafka/raft/utils/DescribeQuorumRpc.java b/raft/src/main/java/org/apache/kafka/raft/utils/DescribeQuorumRpc.java index 9b695513410a4..197302ab7733d 100644 --- a/raft/src/main/java/org/apache/kafka/raft/utils/DescribeQuorumRpc.java +++ b/raft/src/main/java/org/apache/kafka/raft/utils/DescribeQuorumRpc.java @@ -72,7 +72,12 @@ public static DescribeQuorumResponseData singletonDescribeQuorumResponse( .setLeaderEpoch(leaderEpoch) .setHighWatermark(highWatermark) .setCurrentVoters(toReplicaStates(apiVersion, leaderId, voters, currentTimeMs)) - .setObservers(toReplicaStates(apiVersion, leaderId, observers, currentTimeMs)))))); + .setObservers(toReplicaStates(apiVersion, leaderId, observers, currentTimeMs)) + ) + ) + ) + ); + if (apiVersion >= 2) { DescribeQuorumResponseData.NodeCollection nodes = new DescribeQuorumResponseData.NodeCollection(voters.size()); for (LeaderState.ReplicaState voter : voters) { diff --git a/raft/src/main/java/org/apache/kafka/raft/utils/EndQuorumEpochRpc.java b/raft/src/main/java/org/apache/kafka/raft/utils/EndQuorumEpochRpc.java index 7ecf8ea8e73e5..d0d90b00f6800 100644 --- a/raft/src/main/java/org/apache/kafka/raft/utils/EndQuorumEpochRpc.java +++ b/raft/src/main/java/org/apache/kafka/raft/utils/EndQuorumEpochRpc.java @@ -91,7 +91,8 @@ public static EndQuorumEpochResponseData singletonEndQuorumEpochResponse( .setErrorCode(partitionLevelError.code()) .setLeaderId(leaderId) .setLeaderEpoch(leaderEpoch) - ))) + )) + ) ); if (apiVersion >= 1) { diff --git a/raft/src/main/java/org/apache/kafka/raft/utils/VoteRpc.java b/raft/src/main/java/org/apache/kafka/raft/utils/VoteRpc.java index 1085c90828bb2..30033cf283811 100644 --- a/raft/src/main/java/org/apache/kafka/raft/utils/VoteRpc.java +++ b/raft/src/main/java/org/apache/kafka/raft/utils/VoteRpc.java @@ -92,7 +92,10 @@ public static VoteResponseData singletonVoteResponse( .setErrorCode(partitionLevelError.code()) .setLeaderId(leaderId) .setLeaderEpoch(leaderEpoch) - .setVoteGranted(voteGranted))))); + .setVoteGranted(voteGranted)) + ) + ) + ); if (apiVersion >= 1) { Optional address = endpoints.address(listenerName);