You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by su...@apache.org on 2012/03/08 00:38:57 UTC
svn commit: r1298222 - in
/hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common:
CHANGES.txt
src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java
src/test/java/org/apache/hadoop/io/retry/TestFailoverProxy.java
Author: suresh
Date: Wed Mar 7 23:38:56 2012
New Revision: 1298222
URL: http://svn.apache.org/viewvc?rev=1298222&view=rev
Log:
HADOOP-7888. Merge r1211728 from trunk to 0.23
Modified:
hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/CHANGES.txt
hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java
hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestFailoverProxy.java
Modified: hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/CHANGES.txt?rev=1298222&r1=1298221&r2=1298222&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/CHANGES.txt (original)
+++ hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/CHANGES.txt Wed Mar 7 23:38:56 2012
@@ -110,6 +110,9 @@ Release 0.23.3 - UNRELEASED
HADOOP-8119. Fix javac warnings in TestAuthenticationFilter in hadoop-auth.
(szetszwo)
+ HADOOP-7888. TestFailoverProxy fails intermittently on trunk. (Jason Lowe
+ via atm)
+
Release 0.23.2 - UNRELEASED
NEW FEATURES
Modified: hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java?rev=1298222&r1=1298221&r2=1298222&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java Wed Mar 7 23:38:56 2012
@@ -104,15 +104,12 @@ class RetryInvocationHandler implements
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()) {
Modified: hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestFailoverProxy.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestFailoverProxy.java?rev=1298222&r1=1298221&r2=1298222&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestFailoverProxy.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestFailoverProxy.java Wed Mar 7 23:38:56 2012
@@ -36,34 +36,18 @@ 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, 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 class TestFailoverProxy {
// Nothing to do.
}
- public void setLatchEnabled(boolean latchEnabled) {
- this.latchEnabled = latchEnabled;
- }
-
public int getFailoversOccurred() {
return failoversOccurred;
}
@@ -214,6 +194,32 @@ public class TestFailoverProxy {
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 class TestFailoverProxy {
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 class TestFailoverProxy {
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();