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 2023/01/10 14:44:47 UTC

[GitHub] [kafka] tinaselenge opened a new pull request, #13102: KAFKA-14371: Remove unused clusterId field from quorum-state file

tinaselenge opened a new pull request, #13102:
URL: https://github.com/apache/kafka/pull/13102

   Remove clusterId field from the KRaft controller's quorum-state file `$LOG_DIR/__cluster_metadata-0/quorum-state`
   Jira: [KAFKA-14371](https://issues.apache.org/jira/browse/KAFKA-14371)
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


-- 
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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] showuon commented on a diff in pull request #13102: KAFKA-14371: Remove unused clusterId field from quorum-state file

Posted by GitBox <gi...@apache.org>.
showuon commented on code in PR #13102:
URL: https://github.com/apache/kafka/pull/13102#discussion_r1067632210


##########
raft/src/test/java/org/apache/kafka/raft/FileBasedStateStoreTest.java:
##########
@@ -90,6 +98,43 @@ public void testWriteElectionState() throws IOException {
         assertFalse(stateFile.exists());
     }
 
+    @Test
+    public void testCompatibilityWithClusterId() throws IOException {
+        final File stateFile = TestUtils.tempFile();
+        stateStore = new FileBasedStateStore(stateFile);
+
+        // We initialized a state from the metadata log
+        assertTrue(stateFile.exists());
+
+        String jsonString = "{\"clusterId\":\"abc\",\"leaderId\":0,\"leaderEpoch\":0,\"votedId\":-1,\"appliedOffset\":0,\"currentVoters\":[],\"data_version\":0}";
+        writeToStateFile(stateFile, jsonString);
+
+        // verify that we can read the state file that contains the removed "cluserId" field.
+        assertEquals(stateStore.readElectionState(), new ElectionState(0,
+                OptionalInt.of(0), OptionalInt.empty(), Collections.emptySet()));

Review Comment:
   1. we should put the expected result in the first parameter (ref: [here](https://junit.org/junit5/docs/5.0.1/api/org/junit/jupiter/api/Assertions.html#assertEquals-java.lang.Object-java.lang.Object-))
   2. I saw we use `ElectionState.withElectedLeader`, instead of `new ElectionState`. Do you think we can use the former one for consistency?



-- 
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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] showuon commented on pull request #13102: KAFKA-14371: Remove unused clusterId field from quorum-state file

Posted by "showuon (via GitHub)" <gi...@apache.org>.
showuon commented on PR #13102:
URL: https://github.com/apache/kafka/pull/13102#issuecomment-1445268492

   @tinaselenge , could you check the failed test: `testCompatibilityWithClusterId`?
   
   https://ci-builds.apache.org/job/Kafka/job/kafka-pr/job/PR-13102/4/
   
   Also, please try to rebase to the latest trunk to make sure everything else is still working. Thanks.


-- 
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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] showuon commented on pull request #13102: KAFKA-14371: Remove unused clusterId field from quorum-state file

Posted by "showuon (via GitHub)" <gi...@apache.org>.
showuon commented on PR #13102:
URL: https://github.com/apache/kafka/pull/13102#issuecomment-1449214463

   Failed tests are unrelated:
   ```
   Build / JDK 11 and Scala 2.13 / org.apache.kafka.connect.mirror.integration.DedicatedMirrorIntegrationTest.testSingleNodeCluster()
   Build / JDK 11 and Scala 2.13 / org.apache.kafka.tools.MetadataQuorumCommandTest.[1] Type=Raft-CoReside, Name=testDescribeQuorumReplicationSuccessful, MetadataVersion=3.5-IV0, Security=PLAINTEXT
   Build / JDK 17 and Scala 2.13 / org.apache.kafka.clients.consumer.CooperativeStickyAssignorTest.testLargeAssignmentAndGroupWithUniformSubscription()
   Build / JDK 17 and Scala 2.13 / org.apache.kafka.connect.mirror.integration.DedicatedMirrorIntegrationTest.testSingleNodeCluster()
   Build / JDK 17 and Scala 2.13 / kafka.controller.ControllerIntegrationTest.testTopicIdPersistsThroughControllerRestart()
   Build / JDK 17 and Scala 2.13 / kafka.zk.ZkMigrationIntegrationTest.[1] Type=ZK, Name=testMigrate, MetadataVersion=3.4-IV0, Security=PLAINTEXT
   Build / JDK 17 and Scala 2.13 / org.apache.kafka.controller.QuorumControllerTest.testConfigurationOperations()
   Build / JDK 17 and Scala 2.13 / org.apache.kafka.streams.processor.internals.DefaultStateUpdaterTest.shouldPauseActiveTaskAndTransitToUpdateStandby()
   Build / JDK 8 and Scala 2.12 / org.apache.kafka.connect.mirror.integration.DedicatedMirrorIntegrationTest.testMultiNodeCluster()
   Build / JDK 8 and Scala 2.12 / org.apache.kafka.connect.mirror.integration.DedicatedMirrorIntegrationTest.testMultiNodeCluster()
   Build / JDK 8 and Scala 2.12 / kafka.server.ListOffsetsRequestWithRemoteStoreTest.testResponseIncludesLeaderEpoch()
   Build / JDK 8 and Scala 2.12 / org.apache.kafka.streams.processor.internals.DefaultStateUpdaterTest.shouldPauseActiveTaskAndTransitToUpdateStandby()
   Build / JDK 8 and Scala 2.12 / org.apache.kafka.streams.processor.internals.DefaultStateUpdaterTest.shouldPauseActiveTaskAndTransitToUpdateStandby()
   ```


-- 
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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] showuon commented on a diff in pull request #13102: KAFKA-14371: Remove unused clusterId field from quorum-state file

