HADOOP-7888. TestFailoverProxy fails intermittently on trunk. Contributed by Jason Lowe.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1211728 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
58361d3f34
commit
d9690b0922
@ -120,6 +120,9 @@ Trunk (unreleased changes)
|
||||
HADOOP-7887. KerberosAuthenticatorHandler is not setting
|
||||
KerberosName name rules from configuration. (tucu)
|
||||
|
||||
HADOOP-7888. TestFailoverProxy fails intermittently on trunk. (Jason Lowe
|
||||
via atm)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
HADOOP-7761. Improve the performance of raw comparisons. (todd)
|
||||
|
@ -103,15 +103,12 @@ public Object invoke(Object proxy, Method method, Object[] args)
|
||||
if (invocationAttemptFailoverCount == proxyProviderFailoverCount) {
|
||||
proxyProvider.performFailover(currentProxy);
|
||||
proxyProviderFailoverCount++;
|
||||
currentProxy = proxyProvider.getProxy();
|
||||
} else {
|
||||
LOG.warn("A failover has occurred since the start of this method"
|
||||
+ " invocation attempt.");
|
||||
}
|
||||
}
|
||||
// The call to getProxy() could technically only be made in the event
|
||||
// performFailover() is called, but it needs to be out here for the
|
||||
// purpose of testing.
|
||||
currentProxy = proxyProvider.getProxy();
|
||||
invocationFailoverCount++;
|
||||
}
|
||||
if(LOG.isDebugEnabled()) {
|
||||
|
@ -36,34 +36,18 @@ public static class FlipFlopProxyProvider implements FailoverProxyProvider {
|
||||
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, int getProxyCountDown) {
|
||||
Object standbyImpl) {
|
||||
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;
|
||||
}
|
||||
|
||||
@ -83,10 +67,6 @@ public void close() throws IOException {
|
||||
// Nothing to do.
|
||||
}
|
||||
|
||||
public void setLatchEnabled(boolean latchEnabled) {
|
||||
this.latchEnabled = latchEnabled;
|
||||
}
|
||||
|
||||
public int getFailoversOccurred() {
|
||||
return failoversOccurred;
|
||||
}
|
||||
@ -214,6 +194,32 @@ public void testFailoverOnNetworkExceptionIdempotentOperation()
|
||||
assertEquals("impl2", unreliable.succeedsOnceThenFailsReturningStringIdempotent());
|
||||
}
|
||||
|
||||
private static class SynchronizedUnreliableImplementation extends UnreliableImplementation {
|
||||
|
||||
private CountDownLatch methodLatch;
|
||||
|
||||
public SynchronizedUnreliableImplementation(String identifier,
|
||||
TypeOfExceptionToFailWith exceptionToFailWith, int threadCount) {
|
||||
super(identifier, exceptionToFailWith);
|
||||
|
||||
methodLatch = new CountDownLatch(threadCount);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String failsIfIdentifierDoesntMatch(String identifier)
|
||||
throws UnreliableException, StandbyException, IOException {
|
||||
// Wait until all threads are trying to invoke this method
|
||||
methodLatch.countDown();
|
||||
try {
|
||||
methodLatch.await();
|
||||
} catch (InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
return super.failsIfIdentifierDoesntMatch(identifier);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private static class ConcurrentMethodThread extends Thread {
|
||||
|
||||
private UnreliableInterface unreliable;
|
||||
@ -240,11 +246,11 @@ public void run() {
|
||||
public void testConcurrentMethodFailures() throws InterruptedException {
|
||||
FlipFlopProxyProvider proxyProvider = new FlipFlopProxyProvider(
|
||||
UnreliableInterface.class,
|
||||
new UnreliableImplementation("impl1",
|
||||
TypeOfExceptionToFailWith.STANDBY_EXCEPTION),
|
||||
new SynchronizedUnreliableImplementation("impl1",
|
||||
TypeOfExceptionToFailWith.STANDBY_EXCEPTION,
|
||||
2),
|
||||
new UnreliableImplementation("impl2",
|
||||
TypeOfExceptionToFailWith.STANDBY_EXCEPTION),
|
||||
2);
|
||||
TypeOfExceptionToFailWith.STANDBY_EXCEPTION));
|
||||
|
||||
final UnreliableInterface unreliable = (UnreliableInterface)RetryProxy
|
||||
.create(UnreliableInterface.class, proxyProvider,
|
||||
@ -253,9 +259,6 @@ public void testConcurrentMethodFailures() throws InterruptedException {
|
||||
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();
|
||||
|
Loading…
Reference in New Issue
Block a user