You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2018/02/15 02:35:15 UTC

[1/8] hbase git commit: HBASE-19970 Remove unused functions from TableAuthManager.

Repository: hbase
Updated Branches:
  refs/heads/branch-1 e65004aee -> 28ebd29f0
  refs/heads/branch-1.2 0f3bf5489 -> ef847f841
  refs/heads/branch-1.3 0507413fe -> 9b1f379f2
  refs/heads/branch-1.4 7446b8eaf -> 9519ec2ea
  refs/heads/branch-2 9cbf936f9 -> 0743bda05


HBASE-19970 Remove unused functions from TableAuthManager.

Functions deleted: setTableUserPermissions, setTableGroupPermissions, setNamespaceUserPermissions,
setNamespaceGroupPermissions, writeTableToZooKeeper, writeNamespaceToZooKeeper


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

Branch: refs/heads/branch-1.4
Commit: f563b7cf0d3cc15f361f3bb7581db9faf26235a9
Parents: 7446b8e
Author: Apekshit Sharma <ap...@apache.org>
Authored: Mon Feb 12 16:16:38 2018 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Feb 14 14:56:24 2018 -0800

----------------------------------------------------------------------
 .../security/access/AccessControlLists.java     |   9 +-
 .../hbase/security/access/AccessController.java |   4 +-
 .../hbase/security/access/TableAuthManager.java |  75 --------
 .../security/access/TestTablePermissions.java   |   2 +-
 .../access/TestZKPermissionWatcher.java         | 179 +++++++++++++++++++
 .../access/TestZKPermissionsWatcher.java        | 178 ------------------
 6 files changed, 188 insertions(+), 259 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f563b7cf/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
index 57c0f7b..f508110 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
@@ -73,6 +73,10 @@ import org.apache.hadoop.io.Text;
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.ListMultimap;
 import com.google.common.collect.Lists;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.jasper.tagplugins.jstl.core.Remove;
 
 /**
  * Maintains lists of permission grants to users and groups to allow for
@@ -667,8 +671,7 @@ public class AccessControlLists {
    *
    * Writes a set of permission [user: table permission]
    */
-  public static byte[] writePermissionsAsBytes(ListMultimap<String, TablePermission> perms,
-      Configuration conf) {
+  public static byte[] writePermissionsAsBytes(ListMultimap<String, TablePermission> perms) {
     return ProtobufUtil.prependPBMagic(ProtobufUtil.toUserTablePermissions(perms).toByteArray());
   }
 
@@ -755,7 +758,7 @@ public class AccessControlLists {
          // Deserialize the table permissions from the KV
          // TODO: This can be improved. Don't build UsersAndPermissions just to unpack it again,
          // use the builder
-         AccessControlProtos.UsersAndPermissions.Builder builder = 
+         AccessControlProtos.UsersAndPermissions.Builder builder =
            AccessControlProtos.UsersAndPermissions.newBuilder();
          ProtobufUtil.mergeFrom(builder, tag.getBuffer(), tag.getTagOffset(), tag.getTagLength());
          ListMultimap<String,Permission> kvPerms =

http://git-wip-us.apache.org/repos/asf/hbase/blob/f563b7cf/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index fd0a704..c889a3e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -243,7 +243,7 @@ public class AccessController extends BaseMasterAndRegionObserver
       tables.entrySet()) {
       byte[] entry = t.getKey();
       ListMultimap<String,TablePermission> perms = t.getValue();
-      byte[] serialized = AccessControlLists.writePermissionsAsBytes(perms, conf);
+      byte[] serialized = AccessControlLists.writePermissionsAsBytes(perms);
       getAuthManager().getZKPermissionWatcher().writeToZookeeper(entry, serialized);
     }
     initialized = true;
@@ -275,7 +275,7 @@ public class AccessController extends BaseMasterAndRegionObserver
         try (Table t = regionEnv.getTable(AccessControlLists.ACL_TABLE_NAME)) {
           ListMultimap<String,TablePermission> perms =
               AccessControlLists.getPermissions(conf, entry, t);
-          byte[] serialized = AccessControlLists.writePermissionsAsBytes(perms, conf);
+          byte[] serialized = AccessControlLists.writePermissionsAsBytes(perms);
           zkw.writeToZookeeper(entry, serialized);
         }
       } catch (IOException ex) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/f563b7cf/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
index a12757d..0aabcb3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
@@ -659,81 +659,6 @@ public class TableAuthManager implements Closeable {
     tableCache.remove(table);
   }
 