Posted by GitBox <gi...@apache.org>.
showuon commented on code in PR #13102:
URL: https://github.com/apache/kafka/pull/13102#discussion_r1067632210


##########
raft/src/test/java/org/apache/kafka/raft/FileBasedStateStoreTest.java:
##########
@@ -90,6 +98,43 @@ public void testWriteElectionState() throws IOException {
         assertFalse(stateFile.exists());
     }
 
+    @Test
+    public void testCompatibilityWithClusterId() throws IOException {
+        final File stateFile = TestUtils.tempFile();
+        stateStore = new FileBasedStateStore(stateFile);
+
+        // We initialized a state from the metadata log
+        assertTrue(stateFile.exists());
+
+        String jsonString = "{\"clusterId\":\"abc\",\"leaderId\":0,\"leaderEpoch\":0,\"votedId\":-1,\"appliedOffset\":0,\"currentVoters\":[],\"data_version\":0}";
+        writeToStateFile(stateFile, jsonString);
+
+        // verify that we can read the state file that contains the removed "cluserId" field.
+        assertEquals(stateStore.readElectionState(), new ElectionState(0,
+                OptionalInt.of(0), OptionalInt.empty(), Collections.emptySet()));

Review Comment:
   1. we should put the expected result in the first parameter, so that the failed output will be meaningful. (ref: [here](https://junit.org/junit5/docs/5.0.1/api/org/junit/jupiter/api/Assertions.html#assertEquals-java.lang.Object-java.lang.Object-))
   2. I saw we use `ElectionState.withElectedLeader`, instead of `new ElectionState`. Do you think we can use the former one for consistency?



-- 
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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] tinaselenge commented on a diff in pull request #13102: KAFKA-14371: Remove unused clusterId field from quorum-state file

Posted by GitBox <gi...@apache.org>.
tinaselenge commented on code in PR #13102:
URL: https://github.com/apache/kafka/pull/13102#discussion_r1067964100


##########
raft/src/test/java/org/apache/kafka/raft/FileBasedStateStoreTest.java:
##########
@@ -90,6 +98,43 @@ public void testWriteElectionState() throws IOException {
         assertFalse(stateFile.exists());
     }
 
+    @Test
+    public void testCompatibilityWithClusterId() throws IOException {
+        final File stateFile = TestUtils.tempFile();
+        stateStore = new FileBasedStateStore(stateFile);
+
+        // We initialized a state from the metadata log
+        assertTrue(stateFile.exists());
+
+        String jsonString = "{\"clusterId\":\"abc\",\"leaderId\":0,\"leaderEpoch\":0,\"votedId\":-1,\"appliedOffset\":0,\"currentVoters\":[],\"data_version\":0}";
+        writeToStateFile(stateFile, jsonString);
+
+        // verify that we can read the state file that contains the removed "cluserId" field.
+        assertEquals(stateStore.readElectionState(), new ElectionState(0,
+                OptionalInt.of(0), OptionalInt.empty(), Collections.emptySet()));

Review Comment:
   Thanks Luke for pointing them out. Fixed the test now :) 



-- 
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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] showuon commented on pull request #13102: KAFKA-14371: Remove unused clusterId field from quorum-state file

Posted by "showuon (via GitHub)" <gi...@apache.org>.
showuon commented on PR #13102:
URL: https://github.com/apache/kafka/pull/13102#issuecomment-1441441978

   @jsancio @hachikuji , I guess the `clusterId` field is only used in ZK mode, not in KRaft mode. During ZK migrating to KRaft, we also won't rely on this field. So it is safe to delete. If you don't have any other comments, I'll merge it this week. Thanks.


-- 
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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] showuon commented on pull request #13102: KAFKA-14371: Remove unused clusterId field from quorum-state file

