hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ecl...@apache.org
Subject hbase git commit: HBASE-13128 Make HBCK's lock file retry creation and deletion
Date Thu, 05 Mar 2015 23:29:24 GMT
Repository: hbase
Updated Branches:
  refs/heads/master be2ad98b1 -> d58cce169


HBASE-13128 Make HBCK's lock file retry creation and deletion

Signed-off-by: Elliott Clark <eclark@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/d58cce16
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/d58cce16
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/d58cce16

Branch: refs/heads/master
Commit: d58cce1694c22f8f895dceaaf2b2003f69d04ea4
Parents: be2ad98
Author: Victoria Dudin <vdudin@vdudin-mbp.local>
Authored: Fri Feb 27 19:49:08 2015 -0800
Committer: Elliott Clark <eclark@apache.org>
Committed: Thu Mar 5 15:29:14 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/util/HBaseFsck.java | 84 +++++++++++++++++---
 .../apache/hadoop/hbase/util/TestHBaseFsck.java | 38 ++++++++-
 2 files changed, 110 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d58cce16/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index a8b60cd..88cfdf7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.util;
 import java.io.Closeable;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.io.PrintWriter;
 import java.io.StringWriter;
 import java.net.InetAddress;
@@ -59,6 +60,7 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Multimap;
 import com.google.common.collect.TreeMultimap;
 import com.google.protobuf.ServiceException;
+
 import org.apache.commons.lang.RandomStringUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
