Skip to content

Commit

Permalink
Address comments about style
Browse files Browse the repository at this point in the history
  • Loading branch information
frankvicky committed Feb 1, 2025
1 parent cdafa03 commit 74df38e
Show file tree
Hide file tree
Showing 3 changed files with 20 additions and 20 deletions.
28 changes: 10 additions & 18 deletions raft/src/main/java/org/apache/kafka/raft/utils/ApiMessageUtils.java
Original file line number Diff line number Diff line change
Expand Up @@ -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);
};
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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) {
Expand Down
5 changes: 4 additions & 1 deletion raft/src/main/java/org/apache/kafka/raft/utils/VoteRpc.java
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,10 @@ public static VoteResponseData singletonVoteResponse(
.setErrorCode(partitionLevelError.code())
.setLeaderId(leaderId)
.setLeaderEpoch(leaderEpoch)
.setVoteGranted(voteGranted)))));
.setVoteGranted(voteGranted))
)
)
);

if (apiVersion >= 1) {
Optional<InetSocketAddress> address = endpoints.address(listenerName);
Expand Down

0 comments on commit 74df38e

Please sign in to comment.