-  /**
-   * Overwrites the existing permission set for a given user for a table, and
-   * triggers an update for zookeeper synchronization.
-   * @param username
-   * @param table
-   * @param perms
-   */
-  public void setTableUserPermissions(String username, TableName table,
-      List<TablePermission> perms) {
-    PermissionCache<TablePermission> tablePerms = getTablePermissions(table);
-    tablePerms.replaceUser(username, perms);
-    writeTableToZooKeeper(table, tablePerms);
-  }
-
-  /**
-   * Overwrites the existing permission set for a group and triggers an update
-   * for zookeeper synchronization.
-   * @param group
-   * @param table
-   * @param perms
-   */
-  public void setTableGroupPermissions(String group, TableName table,
-      List<TablePermission> perms) {
-    PermissionCache<TablePermission> tablePerms = getTablePermissions(table);
-    tablePerms.replaceGroup(group, perms);
-    writeTableToZooKeeper(table, tablePerms);
-  }
-
-  /**
-   * Overwrites the existing permission set for a given user for a table, and
-   * triggers an update for zookeeper synchronization.
-   * @param username
-   * @param namespace
-   * @param perms
-   */
-  public void setNamespaceUserPermissions(String username, String namespace,
-      List<TablePermission> perms) {
-    PermissionCache<TablePermission> tablePerms = getNamespacePermissions(namespace);
-    tablePerms.replaceUser(username, perms);
-    writeNamespaceToZooKeeper(namespace, tablePerms);
-  }
-
-  /**
-   * Overwrites the existing permission set for a group and triggers an update
-   * for zookeeper synchronization.
-   * @param group
-   * @param namespace
-   * @param perms
-   */
-  public void setNamespaceGroupPermissions(String group, String namespace,
-      List<TablePermission> perms) {
-    PermissionCache<TablePermission> tablePerms = getNamespacePermissions(namespace);
-    tablePerms.replaceGroup(group, perms);
-    writeNamespaceToZooKeeper(namespace, tablePerms);
-  }
-
-  public void writeTableToZooKeeper(TableName table,
-      PermissionCache<TablePermission> tablePerms) {
-    byte[] serialized = new byte[0];
-    if (tablePerms != null) {
-      serialized = AccessControlLists.writePermissionsAsBytes(tablePerms.getAllPermissions(), conf);
-    }
-    zkperms.writeToZookeeper(table.getName(), serialized);
-  }
-
-  public void writeNamespaceToZooKeeper(String namespace,
-      PermissionCache<TablePermission> tablePerms) {
-    byte[] serialized = new byte[0];
-    if (tablePerms != null) {
-      serialized = AccessControlLists.writePermissionsAsBytes(tablePerms.getAllPermissions(), conf);
-    }
-    zkperms.writeToZookeeper(Bytes.toBytes(AccessControlLists.toNamespaceEntry(namespace)),
-        serialized);
-  }
-
   public long getMTime() {
     return mtime.get();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f563b7cf/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java
index f8fad9f..26ca9eb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java
@@ -323,7 +323,7 @@ public class TestTablePermissions {
   public void testSerialization() throws Exception {
     Configuration conf = UTIL.getConfiguration();
     ListMultimap<String,TablePermission> permissions = createPermissions();
-    byte[] permsData = AccessControlLists.writePermissionsAsBytes(permissions, conf);
+    byte[] permsData = AccessControlLists.writePermissionsAsBytes(permissions);
 
     ListMultimap<String, TablePermission> copy =
         AccessControlLists.readPermissions(permsData, conf);

http://git-wip-us.apache.org/repos/asf/hbase/blob/f563b7cf/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionWatcher.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionWatcher.java
new file mode 100644
index 0000000..a80f184
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionWatcher.java
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.security.access;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.ListMultimap;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.Waiter.Predicate;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test the reading and writing of access permissions to and from zookeeper.
+ */
+@Category(LargeTests.class)
+public class TestZKPermissionsWatcher {
+  private static final Log LOG = LogFactory.getLog(TestZKPermissionsWatcher.class);
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+  private static TableAuthManager AUTH_A;
+  private static TableAuthManager AUTH_B;
+  private final static Abortable ABORTABLE = new Abortable() {
+    private final AtomicBoolean abort = new AtomicBoolean(false);
+
+    @Override
+    public void abort(String why, Throwable e) {
+      LOG.info(why, e);
+      abort.set(true);
+    }
+
+    @Override
+    public boolean isAborted() {
+      return abort.get();
+    }
+  };
+
+  private static TableName TEST_TABLE =
+      TableName.valueOf("perms_test");
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    // setup configuration
+    Configuration conf = UTIL.getConfiguration();
+    SecureTestUtil.enableSecurity(conf);
+
+    // start minicluster
+    UTIL.startMiniCluster();
+    AUTH_A = TableAuthManager.getOrCreate(new ZooKeeperWatcher(conf,
+      "TestZKPermissionsWatcher_1", ABORTABLE), conf);
+    AUTH_B = TableAuthManager.getOrCreate(new ZooKeeperWatcher(conf,
+      "TestZKPermissionsWatcher_2", ABORTABLE), conf);
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  private void setTableACL(
+      User user, TableAuthManager srcAuthManager, TableAuthManager destAuthManager,
+      TablePermission.Action... actions) throws Exception{
+    // update ACL: george RW
+    ListMultimap<String, TablePermission> perms = ArrayListMultimap.create();
+    perms.replaceValues(user.getShortName(),
+        Collections.singletonList(new TablePermission(TEST_TABLE, null, actions)));
+    byte[] serialized = AccessControlLists.writePermissionsAsBytes(perms);
+    final long mtime = destAuthManager.getMTime();
+    srcAuthManager.getZKPermissionWatcher().writeToZookeeper(TEST_TABLE.getName(), serialized);
+    // Wait for the update to propagate
+    UTIL.waitFor(10000, 100, new Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        return destAuthManager.getMTime() > mtime;
+      }
+    });
+    Thread.sleep(1000);
+  }
+
+  @Test
+  public void testPermissionsWatcher() throws Exception {
+    Configuration conf = UTIL.getConfiguration();
+    User george = User.createUserForTesting(conf, "george", new String[] { });
+    User hubert = User.createUserForTesting(conf, "hubert", new String[] { });
+
+    assertFalse(AUTH_A.authorizeUser(george, TEST_TABLE, null,
+      TablePermission.Action.READ));
+    assertFalse(AUTH_A.authorizeUser(george, TEST_TABLE, null,
+      TablePermission.Action.WRITE));
+    assertFalse(AUTH_A.authorizeUser(hubert, TEST_TABLE, null,
+      TablePermission.Action.READ));
+    assertFalse(AUTH_A.authorizeUser(hubert, TEST_TABLE, null,
+      TablePermission.Action.WRITE));
+
+    assertFalse(AUTH_B.authorizeUser(george, TEST_TABLE, null,
+      TablePermission.Action.READ));
+    assertFalse(AUTH_B.authorizeUser(george, TEST_TABLE, null,
+      TablePermission.Action.WRITE));
+    assertFalse(AUTH_B.authorizeUser(hubert, TEST_TABLE, null,
+      TablePermission.Action.READ));
+    assertFalse(AUTH_B.authorizeUser(hubert, TEST_TABLE, null,
+      TablePermission.Action.WRITE));
+
+    // update ACL: george, RW
+    setTableACL(george, AUTH_A, AUTH_B,
+        TablePermission.Action.READ, TablePermission.Action.WRITE);
+
+    // check it
+    assertTrue(AUTH_A.authorizeUser(george, TEST_TABLE, null,
+      TablePermission.Action.READ));
+    assertTrue(AUTH_A.authorizeUser(george, TEST_TABLE, null,
+      TablePermission.Action.WRITE));
+    assertTrue(AUTH_B.authorizeUser(george, TEST_TABLE, null,
+      TablePermission.Action.READ));
+    assertTrue(AUTH_B.authorizeUser(george, TEST_TABLE, null,
+      TablePermission.Action.WRITE));
+    assertFalse(AUTH_A.authorizeUser(hubert, TEST_TABLE, null,
+      TablePermission.Action.READ));
+    assertFalse(AUTH_A.authorizeUser(hubert, TEST_TABLE, null,
+      TablePermission.Action.WRITE));
+    assertFalse(AUTH_B.authorizeUser(hubert, TEST_TABLE, null,
+      TablePermission.Action.READ));
+    assertFalse(AUTH_B.authorizeUser(hubert, TEST_TABLE, null,
+      TablePermission.Action.WRITE));
+
+    // update ACL: hubert, Read
+    setTableACL(hubert, AUTH_B, AUTH_A, TablePermission.Action.READ);
+
+    // check it
+    assertTrue(AUTH_A.authorizeUser(george, TEST_TABLE, null,
+      TablePermission.Action.READ));
+    assertTrue(AUTH_A.authorizeUser(george, TEST_TABLE, null,
+      TablePermission.Action.WRITE));
+    assertTrue(AUTH_B.authorizeUser(george, TEST_TABLE, null,
+      TablePermission.Action.READ));
+    assertTrue(AUTH_B.authorizeUser(george, TEST_TABLE, null,
+      TablePermission.Action.WRITE));
+    assertTrue(AUTH_A.authorizeUser(hubert, TEST_TABLE, null,
+      TablePermission.Action.READ));
+    assertFalse(AUTH_A.authorizeUser(hubert, TEST_TABLE, null,
+      TablePermission.Action.WRITE));
+    assertTrue(AUTH_B.authorizeUser(hubert, TEST_TABLE, null,
+      TablePermission.Action.READ));
+    assertFalse(AUTH_B.authorizeUser(hubert, TEST_TABLE, null,
+      TablePermission.Action.WRITE));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/f563b7cf/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionsWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionsWatcher.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionsWatcher.java
deleted file mode 100644
index c99cbaa..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionsWatcher.java
+++ /dev/null
@@ -1,178 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.security.access;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.Waiter.Predicate;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-/**
- * Test the reading and writing of access permissions to and from zookeeper.
- */
-@Category(LargeTests.class)
-public class TestZKPermissionsWatcher {
-  private static final Log LOG = LogFactory.getLog(TestZKPermissionsWatcher.class);
-  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
-  private static TableAuthManager AUTH_A;
-  private static TableAuthManager AUTH_B;
-  private final static Abortable ABORTABLE = new Abortable() {
-    private final AtomicBoolean abort = new AtomicBoolean(false);
-
-    @Override
-    public void abort(String why, Throwable e) {
-      LOG.info(why, e);
-      abort.set(true);
-    }
-
-    @Override
-    public boolean isAborted() {
-      return abort.get();
-    }
-  };
-
-  private static TableName TEST_TABLE =
-      TableName.valueOf("perms_test");
-
-  @BeforeClass
-  public static void beforeClass() throws Exception {
-    // setup configuration
-    Configuration conf = UTIL.getConfiguration();
-    SecureTestUtil.enableSecurity(conf);
-
-    // start minicluster
-    UTIL.startMiniCluster();
-    AUTH_A = TableAuthManager.getOrCreate(new ZooKeeperWatcher(conf,
-      "TestZKPermissionsWatcher_1", ABORTABLE), conf);
-    AUTH_B = TableAuthManager.getOrCreate(new ZooKeeperWatcher(conf,
-      "TestZKPermissionsWatcher_2", ABORTABLE), conf);
-  }
-
-  @AfterClass
-  public static void afterClass() throws Exception {
-    UTIL.shutdownMiniCluster();
-  }
-
-  @Test
-  public void testPermissionsWatcher() throws Exception {
-    Configuration conf = UTIL.getConfiguration();
-    User george = User.createUserForTesting(conf, "george", new String[] { });
-    User hubert = User.createUserForTesting(conf, "hubert", new String[] { });
-    
-    assertFalse(AUTH_A.authorizeUser(george, TEST_TABLE, null,
-      TablePermission.Action.READ));
-    assertFalse(AUTH_A.authorizeUser(george, TEST_TABLE, null,
-      TablePermission.Action.WRITE));
-    assertFalse(AUTH_A.authorizeUser(hubert, TEST_TABLE, null,
-      TablePermission.Action.READ));
-    assertFalse(AUTH_A.authorizeUser(hubert, TEST_TABLE, null,
-      TablePermission.Action.WRITE));
-
-    assertFalse(AUTH_B.authorizeUser(george, TEST_TABLE, null,
-      TablePermission.Action.READ));
-    assertFalse(AUTH_B.authorizeUser(george, TEST_TABLE, null,
-      TablePermission.Action.WRITE));
-    assertFalse(AUTH_B.authorizeUser(hubert, TEST_TABLE, null,
-      TablePermission.Action.READ));
-    assertFalse(AUTH_B.authorizeUser(hubert, TEST_TABLE, null,
-      TablePermission.Action.WRITE));
-
-    // update ACL: george RW
-    List<TablePermission> acl = new ArrayList<TablePermission>();
-    acl.add(new TablePermission(TEST_TABLE, null, TablePermission.Action.READ,
-      TablePermission.Action.WRITE));
-    final long mtimeB = AUTH_B.getMTime();
-    AUTH_A.setTableUserPermissions(george.getShortName(), TEST_TABLE, acl);
-    // Wait for the update to propagate
-    UTIL.waitFor(10000, 100, new Predicate<Exception>() {
-      @Override
-      public boolean evaluate() throws Exception {
-        return AUTH_B.getMTime() > mtimeB;
-      }
-    });
-    Thread.sleep(1000);
-
-    // check it
-    assertTrue(AUTH_A.authorizeUser(george, TEST_TABLE, null,
-      TablePermission.Action.READ));
-    assertTrue(AUTH_A.authorizeUser(george, TEST_TABLE, null,
-      TablePermission.Action.WRITE));
-    assertTrue(AUTH_B.authorizeUser(george, TEST_TABLE, null,
-      TablePermission.Action.READ));
-    assertTrue(AUTH_B.authorizeUser(george, TEST_TABLE, null,
-      TablePermission.Action.WRITE));
-    assertFalse(AUTH_A.authorizeUser(hubert, TEST_TABLE, null,
-      TablePermission.Action.READ));
-    assertFalse(AUTH_A.authorizeUser(hubert, TEST_TABLE, null,
-      TablePermission.Action.WRITE));
-    assertFalse(AUTH_B.authorizeUser(hubert, TEST_TABLE, null,
-      TablePermission.Action.READ));
-    assertFalse(AUTH_B.authorizeUser(hubert, TEST_TABLE, null,
-      TablePermission.Action.WRITE));
-
-    // update ACL: hubert R
-    acl = new ArrayList<TablePermission>();
-    acl.add(new TablePermission(TEST_TABLE, null, TablePermission.Action.READ));
-    final long mtimeA = AUTH_A.getMTime();
-    AUTH_B.setTableUserPermissions("hubert", TEST_TABLE, acl);
-    // Wait for the update to propagate
-    UTIL.waitFor(10000, 100, new Predicate<Exception>() {
-      @Override
-      public boolean evaluate() throws Exception {
-        return AUTH_A.getMTime() > mtimeA;
-      }
-    });
-    Thread.sleep(1000);
-
-    // check it
-    assertTrue(AUTH_A.authorizeUser(george, TEST_TABLE, null,
-      TablePermission.Action.READ));
-    assertTrue(AUTH_A.authorizeUser(george, TEST_TABLE, null,
-      TablePermission.Action.WRITE));
-    assertTrue(AUTH_B.authorizeUser(george, TEST_TABLE, null,
-      TablePermission.Action.READ));
-    assertTrue(AUTH_B.authorizeUser(george, TEST_TABLE, null,
-      TablePermission.Action.WRITE));
-    assertTrue(AUTH_A.authorizeUser(hubert, TEST_TABLE, null,
-      TablePermission.Action.READ));
-    assertFalse(AUTH_A.authorizeUser(hubert, TEST_TABLE, null,
-      TablePermission.Action.WRITE));
-    assertTrue(AUTH_B.authorizeUser(hubert, TEST_TABLE, null,
-      TablePermission.Action.READ));
-    assertFalse(AUTH_B.authorizeUser(hubert, TEST_TABLE, null,
-      TablePermission.Action.WRITE));
-  }
-}


[3/8] hbase git commit: HBASE-18282 ReplicationLogCleaner can delete WALs not yet replicated in case of a KeeperException

Posted by ap...@apache.org.
HBASE-18282 ReplicationLogCleaner can delete WALs not yet replicated in case of a KeeperException

Signed-off-by: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/branch-2
Commit: 0743bda059d5edd6ca7bbb788c54970dc1212055
Parents: 9cbf936
Author: Ben Lau <be...@oath.com>
Authored: Tue Feb 13 17:13:50 2018 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Feb 14 17:23:09 2018 -0800

----------------------------------------------------------------------
 .../hbase/replication/ReplicationQueues.java    |  3 +-
 .../ReplicationQueuesClientZKImpl.java          |  5 ++
 .../replication/ReplicationQueuesZKImpl.java    | 12 +++-
 .../replication/ReplicationStateZKBase.java     |  8 ++-
 .../master/ReplicationLogCleaner.java           | 12 +++-
 .../regionserver/ReplicationSourceManager.java  | 17 ++++-
 .../hbase/master/cleaner/TestLogsCleaner.java   | 70 ++++++++++++++++----
 7 files changed, 107 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0743bda0/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
index 7f440b1..a2d21f7 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
@@ -119,8 +119,9 @@ public interface ReplicationQueues {
    * Get a list of all region servers that have outstanding replication queues. These servers could
    * be alive, dead or from a previous run of the cluster.
    * @return a list of server names
+   * @throws ReplicationException
    */
-  List<String> getListOfReplicators();
+  List<String> getListOfReplicators() throws ReplicationException;
 
   /**
    * Checks if the provided znode is the same as this region server's

http://git-wip-us.apache.org/repos/asf/hbase/blob/0743bda0/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
index e00a7a2..0eeba19 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
@@ -59,6 +59,11 @@ public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implem
       throw new ReplicationException("Internal error while initializing a queues client", e);
     }
   }
+  
+  @Override
+  public List<String> getListOfReplicators() throws KeeperException {
+    return super.getListOfReplicatorsZK();
+  }
 
   @Override
   public List<String> getLogsInQueue(String serverName, String queueId) throws KeeperException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/0743bda0/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
index 7551cb7..40bdeb8 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
@@ -100,7 +100,17 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
       }
     }
   }
-
+  
+  @Override
+  public List<String> getListOfReplicators() throws ReplicationException {
+    try {
+      return super.getListOfReplicatorsZK();
+    } catch (KeeperException e) {
+      LOG.warn("getListOfReplicators() from ZK failed", e);
+      throw new ReplicationException("getListOfReplicators() from ZK failed", e);
+    }
+  }
+  
   @Override
   public void removeQueue(String queueId) {
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/0743bda0/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
index 05bbc84..4e9479f 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
@@ -93,12 +93,18 @@ public abstract class ReplicationStateZKBase {
     this.hfileRefsZNode = ZNodePaths.joinZNode(replicationZNode, hfileRefsZNodeName);
   }
 
-  public List<String> getListOfReplicators() {
+  /**
+   * Subclasses that use ZK explicitly can just call this directly while classes
+   * that are trying to hide internal details of storage can wrap the KeeperException
+   * into a ReplicationException or something else.
+   */
+  protected List<String> getListOfReplicatorsZK() throws KeeperException {
     List<String> result = null;
     try {
       result = ZKUtil.listChildrenNoWatch(this.zookeeper, this.queuesZNode);
     } catch (KeeperException e) {
       this.abortable.abort("Failed to get list of replicators", e);
+      throw e;
     }
     return result;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0743bda0/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
index d2e6d68..5128d58 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
@@ -115,7 +115,15 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
       LOG.error("Error while configuring " + this.getClass().getName(), e);
     }
   }
-
+  
+  @VisibleForTesting
+  public void setConf(Configuration conf, ZKWatcher zk, 
+      ReplicationQueuesClient replicationQueuesClient) {
+    super.setConf(conf);
+    this.zkw = zk;
+    this.replicationQueues = replicationQueuesClient;
+  }
+  
   @Override
   public void stop(String why) {
     if (this.stopped) return;
@@ -131,7 +139,7 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
     return this.stopped;
   }
 
-  private static class WarnOnlyAbortable implements Abortable {
+  public static class WarnOnlyAbortable implements Abortable {
 
     @Override
     public void abort(String why, Throwable e) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/0743bda0/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
index c0c2333..8346824 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
@@ -792,7 +792,22 @@ public class ReplicationSourceManager implements ReplicationListener {
 
     @Override
     public void run() {
-      List<String> currentReplicators = replicationQueues.getListOfReplicators();
+      List<String> currentReplicators = null;
+      while (currentReplicators == null) {
+        try {
+          currentReplicators = replicationQueues.getListOfReplicators();
+        } catch (ReplicationException e1) {
+          LOG.warn("Failure in getListOfReplicators(), will retry later", e1);
+          try {
+            Thread.sleep(ThreadLocalRandom.current().nextInt(10000));
+          } catch (InterruptedException e2) {
+            LOG.warn("Interrupted while waiting for list of replicators to be available, "
+                + "will not adopt any abandoned queues", e2);
+            Thread.currentThread().interrupt();
+            break;
+          }
+        }
+      }
       if (currentReplicators == null || currentReplicators.isEmpty()) {
         return;
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0743bda0/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
index 9c577fe..210e5c9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.doAnswer;
 
 import java.io.IOException;
 import java.lang.reflect.Field;
@@ -30,6 +31,8 @@ import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Random;
+import java.util.concurrent.atomic.AtomicBoolean;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -51,6 +54,8 @@ import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationQueues;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
+import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
+import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesClientZKImpl;
 import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -59,13 +64,14 @@ import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.Stat;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -219,12 +225,10 @@ public class TestLogsCleaner {
     cleaner.getDeletableFiles(new LinkedList<>());
   }
 
-  /**
-   * ReplicationLogCleaner should be able to ride over ZooKeeper errors without aborting.
-   */
-  @Test
-  public void testZooKeeperAbort() throws Exception {
+  @Test(timeout=10000)
+  public void testZooKeeperAbortDuringGetListOfReplicators() throws Exception {
     Configuration conf = TEST_UTIL.getConfiguration();
+
     ReplicationLogCleaner cleaner = new ReplicationLogCleaner();
 
     List<FileStatus> dummyFiles = Lists.newArrayList(
@@ -232,20 +236,56 @@ public class TestLogsCleaner {
         new FileStatus(100, false, 3, 100, System.currentTimeMillis(), new Path("log2"))
     );
 
-    try (FaultyZooKeeperWatcher faultyZK = new FaultyZooKeeperWatcher(conf,
-        "testZooKeeperAbort-faulty", null)) {
+    FaultyZooKeeperWatcher faultyZK =
+        new FaultyZooKeeperWatcher(conf, "testZooKeeperAbort-faulty", null);
+    final AtomicBoolean getListOfReplicatorsFailed = new AtomicBoolean(false);
+
+    try {
       faultyZK.init();
-      cleaner.setConf(conf, faultyZK);
-      cleaner.preClean();
+      ReplicationQueuesClient replicationQueuesClient = spy(ReplicationFactory.getReplicationQueuesClient(
+        new ReplicationQueuesClientArguments(conf, new ReplicationLogCleaner.WarnOnlyAbortable(), faultyZK)));
+      doAnswer(new Answer<Object>() {
+        @Override
+        public Object answer(InvocationOnMock invocation) throws Throwable {
+          try {
+            return invocation.callRealMethod();
+          } catch (KeeperException.ConnectionLossException e) {
+            getListOfReplicatorsFailed.set(true);
+            throw e;
+          }
+        }
+      }).when(replicationQueuesClient).getListOfReplicators();
+      replicationQueuesClient.init();
+
+      cleaner.setConf(conf, faultyZK, replicationQueuesClient);
       // should keep all files due to a ConnectionLossException getting the queues znodes
+      cleaner.preClean();
       Iterable<FileStatus> toDelete = cleaner.getDeletableFiles(dummyFiles);
+
+      assertTrue(getListOfReplicatorsFailed.get());
       assertFalse(toDelete.iterator().hasNext());
       assertFalse(cleaner.isStopped());
+    } finally {
+      faultyZK.close();
     }
+  }
+
+  /**
+   * When zk is working both files should be returned
+   * @throws Exception
+   */
+  @Test(timeout=10000)
+  public void testZooKeeperNormal() throws Exception {
+    Configuration conf = TEST_UTIL.getConfiguration();
+    ReplicationLogCleaner cleaner = new ReplicationLogCleaner();
 
-    // when zk is working both files should be returned
-    cleaner = new ReplicationLogCleaner();
-    try (ZKWatcher zkw = new ZKWatcher(conf, "testZooKeeperAbort-normal", null)) {
+    List<FileStatus> dummyFiles = Lists.newArrayList(
+        new FileStatus(100, false, 3, 100, System.currentTimeMillis(), new Path("log1")),
+        new FileStatus(100, false, 3, 100, System.currentTimeMillis(), new Path("log2"))
+    );
+    
+    ZKWatcher zkw = new ZKWatcher(conf, "testZooKeeperAbort-normal", null);
+    try {
       cleaner.setConf(conf, zkw);
       cleaner.preClean();
       Iterable<FileStatus> filesToDelete = cleaner.getDeletableFiles(dummyFiles);
@@ -255,6 +295,8 @@ public class TestLogsCleaner {
       assertTrue(iter.hasNext());
       assertEquals(new Path("log2"), iter.next().getPath());
       assertFalse(iter.hasNext());
+    } finally {
+      zkw.close();
     }
   }
 
@@ -395,7 +437,7 @@ public class TestLogsCleaner {
     public void init() throws Exception {
       this.zk = spy(super.getRecoverableZooKeeper());
       doThrow(new KeeperException.ConnectionLossException())
-          .when(zk).getData("/hbase/replication/rs", null, new Stat());
+        .when(zk).getChildren("/hbase/replication/rs", null);
     }
 
     @Override


[4/8] hbase git commit: HBASE-18282 ReplicationLogCleaner can delete WALs not yet replicated in case of a KeeperException

Posted by ap...@apache.org.
HBASE-18282 ReplicationLogCleaner can delete WALs not yet replicated in case of a KeeperException

Signed-off-by: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/branch-1
Commit: 28ebd29f0fa2c581d740db5738dd6f69aad3e3ef
Parents: e65004a
Author: Ben Lau <be...@oath.com>
Authored: Wed Feb 14 11:36:04 2018 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Feb 14 17:23:23 2018 -0800

----------------------------------------------------------------------
 .../hbase/replication/ReplicationQueues.java    |  3 +-
 .../ReplicationQueuesClientZKImpl.java          |  5 ++
 .../replication/ReplicationQueuesZKImpl.java    | 10 ++++
 .../replication/ReplicationStateZKBase.java     |  8 ++-
 .../cleaner/ReplicationZKLockCleanerChore.java  |  4 +-
 .../master/ReplicationLogCleaner.java           | 10 +++-
 .../hbase/master/cleaner/TestLogsCleaner.java   | 54 ++++++++++++++++----
 7 files changed, 79 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/28ebd29f/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
index 2409111..ccc7172 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
@@ -126,8 +126,9 @@ public interface ReplicationQueues {
    * Get a list of all region servers that have outstanding replication queues. These servers could
    * be alive, dead or from a previous run of the cluster.
    * @return a list of server names
+   * @throws ReplicationException
    */
-  List<String> getListOfReplicators();
+  List<String> getListOfReplicators() throws ReplicationException;
 
   /**
    * Checks if the provided znode is the same as this region server's

http://git-wip-us.apache.org/repos/asf/hbase/blob/28ebd29f/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
index 9078e40..14b4334 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
@@ -49,6 +49,11 @@ public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implem
   }
 
   @Override
+  public List<String> getListOfReplicators() throws KeeperException {
+    return super.getListOfReplicatorsZK();
+  }
+
+  @Override
   public List<String> getLogsInQueue(String serverName, String queueId) throws KeeperException {
     String znode = ZKUtil.joinZNode(this.queuesZNode, serverName);
     znode = ZKUtil.joinZNode(znode, queueId);

http://git-wip-us.apache.org/repos/asf/hbase/blob/28ebd29f/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
index a1bd829..dda9adf 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
@@ -103,6 +103,16 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
   }
 
   @Override
+  public List<String> getListOfReplicators() throws ReplicationException {
+    try {
+      return super.getListOfReplicatorsZK();
+    } catch (KeeperException e) {
+      LOG.warn("getListOfReplicators() from ZK failed", e);
+      throw new ReplicationException("getListOfReplicators() from ZK failed", e);
+    }
+  }
+
+  @Override
   public void removeQueue(String queueId) {
     try {
       ZKUtil.deleteNodeRecursively(this.zookeeper, ZKUtil.joinZNode(this.myQueuesZnode, queueId));

http://git-wip-us.apache.org/repos/asf/hbase/blob/28ebd29f/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
index d0c3513..0713b99 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
@@ -90,12 +90,18 @@ public abstract class ReplicationStateZKBase {
     this.hfileRefsZNode = ZKUtil.joinZNode(replicationZNode, hfileRefsZNodeName);
   }
 
-  public List<String> getListOfReplicators() {
+  /**
+   * Subclasses that use ZK explicitly can just call this directly while classes
+   * that are trying to hide internal details of storage can wrap the KeeperException
+   * into a ReplicationException or something else.
+   */
+  protected List<String> getListOfReplicatorsZK() throws KeeperException {
     List<String> result = null;
     try {
       result = ZKUtil.listChildrenNoWatch(this.zookeeper, this.queuesZNode);
     } catch (KeeperException e) {
       this.abortable.abort("Failed to get list of replicators", e);
+      throw e;
     }
     return result;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/28ebd29f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKLockCleanerChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKLockCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKLockCleanerChore.java
index 3fa30bf..7c50719 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKLockCleanerChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKLockCleanerChore.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.ScheduledChore;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesZKImpl;
 import org.apache.hadoop.hbase.replication.ReplicationTracker;
@@ -108,7 +109,8 @@ public class ReplicationZKLockCleanerChore extends ScheduledChore {
       }
     } catch (KeeperException e) {
       LOG.warn("zk operation interrupted", e);
+    } catch (ReplicationException e2) {
+      LOG.warn("replication exception", e2);
     }
-
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/28ebd29f/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
index 7731240..42d66a5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
@@ -156,6 +156,14 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
     }
   }
 
+  @VisibleForTesting
+  public void setConf(Configuration conf, ZooKeeperWatcher zk, 
+      ReplicationQueuesClient replicationQueuesClient) {
+    super.setConf(conf);
+    this.zkw = zk;
+    this.replicationQueues = replicationQueuesClient;
+  }
+
   @Override
   public void stop(String why) {
     if (this.stopped) return;
@@ -171,7 +179,7 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
     return this.stopped;
   }
 
-  private static class WarnOnlyAbortable implements Abortable {
+  public static class WarnOnlyAbortable implements Abortable {
 
     @Override
     public void abort(String why, Throwable e) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/28ebd29f/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
index 8efa754..df5916c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.doAnswer;
 
 import java.io.IOException;
 import java.lang.reflect.Field;
@@ -29,6 +30,7 @@ import java.net.URLEncoder;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import com.google.common.collect.Lists;
 import org.apache.hadoop.conf.Configuration;
@@ -55,12 +57,13 @@ import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.Stat;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 @Category(MediumTests.class)
 public class TestLogsCleaner {
@@ -177,13 +180,10 @@ public class TestLogsCleaner {
     cleaner.getDeletableFiles(new LinkedList<FileStatus>());
   }
 
-  /**
-   * ReplicationLogCleaner should be able to ride over ZooKeeper errors without
-   * aborting.
-   */
-  @Test
-  public void testZooKeeperAbort() throws Exception {
+  @Test(timeout=10000)
+  public void testZooKeeperAbortDuringGetListOfReplicators() throws Exception {
     Configuration conf = TEST_UTIL.getConfiguration();
+
     ReplicationLogCleaner cleaner = new ReplicationLogCleaner();
 
     List<FileStatus> dummyFiles = Lists.newArrayList(
@@ -193,19 +193,51 @@ public class TestLogsCleaner {
 
     FaultyZooKeeperWatcher faultyZK =
         new FaultyZooKeeperWatcher(conf, "testZooKeeperAbort-faulty", null);
+    final AtomicBoolean getListOfReplicatorsFailed = new AtomicBoolean(false);
+
     try {
       faultyZK.init();
-      cleaner.setConf(conf, faultyZK);
+      ReplicationQueuesClient replicationQueuesClient = spy(ReplicationFactory.getReplicationQueuesClient(
+        faultyZK, conf, new ReplicationLogCleaner.WarnOnlyAbortable()));
+      doAnswer(new Answer<Object>() {
+        @Override
+        public Object answer(InvocationOnMock invocation) throws Throwable {
+          try {
+            return invocation.callRealMethod();
+          } catch (KeeperException.ConnectionLossException e) {
+            getListOfReplicatorsFailed.set(true);
+            throw e;
+          }
+        }
+      }).when(replicationQueuesClient).getListOfReplicators();
+      replicationQueuesClient.init();
+
+      cleaner.setConf(conf, faultyZK, replicationQueuesClient);
       // should keep all files due to a ConnectionLossException getting the queues znodes
       Iterable<FileStatus> toDelete = cleaner.getDeletableFiles(dummyFiles);
+
+      assertTrue(getListOfReplicatorsFailed.get());
       assertFalse(toDelete.iterator().hasNext());
       assertFalse(cleaner.isStopped());
     } finally {
       faultyZK.close();
     }
+  }
+
+  /**
+   * When zk is working both files should be returned
+   * @throws Exception
+   */
+  @Test(timeout=10000)
+  public void testZooKeeperNormal() throws Exception {
+    Configuration conf = TEST_UTIL.getConfiguration();
+    ReplicationLogCleaner cleaner = new ReplicationLogCleaner();
 
-    // when zk is working both files should be returned
-    cleaner = new ReplicationLogCleaner();
+    List<FileStatus> dummyFiles = Lists.newArrayList(
+        new FileStatus(100, false, 3, 100, System.currentTimeMillis(), new Path("log1")),
+        new FileStatus(100, false, 3, 100, System.currentTimeMillis(), new Path("log2"))
+    );
+    
     ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "testZooKeeperAbort-normal", null);
     try {
       cleaner.setConf(conf, zkw);
@@ -291,7 +323,7 @@ public class TestLogsCleaner {
     public void init() throws Exception {
       this.zk = spy(super.getRecoverableZooKeeper());
       doThrow(new KeeperException.ConnectionLossException())
-          .when(zk).getData("/hbase/replication/rs", null, new Stat());
+        .when(zk).getChildren("/hbase/replication/rs", null);
     }
 
     public RecoverableZooKeeper getRecoverableZooKeeper() {


[2/8] hbase git commit: HBASE-19970 (addendum for 1.x only) Remove unused functions from TableAuthManager.

Posted by ap...@apache.org.
HBASE-19970 (addendum for 1.x only) Remove unused functions from TableAuthManager.


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

Branch: refs/heads/branch-1.4
Commit: 3c5871b4ee8d6c72a6229e88057ac54e4a352d93
Parents: f563b7c
Author: Apekshit Sharma <ap...@apache.org>
Authored: Mon Feb 12 16:30:48 2018 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Feb 14 14:56:36 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/security/access/TestZKPermissionWatcher.java  | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3c5871b4/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionWatcher.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionWatcher.java
index a80f184..0961cab 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionWatcher.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestZKPermissionWatcher.java
@@ -47,8 +47,8 @@ import org.junit.experimental.categories.Category;
  * Test the reading and writing of access permissions to and from zookeeper.
  */
 @Category(LargeTests.class)
-public class TestZKPermissionsWatcher {
-  private static final Log LOG = LogFactory.getLog(TestZKPermissionsWatcher.class);
+public class TestZKPermissionWatcher {
+  private static final Log LOG = LogFactory.getLog(TestZKPermissionWatcher.class);
   private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
   private static TableAuthManager AUTH_A;
   private static TableAuthManager AUTH_B;
@@ -90,7 +90,7 @@ public class TestZKPermissionsWatcher {
   }
 
   private void setTableACL(
-      User user, TableAuthManager srcAuthManager, TableAuthManager destAuthManager,
+      User user, TableAuthManager srcAuthManager, final TableAuthManager destAuthManager,
       TablePermission.Action... actions) throws Exception{
     // update ACL: george RW
     ListMultimap<String, TablePermission> perms = ArrayListMultimap.create();


[5/8] hbase git commit: HBASE-18282 ReplicationLogCleaner can delete WALs not yet replicated in case of a KeeperException

Posted by ap...@apache.org.
HBASE-18282 ReplicationLogCleaner can delete WALs not yet replicated in case of a KeeperException

Signed-off-by: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/branch-1.3
Commit: 9b1f379f2e0f9ccb3bdaacb0aba7973ab05e35a0
Parents: 0507413
Author: Ben Lau <be...@oath.com>
Authored: Wed Feb 14 11:36:04 2018 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Feb 14 17:23:33 2018 -0800

----------------------------------------------------------------------
 .../hbase/replication/ReplicationQueues.java    |  3 +-
 .../ReplicationQueuesClientZKImpl.java          |  5 ++
 .../replication/ReplicationQueuesZKImpl.java    | 10 ++++
 .../replication/ReplicationStateZKBase.java     |  8 ++-
 .../cleaner/ReplicationZKLockCleanerChore.java  |  4 +-
 .../master/ReplicationLogCleaner.java           | 10 +++-
 .../hbase/master/cleaner/TestLogsCleaner.java   | 54 ++++++++++++++++----
 7 files changed, 79 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9b1f379f/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
index 1b1c770..f86d78b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
@@ -125,8 +125,9 @@ public interface ReplicationQueues {
    * Get a list of all region servers that have outstanding replication queues. These servers could
    * be alive, dead or from a previous run of the cluster.
    * @return a list of server names
+   * @throws ReplicationException
    */
-  List<String> getListOfReplicators();
+  List<String> getListOfReplicators() throws ReplicationException;
 
   /**
    * Checks if the provided znode is the same as this region server's

http://git-wip-us.apache.org/repos/asf/hbase/blob/9b1f379f/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
index cc407e3..413d613 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
@@ -47,6 +47,11 @@ public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implem
   }
 
   @Override
+  public List<String> getListOfReplicators() throws KeeperException {
+    return super.getListOfReplicatorsZK();
+  }
+
+  @Override
   public List<String> getLogsInQueue(String serverName, String queueId) throws KeeperException {
     String znode = ZKUtil.joinZNode(this.queuesZNode, serverName);
     znode = ZKUtil.joinZNode(znode, queueId);

http://git-wip-us.apache.org/repos/asf/hbase/blob/9b1f379f/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
index 559ab41..97a1762 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
@@ -98,6 +98,16 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
   }
 
   @Override
+  public List<String> getListOfReplicators() throws ReplicationException {
+    try {
+      return super.getListOfReplicatorsZK();
+    } catch (KeeperException e) {
+      LOG.warn("getListOfReplicators() from ZK failed", e);
+      throw new ReplicationException("getListOfReplicators() from ZK failed", e);
+    }
+  }
+
+  @Override
   public void removeQueue(String queueId) {
     try {
       ZKUtil.deleteNodeRecursively(this.zookeeper, ZKUtil.joinZNode(this.myQueuesZnode, queueId));

http://git-wip-us.apache.org/repos/asf/hbase/blob/9b1f379f/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
index a1dc1c8..77b2a66 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
@@ -84,12 +84,18 @@ public abstract class ReplicationStateZKBase {
     this.hfileRefsZNode = ZKUtil.joinZNode(replicationZNode, hfileRefsZNodeName);
   }
 
-  public List<String> getListOfReplicators() {
+  /**
+   * Subclasses that use ZK explicitly can just call this directly while classes
+   * that are trying to hide internal details of storage can wrap the KeeperException
+   * into a ReplicationException or something else.
+   */
+  protected List<String> getListOfReplicatorsZK() throws KeeperException {
     List<String> result = null;
     try {
       result = ZKUtil.listChildrenNoWatch(this.zookeeper, this.queuesZNode);
     } catch (KeeperException e) {
       this.abortable.abort("Failed to get list of replicators", e);
+      throw e;
     }
     return result;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/9b1f379f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKLockCleanerChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKLockCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKLockCleanerChore.java
index 3fa30bf..7c50719 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKLockCleanerChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKLockCleanerChore.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.ScheduledChore;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesZKImpl;
 import org.apache.hadoop.hbase.replication.ReplicationTracker;
@@ -108,7 +109,8 @@ public class ReplicationZKLockCleanerChore extends ScheduledChore {
       }
     } catch (KeeperException e) {
       LOG.warn("zk operation interrupted", e);
+    } catch (ReplicationException e2) {
+      LOG.warn("replication exception", e2);
     }
-
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/9b1f379f/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
index 7731240..42d66a5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
@@ -156,6 +156,14 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
     }
   }
 
+  @VisibleForTesting
+  public void setConf(Configuration conf, ZooKeeperWatcher zk, 
+      ReplicationQueuesClient replicationQueuesClient) {
+    super.setConf(conf);
+    this.zkw = zk;
+    this.replicationQueues = replicationQueuesClient;
+  }
+
   @Override
   public void stop(String why) {
     if (this.stopped) return;
@@ -171,7 +179,7 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
     return this.stopped;
   }
 
-  private static class WarnOnlyAbortable implements Abortable {
+  public static class WarnOnlyAbortable implements Abortable {
 
     @Override
     public void abort(String why, Throwable e) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/9b1f379f/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
index 8efa754..df5916c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.doAnswer;
 
 import java.io.IOException;
 import java.lang.reflect.Field;
@@ -29,6 +30,7 @@ import java.net.URLEncoder;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import com.google.common.collect.Lists;
 import org.apache.hadoop.conf.Configuration;
@@ -55,12 +57,13 @@ import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.Stat;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 @Category(MediumTests.class)
 public class TestLogsCleaner {
@@ -177,13 +180,10 @@ public class TestLogsCleaner {
     cleaner.getDeletableFiles(new LinkedList<FileStatus>());
   }
 
-  /**
-   * ReplicationLogCleaner should be able to ride over ZooKeeper errors without
-   * aborting.
-   */
-  @Test
-  public void testZooKeeperAbort() throws Exception {
+  @Test(timeout=10000)
+  public void testZooKeeperAbortDuringGetListOfReplicators() throws Exception {
     Configuration conf = TEST_UTIL.getConfiguration();
+
     ReplicationLogCleaner cleaner = new ReplicationLogCleaner();
 
     List<FileStatus> dummyFiles = Lists.newArrayList(
@@ -193,19 +193,51 @@ public class TestLogsCleaner {
 
     FaultyZooKeeperWatcher faultyZK =
         new FaultyZooKeeperWatcher(conf, "testZooKeeperAbort-faulty", null);
+    final AtomicBoolean getListOfReplicatorsFailed = new AtomicBoolean(false);
+
     try {
       faultyZK.init();
-      cleaner.setConf(conf, faultyZK);
+      ReplicationQueuesClient replicationQueuesClient = spy(ReplicationFactory.getReplicationQueuesClient(
+        faultyZK, conf, new ReplicationLogCleaner.WarnOnlyAbortable()));
+      doAnswer(new Answer<Object>() {
+        @Override
+        public Object answer(InvocationOnMock invocation) throws Throwable {
+          try {
+            return invocation.callRealMethod();
+          } catch (KeeperException.ConnectionLossException e) {
+            getListOfReplicatorsFailed.set(true);
+            throw e;
+          }
+        }
+      }).when(replicationQueuesClient).getListOfReplicators();
+      replicationQueuesClient.init();
+
+      cleaner.setConf(conf, faultyZK, replicationQueuesClient);
       // should keep all files due to a ConnectionLossException getting the queues znodes
       Iterable<FileStatus> toDelete = cleaner.getDeletableFiles(dummyFiles);
+
+      assertTrue(getListOfReplicatorsFailed.get());
       assertFalse(toDelete.iterator().hasNext());
       assertFalse(cleaner.isStopped());
     } finally {
       faultyZK.close();
     }
+  }
+
+  /**
+   * When zk is working both files should be returned
+   * @throws Exception
+   */
+  @Test(timeout=10000)
+  public void testZooKeeperNormal() throws Exception {
+    Configuration conf = TEST_UTIL.getConfiguration();
+    ReplicationLogCleaner cleaner = new ReplicationLogCleaner();
 
-    // when zk is working both files should be returned
-    cleaner = new ReplicationLogCleaner();
+    List<FileStatus> dummyFiles = Lists.newArrayList(
+        new FileStatus(100, false, 3, 100, System.currentTimeMillis(), new Path("log1")),
+        new FileStatus(100, false, 3, 100, System.currentTimeMillis(), new Path("log2"))
+    );
+    
     ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "testZooKeeperAbort-normal", null);
     try {
       cleaner.setConf(conf, zkw);
@@ -291,7 +323,7 @@ public class TestLogsCleaner {
     public void init() throws Exception {
       this.zk = spy(super.getRecoverableZooKeeper());
       doThrow(new KeeperException.ConnectionLossException())
-          .when(zk).getData("/hbase/replication/rs", null, new Stat());
+        .when(zk).getChildren("/hbase/replication/rs", null);
     }
 
     public RecoverableZooKeeper getRecoverableZooKeeper() {


[8/8] hbase git commit: HBASE-18282 ReplicationLogCleaner can delete WALs not yet replicated in case of a KeeperException

Posted by ap...@apache.org.
HBASE-18282 ReplicationLogCleaner can delete WALs not yet replicated in case of a KeeperException

Signed-off-by: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/branch-1.4
Commit: 2d83c7637a27603bc6b47795e5c7bfd7688e220e
Parents: 3c5871b
Author: Ben Lau <be...@oath.com>
Authored: Wed Feb 14 11:36:04 2018 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Feb 14 17:26:31 2018 -0800

----------------------------------------------------------------------
 .../hbase/replication/ReplicationQueues.java    |  3 +-
 .../ReplicationQueuesClientZKImpl.java          |  5 ++
 .../replication/ReplicationQueuesZKImpl.java    | 10 ++++
 .../replication/ReplicationStateZKBase.java     |  8 ++-
 .../cleaner/ReplicationZKLockCleanerChore.java  |  4 +-
 .../master/ReplicationLogCleaner.java           | 10 +++-
 .../hbase/master/cleaner/TestLogsCleaner.java   | 54 ++++++++++++++++----
 7 files changed, 79 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2d83c763/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
index 2409111..ccc7172 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
@@ -126,8 +126,9 @@ public interface ReplicationQueues {
    * Get a list of all region servers that have outstanding replication queues. These servers could
    * be alive, dead or from a previous run of the cluster.
    * @return a list of server names
+   * @throws ReplicationException
    */
-  List<String> getListOfReplicators();
+  List<String> getListOfReplicators() throws ReplicationException;
 
   /**
    * Checks if the provided znode is the same as this region server's

http://git-wip-us.apache.org/repos/asf/hbase/blob/2d83c763/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
index 9078e40..14b4334 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
@@ -49,6 +49,11 @@ public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implem
   }
 
   @Override
+  public List<String> getListOfReplicators() throws KeeperException {
+    return super.getListOfReplicatorsZK();
+  }
+
+  @Override
   public List<String> getLogsInQueue(String serverName, String queueId) throws KeeperException {
     String znode = ZKUtil.joinZNode(this.queuesZNode, serverName);
     znode = ZKUtil.joinZNode(znode, queueId);

http://git-wip-us.apache.org/repos/asf/hbase/blob/2d83c763/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
index a1bd829..dda9adf 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
@@ -103,6 +103,16 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
   }
 
   @Override
+  public List<String> getListOfReplicators() throws ReplicationException {
+    try {
+      return super.getListOfReplicatorsZK();
+    } catch (KeeperException e) {
+      LOG.warn("getListOfReplicators() from ZK failed", e);
+      throw new ReplicationException("getListOfReplicators() from ZK failed", e);
+    }
+  }
+
+  @Override
   public void removeQueue(String queueId) {
     try {
       ZKUtil.deleteNodeRecursively(this.zookeeper, ZKUtil.joinZNode(this.myQueuesZnode, queueId));

http://git-wip-us.apache.org/repos/asf/hbase/blob/2d83c763/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
index d0c3513..0713b99 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
@@ -90,12 +90,18 @@ public abstract class ReplicationStateZKBase {
     this.hfileRefsZNode = ZKUtil.joinZNode(replicationZNode, hfileRefsZNodeName);
   }
 
-  public List<String> getListOfReplicators() {
+  /**
+   * Subclasses that use ZK explicitly can just call this directly while classes
+   * that are trying to hide internal details of storage can wrap the KeeperException
+   * into a ReplicationException or something else.
+   */
+  protected List<String> getListOfReplicatorsZK() throws KeeperException {
     List<String> result = null;
     try {
       result = ZKUtil.listChildrenNoWatch(this.zookeeper, this.queuesZNode);
     } catch (KeeperException e) {
       this.abortable.abort("Failed to get list of replicators", e);
+      throw e;
     }
     return result;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/2d83c763/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKLockCleanerChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKLockCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKLockCleanerChore.java
index 3fa30bf..7c50719 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKLockCleanerChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKLockCleanerChore.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.ScheduledChore;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesZKImpl;
 import org.apache.hadoop.hbase.replication.ReplicationTracker;
@@ -108,7 +109,8 @@ public class ReplicationZKLockCleanerChore extends ScheduledChore {
       }
     } catch (KeeperException e) {
       LOG.warn("zk operation interrupted", e);
+    } catch (ReplicationException e2) {
+      LOG.warn("replication exception", e2);
     }
-
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/2d83c763/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
index 7731240..42d66a5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
@@ -156,6 +156,14 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
     }
   }
 
+  @VisibleForTesting
+  public void setConf(Configuration conf, ZooKeeperWatcher zk, 
+      ReplicationQueuesClient replicationQueuesClient) {
+    super.setConf(conf);
+    this.zkw = zk;
+    this.replicationQueues = replicationQueuesClient;
+  }
+
   @Override
   public void stop(String why) {
     if (this.stopped) return;
@@ -171,7 +179,7 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
     return this.stopped;
   }
 
-  private static class WarnOnlyAbortable implements Abortable {
+  public static class WarnOnlyAbortable implements Abortable {
 
     @Override
     public void abort(String why, Throwable e) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/2d83c763/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
index 8efa754..df5916c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.doAnswer;
 
 import java.io.IOException;
 import java.lang.reflect.Field;
@@ -29,6 +30,7 @@ import java.net.URLEncoder;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import com.google.common.collect.Lists;
 import org.apache.hadoop.conf.Configuration;
@@ -55,12 +57,13 @@ import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.Stat;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 @Category(MediumTests.class)
 public class TestLogsCleaner {
@@ -177,13 +180,10 @@ public class TestLogsCleaner {
     cleaner.getDeletableFiles(new LinkedList<FileStatus>());
   }
 
-  /**
-   * ReplicationLogCleaner should be able to ride over ZooKeeper errors without
-   * aborting.
-   */
-  @Test
-  public void testZooKeeperAbort() throws Exception {
+  @Test(timeout=10000)
+  public void testZooKeeperAbortDuringGetListOfReplicators() throws Exception {
     Configuration conf = TEST_UTIL.getConfiguration();
+
     ReplicationLogCleaner cleaner = new ReplicationLogCleaner();
 
     List<FileStatus> dummyFiles = Lists.newArrayList(
@@ -193,19 +193,51 @@ public class TestLogsCleaner {
 
     FaultyZooKeeperWatcher faultyZK =
         new FaultyZooKeeperWatcher(conf, "testZooKeeperAbort-faulty", null);
+    final AtomicBoolean getListOfReplicatorsFailed = new AtomicBoolean(false);
+
     try {
       faultyZK.init();
-      cleaner.setConf(conf, faultyZK);
+      ReplicationQueuesClient replicationQueuesClient = spy(ReplicationFactory.getReplicationQueuesClient(
+        faultyZK, conf, new ReplicationLogCleaner.WarnOnlyAbortable()));
+      doAnswer(new Answer<Object>() {
+        @Override
+        public Object answer(InvocationOnMock invocation) throws Throwable {
+          try {
+            return invocation.callRealMethod();
+          } catch (KeeperException.ConnectionLossException e) {
+            getListOfReplicatorsFailed.set(true);
+            throw e;
+          }
+        }
+      }).when(replicationQueuesClient).getListOfReplicators();
+      replicationQueuesClient.init();
+
+      cleaner.setConf(conf, faultyZK, replicationQueuesClient);
       // should keep all files due to a ConnectionLossException getting the queues znodes
       Iterable<FileStatus> toDelete = cleaner.getDeletableFiles(dummyFiles);
+
+      assertTrue(getListOfReplicatorsFailed.get());
       assertFalse(toDelete.iterator().hasNext());
       assertFalse(cleaner.isStopped());
     } finally {
       faultyZK.close();
     }
+  }
+
+  /**
+   * When zk is working both files should be returned
+   * @throws Exception
+   */
+  @Test(timeout=10000)
+  public void testZooKeeperNormal() throws Exception {
+    Configuration conf = TEST_UTIL.getConfiguration();
+    ReplicationLogCleaner cleaner = new ReplicationLogCleaner();
 
-    // when zk is working both files should be returned
-    cleaner = new ReplicationLogCleaner();
+    List<FileStatus> dummyFiles = Lists.newArrayList(
+        new FileStatus(100, false, 3, 100, System.currentTimeMillis(), new Path("log1")),
+        new FileStatus(100, false, 3, 100, System.currentTimeMillis(), new Path("log2"))
+    );
+    
     ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "testZooKeeperAbort-normal", null);
     try {
       cleaner.setConf(conf, zkw);
@@ -291,7 +323,7 @@ public class TestLogsCleaner {
     public void init() throws Exception {
       this.zk = spy(super.getRecoverableZooKeeper());
       doThrow(new KeeperException.ConnectionLossException())
-          .when(zk).getData("/hbase/replication/rs", null, new Stat());
+        .when(zk).getChildren("/hbase/replication/rs", null);
     }
 
     public RecoverableZooKeeper getRecoverableZooKeeper() {


[6/8] hbase git commit: HBASE-18282 ReplicationLogCleaner can delete WALs not yet replicated in case of a KeeperException

Posted by ap...@apache.org.
HBASE-18282 ReplicationLogCleaner can delete WALs not yet replicated in case of a KeeperException

Signed-off-by: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/branch-1.2
Commit: ef847f8417b0a300f242fe76769d46d7efb86570
Parents: 0f3bf54
Author: Ben Lau <be...@oath.com>
Authored: Wed Feb 14 11:36:04 2018 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Feb 14 17:23:38 2018 -0800

----------------------------------------------------------------------
 .../hbase/replication/ReplicationQueues.java    |  3 +-
 .../ReplicationQueuesClientZKImpl.java          |  5 ++
 .../replication/ReplicationQueuesZKImpl.java    | 10 ++++
 .../replication/ReplicationStateZKBase.java     |  8 ++-
 .../cleaner/ReplicationZKLockCleanerChore.java  |  4 +-
 .../master/ReplicationLogCleaner.java           | 10 +++-
 .../hbase/master/cleaner/TestLogsCleaner.java   | 54 ++++++++++++++++----
 7 files changed, 79 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ef847f84/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
index 3dbbc33..f1457e0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
@@ -104,8 +104,9 @@ public interface ReplicationQueues {
    * Get a list of all region servers that have outstanding replication queues. These servers could
    * be alive, dead or from a previous run of the cluster.
    * @return a list of server names
+   * @throws ReplicationException
    */
-  List<String> getListOfReplicators();
+  List<String> getListOfReplicators() throws ReplicationException;
 
   /**
    * Checks if the provided znode is the same as this region server's

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef847f84/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
index e1a6a49..93a932f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
@@ -47,6 +47,11 @@ public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implem
   }
 
   @Override
+  public List<String> getListOfReplicators() throws KeeperException {
+    return super.getListOfReplicatorsZK();
+  }
+
+  @Override
   public List<String> getLogsInQueue(String serverName, String queueId) throws KeeperException {
     String znode = ZKUtil.joinZNode(this.queuesZNode, serverName);
     znode = ZKUtil.joinZNode(znode, queueId);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef847f84/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
index 35e5087..3085394 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
@@ -88,6 +88,16 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
   }
 
   @Override
+  public List<String> getListOfReplicators() throws ReplicationException {
+    try {
+      return super.getListOfReplicatorsZK();
+    } catch (KeeperException e) {
+      LOG.warn("getListOfReplicators() from ZK failed", e);
+      throw new ReplicationException("getListOfReplicators() from ZK failed", e);
+    }
+  }
+
+  @Override
   public void removeQueue(String queueId) {
     try {
       ZKUtil.deleteNodeRecursively(this.zookeeper, ZKUtil.joinZNode(this.myQueuesZnode, queueId));

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef847f84/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
index 4fbac0f..75c13d8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
@@ -76,12 +76,18 @@ public abstract class ReplicationStateZKBase {
     this.queuesZNode = ZKUtil.joinZNode(replicationZNode, queuesZNodeName);
   }
 
-  public List<String> getListOfReplicators() {
+  /**
+   * Subclasses that use ZK explicitly can just call this directly while classes
+   * that are trying to hide internal details of storage can wrap the KeeperException
+   * into a ReplicationException or something else.
+   */
+  protected List<String> getListOfReplicatorsZK() throws KeeperException {
     List<String> result = null;
     try {
       result = ZKUtil.listChildrenNoWatch(this.zookeeper, this.queuesZNode);
     } catch (KeeperException e) {
       this.abortable.abort("Failed to get list of replicators", e);
+      throw e;
     }
     return result;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef847f84/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKLockCleanerChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKLockCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKLockCleanerChore.java
index 3fa30bf..7c50719 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKLockCleanerChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKLockCleanerChore.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.ScheduledChore;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesZKImpl;
 import org.apache.hadoop.hbase.replication.ReplicationTracker;
@@ -108,7 +109,8 @@ public class ReplicationZKLockCleanerChore extends ScheduledChore {
       }
     } catch (KeeperException e) {
       LOG.warn("zk operation interrupted", e);
+    } catch (ReplicationException e2) {
+      LOG.warn("replication exception", e2);
     }
-
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef847f84/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
index 7731240..42d66a5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
@@ -156,6 +156,14 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
     }
   }
 
+  @VisibleForTesting
+  public void setConf(Configuration conf, ZooKeeperWatcher zk, 
+      ReplicationQueuesClient replicationQueuesClient) {
+    super.setConf(conf);
+    this.zkw = zk;
+    this.replicationQueues = replicationQueuesClient;
+  }
+
   @Override
   public void stop(String why) {
     if (this.stopped) return;
@@ -171,7 +179,7 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
     return this.stopped;
   }
 
-  private static class WarnOnlyAbortable implements Abortable {
+  public static class WarnOnlyAbortable implements Abortable {
 
     @Override
     public void abort(String why, Throwable e) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/ef847f84/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
index 8efa754..df5916c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.doAnswer;
 
 import java.io.IOException;
 import java.lang.reflect.Field;
@@ -29,6 +30,7 @@ import java.net.URLEncoder;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import com.google.common.collect.Lists;
 import org.apache.hadoop.conf.Configuration;
@@ -55,12 +57,13 @@ import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.Stat;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 @Category(MediumTests.class)
 public class TestLogsCleaner {
@@ -177,13 +180,10 @@ public class TestLogsCleaner {
     cleaner.getDeletableFiles(new LinkedList<FileStatus>());
   }
 
-  /**
-   * ReplicationLogCleaner should be able to ride over ZooKeeper errors without
-   * aborting.
-   */
-  @Test
-  public void testZooKeeperAbort() throws Exception {
+  @Test(timeout=10000)
+  public void testZooKeeperAbortDuringGetListOfReplicators() throws Exception {
     Configuration conf = TEST_UTIL.getConfiguration();
+
     ReplicationLogCleaner cleaner = new ReplicationLogCleaner();
 
     List<FileStatus> dummyFiles = Lists.newArrayList(
@@ -193,19 +193,51 @@ public class TestLogsCleaner {
 
     FaultyZooKeeperWatcher faultyZK =
         new FaultyZooKeeperWatcher(conf, "testZooKeeperAbort-faulty", null);
+    final AtomicBoolean getListOfReplicatorsFailed = new AtomicBoolean(false);
+
     try {
       faultyZK.init();
-      cleaner.setConf(conf, faultyZK);
+      ReplicationQueuesClient replicationQueuesClient = spy(ReplicationFactory.getReplicationQueuesClient(
+        faultyZK, conf, new ReplicationLogCleaner.WarnOnlyAbortable()));
+      doAnswer(new Answer<Object>() {
+        @Override
+        public Object answer(InvocationOnMock invocation) throws Throwable {
+          try {
+            return invocation.callRealMethod();
+          } catch (KeeperException.ConnectionLossException e) {
+            getListOfReplicatorsFailed.set(true);
+            throw e;
+          }
+        }
+      }).when(replicationQueuesClient).getListOfReplicators();
+      replicationQueuesClient.init();
+
+      cleaner.setConf(conf, faultyZK, replicationQueuesClient);
       // should keep all files due to a ConnectionLossException getting the queues znodes
       Iterable<FileStatus> toDelete = cleaner.getDeletableFiles(dummyFiles);
+
+      assertTrue(getListOfReplicatorsFailed.get());
       assertFalse(toDelete.iterator().hasNext());
       assertFalse(cleaner.isStopped());
     } finally {
       faultyZK.close();
     }
+  }
+
+  /**
+   * When zk is working both files should be returned
+   * @throws Exception
+   */
+  @Test(timeout=10000)
+  public void testZooKeeperNormal() throws Exception {
+    Configuration conf = TEST_UTIL.getConfiguration();
+    ReplicationLogCleaner cleaner = new ReplicationLogCleaner();
 
-    // when zk is working both files should be returned
-    cleaner = new ReplicationLogCleaner();
+    List<FileStatus> dummyFiles = Lists.newArrayList(
+        new FileStatus(100, false, 3, 100, System.currentTimeMillis(), new Path("log1")),
+        new FileStatus(100, false, 3, 100, System.currentTimeMillis(), new Path("log2"))
+    );
+    
     ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "testZooKeeperAbort-normal", null);
     try {
       cleaner.setConf(conf, zkw);
@@ -291,7 +323,7 @@ public class TestLogsCleaner {
     public void init() throws Exception {
       this.zk = spy(super.getRecoverableZooKeeper());
       doThrow(new KeeperException.ConnectionLossException())
-          .when(zk).getData("/hbase/replication/rs", null, new Stat());
+        .when(zk).getChildren("/hbase/replication/rs", null);
     }
 
     public RecoverableZooKeeper getRecoverableZooKeeper() {


[7/8] hbase git commit: Update POMs and CHANGES.txt for 1.4.2 RC0

Posted by ap...@apache.org.
Update POMs and CHANGES.txt for 1.4.2 RC0


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

Branch: refs/heads/branch-1.4
Commit: 9519ec2ead17ba3cf81c00618952a9be612574c1
Parents: 2d83c76
Author: Andrew Purtell <ap...@apache.org>
Authored: Wed Feb 14 15:28:49 2018 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Feb 14 17:26:31 2018 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     | 31 ++++++++++++++++++++
 hbase-annotations/pom.xml                       |  2 +-
 .../hbase-archetype-builder/pom.xml             |  2 +-
 hbase-archetypes/hbase-client-project/pom.xml   |  2 +-
 .../hbase-shaded-client-project/pom.xml         |  2 +-
 hbase-archetypes/pom.xml                        |  2 +-
 hbase-assembly/pom.xml                          |  2 +-
 hbase-checkstyle/pom.xml                        |  4 +--
 hbase-client/pom.xml                            |  2 +-
 hbase-common/pom.xml                            |  2 +-
 hbase-error-prone/pom.xml                       |  4 +--
 hbase-examples/pom.xml                          |  2 +-
 hbase-external-blockcache/pom.xml               |  2 +-
 hbase-hadoop-compat/pom.xml                     |  2 +-
 hbase-hadoop2-compat/pom.xml                    |  2 +-
 hbase-it/pom.xml                                |  2 +-
 hbase-metrics-api/pom.xml                       |  2 +-
 hbase-metrics/pom.xml                           |  2 +-
 hbase-prefix-tree/pom.xml                       |  2 +-
 hbase-procedure/pom.xml                         |  2 +-
 hbase-protocol/pom.xml                          |  2 +-
 hbase-resource-bundle/pom.xml                   |  2 +-
 hbase-rest/pom.xml                              |  2 +-
 hbase-rsgroup/pom.xml                           |  2 +-
 hbase-server/pom.xml                            |  2 +-
 .../hbase-shaded-check-invariants/pom.xml       |  2 +-
 hbase-shaded/hbase-shaded-client/pom.xml        |  2 +-
 hbase-shaded/hbase-shaded-server/pom.xml        |  2 +-
 hbase-shaded/pom.xml                            |  2 +-
 hbase-shell/pom.xml                             |  2 +-
 hbase-testing-util/pom.xml                      |  2 +-
 hbase-thrift/pom.xml                            |  2 +-
 pom.xml                                         |  2 +-
 33 files changed, 65 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index a64043f..b50c118 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,5 +1,36 @@
 HBase Change Log
 
+Release Notes - HBase - Version 1.4.2 2/23/2018
+
+** Sub-task
+    * [HBASE-19877] - hbase-common and hbase-zookeeper don't add the log4j.properties to the resource path for testing
+
+** Bug
+    * [HBASE-18282] - ReplicationLogCleaner can delete WALs not yet replicated in case of a KeeperException
+    * [HBASE-19728] - Add lock to filesCompacting in all place.
+    * [HBASE-19871] - delete.rb should require user to provide the column
+    * [HBASE-19876] - The exception happening in converting pb mutation to hbase.mutation messes up the CellScanner
+    * [HBASE-19892] - Checking 'patch attach' and yetus 0.7.0 and move to Yetus 0.7.0
+    * [HBASE-19900] - Region-level exception destroy the result of batch
+    * [HBASE-19901] - Up yetus proclimit on nightlies
+    * [HBASE-19905] - ReplicationSyncUp tool will not exit if a peer replication is disabled
+    * [HBASE-19934] - HBaseSnapshotException when read replicas is enabled and online snapshot is taken after region splitting
+    * [HBASE-19937] - Ensure createRSGroupTable be called after ProcedureExecutor and LoadBalancer are initialized 
+    * [HBASE-19972] - Should rethrow  the RetriesExhaustedWithDetailsException when failed to apply the batch in ReplicationSink
+    * [HBASE-19979] - ReplicationSyncUp tool may leak Zookeeper connection
+    * [HBASE-19981] - Boolean#getBoolean is used to parse value
+    * [HBASE-19996] - Some nonce procs might not be cleaned up (follow up HBASE-19756)
+
+** New Feature
+    * [HBASE-19886] - Display maintenance mode in shell, web UI
+
+** Improvement
+    * [HBASE-19917] - Improve RSGroupBasedLoadBalancer#filterServers() to be more efficient
+
+** Test
+    * [HBASE-19949] - TestRSGroupsWithACL fails with ExceptionInInitializerError
+
+
 Release Notes - HBase - Version 1.4.1 2/2/2018
 
 ** Sub-task

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-annotations/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-annotations/pom.xml b/hbase-annotations/pom.xml
index d2d57a1..cb73616 100644
--- a/hbase-annotations/pom.xml
+++ b/hbase-annotations/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <artifactId>hbase</artifactId>
     <groupId>org.apache.hbase</groupId>
-    <version>1.4.1</version>
+    <version>1.4.2</version>
     <relativePath>..</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-archetypes/hbase-archetype-builder/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-archetypes/hbase-archetype-builder/pom.xml b/hbase-archetypes/hbase-archetype-builder/pom.xml
index f6023ae..6fb1c5d 100644
--- a/hbase-archetypes/hbase-archetype-builder/pom.xml
+++ b/hbase-archetypes/hbase-archetype-builder/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>hbase-archetypes</artifactId>
     <groupId>org.apache.hbase</groupId>
-    <version>1.4.1</version>
+    <version>1.4.2</version>
     <relativePath>..</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-archetypes/hbase-client-project/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-archetypes/hbase-client-project/pom.xml b/hbase-archetypes/hbase-client-project/pom.xml
index 93ae9e0..de87dab 100644
--- a/hbase-archetypes/hbase-client-project/pom.xml
+++ b/hbase-archetypes/hbase-client-project/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>hbase-archetypes</artifactId>
     <groupId>org.apache.hbase</groupId>
-    <version>1.4.1</version>
+    <version>1.4.2</version>
     <relativePath>..</relativePath>
   </parent>
   <artifactId>hbase-client-project</artifactId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-archetypes/hbase-shaded-client-project/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-archetypes/hbase-shaded-client-project/pom.xml b/hbase-archetypes/hbase-shaded-client-project/pom.xml
index 22016cb..ba30b98 100644
--- a/hbase-archetypes/hbase-shaded-client-project/pom.xml
+++ b/hbase-archetypes/hbase-shaded-client-project/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <artifactId>hbase-archetypes</artifactId>
     <groupId>org.apache.hbase</groupId>
-    <version>1.4.1</version>
+    <version>1.4.2</version>
     <relativePath>..</relativePath>
   </parent>
   <artifactId>hbase-shaded-client-project</artifactId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-archetypes/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-archetypes/pom.xml b/hbase-archetypes/pom.xml
index 3426e1c..6e60912 100644
--- a/hbase-archetypes/pom.xml
+++ b/hbase-archetypes/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <artifactId>hbase</artifactId>
     <groupId>org.apache.hbase</groupId>
-    <version>1.4.1</version>
+    <version>1.4.2</version>
     <relativePath>..</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-assembly/pom.xml b/hbase-assembly/pom.xml
index 0c085f9..2a6c4cc 100644
--- a/hbase-assembly/pom.xml
+++ b/hbase-assembly/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <artifactId>hbase</artifactId>
     <groupId>org.apache.hbase</groupId>
-    <version>1.4.1</version>
+    <version>1.4.2</version>
     <relativePath>..</relativePath>
   </parent>
   <artifactId>hbase-assembly</artifactId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-checkstyle/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-checkstyle/pom.xml b/hbase-checkstyle/pom.xml
index da18a8a..ed0b129 100644
--- a/hbase-checkstyle/pom.xml
+++ b/hbase-checkstyle/pom.xml
@@ -24,14 +24,14 @@
 <modelVersion>4.0.0</modelVersion>
 <groupId>org.apache.hbase</groupId>
 <artifactId>hbase-checkstyle</artifactId>
-<version>1.4.1</version>
+<version>1.4.2</version>
 <name>Apache HBase - Checkstyle</name>
 <description>Module to hold Checkstyle properties for HBase.</description>
 
   <parent>
     <artifactId>hbase</artifactId>
     <groupId>org.apache.hbase</groupId>
-    <version>1.4.1</version>
+    <version>1.4.2</version>
     <relativePath>..</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-client/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-client/pom.xml b/hbase-client/pom.xml
index 6a372ff..ba5a1cf 100644
--- a/hbase-client/pom.xml
+++ b/hbase-client/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <artifactId>hbase</artifactId>
     <groupId>org.apache.hbase</groupId>
-    <version>1.4.1</version>
+    <version>1.4.2</version>
     <relativePath>..</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-common/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-common/pom.xml b/hbase-common/pom.xml
index f22f362..45ed8e3 100644
--- a/hbase-common/pom.xml
+++ b/hbase-common/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <artifactId>hbase</artifactId>
     <groupId>org.apache.hbase</groupId>
-    <version>1.4.1</version>
+    <version>1.4.2</version>
     <relativePath>..</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-error-prone/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-error-prone/pom.xml b/hbase-error-prone/pom.xml
index 0d04d6e..04768c9 100644
--- a/hbase-error-prone/pom.xml
+++ b/hbase-error-prone/pom.xml
@@ -23,11 +23,11 @@
   <parent>
     <artifactId>hbase</artifactId>
     <groupId>org.apache.hbase</groupId>
-    <version>1.4.1</version>
+    <version>1.4.2</version>
     <relativePath>..</relativePath>
   </parent>
   <artifactId>hbase-error-prone</artifactId>
-  <version>1.4.1</version>
+  <version>1.4.2</version>
   <name>Apache HBase - Error Prone Rules</name>
   <description>Module to hold error prone custom rules for HBase.</description>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-examples/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-examples/pom.xml b/hbase-examples/pom.xml
index db326bf..b37eb20 100644
--- a/hbase-examples/pom.xml
+++ b/hbase-examples/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <artifactId>hbase</artifactId>
     <groupId>org.apache.hbase</groupId>
-    <version>1.4.1</version>
+    <version>1.4.2</version>
     <relativePath>..</relativePath>
   </parent>
   <artifactId>hbase-examples</artifactId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-external-blockcache/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-external-blockcache/pom.xml b/hbase-external-blockcache/pom.xml
index da1069a..727c1eb 100644
--- a/hbase-external-blockcache/pom.xml
+++ b/hbase-external-blockcache/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>hbase</artifactId>
     <groupId>org.apache.hbase</groupId>
-    <version>1.4.1</version>
+    <version>1.4.2</version>
     <relativePath>..</relativePath>
   </parent>
   <artifactId>hbase-external-blockcache</artifactId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-hadoop-compat/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/pom.xml b/hbase-hadoop-compat/pom.xml
index 51b0744..f39f09e 100644
--- a/hbase-hadoop-compat/pom.xml
+++ b/hbase-hadoop-compat/pom.xml
@@ -23,7 +23,7 @@
     <parent>
         <artifactId>hbase</artifactId>
         <groupId>org.apache.hbase</groupId>
-        <version>1.4.1</version>
+        <version>1.4.2</version>
         <relativePath>..</relativePath>
     </parent>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-hadoop2-compat/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/pom.xml b/hbase-hadoop2-compat/pom.xml
index d5b8f0e..5f4e7c9 100644
--- a/hbase-hadoop2-compat/pom.xml
+++ b/hbase-hadoop2-compat/pom.xml
@@ -21,7 +21,7 @@ limitations under the License.
   <parent>
     <artifactId>hbase</artifactId>
     <groupId>org.apache.hbase</groupId>
-    <version>1.4.1</version>
+    <version>1.4.2</version>
     <relativePath>..</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-it/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-it/pom.xml b/hbase-it/pom.xml
index 07dc014..bb4bffe 100644
--- a/hbase-it/pom.xml
+++ b/hbase-it/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <artifactId>hbase</artifactId>
     <groupId>org.apache.hbase</groupId>
-    <version>1.4.1</version>
+    <version>1.4.2</version>
     <relativePath>..</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-metrics-api/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/pom.xml b/hbase-metrics-api/pom.xml
index 620b5a6..e5053c4 100644
--- a/hbase-metrics-api/pom.xml
+++ b/hbase-metrics-api/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <artifactId>hbase</artifactId>
     <groupId>org.apache.hbase</groupId>
-    <version>1.4.1</version>
+    <version>1.4.2</version>
     <relativePath>..</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-metrics/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-metrics/pom.xml b/hbase-metrics/pom.xml
index 89b2536..542a8c2 100644
--- a/hbase-metrics/pom.xml
+++ b/hbase-metrics/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <artifactId>hbase</artifactId>
     <groupId>org.apache.hbase</groupId>
-    <version>1.4.1</version>
+    <version>1.4.2</version>
     <relativePath>..</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-prefix-tree/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-prefix-tree/pom.xml b/hbase-prefix-tree/pom.xml
index 2e2f112..35ed044 100644
--- a/hbase-prefix-tree/pom.xml
+++ b/hbase-prefix-tree/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <artifactId>hbase</artifactId>
     <groupId>org.apache.hbase</groupId>
-    <version>1.4.1</version>
+    <version>1.4.2</version>
     <relativePath>..</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-procedure/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-procedure/pom.xml b/hbase-procedure/pom.xml
index 2f299c6..d825ebb 100644
--- a/hbase-procedure/pom.xml
+++ b/hbase-procedure/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <artifactId>hbase</artifactId>
     <groupId>org.apache.hbase</groupId>
-    <version>1.4.1</version>
+    <version>1.4.2</version>
     <relativePath>..</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-protocol/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-protocol/pom.xml b/hbase-protocol/pom.xml
index a355aeb..d34dc1c 100644
--- a/hbase-protocol/pom.xml
+++ b/hbase-protocol/pom.xml
@@ -23,7 +23,7 @@
     <parent>
         <artifactId>hbase</artifactId>
         <groupId>org.apache.hbase</groupId>
-        <version>1.4.1</version>
+        <version>1.4.2</version>
         <relativePath>..</relativePath>
     </parent>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-resource-bundle/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-resource-bundle/pom.xml b/hbase-resource-bundle/pom.xml
index 9a24ae5..c576107 100644
--- a/hbase-resource-bundle/pom.xml
+++ b/hbase-resource-bundle/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <artifactId>hbase</artifactId>
     <groupId>org.apache.hbase</groupId>
-    <version>1.4.1</version>
+    <version>1.4.2</version>
     <relativePath>..</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-rest/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-rest/pom.xml b/hbase-rest/pom.xml
index 5c71088..8beae2b 100644
--- a/hbase-rest/pom.xml
+++ b/hbase-rest/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <artifactId>hbase</artifactId>
     <groupId>org.apache.hbase</groupId>
-    <version>1.4.1</version>
+    <version>1.4.2</version>
     <relativePath>..</relativePath>
   </parent>
   <artifactId>hbase-rest</artifactId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-rsgroup/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/pom.xml b/hbase-rsgroup/pom.xml
index 0e958d5..b0084d7 100644
--- a/hbase-rsgroup/pom.xml
+++ b/hbase-rsgroup/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <artifactId>hbase</artifactId>
     <groupId>org.apache.hbase</groupId>
-    <version>1.4.1</version>
+    <version>1.4.2</version>
     <relativePath>..</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index bc096dd..bd06ad5 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <artifactId>hbase</artifactId>
     <groupId>org.apache.hbase</groupId>
-    <version>1.4.1</version>
+    <version>1.4.2</version>
     <relativePath>..</relativePath>
   </parent>
   <artifactId>hbase-server</artifactId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-shaded/hbase-shaded-check-invariants/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shaded/hbase-shaded-check-invariants/pom.xml b/hbase-shaded/hbase-shaded-check-invariants/pom.xml
index dd8e77f..7f7eb4f 100644
--- a/hbase-shaded/hbase-shaded-check-invariants/pom.xml
+++ b/hbase-shaded/hbase-shaded-check-invariants/pom.xml
@@ -16,7 +16,7 @@
   <parent>
     <artifactId>hbase</artifactId>
     <groupId>org.apache.hbase</groupId>
-    <version>1.4.1</version>
+    <version>1.4.2</version>
     <relativePath>../..</relativePath>
   </parent>
   <artifactId>hbase-shaded-check-invariants</artifactId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-shaded/hbase-shaded-client/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shaded/hbase-shaded-client/pom.xml b/hbase-shaded/hbase-shaded-client/pom.xml
index fa81ddd..1c5a857 100644
--- a/hbase-shaded/hbase-shaded-client/pom.xml
+++ b/hbase-shaded/hbase-shaded-client/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <artifactId>hbase-shaded</artifactId>
         <groupId>org.apache.hbase</groupId>
-        <version>1.4.1</version>
+        <version>1.4.2</version>
         <relativePath>..</relativePath>
     </parent>
     <artifactId>hbase-shaded-client</artifactId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-shaded/hbase-shaded-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shaded/hbase-shaded-server/pom.xml b/hbase-shaded/hbase-shaded-server/pom.xml
index fbc180f..e69ba75 100644
--- a/hbase-shaded/hbase-shaded-server/pom.xml
+++ b/hbase-shaded/hbase-shaded-server/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <artifactId>hbase-shaded</artifactId>
         <groupId>org.apache.hbase</groupId>
-        <version>1.4.1</version>
+        <version>1.4.2</version>
         <relativePath>..</relativePath>
     </parent>
     <artifactId>hbase-shaded-server</artifactId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-shaded/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shaded/pom.xml b/hbase-shaded/pom.xml
index 20b3b0f..83dfb4c 100644
--- a/hbase-shaded/pom.xml
+++ b/hbase-shaded/pom.xml
@@ -23,7 +23,7 @@
     <parent>
         <artifactId>hbase</artifactId>
         <groupId>org.apache.hbase</groupId>
-        <version>1.4.1</version>
+        <version>1.4.2</version>
         <relativePath>..</relativePath>
     </parent>
     <artifactId>hbase-shaded</artifactId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-shell/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shell/pom.xml b/hbase-shell/pom.xml
index 7e72c01..97c58f3 100644
--- a/hbase-shell/pom.xml
+++ b/hbase-shell/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <artifactId>hbase</artifactId>
     <groupId>org.apache.hbase</groupId>
-    <version>1.4.1</version>
+    <version>1.4.2</version>
     <relativePath>..</relativePath>
   </parent>
   <artifactId>hbase-shell</artifactId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-testing-util/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-testing-util/pom.xml b/hbase-testing-util/pom.xml
index e28c21e..7bf4696 100644
--- a/hbase-testing-util/pom.xml
+++ b/hbase-testing-util/pom.xml
@@ -23,7 +23,7 @@
     <parent>
         <artifactId>hbase</artifactId>
         <groupId>org.apache.hbase</groupId>
-        <version>1.4.1</version>
+        <version>1.4.2</version>
         <relativePath>..</relativePath>
     </parent>
     <artifactId>hbase-testing-util</artifactId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/hbase-thrift/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-thrift/pom.xml b/hbase-thrift/pom.xml
index dea1336..da4f302 100644
--- a/hbase-thrift/pom.xml
+++ b/hbase-thrift/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <artifactId>hbase</artifactId>
     <groupId>org.apache.hbase</groupId>
-    <version>1.4.1</version>
+    <version>1.4.2</version>
     <relativePath>..</relativePath>
   </parent>
   <artifactId>hbase-thrift</artifactId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/9519ec2e/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 0fcc4be..f7699b6 100644
--- a/pom.xml
+++ b/pom.xml
@@ -39,7 +39,7 @@
   <groupId>org.apache.hbase</groupId>
   <artifactId>hbase</artifactId>
   <packaging>pom</packaging>
-  <version>1.4.1</version>
+  <version>1.4.2</version>
   <name>Apache HBase</name>
   <description>
     Apache HBase™ is the Hadoop database. Use it when you need