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 2019/03/07 22:29:13 UTC

[incubator-ratis] branch master updated: RATIS-494. Throw AssumptionViolatedException if a test fails to change leader.

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/incubator-ratis.git


The following commit(s) were added to refs/heads/master by this push:
     new 55030b2  RATIS-494. Throw AssumptionViolatedException if a test fails to change leader.
55030b2 is described below

commit 55030b233bb0a98449141a3a731020960e4403e9
Author: Tsz Wo Nicholas Sze <sz...@apache.org>
AuthorDate: Thu Mar 7 14:28:48 2019 -0800

    RATIS-494. Throw AssumptionViolatedException if a test fails to change leader.
---
 pom.xml                                            |  2 +-
 .../main/java/org/apache/ratis/util/JavaUtils.java | 11 +----------
 .../src/test/java/org/apache/ratis/BaseTest.java   |  2 +-
 .../java/org/apache/ratis/JUnitRunListener.java    | 22 ++++++++++++----------
 .../apache/ratis/server/impl/ServerImplUtils.java  |  7 +++++--
 .../test/java/org/apache/ratis/RaftTestUtil.java   | 18 +++++++++++++-----
 .../apache/ratis/server/ServerRestartTests.java    |  2 +-
 .../ratis/server/impl/LeaderElectionTests.java     | 18 ++++++------------
 .../server/impl/RaftReconfigurationBaseTest.java   |  6 ++++--
 9 files changed, 44 insertions(+), 44 deletions(-)

diff --git a/pom.xml b/pom.xml
index ded40d7..22cf545 100644
--- a/pom.xml
+++ b/pom.xml
@@ -350,7 +350,7 @@
       <dependency>
         <groupId>junit</groupId>
         <artifactId>junit</artifactId>
