You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by wc...@apache.org on 2020/09/10 09:12:30 UTC

[hbase-filesystem] branch master updated: HBASE-24989 [HBOSS] Some code cleanup (#16)

This is an automated email from the ASF dual-hosted git repository.

wchevreuil pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase-filesystem.git


The following commit(s) were added to refs/heads/master by this push:
     new 8ff7832  HBASE-24989 [HBOSS] Some code cleanup (#16)
8ff7832 is described below

commit 8ff7832e3211bc83de1d0fa3de3ed75844e01b9d
Author: Wellington Ramos Chevreuil <wc...@apache.org>
AuthorDate: Thu Sep 10 10:12:23 2020 +0100

    HBASE-24989 [HBOSS] Some code cleanup (#16)
    
    Signed-off-by: Josh Elser <el...@apache.org>
    Signed-off-by: Jan Hentschel <ja...@ultratendency.com>
---
 .../hbase/oss/HBaseObjectStoreSemantics.java       |  1 -
 .../hadoop/hbase/oss/sync/TreeLockManager.java     | 92 +++++++++++++++-------
 .../hadoop/hbase/oss/sync/ZKTreeLockManager.java   |  2 +-
 .../org/apache/hadoop/hbase/oss/EmbeddedS3.java    |  7 +-
 .../hbase/oss/HBaseObjectStoreSemanticsTest.java   |  8 --
 .../apache/hadoop/hbase/oss/TestAtomicRename.java  |  6 --
 .../hadoop/hbase/oss/TestCreateNonRecursive.java   |  4 -
 .../hadoop/hbase/oss/contract/HBOSSContract.java   |  2 +-
 .../hbase/oss/contract/TestHBOSSContract.java      |  5 --
 .../oss/contract/TestHBOSSContractCreate.java      |  2 -
 .../oss/contract/TestHBOSSContractDistCp.java      |  1 -
 .../apache/hadoop/hbase/oss/sync/EmbeddedZK.java   |  2 -
 .../hbase/oss/sync/LocalTreeLockManager.java       | 18 ++---
 .../hadoop/hbase/oss/sync/NullTreeLockManager.java |  1 -
 .../hadoop/hbase/oss/sync/TestTreeLockManager.java |  2 -
 15 files changed, 76 insertions(+), 77 deletions(-)

diff --git a/hbase-oss/src/main/java/org/apache/hadoop/hbase/oss/HBaseObjectStoreSemantics.java b/hbase-oss/src/main/java/org/apache/hadoop/hbase/oss/HBaseObjectStoreSemantics.java
index a9b3d72..73b71e9 100644
--- a/hbase-oss/src/main/java/org/apache/hadoop/hbase/oss/HBaseObjectStoreSemantics.java
+++ b/hbase-oss/src/main/java/org/apache/hadoop/hbase/oss/HBaseObjectStoreSemantics.java
@@ -84,7 +84,6 @@ import org.slf4j.LoggerFactory;
  *     could be made atomic by getting a write lock on the parent of the first
  *     wildcard.
  *   </li>
- *   </li>
  *   <li>
  *     Symlinking is not supported, but not used by HBase at all and not
  *     supported by mainstream object-stores considered in this design.
diff --git a/hbase-oss/src/main/java/org/apache/hadoop/hbase/oss/sync/TreeLockManager.java b/hbase-oss/src/main/java/org/apache/hadoop/hbase/oss/sync/TreeLockManager.java
index 73be990..ea620ce 100644
--- a/hbase-oss/src/main/java/org/apache/hadoop/hbase/oss/sync/TreeLockManager.java
+++ b/hbase-oss/src/main/java/org/apache/hadoop/hbase/oss/sync/TreeLockManager.java
@@ -47,7 +47,7 @@ public abstract class TreeLockManager {
   private static final Logger LOG =
         LoggerFactory.getLogger(TreeLockManager.class);
 
-  public static enum Depth {
+  public enum Depth {
     DIRECTORY, RECURSIVE
   }
 
@@ -82,6 +82,9 @@ public abstract class TreeLockManager {
    * "filesystem" and a path inside it. Assumes a 1:1 mapping between hostnames
    * and filesystems, and assumes the URI scheme mapping is consistent
    * everywhere.
+   *
+   * @param path the path to be normalized.
+   * @return Path the normalized representation.
    */
   public Path norm(Path path) {
     URI uri = fs.makeQualified(path).toUri();
@@ -130,12 +133,17 @@ public abstract class TreeLockManager {
   /**
    * In addition to any implementation-specific setup, implementations must set
    * this.fs = fs in order for path normalization to work.
+   *
+   * @param fs the FileSystem instance to be initialized.
+   * @throws IOException at any possible IO failure.
    */
   public abstract void initialize(FileSystem fs) throws IOException;
 
   /**
    * Performs any shutdown necessary when a client is exiting. Should be
    * considered best-effort and for planned shut downs.
+   *
+   * @throws IOException at any possible IO failure.
    */
   public void close() throws IOException {
   }
@@ -144,6 +152,7 @@ public abstract class TreeLockManager {
    * Acquires a single exclusive (write) lock.
    *
    * @param p Path to lock
+   * @throws IOException at any possible IO failure.
    */
   protected abstract void writeLock(Path p) throws IOException;
 
@@ -151,6 +160,7 @@ public abstract class TreeLockManager {
    * Releases a single exclusive (write) lock.
    *
    * @param p Path to unlock
+   * @throws IOException at any possible IO failure.
    */
   protected abstract void writeUnlock(Path p) throws IOException;
 
@@ -158,6 +168,7 @@ public abstract class TreeLockManager {
    * Acquires a single non-exclusive (read) lock.
    *
    * @param p Path to lock
+   * @throws IOException at any possible IO failure.
    */
   protected abstract void readLock(Path p) throws IOException;
 
@@ -165,6 +176,7 @@ public abstract class TreeLockManager {
    * Releases a single non-exclusive (read) lock.
    *
    * @param p Path to unlock
+   * @throws IOException at any possible IO failure.
    */
   protected abstract void readUnlock(Path p) throws IOException;
 
@@ -174,6 +186,7 @@ public abstract class TreeLockManager {
    *
    * @param p Path to check
    * @return True if a lock is found, false otherwise
+   * @throws IOException at any possible IO failure.
    */
   protected abstract boolean writeLockAbove(Path p) throws IOException;
 
@@ -181,7 +194,10 @@ public abstract class TreeLockManager {
    * Checks for the presence of a write lock on all children of the path.
    *
    * @param p Path to check
+   * @param depth {@link Depth#DIRECTORY} to look for locks on this path only,
+   *              or {@link Depth#RECURSIVE} to go through the whole tree beneath this path.
    * @return True if a lock is found, false otherwise
+   * @throws IOException at any possible IO failure.
    */
   @VisibleForTesting
   public abstract boolean writeLockBelow(Path p, Depth depth) throws IOException;
@@ -191,7 +207,10 @@ public abstract class TreeLockManager {
    * path.
    *
    * @param p Path to check
+   * @param depth {@link Depth#DIRECTORY} to look for locks on this path only,
+   *              or {@link Depth#RECURSIVE} to go through the whole tree beneath this path.
    * @return True if a lock is found, false otherwise
+   * @throws IOException at any possible IO failure.
    */
   @VisibleForTesting
   public abstract boolean readLockBelow(Path p, Depth depth) throws IOException;
@@ -200,6 +219,7 @@ public abstract class TreeLockManager {
    * Recursively cleans up locks that won't be used again.
    *
    * @param p Parent path of all locks to delete
+   * @throws IOException at any possible IO failure.
    */
   protected abstract void recursiveDelete(Path p) throws IOException;
 
@@ -231,25 +251,28 @@ public abstract class TreeLockManager {
    * or write locks should be held by anyone on any child directory.
    *
    * @param path Path to lock
+   * @param depth {@link Depth#DIRECTORY} to look for locks on this path only,
+   *              or {@link Depth#RECURSIVE} to go through the whole tree beneath this path.
+   * @throws IOException at any possible IO failure.
    */
-  protected void treeWriteLock(Path p, Depth depth) throws IOException {
+  protected void treeWriteLock(Path path, Depth depth) throws IOException {
     int outerRetries = 0;
     do {
       int innerRetries = 0;
       do {
         // If there's already a write-lock above or below us in the tree, wait for it to leave
-        if (writeLockAbove(p) || writeLockBelow(p, depth)) {
-          LOG.warn("Blocked on some parent write lock, waiting: {}", p);
+        if (writeLockAbove(path) || writeLockBelow(path, depth)) {
+          LOG.warn("Blocked on some parent write lock, waiting: {}", path);
           continue;
         }
         break;
       } while (retryBackoff(innerRetries++));
       // Try obtain the write lock just for our node
-      writeLock(p);
+      writeLock(path);
       // If there's now a write-lock above or below us in the tree, release and retry
-      if (writeLockAbove(p) || writeLockBelow(p, depth)) {
-        LOG.warn("Blocked on some other write lock, retrying: {}", p);
-        writeUnlock(p);
+      if (writeLockAbove(path) || writeLockBelow(path, depth)) {
+        LOG.warn("Blocked on some other write lock, retrying: {}", path);
+        writeUnlock(path);
         continue;
       }
       break;
@@ -258,8 +281,8 @@ public abstract class TreeLockManager {
     // Once we know we're the only write-lock in our path, drain all read-locks below
     int drainReadLocksRetries = 0;
     do {
-      if (readLockBelow(p, depth)) {
-        LOG.warn("Blocked on some child read lock, writing: {}", p);
+      if (readLockBelow(path, depth)) {
+        LOG.warn("Blocked on some child read lock, writing: {}", path);
         continue;
       }
       break;
@@ -272,27 +295,28 @@ public abstract class TreeLockManager {
    * no write locks should be held by anyone on any parent directory.
    *
    * @param path Path to lock
+   * @throws IOException at any possible IO failure.
    */
-  protected void treeReadLock(Path p) throws IOException {
+  protected void treeReadLock(Path path) throws IOException {
     int outerRetries = 0;
     do {
       int innerRetries = 0;
       do {
         // If there's a write lock above us, wait
-        if (writeLockAbove(p)) {
+        if (writeLockAbove(path)) {
           LOG.warn("Blocked waiting for some parent write lock, waiting: {}",
-                p);
+                path);
           continue;
         }
         break;
       } while (retryBackoff(innerRetries++));
       // Try obtain the read-lock just for our node
-      readLock(p);
+      readLock(path);
       // If there's a write lock above us, release the lock and try again
-      if (writeLockAbove(p)) {
+      if (writeLockAbove(path)) {
         LOG.warn("Blocked waiting for some parent write lock, retrying: {}",
-              p);
-        readUnlock(p);
+              path);
+        readUnlock(path);
         continue;
       }
       break;
@@ -305,8 +329,9 @@ public abstract class TreeLockManager {
    * atomic because the underlying file may not be created until all data has
    * been written.
    *
-   * @param path Path of the create operation
+   * @param rawPath Path of the create operation
    * @return AutoLock to release this path
+   * @throws IOException at any possible IO failure.
    */
   public AutoLock lockWrite(Path rawPath) throws IOException {
     Path path = norm(rawPath);
@@ -327,8 +352,9 @@ public abstract class TreeLockManager {
    * and those of all children. The lock ensures this doesn't interfere with any
    * renames or other listing operations above this path.
    *
-   * @param path Path of the create operation
+   * @param rawPath Path of the create operation
    * @return AutoLock to release this path
+   * @throws IOException at any possible IO failure.
    */
   public AutoLock lockDelete(Path rawPath) throws IOException {
     Path path = norm(rawPath);
@@ -350,8 +376,11 @@ public abstract class TreeLockManager {
    * listings. Other listings should only need a read lock on the root and all
    * children, but that is not implemented.
    *
-   * @param path Root of the listing operation
+   * @param rawPath Root of the listing operation
+   * @param depth Depth.DIRECTORY to look for locks on this path only, or Depth.RECURSIVE to
+   *    *              go through the whole tree beneath this path.
    * @return AutoCloseable to release this path
+   * @throws IOException at any possible IO failure.
    */
   public AutoLock lockListing(Path rawPath, Depth depth) throws IOException {
     Path path = norm(rawPath);
@@ -369,8 +398,11 @@ public abstract class TreeLockManager {
    * Same considerations of lockListing, but locks an array of paths in order
    * and returns an AutoLock that encapsulates all of them.
    *
-   * @param paths
+   * @param rawPaths an array of paths to lock.
+   * @param depth Depth.DIRECTORY to look for locks on this path only, or Depth.RECURSIVE to
+   *    *              go through the whole tree beneath this path.
    * @return AutoCloseable that encapsulate all paths
+   * @throws IOException at any possible IO failure.
    */
   public AutoLock lockListings(Path[] rawPaths, Depth depth) throws IOException {
     Path[] paths = norm(rawPaths);
@@ -406,10 +438,10 @@ public abstract class TreeLockManager {
    * may also record the start of the rename in something like a write-ahead log
    * to recover in-progress renames in the event of a failure.
    *
-   * @param src Source of the rename
-   * @param dst Destination of the rename
+   * @param rawSrc Source of the rename
+   * @param rawDst Destination of the rename
    * @return AutoCloseable to release both paths
-   * @throws IOException
+   * @throws IOException at any possible IO failure.
    */
   public AutoLock lockRename(Path rawSrc, Path rawDst) throws IOException {
     Path src = norm(rawSrc);
@@ -439,9 +471,9 @@ public abstract class TreeLockManager {
    * that read or modify the file-system but that don't necessarily need
    * exclusive access if no other concurrent operations do.
    *
-   * @param path Path to lock
+   * @param rawPath Path to lock
    * @return AutoCloseable that will release the path
-   * @throws IOException
+   * @throws IOException at any possible IO failure.
    */
   public AutoLock lock(Path rawPath) throws IOException {
     Path path = norm(rawPath);
@@ -460,9 +492,9 @@ public abstract class TreeLockManager {
    * cases that read or modify the file-system but that don't necessarily need
    * exclusive access if no other concurrent operations do.
    *
-   * @param paths Path to lock
+   * @param rawPaths array of Path to lock
    * @return AutoCloseable that will release all the paths
-   * @throws IOException
+   * @throws IOException at any possible IO failure.
    */
   public AutoLock lock(Path[] rawPaths) throws IOException {
     return innerLock(norm(rawPaths));
@@ -474,9 +506,9 @@ public abstract class TreeLockManager {
    * convenience in the FileSystem implementation where there is a distinction.
    *
    * @param extraPath Extra path to lock
-   * @param paths Paths to lock
+   * @param rawPaths array of Paths to lock
    * @return AutoCloseable that will release all the paths
-   * @throws IOException
+   * @throws IOException at any possible IO failure.
    */
   public AutoLock lock(Path extraPath, Path[] rawPaths) throws IOException {
     return innerLock(norm(rawPaths, extraPath));
diff --git a/hbase-oss/src/main/java/org/apache/hadoop/hbase/oss/sync/ZKTreeLockManager.java b/hbase-oss/src/main/java/org/apache/hadoop/hbase/oss/sync/ZKTreeLockManager.java
index 70c5433..333d6de 100644
--- a/hbase-oss/src/main/java/org/apache/hadoop/hbase/oss/sync/ZKTreeLockManager.java
+++ b/hbase-oss/src/main/java/org/apache/hadoop/hbase/oss/sync/ZKTreeLockManager.java
@@ -52,7 +52,7 @@ import static java.util.Objects.requireNonNull;
  * HBOSS to re-use an Apache HBase cluster's ZooKeeper ensemble for file
  * system locking.
  *
- * Can be enabled in JUnit tests with -Pzk. If {@link Constants.ZK_CONN_STRING}
+ * Can be enabled in JUnit tests with -Pzk. If {@link org.apache.hadoop.hbase.oss.Constants#ZK_CONN_STRING}
  * isn't specified, an embedded ZooKeeper process will be spun up for tests.
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
diff --git a/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/EmbeddedS3.java b/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/EmbeddedS3.java
index de27d1b..f5411b7 100644
--- a/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/EmbeddedS3.java
+++ b/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/EmbeddedS3.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.oss;
 import com.amazonaws.AmazonServiceException;
 import com.amazonaws.SdkClientException;
 import com.amazonaws.auth.AWSCredentialsProvider;
-import com.amazonaws.auth.AWSStaticCredentialsProvider;
 import com.amazonaws.services.s3.AbstractAmazonS3;
 import com.amazonaws.services.s3.AmazonS3;
 import com.amazonaws.services.s3.model.Bucket;
@@ -74,7 +73,7 @@ public class EmbeddedS3 {
 
   private static final String BUCKET = "embedded";
 
-  public static void conditionalStart(Configuration conf) throws Exception {
+  public static void conditionalStart(Configuration conf) {
     if (StringUtils.isEmpty(conf.get(S3_METADATA_STORE_IMPL))) {
       conf.set(S3_METADATA_STORE_IMPL, LocalMetadataStore.class.getName());
     }
@@ -95,8 +94,12 @@ public class EmbeddedS3 {
    * instance. This is currently a private API in Hadoop, but is the same method
    * used by S3Guard's inconsistency-injection tests. The method signature
    * defined in the interface varies depending on the Hadoop version.
+   *
+   * Due to compatibility purposes for both hadoop 2 and 3 main versions,
+   * we are omitting "@override" annotation from overridden methods.
    */
   public static class EmbeddedS3ClientFactory implements S3ClientFactory {
+
     public AmazonS3 createS3Client(URI name) {
       AmazonS3 s3 = new EmbeddedAmazonS3();
       s3.createBucket(BUCKET);
diff --git a/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/HBaseObjectStoreSemanticsTest.java b/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/HBaseObjectStoreSemanticsTest.java
index 21299db..1c1c883 100644
--- a/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/HBaseObjectStoreSemanticsTest.java
+++ b/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/HBaseObjectStoreSemanticsTest.java
@@ -25,14 +25,10 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 import org.junit.After;
 import org.junit.Before;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public class HBaseObjectStoreSemanticsTest {
-  public static final Logger LOG =
-        LoggerFactory.getLogger(HBaseObjectStoreSemanticsTest.class);
 
   protected HBaseObjectStoreSemantics hboss = null;
   protected TreeLockManager sync = null;
@@ -45,10 +41,6 @@ public class HBaseObjectStoreSemanticsTest {
     return TestUtils.testPath(hboss, path);
   }
 
-  public Path testPath(Path path) {
-    return TestUtils.testPath(hboss, path);
-  }
-
   @Before
   public void setup() throws Exception {
     Configuration conf = new Configuration();
diff --git a/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/TestAtomicRename.java b/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/TestAtomicRename.java
index be95dbf..556bb0a 100644
--- a/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/TestAtomicRename.java
+++ b/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/TestAtomicRename.java
@@ -21,16 +21,10 @@ package org.apache.hadoop.hbase.oss;
 import java.io.IOException;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.oss.sync.TreeLockManager;
 import org.junit.Assert;
-import org.junit.Assume;
 import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 public class TestAtomicRename extends HBaseObjectStoreSemanticsTest {
-  public static final Logger LOG =
-        LoggerFactory.getLogger(TestAtomicRename.class);
 
   @Test
   public void testAtomicRename() throws Exception {
diff --git a/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/TestCreateNonRecursive.java b/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/TestCreateNonRecursive.java
index 9208e9f..6b5987a 100644
--- a/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/TestCreateNonRecursive.java
+++ b/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/TestCreateNonRecursive.java
@@ -29,12 +29,8 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.junit.Assert;
 import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 public class TestCreateNonRecursive extends HBaseObjectStoreSemanticsTest {
-  public static final Logger LOG =
-        LoggerFactory.getLogger(TestCreateNonRecursive.class);
 
   @Test
   public void testCreateNonRecursiveSerial() throws Exception {
diff --git a/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/contract/HBOSSContract.java b/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/contract/HBOSSContract.java
index 948b46d..1dd6a46 100644
--- a/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/contract/HBOSSContract.java
+++ b/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/contract/HBOSSContract.java
@@ -54,7 +54,7 @@ public class HBOSSContract extends AbstractFSContract {
    * Any initialisation logic can go here
    * @throws IOException IO problems
    */
-  public void init() throws IOException {
+  public void init() {
 
   }
 
diff --git a/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/contract/TestHBOSSContract.java b/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/contract/TestHBOSSContract.java
index 4ed4be0..26c5a96 100644
--- a/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/contract/TestHBOSSContract.java
+++ b/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/contract/TestHBOSSContract.java
@@ -30,14 +30,9 @@ import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 public class TestHBOSSContract extends FileSystemContractBaseTest {
 
-  protected static final Logger LOG =
-      LoggerFactory.getLogger(TestHBOSSContract.class);
-
   private Path basePath;
   private Configuration conf;
 
diff --git a/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/contract/TestHBOSSContractCreate.java b/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/contract/TestHBOSSContractCreate.java
index ec088fa..1f24ac4 100644
--- a/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/contract/TestHBOSSContractCreate.java
+++ b/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/contract/TestHBOSSContractCreate.java
@@ -24,9 +24,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.AbstractContractCreateTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.apache.hadoop.hbase.oss.EmbeddedS3;
 import org.apache.hadoop.hbase.oss.TestUtils;
-import org.apache.hadoop.hbase.oss.sync.TreeLockManager;
 import org.junit.Assume;
 import org.junit.Test;
 
diff --git a/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/contract/TestHBOSSContractDistCp.java b/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/contract/TestHBOSSContractDistCp.java
index 73afb07..7eed5cf 100644
--- a/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/contract/TestHBOSSContractDistCp.java
+++ b/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/contract/TestHBOSSContractDistCp.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.oss.contract;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
-import org.apache.hadoop.hbase.oss.TestUtils;
 import org.apache.hadoop.tools.contract.AbstractContractDistCpTest;
 
 public class TestHBOSSContractDistCp extends AbstractContractDistCpTest {
diff --git a/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/sync/EmbeddedZK.java b/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/sync/EmbeddedZK.java
index dd9e293..77f7d58 100644
--- a/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/sync/EmbeddedZK.java
+++ b/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/sync/EmbeddedZK.java
@@ -24,8 +24,6 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.oss.Constants;
-import org.apache.hadoop.hbase.oss.sync.TreeLockManager;
-import org.apache.hadoop.hbase.oss.sync.ZKTreeLockManager;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 
diff --git a/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/sync/LocalTreeLockManager.java b/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/sync/LocalTreeLockManager.java
index 83d4227..4d4a8e2 100644
--- a/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/sync/LocalTreeLockManager.java
+++ b/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/sync/LocalTreeLockManager.java
@@ -18,14 +18,10 @@
 
 package org.apache.hadoop.hbase.oss.sync;
 
-import java.io.IOException;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
@@ -49,13 +45,13 @@ public class LocalTreeLockManager extends TreeLockManager {
   }
 
   @Override
-  protected void writeLock(Path p) throws IOException {
+  protected void writeLock(Path p) {
     createLocksIfNeeded(p);
     index.get(p).lock.writeLock().lock();
   }
 
   @Override
-  protected void writeUnlock(Path p) throws IOException {
+  protected void writeUnlock(Path p) {
     try {
       LockNode node = index.get(p);
       // Node to unlock may already be gone after deletes
@@ -70,13 +66,13 @@ public class LocalTreeLockManager extends TreeLockManager {
   }
 
   @Override
-  protected void readLock(Path p) throws IOException {
+  protected void readLock(Path p) {
     createLocksIfNeeded(p);
     index.get(p).lock.readLock().lock();
   }
 
   @Override
-  protected void readUnlock(Path p) throws IOException {
+  protected void readUnlock(Path p) {
     try {
       index.get(p).lock.readLock().unlock();
     } catch(IllegalMonitorStateException e) {
@@ -87,7 +83,7 @@ public class LocalTreeLockManager extends TreeLockManager {
   }
 
   @Override
-  protected boolean writeLockAbove(Path p) throws IOException {
+  protected boolean writeLockAbove(Path p) {
     createLocksIfNeeded(p);
     while (!p.isRoot()) {
       p = p.getParent();
@@ -103,7 +99,7 @@ public class LocalTreeLockManager extends TreeLockManager {
 
   @Override
   @VisibleForTesting
-  public boolean writeLockBelow(Path p, Depth depth) throws IOException {
+  public boolean writeLockBelow(Path p, Depth depth) {
     createLocksIfNeeded(p);
     int maxLevel = (depth == Depth.DIRECTORY) ? 1 : Integer.MAX_VALUE;
     return writeLockBelow(p, 0, maxLevel);
@@ -111,7 +107,7 @@ public class LocalTreeLockManager extends TreeLockManager {
 
   @Override
   @VisibleForTesting
-  public boolean readLockBelow(Path p, Depth depth) throws IOException {
+  public boolean readLockBelow(Path p, Depth depth) {
     createLocksIfNeeded(p);
     int maxLevel = (depth == Depth.DIRECTORY) ? 1 : Integer.MAX_VALUE;
     return readLockBelow(p, 0, maxLevel);
diff --git a/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/sync/NullTreeLockManager.java b/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/sync/NullTreeLockManager.java
index c9f3fbb..1027988 100644
--- a/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/sync/NullTreeLockManager.java
+++ b/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/sync/NullTreeLockManager.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.hbase.oss.sync;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
diff --git a/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/sync/TestTreeLockManager.java b/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/sync/TestTreeLockManager.java
index bafc121..e9ef0ce 100644
--- a/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/sync/TestTreeLockManager.java
+++ b/hbase-oss/src/test/java/org/apache/hadoop/hbase/oss/sync/TestTreeLockManager.java
@@ -22,8 +22,6 @@ import java.io.IOException;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.hadoop.hbase.oss.HBaseObjectStoreSemanticsTest;
-import org.apache.hadoop.hbase.oss.sync.AutoLock;
-import org.apache.hadoop.hbase.oss.sync.NullTreeLockManager;
 import org.apache.hadoop.hbase.oss.sync.TreeLockManager.Depth;
 import org.apache.hadoop.fs.Path;
 import org.junit.Assert;