You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2021/06/01 17:20:28 UTC

[GitHub] [kafka] jsancio commented on a change in pull request #10749: KAFKA-12773: Use UncheckedIOException when wrapping IOException

jsancio commented on a change in pull request #10749:
URL: https://github.com/apache/kafka/pull/10749#discussion_r643303043



##########
File path: raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java
##########
@@ -372,27 +371,23 @@ private void maybeFireLeaderChange() {
 
     @Override
     public void initialize() {
-        try {
-            quorum.initialize(new OffsetAndEpoch(log.endOffset().offset, log.lastFetchedEpoch()));
+        quorum.initialize(new OffsetAndEpoch(log.endOffset().offset, log.lastFetchedEpoch()));
 
-            long currentTimeMs = time.milliseconds();
-            if (quorum.isLeader()) {
-                throw new IllegalStateException("Voter cannot initialize as a Leader");
-            } else if (quorum.isCandidate()) {
-                onBecomeCandidate(currentTimeMs);
-            } else if (quorum.isFollower()) {
-                onBecomeFollower(currentTimeMs);
-            }
+        long currentTimeMs = time.milliseconds();
+        if (quorum.isLeader()) {
+            throw new IllegalStateException("Voter cannot initialize as a Leader");
+        } else if (quorum.isCandidate()) {
+            onBecomeCandidate(currentTimeMs);
+        } else if (quorum.isFollower()) {
+            onBecomeFollower(currentTimeMs);
+        }
 
-            // When there is only a single voter, become candidate immediately
-            if (quorum.isVoter()
-                && quorum.remoteVoters().isEmpty()
-                && !quorum.isCandidate()) {
+        // When there is only a single voter, become candidate immediately
+        if (quorum.isVoter()
+            && quorum.remoteVoters().isEmpty()
+            && !quorum.isCandidate()) {
 
-                transitionToCandidate(currentTimeMs);
-            }
-        } catch (IOException e) {
-            throw new RuntimeException(e);
+            transitionToCandidate(currentTimeMs);

Review comment:
       This is semantically different from the previous code but I think it is okay. If `initialize` throws an io exception the JVM process for the broker will terminate as this is called in the `RaftManager` constructor which is indirectly called from `kafka.Kafka`'s `main`.

##########
File path: raft/src/main/java/org/apache/kafka/raft/QuorumState.java
##########
@@ -463,9 +464,14 @@ public void transitionToCandidate() throws IOException {
         return state;
     }
 
-    private void transitionTo(EpochState state) throws IOException {
+    private void transitionTo(EpochState state) {
         if (this.state != null) {
-            this.state.close();
+            try {
+                this.state.close();
+            } catch (IOException e) {
+                throw new UncheckedIOException(
+                        "Failed to close Quorum status during transition to candidate, state is " + this.state, e);

Review comment:
       How about "Failed to transition from ${this.state.name()} to ${state.name()}"? Note that this string is  not valid Java.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org