Posted by GitBox <gi...@apache.org>.
showuon commented on PR #13102:
URL: https://github.com/apache/kafka/pull/13102#issuecomment-1378254330

   This file is not a public API, so I think updating it doesn't need a KIP. Thanks.


-- 
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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] showuon commented on a diff in pull request #13102: KAFKA-14371: Remove unused clusterId field from quorum-state file

Posted by GitBox <gi...@apache.org>.
showuon commented on code in PR #13102:
URL: https://github.com/apache/kafka/pull/13102#discussion_r1066580834


##########
raft/src/main/resources/common/message/QuorumStateData.json:
##########
@@ -19,7 +19,6 @@
   "validVersions": "0",

Review Comment:
   Bumping the version is a good point, and add a comment to mention what have changed like [this](https://github.com/apache/kafka/blob/trunk/clients/src/main/resources/common/message/FetchRequest.json#L24) did.



-- 
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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jsancio commented on pull request #13102: KAFKA-14371: Remove unused clusterId field from quorum-state file

Posted by "jsancio (via GitHub)" <gi...@apache.org>.
jsancio commented on PR #13102:
URL: https://github.com/apache/kafka/pull/13102#issuecomment-1450767005

   Thanks for the change and review. I think it is safe to remove the cluster id from `quorum-state`. The cluster id is stored and read from the file `meta.properties` in `KafkaConfig.metadataLogDir`. See https://github.com/apache/kafka/blob/510e99e1a2636f9a7035020f682ab7df8530986b/core/src/main/scala/kafka/raft/RaftManager.scala#L244


-- 
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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] ijuma commented on pull request #13102: KAFKA-14371: Remove unused clusterId field from quorum-state file

Posted by GitBox <gi...@apache.org>.
ijuma commented on PR #13102:
URL: https://github.com/apache/kafka/pull/13102#issuecomment-1383075297

   @jsancio @hachikuji Was there a reason to have this field or was it added accidentally?


-- 
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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] tinaselenge commented on pull request #13102: KAFKA-14371: Remove unused clusterId field from quorum-state file

Posted by "tinaselenge (via GitHub)" <gi...@apache.org>.
tinaselenge commented on PR #13102:
URL: https://github.com/apache/kafka/pull/13102#issuecomment-1458147611

   @jsancio thank you for catching this and reverting the change!


-- 
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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] dengziming commented on a diff in pull request #13102: KAFKA-14371: Remove unused clusterId field from quorum-state file

Posted by GitBox <gi...@apache.org>.
dengziming commented on code in PR #13102:
URL: https://github.com/apache/kafka/pull/13102#discussion_r1066561563


##########
raft/src/main/resources/common/message/QuorumStateData.json:
##########
@@ -19,7 +19,6 @@
   "validVersions": "0",

Review Comment:
   I guess this change is compatible since the structure is only used in json schema and json is friendly to adding and removing fields, but do you think we should bump this version.



-- 
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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jsancio commented on pull request #13102: KAFKA-14371: Remove unused clusterId field from quorum-state file

