You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ratis.apache.org by "runzhiwang (Jira)" <ji...@apache.org> on 2020/05/07 11:11:00 UTC

[jira] [Comment Edited] (RATIS-900) Failed UT: RaftExceptionBaseTest.testHandleNotLeaderAndIOException

    [ https://issues.apache.org/jira/browse/RATIS-900?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17101557#comment-17101557 ] 

runzhiwang edited comment on RATIS-900 at 5/7/20, 11:10 AM:
------------------------------------------------------------

 In the test, the cluster first select an oldLeader, then force change to newLeader, and kill newLeader. Then client connect to oldLeader and expect the oldLeader reply the info of real leader i.e. newLeader in the NotLeaderException. But the newLeader has been killed,  ratis will triger a leader election, so the original newLeader can not be included in the NotLeaderException. so the test failed. I want to delete this test i.e. testHandleNotLeaderAndIOException, what do you think [~jing9] ?

{code:java}
 @Test
  public void testHandleNotLeaderAndIOException() throws Exception {
    runWithNewCluster(NUM_PEERS, cluster -> runTestHandleNotLeaderException(true, cluster));
  }

  void runTestHandleNotLeaderException(boolean killNewLeader, CLUSTER cluster) throws Exception {
    final RaftPeerId oldLeader = RaftTestUtil.waitForLeader(cluster).getId();
    try(final RaftClient client = cluster.createClient(oldLeader)) {
      sendMessage("m1", client);

      // enforce leader change
      final RaftPeerId newLeader = RaftTestUtil.changeLeader(cluster, oldLeader);

      if (killNewLeader) {
        // kill the new leader
        cluster.killServer(newLeader);
      }

      final RaftClientRpc rpc = client.getClientRpc();
      JavaUtils.attemptRepeatedly(() -> assertNotLeaderException(newLeader, "m2", oldLeader, rpc, cluster),
          10, ONE_SECOND, "assertNotLeaderException", LOG);

      sendMessage("m3", client);
    }
  }
{code}



was (Author: yjxxtd):
 In the test, the cluster first select an oldLeader, then force change to newLeader, and kill newLeader. Then client connect to oldLeader and expect the oldLeader reply the info of real leader i.e. newLeader in the NotLeaderException. But the newLeader has been killed,  ratis will triger a leader election, so the original newLeader can not be included in the NotLeaderException. so the test failed. I want to delete this test, what do you think [~jing9] ?

{code:java}
 @Test
  public void testHandleNotLeaderAndIOException() throws Exception {
    runWithNewCluster(NUM_PEERS, cluster -> runTestHandleNotLeaderException(true, cluster));
  }

  void runTestHandleNotLeaderException(boolean killNewLeader, CLUSTER cluster) throws Exception {
    final RaftPeerId oldLeader = RaftTestUtil.waitForLeader(cluster).getId();
    try(final RaftClient client = cluster.createClient(oldLeader)) {
      sendMessage("m1", client);

      // enforce leader change
      final RaftPeerId newLeader = RaftTestUtil.changeLeader(cluster, oldLeader);

      if (killNewLeader) {
        // kill the new leader
        cluster.killServer(newLeader);
      }

      final RaftClientRpc rpc = client.getClientRpc();
      JavaUtils.attemptRepeatedly(() -> assertNotLeaderException(newLeader, "m2", oldLeader, rpc, cluster),
          10, ONE_SECOND, "assertNotLeaderException", LOG);

      sendMessage("m3", client);
    }
  }
{code}


> Failed UT: RaftExceptionBaseTest.testHandleNotLeaderAndIOException
> ------------------------------------------------------------------
>
>                 Key: RATIS-900
>                 URL: https://issues.apache.org/jira/browse/RATIS-900
>             Project: Ratis
>          Issue Type: Sub-task
>            Reporter: runzhiwang
>            Assignee: runzhiwang
>            Priority: Major
>         Attachments: image.png, screenshot-1.png, screenshot-2.png
>
>          Time Spent: 10m
>  Remaining Estimate: 0h
>
>  !screenshot-1.png! 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)