Pārlūkot izejas kodu

HADOOP-10585. Retry polices ignore interrupted exceptions (Daryn Sharp via jeagles)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1594267 13f79535-47bb-0310-9956-ffa450edef68
Jonathan Turner Eagles 11 gadi atpakaļ
vecāks
revīzija
335f61a72f

+ 3 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -465,6 +465,9 @@ Release 2.5.0 - UNRELEASED
     because groups stored in Set and ArrayList are compared. 
     (Mit Desai via kihwal)
 
+    HADOOP-10585. Retry polices ignore interrupted exceptions (Daryn Sharp via
+    jeagles)
+
 Release 2.4.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java

@@ -150,7 +150,7 @@ public class RetryInvocationHandler<T> implements RpcInvocationHandler {
           }
           
           if (action.delayMillis > 0) {
-            ThreadUtil.sleepAtLeastIgnoreInterrupts(action.delayMillis);
+            Thread.sleep(action.delayMillis);
           }
           
           if (action.action == RetryAction.RetryDecision.FAILOVER_AND_RETRY) {

+ 54 - 5
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestRetryProxy.java

@@ -26,27 +26,37 @@ import static org.apache.hadoop.io.retry.RetryPolicies.retryUpToMaximumCountWith
 import static org.apache.hadoop.io.retry.RetryPolicies.retryUpToMaximumCountWithProportionalSleep;
 import static org.apache.hadoop.io.retry.RetryPolicies.retryUpToMaximumTimeWithFixedSleep;
 import static org.apache.hadoop.io.retry.RetryPolicies.exponentialBackoffRetry;
+import static org.junit.Assert.*;
 
 import java.util.Collections;
 import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
-
-import junit.framework.TestCase;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.hadoop.io.retry.UnreliableInterface.FatalException;
 import org.apache.hadoop.io.retry.UnreliableInterface.UnreliableException;
 import org.apache.hadoop.ipc.ProtocolTranslator;
 import org.apache.hadoop.ipc.RemoteException;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.lang.reflect.UndeclaredThrowableException;
 
-public class TestRetryProxy extends TestCase {
+public class TestRetryProxy {
   
   private UnreliableImplementation unreliableImpl;
   
-  @Override
-  protected void setUp() throws Exception {
+  @Before
+  public void setUp() throws Exception {
     unreliableImpl = new UnreliableImplementation();
   }
 
+  @Test
   public void testTryOnceThenFail() throws UnreliableException {
     UnreliableInterface unreliable = (UnreliableInterface)
       RetryProxy.create(UnreliableInterface.class, unreliableImpl, TRY_ONCE_THEN_FAIL);
@@ -62,6 +72,7 @@ public class TestRetryProxy extends TestCase {
   /**
    * Test for {@link RetryInvocationHandler#isRpcInvocation(Object)}
    */
+  @Test
   public void testRpcInvocation() throws Exception {
     // For a proxy method should return true
     final UnreliableInterface unreliable = (UnreliableInterface)
@@ -91,6 +102,7 @@ public class TestRetryProxy extends TestCase {
     assertFalse(RetryInvocationHandler.isRpcInvocation(new Object()));
   }
   
+  @Test
   public void testRetryForever() throws UnreliableException {
     UnreliableInterface unreliable = (UnreliableInterface)
       RetryProxy.create(UnreliableInterface.class, unreliableImpl, RETRY_FOREVER);
@@ -99,6 +111,7 @@ public class TestRetryProxy extends TestCase {
     unreliable.failsTenTimesThenSucceeds();
   }
   
+  @Test
   public void testRetryUpToMaximumCountWithFixedSleep() throws UnreliableException {
     UnreliableInterface unreliable = (UnreliableInterface)
       RetryProxy.create(UnreliableInterface.class, unreliableImpl,
@@ -113,6 +126,7 @@ public class TestRetryProxy extends TestCase {
     }
   }
   
+  @Test
   public void testRetryUpToMaximumTimeWithFixedSleep() throws UnreliableException {
     UnreliableInterface unreliable = (UnreliableInterface)
       RetryProxy.create(UnreliableInterface.class, unreliableImpl,
@@ -127,6 +141,7 @@ public class TestRetryProxy extends TestCase {
     }
   }
   
+  @Test
   public void testRetryUpToMaximumCountWithProportionalSleep() throws UnreliableException {
     UnreliableInterface unreliable = (UnreliableInterface)
       RetryProxy.create(UnreliableInterface.class, unreliableImpl,
@@ -141,6 +156,7 @@ public class TestRetryProxy extends TestCase {
     }
   }
   
+  @Test
   public void testExponentialRetry() throws UnreliableException {
     UnreliableInterface unreliable = (UnreliableInterface)
       RetryProxy.create(UnreliableInterface.class, unreliableImpl,
@@ -155,6 +171,7 @@ public class TestRetryProxy extends TestCase {
     }
   }
   
+  @Test
   public void testRetryByException() throws UnreliableException {
     Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap =
       Collections.<Class<? extends Exception>, RetryPolicy>singletonMap(FatalException.class, TRY_ONCE_THEN_FAIL);
@@ -171,6 +188,7 @@ public class TestRetryProxy extends TestCase {
     }
   }
   
+  @Test
   public void testRetryByRemoteException() {
     Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap =
       Collections.<Class<? extends Exception>, RetryPolicy>singletonMap(FatalException.class, TRY_ONCE_THEN_FAIL);
@@ -186,4 +204,35 @@ public class TestRetryProxy extends TestCase {
     }
   }  
   
+  @Test
+  public void testRetryInterruptible() throws Throwable {
+    final UnreliableInterface unreliable = (UnreliableInterface)
+        RetryProxy.create(UnreliableInterface.class, unreliableImpl,
+            retryUpToMaximumTimeWithFixedSleep(10, 10, TimeUnit.SECONDS));
+    
+    final CountDownLatch latch = new CountDownLatch(1);
+    final AtomicReference<Thread> futureThread = new AtomicReference<Thread>();
+    ExecutorService exec = Executors.newSingleThreadExecutor();
+    Future<Throwable> future = exec.submit(new Callable<Throwable>(){
+      @Override
+      public Throwable call() throws Exception {
+        futureThread.set(Thread.currentThread());
+        latch.countDown();
+        try {
+          unreliable.alwaysFailsWithFatalException();
+        } catch (UndeclaredThrowableException ute) {
+          return ute.getCause();
+        }
+        return null;
+      }
+    });
+    latch.await();
+    Thread.sleep(1000); // time to fail and sleep
+    assertTrue(futureThread.get().isAlive());
+    futureThread.get().interrupt();
+    Throwable e = future.get(1, TimeUnit.SECONDS); // should return immediately 
+    assertNotNull(e);
+    assertEquals(InterruptedException.class, e.getClass());
+    assertEquals("sleep interrupted", e.getMessage());
+  }
 }