|
@@ -20,6 +20,7 @@ package org.apache.hadoop.io.retry;
|
|
|
import static org.junit.Assert.*;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
+import java.util.concurrent.CountDownLatch;
|
|
|
|
|
|
import org.apache.hadoop.io.retry.UnreliableImplementation.TypeOfExceptionToFailWith;
|
|
|
import org.apache.hadoop.io.retry.UnreliableInterface.UnreliableException;
|
|
@@ -35,22 +36,41 @@ public class TestFailoverProxy {
|
|
|
private Object impl1;
|
|
|
private Object impl2;
|
|
|
|
|
|
+ private boolean latchEnabled = false;
|
|
|
+ private CountDownLatch getProxyLatch;
|
|
|
+ private int failoversOccurred = 0;
|
|
|
+
|
|
|
public FlipFlopProxyProvider(Class<?> iface, Object activeImpl,
|
|
|
- Object standbyImpl) {
|
|
|
+ Object standbyImpl, int getProxyCountDown) {
|
|
|
this.iface = iface;
|
|
|
this.impl1 = activeImpl;
|
|
|
this.impl2 = standbyImpl;
|
|
|
currentlyActive = impl1;
|
|
|
+ getProxyLatch = new CountDownLatch(getProxyCountDown);
|
|
|
+ }
|
|
|
+
|
|
|
+ public FlipFlopProxyProvider(Class<?> iface, Object activeImpl,
|
|
|
+ Object standbyImpl) {
|
|
|
+ this(iface, activeImpl, standbyImpl, 0);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public Object getProxy() {
|
|
|
+ if (latchEnabled) {
|
|
|
+ getProxyLatch.countDown();
|
|
|
+ try {
|
|
|
+ getProxyLatch.await();
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ throw new RuntimeException(e);
|
|
|
+ }
|
|
|
+ }
|
|
|
return currentlyActive;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public void performFailover(Object currentProxy) {
|
|
|
+ public synchronized void performFailover(Object currentProxy) {
|
|
|
currentlyActive = impl1 == currentProxy ? impl2 : impl1;
|
|
|
+ failoversOccurred++;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -63,6 +83,13 @@ public class TestFailoverProxy {
|
|
|
// Nothing to do.
|
|
|
}
|
|
|
|
|
|
+ public void setLatchEnabled(boolean latchEnabled) {
|
|
|
+ this.latchEnabled = latchEnabled;
|
|
|
+ }
|
|
|
+
|
|
|
+ public int getFailoversOccurred() {
|
|
|
+ return failoversOccurred;
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
public static class FailOverOnceOnAnyExceptionPolicy implements RetryPolicy {
|
|
@@ -186,4 +213,55 @@ public class TestFailoverProxy {
|
|
|
// IOException and this method is idempotent.
|
|
|
assertEquals("impl2", unreliable.succeedsOnceThenFailsReturningStringIdempotent());
|
|
|
}
|
|
|
-}
|
|
|
+
|
|
|
+ private static class ConcurrentMethodThread extends Thread {
|
|
|
+
|
|
|
+ private UnreliableInterface unreliable;
|
|
|
+ public String result;
|
|
|
+
|
|
|
+ public ConcurrentMethodThread(UnreliableInterface unreliable) {
|
|
|
+ this.unreliable = unreliable;
|
|
|
+ }
|
|
|
+
|
|
|
+ public void run() {
|
|
|
+ try {
|
|
|
+ result = unreliable.failsIfIdentifierDoesntMatch("impl2");
|
|
|
+ } catch (Exception e) {
|
|
|
+ throw new RuntimeException(e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test that concurrent failed method invocations only result in a single
|
|
|
+ * failover.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testConcurrentMethodFailures() throws InterruptedException {
|
|
|
+ FlipFlopProxyProvider proxyProvider = new FlipFlopProxyProvider(
|
|
|
+ UnreliableInterface.class,
|
|
|
+ new UnreliableImplementation("impl1",
|
|
|
+ TypeOfExceptionToFailWith.STANDBY_EXCEPTION),
|
|
|
+ new UnreliableImplementation("impl2",
|
|
|
+ TypeOfExceptionToFailWith.STANDBY_EXCEPTION),
|
|
|
+ 2);
|
|
|
+
|
|
|
+ final UnreliableInterface unreliable = (UnreliableInterface)RetryProxy
|
|
|
+ .create(UnreliableInterface.class, proxyProvider,
|
|
|
+ RetryPolicies.failoverOnNetworkException(10));
|
|
|
+
|
|
|
+ ConcurrentMethodThread t1 = new ConcurrentMethodThread(unreliable);
|
|
|
+ ConcurrentMethodThread t2 = new ConcurrentMethodThread(unreliable);
|
|
|
+
|
|
|
+ // Getting a proxy will now wait on a latch.
|
|
|
+ proxyProvider.setLatchEnabled(true);
|
|
|
+
|
|
|
+ t1.start();
|
|
|
+ t2.start();
|
|
|
+ t1.join();
|
|
|
+ t2.join();
|
|
|
+ assertEquals("impl2", t1.result);
|
|
|
+ assertEquals("impl2", t2.result);
|
|
|
+ assertEquals(1, proxyProvider.getFailoversOccurred());
|
|
|
+ }
|
|
|
+}
|