Repository: hbase
Updated Branches:
  refs/heads/branch-1 e7795007b -> f9873622b


HBASE-12785 Use FutureTask to timeout the attempt to get the lock for hbck


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

Branch: refs/heads/branch-1
Commit: f9873622b86e7d8a912dfa15065e9bf96e10bacf
Parents: e779500
Author: tedyu <yuzhih...@gmail.com>
Authored: Mon Jan 5 14:10:48 2015 -0800
Committer: tedyu <yuzhih...@gmail.com>
Committed: Mon Jan 5 14:10:48 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/util/HBaseFsck.java | 69 +++++++++++++-------
 1 file changed, 46 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f9873622/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 4664a6b..aad8956 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
@@ -45,8 +45,12 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
+import java.util.concurrent.FutureTask;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -323,6 +327,30 @@ public class HBaseFsck extends Configured implements 
Closeable {
     this.executor = exec;
   }
   
+  private class FileLockCallable implements Callable<FSDataOutputStream> {
+    @Override
+    public FSDataOutputStream call() throws IOException {
+      try {
+        FileSystem fs = FSUtils.getCurrentFileSystem(getConf());
+        FsPermission defaultPerms = FSUtils.getFilePermissions(fs, getConf(),
+            HConstants.DATA_FILE_UMASK_KEY);
+        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);
+        out.writeBytes(InetAddress.getLocalHost().toString());
+        out.flush();
+        return out;
+      } catch(RemoteException e) {
+        
if(AlreadyBeingCreatedException.class.getName().equals(e.getClassName())){
+          return null;
+        } else {
+          throw e;
+        }
+      }
+    }
+  }
+
   /**
    * This method maintains a lock using a file. If the creation fails we 
return null
    *
@@ -330,32 +358,27 @@ public class HBaseFsck extends Configured implements 
Closeable {
    * @throws IOException
    */
   private FSDataOutputStream checkAndMarkRunningHbck() throws IOException {
-    long start = EnvironmentEdgeManager.currentTime();
+    FileLockCallable callable = new FileLockCallable();
+    ExecutorService executor = Executors.newFixedThreadPool(1);
+    FutureTask<FSDataOutputStream> futureTask = new 
FutureTask<FSDataOutputStream>(callable);
+    executor.execute(futureTask);
+    final int timeoutInSeconds = 30;
+    FSDataOutputStream stream = null;
     try {
-      FileSystem fs = FSUtils.getCurrentFileSystem(getConf());
-      FsPermission defaultPerms = FSUtils.getFilePermissions(fs, getConf(),
-          HConstants.DATA_FILE_UMASK_KEY);
-      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);
-      out.writeBytes(InetAddress.getLocalHost().toString());
-      out.flush();
-      return out;
-    } catch(RemoteException e) {
-      
if(AlreadyBeingCreatedException.class.getName().equals(e.getClassName())){
-        return null;
-      } else {
-        throw e;
-      }
+      stream = futureTask.get(30, TimeUnit.SECONDS);
+    } catch (ExecutionException ee) {
+      LOG.warn("Encountered exception when opening lock file", ee);
+    } catch (InterruptedException ie) {
+      LOG.warn("Interrupted when opening lock file", ie);
+      Thread.currentThread().interrupt();
+    } catch (TimeoutException exception) {
+      // took too long to obtain lock
+      LOG.warn("Took more than " + timeoutInSeconds + " seconds in obtaining 
lock");
+      futureTask.cancel(true);
     } finally {
-      long duration = EnvironmentEdgeManager.currentTime() - start;
-      if (duration > 30000) {
-        LOG.warn("Took " + duration + " milliseconds to obtain lock");
-        // took too long to obtain lock
-        return null;
-      }
+      executor.shutdownNow();
     }
+    return stream;
   }
 
   private void unlockHbck() {

Reply via email to