You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ratis.apache.org by sz...@apache.org on 2021/05/05 11:35:55 UTC

[ratis] branch master updated: RATIS-1367. Add null check for RaftConfigurationImpl (#469)

This is an automated email from the ASF dual-hosted git repository.

szetszwo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ratis.git


The following commit(s) were added to refs/heads/master by this push:
     new 040bc52  RATIS-1367. Add null check for RaftConfigurationImpl (#469)
040bc52 is described below

commit 040bc52e19a5e36f5710ccd4fc1981e862e691e8
Author: Roni Juntunen <18...@users.noreply.github.com>
AuthorDate: Wed May 5 14:35:48 2021 +0300

    RATIS-1367. Add null check for RaftConfigurationImpl (#469)
---
 .../ratis/server/impl/RaftConfigurationImpl.java   |  3 ++
 .../ratis/server/impl/RaftServerTestUtil.java      |  4 ++
 .../ratis/server/impl/TestRaftConfiguration.java   | 59 ++++++++++++++++++++++
 3 files changed, 66 insertions(+)

diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java
index bb85f75..dd85078 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftConfigurationImpl.java
@@ -147,6 +147,9 @@ final class RaftConfigurationImpl implements RaftConfiguration {
 
   boolean isHighestPriority(RaftPeerId peerId) {
     RaftPeer target = getPeer(peerId);
+    if (target == null) {
+      return false;
+    }
     Collection<RaftPeer> peers = getCurrentPeers();
     for (RaftPeer peer : peers) {
       if (peer.getPriority() >= target.getPriority() && !peer.equals(target)) {
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java
index f7303f9..74cc079 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftServerTestUtil.java
@@ -172,4 +172,8 @@ public class RaftServerTestUtil {
         server::submitUpdateCommitEvent,
         storage, () -> -1, properties);
   }
+
+  public static boolean isHighestPriority(RaftConfiguration config, RaftPeerId peerId) {
+    return ((RaftConfigurationImpl)config).isHighestPriority(peerId);
+  }
 }
diff --git a/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftConfiguration.java b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftConfiguration.java
new file mode 100644
index 0000000..23cae78
--- /dev/null
+++ b/ratis-test/src/test/java/org/apache/ratis/server/impl/TestRaftConfiguration.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.server.impl;
+
+import org.apache.ratis.BaseTest;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.protocol.RaftPeerId;
+import org.apache.ratis.server.RaftConfiguration;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.stream.Collectors;
+
+
+public class TestRaftConfiguration extends BaseTest {
+  @Test
+  public void testIsHighestPriority() {
+    Integer node1 = 0;
+    Integer node2 = 1;
+    Integer node3 = 2;
+    PeerConfiguration peerConfig = new PeerConfiguration(raftPeersWithPriority(node1, node2, node3));
+    RaftConfiguration config = RaftConfigurationImpl.newBuilder().setConf(peerConfig).build();
+    RaftPeer[] allRaftPeers = peerConfig.getPeers().toArray(new RaftPeer[peerConfig.getPeers().size()]);
+
+    // First member should not have highest priority
+    Assert.assertFalse(RaftServerTestUtil.isHighestPriority(config,
+        allRaftPeers[0].getId()));
+
+    // Last member should have highest priority
+    Assert.assertTrue(RaftServerTestUtil.isHighestPriority(config,
+        allRaftPeers[allRaftPeers.length - 1].getId()));
+
+    // Should return false for non existent peer id
+    Assert.assertFalse(RaftServerTestUtil.isHighestPriority(config, RaftPeerId.valueOf("123456789")));
+  }
+
+  private Collection<RaftPeer> raftPeersWithPriority(Integer... voters) {
+    return Arrays.stream(voters)
+        .map(id -> RaftPeer.newBuilder().setPriority(id).setId(id.toString()).build())
+        .collect(Collectors.toSet());
+  }
+}
\ No newline at end of file