Posted by "jsancio (via GitHub)" <gi...@apache.org>.
jsancio commented on PR #13102:
URL: https://github.com/apache/kafka/pull/13102#issuecomment-1457138403

   I was thinking about this change today and realized that this change is not backwards compatible. For example, the user upgrades the cluster and then they downgrade the cluster, it will stop working. For example, I got the following error if the `clusterId` is removed:
   
   ```
   [2023-03-06 14:31:09,673] ERROR [SharedServer id=1] Got exception while starting SharedServer (kafka.server.SharedServer)
   java.lang.RuntimeException: QuorumStateData: unable to locate field 'clusterId', which is mandatory in version 1
           at org.apache.kafka.raft.generated.QuorumStateDataJsonConverter.read(QuorumStateDataJsonConverter.java:40)
           at org.apache.kafka.raft.FileBasedStateStore.readStateFromFile(FileBasedStateStore.java:94)
           at org.apache.kafka.raft.FileBasedStateStore.readElectionState(FileBasedStateStore.java:110)
           at org.apache.kafka.raft.QuorumState.initialize(QuorumState.java:116)
           at org.apache.kafka.raft.KafkaRaftClient.initialize(KafkaRaftClient.java:387)
           at kafka.raft.KafkaRaftManager.buildRaftClient(RaftManager.scala:256)
           at kafka.raft.KafkaRaftManager.<init>(RaftManager.scala:179)
           at kafka.server.SharedServer.start(SharedServer.scala:245)
           at kafka.server.SharedServer.startForController(SharedServer.scala:137)
           at kafka.server.ControllerServer.startup(ControllerServer.scala:332)
           at kafka.server.KafkaRaftServer.$anonfun$startup$1(KafkaRaftServer.scala:134)
           at kafka.server.KafkaRaftServer.$anonfun$startup$1$adapted(KafkaRaftServer.scala:134)
           at scala.Option.foreach(Option.scala:437)
           at kafka.server.KafkaRaftServer.startup(KafkaRaftServer.scala:134)
           at kafka.Kafka$.main(Kafka.scala:114)
           at kafka.Kafka.main(Kafka.scala)
   [2023-03-06 14:31:09,675] INFO Waiting for controller quorum voters future (kafka.server.ControllerServer)
   [2023-03-06 14:31:09,676] INFO Finished waiting for controller quorum voters future (kafka.server.ControllerServer)
   [2023-03-06 14:31:09,677] ERROR [ControllerServer id=1] Fatal error during controller startup. Prepare to shutdown (kafka.server.ControllerServer)
   java.lang.NullPointerException: Cannot invoke "kafka.raft.KafkaRaftManager.apiVersions()" because the return value of "kafka.server.SharedServer.raftManager()" is null
           at kafka.server.ControllerServer.startup(ControllerServer.scala:349)
           at kafka.server.KafkaRaftServer.$anonfun$startup$1(KafkaRaftServer.scala:134)
           at kafka.server.KafkaRaftServer.$anonfun$startup$1$adapted(KafkaRaftServer.scala:134)
           at scala.Option.foreach(Option.scala:437)
           at kafka.server.KafkaRaftServer.startup(KafkaRaftServer.scala:134)
           at kafka.Kafka$.main(Kafka.scala:114)
           at kafka.Kafka.main(Kafka.scala)
   [2023-03-06 14:31:09,678] INFO [ControllerServer id=1] shutting down (kafka.server.ControllerServer)
   [2023-03-06 14:31:09,680] INFO [SocketServer listenerType=CONTROLLER, nodeId=1] Stopping socket server request processors (kafka.network.SocketServer)
   [2023-03-06 14:31:09,683] INFO [SocketServer listenerType=CONTROLLER, nodeId=1] Stopped socket server request processors (kafka.network.SocketServer)
   [2023-03-06 14:31:09,684] INFO [SocketServer listenerType=CONTROLLER, nodeId=1] Shutting down socket server (kafka.network.SocketServer)
   [2023-03-06 14:31:09,698] INFO [SocketServer listenerType=CONTROLLER, nodeId=1] Shutdown completed (kafka.network.SocketServer)
   [2023-03-06 14:31:09,699] ERROR Exiting Kafka due to fatal exception during startup. (kafka.Kafka$)
   java.lang.NullPointerException: Cannot invoke "kafka.raft.KafkaRaftManager.apiVersions()" because the return value of "kafka.server.SharedServer.raftManager()" is null
           at kafka.server.ControllerServer.startup(ControllerServer.scala:349)
           at kafka.server.KafkaRaftServer.$anonfun$startup$1(KafkaRaftServer.scala:134)
           at kafka.server.KafkaRaftServer.$anonfun$startup$1$adapted(KafkaRaftServer.scala:134)
           at scala.Option.foreach(Option.scala:437)
           at kafka.server.KafkaRaftServer.startup(KafkaRaftServer.scala:134)
           at kafka.Kafka$.main(Kafka.scala:114)
           at kafka.Kafka.main(Kafka.scala)
   ```
   
   I am going to revert this change.
   
   To make a similar change in the future we need a KIP since it requires changes to the on-disk format of a file.
   
   cc @tinaselenge @showuon 


