Return-Path: Delivered-To: apmail-lucene-hadoop-commits-archive@locus.apache.org Received: (qmail 84261 invoked from network); 25 May 2007 21:36:44 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.2) by minotaur.apache.org with SMTP; 25 May 2007 21:36:44 -0000 Received: (qmail 41176 invoked by uid 500); 25 May 2007 21:36:49 -0000 Delivered-To: apmail-lucene-hadoop-commits-archive@lucene.apache.org Received: (qmail 41081 invoked by uid 500); 25 May 2007 21:36:49 -0000 Mailing-List: contact hadoop-commits-help@lucene.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hadoop-dev@lucene.apache.org Delivered-To: mailing list hadoop-commits@lucene.apache.org Received: (qmail 41071 invoked by uid 99); 25 May 2007 21:36:49 -0000 Received: from herse.apache.org (HELO herse.apache.org) (140.211.11.133) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 25 May 2007 14:36:49 -0700 X-ASF-Spam-Status: No, hits=-98.6 required=10.0 tests=ALL_TRUSTED,INFO_TLD,NO_REAL_NAME X-Spam-Check-By: apache.org Received: from [140.211.11.3] (HELO eris.apache.org) (140.211.11.3) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 25 May 2007 14:36:43 -0700 Received: by eris.apache.org (Postfix, from userid 65534) id 7535F1A981D; Fri, 25 May 2007 14:36:23 -0700 (PDT) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r541785 - in /lucene/hadoop/branches/branch-0.13: ./ src/java/org/apache/hadoop/dfs/ src/java/org/apache/hadoop/io/retry/ src/test/org/apache/hadoop/io/retry/ Date: Fri, 25 May 2007 21:36:23 -0000 To: hadoop-commits@lucene.apache.org From: tomwhite@apache.org X-Mailer: svnmailer-1.1.0 Message-Id: <20070525213623.7535F1A981D@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: tomwhite Date: Fri May 25 14:36:22 2007 New Revision: 541785 URL: http://svn.apache.org/viewvc?view=rev&rev=541785 Log: Merge -r 541782:541783 from trunk to 0.13 branch. Fixes: HADOOP-1411. Modified: lucene/hadoop/branches/branch-0.13/CHANGES.txt lucene/hadoop/branches/branch-0.13/src/java/org/apache/hadoop/dfs/DFSClient.java lucene/hadoop/branches/branch-0.13/src/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java lucene/hadoop/branches/branch-0.13/src/java/org/apache/hadoop/io/retry/RetryPolicies.java lucene/hadoop/branches/branch-0.13/src/test/org/apache/hadoop/io/retry/TestRetryProxy.java lucene/hadoop/branches/branch-0.13/src/test/org/apache/hadoop/io/retry/UnreliableImplementation.java lucene/hadoop/branches/branch-0.13/src/test/org/apache/hadoop/io/retry/UnreliableInterface.java Modified: lucene/hadoop/branches/branch-0.13/CHANGES.txt URL: http://svn.apache.org/viewvc/lucene/hadoop/branches/branch-0.13/CHANGES.txt?view=diff&rev=541785&r1=541784&r2=541785 ============================================================================== --- lucene/hadoop/branches/branch-0.13/CHANGES.txt (original) +++ lucene/hadoop/branches/branch-0.13/CHANGES.txt Fri May 25 14:36:22 2007 @@ -426,6 +426,10 @@ to indicate the default directory, per HADOOP-1386. (Hairong Kuang via cutting) +128. HADOOP-1411. Make task retry framework handle + AlreadyBeingCreatedException when wrapped as a RemoteException. + (Hairong Kuang via tomwhite) + Release 0.12.3 - 2007-04-06 Modified: lucene/hadoop/branches/branch-0.13/src/java/org/apache/hadoop/dfs/DFSClient.java URL: http://svn.apache.org/viewvc/lucene/hadoop/branches/branch-0.13/src/java/org/apache/hadoop/dfs/DFSClient.java?view=diff&rev=541785&r1=541784&r2=541785 ============================================================================== --- lucene/hadoop/branches/branch-0.13/src/java/org/apache/hadoop/dfs/DFSClient.java (original) +++ lucene/hadoop/branches/branch-0.13/src/java/org/apache/hadoop/dfs/DFSClient.java Fri May 25 14:36:22 2007 @@ -106,11 +106,16 @@ RetryPolicy createPolicy = RetryPolicies.retryUpToMaximumCountWithFixedSleep( 5, LEASE_SOFTLIMIT_PERIOD, TimeUnit.MILLISECONDS); + Map,RetryPolicy> remoteExceptionToPolicyMap = + new HashMap, RetryPolicy>(); + remoteExceptionToPolicyMap.put(AlreadyBeingCreatedException.class, createPolicy); + Map,RetryPolicy> exceptionToPolicyMap = new HashMap, RetryPolicy>(); + exceptionToPolicyMap.put(RemoteException.class, + RetryPolicies.retryByRemoteException( + RetryPolicies.TRY_ONCE_THEN_FAIL, remoteExceptionToPolicyMap)); exceptionToPolicyMap.put(SocketTimeoutException.class, timeoutPolicy); - exceptionToPolicyMap.put(AlreadyBeingCreatedException.class, createPolicy); - RetryPolicy methodPolicy = RetryPolicies.retryByException( RetryPolicies.TRY_ONCE_THEN_FAIL, exceptionToPolicyMap); Map methodNameToPolicyMap = new HashMap(); Modified: lucene/hadoop/branches/branch-0.13/src/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java URL: http://svn.apache.org/viewvc/lucene/hadoop/branches/branch-0.13/src/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java?view=diff&rev=541785&r1=541784&r2=541785 ============================================================================== --- lucene/hadoop/branches/branch-0.13/src/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java (original) +++ lucene/hadoop/branches/branch-0.13/src/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java Fri May 25 14:36:22 2007 @@ -59,7 +59,7 @@ return invokeMethod(method, args); } catch (Exception e) { if (!policy.shouldRetry(e, retries++)) { - LOG.warn("Exception while invoking " + method.getName() + LOG.info("Exception while invoking " + method.getName() + " of " + implementation.getClass() + ". Not retrying." + StringUtils.stringifyException(e)); if (!method.getReturnType().equals(Void.TYPE)) { @@ -67,7 +67,7 @@ } return null; } - LOG.info("Exception while invoking " + method.getName() + LOG.debug("Exception while invoking " + method.getName() + " of " + implementation.getClass() + ". Retrying." + StringUtils.stringifyException(e)); } Modified: lucene/hadoop/branches/branch-0.13/src/java/org/apache/hadoop/io/retry/RetryPolicies.java URL: http://svn.apache.org/viewvc/lucene/hadoop/branches/branch-0.13/src/java/org/apache/hadoop/io/retry/RetryPolicies.java?view=diff&rev=541785&r1=541784&r2=541785 ============================================================================== --- lucene/hadoop/branches/branch-0.13/src/java/org/apache/hadoop/io/retry/RetryPolicies.java (original) +++ lucene/hadoop/branches/branch-0.13/src/java/org/apache/hadoop/io/retry/RetryPolicies.java Fri May 25 14:36:22 2007 @@ -17,10 +17,15 @@ */ package org.apache.hadoop.io.retry; +import java.util.HashMap; import java.util.Map; import java.util.Random; +import java.util.Set; +import java.util.Map.Entry; import java.util.concurrent.TimeUnit; +import org.apache.hadoop.ipc.RemoteException; + /** *