@@ -197,7 +199,8 @@ public class HBaseFsck extends Configured implements Closeable {
   private static final int DEFAULT_MAX_MERGE = 5;
   private static final String TO_BE_LOADED = "to_be_loaded";
   private static final String HBCK_LOCK_FILE = "hbase-hbck.lock";
-
+  private static final int DEFAULT_MAX_LOCK_FILE_ATTEMPTS = 5;
+  private static final int DEFAULT_LOCK_FILE_ATTEMPT_SLEEP_INTERVAL = 200;
 
   /**********************
    * Internal resources
@@ -290,6 +293,8 @@ public class HBaseFsck extends Configured implements Closeable {
       new HashMap<TableName, Set<String>>();
   private Map<TableName, TableState> tableStates =
       new HashMap<TableName, TableState>();
+  private final RetryCounterFactory lockFileRetryCounterFactory;
+
 
   /**
    * Constructor
@@ -311,6 +316,10 @@ public class HBaseFsck extends Configured implements Closeable {
 
     int numThreads = conf.getInt("hbasefsck.numthreads", MAX_NUM_THREADS);
     executor = new ScheduledThreadPoolExecutor(numThreads, Threads.newDaemonThreadFactory("hbasefsck"));
+    lockFileRetryCounterFactory = new RetryCounterFactory(
+        getConf().getInt("hbase.hbck.lockfile.attempts", DEFAULT_MAX_LOCK_FILE_ATTEMPTS),

+        getConf().getInt("hbase.hbck.lockfile.attempt.sleep.interval",
+            DEFAULT_LOCK_FILE_ATTEMPT_SLEEP_INTERVAL));
   }
 
   /**
@@ -328,9 +337,17 @@ public class HBaseFsck extends Configured implements Closeable {
     super(conf);
     errors = getErrorReporter(getConf());
     this.executor = exec;
+    lockFileRetryCounterFactory = new RetryCounterFactory(
+        getConf().getInt("hbase.hbck.lockfile.attempts", DEFAULT_MAX_LOCK_FILE_ATTEMPTS),
+        getConf().getInt("hbase.hbck.lockfile.attempt.sleep.interval", DEFAULT_LOCK_FILE_ATTEMPT_SLEEP_INTERVAL));
   }
   
   private class FileLockCallable implements Callable<FSDataOutputStream> {
+    RetryCounter retryCounter;
+
+    public FileLockCallable(RetryCounter retryCounter) {
+      this.retryCounter = retryCounter;
+    }
     @Override
     public FSDataOutputStream call() throws IOException {
       try {
@@ -340,7 +357,7 @@ public class HBaseFsck extends Configured implements Closeable {
         Path tmpDir = new Path(FSUtils.getRootDir(getConf()), HConstants.HBASE_TEMP_DIRECTORY);
         fs.mkdirs(tmpDir);
         HBCK_LOCK_PATH = new Path(tmpDir, HBCK_LOCK_FILE);
-        final FSDataOutputStream out = FSUtils.create(fs, HBCK_LOCK_PATH, defaultPerms, false);
+        final FSDataOutputStream out = createFileWithRetries(fs, HBCK_LOCK_PATH, defaultPerms);
         out.writeBytes(InetAddress.getLocalHost().toString());
         out.flush();
         return out;
@@ -352,6 +369,34 @@ public class HBaseFsck extends Configured implements Closeable {
         }
       }
     }
+
+    private FSDataOutputStream createFileWithRetries(final FileSystem fs,
+        final Path hbckLockFilePath, final FsPermission defaultPerms)
+        throws IOException {
+
+      IOException exception = null;
+      do {
+        try {
+          return FSUtils.create(fs, hbckLockFilePath, defaultPerms, false);
+        } catch (IOException ioe) {
+          LOG.info("Failed to create lock file " + hbckLockFilePath.getName()
+              + ", try=" + (retryCounter.getAttemptTimes() + 1) + " of "
+              + retryCounter.getMaxAttempts());
+          LOG.debug("Failed to create lock file " + hbckLockFilePath.getName(), 
+              ioe);
+          try {
+            exception = ioe;
+            retryCounter.sleepUntilNextRetry();
+          } catch (InterruptedException ie) {
+            throw (InterruptedIOException) new InterruptedIOException(
+                "Can't create lock file " + hbckLockFilePath.getName())
+            .initCause(ie);
+          }
+        }
+      } while (retryCounter.shouldRetry());
+
+      throw exception;
+    }
   }
 
   /**
@@ -361,7 +406,8 @@ public class HBaseFsck extends Configured implements Closeable {
    * @throws IOException
    */
   private FSDataOutputStream checkAndMarkRunningHbck() throws IOException {
-    FileLockCallable callable = new FileLockCallable();
+    RetryCounter retryCounter = lockFileRetryCounterFactory.create();
+    FileLockCallable callable = new FileLockCallable(retryCounter);
     ExecutorService executor = Executors.newFixedThreadPool(1);
     FutureTask<FSDataOutputStream> futureTask = new FutureTask<FSDataOutputStream>(callable);
     executor.execute(futureTask);
@@ -385,14 +431,30 @@ public class HBaseFsck extends Configured implements Closeable {
   }
 
   private void unlockHbck() {
-    if(hbckLockCleanup.compareAndSet(true, false)){
-      IOUtils.closeStream(hbckOutFd);
-      try{
-        FSUtils.delete(FSUtils.getCurrentFileSystem(getConf()), HBCK_LOCK_PATH, true);
-      } catch(IOException ioe) {
-        LOG.warn("Failed to delete " + HBCK_LOCK_PATH);
-        LOG.debug(ioe);
-      }
+    if (hbckLockCleanup.compareAndSet(true, false)) {
+      RetryCounter retryCounter = lockFileRetryCounterFactory.create();
+      do {
+        try {
+          IOUtils.closeStream(hbckOutFd);
+          FSUtils.delete(FSUtils.getCurrentFileSystem(getConf()),
+              HBCK_LOCK_PATH, true);
+          return;
+        } catch (IOException ioe) {
+          LOG.info("Failed to delete " + HBCK_LOCK_PATH + ", try="
+              + (retryCounter.getAttemptTimes() + 1) + " of "
+              + retryCounter.getMaxAttempts());
+          LOG.debug("Failed to delete " + HBCK_LOCK_PATH, ioe);
+          try {
+            retryCounter.sleepUntilNextRetry();
+          } catch (InterruptedException ie) {
+            Thread.currentThread().interrupt();
+            LOG.warn("Interrupted while deleting lock file" +
+                HBCK_LOCK_PATH);
+            return;
+          }
+        }
+      } while (retryCounter.shouldRetry());
+
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/d58cce16/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
index 0d3a94e..d635ce0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
@@ -555,8 +555,10 @@ public class TestHBaseFsck {
       boolean fail = true;
       @Override
       public HBaseFsck call(){
+        Configuration c = new Configuration(conf);
+        c.setInt("hbase.hbck.lockfile.attempts", 1);
         try{
-          return doFsck(conf, false);
+          return doFsck(c, false);
         } catch(Exception e){
           if (e.getMessage().contains("Duplicate hbck")) {
             fail = false;
@@ -584,6 +586,40 @@ public class TestHBaseFsck {
       assert(h2.getRetCode() >= 0);
     }
   }
+  
+  /**
+   * This test makes sure that with 5 retries both parallel instances 
+   * of hbck will be completed successfully.
+   *
+   * @throws Exception
+   */
+  @Test (timeout=180000)
+  public void testParallelWithRetriesHbck() throws Exception {
+    final ExecutorService service;
+    final Future<HBaseFsck> hbck1,hbck2;
+
+    class RunHbck implements Callable<HBaseFsck>{
+
+      @Override
+      public HBaseFsck call() throws Exception {
+        return doFsck(conf, false);        
+      }
+    }
+    service = Executors.newFixedThreadPool(2);
+    hbck1 = service.submit(new RunHbck());
+    hbck2 = service.submit(new RunHbck());
+    service.shutdown();
+    //wait for 15 seconds, for both hbck calls finish
+    service.awaitTermination(15, TimeUnit.SECONDS);
+    HBaseFsck h1 = hbck1.get();
+    HBaseFsck h2 = hbck2.get();
+    // Both should be successful
+    assertNotNull(h1);
+    assertNotNull(h2);
+    assert(h1.getRetCode() >= 0);
+    assert(h2.getRetCode() >= 0);
+  
+  }
 
   /**
    * This create and fixes a bad table with regions that have a duplicate


Mime
View raw message