-        <version>4.11</version>
+        <version>4.12</version>
       </dependency>
       <dependency>
         <groupId>org.mockito</groupId>
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java b/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java
index 1572aa8..916d27a 100644
--- a/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java
+++ b/ratis-common/src/main/java/org/apache/ratis/util/JavaUtils.java
@@ -75,7 +75,7 @@ public interface JavaUtils {
   }
 
   static <T extends Throwable> void runAsUnchecked(CheckedRunnable<T> runnable) {
-    runAsUnchecked(runnable::run, RuntimeException::new);
+    runAsUnchecked(runnable, RuntimeException::new);
   }
 
   static <THROWABLE extends Throwable> void runAsUnchecked(
@@ -133,15 +133,6 @@ public interface JavaUtils {
     return ROOT_THREAD_GROUP.get();
   }
 
-  /** @deprecated use {@link #attempt(CheckedSupplier, int, TimeDuration, String, Logger)} */
-  @Deprecated
-  static <RETURN, THROWABLE extends Throwable> RETURN attempt(
-      CheckedSupplier<RETURN, THROWABLE> supplier,
-      int numAttempts, long sleepMs, String name, Logger log)
-      throws THROWABLE, InterruptedException {
-    return attempt(supplier, numAttempts, TimeDuration.valueOf(sleepMs, TimeUnit.MILLISECONDS), name, log);
-  }
-
   /** Attempt to get a return value from the given supplier multiple times. */
   static <RETURN, THROWABLE extends Throwable> RETURN attempt(
       CheckedSupplier<RETURN, THROWABLE> supplier,
diff --git a/ratis-common/src/test/java/org/apache/ratis/BaseTest.java b/ratis-common/src/test/java/org/apache/ratis/BaseTest.java
index 55024bb..278f9ca 100644
--- a/ratis-common/src/test/java/org/apache/ratis/BaseTest.java
+++ b/ratis-common/src/test/java/org/apache/ratis/BaseTest.java
@@ -60,7 +60,7 @@ public abstract class BaseTest {
   }
 
   @Rule
-  public final Timeout globalTimeout = new Timeout(getGlobalTimeoutSeconds() * 1000);
+  public final Timeout globalTimeout = new Timeout(getGlobalTimeoutSeconds(), TimeUnit.SECONDS );
 
   @Rule
   public final TestName testName = new TestName();
diff --git a/ratis-common/src/test/java/org/apache/ratis/JUnitRunListener.java b/ratis-common/src/test/java/org/apache/ratis/JUnitRunListener.java
index 723eeab..144c806 100644
--- a/ratis-common/src/test/java/org/apache/ratis/JUnitRunListener.java
+++ b/ratis-common/src/test/java/org/apache/ratis/JUnitRunListener.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -22,34 +22,36 @@ import org.junit.internal.runners.statements.FailOnTimeout;
 import org.junit.runner.notification.Failure;
 import org.junit.runner.notification.RunListener;
 import org.junit.runners.model.Statement;
+import org.junit.runners.model.TestTimedOutException;
 
 import java.io.PrintStream;
+import java.util.concurrent.TimeUnit;
 
 /**
  * A {@link RunListener} to dump all threads after a test timeout failure.
  */
 public class JUnitRunListener extends RunListener {
   private static final Throwable TIMEOUT_EXCEPTION = getTimeoutException();
-  private static final String TIMEOUT_EXCEPTION_PREFIX = getTimeoutExceptionPrefix(TIMEOUT_EXCEPTION);
+  private static final String TIMEOUT_EXCEPTION_PREFIX;
 
   private static Throwable getTimeoutException() {
-    final FailOnTimeout f = new FailOnTimeout(new Statement() {
+    final FailOnTimeout f = FailOnTimeout.builder().withTimeout(1, TimeUnit.NANOSECONDS).build(new Statement() {
       @Override
       public void evaluate() throws InterruptedException {
         Thread.sleep(1000);
       }
-    }, 1);
+    });
     try {
       f.evaluate();
     } catch(Throwable throwable) {
       return throwable;
     }
-    return null;
+    throw new IllegalStateException("Failed to getTimeoutException");
   }
 
-  private static String getTimeoutExceptionPrefix(Throwable timeoutException) {
-    final String message = timeoutException.getMessage();
-    return message.substring(0, message.indexOf('1'));
+  static {
+    final String message = JUnitRunListener.TIMEOUT_EXCEPTION.getMessage();
+    TIMEOUT_EXCEPTION_PREFIX = message.substring(0, message.indexOf('1'));
   }
 
   private final PrintStream out = System.out;
@@ -81,9 +83,9 @@ public class JUnitRunListener extends RunListener {
 
   public static void main(String[] args) {
     final JUnitRunListener listener = new JUnitRunListener();
-    listener.out.println("TIMEOUT_EXCEPTION_PREFIX = " + TIMEOUT_EXCEPTION_PREFIX);
+    listener.out.println("TIMEOUT_EXCEPTION_PREFIX = '" + TIMEOUT_EXCEPTION_PREFIX + "'");
     TIMEOUT_EXCEPTION.printStackTrace(listener.out);
 
-    listener.testFailure(new Failure(null, new Exception(TIMEOUT_EXCEPTION_PREFIX + "999 milliseconds")));
+    listener.testFailure(new Failure(null, new TestTimedOutException(999, TimeUnit.MILLISECONDS)));
   }
 }
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java
index 24b5530..65494e1 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerImplUtils.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -25,8 +25,10 @@ import org.apache.ratis.server.protocol.TermIndex;
 import org.apache.ratis.statemachine.StateMachine;
 import org.apache.ratis.util.IOUtils;
 import org.apache.ratis.util.JavaUtils;
+import org.apache.ratis.util.TimeDuration;
 
 import java.io.IOException;
+import java.util.concurrent.TimeUnit;
 
 /** Server utilities for internal use. */
 public class ServerImplUtils {
@@ -43,12 +45,13 @@ public class ServerImplUtils {
   private static RaftServerProxy newRaftServer(
       RaftPeerId id, StateMachine.Registry stateMachineRegistry, RaftProperties properties, Parameters parameters)
       throws IOException {
+    final TimeDuration sleepTime = TimeDuration.valueOf(500, TimeUnit.MILLISECONDS);
     final RaftServerProxy proxy;
     try {
       // attempt multiple times to avoid temporary bind exception
       proxy = JavaUtils.attempt(
           () -> new RaftServerProxy(id, stateMachineRegistry, properties, parameters),
-          5, 500L, "new RaftServerProxy", RaftServerProxy.LOG);
+          5, sleepTime, "new RaftServerProxy", RaftServerProxy.LOG);
     } catch (InterruptedException e) {
       throw IOUtils.toInterruptedIOException(
           "Interrupted when creating RaftServer " + id, e);
diff --git a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java
index 96fc578..5f782a0 100644
--- a/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java
+++ b/ratis-server/src/test/java/org/apache/ratis/RaftTestUtil.java
@@ -38,10 +38,10 @@ import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
 import org.apache.ratis.util.AutoCloseableLock;
 import org.apache.ratis.util.CollectionUtils;
 import org.apache.ratis.util.JavaUtils;
-import org.apache.ratis.util.Preconditions;
 import org.apache.ratis.util.ProtoUtils;
 import org.apache.ratis.util.TimeDuration;
 import org.junit.Assert;
+import org.junit.AssumptionViolatedException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -59,6 +59,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.BooleanSupplier;
 import java.util.function.Consumer;
+import java.util.function.Function;
 import java.util.function.IntSupplier;
 import java.util.function.Predicate;
 
@@ -345,16 +346,23 @@ public interface RaftTestUtil {
   }
 
   static RaftPeerId changeLeader(MiniRaftCluster cluster, RaftPeerId oldLeader)
-      throws InterruptedException {
+      throws Exception {
+    return changeLeader(cluster, oldLeader, AssumptionViolatedException::new);
+  }
+
+  static RaftPeerId changeLeader(MiniRaftCluster cluster, RaftPeerId oldLeader, Function<String, Exception> constructor)
+      throws Exception {
+    final String name = JavaUtils.getCallerStackTraceElement().getMethodName() + "-changeLeader";
     cluster.setBlockRequestsFrom(oldLeader.toString(), true);
     try {
       return JavaUtils.attempt(() -> {
         final RaftPeerId newLeader = waitForLeader(cluster).getId();
-        Preconditions.assertTrue(!newLeader.equals(oldLeader),
-            () -> "Failed to change leader: newLeader=" + newLeader + " equals oldLeader=" + oldLeader);
+        if (newLeader.equals(oldLeader)) {
+          throw constructor.apply("Failed to change leader: newLeader == oldLeader == " + oldLeader);
+        }
         LOG.info("Changed leader from " + oldLeader + " to " + newLeader);
         return newLeader;
-      }, 10, 100L, "changeLeader", LOG);
+      }, 20, BaseTest.HUNDRED_MILLIS, name, LOG);
     } finally {
       cluster.setBlockRequestsFrom(oldLeader.toString(), false);
     }
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/ServerRestartTests.java b/ratis-server/src/test/java/org/apache/ratis/server/ServerRestartTests.java
index f91cd27..f8f1a4c 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/ServerRestartTests.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/ServerRestartTests.java
@@ -188,7 +188,7 @@ public abstract class ServerRestartTests<CLUSTER extends MiniRaftCluster>
 
     for(RaftServerImpl impl : cluster.iterateServerImpls()) {
       final File openLogFile = JavaUtils.attempt(() -> getOpenLogFile(impl),
-          10, 100, impl.getId() + "-getOpenLogFile", LOG);
+          10, HUNDRED_MILLIS, impl.getId() + "-getOpenLogFile", LOG);
       for(int i = 0; i < SegmentedRaftLogFormat.getHeaderLength(); i++) {
         assertCorruptedLogHeader(impl.getId(), openLogFile, i, cluster, LOG);
         Assert.assertTrue(getOpenLogFiles(impl).isEmpty());
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java
index a48edc4..f31620f 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -33,6 +33,7 @@ import org.junit.Test;
 import org.slf4j.Logger;
 
 import java.util.Iterator;
+import java.util.Optional;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
 
@@ -69,7 +70,7 @@ public abstract class LeaderElectionTests<CLUSTER extends MiniRaftCluster>
 
     RaftPeerId leader = RaftTestUtil.waitForLeader(cluster).getId();
     for(int i = 0; i < 10; i++) {
-      leader = RaftTestUtil.changeLeader(cluster, leader);
+      leader = RaftTestUtil.changeLeader(cluster, leader, IllegalStateException::new);
       ExitUtils.assertNotTerminated();
     }
     RaftStorageTestUtils.setRaftLogWorkerLogLevel(Level.INFO);
@@ -129,17 +130,10 @@ public abstract class LeaderElectionTests<CLUSTER extends MiniRaftCluster>
     final RaftServerProxy lastServer = i.next();
     lastServer.start();
     final RaftPeerId lastServerLeaderId = JavaUtils.attempt(
-        () -> getLeader(lastServer.getImpls().iterator().next().getState()),
-        10, 1000, "getLeaderId", LOG);
+        () -> Optional.ofNullable(lastServer.getImpls().iterator().next().getState().getLeaderId())
+            .orElseThrow(() -> new IllegalStateException("No leader yet")),
+        10, ONE_SECOND, "getLeaderId", LOG);
     LOG.info(cluster.printServers());
     Assert.assertEquals(leader.getId(), lastServerLeaderId);
   }
-
-  static RaftPeerId getLeader(ServerState state) {
-    final RaftPeerId leader = state.getLeaderId();
-    if (leader == null) {
-      throw new IllegalStateException("No leader yet");
-    }
-    return leader;
-  }
 }
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java
index a8f4c7b..39f39e7 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/RaftReconfigurationBaseTest.java
@@ -32,6 +32,7 @@ import org.apache.ratis.server.storage.RaftLog;
 import org.apache.ratis.server.storage.RaftStorageTestUtils;
 import org.apache.ratis.util.JavaUtils;
 import org.apache.ratis.util.LogUtils;
+import org.apache.ratis.util.TimeDuration;
 import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.internal.util.reflection.Whitebox;
@@ -518,6 +519,7 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER extends MiniRaftCluste
       clientThread.start();
 
       // find ConfigurationEntry
+      final TimeDuration sleepTime = TimeDuration.valueOf(500, TimeUnit.MILLISECONDS);
       final long confIndex = JavaUtils.attempt(() -> {
         final long last = log.getLastEntryTermIndex().getIndex();
         for (long i = last; i >= 1; i--) {
@@ -526,11 +528,11 @@ public abstract class RaftReconfigurationBaseTest<CLUSTER extends MiniRaftCluste
           }
         }
         throw new Exception("ConfigurationEntry not found: last=" + last);
-      }, 10, 500, "confIndex", LOG);
+      }, 10, sleepTime, "confIndex", LOG);
 
       // wait till the old leader persist the new conf
       JavaUtils.attempt(() -> log.getLatestFlushedIndex() >= confIndex,
-          10, 500L, "FLUSH", LOG);
+          10, sleepTime, "FLUSH", LOG);
       final long committed = log.getLastCommittedIndex();
       Assert.assertTrue(committed < confIndex);