You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2018/02/27 23:02:14 UTC

[GitHub] merlimat closed pull request #1301: Make BackoffTest deterministic, fixes #1299

merlimat closed pull request #1301: Make BackoffTest deterministic, fixes #1299
URL: https://github.com/apache/incubator-pulsar/pull/1301
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/Backoff.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/Backoff.java
index 9f796800a..aa506da90 100644
--- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/Backoff.java
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/Backoff.java
@@ -18,6 +18,8 @@
  */
 package org.apache.pulsar.client.impl;
 
+import com.google.common.annotations.VisibleForTesting;
+import java.time.Clock;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
 
@@ -27,19 +29,28 @@
     private static final long MAX_BACKOFF_INTERVAL_NANOSECONDS = TimeUnit.SECONDS.toNanos(30);
     private final long initial;
     private final long max;
+    private final Clock clock;
     private long next;
     private long mandatoryStop;
-    long firstBackoffTimeInMillis;
+
+    private long firstBackoffTimeInMillis;
     private boolean mandatoryStopMade = false;
 
     private static final Random random = new Random();
 
-    public Backoff(long initial, TimeUnit unitInitial, long max, TimeUnit unitMax, long mandatoryStop,
-            TimeUnit unitMandatoryStop) {
+    @VisibleForTesting
+    Backoff(long initial, TimeUnit unitInitial, long max, TimeUnit unitMax, long mandatoryStop,
+            TimeUnit unitMandatoryStop, Clock clock) {
         this.initial = unitInitial.toMillis(initial);
         this.max = unitMax.toMillis(max);
         this.next = this.initial;
         this.mandatoryStop = unitMandatoryStop.toMillis(mandatoryStop);
+        this.clock = clock;
+    }
+
+    public Backoff(long initial, TimeUnit unitInitial, long max, TimeUnit unitMax, long mandatoryStop,
+            TimeUnit unitMandatoryStop) {
+        this(initial, unitInitial, max, unitMax, mandatoryStop, unitMandatoryStop, Clock.systemDefaultZone());
     }
 
     public long next() {
@@ -50,7 +61,7 @@ public long next() {
         
         // Check for mandatory stop
         if (!mandatoryStopMade) {
-            long now = System.currentTimeMillis();
+            long now = clock.millis();
             long timeElapsedSinceFirstBackoff = 0;
             if (initial == current) {
                 firstBackoffTimeInMillis = now;
@@ -83,6 +94,11 @@ public void reset() {
         this.mandatoryStopMade = false;
     }
 
+    @VisibleForTesting
+    long getFirstBackoffTimeInMillis() {
+        return firstBackoffTimeInMillis;
+    }
+
     public static boolean shouldBackoff(long initialTimestamp, TimeUnit unitInitial, int failedAttempts) {
         long initialTimestampInNano = unitInitial.toNanos(initialTimestamp);
         long currentTime = System.nanoTime();
diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BackoffTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BackoffTest.java
index 0463954b6..6bdb2136b 100644
--- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BackoffTest.java
+++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BackoffTest.java
@@ -18,35 +18,34 @@
  */
 package org.apache.pulsar.client.impl;
 
-import static org.testng.Assert.*;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertTrue;
 
+import java.time.Clock;
+import java.time.Instant;
+import java.time.ZoneId;
 import java.util.concurrent.TimeUnit;
-
-import org.apache.pulsar.client.impl.Backoff;
+import org.mockito.Mockito;
 import org.testng.annotations.Test;
 
 public class BackoffTest {
     boolean withinTenPercentAndDecrementTimer(Backoff backoff, long t2) {
         long t1 = backoff.next();
-        backoff.firstBackoffTimeInMillis -= t2;
         return (t1 >= t2 * 0.9 && t1 <= t2);
     }
 
     boolean checkExactAndDecrementTimer(Backoff backoff, long t2) {
         long t1 = backoff.next();
-        backoff.firstBackoffTimeInMillis -= t2;
         return t1 == t2;
     }
     @Test
     public void shouldBackoffTest() {
-        long currentTimestamp = System.nanoTime();
-        Backoff testBackoff = new Backoff(currentTimestamp, TimeUnit.NANOSECONDS, 100, TimeUnit.MICROSECONDS, 0,
-                TimeUnit.NANOSECONDS);
         // gives false
-        assertTrue(!testBackoff.shouldBackoff(0L, TimeUnit.NANOSECONDS, 0));
-        currentTimestamp = System.nanoTime();
+        assertTrue(!Backoff.shouldBackoff(0L, TimeUnit.NANOSECONDS, 0));
+        long currentTimestamp = System.nanoTime();
         // gives true
-        assertTrue(testBackoff.shouldBackoff(currentTimestamp, TimeUnit.NANOSECONDS, 100));
+        assertTrue(Backoff.shouldBackoff(currentTimestamp, TimeUnit.NANOSECONDS, 100));
     }
 
     @Test
@@ -60,23 +59,32 @@ public void mandatoryStopTestNegativeTest() {
     }
     
     @Test
-    public void firstBackoffTimerTest() throws InterruptedException {
-        Backoff backoff = new Backoff(100, TimeUnit.MILLISECONDS, 60, TimeUnit.SECONDS, 1900, TimeUnit.MILLISECONDS);
+    public void firstBackoffTimerTest() {
+        Clock mockClock = Mockito.mock(Clock.class);
+        Mockito.when(mockClock.millis())
+            .thenReturn(0L)
+            .thenReturn(300L);
+
+        Backoff backoff = new Backoff(
+            100, TimeUnit.MILLISECONDS,
+            60, TimeUnit.SECONDS,
+            1900, TimeUnit.MILLISECONDS,
+            mockClock
+        );
+
         assertEquals(backoff.next(), 100);
-        long firstBackOffTime = backoff.firstBackoffTimeInMillis;
-        Thread.sleep(300);
-        long diffBackOffTime = backoff.firstBackoffTimeInMillis - firstBackOffTime;
-        assertEquals(diffBackOffTime, 0);
-        
+
+        long firstBackOffTime = backoff.getFirstBackoffTimeInMillis();
         backoff.reset();
         assertEquals(backoff.next(), 100);
-        diffBackOffTime = backoff.firstBackoffTimeInMillis - firstBackOffTime;
-        assertTrue(diffBackOffTime >= 300 && diffBackOffTime < 310);
+        long diffBackOffTime = backoff.getFirstBackoffTimeInMillis() - firstBackOffTime;
+        assertTrue(diffBackOffTime == 300);
     }
     
     @Test
     public void basicTest() {
-        Backoff backoff = new Backoff(5, TimeUnit.MILLISECONDS, 60, TimeUnit.SECONDS, 60, TimeUnit.SECONDS);
+        Clock mockClock = Clock.fixed(Instant.EPOCH, ZoneId.systemDefault());
+        Backoff backoff = new Backoff(5, TimeUnit.MILLISECONDS, 60, TimeUnit.SECONDS, 60, TimeUnit.SECONDS, mockClock);
         assertTrue(checkExactAndDecrementTimer(backoff, 5));
         assertTrue(withinTenPercentAndDecrementTimer(backoff, 10));
         backoff.reset();
@@ -85,7 +93,20 @@ public void basicTest() {
 
     @Test
     public void maxTest() {
-        Backoff backoff = new Backoff(5, TimeUnit.MILLISECONDS, 20, TimeUnit.MILLISECONDS, 20, TimeUnit.MILLISECONDS);
+        Clock mockClock = Mockito.mock(Clock.class);
+        Mockito.when(mockClock.millis())
+            .thenReturn(0L)
+            .thenReturn(10L)
+            .thenReturn(20L)
+            .thenReturn(40L);
+
+        Backoff backoff = new Backoff(
+            5, TimeUnit.MILLISECONDS,
+            20, TimeUnit.MILLISECONDS,
+            20, TimeUnit.MILLISECONDS,
+            mockClock
+        );
+
         assertTrue(checkExactAndDecrementTimer(backoff, 5));
         assertTrue(withinTenPercentAndDecrementTimer(backoff, 10));
         assertTrue(withinTenPercentAndDecrementTimer(backoff, 5));
@@ -94,68 +115,75 @@ public void maxTest() {
 
     @Test
     public void mandatoryStopTest() {
-        Backoff backoff = new Backoff(100, TimeUnit.MILLISECONDS, 60, TimeUnit.SECONDS, 1900, TimeUnit.MILLISECONDS);
+        Clock mockClock = Mockito.mock(Clock.class);
+
+        Backoff backoff = new Backoff(
+            100, TimeUnit.MILLISECONDS,
+            60, TimeUnit.SECONDS,
+            1900, TimeUnit.MILLISECONDS,
+            mockClock
+        );
+
+        Mockito.when(mockClock.millis()).thenReturn(0L);
         assertTrue(checkExactAndDecrementTimer(backoff, 100));
+        Mockito.when(mockClock.millis()).thenReturn(100L);
         assertTrue(withinTenPercentAndDecrementTimer(backoff, 200));
+        Mockito.when(mockClock.millis()).thenReturn(300L);
         assertTrue(withinTenPercentAndDecrementTimer(backoff, 400));
+        Mockito.when(mockClock.millis()).thenReturn(700L);
         assertTrue(withinTenPercentAndDecrementTimer(backoff, 800));
-        // would have been 1600 w/o the mandatory stop 
+        Mockito.when(mockClock.millis()).thenReturn(1500L);
+
+        // would have been 1600 w/o the mandatory stop
         assertTrue(withinTenPercentAndDecrementTimer(backoff, 400));
+        Mockito.when(mockClock.millis()).thenReturn(1900L);
         assertTrue(withinTenPercentAndDecrementTimer(backoff, 3200));
+        Mockito.when(mockClock.millis()).thenReturn(3200L);
         assertTrue(withinTenPercentAndDecrementTimer(backoff, 6400));
+        Mockito.when(mockClock.millis()).thenReturn(3200L);
         assertTrue(withinTenPercentAndDecrementTimer(backoff, 12800));
+        Mockito.when(mockClock.millis()).thenReturn(6400L);
         assertTrue(withinTenPercentAndDecrementTimer(backoff, 25600));
+        Mockito.when(mockClock.millis()).thenReturn(12800L);
         assertTrue(withinTenPercentAndDecrementTimer(backoff, 51200));
+        Mockito.when(mockClock.millis()).thenReturn(25600L);
         assertTrue(withinTenPercentAndDecrementTimer(backoff, 60000));
+        Mockito.when(mockClock.millis()).thenReturn(51200L);
         assertTrue(withinTenPercentAndDecrementTimer(backoff, 60000));
+        Mockito.when(mockClock.millis()).thenReturn(60000L);
+
         backoff.reset();
+        Mockito.when(mockClock.millis()).thenReturn(0L);
         assertTrue(checkExactAndDecrementTimer(backoff, 100));
+        Mockito.when(mockClock.millis()).thenReturn(100L);
         assertTrue(withinTenPercentAndDecrementTimer(backoff, 200));
+        Mockito.when(mockClock.millis()).thenReturn(300L);
         assertTrue(withinTenPercentAndDecrementTimer(backoff, 400));
+        Mockito.when(mockClock.millis()).thenReturn(700L);
         assertTrue(withinTenPercentAndDecrementTimer(backoff, 800));
+        Mockito.when(mockClock.millis()).thenReturn(1500L);
         // would have been 1600 w/o the mandatory stop
         assertTrue(withinTenPercentAndDecrementTimer(backoff, 400));
 
         backoff.reset();
+        Mockito.when(mockClock.millis()).thenReturn(0L);
         assertTrue(checkExactAndDecrementTimer(backoff, 100));
+        Mockito.when(mockClock.millis()).thenReturn(100L);
         assertTrue(withinTenPercentAndDecrementTimer(backoff, 200));
+        Mockito.when(mockClock.millis()).thenReturn(300L);
         assertTrue(withinTenPercentAndDecrementTimer(backoff, 400));
+        Mockito.when(mockClock.millis()).thenReturn(700L);
         assertTrue(withinTenPercentAndDecrementTimer(backoff, 800));
 
         backoff.reset();
+        Mockito.when(mockClock.millis()).thenReturn(0L);
         assertTrue(checkExactAndDecrementTimer(backoff, 100));
+        Mockito.when(mockClock.millis()).thenReturn(100L);
         assertTrue(withinTenPercentAndDecrementTimer(backoff, 200));
+        Mockito.when(mockClock.millis()).thenReturn(300L);
         assertTrue(withinTenPercentAndDecrementTimer(backoff, 400));
+        Mockito.when(mockClock.millis()).thenReturn(700L);
         assertTrue(withinTenPercentAndDecrementTimer(backoff, 800));
     }
 
-    public void ignoringMandatoryStopTest() {
-        Backoff backoff = new Backoff(100, TimeUnit.MILLISECONDS, 60, TimeUnit.SECONDS, 0, TimeUnit.MILLISECONDS);
-        assertTrue(checkExactAndDecrementTimer(backoff, 100));
-        assertTrue(withinTenPercentAndDecrementTimer(backoff, 200));
-        assertTrue(withinTenPercentAndDecrementTimer(backoff, 400));
-        assertTrue(withinTenPercentAndDecrementTimer(backoff, 800));
-        assertTrue(withinTenPercentAndDecrementTimer(backoff, 1600));
-        assertTrue(withinTenPercentAndDecrementTimer(backoff, 3200));
-        assertTrue(withinTenPercentAndDecrementTimer(backoff, 6400));
-        assertTrue(withinTenPercentAndDecrementTimer(backoff, 12800));
-        assertTrue(withinTenPercentAndDecrementTimer(backoff, 25600));
-        assertTrue(withinTenPercentAndDecrementTimer(backoff, 51200));
-        assertTrue(withinTenPercentAndDecrementTimer(backoff, 60000));
-        assertTrue(withinTenPercentAndDecrementTimer(backoff, 60000));
-
-        backoff.reset();
-        assertTrue(checkExactAndDecrementTimer(backoff, 100));
-        assertTrue(withinTenPercentAndDecrementTimer(backoff, 200));
-        assertTrue(withinTenPercentAndDecrementTimer(backoff, 400));
-        assertTrue(withinTenPercentAndDecrementTimer(backoff, 800));
-        assertTrue(withinTenPercentAndDecrementTimer(backoff, 1600));
-        assertTrue(withinTenPercentAndDecrementTimer(backoff, 3200));
-        assertTrue(withinTenPercentAndDecrementTimer(backoff, 6400));
-        assertTrue(withinTenPercentAndDecrementTimer(backoff, 12800));
-        assertTrue(withinTenPercentAndDecrementTimer(backoff, 25600));
-        assertTrue(withinTenPercentAndDecrementTimer(backoff, 51200));
-        assertTrue(withinTenPercentAndDecrementTimer(backoff, 60000));
-        assertTrue(withinTenPercentAndDecrementTimer(backoff, 60000));
-    }
 }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services