* A collection of useful implementations of {@link RetryPolicy}. @@ -83,10 +88,19 @@ return new RetryUpToMaximumCountWithProportionalSleep(maxRetries, sleepTime, timeUnit); } + /** + *

+ * Keep trying a limited number of times, waiting a growing amount of time between attempts, + * and then fail by re-throwing the exception. + * The time between attempts is sleepTime mutliplied by a random + * number in the range of [0, 2 to the number of retries) + *

+ */ public static final RetryPolicy exponentialBackoffRetry( int maxRetries, long sleepTime, TimeUnit timeUnit) { return new ExponentialBackoffRetry(maxRetries, sleepTime, timeUnit); } + /** *

* Set a default policy with some explicit handlers for specific exceptions. @@ -97,6 +111,18 @@ return new ExceptionDependentRetry(defaultPolicy, exceptionToPolicyMap); } + /** + *

+ * A retry policy for RemoteException + * Set a default policy with some explicit handlers for specific exceptions. + *

+ */ + public static final RetryPolicy retryByRemoteException( + RetryPolicy defaultPolicy, + Map, RetryPolicy> exceptionToPolicyMap) { + return new RemoteExceptionDependentRetry(defaultPolicy, exceptionToPolicyMap); + } + static class TryOnceThenFail implements RetryPolicy { public boolean shouldRetry(Exception e, int retries) throws Exception { throw e; @@ -187,6 +213,35 @@ return policy.shouldRetry(e, retries); } + } + + static class RemoteExceptionDependentRetry implements RetryPolicy { + + RetryPolicy defaultPolicy; + Map exceptionNameToPolicyMap; + + public RemoteExceptionDependentRetry(RetryPolicy defaultPolicy, + Map, + RetryPolicy> exceptionToPolicyMap) { + this.defaultPolicy = defaultPolicy; + this.exceptionNameToPolicyMap = new HashMap(); + for (Entry, RetryPolicy> e : + exceptionToPolicyMap.entrySet()) { + exceptionNameToPolicyMap.put(e.getKey().getName(), e.getValue()); + } + } + + public boolean shouldRetry(Exception e, int retries) throws Exception { + RetryPolicy policy = null; + if (e instanceof RemoteException) { + policy = exceptionNameToPolicyMap.get( + ((RemoteException) e).getClassName()); + } + if (policy == null) { + policy = defaultPolicy; + } + return policy.shouldRetry(e, retries); + } } static class ExponentialBackoffRetry extends RetryLimited { Modified: lucene/hadoop/branches/branch-0.13/src/test/org/apache/hadoop/io/retry/TestRetryProxy.java URL: http://svn.apache.org/viewvc/lucene/hadoop/branches/branch-0.13/src/test/org/apache/hadoop/io/retry/TestRetryProxy.java?view=diff&rev=541785&r1=541784&r2=541785 ============================================================================== --- lucene/hadoop/branches/branch-0.13/src/test/org/apache/hadoop/io/retry/TestRetryProxy.java (original) +++ lucene/hadoop/branches/branch-0.13/src/test/org/apache/hadoop/io/retry/TestRetryProxy.java Fri May 25 14:36:22 2007 @@ -4,6 +4,7 @@ import static org.apache.hadoop.io.retry.RetryPolicies.TRY_ONCE_DONT_FAIL; import static org.apache.hadoop.io.retry.RetryPolicies.TRY_ONCE_THEN_FAIL; import static org.apache.hadoop.io.retry.RetryPolicies.retryByException; +import static org.apache.hadoop.io.retry.RetryPolicies.retryByRemoteException; import static org.apache.hadoop.io.retry.RetryPolicies.retryUpToMaximumCountWithFixedSleep; import static org.apache.hadoop.io.retry.RetryPolicies.retryUpToMaximumCountWithProportionalSleep; import static org.apache.hadoop.io.retry.RetryPolicies.retryUpToMaximumTimeWithFixedSleep; @@ -17,6 +18,7 @@ import org.apache.hadoop.io.retry.UnreliableInterface.FatalException; import org.apache.hadoop.io.retry.UnreliableInterface.UnreliableException; +import org.apache.hadoop.ipc.RemoteException; public class TestRetryProxy extends TestCase { @@ -125,11 +127,26 @@ retryByException(RETRY_FOREVER, exceptionToPolicyMap)); unreliable.failsOnceThenSucceeds(); try { - unreliable.alwaysfailsWithFatalException(); + unreliable.alwaysFailsWithFatalException(); fail("Should fail"); } catch (FatalException e) { // expected } } + + public void testRetryByRemoteException() throws UnreliableException { + Map, RetryPolicy> exceptionToPolicyMap = + Collections., RetryPolicy>singletonMap(FatalException.class, TRY_ONCE_THEN_FAIL); + + UnreliableInterface unreliable = (UnreliableInterface) + RetryProxy.create(UnreliableInterface.class, unreliableImpl, + retryByRemoteException(RETRY_FOREVER, exceptionToPolicyMap)); + try { + unreliable.alwaysFailsWithRemoteFatalException(); + fail("Should fail"); + } catch (RemoteException e) { + // expected + } + } } Modified: lucene/hadoop/branches/branch-0.13/src/test/org/apache/hadoop/io/retry/UnreliableImplementation.java URL: http://svn.apache.org/viewvc/lucene/hadoop/branches/branch-0.13/src/test/org/apache/hadoop/io/retry/UnreliableImplementation.java?view=diff&rev=541785&r1=541784&r2=541785 ============================================================================== --- lucene/hadoop/branches/branch-0.13/src/test/org/apache/hadoop/io/retry/UnreliableImplementation.java (original) +++ lucene/hadoop/branches/branch-0.13/src/test/org/apache/hadoop/io/retry/UnreliableImplementation.java Fri May 25 14:36:22 2007 @@ -1,5 +1,7 @@ package org.apache.hadoop.io.retry; +import org.apache.hadoop.ipc.RemoteException; + public class UnreliableImplementation implements UnreliableInterface { private int failsOnceInvocationCount, @@ -10,8 +12,12 @@ // do nothing } - public void alwaysfailsWithFatalException() throws FatalException { + public void alwaysFailsWithFatalException() throws FatalException { throw new FatalException(); + } + + public void alwaysFailsWithRemoteFatalException() throws RemoteException { + throw new RemoteException(FatalException.class.getName(), "Oops"); } public void failsOnceThenSucceeds() throws UnreliableException { Modified: lucene/hadoop/branches/branch-0.13/src/test/org/apache/hadoop/io/retry/UnreliableInterface.java URL: http://svn.apache.org/viewvc/lucene/hadoop/branches/branch-0.13/src/test/org/apache/hadoop/io/retry/UnreliableInterface.java?view=diff&rev=541785&r1=541784&r2=541785 ============================================================================== --- lucene/hadoop/branches/branch-0.13/src/test/org/apache/hadoop/io/retry/UnreliableInterface.java (original) +++ lucene/hadoop/branches/branch-0.13/src/test/org/apache/hadoop/io/retry/UnreliableInterface.java Fri May 25 14:36:22 2007 @@ -1,5 +1,7 @@ package org.apache.hadoop.io.retry; +import org.apache.hadoop.ipc.RemoteException; + public interface UnreliableInterface { public static class UnreliableException extends Exception { @@ -12,7 +14,8 @@ void alwaysSucceeds() throws UnreliableException; - void alwaysfailsWithFatalException() throws FatalException; + void alwaysFailsWithFatalException() throws FatalException; + void alwaysFailsWithRemoteFatalException() throws RemoteException; void failsOnceThenSucceeds() throws UnreliableException; boolean failsOnceThenSucceedsWithReturnValue() throws UnreliableException;