You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ec...@apache.org on 2013/08/16 23:40:47 UTC

svn commit: r1514898 - /hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Leases.java

Author: eclark
Date: Fri Aug 16 21:40:47 2013
New Revision: 1514898

URL: http://svn.apache.org/r1514898
Log:
HBASE-9250 Make sure lease time isn't ever negative.

Sometimes nextLease.getDelay can return a negative number.  Trying to
sleep for a negative time thows an uncaught exception bringing down the
whole regionserver.

Modified:
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Leases.java

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Leases.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Leases.java?rev=1514898&r1=1514897&r2=1514898&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Leases.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Leases.java Fri Aug 16 21:40:47 2013
@@ -56,6 +56,7 @@ import java.io.IOException;
 @InterfaceAudience.Private
 public class Leases extends HasThread {
   private static final Log LOG = LogFactory.getLog(Leases.class.getName());
+  public static final int MIN_WAIT_TIME = 100;
   private final Map<String, Lease> leases = new ConcurrentHashMap<String, Lease>();
 
   protected final int leaseCheckFrequency;
@@ -87,7 +88,10 @@ public class Leases extends HasThread {
         if (nextLease != null) {
           toWait = nextLease.getDelay(TimeUnit.MILLISECONDS);
         }
+
         toWait = Math.min(leaseCheckFrequency, toWait);
+        toWait = Math.max(MIN_WAIT_TIME, toWait);
+
         Thread.sleep(toWait);
       } catch (InterruptedException e) {
         continue;