-- 
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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] showuon merged pull request #13102: KAFKA-14371: Remove unused clusterId field from quorum-state file

Posted by "showuon (via GitHub)" <gi...@apache.org>.
showuon merged PR #13102:
URL: https://github.com/apache/kafka/pull/13102


-- 
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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] tinaselenge commented on pull request #13102: KAFKA-14371: Remove unused clusterId field from quorum-state file

Posted by "tinaselenge (via GitHub)" <gi...@apache.org>.
tinaselenge commented on PR #13102:
URL: https://github.com/apache/kafka/pull/13102#issuecomment-1434560483

   Can this please be merged now? Or should we wait for responses to @ijuma's question? 


-- 
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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] showuon commented on pull request #13102: KAFKA-14371: Remove unused clusterId field from quorum-state file

Posted by "showuon (via GitHub)" <gi...@apache.org>.
showuon commented on PR #13102:
URL: https://github.com/apache/kafka/pull/13102#issuecomment-1457302953

   Nice catch @jsancio !


-- 
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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] jsancio commented on pull request #13102: KAFKA-14371: Remove unused clusterId field from quorum-state file

Posted by "jsancio (via GitHub)" <gi...@apache.org>.
jsancio commented on PR #13102:
URL: https://github.com/apache/kafka/pull/13102#issuecomment-1450774546

   > @jsancio @hachikuji Was there a reason to have this field or was it added accidentally?
   
   @ijuma I think this is just an artifact that `quorum-state` was implemented before `KIP-631` added cluster id to `meta.properites`. In that change we missed removing cluster id from `quorum-state`.
   
   


-- 
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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] tinaselenge commented on a diff in pull request #13102: KAFKA-14371: Remove unused clusterId field from quorum-state file

Posted by GitBox <gi...@apache.org>.
tinaselenge commented on code in PR #13102:
URL: https://github.com/apache/kafka/pull/13102#discussion_r1067142496


##########
raft/src/main/resources/common/message/QuorumStateData.json:
##########
@@ -19,7 +19,6 @@
   "validVersions": "0",

Review Comment:
   Thank you for the feedback. I have addressed this in the latest commit. 



-- 
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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] showuon commented on a diff in pull request #13102: KAFKA-14371: Remove unused clusterId field from quorum-state file

Posted by GitBox <gi...@apache.org>.
showuon commented on code in PR #13102:
URL: https://github.com/apache/kafka/pull/13102#discussion_r1067632210


##########
raft/src/test/java/org/apache/kafka/raft/FileBasedStateStoreTest.java:
##########
@@ -90,6 +98,43 @@ public void testWriteElectionState() throws IOException {
         assertFalse(stateFile.exists());
     }
 
+    @Test
+    public void testCompatibilityWithClusterId() throws IOException {
+        final File stateFile = TestUtils.tempFile();
+        stateStore = new FileBasedStateStore(stateFile);
+
+        // We initialized a state from the metadata log
+        assertTrue(stateFile.exists());
+
+        String jsonString = "{\"clusterId\":\"abc\",\"leaderId\":0,\"leaderEpoch\":0,\"votedId\":-1,\"appliedOffset\":0,\"currentVoters\":[],\"data_version\":0}";
+        writeToStateFile(stateFile, jsonString);
+
+        // verify that we can read the state file that contains the removed "cluserId" field.
+        assertEquals(stateStore.readElectionState(), new ElectionState(0,
+                OptionalInt.of(0), OptionalInt.empty(), Collections.emptySet()));

Review Comment:
   1. we should put the expected result in the first parameter, so that the failed output will be meaningful. (ref: [here](https://junit.org/junit5/docs/5.0.1/api/org/junit/jupiter/api/Assertions.html#assertEquals-java.lang.Object-java.lang.Object-))
   2. I saw we use `ElectionState.withElectedLeader`, instead of `new ElectionState` in other tests. Do you think we can use the former one for consistency?



-- 
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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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