Browse Source

ZOOKEEPER-3470: Fix flaky LearnerMetricsTest (again)

- based on the logs, this flakyness happened because some time the client session gets initiated before the observer joined
- the client session is now initiated against the observer (to make sure it is already in a CONNECTED state)
- the number of participants set to 3 to make sure we have every participant connected after leader election
- the comments were a bit misleading in the test (the leader election messages are not counted here)

Author: Mate Szalay-Beko <szalay.beko.mate@gmail.com>

Reviewers: Andor Molnar <andor@apache.org>, Norbert Kalmar <nkalmar@apache.org>

Closes #1094 from symat/ZOOKEEPER-3470
Mate Szalay-Beko 5 năm trước cách đây
mục cha
commit
93dca3c254

+ 20 - 11
zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LearnerMetricsTest.java

@@ -36,7 +36,7 @@ import org.junit.Test;
 public class LearnerMetricsTest extends QuorumPeerTestBase {
 
     private static final int TIMEOUT_SECONDS = 30;
-    private static final int SERVER_COUNT = 6; // 5 participants, 1 observer
+    private static final int SERVER_COUNT = 4; // 1 observer, 3 participants
     private final QuorumPeerTestBase.MainThread[] mt = new QuorumPeerTestBase.MainThread[SERVER_COUNT];
     private ZooKeeper zk_client;
 
@@ -57,7 +57,7 @@ public class LearnerMetricsTest extends QuorumPeerTestBase {
             sb.append("server." + i + "=127.0.0.1:" + PortAssignment.unique() + ":" + PortAssignment.unique() + "\n");
         }
 
-        // start the participants
+        // start the three participants
         String quorumCfgSection = sb.toString();
         for (int i = 1; i < SERVER_COUNT; i++) {
             mt[i] = new QuorumPeerTestBase.MainThread(i, clientPorts[i], quorumCfgSection);
@@ -70,21 +70,30 @@ public class LearnerMetricsTest extends QuorumPeerTestBase {
         mt[observer] = new QuorumPeerTestBase.MainThread(observer, clientPorts[observer], quorumCfgSection, observerConfig);
         mt[observer].start();
 
-        // send one create request
-        zk_client = new ZooKeeper("127.0.0.1:" + clientPorts[1], ClientBase.CONNECTION_TIMEOUT, this);
+        // connect to the observer node and wait for CONNECTED state
+        // (this way we make sure to wait until the leader election finished and the observer node joined as well)
+        zk_client = new ZooKeeper("127.0.0.1:" + clientPorts[observer], ClientBase.CONNECTION_TIMEOUT, this);
         waitForOne(zk_client, ZooKeeper.States.CONNECTED);
+
+        // creating a node
         zk_client.create(path, data, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
 
-        // there are 4 followers, each received two proposals, one for leader election, one for the create request
-        waitForMetric("learner_proposal_received_count", is(8L));
-        waitForMetric("cnt_proposal_latency", is(8L));
+
+        // there are two proposals by now, one for the global client session creation, one for the create request
+
+        // there are two followers, each received two PROPOSALs
+        waitForMetric("learner_proposal_received_count", is(4L));
+        waitForMetric("cnt_proposal_latency", is(4L));
         waitForMetric("min_proposal_latency", greaterThanOrEqualTo(0L));
-        waitForMetric("cnt_proposal_ack_creation_latency", is(10L));
+
+        // the two ACKs are processed by the leader and by each of the two followers
+        waitForMetric("cnt_proposal_ack_creation_latency", is(6L));
         waitForMetric("min_proposal_ack_creation_latency", greaterThanOrEqualTo(0L));
 
-        // there are five learners, each received two commits, one for leader election, one for the create request
-        waitForMetric("learner_commit_received_count", is(10L));
-        waitForMetric("cnt_commit_propagation_latency", is(10L));
+        // two COMMITs are received by each of the two followers, and two INFORMs are received by the single observer
+        // (the INFORM message is also counted into the "commit_received" metrics)
+        waitForMetric("learner_commit_received_count", is(6L));
+        waitForMetric("cnt_commit_propagation_latency", is(6L));
         waitForMetric("min_commit_propagation_latency", greaterThanOrEqualTo(0L));
     }