You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2017/12/22 13:44:59 UTC

[01/26] hbase git commit: HBASE-19578 MasterProcWALs cleaning is incorrect [Forced Update!]

Repository: hbase
Updated Branches:
  refs/heads/HBASE-19397 61b974789 -> 557f5d4d8 (forced update)


HBASE-19578 MasterProcWALs cleaning is incorrect

Signed-off-by: tedyu <yu...@gmail.com>


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

Branch: refs/heads/HBASE-19397
Commit: 35728acd217dcfa52a2f274a064ba6e847835b9f
Parents: 00b0288
Author: Peter Somogyi <ps...@cloudera.com>
Authored: Thu Dec 21 13:15:05 2017 +0100
Committer: tedyu <yu...@gmail.com>
Committed: Thu Dec 21 09:38:25 2017 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java      | 2 +-
 .../hadoop/hbase/master/procedure/MasterProcedureUtil.java       | 2 +-
 .../org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java  | 4 ++--
 3 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/35728acd/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
index 1e9ef6e..6226350 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
@@ -160,7 +160,7 @@ public class ProcedureWALFile implements Comparable<ProcedureWALFile> {
     boolean archived = false;
     if (walArchiveDir != null) {
       Path archivedFile = new Path(walArchiveDir, logFile.getName());
-      LOG.info("ARCHIVED (TODO: FILES ARE NOT PURGED FROM ARCHIVE!) " + logFile + " to " + archivedFile);
+      LOG.info("Archiving " + logFile + " to " + archivedFile);
       if (!fs.rename(logFile, archivedFile)) {
         LOG.warn("Failed archive of " + logFile + ", deleting");
       } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/35728acd/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureUtil.java
index b87f437..4afd711 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureUtil.java
@@ -147,7 +147,7 @@ public final class MasterProcedureUtil {
    * Pattern used to validate a Procedure WAL file name see
    * {@link #validateProcedureWALFilename(String)} for description.
    */
-  private static final Pattern pattern = Pattern.compile(".*pv-\\d{20}.log");
+  private static final Pattern pattern = Pattern.compile(".*pv2-\\d{20}.log");
 
   /**
    * A Procedure WAL file name is of the format: pv-&lt;wal-id&gt;.log where wal-id is 20 digits.

http://git-wip-us.apache.org/repos/asf/hbase/blob/35728acd/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 43fc6a4..2d517c4 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
@@ -134,7 +134,7 @@ public class TestLogsCleaner {
 
     // Case 2: 5 Procedure WALs that are old which would be deleted
     for (int i = 1; i < 6; i++) {
-      Path fileName = new Path(oldProcedureWALDir, String.format("pv-%020d.log", i));
+      Path fileName = new Path(oldProcedureWALDir, String.format("pv2-%020d.log", i));
       fs.createNewFile(fileName);
     }
 
@@ -155,7 +155,7 @@ public class TestLogsCleaner {
 
     // Case 5: 5 Procedure WALs that are new, will stay
     for (int i = 6; i < 11; i++) {
-      Path fileName = new Path(oldProcedureWALDir, String.format("pv-%020d.log", i));
+      Path fileName = new Path(oldProcedureWALDir, String.format("pv2-%020d.log", i));
       fs.createNewFile(fileName);
     }
 


[20/26] hbase git commit: HBASE-19536 Client side changes for moving peer modification from zk watcher to procedure

Posted by zh...@apache.org.
HBASE-19536 Client side changes for moving peer modification from zk watcher to procedure

Signed-off-by: zhangduo <zh...@apache.org>


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

Branch: refs/heads/HBASE-19397
Commit: f2077c962f4d507737940e89effae3f2ceebe95d
Parents: 577c19a
Author: Guanghao Zhang <zg...@apache.org>
Authored: Tue Dec 19 15:50:57 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Fri Dec 22 21:41:28 2017 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Admin.java   |  87 ++++++++++-
 .../apache/hadoop/hbase/client/HBaseAdmin.java  | 149 ++++++++++++++-----
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java |  82 +++++-----
 3 files changed, 238 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f2077c96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index de022d3..a836811 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -2463,7 +2463,7 @@ public interface Admin extends Abortable, Closeable {
   /**
    * Add a new replication peer for replicating data to slave cluster.
    * @param peerId a short name that identifies the peer
-   * @param peerConfig configuration for the replication slave cluster
+   * @param peerConfig configuration for the replication peer
    * @throws IOException if a remote or network exception occurs
    */
   default void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig)
@@ -2474,7 +2474,7 @@ public interface Admin extends Abortable, Closeable {
   /**
    * Add a new replication peer for replicating data to slave cluster.
    * @param peerId a short name that identifies the peer
-   * @param peerConfig configuration for the replication slave cluster
+   * @param peerConfig configuration for the replication peer
    * @param enabled peer state, true if ENABLED and false if DISABLED
    * @throws IOException if a remote or network exception occurs
    */
@@ -2482,6 +2482,37 @@ public interface Admin extends Abortable, Closeable {
       throws IOException;
 
   /**
+   * Add a new replication peer but does not block and wait for it.
+   * <p>
+   * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
+   * ExecutionException if there was an error while executing the operation or TimeoutException in
+   * case the wait timeout was not long enough to allow the operation to complete.
+   * @param peerId a short name that identifies the peer
+   * @param peerConfig configuration for the replication peer
+   * @return the result of the async operation
+   * @throws IOException IOException if a remote or network exception occurs
+   */
+  default Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig)
+      throws IOException {
+    return addReplicationPeerAsync(peerId, peerConfig, true);
+  }
+
+  /**
+   * Add a new replication peer but does not block and wait for it.
+   * <p>
+   * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
+   * ExecutionException if there was an error while executing the operation or TimeoutException in
+   * case the wait timeout was not long enough to allow the operation to complete.
+   * @param peerId a short name that identifies the peer
+   * @param peerConfig configuration for the replication peer
+   * @param enabled peer state, true if ENABLED and false if DISABLED
+   * @return the result of the async operation
+   * @throws IOException IOException if a remote or network exception occurs
+   */
+  Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig,
+      boolean enabled) throws IOException;
+
+  /**
    * Remove a peer and stop the replication.
    * @param peerId a short name that identifies the peer
    * @throws IOException if a remote or network exception occurs
@@ -2489,6 +2520,18 @@ public interface Admin extends Abortable, Closeable {
   void removeReplicationPeer(String peerId) throws IOException;
 
   /**
+   * Remove a replication peer but does not block and wait for it.
+   * <p>
+   * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
+   * ExecutionException if there was an error while executing the operation or TimeoutException in
+   * case the wait timeout was not long enough to allow the operation to complete.
+   * @param peerId a short name that identifies the peer
+   * @return the result of the async operation
+   * @throws IOException IOException if a remote or network exception occurs
+   */
+  Future<Void> removeReplicationPeerAsync(String peerId) throws IOException;
+
+  /**
    * Restart the replication stream to the specified peer.
    * @param peerId a short name that identifies the peer
    * @throws IOException if a remote or network exception occurs
@@ -2496,6 +2539,18 @@ public interface Admin extends Abortable, Closeable {
   void enableReplicationPeer(String peerId) throws IOException;
 
   /**
+   * Enable a replication peer but does not block and wait for it.
+   * <p>
+   * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
+   * ExecutionException if there was an error while executing the operation or TimeoutException in
+   * case the wait timeout was not long enough to allow the operation to complete.
+   * @param peerId a short name that identifies the peer
+   * @return the result of the async operation
+   * @throws IOException IOException if a remote or network exception occurs
+   */
+  Future<Void> enableReplicationPeerAsync(String peerId) throws IOException;
+
+  /**
    * Stop the replication stream to the specified peer.
    * @param peerId a short name that identifies the peer
    * @throws IOException if a remote or network exception occurs
@@ -2503,6 +2558,18 @@ public interface Admin extends Abortable, Closeable {
   void disableReplicationPeer(String peerId) throws IOException;
 
   /**
+   * Disable a replication peer but does not block and wait for it.
+   * <p>
+   * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
+   * ExecutionException if there was an error while executing the operation or TimeoutException in
+   * case the wait timeout was not long enough to allow the operation to complete.
+   * @param peerId a short name that identifies the peer
+   * @return the result of the async operation
+   * @throws IOException IOException if a remote or network exception occurs
+   */
+  Future<Void> disableReplicationPeerAsync(String peerId) throws IOException;
+
+  /**
    * Returns the configured ReplicationPeerConfig for the specified peer.
    * @param peerId a short name that identifies the peer
    * @return ReplicationPeerConfig for the peer
@@ -2513,13 +2580,27 @@ public interface Admin extends Abortable, Closeable {
   /**
    * Update the peerConfig for the specified peer.
    * @param peerId a short name that identifies the peer
-   * @param peerConfig new config for the peer
+   * @param peerConfig new config for the replication peer
    * @throws IOException if a remote or network exception occurs
    */
   void updateReplicationPeerConfig(String peerId,
       ReplicationPeerConfig peerConfig) throws IOException;
 
   /**
+   * Update the peerConfig for the specified peer but does not block and wait for it.
+   * <p>
+   * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
+   * ExecutionException if there was an error while executing the operation or TimeoutException in
+   * case the wait timeout was not long enough to allow the operation to complete.
+   * @param peerId a short name that identifies the peer
+   * @param peerConfig new config for the replication peer
+   * @return the result of the async operation
+   * @throws IOException IOException if a remote or network exception occurs
+   */
+  Future<Void> updateReplicationPeerConfigAsync(String peerId, ReplicationPeerConfig peerConfig)
+      throws IOException;
+
+  /**
    * Append the replicable table column family config from the specified peer.
    * @param id a short that identifies the cluster
    * @param tableCfs A map from tableName to column family names

http://git-wip-us.apache.org/repos/asf/hbase/blob/f2077c96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index bbcc825..ac3aa38 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -41,6 +41,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
@@ -200,7 +201,12 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTa
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
 
 /**
@@ -3780,6 +3786,25 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  @InterfaceAudience.Private
+  @InterfaceStability.Evolving
+  private static class ReplicationFuture extends ProcedureFuture<Void> {
+    private final String peerId;
+    private final Supplier<String> getOperation;
+
+    public ReplicationFuture(HBaseAdmin admin, String peerId, Long procId,
+        Supplier<String> getOperation) {
+      super(admin, procId);
+      this.peerId = peerId;
+      this.getOperation = getOperation;
+    }
+
+    @Override
+    public String toString() {
+      return "Operation: " + getOperation.get() + ", peerId: " + peerId;
+    }
+  }
+
   @Override
   public List<SecurityCapability> getSecurityCapabilities() throws IOException {
     try {
@@ -3852,50 +3877,82 @@ public class HBaseAdmin implements Admin {
   @Override
   public void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
       throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Void rpcCall() throws Exception {
-        master.addReplicationPeer(getRpcController(),
-          RequestConverter.buildAddReplicationPeerRequest(peerId, peerConfig, enabled));
-        return null;
-      }
-    });
+    get(addReplicationPeerAsync(peerId, peerConfig, enabled), this.syncWaitTimeout,
+      TimeUnit.MILLISECONDS);
+  }
+
+  @Override
+  public Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig,
+      boolean enabled) throws IOException {
+    AddReplicationPeerResponse response = executeCallable(
+      new MasterCallable<AddReplicationPeerResponse>(getConnection(), getRpcControllerFactory()) {
+        @Override
+        protected AddReplicationPeerResponse rpcCall() throws Exception {
+          return master.addReplicationPeer(getRpcController(),
+            RequestConverter.buildAddReplicationPeerRequest(peerId, peerConfig, enabled));
+        }
+      });
+    return new ReplicationFuture(this, peerId, response.getProcId(), () -> "ADD_REPLICATION_PEER");
   }
 
   @Override
   public void removeReplicationPeer(String peerId) throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Void rpcCall() throws Exception {
-        master.removeReplicationPeer(getRpcController(),
-          RequestConverter.buildRemoveReplicationPeerRequest(peerId));
-        return null;
-      }
-    });
+    get(removeReplicationPeerAsync(peerId), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
+  }
+
+  @Override
+  public Future<Void> removeReplicationPeerAsync(String peerId) throws IOException {
+    RemoveReplicationPeerResponse response =
+      executeCallable(new MasterCallable<RemoveReplicationPeerResponse>(getConnection(),
+          getRpcControllerFactory()) {
+        @Override
+        protected RemoveReplicationPeerResponse rpcCall() throws Exception {
+          return master.removeReplicationPeer(getRpcController(),
+            RequestConverter.buildRemoveReplicationPeerRequest(peerId));
+        }
+      });
+    return new ReplicationFuture(this, peerId, response.getProcId(),
+      () -> "REMOVE_REPLICATION_PEER");
   }
 
   @Override
   public void enableReplicationPeer(final String peerId) throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Void rpcCall() throws Exception {
-        master.enableReplicationPeer(getRpcController(),
-          RequestConverter.buildEnableReplicationPeerRequest(peerId));
-        return null;
-      }
-    });
+    get(enableReplicationPeerAsync(peerId), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
+  }
+
+  @Override
+  public Future<Void> enableReplicationPeerAsync(final String peerId) throws IOException {
+    EnableReplicationPeerResponse response =
+      executeCallable(new MasterCallable<EnableReplicationPeerResponse>(getConnection(),
+          getRpcControllerFactory()) {
+        @Override
+        protected EnableReplicationPeerResponse rpcCall() throws Exception {
+          return master.enableReplicationPeer(getRpcController(),
+            RequestConverter.buildEnableReplicationPeerRequest(peerId));
+        }
+      });
+    return new ReplicationFuture(this, peerId, response.getProcId(),
+      () -> "ENABLE_REPLICATION_PEER");
   }
 
   @Override
   public void disableReplicationPeer(final String peerId) throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Void rpcCall() throws Exception {
-        master.disableReplicationPeer(getRpcController(),
-          RequestConverter.buildDisableReplicationPeerRequest(peerId));
-        return null;
-      }
-    });
+    get(disableReplicationPeerAsync(peerId), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
+  }
+
+  @Override
+  public Future<Void> disableReplicationPeerAsync(final String peerId) throws IOException {
+    DisableReplicationPeerResponse response =
+      executeCallable(new MasterCallable<DisableReplicationPeerResponse>(getConnection(),
+          getRpcControllerFactory()) {
+        @Override
+        protected DisableReplicationPeerResponse rpcCall() throws Exception {
+          return master.disableReplicationPeer(getRpcController(),
+            RequestConverter.buildDisableReplicationPeerRequest(peerId));
+        }
+      });
+    return new ReplicationFuture(this, peerId, response.getProcId(),
+      () -> "DISABLE_REPLICATION_PEER");
   }
 
   @Override
@@ -3914,14 +3971,24 @@ public class HBaseAdmin implements Admin {
   @Override
   public void updateReplicationPeerConfig(final String peerId,
       final ReplicationPeerConfig peerConfig) throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Void rpcCall() throws Exception {
-        master.updateReplicationPeerConfig(getRpcController(),
-          RequestConverter.buildUpdateReplicationPeerConfigRequest(peerId, peerConfig));
-        return null;
-      }
-    });
+    get(updateReplicationPeerConfigAsync(peerId, peerConfig), this.syncWaitTimeout,
+      TimeUnit.MILLISECONDS);
+  }
+
+  @Override
+  public Future<Void> updateReplicationPeerConfigAsync(final String peerId,
+      final ReplicationPeerConfig peerConfig) throws IOException {
+    UpdateReplicationPeerConfigResponse response =
+      executeCallable(new MasterCallable<UpdateReplicationPeerConfigResponse>(getConnection(),
+          getRpcControllerFactory()) {
+        @Override
+        protected UpdateReplicationPeerConfigResponse rpcCall() throws Exception {
+          return master.updateReplicationPeerConfig(getRpcController(),
+            RequestConverter.buildUpdateReplicationPeerConfigRequest(peerId, peerConfig));
+        }
+      });
+    return new ReplicationFuture(this, peerId, response.getProcId(),
+      () -> "UPDATE_REPLICATION_PEER_CONFIG");
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/f2077c96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index 5e1c654..c442d7b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -39,6 +39,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.BiConsumer;
 import java.util.function.Function;
+import java.util.function.Supplier;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
@@ -1523,47 +1524,34 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   @Override
   public CompletableFuture<Void> addReplicationPeer(String peerId,
       ReplicationPeerConfig peerConfig, boolean enabled) {
-    return this
-        .<Void> newMasterCaller()
-        .action(
-          (controller, stub) -> this
-              .<AddReplicationPeerRequest, AddReplicationPeerResponse, Void> call(controller, stub,
-                RequestConverter.buildAddReplicationPeerRequest(peerId, peerConfig, enabled), (s,
-                    c, req, done) -> s.addReplicationPeer(c, req, done), (resp) -> null)).call();
+    return this.<AddReplicationPeerRequest, AddReplicationPeerResponse> procedureCall(
+      RequestConverter.buildAddReplicationPeerRequest(peerId, peerConfig, enabled),
+      (s, c, req, done) -> s.addReplicationPeer(c, req, done), (resp) -> resp.getProcId(),
+      new ReplicationProcedureBiConsumer(peerId, () -> "ADD_REPLICATION_PEER"));
   }
 
   @Override
   public CompletableFuture<Void> removeReplicationPeer(String peerId) {
-    return this
-        .<Void> newMasterCaller()
-        .action(
-          (controller, stub) -> this
-              .<RemoveReplicationPeerRequest, RemoveReplicationPeerResponse, Void> call(controller,
-                stub, RequestConverter.buildRemoveReplicationPeerRequest(peerId),
-                (s, c, req, done) -> s.removeReplicationPeer(c, req, done), (resp) -> null)).call();
+    return this.<RemoveReplicationPeerRequest, RemoveReplicationPeerResponse> procedureCall(
+      RequestConverter.buildRemoveReplicationPeerRequest(peerId),
+      (s, c, req, done) -> s.removeReplicationPeer(c, req, done), (resp) -> resp.getProcId(),
+      new ReplicationProcedureBiConsumer(peerId, () -> "REMOVE_REPLICATION_PEER"));
   }
 
   @Override
   public CompletableFuture<Void> enableReplicationPeer(String peerId) {
-    return this
-        .<Void> newMasterCaller()
-        .action(
-          (controller, stub) -> this
-              .<EnableReplicationPeerRequest, EnableReplicationPeerResponse, Void> call(controller,
-                stub, RequestConverter.buildEnableReplicationPeerRequest(peerId),
-                (s, c, req, done) -> s.enableReplicationPeer(c, req, done), (resp) -> null)).call();
+    return this.<EnableReplicationPeerRequest, EnableReplicationPeerResponse> procedureCall(
+      RequestConverter.buildEnableReplicationPeerRequest(peerId),
+      (s, c, req, done) -> s.enableReplicationPeer(c, req, done), (resp) -> resp.getProcId(),
+      new ReplicationProcedureBiConsumer(peerId, () -> "ENABLE_REPLICATION_PEER"));
   }
 
   @Override
   public CompletableFuture<Void> disableReplicationPeer(String peerId) {
-    return this
-        .<Void> newMasterCaller()
-        .action(
-          (controller, stub) -> this
-              .<DisableReplicationPeerRequest, DisableReplicationPeerResponse, Void> call(
-                controller, stub, RequestConverter.buildDisableReplicationPeerRequest(peerId), (s,
-                    c, req, done) -> s.disableReplicationPeer(c, req, done), (resp) -> null))
-        .call();
+    return this.<DisableReplicationPeerRequest, DisableReplicationPeerResponse> procedureCall(
+      RequestConverter.buildDisableReplicationPeerRequest(peerId),
+      (s, c, req, done) -> s.disableReplicationPeer(c, req, done), (resp) -> resp.getProcId(),
+      new ReplicationProcedureBiConsumer(peerId, () -> "DISABLE_REPLICATION_PEER"));
   }
 
   @Override
@@ -1582,13 +1570,11 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   public CompletableFuture<Void> updateReplicationPeerConfig(String peerId,
       ReplicationPeerConfig peerConfig) {
     return this
-        .<Void> newMasterCaller()
-        .action(
-          (controller, stub) -> this
-              .<UpdateReplicationPeerConfigRequest, UpdateReplicationPeerConfigResponse, Void> call(
-                controller, stub, RequestConverter.buildUpdateReplicationPeerConfigRequest(peerId,
-                  peerConfig), (s, c, req, done) -> s.updateReplicationPeerConfig(c, req, done), (
-                    resp) -> null)).call();
+        .<UpdateReplicationPeerConfigRequest, UpdateReplicationPeerConfigResponse> procedureCall(
+          RequestConverter.buildUpdateReplicationPeerConfigRequest(peerId, peerConfig),
+          (s, c, req, done) -> s.updateReplicationPeerConfig(c, req, done),
+          (resp) -> resp.getProcId(),
+          new ReplicationProcedureBiConsumer(peerId, () -> "UPDATE_REPLICATION_PEER_CONFIG"));
   }
 
   @Override
@@ -2545,6 +2531,30 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     }
   }
 
+  private class ReplicationProcedureBiConsumer extends ProcedureBiConsumer {
+    private final String peerId;
+    private final Supplier<String> getOperation;
+
+    ReplicationProcedureBiConsumer(String peerId, Supplier<String> getOperation) {
+      this.peerId = peerId;
+      this.getOperation = getOperation;
+    }
+
+    String getDescription() {
+      return "Operation: " + getOperation.get() + ", peerId: " + peerId;
+    }
+
+    @Override
+    void onFinished() {
+      LOG.info(getDescription() + " completed");
+    }
+
+    @Override
+    void onError(Throwable error) {
+      LOG.info(getDescription() + " failed with " + error.getMessage());
+    }
+  }
+
   private CompletableFuture<Void> waitProcedureResult(CompletableFuture<Long> procFuture) {
     CompletableFuture<Void> future = new CompletableFuture<>();
     procFuture.whenComplete((procId, error) -> {


[25/26] hbase git commit: HBASE-19543 Abstract a replication storage interface to extract the zk specific code

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/557f5d4d/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java
deleted file mode 100644
index b6f8784..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java
+++ /dev/null
@@ -1,199 +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.master.replication;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.regex.Pattern;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.ReplicationPeerNotFoundException;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.replication.BaseReplicationEndpoint;
-import org.apache.hadoop.hbase.replication.ReplicationException;
-import org.apache.hadoop.hbase.replication.ReplicationFactory;
-import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
-import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
-import org.apache.hadoop.hbase.replication.ReplicationPeers;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.apache.yetus.audience.InterfaceAudience;
-
-/**
- * Manages and performs all replication admin operations.
- * <p>
- * Used to add/remove a replication peer.
- */
-@InterfaceAudience.Private
-public class ReplicationManager {
-  private final ReplicationQueuesClient replicationQueuesClient;
-  private final ReplicationPeers replicationPeers;
-
-  public ReplicationManager(Configuration conf, ZKWatcher zkw, Abortable abortable)
-      throws IOException {
-    try {
-      this.replicationQueuesClient = ReplicationFactory
-          .getReplicationQueuesClient(new ReplicationQueuesClientArguments(conf, abortable, zkw));
-      this.replicationQueuesClient.init();
-      this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf,
-        this.replicationQueuesClient, abortable);
-      this.replicationPeers.init();
-    } catch (Exception e) {
-      throw new IOException("Failed to construct ReplicationManager", e);
-    }
-  }
-
-  public void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
-      throws ReplicationException {
-    checkPeerConfig(peerConfig);
-    replicationPeers.registerPeer(peerId, peerConfig, enabled);
-    replicationPeers.peerConnected(peerId);
-  }
-
-  public void removeReplicationPeer(String peerId) throws ReplicationException {
-    replicationPeers.peerDisconnected(peerId);
-    replicationPeers.unregisterPeer(peerId);
-  }
-
-  public void enableReplicationPeer(String peerId) throws ReplicationException {
-    this.replicationPeers.enablePeer(peerId);
-  }
-
-  public void disableReplicationPeer(String peerId) throws ReplicationException {
-    this.replicationPeers.disablePeer(peerId);
-  }
-
-  public ReplicationPeerConfig getPeerConfig(String peerId)
-      throws ReplicationException, ReplicationPeerNotFoundException {
-    ReplicationPeerConfig peerConfig = replicationPeers.getReplicationPeerConfig(peerId);
-    if (peerConfig == null) {
-      throw new ReplicationPeerNotFoundException(peerId);
-    }
-    return peerConfig;
-  }
-
-  public void updatePeerConfig(String peerId, ReplicationPeerConfig peerConfig)
-      throws ReplicationException, IOException {
-    checkPeerConfig(peerConfig);
-    this.replicationPeers.updatePeerConfig(peerId, peerConfig);
-  }
-
-  public List<ReplicationPeerDescription> listReplicationPeers(Pattern pattern)
-      throws ReplicationException {
-    List<ReplicationPeerDescription> peers = new ArrayList<>();
-    List<String> peerIds = replicationPeers.getAllPeerIds();
-    for (String peerId : peerIds) {
-      if (pattern == null || (pattern != null && pattern.matcher(peerId).matches())) {
-        peers.add(new ReplicationPeerDescription(peerId,
-            replicationPeers.getStatusOfPeerFromBackingStore(peerId),
-            replicationPeers.getReplicationPeerConfig(peerId)));
-      }
-    }
-    return peers;
-  }
-
-  /**
-   * If replicate_all flag is true, it means all user tables will be replicated to peer cluster.
-   * Then allow config exclude namespaces or exclude table-cfs which can't be replicated to
-   * peer cluster.
-   *
-   * If replicate_all flag is false, it means all user tables can't be replicated to peer cluster.
-   * Then allow to config namespaces or table-cfs which will be replicated to peer cluster.
-   */
-  private void checkPeerConfig(ReplicationPeerConfig peerConfig) {
-    if (peerConfig.replicateAllUserTables()) {
-      if ((peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty()) ||
-        (peerConfig.getTableCFsMap() != null && !peerConfig.getTableCFsMap().isEmpty())) {
-        throw new IllegalArgumentException("Need clean namespaces or table-cfs config firstly " +
-          "when you want replicate all cluster");
-      }
-      checkNamespacesAndTableCfsConfigConflict(peerConfig.getExcludeNamespaces(),
-        peerConfig.getExcludeTableCFsMap());
-    } else {
-      if ((peerConfig.getExcludeNamespaces() != null
-          && !peerConfig.getExcludeNamespaces().isEmpty())
-          || (peerConfig.getExcludeTableCFsMap() != null
-              && !peerConfig.getExcludeTableCFsMap().isEmpty())) {
-        throw new IllegalArgumentException(
-            "Need clean exclude-namespaces or exclude-table-cfs config firstly"
-                + " when replicate_all flag is false");
-      }
-      checkNamespacesAndTableCfsConfigConflict(peerConfig.getNamespaces(),
-        peerConfig.getTableCFsMap());
-    }
-    checkConfiguredWALEntryFilters(peerConfig);
-  }
-
-  /**
-   * Set a namespace in the peer config means that all tables in this namespace will be replicated
-   * to the peer cluster.
-   * <ol>
-   * <li>If peer config already has a namespace, then not allow set any table of this namespace to
-   * the peer config.</li>
-   * <li>If peer config already has a table, then not allow set this table's namespace to the peer
-   * config.</li>
-   * </ol>
-   * <p>
-   * Set a exclude namespace in the peer config means that all tables in this namespace can't be
-   * replicated to the peer cluster.
-   * <ol>
-   * <li>If peer config already has a exclude namespace, then not allow set any exclude table of
-   * this namespace to the peer config.</li>
-   * <li>If peer config already has a exclude table, then not allow set this table's namespace as a
-   * exclude namespace.</li>
-   * </ol>
-   */
-  private void checkNamespacesAndTableCfsConfigConflict(Set<String> namespaces,
-      Map<TableName, ? extends Collection<String>> tableCfs) {
-    if (namespaces == null || namespaces.isEmpty()) {
-      return;
-    }
-    if (tableCfs == null || tableCfs.isEmpty()) {
-      return;
-    }
-    for (Map.Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
-      TableName table = entry.getKey();
-      if (namespaces.contains(table.getNamespaceAsString())) {
-        throw new IllegalArgumentException("Table-cfs " + table + " is conflict with namespaces "
-            + table.getNamespaceAsString() + " in peer config");
-      }
-    }
-  }
-
-  private void checkConfiguredWALEntryFilters(ReplicationPeerConfig peerConfig) {
-    String filterCSV = peerConfig.getConfiguration()
-        .get(BaseReplicationEndpoint.REPLICATION_WALENTRYFILTER_CONFIG_KEY);
-    if (filterCSV != null && !filterCSV.isEmpty()) {
-      String[] filters = filterCSV.split(",");
-      for (String filter : filters) {
-        try {
-          Class.forName(filter).newInstance();
-        } catch (Exception e) {
-          throw new IllegalArgumentException("Configured WALEntryFilter " + filter +
-              " could not be created. Failing add/update " + "peer operation.", e);
-        }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/557f5d4d/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
new file mode 100644
index 0000000..5abd874
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
@@ -0,0 +1,331 @@
+/**
+ * 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.master.replication;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.replication.BaseReplicationEndpoint;
+import org.apache.hadoop.hbase.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
+import org.apache.hadoop.hbase.replication.ReplicationPeerStorage;
+import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Manages and performs all replication admin operations.
+ * <p>
+ * Used to add/remove a replication peer.
+ */
+@InterfaceAudience.Private
+public final class ReplicationPeerManager {
+
+  private final ReplicationPeerStorage peerStorage;
+
+  private final ReplicationQueueStorage queueStorage;
+
+  private final ConcurrentMap<String, ReplicationPeerDescription> peers;
+
+  private ReplicationPeerManager(ReplicationPeerStorage peerStorage,
+      ReplicationQueueStorage queueStorage,
+      ConcurrentMap<String, ReplicationPeerDescription> peers) {
+    this.peerStorage = peerStorage;
+    this.queueStorage = queueStorage;
+    this.peers = peers;
+  }
+
+  private void checkQueuesDeleted(String peerId)
+      throws ReplicationException, DoNotRetryIOException {
+    for (ServerName replicator : queueStorage.getListOfReplicators()) {
+      List<String> queueIds = queueStorage.getAllQueues(replicator);
+      for (String queueId : queueIds) {
+        ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
+        if (queueInfo.getPeerId().equals(peerId)) {
+          throw new DoNotRetryIOException("undeleted queue for peerId: " + peerId +
+            ", replicator: " + replicator + ", queueId: " + queueId);
+        }
+      }
+    }
+    if (queueStorage.getAllPeersFromHFileRefsQueue().contains(peerId)) {
+      throw new DoNotRetryIOException("Undeleted queue for peer " + peerId + " in hfile-refs");
+    }
+  }
+
+  public void preAddPeer(String peerId, ReplicationPeerConfig peerConfig)
+      throws DoNotRetryIOException, ReplicationException {
+    if (peerId.contains("-")) {
+      throw new DoNotRetryIOException("Found invalid peer name: " + peerId);
+    }
+    checkPeerConfig(peerConfig);
+    if (peers.containsKey(peerId)) {
+      throw new DoNotRetryIOException("Replication peer " + peerId + " already exists");
+    }
+    // make sure that there is no queues with the same peer id. This may happen when we create a
+    // peer with the same id with a old deleted peer. If the replication queues for the old peer
+    // have not been cleaned up yet then we should not create the new peer, otherwise the old wal
+    // file may also be replicated.
+    checkQueuesDeleted(peerId);
+  }
+
+  private ReplicationPeerDescription checkPeerExists(String peerId) throws DoNotRetryIOException {
+    ReplicationPeerDescription desc = peers.get(peerId);
+    if (desc == null) {
+      throw new DoNotRetryIOException("Replication peer " + peerId + " does not exist");
+    }
+    return desc;
+  }
+
+  public void preRemovePeer(String peerId) throws DoNotRetryIOException {
+    checkPeerExists(peerId);
+  }
+
+  public void preEnablePeer(String peerId) throws DoNotRetryIOException {
+    ReplicationPeerDescription desc = checkPeerExists(peerId);
+    if (desc.isEnabled()) {
+      throw new DoNotRetryIOException("Replication peer " + peerId + " has already been enabled");
+    }
+  }
+
+  public void preDisablePeer(String peerId) throws DoNotRetryIOException {
+    ReplicationPeerDescription desc = checkPeerExists(peerId);
+    if (!desc.isEnabled()) {
+      throw new DoNotRetryIOException("Replication peer " + peerId + " has already been disabled");
+    }
+  }
+
+  public void preUpdatePeerConfig(String peerId, ReplicationPeerConfig peerConfig)
+      throws DoNotRetryIOException {
+    checkPeerConfig(peerConfig);
+    ReplicationPeerDescription desc = checkPeerExists(peerId);
+    ReplicationPeerConfig oldPeerConfig = desc.getPeerConfig();
+    if (!StringUtils.isBlank(peerConfig.getClusterKey()) &&
+      !peerConfig.getClusterKey().equals(oldPeerConfig.getClusterKey())) {
+      throw new DoNotRetryIOException(
+          "Changing the cluster key on an existing peer is not allowed. Existing key '" +
+            oldPeerConfig.getClusterKey() + "' for peer " + peerId + " does not match new key '" +
+            peerConfig.getClusterKey() + "'");
+    }
+
+    if (!StringUtils.isBlank(peerConfig.getReplicationEndpointImpl()) &&
+      !peerConfig.getReplicationEndpointImpl().equals(oldPeerConfig.getReplicationEndpointImpl())) {
+      throw new DoNotRetryIOException("Changing the replication endpoint implementation class " +
+        "on an existing peer is not allowed. Existing class '" +
+        oldPeerConfig.getReplicationEndpointImpl() + "' for peer " + peerId +
+        " does not match new class '" + peerConfig.getReplicationEndpointImpl() + "'");
+    }
+  }
+
+  private ReplicationPeerConfig copy(ReplicationPeerConfig peerConfig) {
+    ReplicationPeerConfig copiedPeerConfig = new ReplicationPeerConfig();
+    copiedPeerConfig.getConfiguration().putAll(peerConfig.getConfiguration());
+    copiedPeerConfig.getPeerData().putAll(peerConfig.getPeerData());
+    copiedPeerConfig.setTableCFsMap(peerConfig.getTableCFsMap());
+    copiedPeerConfig.setNamespaces(peerConfig.getNamespaces());
+    copiedPeerConfig.setExcludeTableCFsMap(peerConfig.getExcludeTableCFsMap());
+    copiedPeerConfig.setExcludeNamespaces(peerConfig.getExcludeNamespaces());
+    copiedPeerConfig.setBandwidth(peerConfig.getBandwidth());
+    copiedPeerConfig.setReplicateAllUserTables(peerConfig.replicateAllUserTables());
+    copiedPeerConfig.setClusterKey(peerConfig.getClusterKey());
+    copiedPeerConfig.setReplicationEndpointImpl(peerConfig.getReplicationEndpointImpl());
+    return copiedPeerConfig;
+  }
+
+  public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
+      throws ReplicationException {
+    if (peers.containsKey(peerId)) {
+      // this should be a retry, just return
+      return;
+    }
+    ReplicationPeerConfig copiedPeerConfig = copy(peerConfig);
+    peerStorage.addPeer(peerId, copiedPeerConfig, enabled);
+    peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, copiedPeerConfig));
+  }
+
+  public void removePeer(String peerId) throws ReplicationException {
+    if (!peers.containsKey(peerId)) {
+      // this should be a retry, just return
+      return;
+    }
+    peerStorage.removePeer(peerId);
+    peers.remove(peerId);
+  }
+
+  private void setPeerState(String peerId, boolean enabled) throws ReplicationException {
+    ReplicationPeerDescription desc = peers.get(peerId);
+    if (desc.isEnabled() == enabled) {
+      // this should be a retry, just return
+      return;
+    }
+    peerStorage.setPeerState(peerId, enabled);
+    peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, desc.getPeerConfig()));
+  }
+
+  public void enablePeer(String peerId) throws ReplicationException {
+    setPeerState(peerId, true);
+  }
+
+  public void disablePeer(String peerId) throws ReplicationException {
+    setPeerState(peerId, false);
+  }
+
+  public void updatePeerConfig(String peerId, ReplicationPeerConfig peerConfig)
+      throws ReplicationException {
+    // the checking rules are too complicated here so we give up checking whether this is a retry.
+    ReplicationPeerDescription desc = peers.get(peerId);
+    ReplicationPeerConfig oldPeerConfig = desc.getPeerConfig();
+    ReplicationPeerConfig newPeerConfig = copy(peerConfig);
+    // we need to use the new conf to overwrite the old one.
+    newPeerConfig.getConfiguration().putAll(oldPeerConfig.getConfiguration());
+    newPeerConfig.getConfiguration().putAll(peerConfig.getConfiguration());
+    newPeerConfig.getPeerData().putAll(oldPeerConfig.getPeerData());
+    newPeerConfig.getPeerData().putAll(peerConfig.getPeerData());
+
+    peerStorage.updatePeerConfig(peerId, newPeerConfig);
+    peers.put(peerId, new ReplicationPeerDescription(peerId, desc.isEnabled(), newPeerConfig));
+  }
+
+  public List<ReplicationPeerDescription> listPeers(Pattern pattern) {
+    if (pattern == null) {
+      return new ArrayList<>(peers.values());
+    }
+    return peers.values().stream().filter(r -> pattern.matcher(r.getPeerId()).matches())
+        .collect(Collectors.toList());
+  }
+
+  public Optional<ReplicationPeerConfig> getPeerConfig(String peerId) {
+    ReplicationPeerDescription desc = peers.get(peerId);
+    return desc != null ? Optional.of(desc.getPeerConfig()) : Optional.empty();
+  }
+
+  /**
+   * If replicate_all flag is true, it means all user tables will be replicated to peer cluster.
+   * Then allow config exclude namespaces or exclude table-cfs which can't be replicated to peer
+   * cluster.
+   * <p>
+   * If replicate_all flag is false, it means all user tables can't be replicated to peer cluster.
+   * Then allow to config namespaces or table-cfs which will be replicated to peer cluster.
+   */
+  private static void checkPeerConfig(ReplicationPeerConfig peerConfig)
+      throws DoNotRetryIOException {
+    if (peerConfig.replicateAllUserTables()) {
+      if ((peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty()) ||
+        (peerConfig.getTableCFsMap() != null && !peerConfig.getTableCFsMap().isEmpty())) {
+        throw new DoNotRetryIOException("Need clean namespaces or table-cfs config firstly " +
+          "when you want replicate all cluster");
+      }
+      checkNamespacesAndTableCfsConfigConflict(peerConfig.getExcludeNamespaces(),
+        peerConfig.getExcludeTableCFsMap());
+    } else {
+      if ((peerConfig.getExcludeNamespaces() != null &&
+        !peerConfig.getExcludeNamespaces().isEmpty()) ||
+        (peerConfig.getExcludeTableCFsMap() != null &&
+          !peerConfig.getExcludeTableCFsMap().isEmpty())) {
+        throw new DoNotRetryIOException(
+            "Need clean exclude-namespaces or exclude-table-cfs config firstly" +
+              " when replicate_all flag is false");
+      }
+      checkNamespacesAndTableCfsConfigConflict(peerConfig.getNamespaces(),
+        peerConfig.getTableCFsMap());
+    }
+    checkConfiguredWALEntryFilters(peerConfig);
+  }
+
+  /**
+   * Set a namespace in the peer config means that all tables in this namespace will be replicated
+   * to the peer cluster.
+   * <ol>
+   * <li>If peer config already has a namespace, then not allow set any table of this namespace to
+   * the peer config.</li>
+   * <li>If peer config already has a table, then not allow set this table's namespace to the peer
+   * config.</li>
+   * </ol>
+   * <p>
+   * Set a exclude namespace in the peer config means that all tables in this namespace can't be
+   * replicated to the peer cluster.
+   * <ol>
+   * <li>If peer config already has a exclude namespace, then not allow set any exclude table of
+   * this namespace to the peer config.</li>
+   * <li>If peer config already has a exclude table, then not allow set this table's namespace as a
+   * exclude namespace.</li>
+   * </ol>
+   */
+  private static void checkNamespacesAndTableCfsConfigConflict(Set<String> namespaces,
+      Map<TableName, ? extends Collection<String>> tableCfs) throws DoNotRetryIOException {
+    if (namespaces == null || namespaces.isEmpty()) {
+      return;
+    }
+    if (tableCfs == null || tableCfs.isEmpty()) {
+      return;
+    }
+    for (Map.Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
+      TableName table = entry.getKey();
+      if (namespaces.contains(table.getNamespaceAsString())) {
+        throw new DoNotRetryIOException("Table-cfs " + table + " is conflict with namespaces " +
+          table.getNamespaceAsString() + " in peer config");
+      }
+    }
+  }
+
+  private static void checkConfiguredWALEntryFilters(ReplicationPeerConfig peerConfig)
+      throws DoNotRetryIOException {
+    String filterCSV = peerConfig.getConfiguration()
+        .get(BaseReplicationEndpoint.REPLICATION_WALENTRYFILTER_CONFIG_KEY);
+    if (filterCSV != null && !filterCSV.isEmpty()) {
+      String[] filters = filterCSV.split(",");
+      for (String filter : filters) {
+        try {
+          Class.forName(filter).newInstance();
+        } catch (Exception e) {
+          throw new DoNotRetryIOException("Configured WALEntryFilter " + filter +
+            " could not be created. Failing add/update " + "peer operation.", e);
+        }
+      }
+    }
+  }
+
+  public static ReplicationPeerManager create(ZKWatcher zk, Configuration conf)
+      throws ReplicationException {
+    ReplicationPeerStorage peerStorage =
+      ReplicationStorageFactory.getReplicationPeerStorage(zk, conf);
+    ConcurrentMap<String, ReplicationPeerDescription> peers = new ConcurrentHashMap<>();
+    for (String peerId : peerStorage.listPeerIds()) {
+      Optional<ReplicationPeerConfig> peerConfig = peerStorage.getPeerConfig(peerId);
+      boolean enabled = peerStorage.isPeerEnabled(peerId);
+      peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, peerConfig.get()));
+    }
+    return new ReplicationPeerManager(peerStorage,
+        ReplicationStorageFactory.getReplicationQueueStorage(zk, conf), peers);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/557f5d4d/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
index d8154dc..a43532d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -59,12 +60,12 @@ public class UpdatePeerConfigProcedure extends ModifyPeerProcedure {
     if (cpHost != null) {
       cpHost.preUpdateReplicationPeerConfig(peerId, peerConfig);
     }
+    env.getReplicationPeerManager().preUpdatePeerConfig(peerId, peerConfig);
   }
 
   @Override
-  protected void updatePeerStorage(MasterProcedureEnv env)
-      throws IllegalArgumentException, Exception {
-    env.getReplicationManager().updatePeerConfig(peerId, peerConfig);
+  protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException {
+    env.getReplicationPeerManager().updatePeerConfig(peerId, peerConfig);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/557f5d4d/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
index 67c635b..2f3de9c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
@@ -17,6 +17,12 @@
  */
 package org.apache.hadoop.hbase.client.replication;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -31,32 +37,23 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ReplicationPeerNotFoundException;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.replication.ReplicationException;
-import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 /**
  * Unit testing of ReplicationAdmin
@@ -64,8 +61,6 @@ import static org.junit.Assert.fail;
 @Category({MediumTests.class, ClientTests.class})
 public class TestReplicationAdmin {
 
-  private static final Logger LOG =
-      LoggerFactory.getLogger(TestReplicationAdmin.class);
   private final static HBaseTestingUtility TEST_UTIL =
       new HBaseTestingUtility();
 
@@ -99,6 +94,21 @@ public class TestReplicationAdmin {
     TEST_UTIL.shutdownMiniCluster();
   }
 
+  @After
+  public void tearDown() throws Exception {
+    for (ReplicationPeerDescription desc : hbaseAdmin.listReplicationPeers()) {
+      hbaseAdmin.removeReplicationPeer(desc.getPeerId());
+    }
+    ReplicationQueueStorage queueStorage = ReplicationStorageFactory
+        .getReplicationQueueStorage(TEST_UTIL.getZooKeeperWatcher(), TEST_UTIL.getConfiguration());
+    for (ServerName serverName : queueStorage.getListOfReplicators()) {
+      for (String queue : queueStorage.getAllQueues(serverName)) {
+        queueStorage.removeQueue(serverName, queue);
+      }
+      queueStorage.removeReplicatorIfQueueIsEmpty(serverName);
+    }
+  }
+
   /**
    * Simple testing of adding and removing peers, basically shows that
    * all interactions with ZK work
@@ -185,32 +195,29 @@ public class TestReplicationAdmin {
     ReplicationPeerConfig rpc2 = new ReplicationPeerConfig();
     rpc2.setClusterKey(KEY_SECOND);
     Configuration conf = TEST_UTIL.getConfiguration();
-    ZKWatcher zkw = new ZKWatcher(conf, "Test HBaseAdmin", null);
-    ReplicationQueues repQueues =
-        ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, null, zkw));
-    repQueues.init("server1");
+    ReplicationQueueStorage queueStorage =
+      ReplicationStorageFactory.getReplicationQueueStorage(TEST_UTIL.getZooKeeperWatcher(), conf);
 
+    ServerName serverName = ServerName.valueOf("server1", 8000, 1234);
     // add queue for ID_ONE
-    repQueues.addLog(ID_ONE, "file1");
+    queueStorage.addWAL(serverName, ID_ONE, "file1");
     try {
       admin.addPeer(ID_ONE, rpc1, null);
       fail();
     } catch (Exception e) {
       // OK!
     }
-    repQueues.removeQueue(ID_ONE);
-    assertEquals(0, repQueues.getAllQueues().size());
+    queueStorage.removeQueue(serverName, ID_ONE);
+    assertEquals(0, queueStorage.getAllQueues(serverName).size());
 
     // add recovered queue for ID_ONE
-    repQueues.addLog(ID_ONE + "-server2", "file1");
+    queueStorage.addWAL(serverName, ID_ONE + "-server2", "file1");
     try {
       admin.addPeer(ID_ONE, rpc2, null);
       fail();
     } catch (Exception e) {
       // OK!
     }
-    repQueues.removeAllQueues();
-    zkw.close();
   }
 
   /**
@@ -406,7 +413,7 @@ public class TestReplicationAdmin {
       tableCFs.clear();
       tableCFs.put(tableName2, null);
       admin.removePeerTableCFs(ID_ONE, tableCFs);
-      assertTrue(false);
+      fail();
     } catch (ReplicationException e) {
     }
     tableCFs.clear();

http://git-wip-us.apache.org/repos/asf/hbase/blob/557f5d4d/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
index 540a67c..7196b7c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
@@ -19,6 +19,8 @@ package org.apache.hadoop.hbase.master;
 
 import static org.mockito.Mockito.mock;
 
+import com.google.protobuf.Service;
+
 import java.io.IOException;
 import java.util.List;
 
@@ -42,7 +44,7 @@ import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.locking.LockManager;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-import org.apache.hadoop.hbase.master.replication.ReplicationManager;
+import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
 import org.apache.hadoop.hbase.procedure2.LockedResource;
@@ -56,8 +58,6 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 
-import com.google.protobuf.Service;
-
 public class MockNoopMasterServices implements MasterServices, Server {
   private final Configuration conf;
   private final MetricsMaster metricsMaster;
@@ -462,7 +462,7 @@ public class MockNoopMasterServices implements MasterServices, Server {
   }
 
   @Override
-  public ProcedureEvent getInitializedEvent() {
+  public ProcedureEvent<?> getInitializedEvent() {
     return null;
   }
 
@@ -477,7 +477,7 @@ public class MockNoopMasterServices implements MasterServices, Server {
   }
 
   @Override
-  public ReplicationManager getReplicationManager() {
+  public ReplicationPeerManager getReplicationPeerManager() {
     return null;
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/557f5d4d/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
index 6119e43..eaace03 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
@@ -44,6 +44,8 @@ import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
+import org.apache.hadoop.hbase.replication.ReplicationException;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -270,7 +272,7 @@ public class TestMasterNoCluster {
 
       @Override
       void initializeZKBasedSystemTrackers() throws IOException, InterruptedException,
-          KeeperException, CoordinatedStateException {
+          KeeperException, CoordinatedStateException, ReplicationException {
         super.initializeZKBasedSystemTrackers();
         // Record a newer server in server manager at first
         getServerManager().recordNewServerWithLock(newServer,

http://git-wip-us.apache.org/repos/asf/hbase/blob/557f5d4d/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java
index 1675496..24bb4d0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * 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
@@ -47,9 +46,6 @@ public class TestReplicationDisableInactivePeer extends TestReplicationBase {
    */
   @Test(timeout = 600000)
   public void testDisableInactivePeer() throws Exception {
-
-    // enabling and shutdown the peer
-    admin.enablePeer("2");
     utility2.shutdownMiniHBaseCluster();
 
     byte[] rowkey = Bytes.toBytes("disable inactive peer");


[15/26] hbase git commit: HBASE-19496 Reusing the ByteBuffer in rpc layer corrupt the ServerLoad and RegionLoad

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
index 2fdb731..2a56e57 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerLoad.java
@@ -24,23 +24,26 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
-import java.util.TreeSet;
-
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Objects;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor;
+import java.util.stream.Collectors;
 import org.apache.hadoop.hbase.replication.ReplicationLoadSink;
 import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Strings;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Objects;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
 
 /**
  * This class is used for exporting current state of load on a RegionServer.
+ *
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+ *             Use {@link ServerMetrics} instead.
  */
 @InterfaceAudience.Public
-public class ServerLoad {
+@Deprecated
+public class ServerLoad implements ServerMetrics {
+  private final ServerMetrics metrics;
   private int stores = 0;
   private int storefiles = 0;
   private int storeUncompressedSizeMB = 0;
@@ -55,113 +58,200 @@ public class ServerLoad {
   private int totalStaticBloomSizeKB = 0;
   private long totalCompactingKVs = 0;
   private long currentCompactedKVs = 0;
-  private long reportTime = 0;
 
+  /**
+   * DONT USE this construction. It make a fake server name;
+   */
   @InterfaceAudience.Private
   public ServerLoad(ClusterStatusProtos.ServerLoad serverLoad) {
+    this(ServerName.valueOf("localhost,1,1"), serverLoad);
+  }
+
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="URF_UNREAD_PUBLIC_OR_PROTECTED_FIELD")
+  @InterfaceAudience.Private
+  public ServerLoad(ServerName name, ClusterStatusProtos.ServerLoad serverLoad) {
+    this(ServerMetricsBuilder.toServerMetrics(name, serverLoad));
     this.serverLoad = serverLoad;
-    this.reportTime = System.currentTimeMillis();
-    for (ClusterStatusProtos.RegionLoad rl: serverLoad.getRegionLoadsList()) {
-      stores += rl.getStores();
-      storefiles += rl.getStorefiles();
-      storeUncompressedSizeMB += rl.getStoreUncompressedSizeMB();
-      storefileSizeMB += rl.getStorefileSizeMB();
-      memstoreSizeMB += rl.getMemStoreSizeMB();
-      storefileIndexSizeKB += rl.getStorefileIndexSizeKB();
-      readRequestsCount += rl.getReadRequestsCount();
-      filteredReadRequestsCount += rl.getFilteredReadRequestsCount();
-      writeRequestsCount += rl.getWriteRequestsCount();
-      rootIndexSizeKB += rl.getRootIndexSizeKB();
-      totalStaticIndexSizeKB += rl.getTotalStaticIndexSizeKB();
-      totalStaticBloomSizeKB += rl.getTotalStaticBloomSizeKB();
-      totalCompactingKVs += rl.getTotalCompactingKVs();
-      currentCompactedKVs += rl.getCurrentCompactedKVs();
+  }
+
+  @InterfaceAudience.Private
+  public ServerLoad(ServerMetrics metrics) {
+    this.metrics = metrics;
+    this.serverLoad = ServerMetricsBuilder.toServerLoad(metrics);
+    for (RegionMetrics rl : metrics.getRegionMetrics().values()) {
+      stores += rl.getStoreCount();
+      storefiles += rl.getStoreFileCount();
+      storeUncompressedSizeMB += rl.getUncompressedStoreFileSize().get(Size.Unit.MEGABYTE);
+      storefileSizeMB += rl.getStoreFileSize().get(Size.Unit.MEGABYTE);
+      memstoreSizeMB += rl.getMemStoreSize().get(Size.Unit.MEGABYTE);
+      readRequestsCount += rl.getReadRequestCount();
+      filteredReadRequestsCount += rl.getFilteredReadRequestCount();
+      writeRequestsCount += rl.getWriteRequestCount();
+      storefileIndexSizeKB += rl.getStoreFileIndexSize().get(Size.Unit.KILOBYTE);
+      rootIndexSizeKB += rl.getStoreFileRootLevelIndexSize().get(Size.Unit.KILOBYTE);
+      totalStaticIndexSizeKB += rl.getStoreFileUncompressedDataIndexSize().get(Size.Unit.KILOBYTE);
+      totalStaticBloomSizeKB += rl.getBloomFilterSize().get(Size.Unit.KILOBYTE);
+      totalCompactingKVs += rl.getCompactingCellCount();
+      currentCompactedKVs += rl.getCompactedCellCount();
     }
   }
 
-  // NOTE: Function name cannot start with "get" because then an OpenDataException is thrown because
-  // HBaseProtos.ServerLoad cannot be converted to an open data type(see HBASE-5967).
-  /* @return the underlying ServerLoad protobuf object */
+  /**
+   * NOTE: Function name cannot start with "get" because then an OpenDataException is thrown because
+   * HBaseProtos.ServerLoad cannot be converted to an open data type(see HBASE-5967).
+   * @return the underlying ServerLoad protobuf object
+   * @deprecated DONT use this pb object since the byte array backed may be modified in rpc layer
+   */
   @InterfaceAudience.Private
+  @Deprecated
   public ClusterStatusProtos.ServerLoad obtainServerLoadPB() {
     return serverLoad;
   }
 
   protected ClusterStatusProtos.ServerLoad serverLoad;
 
-  /* @return number of requests  since last report. */
+  /**
+   * @return number of requests  since last report.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
+   *             Use {@link #getRequestCountPerSecond} instead.
+   */
+  @Deprecated
   public long getNumberOfRequests() {
-    return serverLoad.getNumberOfRequests();
+    return getRequestCountPerSecond();
   }
+
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             No flag in 2.0
+   */
+  @Deprecated
   public boolean hasNumberOfRequests() {
-    return serverLoad.hasNumberOfRequests();
+    return true;
   }
 
-  /* @return total Number of requests from the start of the region server. */
+  /**
+   * @return total Number of requests from the start of the region server.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
+   *             Use {@link #getRequestCount} instead.
+   */
+  @Deprecated
   public long getTotalNumberOfRequests() {
-    return serverLoad.getTotalNumberOfRequests();
+    return getRequestCount();
   }
+
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             No flag in 2.0
+   */
+  @Deprecated
   public boolean hasTotalNumberOfRequests() {
-    return serverLoad.hasTotalNumberOfRequests();
+    return true;
   }
 
-  /* @return the amount of used heap, in MB. */
+  /**
+   * @return the amount of used heap, in MB.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
+   *             Use {@link #getUsedHeapSize} instead.
+   */
+  @Deprecated
   public int getUsedHeapMB() {
-    return serverLoad.getUsedHeapMB();
+    return (int) getUsedHeapSize().get(Size.Unit.MEGABYTE);
   }
+
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             No flag in 2.0
+   */
+  @Deprecated
   public boolean hasUsedHeapMB() {
-    return serverLoad.hasUsedHeapMB();
+    return true;
   }
 
-  /* @return the maximum allowable size of the heap, in MB. */
+  /**
+   * @return the maximum allowable size of the heap, in MB.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getMaxHeapSize} instead.
+   */
+  @Deprecated
   public int getMaxHeapMB() {
-    return serverLoad.getMaxHeapMB();
+    return (int) getMaxHeapSize().get(Size.Unit.MEGABYTE);
   }
+
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             No flag in 2.0
+   */
+  @Deprecated
   public boolean hasMaxHeapMB() {
-    return serverLoad.hasMaxHeapMB();
+    return true;
   }
 
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getRegionMetrics} instead.
+   */
+  @Deprecated
   public int getStores() {
     return stores;
   }
 
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
+   *             Use {@link #getRegionMetrics} instead.
+   */
+  @Deprecated
   public int getStorefiles() {
     return storefiles;
   }
 
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getRegionMetrics} instead.
+   */
+  @Deprecated
   public int getStoreUncompressedSizeMB() {
     return storeUncompressedSizeMB;
   }
 
   /**
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   * Use {@link #getStorefileSizeMB()} instead.
+   *             Use {@link #getRegionMetrics} instead.
    */
   @Deprecated
   public int getStorefileSizeInMB() {
     return storefileSizeMB;
   }
 
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getRegionMetrics} instead.
+   */
+  @Deprecated
   public int getStorefileSizeMB() {
     return storefileSizeMB;
   }
 
   /**
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   * Use {@link #getMemStoreSizeMB()} instead.
+   *             Use {@link #getRegionMetrics} instead.
    */
   @Deprecated
   public int getMemstoreSizeInMB() {
     return memstoreSizeMB;
   }
 
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *     Use {@link #getRegionMetrics} instead.
+   */
+  @Deprecated
   public int getMemStoreSizeMB() {
     return memstoreSizeMB;
   }
 
   /**
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   * Use {@link #getStorefileIndexSizeKB()} instead.
+   *     Use {@link #getRegionMetrics} instead.
    */
   @Deprecated
   public int getStorefileIndexSizeInMB() {
@@ -169,71 +259,162 @@ public class ServerLoad {
     return (int) (getStorefileIndexSizeKB() >> 10);
   }
 
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *     Use {@link #getRegionMetrics} instead.
+   */
+  @Deprecated
   public long getStorefileIndexSizeKB() {
     return storefileIndexSizeKB;
   }
 
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *     Use {@link #getRegionMetrics} instead.
+   */
+  @Deprecated
   public long getReadRequestsCount() {
     return readRequestsCount;
   }
 
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *     Use {@link #getRegionMetrics} instead.
+   */
+  @Deprecated
   public long getFilteredReadRequestsCount() {
     return filteredReadRequestsCount;
   }
 
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *     Use {@link #getRegionMetrics} instead.
+   */
+  @Deprecated
   public long getWriteRequestsCount() {
     return writeRequestsCount;
   }
 
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *     Use {@link #getRegionMetrics} instead.
+   */
+  @Deprecated
   public int getRootIndexSizeKB() {
     return rootIndexSizeKB;
   }
 
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *     Use {@link #getRegionMetrics} instead.
+   */
+  @Deprecated
   public int getTotalStaticIndexSizeKB() {
     return totalStaticIndexSizeKB;
   }
 
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *     Use {@link #getRegionMetrics} instead.
+   */
+  @Deprecated
   public int getTotalStaticBloomSizeKB() {
     return totalStaticBloomSizeKB;
   }
 
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *     Use {@link #getRegionMetrics} instead.
+   */
+  @Deprecated
   public long getTotalCompactingKVs() {
     return totalCompactingKVs;
   }
 
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *     Use {@link #getRegionMetrics} instead.
+   */
+  @Deprecated
   public long getCurrentCompactedKVs() {
     return currentCompactedKVs;
   }
 
   /**
-   * @return the number of regions
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getRegionMetrics} instead.
    */
+  @Deprecated
   public int getNumberOfRegions() {
-    return serverLoad.getRegionLoadsCount();
+    return metrics.getRegionMetrics().size();
+  }
+
+  @Override
+  public ServerName getServerName() {
+    return metrics.getServerName();
   }
 
+  @Override
+  public long getRequestCountPerSecond() {
+    return metrics.getRequestCountPerSecond();
+  }
+
+  @Override
+  public long getRequestCount() {
+    return metrics.getRequestCount();
+  }
+
+  @Override
+  public Size getUsedHeapSize() {
+    return metrics.getUsedHeapSize();
+  }
+
+  @Override
+  public Size getMaxHeapSize() {
+    return metrics.getMaxHeapSize();
+  }
+
+  @Override
   public int getInfoServerPort() {
-    return serverLoad.getInfoServerPort();
+    return metrics.getInfoServerPort();
   }
 
   /**
    * Call directly from client such as hbase shell
    * @return the list of ReplicationLoadSource
    */
+  @Override
   public List<ReplicationLoadSource> getReplicationLoadSourceList() {
-    return ProtobufUtil.toReplicationLoadSourceList(serverLoad.getReplLoadSourceList());
+    return metrics.getReplicationLoadSourceList();
   }
 
   /**
    * Call directly from client such as hbase shell
    * @return ReplicationLoadSink
    */
+  @Override
   public ReplicationLoadSink getReplicationLoadSink() {
-    if (serverLoad.hasReplLoadSink()) {
-      return ProtobufUtil.toReplicationLoadSink(serverLoad.getReplLoadSink());
-    } else {
-      return null;
-    }
+    return metrics.getReplicationLoadSink();
+  }
+
+  @Override
+  public Map<byte[], RegionMetrics> getRegionMetrics() {
+    return metrics.getRegionMetrics();
+  }
+
+  @Override
+  public List<String> getCoprocessorNames() {
+    return metrics.getCoprocessorNames();
+  }
+
+  @Override
+  public long getReportTimestamp() {
+    return metrics.getReportTimestamp();
+  }
+
+  @Override
+  public long getLastReportTimestamp() {
+    return metrics.getLastReportTimestamp();
   }
 
   /**
@@ -243,8 +424,11 @@ public class ServerLoad {
    * interim, until we can figure out how to make rebalancing use all the info
    * available, we're just going to make load purely the number of regions.
    *
-   * @return load factor for this server
+   * @return load factor for this server.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getNumberOfRegions} instead.
    */
+  @Deprecated
   public int getLoad() {
     // See above comment
     // int load = numberOfRequests == 0 ? 1 : numberOfRequests;
@@ -254,53 +438,43 @@ public class ServerLoad {
   }
 
   /**
-   * @return region load metrics
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getRegionMetrics} instead.
    */
+  @Deprecated
   public Map<byte[], RegionLoad> getRegionsLoad() {
-    Map<byte[], RegionLoad> regionLoads =
-      new TreeMap<>(Bytes.BYTES_COMPARATOR);
-    for (ClusterStatusProtos.RegionLoad rl : serverLoad.getRegionLoadsList()) {
-      RegionLoad regionLoad = new RegionLoad(rl);
-      regionLoads.put(regionLoad.getName(), regionLoad);
-    }
-    return regionLoads;
+    return getRegionMetrics().entrySet().stream()
+        .collect(Collectors.toMap(Map.Entry::getKey, e -> new RegionLoad(e.getValue()),
+          (v1, v2) -> {
+            throw new RuntimeException("key collisions?");
+          }, () -> new TreeMap(Bytes.BYTES_COMPARATOR)));
   }
 
   /**
-   * Return the RegionServer-level coprocessors
-   * @return string array of loaded RegionServer-level coprocessors
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getCoprocessorNames} instead.
    */
+  @Deprecated
   public String[] getRegionServerCoprocessors() {
-    List<Coprocessor> list = obtainServerLoadPB().getCoprocessorsList();
-    String [] ret = new String[list.size()];
-    int i = 0;
-    for (Coprocessor elem : list) {
-      ret[i++] = elem.getName();
-    }
-
-    return ret;
+    return getCoprocessorNames().toArray(new String[getCoprocessorNames().size()]);
   }
 
   /**
-   * Return the RegionServer-level and Region-level coprocessors
-   * @return string array of loaded RegionServer-level and
-   *         Region-level coprocessors
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getCoprocessorNames} instead.
    */
+  @Deprecated
   public String[] getRsCoprocessors() {
-    // Need a set to remove duplicates, but since generated Coprocessor class
-    // is not Comparable, make it a Set<String> instead of Set<Coprocessor>
-    TreeSet<String> coprocessSet = new TreeSet<>();
-    for (Coprocessor coprocessor : obtainServerLoadPB().getCoprocessorsList()) {
-      coprocessSet.add(coprocessor.getName());
-    }
-    return coprocessSet.toArray(new String[coprocessSet.size()]);
+    return getRegionServerCoprocessors();
   }
 
   /**
-   * @return number of requests per second received since the last report
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getRequestCountPerSecond} instead.
    */
+  @Deprecated
   public double getRequestsPerSecond() {
-    return getNumberOfRequests();
+    return getRequestCountPerSecond();
   }
 
   /**
@@ -308,70 +482,73 @@ public class ServerLoad {
    */
   @Override
   public String toString() {
-     StringBuilder sb =
-        Strings.appendKeyValue(new StringBuilder(), "requestsPerSecond",
-          Double.valueOf(getRequestsPerSecond()));
+    StringBuilder sb = Strings.appendKeyValue(new StringBuilder(), "requestsPerSecond",
+      Double.valueOf(getRequestsPerSecond()));
     Strings.appendKeyValue(sb, "numberOfOnlineRegions", Integer.valueOf(getNumberOfRegions()));
-    sb = Strings.appendKeyValue(sb, "usedHeapMB", Integer.valueOf(this.getUsedHeapMB()));
-    sb = Strings.appendKeyValue(sb, "maxHeapMB", Integer.valueOf(getMaxHeapMB()));
-    sb = Strings.appendKeyValue(sb, "numberOfStores", Integer.valueOf(this.stores));
-    sb = Strings.appendKeyValue(sb, "numberOfStorefiles", Integer.valueOf(this.storefiles));
-    sb =
-        Strings.appendKeyValue(sb, "storefileUncompressedSizeMB",
-          Integer.valueOf(this.storeUncompressedSizeMB));
-    sb = Strings.appendKeyValue(sb, "storefileSizeMB", Integer.valueOf(this.storefileSizeMB));
+    Strings.appendKeyValue(sb, "usedHeapMB", Integer.valueOf(this.getUsedHeapMB()));
+    Strings.appendKeyValue(sb, "maxHeapMB", Integer.valueOf(getMaxHeapMB()));
+    Strings.appendKeyValue(sb, "numberOfStores", Integer.valueOf(this.stores));
+    Strings.appendKeyValue(sb, "numberOfStorefiles", Integer.valueOf(this.storefiles));
+    Strings.appendKeyValue(sb, "storefileUncompressedSizeMB",
+        Integer.valueOf(this.storeUncompressedSizeMB));
+    Strings.appendKeyValue(sb, "storefileSizeMB", Integer.valueOf(this.storefileSizeMB));
     if (this.storeUncompressedSizeMB != 0) {
-      sb =
-          Strings.appendKeyValue(
-            sb,
-            "compressionRatio",
-            String.format("%.4f", (float) this.storefileSizeMB
-                / (float) this.storeUncompressedSizeMB));
+      Strings.appendKeyValue(sb, "compressionRatio", String.format("%.4f",
+          (float) this.storefileSizeMB / (float) this.storeUncompressedSizeMB));
     }
-    sb = Strings.appendKeyValue(sb, "memstoreSizeMB", Integer.valueOf(this.memstoreSizeMB));
-    sb =
-        Strings.appendKeyValue(sb, "storefileIndexSizeKB",
-          Long.valueOf(this.storefileIndexSizeKB));
-    sb = Strings.appendKeyValue(sb, "readRequestsCount", Long.valueOf(this.readRequestsCount));
-    sb = Strings.appendKeyValue(sb, "filteredReadRequestsCount",
-      Long.valueOf(this.filteredReadRequestsCount));
-    sb = Strings.appendKeyValue(sb, "writeRequestsCount", Long.valueOf(this.writeRequestsCount));
-    sb = Strings.appendKeyValue(sb, "rootIndexSizeKB", Integer.valueOf(this.rootIndexSizeKB));
-    sb =
-        Strings.appendKeyValue(sb, "totalStaticIndexSizeKB",
-          Integer.valueOf(this.totalStaticIndexSizeKB));
-    sb =
-        Strings.appendKeyValue(sb, "totalStaticBloomSizeKB",
-          Integer.valueOf(this.totalStaticBloomSizeKB));
-    sb = Strings.appendKeyValue(sb, "totalCompactingKVs", Long.valueOf(this.totalCompactingKVs));
-    sb = Strings.appendKeyValue(sb, "currentCompactedKVs", Long.valueOf(this.currentCompactedKVs));
+    Strings.appendKeyValue(sb, "memstoreSizeMB", Integer.valueOf(this.memstoreSizeMB));
+    Strings.appendKeyValue(sb, "storefileIndexSizeKB",
+        Long.valueOf(this.storefileIndexSizeKB));
+    Strings.appendKeyValue(sb, "readRequestsCount", Long.valueOf(this.readRequestsCount));
+    Strings.appendKeyValue(sb, "filteredReadRequestsCount",
+        Long.valueOf(this.filteredReadRequestsCount));
+    Strings.appendKeyValue(sb, "writeRequestsCount", Long.valueOf(this.writeRequestsCount));
+    Strings.appendKeyValue(sb, "rootIndexSizeKB", Integer.valueOf(this.rootIndexSizeKB));
+    Strings.appendKeyValue(sb, "totalStaticIndexSizeKB",
+        Integer.valueOf(this.totalStaticIndexSizeKB));
+    Strings.appendKeyValue(sb, "totalStaticBloomSizeKB",
+        Integer.valueOf(this.totalStaticBloomSizeKB));
+    Strings.appendKeyValue(sb, "totalCompactingKVs", Long.valueOf(this.totalCompactingKVs));
+    Strings.appendKeyValue(sb, "currentCompactedKVs", Long.valueOf(this.currentCompactedKVs));
     float compactionProgressPct = Float.NaN;
     if (this.totalCompactingKVs > 0) {
       compactionProgressPct =
           Float.valueOf((float) this.currentCompactedKVs / this.totalCompactingKVs);
     }
-    sb = Strings.appendKeyValue(sb, "compactionProgressPct", compactionProgressPct);
+    Strings.appendKeyValue(sb, "compactionProgressPct", compactionProgressPct);
 
     String[] coprocessorStrings = getRsCoprocessors();
     if (coprocessorStrings != null) {
-      sb = Strings.appendKeyValue(sb, "coprocessors", Arrays.toString(coprocessorStrings));
+      Strings.appendKeyValue(sb, "coprocessors", Arrays.toString(coprocessorStrings));
     }
     return sb.toString();
   }
 
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link ServerMetricsBuilder#of(ServerName)} instead.
+   */
+  @Deprecated
   public static final ServerLoad EMPTY_SERVERLOAD =
-    new ServerLoad(ClusterStatusProtos.ServerLoad.newBuilder().build());
+      new ServerLoad(ServerName.valueOf("localhost,1,1"),
+          ClusterStatusProtos.ServerLoad.newBuilder().build());
 
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getReportTimestamp} instead.
+   */
+  @Deprecated
   public long getReportTime() {
-    return reportTime;
+    return getReportTimestamp();
   }
 
   @Override
   public int hashCode() {
-    return Objects.hashCode(stores, storefiles, storeUncompressedSizeMB,
-      storefileSizeMB, memstoreSizeMB, storefileIndexSizeKB, readRequestsCount,
-      filteredReadRequestsCount, writeRequestsCount, rootIndexSizeKB, totalStaticIndexSizeKB,
-      totalStaticBloomSizeKB, totalCompactingKVs, currentCompactedKVs);
+    return Objects
+        .hashCode(stores, storefiles, storeUncompressedSizeMB, storefileSizeMB, memstoreSizeMB,
+            storefileIndexSizeKB, readRequestsCount, filteredReadRequestsCount, writeRequestsCount,
+            rootIndexSizeKB, totalStaticIndexSizeKB, totalStaticBloomSizeKB, totalCompactingKVs,
+            currentCompactedKVs);
   }
 
   @Override
@@ -379,16 +556,17 @@ public class ServerLoad {
     if (other == this) return true;
     if (other instanceof ServerLoad) {
       ServerLoad sl = ((ServerLoad) other);
-      return stores == sl.stores && storefiles == sl.storefiles &&
-        storeUncompressedSizeMB == sl.storeUncompressedSizeMB &&
-        storefileSizeMB == sl.storefileSizeMB && memstoreSizeMB == sl.memstoreSizeMB &&
-        storefileIndexSizeKB == sl.storefileIndexSizeKB && readRequestsCount == sl.readRequestsCount &&
-        filteredReadRequestsCount == sl.filteredReadRequestsCount &&
-        writeRequestsCount == sl.writeRequestsCount && rootIndexSizeKB == sl.rootIndexSizeKB &&
-        totalStaticIndexSizeKB == sl.totalStaticIndexSizeKB &&
-        totalStaticBloomSizeKB == sl.totalStaticBloomSizeKB &&
-        totalCompactingKVs == sl.totalCompactingKVs &&
-        currentCompactedKVs == sl.currentCompactedKVs;
+      return stores == sl.stores && storefiles == sl.storefiles
+          && storeUncompressedSizeMB == sl.storeUncompressedSizeMB
+          && storefileSizeMB == sl.storefileSizeMB && memstoreSizeMB == sl.memstoreSizeMB
+          && storefileIndexSizeKB == sl.storefileIndexSizeKB
+          && readRequestsCount == sl.readRequestsCount
+          && filteredReadRequestsCount == sl.filteredReadRequestsCount
+          && writeRequestsCount == sl.writeRequestsCount && rootIndexSizeKB == sl.rootIndexSizeKB
+          && totalStaticIndexSizeKB == sl.totalStaticIndexSizeKB
+          && totalStaticBloomSizeKB == sl.totalStaticBloomSizeKB
+          && totalCompactingKVs == sl.totalCompactingKVs
+          && currentCompactedKVs == sl.currentCompactedKVs;
     }
     return false;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerMetrics.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerMetrics.java
new file mode 100644
index 0000000..1ef3126
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerMetrics.java
@@ -0,0 +1,90 @@
+/**
+ * Copyright The Apache Software Foundation
+ * 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;
+
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.util.List;
+import java.util.Map;
+import org.apache.hadoop.hbase.replication.ReplicationLoadSink;
+import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * This class is used for exporting current state of load on a RegionServer.
+ */
+@InterfaceAudience.Public
+public interface ServerMetrics {
+
+  ServerName getServerName();
+  /**
+   * @return the number of requests per second.
+   */
+  long getRequestCountPerSecond();
+
+  /**
+   * @return total Number of requests from the start of the region server.
+   */
+  long getRequestCount();
+
+  /**
+   * @return the amount of used heap
+   */
+  Size getUsedHeapSize();
+
+  /**
+   * @return the maximum allowable size of the heap
+   */
+  Size getMaxHeapSize();
+
+  int getInfoServerPort();
+
+  /**
+   * Call directly from client such as hbase shell
+   * @return the list of ReplicationLoadSource
+   */
+  List<ReplicationLoadSource> getReplicationLoadSourceList();
+
+  /**
+   * Call directly from client such as hbase shell
+   * @return ReplicationLoadSink
+   */
+  @Nullable
+  ReplicationLoadSink getReplicationLoadSink();
+
+  /**
+   * @return region load metrics
+   */
+  Map<byte[], RegionMetrics> getRegionMetrics();
+
+  /**
+   * Return the RegionServer-level and Region-level coprocessors
+   * @return string list of loaded RegionServer-level and Region-level coprocessors
+   */
+  List<String> getCoprocessorNames();
+
+  /**
+   * @return the timestamp (server side) of generating this metrics
+   */
+  long getReportTimestamp();
+
+  /**
+   * @return the last timestamp (server side) of generating this metrics
+   */
+  long getLastReportTimestamp();
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerMetricsBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerMetricsBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerMetricsBuilder.java
new file mode 100644
index 0000000..e501c43
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ServerMetricsBuilder.java
@@ -0,0 +1,352 @@
+/**
+ * Copyright The Apache Software Foundation
+ * 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;
+
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hbase.replication.ReplicationLoadSink;
+import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Strings;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+
+@InterfaceAudience.Private
+public final class ServerMetricsBuilder {
+
+  /**
+   * @param sn the server name
+   * @return a empty metrics
+   */
+  public static ServerMetrics of(ServerName sn) {
+    return newBuilder(sn).build();
+  }
+
+  public static ServerMetrics toServerMetrics(ClusterStatusProtos.LiveServerInfo serverInfo) {
+    return toServerMetrics(ProtobufUtil.toServerName(serverInfo.getServer()),
+        serverInfo.getServerLoad());
+  }
+
+  public static ServerMetrics toServerMetrics(ServerName serverName,
+      ClusterStatusProtos.ServerLoad serverLoadPB) {
+    return ServerMetricsBuilder.newBuilder(serverName)
+        .setRequestCountPerSecond(serverLoadPB.getNumberOfRequests())
+        .setRequestCount(serverLoadPB.getTotalNumberOfRequests())
+        .setInfoServerPort(serverLoadPB.getInfoServerPort())
+        .setMaxHeapSize(new Size(serverLoadPB.getMaxHeapMB(), Size.Unit.MEGABYTE))
+        .setUsedHeapSize(new Size(serverLoadPB.getUsedHeapMB(), Size.Unit.MEGABYTE))
+        .setCoprocessorNames(serverLoadPB.getCoprocessorsList().stream()
+            .map(HBaseProtos.Coprocessor::getName).collect(Collectors.toList()))
+        .setRegionMetrics(serverLoadPB.getRegionLoadsList().stream()
+            .map(RegionMetricsBuilder::toRegionMetrics)
+            .collect(Collectors.toList()))
+        .setReplicationLoadSources(serverLoadPB.getReplLoadSourceList().stream()
+            .map(ProtobufUtil::toReplicationLoadSource)
+            .collect(Collectors.toList()))
+        .setReplicationLoadSink(serverLoadPB.hasReplLoadSink() ?
+            ProtobufUtil.toReplicationLoadSink(serverLoadPB.getReplLoadSink()) : null)
+        .setReportTimestamp(serverLoadPB.getReportEndTime())
+        .setLastReportTimestamp(serverLoadPB.getReportStartTime())
+        .build();
+  }
+
+  public static List<HBaseProtos.Coprocessor> toCoprocessor(List<String> names) {
+    return names.stream()
+        .map(n -> HBaseProtos.Coprocessor.newBuilder().setName(n).build())
+        .collect(Collectors.toList());
+  }
+
+  public static ClusterStatusProtos.ServerLoad toServerLoad(ServerMetrics metrics) {
+    ClusterStatusProtos.ServerLoad.Builder builder = ClusterStatusProtos.ServerLoad.newBuilder()
+        .setNumberOfRequests(metrics.getRequestCountPerSecond())
+        .setTotalNumberOfRequests(metrics.getRequestCount())
+        .setInfoServerPort(metrics.getInfoServerPort())
+        .setMaxHeapMB((int) metrics.getMaxHeapSize().get(Size.Unit.MEGABYTE))
+        .setUsedHeapMB((int) metrics.getUsedHeapSize().get(Size.Unit.MEGABYTE))
+        .addAllCoprocessors(toCoprocessor(metrics.getCoprocessorNames()))
+        .addAllRegionLoads(metrics.getRegionMetrics().values().stream()
+            .map(RegionMetricsBuilder::toRegionLoad)
+            .collect(Collectors.toList()))
+        .addAllReplLoadSource(metrics.getReplicationLoadSourceList().stream()
+            .map(ProtobufUtil::toReplicationLoadSource)
+            .collect(Collectors.toList()))
+        .setReportStartTime(metrics.getLastReportTimestamp())
+        .setReportEndTime(metrics.getReportTimestamp());
+    if (metrics.getReplicationLoadSink() != null) {
+      builder.setReplLoadSink(ProtobufUtil.toReplicationLoadSink(
+          metrics.getReplicationLoadSink()));
+    }
+    return builder.build();
+  }
+
+  public static ServerMetricsBuilder newBuilder(ServerName sn) {
+    return new ServerMetricsBuilder(sn);
+  }
+
+  private final ServerName serverName;
+  private long requestCountPerSecond;
+  private long requestCount;
+  private Size usedHeapSize = Size.ZERO;
+  private Size maxHeapSize = Size.ZERO;
+  private int infoServerPort;
+  private List<ReplicationLoadSource> sources = Collections.emptyList();
+  @Nullable
+  private ReplicationLoadSink sink = null;
+  private final Map<byte[], RegionMetrics> regionStatus = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+  private List<String> coprocessorNames = Collections.emptyList();
+  private long reportTimestamp = System.currentTimeMillis();
+  private long lastReportTimestamp = 0;
+  private ServerMetricsBuilder(ServerName serverName) {
+    this.serverName = serverName;
+  }
+
+  public ServerMetricsBuilder setRequestCountPerSecond(long value) {
+    this.requestCountPerSecond = value;
+    return this;
+  }
+
+  public ServerMetricsBuilder setRequestCount(long value) {
+    this.requestCount = value;
+    return this;
+  }
+
+  public ServerMetricsBuilder setUsedHeapSize(Size value) {
+    this.usedHeapSize = value;
+    return this;
+  }
+
+  public ServerMetricsBuilder setMaxHeapSize(Size value) {
+    this.maxHeapSize = value;
+    return this;
+  }
+
+  public ServerMetricsBuilder setInfoServerPort(int value) {
+    this.infoServerPort = value;
+    return this;
+  }
+
+  public ServerMetricsBuilder setReplicationLoadSources(List<ReplicationLoadSource> value) {
+    this.sources = value;
+    return this;
+  }
+
+  public ServerMetricsBuilder setReplicationLoadSink(ReplicationLoadSink value) {
+    this.sink = value;
+    return this;
+  }
+
+  public ServerMetricsBuilder setRegionMetrics(List<RegionMetrics> value) {
+    value.forEach(v -> this.regionStatus.put(v.getRegionName(), v));
+    return this;
+  }
+
+  public ServerMetricsBuilder setCoprocessorNames(List<String> value) {
+    this.coprocessorNames = value;
+    return this;
+  }
+
+  public ServerMetricsBuilder setReportTimestamp(long value) {
+    this.reportTimestamp = value;
+    return this;
+  }
+
+  public ServerMetricsBuilder setLastReportTimestamp(long value) {
+    this.lastReportTimestamp = value;
+    return this;
+  }
+
+  public ServerMetrics build() {
+    return new ServerMetricsImpl(
+        serverName,
+        requestCountPerSecond,
+        requestCount,
+        usedHeapSize,
+        maxHeapSize,
+        infoServerPort,
+        sources,
+        sink,
+        regionStatus,
+        coprocessorNames,
+        reportTimestamp,
+        lastReportTimestamp);
+  }
+
+  private static class ServerMetricsImpl implements ServerMetrics {
+    private final ServerName serverName;
+    private final long requestCountPerSecond;
+    private final long requestCount;
+    private final Size usedHeapSize;
+    private final Size maxHeapSize;
+    private final int infoServerPort;
+    private final List<ReplicationLoadSource> sources;
+    @Nullable
+    private final ReplicationLoadSink sink;
+    private final Map<byte[], RegionMetrics> regionStatus;
+    private final List<String> coprocessorNames;
+    private final long reportTimestamp;
+    private final long lastReportTimestamp;
+
+    ServerMetricsImpl(ServerName serverName, long requestCountPerSecond, long requestCount,
+      Size usedHeapSize, Size maxHeapSize, int infoServerPort, List<ReplicationLoadSource> sources,
+      ReplicationLoadSink sink, Map<byte[], RegionMetrics> regionStatus,
+      List<String> coprocessorNames, long reportTimestamp, long lastReportTimestamp) {
+      this.serverName = Preconditions.checkNotNull(serverName);
+      this.requestCountPerSecond = requestCountPerSecond;
+      this.requestCount = requestCount;
+      this.usedHeapSize = Preconditions.checkNotNull(usedHeapSize);
+      this.maxHeapSize = Preconditions.checkNotNull(maxHeapSize);
+      this.infoServerPort = infoServerPort;
+      this.sources = Preconditions.checkNotNull(sources);
+      this.sink = sink;
+      this.regionStatus = Preconditions.checkNotNull(regionStatus);
+      this.coprocessorNames =Preconditions.checkNotNull(coprocessorNames);
+      this.reportTimestamp = reportTimestamp;
+      this.lastReportTimestamp = lastReportTimestamp;
+    }
+
+    @Override
+    public ServerName getServerName() {
+      return serverName;
+    }
+    @Override
+    public long getRequestCountPerSecond() {
+      return requestCountPerSecond;
+    }
+
+    @Override
+    public long getRequestCount() {
+      return requestCount;
+    }
+
+    @Override
+    public Size getUsedHeapSize() {
+      return usedHeapSize;
+    }
+
+    @Override
+    public Size getMaxHeapSize() {
+      return maxHeapSize;
+    }
+
+    @Override
+    public int getInfoServerPort() {
+      return infoServerPort;
+    }
+
+    @Override
+    public List<ReplicationLoadSource> getReplicationLoadSourceList() {
+      return Collections.unmodifiableList(sources);
+    }
+
+    @Override
+    public ReplicationLoadSink getReplicationLoadSink() {
+      return sink;
+    }
+
+    @Override
+    public Map<byte[], RegionMetrics> getRegionMetrics() {
+      return Collections.unmodifiableMap(regionStatus);
+    }
+
+    @Override
+    public List<String> getCoprocessorNames() {
+      return Collections.unmodifiableList(coprocessorNames);
+    }
+
+    @Override
+    public long getReportTimestamp() {
+      return reportTimestamp;
+    }
+
+    @Override
+    public long getLastReportTimestamp() {
+      return lastReportTimestamp;
+    }
+
+    @Override
+    public String toString() {
+      int storeCount = 0;
+      int storeFileCount = 0;
+      long uncompressedStoreFileSizeMB = 0;
+      long storeFileSizeMB = 0;
+      long memStoreSizeMB = 0;
+      long storefileIndexSizeKB = 0;
+      long rootLevelIndexSizeKB = 0;
+      long readRequestsCount = 0;
+      long writeRequestsCount = 0;
+      long filteredReadRequestsCount = 0;
+      long bloomFilterSizeMB = 0;
+      long compactingCellCount = 0;
+      long compactedCellCount = 0;
+      for (RegionMetrics r : getRegionMetrics().values()) {
+        storeCount += r.getStoreCount();
+        storeFileCount += r.getStoreFileCount();
+        uncompressedStoreFileSizeMB += r.getUncompressedStoreFileSize().get(Size.Unit.MEGABYTE);
+        storeFileSizeMB += r.getStoreFileSize().get(Size.Unit.MEGABYTE);
+        memStoreSizeMB += r.getMemStoreSize().get(Size.Unit.MEGABYTE);
+        storefileIndexSizeKB += r.getStoreFileUncompressedDataIndexSize().get(Size.Unit.KILOBYTE);
+        readRequestsCount += r.getReadRequestCount();
+        writeRequestsCount += r.getWriteRequestCount();
+        filteredReadRequestsCount += r.getFilteredReadRequestCount();
+        rootLevelIndexSizeKB += r.getStoreFileRootLevelIndexSize().get(Size.Unit.KILOBYTE);
+        bloomFilterSizeMB += r.getBloomFilterSize().get(Size.Unit.MEGABYTE);
+        compactedCellCount += r.getCompactedCellCount();
+        compactingCellCount += r.getCompactingCellCount();
+      }
+      StringBuilder sb = Strings.appendKeyValue(new StringBuilder(), "requestsPerSecond",
+            Double.valueOf(getRequestCountPerSecond()));
+      Strings.appendKeyValue(sb, "numberOfOnlineRegions",
+          Integer.valueOf(getRegionMetrics().size()));
+      Strings.appendKeyValue(sb, "usedHeapMB", getUsedHeapSize());
+      Strings.appendKeyValue(sb, "maxHeapMB", getMaxHeapSize());
+      Strings.appendKeyValue(sb, "numberOfStores", storeCount);
+      Strings.appendKeyValue(sb, "numberOfStorefiles", storeFileCount);
+      Strings.appendKeyValue(sb, "storefileUncompressedSizeMB", uncompressedStoreFileSizeMB);
+      Strings.appendKeyValue(sb, "storefileSizeMB", storeFileSizeMB);
+      if (uncompressedStoreFileSizeMB != 0) {
+        Strings.appendKeyValue(sb, "compressionRatio", String.format("%.4f",
+            (float) storeFileSizeMB / (float) uncompressedStoreFileSizeMB));
+      }
+      Strings.appendKeyValue(sb, "memstoreSizeMB", memStoreSizeMB);
+      Strings.appendKeyValue(sb, "readRequestsCount", readRequestsCount);
+      Strings.appendKeyValue(sb, "filteredReadRequestsCount", filteredReadRequestsCount);
+      Strings.appendKeyValue(sb, "writeRequestsCount", writeRequestsCount);
+      Strings.appendKeyValue(sb, "rootIndexSizeKB", rootLevelIndexSizeKB);
+      Strings.appendKeyValue(sb, "totalStaticIndexSizeKB", storefileIndexSizeKB);
+      Strings.appendKeyValue(sb, "totalStaticBloomSizeKB", bloomFilterSizeMB);
+      Strings.appendKeyValue(sb, "totalCompactingKVs", compactingCellCount);
+      Strings.appendKeyValue(sb, "currentCompactedKVs", compactedCellCount);
+      float compactionProgressPct = Float.NaN;
+      if (compactingCellCount > 0) {
+        compactionProgressPct =
+            Float.valueOf((float) compactedCellCount / compactingCellCount);
+      }
+      Strings.appendKeyValue(sb, "compactionProgressPct", compactionProgressPct);
+      Strings.appendKeyValue(sb, "coprocessors", getCoprocessorNames());
+      return sb.toString();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-client/src/main/java/org/apache/hadoop/hbase/Size.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/Size.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/Size.java
new file mode 100644
index 0000000..87d7554
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/Size.java
@@ -0,0 +1,158 @@
+/**
+ * Copyright The Apache Software Foundation
+ * 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;
+
+import java.math.BigDecimal;
+import java.util.Objects;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
+
+/**
+ * It is used to represent the size with different units.
+ * This class doesn't serve for the precise computation.
+ */
+@InterfaceAudience.Public
+public final class Size implements Comparable<Size> {
+  public static final Size ZERO = new Size(0, Unit.KILOBYTE);
+  private static final BigDecimal SCALE_BASE = BigDecimal.valueOf(1024D);
+
+  public enum Unit {
+    // keep the room to add more units for HBase 10.x
+    PETABYTE(100, "PB"),
+    TERABYTE(99, "TB"),
+    GIGABYTE(98, "GB"),
+    MEGABYTE(97, "MB"),
+    KILOBYTE(96, "KB"),
+    BYTE(95, "B");
+    private final int orderOfSize;
+    private final String simpleName;
+
+    Unit(int orderOfSize, String simpleName) {
+      this.orderOfSize = orderOfSize;
+      this.simpleName = simpleName;
+    }
+
+    public int getOrderOfSize() {
+      return orderOfSize;
+    }
+
+    public String getSimpleName() {
+      return simpleName;
+    }
+  }
+
+  private final double value;
+  private final Unit unit;
+
+  public Size(double value, Unit unit) {
+    if (value < 0) {
+      throw new IllegalArgumentException("The value:" + value + " can't be negative");
+    }
+    this.value = value;
+    this.unit = Preconditions.checkNotNull(unit);
+  }
+
+  /**
+   * @return size unit
+   */
+  public Unit getUnit() {
+    return unit;
+  }
+
+  /**
+   * get the value
+   */
+  public long getLongValue() {
+    return (long) value;
+  }
+
+  /**
+   * get the value
+   */
+  public double get() {
+    return value;
+  }
+
+  /**
+   * get the value which is converted to specified unit.
+   *
+   * @param unit size unit
+   * @return the converted value
+   */
+  public double get(Unit unit) {
+    if (value == 0) {
+      return value;
+    }
+    int diff = this.unit.getOrderOfSize() - unit.getOrderOfSize();
+    if (diff == 0) {
+      return value;
+    }
+
+    BigDecimal rval = BigDecimal.valueOf(value);
+    for (int i = 0; i != Math.abs(diff); ++i) {
+      rval = diff > 0 ? rval.multiply(SCALE_BASE) : rval.divide(SCALE_BASE);
+    }
+    return rval.doubleValue();
+  }
+
+  @Override
+  public int compareTo(Size other) {
+    int diff = unit.getOrderOfSize() - other.unit.getOrderOfSize();
+    if (diff == 0) {
+      return Double.compare(value, other.value);
+    }
+
+    BigDecimal thisValue = BigDecimal.valueOf(value);
+    BigDecimal otherValue = BigDecimal.valueOf(other.value);
+    if (diff > 0) {
+      for (int i = 0; i != Math.abs(diff); ++i) {
+        thisValue = thisValue.multiply(SCALE_BASE);
+      }
+    } else {
+      for (int i = 0; i != Math.abs(diff); ++i) {
+        otherValue = otherValue.multiply(SCALE_BASE);
+      }
+    }
+    return thisValue.compareTo(otherValue);
+  }
+
+  @Override
+  public String toString() {
+    return value + unit.getSimpleName();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == null) {
+      return false;
+    }
+    if (obj == this) {
+      return true;
+    }
+    if (obj instanceof Size) {
+      return compareTo((Size)obj) == 0;
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(value, unit);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index 0567e8e..de022d3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -28,12 +28,11 @@ import java.util.Set;
 import java.util.TreeMap;
 import java.util.concurrent.Future;
 import java.util.regex.Pattern;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.CacheEvictionStats;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.ClusterStatus;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
@@ -43,7 +42,6 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.replication.TableCFs;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
@@ -60,6 +58,7 @@ import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
 import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * The administrative API for HBase. Obtain an instance from {@link Connection#getAdmin()} and

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
index 2ae51ac..c01d5fa 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.client;
 
 import com.google.protobuf.RpcChannel;
-
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.EnumSet;
@@ -29,9 +28,8 @@ import java.util.Set;
 import java.util.concurrent.CompletableFuture;
 import java.util.function.Function;
 import java.util.regex.Pattern;
-
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.ClusterStatus;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.RegionLoad;
 import org.apache.hadoop.hbase.ServerName;

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
index 0f0679d..09fdeff 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.client;
 
 import com.google.protobuf.RpcChannel;
-
 import java.util.Collection;
 import java.util.EnumSet;
 import java.util.List;
@@ -29,9 +28,8 @@ import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutorService;
 import java.util.function.Function;
 import java.util.regex.Pattern;
-
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.ClusterStatus;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.RegionLoad;
 import org.apache.hadoop.hbase.ServerName;

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java
index 5ff1b67..0c53ee4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java
@@ -264,7 +264,7 @@ class ClusterStatusListener implements Closeable {
         ByteBufInputStream bis = new ByteBufInputStream(dp.content());
         try {
           ClusterStatusProtos.ClusterStatus csp = ClusterStatusProtos.ClusterStatus.parseFrom(bis);
-          ClusterStatus ncs = ProtobufUtil.convert(csp);
+          ClusterStatus ncs = ProtobufUtil.toClusterStatus(csp);
           receive(ncs);
         } finally {
           bis.close();

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index 600ee69..bbcc825 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.client;
 import com.google.protobuf.Descriptors;
 import com.google.protobuf.Message;
 import com.google.protobuf.RpcController;
-
 import java.io.Closeable;
 import java.io.IOException;
 import java.io.InterruptedIOException;
@@ -45,13 +44,12 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.CacheEvictionStats;
 import org.apache.hadoop.hbase.CacheEvictionStatsBuilder;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.ClusterStatus;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
@@ -103,6 +101,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -2086,7 +2085,7 @@ public class HBaseAdmin implements Admin {
       @Override
       protected ClusterStatus rpcCall() throws Exception {
         GetClusterStatusRequest req = RequestConverter.buildGetClusterStatusRequest(options);
-        return ProtobufUtil.convert(
+        return ProtobufUtil.toClusterStatus(
           master.getClusterStatus(getRpcController(), req).getClusterStatus());
       }
     });

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index bb427b1..5e1c654 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -23,7 +23,6 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.protobuf.Message;
 import com.google.protobuf.RpcChannel;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -43,12 +42,11 @@ import java.util.function.Function;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
-
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.AsyncMetaTableAccessor;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.ClusterStatus;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.MetaTableAccessor;
@@ -87,6 +85,7 @@ import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
 import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
 import org.apache.hadoop.hbase.shaded.io.netty.util.Timeout;
@@ -2617,7 +2616,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
               .<GetClusterStatusRequest, GetClusterStatusResponse, ClusterStatus> call(controller,
                 stub, RequestConverter.buildGetClusterStatusRequest(options),
                 (s, c, req, done) -> s.getClusterStatus(c, req, done),
-                resp -> ProtobufUtil.convert(resp.getClusterStatus()))).call();
+                resp -> ProtobufUtil.toClusterStatus(resp.getClusterStatus()))).call();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSink.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSink.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSink.java
index 7ae0e0f..de4d4ca 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSink.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSink.java
@@ -15,11 +15,13 @@ import org.apache.yetus.audience.InterfaceAudience;
 /**
  * A HBase ReplicationLoad to present MetricsSink information
  */
-@InterfaceAudience.Private
+@InterfaceAudience.Public
 public class ReplicationLoadSink {
-  private long ageOfLastAppliedOp;
-  private long timeStampsOfLastAppliedOp;
+  private final long ageOfLastAppliedOp;
+  private final long timeStampsOfLastAppliedOp;
 
+  // TODO: add the builder for this class
+  @InterfaceAudience.Private
   public ReplicationLoadSink(long age, long timeStamp) {
     this.ageOfLastAppliedOp = age;
     this.timeStampsOfLastAppliedOp = timeStamp;

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSource.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSource.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSource.java
index 0b02805..845320c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSource.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationLoadSource.java
@@ -15,14 +15,16 @@ import org.apache.yetus.audience.InterfaceAudience;
 /**
  * A HBase ReplicationLoad to present MetricsSource information
  */
-@InterfaceAudience.Private
+@InterfaceAudience.Public
 public class ReplicationLoadSource {
-  private String peerID;
-  private long ageOfLastShippedOp;
-  private int sizeOfLogQueue;
-  private long timeStampOfLastShippedOp;
-  private long replicationLag;
-
+  private final String peerID;
+  private final long ageOfLastShippedOp;
+  private final int sizeOfLogQueue;
+  private final long timeStampOfLastShippedOp;
+  private final long replicationLag;
+
+  // TODO: add the builder for this class
+  @InterfaceAudience.Private
   public ReplicationLoadSource(String id, long age, int size, long timeStamp, long lag) {
     this.peerID = id;
     this.ageOfLastShippedOp = age;

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index c9ea5a5..17b1141 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -25,7 +25,6 @@ import java.lang.reflect.Method;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Locale;
@@ -49,9 +48,8 @@ import org.apache.hadoop.hbase.Cell.DataType;
 import org.apache.hadoop.hbase.CellBuilderType;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.ClusterId;
+import org.apache.hadoop.hbase.ClusterMetricsBuilder;
 import org.apache.hadoop.hbase.ClusterStatus;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.ExtendedCellBuilder;
 import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
@@ -60,7 +58,6 @@ import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Append;
@@ -92,7 +89,6 @@ import org.apache.hadoop.hbase.filter.ByteArrayComparable;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.io.LimitInputStream;
 import org.apache.hadoop.hbase.io.TimeRange;
-import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.protobuf.ProtobufMagic;
 import org.apache.hadoop.hbase.protobuf.ProtobufMessageConverter;
 import org.apache.hadoop.hbase.quotas.QuotaScope;
@@ -150,11 +146,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationPr
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.LiveServerInfo;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionInTransition;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos.HBaseVersionFileContent;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair;
@@ -2731,23 +2724,14 @@ public final class ProtobufUtil {
   }
 
   public static ReplicationLoadSink toReplicationLoadSink(
-      ClusterStatusProtos.ReplicationLoadSink cls) {
-    return new ReplicationLoadSink(cls.getAgeOfLastAppliedOp(), cls.getTimeStampsOfLastAppliedOp());
+      ClusterStatusProtos.ReplicationLoadSink rls) {
+    return new ReplicationLoadSink(rls.getAgeOfLastAppliedOp(), rls.getTimeStampsOfLastAppliedOp());
   }
 
   public static ReplicationLoadSource toReplicationLoadSource(
-      ClusterStatusProtos.ReplicationLoadSource cls) {
-    return new ReplicationLoadSource(cls.getPeerID(), cls.getAgeOfLastShippedOp(),
-        cls.getSizeOfLogQueue(), cls.getTimeStampOfLastShippedOp(), cls.getReplicationLag());
-  }
-
-  public static List<ReplicationLoadSource> toReplicationLoadSourceList(
-      List<ClusterStatusProtos.ReplicationLoadSource> clsList) {
-    ArrayList<ReplicationLoadSource> rlsList = new ArrayList<>(clsList.size());
-    for (ClusterStatusProtos.ReplicationLoadSource cls : clsList) {
-      rlsList.add(toReplicationLoadSource(cls));
-    }
-    return rlsList;
+      ClusterStatusProtos.ReplicationLoadSource rls) {
+    return new ReplicationLoadSource(rls.getPeerID(), rls.getAgeOfLastShippedOp(),
+      rls.getSizeOfLogQueue(), rls.getTimeStampOfLastShippedOp(), rls.getReplicationLag());
   }
 
   /**
@@ -2991,213 +2975,8 @@ public final class ProtobufUtil {
    * @param proto the protobuf ClusterStatus
    * @return the converted ClusterStatus
    */
-  public static ClusterStatus convert(ClusterStatusProtos.ClusterStatus proto) {
-    ClusterStatus.Builder builder = ClusterStatus.newBuilder();
-
-    Map<ServerName, ServerLoad> servers = null;
-    servers = new HashMap<>(proto.getLiveServersList().size());
-    for (LiveServerInfo lsi : proto.getLiveServersList()) {
-      servers.put(ProtobufUtil.toServerName(
-          lsi.getServer()), new ServerLoad(lsi.getServerLoad()));
-    }
-
-    List<ServerName> deadServers = new ArrayList<>(proto.getDeadServersList().size());
-    for (HBaseProtos.ServerName sn : proto.getDeadServersList()) {
-      deadServers.add(ProtobufUtil.toServerName(sn));
-    }
-
-    List<ServerName> backupMasters = new ArrayList<>(proto.getBackupMastersList().size());
-    for (HBaseProtos.ServerName sn : proto.getBackupMastersList()) {
-      backupMasters.add(ProtobufUtil.toServerName(sn));
-    }
-
-    List<RegionState> rit =
-      new ArrayList<>(proto.getRegionsInTransitionList().size());
-    for (RegionInTransition region : proto.getRegionsInTransitionList()) {
-      RegionState value = RegionState.convert(region.getRegionState());
-      rit.add(value);
-    }
-
-    String[] masterCoprocessors = null;
-    final int numMasterCoprocessors = proto.getMasterCoprocessorsCount();
-    masterCoprocessors = new String[numMasterCoprocessors];
-    for (int i = 0; i < numMasterCoprocessors; i++) {
-      masterCoprocessors[i] = proto.getMasterCoprocessors(i).getName();
-    }
-
-    String clusterId = null;
-    if (proto.hasClusterId()) {
-      clusterId = ClusterId.convert(proto.getClusterId()).toString();
-    }
-
-    String hbaseVersion = null;
-    if (proto.hasHbaseVersion()) {
-      hbaseVersion = proto.getHbaseVersion().getVersion();
-    }
-
-    ServerName master = null;
-    if (proto.hasMaster()) {
-      master = ProtobufUtil.toServerName(proto.getMaster());
-    }
-
-    Boolean balancerOn = null;
-    if (proto.hasBalancerOn()) {
-      balancerOn = proto.getBalancerOn();
-    }
-
-    builder.setHBaseVersion(hbaseVersion)
-           .setClusterId(clusterId)
-           .setLiveServers(servers)
-           .setDeadServers(deadServers)
-           .setMaster(master)
-           .setBackupMasters(backupMasters)
-           .setRegionState(rit)
-           .setMasterCoprocessors(masterCoprocessors)
-           .setBalancerOn(balancerOn);
-    if (proto.hasMasterInfoPort()) {
-      builder.setMasterInfoPort(proto.getMasterInfoPort());
-    }
-    return builder.build();
-  }
-
-  /**
-   * Convert ClusterStatusProtos.Option to ClusterStatus.Option
-   * @param option a ClusterStatusProtos.Option
-   * @return converted ClusterStatus.Option
-   */
-  public static ClusterStatus.Option toOption(ClusterStatusProtos.Option option) {
-    switch (option) {
-      case HBASE_VERSION: return ClusterStatus.Option.HBASE_VERSION;
-      case LIVE_SERVERS: return ClusterStatus.Option.LIVE_SERVERS;
-      case DEAD_SERVERS: return ClusterStatus.Option.DEAD_SERVERS;
-      case REGIONS_IN_TRANSITION: return ClusterStatus.Option.REGIONS_IN_TRANSITION;
-      case CLUSTER_ID: return ClusterStatus.Option.CLUSTER_ID;
-      case MASTER_COPROCESSORS: return ClusterStatus.Option.MASTER_COPROCESSORS;
-      case MASTER: return ClusterStatus.Option.MASTER;
-      case BACKUP_MASTERS: return ClusterStatus.Option.BACKUP_MASTERS;
-      case BALANCER_ON: return ClusterStatus.Option.BALANCER_ON;
-      case MASTER_INFO_PORT: return ClusterStatus.Option.MASTER_INFO_PORT;
-      // should not reach here
-      default: throw new IllegalArgumentException("Invalid option: " + option);
-    }
-  }
-
-  /**
-   * Convert ClusterStatus.Option to ClusterStatusProtos.Option
-   * @param option a ClusterStatus.Option
-   * @return converted ClusterStatusProtos.Option
-   */
-  public static ClusterStatusProtos.Option toOption(ClusterStatus.Option option) {
-    switch (option) {
-      case HBASE_VERSION: return ClusterStatusProtos.Option.HBASE_VERSION;
-      case LIVE_SERVERS: return ClusterStatusProtos.Option.LIVE_SERVERS;
-      case DEAD_SERVERS: return ClusterStatusProtos.Option.DEAD_SERVERS;
-      case REGIONS_IN_TRANSITION: return ClusterStatusProtos.Option.REGIONS_IN_TRANSITION;
-      case CLUSTER_ID: return ClusterStatusProtos.Option.CLUSTER_ID;
-      case MASTER_COPROCESSORS: return ClusterStatusProtos.Option.MASTER_COPROCESSORS;
-      case MASTER: return ClusterStatusProtos.Option.MASTER;
-      case BACKUP_MASTERS: return ClusterStatusProtos.Option.BACKUP_MASTERS;
-      case BALANCER_ON: return ClusterStatusProtos.Option.BALANCER_ON;
-      case MASTER_INFO_PORT: return ClusterStatusProtos.Option.MASTER_INFO_PORT;
-      // should not reach here
-      default: throw new IllegalArgumentException("Invalid option: " + option);
-    }
-  }
-
-  /**
-   * Convert a list of ClusterStatusProtos.Option to an enum set of ClusterStatus.Option
-   * @param options
-   * @return an enum set of ClusterStatus.Option
-   */
-  public static EnumSet<Option> toOptions(List<ClusterStatusProtos.Option> options) {
-    EnumSet<Option> result = EnumSet.noneOf(Option.class);
-    for (ClusterStatusProtos.Option opt : options) {
-      result.add(toOption(opt));
-    }
-    return result;
-  }
-
-  /**
-   * Convert an enum set of ClusterStatus.Option to a list of ClusterStatusProtos.Option
-   * @param options
-   * @return a list of ClusterStatusProtos.Option
-   */
-  public static List<ClusterStatusProtos.Option> toOptions(EnumSet<Option> options) {
-    List<ClusterStatusProtos.Option> result = new ArrayList<>(options.size());
-    for (ClusterStatus.Option opt : options) {
-      result.add(toOption(opt));
-    }
-    return result;
-  }
-
-  /**
-   * Convert a ClusterStatus to a protobuf ClusterStatus
-   *
-   * @return the protobuf ClusterStatus
-   */
-  public static ClusterStatusProtos.ClusterStatus convert(ClusterStatus status) {
-    ClusterStatusProtos.ClusterStatus.Builder builder =
-        ClusterStatusProtos.ClusterStatus.newBuilder();
-    if (status.getHBaseVersion() != null) {
-       builder.setHbaseVersion(
-         HBaseVersionFileContent.newBuilder()
-                                .setVersion(status.getHBaseVersion()));
-    }
-
-    if (status.getServers() != null) {
-      for (ServerName serverName : status.getServers()) {
-        LiveServerInfo.Builder lsi =
-            LiveServerInfo.newBuilder().setServer(ProtobufUtil.toServerName(serverName));
-        lsi.setServerLoad(status.getLoad(serverName).obtainServerLoadPB());
-        builder.addLiveServers(lsi.build());
-      }
-    }
-
-    if (status.getDeadServerNames() != null) {
-      for (ServerName deadServer : status.getDeadServerNames()) {
-        builder.addDeadServers(ProtobufUtil.toServerName(deadServer));
-      }
-    }
-
-    if (status.getRegionsInTransition() != null) {
-      for (RegionState rit : status.getRegionsInTransition()) {
-        ClusterStatusProtos.RegionState rs = rit.convert();
-        RegionSpecifier.Builder spec =
-            RegionSpecifier.newBuilder().setType(RegionSpecifierType.REGION_NAME);
-        spec.setValue(UnsafeByteOperations.unsafeWrap(rit.getRegion().getRegionName()));
-
-        RegionInTransition pbRIT =
-            RegionInTransition.newBuilder().setSpec(spec.build()).setRegionState(rs).build();
-        builder.addRegionsInTransition(pbRIT);
-      }
-    }
-
-    if (status.getClusterId() != null) {
-      builder.setClusterId(new ClusterId(status.getClusterId()).convert());
-    }
-
-    if (status.getMasterCoprocessors() != null) {
-      for (String coprocessor : status.getMasterCoprocessors()) {
-        builder.addMasterCoprocessors(HBaseProtos.Coprocessor.newBuilder().setName(coprocessor));
-      }
-    }
-
-    if (status.getMaster() != null) {
-      builder.setMaster(ProtobufUtil.toServerName(status.getMaster()));
-    }
-
-    if (status.getBackupMasters() != null) {
-      for (ServerName backup : status.getBackupMasters()) {
-        builder.addBackupMasters(ProtobufUtil.toServerName(backup));
-      }
-    }
-
-    if (status.getBalancerOn() != null) {
-      builder.setBalancerOn(status.getBalancerOn());
-    }
-
-    builder.setMasterInfoPort(status.getMasterInfoPort());
-    return builder.build();
+  public static ClusterStatus toClusterStatus(ClusterStatusProtos.ClusterStatus proto) {
+    return new ClusterStatus(ClusterMetricsBuilder.toClusterMetrics(proto));
   }
 
   public static RegionLoadStats createRegionLoadStats(ClientProtos.RegionLoadStats stats) {
@@ -3459,4 +3238,23 @@ public final class ProtobufUtil {
         .setMaxCacheSize(cacheEvictionStats.getMaxCacheSize())
         .build();
   }
+
+  public static ClusterStatusProtos.ReplicationLoadSource toReplicationLoadSource(
+      ReplicationLoadSource rls) {
+    return ClusterStatusProtos.ReplicationLoadSource.newBuilder()
+        .setPeerID(rls.getPeerID())
+        .setAgeOfLastShippedOp(rls.getAgeOfLastShippedOp())
+        .setSizeOfLogQueue((int) rls.getSizeOfLogQueue())
+        .setTimeStampOfLastShippedOp(rls.getTimeStampOfLastShippedOp())
+        .setReplicationLag(rls.getReplicationLag())
+        .build();
+  }
+
+  public static ClusterStatusProtos.ReplicationLoadSink toReplicationLoadSink(
+      ReplicationLoadSink rls) {
+    return ClusterStatusProtos.ReplicationLoadSink.newBuilder()
+        .setAgeOfLastAppliedOp(rls.getAgeOfLastAppliedOp())
+        .setTimeStampsOfLastAppliedOp(rls.getTimeStampsOfLastAppliedOp())
+        .build();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
index 5cbd2bf..7e8e903 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
@@ -27,7 +27,8 @@ import java.util.Set;
 import java.util.regex.Pattern;
 
 import org.apache.hadoop.hbase.CellScannable;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
+import org.apache.hadoop.hbase.ClusterMetricsBuilder;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
@@ -1523,7 +1524,7 @@ public final class RequestConverter {
    */
   public static GetClusterStatusRequest buildGetClusterStatusRequest(EnumSet<Option> options) {
     return GetClusterStatusRequest.newBuilder()
-                                  .addAllOptions(ProtobufUtil.toOptions(options))
+                                  .addAllOptions(ClusterMetricsBuilder.toOptions(options))
                                   .build();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java
index 47fcd84..299e51b 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java
@@ -22,12 +22,11 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.List;
-
 import org.apache.commons.lang3.RandomUtils;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
 import org.apache.hadoop.hbase.chaos.factories.MonkeyConstants;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.util.Bytes;

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java
index 2588e63..6c2e635 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java
@@ -38,7 +38,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.IntegrationTestBase;

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
index 826db07..e1e73ff 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
@@ -49,7 +49,7 @@ import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterStatusResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterStatusResource.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterStatusResource.java
index 90ebccb..7ee1065 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterStatusResource.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterStatusResource.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.rest;
 
 import java.io.IOException;
 import java.util.EnumSet;
-
 import javax.ws.rs.GET;
 import javax.ws.rs.Produces;
 import javax.ws.rs.core.CacheControl;
@@ -29,16 +28,15 @@ import javax.ws.rs.core.Context;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.core.Response.ResponseBuilder;
 import javax.ws.rs.core.UriInfo;
-
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.ClusterStatus;
-import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.RegionLoad;
+import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
 import org.apache.hadoop.hbase.rest.model.StorageClusterStatusModel;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @InterfaceAudience.Private
 public class StorageClusterStatusResource extends ResourceBase {

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterVersionResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterVersionResource.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterVersionResource.java
index 3d70410..3ac6566 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterVersionResource.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/StorageClusterVersionResource.java
@@ -21,20 +21,18 @@ package org.apache.hadoop.hbase.rest;
 
 import java.io.IOException;
 import java.util.EnumSet;
-
 import javax.ws.rs.GET;
 import javax.ws.rs.Produces;
 import javax.ws.rs.core.CacheControl;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.Response;
-import javax.ws.rs.core.UriInfo;
 import javax.ws.rs.core.Response.ResponseBuilder;
-
+import javax.ws.rs.core.UriInfo;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
+import org.apache.hadoop.hbase.rest.model.StorageClusterVersionModel;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
-import org.apache.hadoop.hbase.rest.model.StorageClusterVersionModel;
 
 @InterfaceAudience.Private
 public class StorageClusterVersionResource extends ResourceBase {

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
index cd8c386..e4386bc 100644
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
@@ -33,9 +33,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeMap;
-
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.ClusterStatus;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
 import org.apache.hadoop.hbase.HBaseCluster;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -59,6 +58,7 @@ import org.junit.Test;
 import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -525,7 +525,7 @@ public abstract class TestRSGroupsBase {
             getTableRegionMap().get(tableName) != null &&
                 getTableRegionMap().get(tableName).size() == 6 &&
                 admin.getClusterStatus(EnumSet.of(Option.REGIONS_IN_TRANSITION))
-                     .getRegionsInTransition().size() < 1;
+                     .getRegionStatesInTransition().size() < 1;
       }
     });
 
@@ -609,7 +609,7 @@ public abstract class TestRSGroupsBase {
     TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
       @Override
       public boolean evaluate() throws Exception {
-        return cluster.getClusterStatus().getRegionsInTransition().isEmpty();
+        return cluster.getClusterStatus().getRegionStatesInTransition().isEmpty();
       }
     });
     Set<Address> newServers = Sets.newHashSet();
@@ -626,7 +626,7 @@ public abstract class TestRSGroupsBase {
     TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
       @Override
       public boolean evaluate() throws Exception {
-        return cluster.getClusterStatus().getRegionsInTransition().isEmpty();
+        return cluster.getClusterStatus().getRegionStatesInTransition().isEmpty();
       }
     });
 
@@ -830,7 +830,7 @@ public abstract class TestRSGroupsBase {
                 getTableRegionMap().get(tableName).size() == 5 &&
                 getTableServerRegionMap().get(tableName).size() == 1 &&
                 admin.getClusterStatus(EnumSet.of(Option.REGIONS_IN_TRANSITION))
-                     .getRegionsInTransition().size() < 1;
+                     .getRegionStatesInTransition().size() < 1;
       }
     });
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/BackupMasterStatusTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/BackupMasterStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/BackupMasterStatusTmpl.jamon
index ee5b33f..e1f864e 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/BackupMasterStatusTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/BackupMasterStatusTmpl.jamon
@@ -22,7 +22,7 @@ HMaster master;
 <%import>
 java.util.*;
 org.apache.hadoop.hbase.ServerName;
-org.apache.hadoop.hbase.ClusterStatus;
+org.apache.hadoop.hbase.ClusterMetrics;
 org.apache.hadoop.hbase.master.HMaster;
 org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
 </%import>
@@ -55,7 +55,7 @@ MasterAddressTracker masterAddressTracker = master.getMasterAddressTracker();
     </tr>
     <%java>
     Collection<ServerName> backup_masters = master.getClusterStatusWithoutCoprocessor(
-        EnumSet.of(ClusterStatus.Option.BACKUP_MASTERS)).getBackupMasters();
+        EnumSet.of(ClusterMetrics.Option.BACKUP_MASTERS)).getBackupMasters();
     ServerName [] backupServerNames = backup_masters.toArray(new ServerName[backup_masters.size()]);
     Arrays.sort(backupServerNames);
     for (ServerName serverName : backupServerNames) {


[26/26] hbase git commit: HBASE-19543 Abstract a replication storage interface to extract the zk specific code

Posted by zh...@apache.org.
HBASE-19543 Abstract a replication storage interface to extract the zk specific code


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

Branch: refs/heads/HBASE-19397
Commit: 557f5d4d852a4fd3177ce68c45d7efef062ea659
Parents: 72e30d8
Author: zhangduo <zh...@apache.org>
Authored: Fri Dec 22 14:37:28 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Fri Dec 22 21:44:04 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/util/CollectionUtils.java      |   3 +
 hbase-replication/pom.xml                       |  12 +
 .../replication/ReplicationPeerStorage.java     |  74 ++++
 .../replication/ReplicationQueueStorage.java    | 164 +++++++
 .../replication/ReplicationStateZKBase.java     |   1 -
 .../replication/ReplicationStorageFactory.java  |  49 +++
 .../replication/ZKReplicationPeerStorage.java   | 164 +++++++
 .../replication/ZKReplicationQueueStorage.java  | 425 +++++++++++++++++++
 .../replication/ZKReplicationStorageBase.java   |  75 ++++
 .../TestZKReplicationPeerStorage.java           | 171 ++++++++
 .../TestZKReplicationQueueStorage.java          | 171 ++++++++
 .../org/apache/hadoop/hbase/master/HMaster.java |  36 +-
 .../hadoop/hbase/master/MasterServices.java     |   6 +-
 .../master/procedure/MasterProcedureEnv.java    |  24 +-
 .../master/replication/AddPeerProcedure.java    |   6 +-
 .../replication/DisablePeerProcedure.java       |   7 +-
 .../master/replication/EnablePeerProcedure.java |   6 +-
 .../master/replication/ModifyPeerProcedure.java |  41 +-
 .../master/replication/RemovePeerProcedure.java |   6 +-
 .../master/replication/ReplicationManager.java  | 199 ---------
 .../replication/ReplicationPeerManager.java     | 331 +++++++++++++++
 .../replication/UpdatePeerConfigProcedure.java  |   7 +-
 .../replication/TestReplicationAdmin.java       |  59 +--
 .../hbase/master/MockNoopMasterServices.java    |  10 +-
 .../hbase/master/TestMasterNoCluster.java       |   4 +-
 .../TestReplicationDisableInactivePeer.java     |   6 +-
 26 files changed, 1755 insertions(+), 302 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/557f5d4d/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CollectionUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CollectionUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CollectionUtils.java
index 875b124..8bbb6f1 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CollectionUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CollectionUtils.java
@@ -107,6 +107,9 @@ public class CollectionUtils {
     return list.get(list.size() - 1);
   }
 
+  public static <T> List<T> nullToEmpty(List<T> list) {
+    return list != null ? list : Collections.emptyList();
+  }
   /**
    * In HBASE-16648 we found that ConcurrentHashMap.get is much faster than computeIfAbsent if the
    * value already exists. Notice that the implementation does not guarantee that the supplier will

http://git-wip-us.apache.org/repos/asf/hbase/blob/557f5d4d/hbase-replication/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-replication/pom.xml b/hbase-replication/pom.xml
index ab22199..4e3cea0 100644
--- a/hbase-replication/pom.xml
+++ b/hbase-replication/pom.xml
@@ -121,6 +121,18 @@
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-zookeeper</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-zookeeper</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
     <!-- General dependencies -->
     <dependency>
       <groupId>org.apache.commons</groupId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/557f5d4d/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java
new file mode 100644
index 0000000..e00cd0d
--- /dev/null
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java
@@ -0,0 +1,74 @@
+/**
+ * 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.replication;
+
+import java.util.List;
+import java.util.Optional;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Perform read/write to the replication peer storage.
+ */
+@InterfaceAudience.Private
+public interface ReplicationPeerStorage {
+
+  /**
+   * Add a replication peer.
+   * @throws ReplicationException if there are errors accessing the storage service.
+   */
+  void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
+      throws ReplicationException;
+
+  /**
+   * Remove a replication peer.
+   * @throws ReplicationException if there are errors accessing the storage service.
+   */
+  void removePeer(String peerId) throws ReplicationException;
+
+  /**
+   * Set the state of peer, {@code true} to {@code ENABLED}, otherwise to {@code DISABLED}.
+   * @throws ReplicationException if there are errors accessing the storage service.
+   */
+  void setPeerState(String peerId, boolean enabled) throws ReplicationException;
+
+  /**
+   * Update the config a replication peer.
+   * @throws ReplicationException if there are errors accessing the storage service.
+   */
+  void updatePeerConfig(String peerId, ReplicationPeerConfig peerConfig)
+      throws ReplicationException;
+
+  /**
+   * Return the peer ids of all replication peers.
+   * @throws ReplicationException if there are errors accessing the storage service.
+   */
+  List<String> listPeerIds() throws ReplicationException;
+
+  /**
+   * Test whether a replication peer is enabled.
+   * @throws ReplicationException if there are errors accessing the storage service.
+   */
+  boolean isPeerEnabled(String peerId) throws ReplicationException;
+
+  /**
+   * Get the peer config of a replication peer.
+   * @throws ReplicationException if there are errors accessing the storage service.
+   */
+  Optional<ReplicationPeerConfig> getPeerConfig(String peerId) throws ReplicationException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/557f5d4d/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
new file mode 100644
index 0000000..7210d9a
--- /dev/null
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
@@ -0,0 +1,164 @@
+/**
+ * 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.replication;
+
+import java.util.List;
+import java.util.Set;
+import java.util.SortedSet;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Perform read/write to the replication queue storage.
+ */
+@InterfaceAudience.Private
+public interface ReplicationQueueStorage {
+
+  /**
+   * Remove a replication queue for a given regionserver.
+   * @param serverName the name of the regionserver
+   * @param queueId a String that identifies the queue.
+   */
+  void removeQueue(ServerName serverName, String queueId) throws ReplicationException;
+
+  /**
+   * Add a new WAL file to the given queue for a given regionserver. If the queue does not exist it
+   * is created.
+   * @param serverName the name of the regionserver
+   * @param queueId a String that identifies the queue.
+   * @param fileName name of the WAL
+   */
+  void addWAL(ServerName serverName, String queueId, String fileName) throws ReplicationException;
+
+  /**
+   * Remove an WAL file from the given queue for a given regionserver.
+   * @param serverName the name of the regionserver
+   * @param queueId a String that identifies the queue.
+   * @param fileName name of the WAL
+   */
+  void removeWAL(ServerName serverName, String queueId, String fileName)
+      throws ReplicationException;
+
+  /**
+   * Set the current position for a specific WAL in a given queue for a given regionserver.
+   * @param serverName the name of the regionserver
+   * @param queueId a String that identifies the queue
+   * @param fileName name of the WAL
+   * @param position the current position in the file
+   */
+  void setWALPosition(ServerName serverName, String queueId, String fileName, long position)
+      throws ReplicationException;
+
+  /**
+   * Get the current position for a specific WAL in a given queue for a given regionserver.
+   * @param serverName the name of the regionserver
+   * @param queueId a String that identifies the queue
+   * @param fileName name of the WAL
+   * @return the current position in the file
+   */
+  long getWALPosition(ServerName serverName, String queueId, String fileName)
+      throws ReplicationException;
+
+  /**
+   * Get a list of all queues for the specified region server.
+   * @param serverName the server name of the region server that owns the set of queues
+   * @return a list of queueIds
+   */
+  List<String> getAllQueues(ServerName serverName) throws ReplicationException;
+
+  /**
+   * Change ownership for the queue identified by queueId and belongs to a dead region server.
+   * @param sourceServerName the name of the dead region server
+   * @param destServerName the name of the target region server
+   * @param queueId the id of the queue
+   * @return the new PeerId and A SortedSet of WALs in its queue
+   */
+  Pair<String, SortedSet<String>> claimQueue(ServerName sourceServerName, String queueId,
+      ServerName destServerName) throws ReplicationException;
+
+  /**
+   * Remove the record of region server if the queue is empty.
+   */
+  void removeReplicatorIfQueueIsEmpty(ServerName serverName) throws ReplicationException;
+
+  /**
+   * 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
+   */
+  List<ServerName> getListOfReplicators() throws ReplicationException;
+
+  /**
+   * Load all wals in all replication queues. This method guarantees to return a snapshot which
+   * contains all WALs in the zookeeper at the start of this call even there is concurrent queue
+   * failover. However, some newly created WALs during the call may not be included.
+   */
+  Set<String> getAllWALs() throws ReplicationException;
+
+  /**
+   * Add a peer to hfile reference queue if peer does not exist.
+   * @param peerId peer cluster id to be added
+   * @throws ReplicationException if fails to add a peer id to hfile reference queue
+   */
+  void addPeerToHFileRefs(String peerId) throws ReplicationException;
+
+  /**
+   * Remove a peer from hfile reference queue.
+   * @param peerId peer cluster id to be removed
+   */
+  void removePeerFromHFileRefs(String peerId) throws ReplicationException;
+
+  /**
+   * Add new hfile references to the queue.
+   * @param peerId peer cluster id to which the hfiles need to be replicated
+   * @param pairs list of pairs of { HFile location in staging dir, HFile path in region dir which
+   *          will be added in the queue }
+   * @throws ReplicationException if fails to add a hfile reference
+   */
+  void addHFileRefs(String peerId, List<Pair<Path, Path>> pairs) throws ReplicationException;
+
+  /**
+   * Remove hfile references from the queue.
+   * @param peerId peer cluster id from which this hfile references needs to be removed
+   * @param files list of hfile references to be removed
+   */
+  void removeHFileRefs(String peerId, List<String> files) throws ReplicationException;
+
+  /**
+   * Get the change version number of replication hfile references node. This can be used as
+   * optimistic locking to get a consistent snapshot of the replication queues of hfile references.
+   * @return change version number of hfile references node
+   */
+  int getHFileRefsNodeChangeVersion() throws ReplicationException;
+
+  /**
+   * Get list of all peers from hfile reference queue.
+   * @return a list of peer ids
+   */
+  List<String> getAllPeersFromHFileRefsQueue() throws ReplicationException;
+
+  /**
+   * Get a list of all hfile references in the given peer.
+   * @param peerId a String that identifies the peer
+   * @return a list of hfile references
+   */
+  List<String> getReplicableHFiles(String peerId) throws ReplicationException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/557f5d4d/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 ad970c6..edbbd4b 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
@@ -63,7 +63,6 @@ public abstract class ReplicationStateZKBase {
   protected final Configuration conf;
   protected final Abortable abortable;
 
-  // Public for testing
   public static final byte[] ENABLED_ZNODE_BYTES =
       toByteArray(ReplicationProtos.ReplicationState.State.ENABLED);
   public static final byte[] DISABLED_ZNODE_BYTES =

http://git-wip-us.apache.org/repos/asf/hbase/blob/557f5d4d/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStorageFactory.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStorageFactory.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStorageFactory.java
new file mode 100644
index 0000000..60d0749
--- /dev/null
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStorageFactory.java
@@ -0,0 +1,49 @@
+/**
+ * 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.replication;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Used to create replication storage(peer, queue) classes.
+ * <p>
+ * For now we only have zk based implementation.
+ */
+@InterfaceAudience.Private
+public class ReplicationStorageFactory {
+
+  private ReplicationStorageFactory() {
+  }
+
+  /**
+   * Create a new {@link ReplicationPeerStorage}.
+   */
+  public static ReplicationPeerStorage getReplicationPeerStorage(ZKWatcher zk, Configuration conf) {
+    return new ZKReplicationPeerStorage(zk, conf);
+  }
+
+  /**
+   * Create a new {@link ReplicationQueueStorage}.
+   */
+  public static ReplicationQueueStorage getReplicationQueueStorage(ZKWatcher zk,
+      Configuration conf) {
+    return new ZKReplicationQueueStorage(zk, conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/557f5d4d/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java
new file mode 100644
index 0000000..49af4c3
--- /dev/null
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java
@@ -0,0 +1,164 @@
+/**
+ * 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.replication;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.util.CollectionUtils;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
+
+/**
+ * ZK based replication peer storage.
+ */
+@InterfaceAudience.Private
+class ZKReplicationPeerStorage extends ZKReplicationStorageBase implements ReplicationPeerStorage {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ZKReplicationPeerStorage.class);
+
+  public static final byte[] ENABLED_ZNODE_BYTES =
+    toByteArray(ReplicationProtos.ReplicationState.State.ENABLED);
+  public static final byte[] DISABLED_ZNODE_BYTES =
+    toByteArray(ReplicationProtos.ReplicationState.State.DISABLED);
+
+  /**
+   * The name of the znode that contains the replication status of a remote slave (i.e. peer)
+   * cluster.
+   */
+  private final String peerStateNodeName;
+
+  /**
+   * The name of the znode that contains a list of all remote slave (i.e. peer) clusters.
+   */
+  private final String peersZNode;
+
+  public ZKReplicationPeerStorage(ZKWatcher zookeeper, Configuration conf) {
+    super(zookeeper, conf);
+    this.peerStateNodeName = conf.get("zookeeper.znode.replication.peers.state", "peer-state");
+    String peersZNodeName = conf.get("zookeeper.znode.replication.peers", "peers");
+    this.peersZNode = ZNodePaths.joinZNode(replicationZNode, peersZNodeName);
+  }
+
+  private String getPeerStateNode(String peerId) {
+    return ZNodePaths.joinZNode(getPeerNode(peerId), peerStateNodeName);
+  }
+
+  private String getPeerNode(String peerId) {
+    return ZNodePaths.joinZNode(peersZNode, peerId);
+  }
+
+  @Override
+  public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
+      throws ReplicationException {
+    try {
+      ZKUtil.createWithParents(zookeeper, peersZNode);
+      ZKUtil.multiOrSequential(zookeeper,
+        Arrays.asList(
+          ZKUtilOp.createAndFailSilent(getPeerNode(peerId),
+            ReplicationPeerConfigUtil.toByteArray(peerConfig)),
+          ZKUtilOp.createAndFailSilent(getPeerStateNode(peerId),
+            enabled ? ENABLED_ZNODE_BYTES : DISABLED_ZNODE_BYTES)),
+        false);
+    } catch (KeeperException e) {
+      throw new ReplicationException("Could not add peer with id=" + peerId + ", peerConfif=>" +
+        peerConfig + ", state=" + (enabled ? "ENABLED" : "DISABLED"), e);
+    }
+  }
+
+  @Override
+  public void removePeer(String peerId) throws ReplicationException {
+    try {
+      ZKUtil.deleteNodeRecursively(zookeeper, getPeerNode(peerId));
+    } catch (KeeperException e) {
+      throw new ReplicationException("Could not remove peer with id=" + peerId, e);
+    }
+  }
+
+  @Override
+  public void setPeerState(String peerId, boolean enabled) throws ReplicationException {
+    byte[] stateBytes = enabled ? ENABLED_ZNODE_BYTES : DISABLED_ZNODE_BYTES;
+    try {
+      ZKUtil.setData(zookeeper, getPeerStateNode(peerId), stateBytes);
+    } catch (KeeperException e) {
+      throw new ReplicationException("Unable to change state of the peer with id=" + peerId, e);
+    }
+  }
+
+  @Override
+  public void updatePeerConfig(String peerId, ReplicationPeerConfig peerConfig)
+      throws ReplicationException {
+    try {
+      ZKUtil.setData(this.zookeeper, getPeerNode(peerId),
+        ReplicationPeerConfigUtil.toByteArray(peerConfig));
+    } catch (KeeperException e) {
+      throw new ReplicationException(
+          "There was a problem trying to save changes to the " + "replication peer " + peerId, e);
+    }
+  }
+
+  @Override
+  public List<String> listPeerIds() throws ReplicationException {
+    try {
+      return CollectionUtils.nullToEmpty(ZKUtil.listChildrenAndWatchThem(zookeeper, peersZNode));
+    } catch (KeeperException e) {
+      throw new ReplicationException("Cannot get the list of peers", e);
+    }
+  }
+
+  @Override
+  public boolean isPeerEnabled(String peerId) throws ReplicationException {
+    try {
+      return Arrays.equals(ENABLED_ZNODE_BYTES,
+        ZKUtil.getData(zookeeper, getPeerStateNode(peerId)));
+    } catch (KeeperException | InterruptedException e) {
+      throw new ReplicationException("Unable to get status of the peer with id=" + peerId, e);
+    }
+  }
+
+  @Override
+  public Optional<ReplicationPeerConfig> getPeerConfig(String peerId) throws ReplicationException {
+    byte[] data;
+    try {
+      data = ZKUtil.getData(zookeeper, getPeerNode(peerId));
+    } catch (KeeperException | InterruptedException e) {
+      throw new ReplicationException("Error getting configuration for peer with id=" + peerId, e);
+    }
+    if (data == null || data.length == 0) {
+      return Optional.empty();
+    }
+    try {
+      return Optional.of(ReplicationPeerConfigUtil.parsePeerFrom(data));
+    } catch (DeserializationException e) {
+      LOG.warn("Failed to parse replication peer config for peer with id=" + peerId, e);
+      return Optional.empty();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/557f5d4d/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
new file mode 100644
index 0000000..7015d7f
--- /dev/null
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
@@ -0,0 +1,425 @@
+/**
+ * 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.replication;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.hadoop.hbase.util.CollectionUtils.nullToEmpty;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CollectionUtils;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.BadVersionException;
+import org.apache.zookeeper.KeeperException.NoNodeException;
+import org.apache.zookeeper.KeeperException.NodeExistsException;
+import org.apache.zookeeper.KeeperException.NotEmptyException;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
+
+/**
+ * ZK based replication queue storage.
+ */
+@InterfaceAudience.Private
+class ZKReplicationQueueStorage extends ZKReplicationStorageBase
+    implements ReplicationQueueStorage {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ZKReplicationQueueStorage.class);
+
+  public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_KEY =
+    "zookeeper.znode.replication.hfile.refs";
+  public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT = "hfile-refs";
+
+  /**
+   * The name of the znode that contains all replication queues
+   */
+  private final String queuesZNode;
+
+  /**
+   * The name of the znode that contains queues of hfile references to be replicated
+   */
+  private final String hfileRefsZNode;
+
+  public ZKReplicationQueueStorage(ZKWatcher zookeeper, Configuration conf) {
+    super(zookeeper, conf);
+
+    String queuesZNodeName = conf.get("zookeeper.znode.replication.rs", "rs");
+    String hfileRefsZNodeName = conf.get(ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_KEY,
+      ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT);
+    this.queuesZNode = ZNodePaths.joinZNode(replicationZNode, queuesZNodeName);
+    this.hfileRefsZNode = ZNodePaths.joinZNode(replicationZNode, hfileRefsZNodeName);
+  }
+
+  private String getRsNode(ServerName serverName) {
+    return ZNodePaths.joinZNode(queuesZNode, serverName.getServerName());
+  }
+
+  private String getQueueNode(ServerName serverName, String queueId) {
+    return ZNodePaths.joinZNode(getRsNode(serverName), queueId);
+  }
+
+  private String getFileNode(String queueNode, String fileName) {
+    return ZNodePaths.joinZNode(queueNode, fileName);
+  }
+
+  private String getFileNode(ServerName serverName, String queueId, String fileName) {
+    return getFileNode(getQueueNode(serverName, queueId), fileName);
+  }
+
+  @Override
+  public void removeQueue(ServerName serverName, String queueId) throws ReplicationException {
+    try {
+      ZKUtil.deleteNodeRecursively(zookeeper, getQueueNode(serverName, queueId));
+    } catch (KeeperException e) {
+      throw new ReplicationException(
+          "Failed to delete queue (serverName=" + serverName + ", queueId=" + queueId + ")", e);
+    }
+  }
+
+  @Override
+  public void addWAL(ServerName serverName, String queueId, String fileName)
+      throws ReplicationException {
+    try {
+      ZKUtil.createWithParents(zookeeper, getFileNode(serverName, queueId, fileName));
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to add wal to queue (serverName=" + serverName +
+        ", queueId=" + queueId + ", fileName=" + fileName + ")", e);
+    }
+  }
+
+  @Override
+  public void removeWAL(ServerName serverName, String queueId, String fileName)
+      throws ReplicationException {
+    String fileNode = getFileNode(serverName, queueId, fileName);
+    try {
+      ZKUtil.deleteNode(zookeeper, fileNode);
+    } catch (NoNodeException e) {
+      LOG.warn(fileNode + " has already been deleted when removing log");
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to remove wal from queue (serverName=" + serverName +
+        ", queueId=" + queueId + ", fileName=" + fileName + ")", e);
+    }
+  }
+
+  @Override
+  public void setWALPosition(ServerName serverName, String queueId, String fileName, long position)
+      throws ReplicationException {
+    try {
+      ZKUtil.setData(zookeeper, getFileNode(serverName, queueId, fileName),
+        ZKUtil.positionToByteArray(position));
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to set log position (serverName=" + serverName +
+        ", queueId=" + queueId + ", fileName=" + fileName + ", position=" + position + ")", e);
+    }
+  }
+
+  @Override
+  public long getWALPosition(ServerName serverName, String queueId, String fileName)
+      throws ReplicationException {
+    byte[] bytes;
+    try {
+      bytes = ZKUtil.getData(zookeeper, getFileNode(serverName, queueId, fileName));
+    } catch (KeeperException | InterruptedException e) {
+      throw new ReplicationException("Failed to get log position (serverName=" + serverName +
+        ", queueId=" + queueId + ", fileName=" + fileName + ")", e);
+    }
+    try {
+      return ZKUtil.parseWALPositionFrom(bytes);
+    } catch (DeserializationException de) {
+      LOG.warn("Failed to parse log position (serverName=" + serverName + ", queueId=" + queueId +
+        ", fileName=" + fileName + ")");
+    }
+    // if we can not parse the position, start at the beginning of the wal file again
+    return 0;
+  }
+
+  @Override
+  public Pair<String, SortedSet<String>> claimQueue(ServerName sourceServerName, String queueId,
+      ServerName destServerName) throws ReplicationException {
+    LOG.info(
+      "Atomically moving " + sourceServerName + "/" + queueId + "'s WALs to " + destServerName);
+    try {
+      ZKUtil.createWithParents(zookeeper, getRsNode(destServerName));
+    } catch (KeeperException e) {
+      throw new ReplicationException(
+          "Claim queue queueId=" + queueId + " from " + sourceServerName + " to " + destServerName +
+            " failed when creating the node for " + destServerName,
+          e);
+    }
+    try {
+      String oldQueueNode = getQueueNode(sourceServerName, queueId);
+      List<String> wals = ZKUtil.listChildrenNoWatch(zookeeper, oldQueueNode);
+      String newQueueId = queueId + "-" + sourceServerName;
+      if (CollectionUtils.isEmpty(wals)) {
+        ZKUtil.deleteNodeFailSilent(zookeeper, oldQueueNode);
+        LOG.info("Removed " + sourceServerName + "/" + queueId + " since it's empty");
+        return new Pair<>(newQueueId, Collections.emptySortedSet());
+      }
+      String newQueueNode = getQueueNode(destServerName, newQueueId);
+      List<ZKUtilOp> listOfOps = new ArrayList<>();
+      SortedSet<String> logQueue = new TreeSet<>();
+      // create the new cluster znode
+      listOfOps.add(ZKUtilOp.createAndFailSilent(newQueueNode, HConstants.EMPTY_BYTE_ARRAY));
+      // get the offset of the logs and set it to new znodes
+      for (String wal : wals) {
+        String oldWalNode = getFileNode(oldQueueNode, wal);
+        byte[] logOffset = ZKUtil.getData(this.zookeeper, oldWalNode);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Creating " + wal + " with data " + Bytes.toStringBinary(logOffset));
+        }
+        String newWalNode = getFileNode(newQueueNode, wal);
+        listOfOps.add(ZKUtilOp.createAndFailSilent(newWalNode, logOffset));
+        listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldWalNode));
+        logQueue.add(wal);
+      }
+      // add delete op for peer
+      listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldQueueNode));
+
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("The multi list size is: " + listOfOps.size());
+      }
+      ZKUtil.multiOrSequential(zookeeper, listOfOps, false);
+
+      LOG.info(
+        "Atomically moved " + sourceServerName + "/" + queueId + "'s WALs to " + destServerName);
+      return new Pair<>(newQueueId, logQueue);
+    } catch (NoNodeException | NodeExistsException | NotEmptyException | BadVersionException e) {
+      // Multi call failed; it looks like some other regionserver took away the logs.
+      // These exceptions mean that zk tells us the request can not be execute so it is safe to just
+      // return a null. For other types of exception should be thrown out to notify the upper layer.
+      LOG.info(
+        "Claim queue queueId=" + queueId + " from " + sourceServerName + " to " + destServerName +
+          " failed with " + e.toString() + ", maybe someone else has already took away the logs");
+      return null;
+    } catch (KeeperException | InterruptedException e) {
+      throw new ReplicationException("Claim queue queueId=" + queueId + " from " +
+        sourceServerName + " to " + destServerName + " failed", e);
+    }
+  }
+
+  @Override
+  public void removeReplicatorIfQueueIsEmpty(ServerName serverName) throws ReplicationException {
+    try {
+      ZKUtil.deleteNodeFailSilent(zookeeper, getRsNode(serverName));
+    } catch (NotEmptyException e) {
+      // keep silence to avoid logging too much.
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to remove replicator for " + serverName, e);
+    }
+  }
+
+  private List<ServerName> getListOfReplicators0() throws KeeperException {
+    return nullToEmpty(ZKUtil.listChildrenNoWatch(zookeeper, queuesZNode)).stream()
+        .map(ServerName::parseServerName).collect(toList());
+  }
+
+  @Override
+  public List<ServerName> getListOfReplicators() throws ReplicationException {
+    try {
+      return getListOfReplicators0();
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to get list of replicators", e);
+    }
+  }
+
+  private List<String> getLogsInQueue0(ServerName serverName, String queueId)
+      throws KeeperException {
+    return nullToEmpty(ZKUtil.listChildrenNoWatch(zookeeper, getQueueNode(serverName, queueId)));
+  }
+
+  private List<String> getAllQueues0(ServerName serverName) throws KeeperException {
+    return nullToEmpty(ZKUtil.listChildrenNoWatch(zookeeper, getRsNode(serverName)));
+  }
+
+  @Override
+  public List<String> getAllQueues(ServerName serverName) throws ReplicationException {
+    try {
+      return getAllQueues0(serverName);
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to get all queues (serverName=" + serverName + ")", e);
+    }
+  }
+
+  private int getQueuesZNodeCversion() throws KeeperException {
+    Stat stat = new Stat();
+    ZKUtil.getDataNoWatch(this.zookeeper, this.queuesZNode, stat);
+    return stat.getCversion();
+  }
+
+  @Override
+  public Set<String> getAllWALs() throws ReplicationException {
+    try {
+      for (int retry = 0;; retry++) {
+        int v0 = getQueuesZNodeCversion();
+        List<ServerName> rss = getListOfReplicators0();
+        if (rss.isEmpty()) {
+          LOG.debug("Didn't find any region server that replicates, won't prevent any deletions.");
+          return Collections.emptySet();
+        }
+        Set<String> wals = Sets.newHashSet();
+        for (ServerName rs : rss) {
+          for (String queueId : getAllQueues0(rs)) {
+            wals.addAll(getLogsInQueue0(rs, queueId));
+          }
+        }
+        int v1 = getQueuesZNodeCversion();
+        if (v0 == v1) {
+          return wals;
+        }
+        LOG.info(String.format("Replication queue node cversion changed from %d to %d, retry = %d",
+          v0, v1, retry));
+      }
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to get all wals", e);
+    }
+  }
+
+  private String getHFileRefsPeerNode(String peerId) {
+    return ZNodePaths.joinZNode(hfileRefsZNode, peerId);
+  }
+
+  private String getHFileNode(String peerNode, String fileName) {
+    return ZNodePaths.joinZNode(peerNode, fileName);
+  }
+
+  @Override
+  public void addPeerToHFileRefs(String peerId) throws ReplicationException {
+    String peerNode = getHFileRefsPeerNode(peerId);
+    try {
+      if (ZKUtil.checkExists(zookeeper, peerNode) == -1) {
+        LOG.info("Adding peer " + peerId + " to hfile reference queue.");
+        ZKUtil.createWithParents(zookeeper, peerNode);
+      }
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to add peer " + peerId + " to hfile reference queue.",
+          e);
+    }
+  }
+
+  @Override
+  public void removePeerFromHFileRefs(String peerId) throws ReplicationException {
+    String peerNode = getHFileRefsPeerNode(peerId);
+    try {
+      if (ZKUtil.checkExists(zookeeper, peerNode) == -1) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Peer " + peerNode + " not found in hfile reference queue.");
+        }
+      } else {
+        LOG.info("Removing peer " + peerNode + " from hfile reference queue.");
+        ZKUtil.deleteNodeRecursively(zookeeper, peerNode);
+      }
+    } catch (KeeperException e) {
+      throw new ReplicationException(
+          "Failed to remove peer " + peerId + " from hfile reference queue.", e);
+    }
+  }
+
+  @Override
+  public void addHFileRefs(String peerId, List<Pair<Path, Path>> pairs)
+      throws ReplicationException {
+    String peerNode = getHFileRefsPeerNode(peerId);
+    boolean debugEnabled = LOG.isDebugEnabled();
+    if (debugEnabled) {
+      LOG.debug("Adding hfile references " + pairs + " in queue " + peerNode);
+    }
+    List<ZKUtilOp> listOfOps =
+      pairs.stream().map(p -> p.getSecond().getName()).map(n -> getHFileNode(peerNode, n))
+          .map(f -> ZKUtilOp.createAndFailSilent(f, HConstants.EMPTY_BYTE_ARRAY)).collect(toList());
+    if (debugEnabled) {
+      LOG.debug("The multi list size for adding hfile references in zk for node " + peerNode +
+        " is " + listOfOps.size());
+    }
+    try {
+      ZKUtil.multiOrSequential(this.zookeeper, listOfOps, true);
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to add hfile reference to peer " + peerId, e);
+    }
+  }
+
+  @Override
+  public void removeHFileRefs(String peerId, List<String> files) throws ReplicationException {
+    String peerNode = getHFileRefsPeerNode(peerId);
+    boolean debugEnabled = LOG.isDebugEnabled();
+    if (debugEnabled) {
+      LOG.debug("Removing hfile references " + files + " from queue " + peerNode);
+    }
+
+    List<ZKUtilOp> listOfOps = files.stream().map(n -> getHFileNode(peerNode, n))
+        .map(ZKUtilOp::deleteNodeFailSilent).collect(toList());
+    if (debugEnabled) {
+      LOG.debug("The multi list size for removing hfile references in zk for node " + peerNode +
+        " is " + listOfOps.size());
+    }
+    try {
+      ZKUtil.multiOrSequential(this.zookeeper, listOfOps, true);
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to remove hfile reference from peer " + peerId, e);
+    }
+  }
+
+  @Override
+  public int getHFileRefsNodeChangeVersion() throws ReplicationException {
+    Stat stat = new Stat();
+    try {
+      ZKUtil.getDataNoWatch(zookeeper, hfileRefsZNode, stat);
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to get stat of replication hfile references node.", e);
+    }
+    return stat.getCversion();
+  }
+
+  @Override
+  public List<String> getAllPeersFromHFileRefsQueue() throws ReplicationException {
+    try {
+      return nullToEmpty(ZKUtil.listChildrenNoWatch(zookeeper, hfileRefsZNode));
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to get list of all peers in hfile references node.",
+          e);
+    }
+  }
+
+  @Override
+  public List<String> getReplicableHFiles(String peerId) throws ReplicationException {
+    try {
+      return nullToEmpty(ZKUtil.listChildrenNoWatch(this.zookeeper, getHFileRefsPeerNode(peerId)));
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to get list of hfile references for peer " + peerId,
+          e);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/557f5d4d/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java
new file mode 100644
index 0000000..b8a2044
--- /dev/null
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java
@@ -0,0 +1,75 @@
+/**
+ * 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.replication;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
+
+/**
+ * This is a base class for maintaining replication related data,for example, peer, queue, etc, in
+ * zookeeper.
+ */
+@InterfaceAudience.Private
+class ZKReplicationStorageBase {
+
+  /** The name of the base znode that contains all replication state. */
+  protected final String replicationZNode;
+
+  protected final ZKWatcher zookeeper;
+  protected final Configuration conf;
+
+  protected ZKReplicationStorageBase(ZKWatcher zookeeper, Configuration conf) {
+    this.zookeeper = zookeeper;
+    this.conf = conf;
+    String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication");
+
+    this.replicationZNode =
+      ZNodePaths.joinZNode(this.zookeeper.znodePaths.baseZNode, replicationZNodeName);
+
+  }
+
+  /**
+   * Serialized protobuf of <code>state</code> with pb magic prefix prepended suitable for use as
+   * content of a peer-state znode under a peer cluster id as in
+   * /hbase/replication/peers/PEER_ID/peer-state.
+   */
+  protected static byte[] toByteArray(final ReplicationProtos.ReplicationState.State state) {
+    ReplicationProtos.ReplicationState msg =
+      ReplicationProtos.ReplicationState.newBuilder().setState(state).build();
+    // There is no toByteArray on this pb Message?
+    // 32 bytes is default which seems fair enough here.
+    try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
+      CodedOutputStream cos = CodedOutputStream.newInstance(baos, 16);
+      msg.writeTo(cos);
+      cos.flush();
+      baos.flush();
+      return ProtobufUtil.prependPBMagic(baos.toByteArray());
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/557f5d4d/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java
new file mode 100644
index 0000000..a3be1e6
--- /dev/null
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java
@@ -0,0 +1,171 @@
+/**
+ * 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.replication;
+
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toSet;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Stream;
+
+import org.apache.hadoop.hbase.HBaseZKTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ ReplicationTests.class, MediumTests.class })
+public class TestZKReplicationPeerStorage {
+
+  private static final HBaseZKTestingUtility UTIL = new HBaseZKTestingUtility();
+
+  private static ZKReplicationPeerStorage STORAGE;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    UTIL.startMiniZKCluster();
+    STORAGE = new ZKReplicationPeerStorage(UTIL.getZooKeeperWatcher(), UTIL.getConfiguration());
+  }
+
+  @AfterClass
+  public static void tearDown() throws IOException {
+    UTIL.shutdownMiniZKCluster();
+  }
+
+  private Set<String> randNamespaces(Random rand) {
+    return Stream.generate(() -> Long.toHexString(rand.nextLong())).limit(rand.nextInt(5))
+        .collect(toSet());
+  }
+
+  private Map<TableName, List<String>> randTableCFs(Random rand) {
+    int size = rand.nextInt(5);
+    Map<TableName, List<String>> map = new HashMap<>();
+    for (int i = 0; i < size; i++) {
+      TableName tn = TableName.valueOf(Long.toHexString(rand.nextLong()));
+      List<String> cfs = Stream.generate(() -> Long.toHexString(rand.nextLong()))
+          .limit(rand.nextInt(5)).collect(toList());
+      map.put(tn, cfs);
+    }
+    return map;
+  }
+
+  private ReplicationPeerConfig getConfig(int seed) {
+    Random rand = new Random(seed);
+    ReplicationPeerConfig config = new ReplicationPeerConfig();
+    config.setClusterKey(Long.toHexString(rand.nextLong()));
+    config.setReplicationEndpointImpl(Long.toHexString(rand.nextLong()));
+    config.setNamespaces(randNamespaces(rand));
+    config.setExcludeNamespaces(randNamespaces(rand));
+    config.setTableCFsMap(randTableCFs(rand));
+    config.setReplicateAllUserTables(rand.nextBoolean());
+    config.setBandwidth(rand.nextInt(1000));
+    return config;
+  }
+
+  private void assertSetEquals(Set<String> expected, Set<String> actual) {
+    if (expected == null || expected.size() == 0) {
+      assertTrue(actual == null || actual.size() == 0);
+      return;
+    }
+    assertEquals(expected.size(), actual.size());
+    expected.forEach(s -> assertTrue(actual.contains(s)));
+  }
+
+  private void assertMapEquals(Map<TableName, List<String>> expected,
+      Map<TableName, List<String>> actual) {
+    if (expected == null || expected.size() == 0) {
+      assertTrue(actual == null || actual.size() == 0);
+      return;
+    }
+    assertEquals(expected.size(), actual.size());
+    expected.forEach((expectedTn, expectedCFs) -> {
+      List<String> actualCFs = actual.get(expectedTn);
+      if (expectedCFs == null || expectedCFs.size() == 0) {
+        assertTrue(actual.containsKey(expectedTn));
+        assertTrue(actualCFs == null || actualCFs.size() == 0);
+      } else {
+        assertNotNull(actualCFs);
+        assertEquals(expectedCFs.size(), actualCFs.size());
+        for (Iterator<String> expectedIt = expectedCFs.iterator(), actualIt = actualCFs.iterator();
+          expectedIt.hasNext();) {
+          assertEquals(expectedIt.next(), actualIt.next());
+        }
+      }
+    });
+  }
+
+  private void assertConfigEquals(ReplicationPeerConfig expected, ReplicationPeerConfig actual) {
+    assertEquals(expected.getClusterKey(), actual.getClusterKey());
+    assertEquals(expected.getReplicationEndpointImpl(), actual.getReplicationEndpointImpl());
+    assertSetEquals(expected.getNamespaces(), actual.getNamespaces());
+    assertSetEquals(expected.getExcludeNamespaces(), actual.getExcludeNamespaces());
+    assertMapEquals(expected.getTableCFsMap(), actual.getTableCFsMap());
+    assertMapEquals(expected.getExcludeTableCFsMap(), actual.getExcludeTableCFsMap());
+    assertEquals(expected.replicateAllUserTables(), actual.replicateAllUserTables());
+    assertEquals(expected.getBandwidth(), actual.getBandwidth());
+  }
+
+  @Test
+  public void test() throws ReplicationException {
+    int peerCount = 10;
+    for (int i = 0; i < peerCount; i++) {
+      STORAGE.addPeer(Integer.toString(i), getConfig(i), i % 2 == 0);
+    }
+    List<String> peerIds = STORAGE.listPeerIds();
+    assertEquals(peerCount, peerIds.size());
+    for (String peerId : peerIds) {
+      int seed = Integer.parseInt(peerId);
+      assertConfigEquals(getConfig(seed), STORAGE.getPeerConfig(peerId).get());
+    }
+    for (int i = 0; i < peerCount; i++) {
+      STORAGE.updatePeerConfig(Integer.toString(i), getConfig(i + 1));
+    }
+    for (String peerId : peerIds) {
+      int seed = Integer.parseInt(peerId);
+      assertConfigEquals(getConfig(seed + 1), STORAGE.getPeerConfig(peerId).get());
+    }
+    for (int i = 0; i < peerCount; i++) {
+      assertEquals(i % 2 == 0, STORAGE.isPeerEnabled(Integer.toString(i)));
+    }
+    for (int i = 0; i < peerCount; i++) {
+      STORAGE.setPeerState(Integer.toString(i), i % 2 != 0);
+    }
+    for (int i = 0; i < peerCount; i++) {
+      assertEquals(i % 2 != 0, STORAGE.isPeerEnabled(Integer.toString(i)));
+    }
+    String toRemove = Integer.toString(peerCount / 2);
+    STORAGE.removePeer(toRemove);
+    peerIds = STORAGE.listPeerIds();
+    assertEquals(peerCount - 1, peerIds.size());
+    assertFalse(peerIds.contains(toRemove));
+    assertFalse(STORAGE.getPeerConfig(toRemove).isPresent());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/557f5d4d/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
new file mode 100644
index 0000000..d5bba0d
--- /dev/null
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
@@ -0,0 +1,171 @@
+/**
+ * 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.replication;
+
+import static org.hamcrest.CoreMatchers.hasItems;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+import java.util.SortedSet;
+
+import org.apache.hadoop.hbase.HBaseZKTestingUtility;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.util.Pair;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ ReplicationTests.class, MediumTests.class })
+public class TestZKReplicationQueueStorage {
+  private static final HBaseZKTestingUtility UTIL = new HBaseZKTestingUtility();
+
+  private static ZKReplicationQueueStorage STORAGE;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    UTIL.startMiniZKCluster();
+    STORAGE = new ZKReplicationQueueStorage(UTIL.getZooKeeperWatcher(), UTIL.getConfiguration());
+  }
+
+  @AfterClass
+  public static void tearDown() throws IOException {
+    UTIL.shutdownMiniZKCluster();
+  }
+
+  @After
+  public void tearDownAfterTest() throws ReplicationException {
+    for (ServerName serverName : STORAGE.getListOfReplicators()) {
+      for (String queue : STORAGE.getAllQueues(serverName)) {
+        STORAGE.removeQueue(serverName, queue);
+      }
+      STORAGE.removeReplicatorIfQueueIsEmpty(serverName);
+    }
+    for (String peerId : STORAGE.getAllPeersFromHFileRefsQueue()) {
+      STORAGE.removePeerFromHFileRefs(peerId);
+    }
+  }
+
+  private ServerName getServerName(int i) {
+    return ServerName.valueOf("127.0.0.1", 8000 + i, 10000 + i);
+  }
+
+  @Test
+  public void testReplicator() throws ReplicationException {
+    assertTrue(STORAGE.getListOfReplicators().isEmpty());
+    String queueId = "1";
+    for (int i = 0; i < 10; i++) {
+      STORAGE.addWAL(getServerName(i), queueId, "file" + i);
+    }
+    List<ServerName> replicators = STORAGE.getListOfReplicators();
+    assertEquals(10, replicators.size());
+    for (int i = 0; i < 10; i++) {
+      assertThat(replicators, hasItems(getServerName(i)));
+    }
+    for (int i = 0; i < 5; i++) {
+      STORAGE.removeQueue(getServerName(i), queueId);
+    }
+    for (int i = 0; i < 10; i++) {
+      STORAGE.removeReplicatorIfQueueIsEmpty(getServerName(i));
+    }
+    replicators = STORAGE.getListOfReplicators();
+    assertEquals(5, replicators.size());
+    for (int i = 5; i < 10; i++) {
+      assertThat(replicators, hasItems(getServerName(i)));
+    }
+  }
+
+  private String getFileName(String base, int i) {
+    return String.format(base + "-%04d", i);
+  }
+
+  @Test
+  public void testAddRemoveLog() throws ReplicationException {
+    ServerName serverName1 = ServerName.valueOf("127.0.0.1", 8000, 10000);
+    assertTrue(STORAGE.getAllQueues(serverName1).isEmpty());
+    String queue1 = "1";
+    String queue2 = "2";
+    for (int i = 0; i < 10; i++) {
+      STORAGE.addWAL(serverName1, queue1, getFileName("file1", i));
+      STORAGE.addWAL(serverName1, queue2, getFileName("file2", i));
+    }
+    List<String> queueIds = STORAGE.getAllQueues(serverName1);
+    assertEquals(2, queueIds.size());
+    assertThat(queueIds, hasItems("1", "2"));
+
+    for (int i = 0; i < 10; i++) {
+      assertEquals(0, STORAGE.getWALPosition(serverName1, queue1, getFileName("file1", i)));
+      assertEquals(0, STORAGE.getWALPosition(serverName1, queue2, getFileName("file2", i)));
+      STORAGE.setWALPosition(serverName1, queue1, getFileName("file1", i), (i + 1) * 100);
+      STORAGE.setWALPosition(serverName1, queue2, getFileName("file2", i), (i + 1) * 100 + 10);
+    }
+
+    for (int i = 0; i < 10; i++) {
+      assertEquals((i + 1) * 100,
+        STORAGE.getWALPosition(serverName1, queue1, getFileName("file1", i)));
+      assertEquals((i + 1) * 100 + 10,
+        STORAGE.getWALPosition(serverName1, queue2, getFileName("file2", i)));
+    }
+
+    for (int i = 0; i < 10; i++) {
+      if (i % 2 == 0) {
+        STORAGE.removeWAL(serverName1, queue1, getFileName("file1", i));
+      } else {
+        STORAGE.removeWAL(serverName1, queue2, getFileName("file2", i));
+      }
+    }
+
+    queueIds = STORAGE.getAllQueues(serverName1);
+    assertEquals(2, queueIds.size());
+    assertThat(queueIds, hasItems("1", "2"));
+
+    ServerName serverName2 = ServerName.valueOf("127.0.0.1", 8001, 10001);
+    Pair<String, SortedSet<String>> peer1 = STORAGE.claimQueue(serverName1, "1", serverName2);
+
+    assertEquals("1-" + serverName1.getServerName(), peer1.getFirst());
+    assertEquals(5, peer1.getSecond().size());
+    int i = 1;
+    for (String wal : peer1.getSecond()) {
+      assertEquals(getFileName("file1", i), wal);
+      assertEquals((i + 1) * 100,
+        STORAGE.getWALPosition(serverName2, peer1.getFirst(), getFileName("file1", i)));
+      i += 2;
+    }
+
+    queueIds = STORAGE.getAllQueues(serverName1);
+    assertEquals(1, queueIds.size());
+    assertThat(queueIds, hasItems("2"));
+
+    queueIds = STORAGE.getAllQueues(serverName2);
+    assertEquals(1, queueIds.size());
+    assertThat(queueIds, hasItems(peer1.getFirst()));
+
+    Set<String> allWals = STORAGE.getAllWALs();
+    assertEquals(10, allWals.size());
+    for (i = 0; i < 10; i++) {
+      assertThat(allWals, hasItems(i % 2 == 0 ? getFileName("file2", i) : getFileName("file1", i)));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/557f5d4d/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 024a6ff..b72255a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -39,6 +39,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Objects;
+import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
@@ -135,7 +136,7 @@ import org.apache.hadoop.hbase.master.replication.DisablePeerProcedure;
 import org.apache.hadoop.hbase.master.replication.EnablePeerProcedure;
 import org.apache.hadoop.hbase.master.replication.ModifyPeerProcedure;
 import org.apache.hadoop.hbase.master.replication.RemovePeerProcedure;
-import org.apache.hadoop.hbase.master.replication.ReplicationManager;
+import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
 import org.apache.hadoop.hbase.master.replication.UpdatePeerConfigProcedure;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.mob.MobConstants;
@@ -329,7 +330,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   private AssignmentManager assignmentManager;
 
   // manager of replication
-  private ReplicationManager replicationManager;
+  private ReplicationPeerManager replicationPeerManager;
 
   // buffer for "fatal error" notices from region servers
   // in the cluster. This is only used for assisting
@@ -700,8 +701,8 @@ public class HMaster extends HRegionServer implements MasterServices {
   /**
    * Initialize all ZK based system trackers.
    */
-  void initializeZKBasedSystemTrackers() throws IOException,
-      InterruptedException, KeeperException, CoordinatedStateException {
+  void initializeZKBasedSystemTrackers() throws IOException, InterruptedException, KeeperException,
+      CoordinatedStateException, ReplicationException {
     this.balancer = LoadBalancerFactory.getLoadBalancer(conf);
     this.normalizer = RegionNormalizerFactory.getRegionNormalizer(conf);
     this.normalizer.setMasterServices(this);
@@ -719,7 +720,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     this.assignmentManager = new AssignmentManager(this);
     this.assignmentManager.start();
 
-    this.replicationManager = new ReplicationManager(conf, zooKeeper, this);
+    this.replicationPeerManager = ReplicationPeerManager.create(zooKeeper, conf);
 
     this.regionServerTracker = new RegionServerTracker(zooKeeper, this, this.serverManager);
     this.regionServerTracker.start();
@@ -764,9 +765,8 @@ public class HMaster extends HRegionServer implements MasterServices {
    * <li>Handle either fresh cluster start or master failover</li>
    * </ol>
    */
-  private void finishActiveMasterInitialization(MonitoredTask status)
-      throws IOException, InterruptedException, KeeperException, CoordinatedStateException {
-
+  private void finishActiveMasterInitialization(MonitoredTask status) throws IOException,
+      InterruptedException, KeeperException, CoordinatedStateException, ReplicationException {
     activeMaster = true;
     Thread zombieDetector = new Thread(new InitializationMonitor(this),
         "ActiveMasterInitializationMonitor-" + System.currentTimeMillis());
@@ -3356,18 +3356,19 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   @Override
-  public ReplicationPeerConfig getReplicationPeerConfig(String peerId) throws ReplicationException,
-      IOException {
+  public ReplicationPeerConfig getReplicationPeerConfig(String peerId)
+      throws ReplicationException, IOException {
     if (cpHost != null) {
       cpHost.preGetReplicationPeerConfig(peerId);
     }
-    final ReplicationPeerConfig peerConfig = this.replicationManager.getPeerConfig(peerId);
-    LOG.info(getClientIdAuditPrefix() + " get replication peer config, id=" + peerId + ", config="
-        + peerConfig);
+    LOG.info(getClientIdAuditPrefix() + " get replication peer config, id=" + peerId);
+    Optional<ReplicationPeerConfig> peerConfig =
+      this.replicationPeerManager.getPeerConfig(peerId);
+
     if (cpHost != null) {
       cpHost.postGetReplicationPeerConfig(peerId);
     }
-    return peerConfig;
+    return peerConfig.orElse(null);
   }
 
   @Override
@@ -3386,7 +3387,8 @@ public class HMaster extends HRegionServer implements MasterServices {
     }
     LOG.info(getClientIdAuditPrefix() + " list replication peers, regex=" + regex);
     Pattern pattern = regex == null ? null : Pattern.compile(regex);
-    List<ReplicationPeerDescription> peers = this.replicationManager.listReplicationPeers(pattern);
+    List<ReplicationPeerDescription> peers =
+      this.replicationPeerManager.listPeers(pattern);
     if (cpHost != null) {
       cpHost.postListReplicationPeers(regex);
     }
@@ -3536,7 +3538,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   @Override
-  public ReplicationManager getReplicationManager() {
-    return replicationManager;
+  public ReplicationPeerManager getReplicationPeerManager() {
+    return replicationPeerManager;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/557f5d4d/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index 43df8b1..67baf14 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.locking.LockManager;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-import org.apache.hadoop.hbase.master.replication.ReplicationManager;
+import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
 import org.apache.hadoop.hbase.procedure2.LockedResource;
@@ -459,9 +459,9 @@ public interface MasterServices extends Server {
       IOException;
 
   /**
-   * Returns the {@link ReplicationManager}.
+   * Returns the {@link ReplicationPeerManager}.
    */
-  ReplicationManager getReplicationManager();
+  ReplicationPeerManager getReplicationPeerManager();
 
   /**
    * Update the peerConfig for the specified peer

http://git-wip-us.apache.org/repos/asf/hbase/blob/557f5d4d/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
index fa4d371..ae038a5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
@@ -24,24 +24,24 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.conf.ConfigurationObserver;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
-import org.apache.hadoop.hbase.master.replication.ReplicationManager;
+import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
 import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
-import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.Superusers;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
@@ -138,8 +138,8 @@ public class MasterProcedureEnv implements ConfigurationObserver {
     return remoteDispatcher;
   }
 
-  public ReplicationManager getReplicationManager() {
-    return master.getReplicationManager();
+  public ReplicationPeerManager getReplicationPeerManager() {
+    return master.getReplicationPeerManager();
   }
 
   public boolean isRunning() {
@@ -151,22 +151,22 @@ public class MasterProcedureEnv implements ConfigurationObserver {
     return master.isInitialized();
   }
 
-  public boolean waitInitialized(Procedure proc) {
+  public boolean waitInitialized(Procedure<?> proc) {
     return master.getInitializedEvent().suspendIfNotReady(proc);
   }
 
-  public boolean waitServerCrashProcessingEnabled(Procedure proc) {
+  public boolean waitServerCrashProcessingEnabled(Procedure<?> proc) {
     if (master instanceof HMaster) {
       return ((HMaster)master).getServerCrashProcessingEnabledEvent().suspendIfNotReady(proc);
     }
     return false;
   }
 
-  public boolean waitFailoverCleanup(Procedure proc) {
+  public boolean waitFailoverCleanup(Procedure<?> proc) {
     return master.getAssignmentManager().getFailoverCleanupEvent().suspendIfNotReady(proc);
   }
 
-  public void setEventReady(ProcedureEvent event, boolean isReady) {
+  public void setEventReady(ProcedureEvent<?> event, boolean isReady) {
     if (isReady) {
       event.wake(procSched);
     } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/557f5d4d/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
index 066c3e7..a4f9b32 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
@@ -58,16 +58,18 @@ public class AddPeerProcedure extends ModifyPeerProcedure {
   }
 
   @Override
-  protected void prePeerModification(MasterProcedureEnv env) throws IOException {
+  protected void prePeerModification(MasterProcedureEnv env)
+      throws IOException, ReplicationException {
     MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
     if (cpHost != null) {
       cpHost.preAddReplicationPeer(peerId, peerConfig);
     }
+    env.getReplicationPeerManager().preAddPeer(peerId, peerConfig);
   }
 
   @Override
   protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException {
-    env.getReplicationManager().addReplicationPeer(peerId, peerConfig, enabled);
+    env.getReplicationPeerManager().addPeer(peerId, peerConfig, enabled);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/557f5d4d/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
index 9a28de6..10e35a8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -51,12 +52,12 @@ public class DisablePeerProcedure extends ModifyPeerProcedure {
     if (cpHost != null) {
       cpHost.preDisableReplicationPeer(peerId);
     }
+    env.getReplicationPeerManager().preDisablePeer(peerId);
   }
 
   @Override
-  protected void updatePeerStorage(MasterProcedureEnv env)
-      throws IllegalArgumentException, Exception {
-    env.getReplicationManager().disableReplicationPeer(peerId);
+  protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException {
+    env.getReplicationPeerManager().disablePeer(peerId);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/557f5d4d/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
index 4855901..f2a9f01 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -51,11 +52,12 @@ public class EnablePeerProcedure extends ModifyPeerProcedure {
     if (cpHost != null) {
       cpHost.preEnableReplicationPeer(peerId);
     }
+    env.getReplicationPeerManager().preEnablePeer(peerId);
   }
 
   @Override
-  protected void updatePeerStorage(MasterProcedureEnv env) throws Exception {
-    env.getReplicationManager().enableReplicationPeer(peerId);
+  protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException {
+    env.getReplicationPeerManager().enablePeer(peerId);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/557f5d4d/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
index c4552ed..279fbc7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
 
-import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface;
 import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
@@ -27,6 +26,7 @@ import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
 import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
 import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -67,17 +67,16 @@ public abstract class ModifyPeerProcedure
   }
 
   /**
-   * Called before we start the actual processing. If an exception is thrown then we will give up
-   * and mark the procedure as failed directly.
+   * Called before we start the actual processing. The implementation should call the pre CP hook,
+   * and also the pre-check for the peer modification.
+   * <p>
+   * If an IOException is thrown then we will give up and mark the procedure as failed directly. If
+   * all checks passes then the procedure can not be rolled back any more.
    */
-  protected abstract void prePeerModification(MasterProcedureEnv env) throws IOException;
+  protected abstract void prePeerModification(MasterProcedureEnv env)
+      throws IOException, ReplicationException;
 
-  /**
-   * We will give up and mark the procedure as failure if {@link IllegalArgumentException} is
-   * thrown, for other type of Exception we will retry.
-   */
-  protected abstract void updatePeerStorage(MasterProcedureEnv env)
-      throws IllegalArgumentException, Exception;
+  protected abstract void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException;
 
   /**
    * Called before we finish the procedure. The implementation can do some logging work, and also
@@ -100,23 +99,24 @@ public abstract class ModifyPeerProcedure
         try {
           prePeerModification(env);
         } catch (IOException e) {
-          LOG.warn(getClass().getName() + " failed to call prePeerModification for peer " + peerId +
-            ", mark the procedure as failure and give up", e);
-          setFailure("prePeerModification", e);
+          LOG.warn(
+            getClass().getName() + " failed to call CP hook or the pre check is failed for peer " +
+              peerId + ", mark the procedure as failure and give up",
+            e);
+          setFailure("master-" + getPeerOperationType().name().toLowerCase() + "-peer", e);
           releaseLatch();
           return Flow.NO_MORE_STATE;
+        } catch (ReplicationException e) {
+          LOG.warn(getClass().getName() + " failed to call prePeerModification for peer " + peerId +
+            ", retry", e);
+          throw new ProcedureYieldException();
         }
         setNextState(PeerModificationState.UPDATE_PEER_STORAGE);
         return Flow.HAS_MORE_STATE;
       case UPDATE_PEER_STORAGE:
         try {
           updatePeerStorage(env);
-        } catch (IllegalArgumentException e) {
-          setFailure("master-" + getPeerOperationType().name().toLowerCase() + "-peer",
-            new DoNotRetryIOException(e));
-          releaseLatch();
-          return Flow.NO_MORE_STATE;
-        } catch (Exception e) {
+        } catch (ReplicationException e) {
           LOG.warn(
             getClass().getName() + " update peer storage for peer " + peerId + " failed, retry", e);
           throw new ProcedureYieldException();
@@ -158,8 +158,7 @@ public abstract class ModifyPeerProcedure
   @Override
   protected void rollbackState(MasterProcedureEnv env, PeerModificationState state)
       throws IOException, InterruptedException {
-    if (state == PeerModificationState.PRE_PEER_MODIFICATION ||
-      state == PeerModificationState.UPDATE_PEER_STORAGE) {
+    if (state == PeerModificationState.PRE_PEER_MODIFICATION) {
       // actually the peer related operations has no rollback, but if we haven't done any
       // modifications on the peer storage, we can just return.
       return;

http://git-wip-us.apache.org/repos/asf/hbase/blob/557f5d4d/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
index d40df02..6e9c384 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -51,11 +52,12 @@ public class RemovePeerProcedure extends ModifyPeerProcedure {
     if (cpHost != null) {
       cpHost.preRemoveReplicationPeer(peerId);
     }
+    env.getReplicationPeerManager().preRemovePeer(peerId);
   }
 
   @Override
-  protected void updatePeerStorage(MasterProcedureEnv env) throws Exception {
-    env.getReplicationManager().removeReplicationPeer(peerId);
+  protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException {
+    env.getReplicationPeerManager().removePeer(peerId);
   }
 
   @Override


[14/26] hbase git commit: HBASE-19496 Reusing the ByteBuffer in rpc layer corrupt the ServerLoad and RegionLoad

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java
index cbf4b1c..2988ddf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterStatusPublisher.java
@@ -20,7 +20,6 @@
 
 package org.apache.hadoop.hbase.master;
 
-
 import java.io.Closeable;
 import java.io.IOException;
 import java.net.Inet6Address;
@@ -35,8 +34,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ClusterMetricsBuilder;
 import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
@@ -66,7 +65,6 @@ import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.InternetProtocolFa
 import org.apache.hadoop.hbase.shaded.io.netty.channel.socket.nio.NioDatagramChannel;
 import org.apache.hadoop.hbase.shaded.io.netty.handler.codec.MessageToMessageEncoder;
 import org.apache.hadoop.hbase.shaded.io.netty.util.internal.StringUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
 
 
@@ -161,12 +159,12 @@ public class ClusterStatusPublisher extends ScheduledChore {
     // We're reusing an existing protobuf message, but we don't send everything.
     // This could be extended in the future, for example if we want to send stuff like the
     //  hbase:meta server name.
-    ClusterStatus.Builder csBuilder = ClusterStatus.newBuilder();
-    csBuilder.setHBaseVersion(VersionInfo.getVersion())
-             .setClusterId(master.getMasterFileSystem().getClusterId().toString())
-             .setMaster(master.getServerName())
-             .setDeadServers(sns);
-    publisher.publish(csBuilder.build());
+    publisher.publish(new ClusterStatus(ClusterMetricsBuilder.newBuilder()
+        .setHBaseVersion(VersionInfo.getVersion())
+        .setClusterId(master.getMasterFileSystem().getClusterId().toString())
+        .setMasterName(master.getServerName())
+        .setDeadServerNames(sns)
+        .build()));
   }
 
   protected void cleanup() {
@@ -340,7 +338,8 @@ public class ClusterStatusPublisher extends ScheduledChore {
       @Override
       protected void encode(ChannelHandlerContext channelHandlerContext,
                             ClusterStatus clusterStatus, List<Object> objects) {
-        ClusterStatusProtos.ClusterStatus csp = ProtobufUtil.convert(clusterStatus);
+        ClusterStatusProtos.ClusterStatus csp
+          = ClusterMetricsBuilder.toClusterStatus(clusterStatus);
         objects.add(new DatagramPacket(Unpooled.wrappedBuffer(csp.toByteArray()), isa));
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index e31db82..cf95030 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -18,10 +18,8 @@
  */
 package org.apache.hadoop.hbase.master;
 
-import javax.servlet.ServletException;
-import javax.servlet.http.HttpServlet;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Service;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.lang.reflect.Constructor;
@@ -30,6 +28,7 @@ import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
@@ -50,11 +49,16 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Function;
 import java.util.regex.Pattern;
-
+import java.util.stream.Collectors;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
+import org.apache.hadoop.hbase.ClusterMetricsBuilder;
 import org.apache.hadoop.hbase.ClusterStatus;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
 import org.apache.hadoop.hbase.CoordinatedStateException;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseIOException;
@@ -66,6 +70,7 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.PleaseHoldException;
 import org.apache.hadoop.hbase.ServerLoad;
+import org.apache.hadoop.hbase.ServerMetricsBuilder;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
@@ -180,8 +185,8 @@ import org.apache.hadoop.hbase.zookeeper.MasterMaintenanceModeTracker;
 import org.apache.hadoop.hbase.zookeeper.RegionNormalizerTracker;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.eclipse.jetty.server.Server;
@@ -190,6 +195,7 @@ import org.eclipse.jetty.servlet.ServletHolder;
 import org.eclipse.jetty.webapp.WebAppContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
@@ -201,9 +207,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceViolat
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
 
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Service;
-
 /**
  * HMaster is the "master server" for HBase. An HBase cluster has one active
  * master.  If many masters are started, all compete.  Whichever wins goes on to
@@ -1021,7 +1024,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     for (ServerName sn: this.regionServerTracker.getOnlineServers()) {
       // The isServerOnline check is opportunistic, correctness is handled inside
       if (!this.serverManager.isServerOnline(sn) &&
-          serverManager.checkAndRecordNewServer(sn, ServerLoad.EMPTY_SERVERLOAD)) {
+          serverManager.checkAndRecordNewServer(sn, new ServerLoad(ServerMetricsBuilder.of(sn)))) {
         LOG.info("Registered server found up in zk but who has not yet reported in: " + sn);
       }
     }
@@ -2412,7 +2415,7 @@ public class HMaster extends HRegionServer implements MasterServices {
 
   public ClusterStatus getClusterStatusWithoutCoprocessor(EnumSet<Option> options)
       throws InterruptedIOException {
-    ClusterStatus.Builder builder = ClusterStatus.newBuilder();
+    ClusterMetricsBuilder builder = ClusterMetricsBuilder.newBuilder();
     // given that hbase1 can't submit the request with Option,
     // we return all information to client if the list of Option is empty.
     if (options.isEmpty()) {
@@ -2423,30 +2426,32 @@ public class HMaster extends HRegionServer implements MasterServices {
       switch (opt) {
         case HBASE_VERSION: builder.setHBaseVersion(VersionInfo.getVersion()); break;
         case CLUSTER_ID: builder.setClusterId(getClusterId()); break;
-        case MASTER: builder.setMaster(getServerName()); break;
-        case BACKUP_MASTERS: builder.setBackupMasters(getBackupMasters()); break;
+        case MASTER: builder.setMasterName(getServerName()); break;
+        case BACKUP_MASTERS: builder.setBackerMasterNames(getBackupMasters()); break;
         case LIVE_SERVERS: {
           if (serverManager != null) {
-            builder.setLiveServers(serverManager.getOnlineServers());
+            builder.setLiveServerMetrics(serverManager.getOnlineServers().entrySet().stream()
+              .collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue())));
           }
           break;
         }
         case DEAD_SERVERS: {
           if (serverManager != null) {
-            builder.setDeadServers(new ArrayList<>(
+            builder.setDeadServerNames(new ArrayList<>(
               serverManager.getDeadServers().copyServerNames()));
           }
           break;
         }
         case MASTER_COPROCESSORS: {
           if (cpHost != null) {
-            builder.setMasterCoprocessors(getMasterCoprocessors());
+            builder.setMasterCoprocessorNames(Arrays.asList(getMasterCoprocessors()));
           }
           break;
         }
         case REGIONS_IN_TRANSITION: {
           if (assignmentManager != null) {
-            builder.setRegionState(assignmentManager.getRegionStates().getRegionsStateInTransition());
+            builder.setRegionsInTransition(assignmentManager.getRegionStates()
+                .getRegionsStateInTransition());
           }
           break;
         }
@@ -2464,7 +2469,7 @@ public class HMaster extends HRegionServer implements MasterServices {
         }
       }
     }
-    return builder.build();
+    return new ClusterStatus(builder.build());
   }
 
   /**
@@ -2496,7 +2501,7 @@ public class HMaster extends HRegionServer implements MasterServices {
       backupMasterStrings = null;
     }
 
-    List<ServerName> backupMasters = null;
+    List<ServerName> backupMasters = Collections.emptyList();
     if (backupMasterStrings != null && !backupMasterStrings.isEmpty()) {
       backupMasters = new ArrayList<>(backupMasterStrings.size());
       for (String s: backupMasterStrings) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 4a4bbe1..f0f2e10 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -32,6 +32,7 @@ import java.util.Set;
 import java.util.stream.Collectors;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ClusterMetricsBuilder;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.MetaTableAccessor;
@@ -446,7 +447,7 @@ public class MasterRpcServices extends RSRpcServices
       ClusterStatusProtos.ServerLoad sl = request.getLoad();
       ServerName serverName = ProtobufUtil.toServerName(request.getServer());
       ServerLoad oldLoad = master.getServerManager().getLoad(serverName);
-      ServerLoad newLoad = new ServerLoad(sl);
+      ServerLoad newLoad = new ServerLoad(serverName, sl);
       master.getServerManager().regionServerReport(serverName, newLoad);
       int version = VersionInfoUtil.getCurrentClientVersionNumber();
       master.getAssignmentManager().reportOnlineRegions(serverName,
@@ -902,8 +903,8 @@ public class MasterRpcServices extends RSRpcServices
     GetClusterStatusResponse.Builder response = GetClusterStatusResponse.newBuilder();
     try {
       master.checkInitialized();
-      response.setClusterStatus(ProtobufUtil.convert(
-        master.getClusterStatus(ProtobufUtil.toOptions(req.getOptionsList()))));
+      response.setClusterStatus(ClusterMetricsBuilder.toClusterStatus(
+        master.getClusterStatus(ClusterMetricsBuilder.toOptions(req.getOptionsList()))));
     } catch (IOException e) {
       throw new ServiceException(e);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java
index 370f1f2..96b31c7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java
@@ -31,7 +31,6 @@ import java.util.Random;
 import java.util.Scanner;
 import java.util.Set;
 import java.util.TreeMap;
-
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.GnuParser;
 import org.apache.commons.cli.HelpFormatter;
@@ -40,7 +39,7 @@ import org.apache.commons.cli.ParseException;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ServerName;
@@ -58,6 +57,7 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface;

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
index 923a0a7..f84391f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.RegionLoad;
 import org.apache.hadoop.hbase.ServerLoad;
+import org.apache.hadoop.hbase.ServerMetricsBuilder;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.YouAreDeadException;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
@@ -244,7 +245,7 @@ public class ServerManager {
       request.getServerStartCode());
     checkClockSkew(sn, request.getServerCurrentTime());
     checkIsDead(sn, "STARTUP");
-    if (!checkAndRecordNewServer(sn, ServerLoad.EMPTY_SERVERLOAD)) {
+    if (!checkAndRecordNewServer(sn, new ServerLoad(ServerMetricsBuilder.of(sn)))) {
       LOG.warn("THIS SHOULD NOT HAPPEN, RegionServerStartup"
         + " could not record the server: " + sn);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 9bb8013..8ead08c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -1614,7 +1614,7 @@ public class HRegionServer extends HasThread implements
     int storefileSizeMB = 0;
     int memstoreSizeMB = (int) (r.getMemStoreSize() / 1024 / 1024);
     long storefileIndexSizeKB = 0;
-    int rootIndexSizeKB = 0;
+    int rootLevelIndexSizeKB = 0;
     int totalStaticIndexSizeKB = 0;
     int totalStaticBloomSizeKB = 0;
     long totalCompactingKVs = 0;
@@ -1625,13 +1625,14 @@ public class HRegionServer extends HasThread implements
       storefiles += store.getStorefilesCount();
       storeUncompressedSizeMB += (int) (store.getStoreSizeUncompressed() / 1024 / 1024);
       storefileSizeMB += (int) (store.getStorefilesSize() / 1024 / 1024);
-      storefileIndexSizeKB += store.getStorefilesIndexSize() / 1024;
+      //TODO: storefileIndexSizeKB is same with rootLevelIndexSizeKB?
+      storefileIndexSizeKB += store.getStorefilesRootLevelIndexSize() / 1024;
       CompactionProgress progress = store.getCompactionProgress();
       if (progress != null) {
         totalCompactingKVs += progress.totalCompactingKVs;
         currentCompactedKVs += progress.currentCompactedKVs;
       }
-      rootIndexSizeKB += (int) (store.getStorefilesIndexSize() / 1024);
+      rootLevelIndexSizeKB += (int) (store.getStorefilesRootLevelIndexSize() / 1024);
       totalStaticIndexSizeKB += (int) (store.getTotalStaticIndexSize() / 1024);
       totalStaticBloomSizeKB += (int) (store.getTotalStaticBloomSize() / 1024);
     }
@@ -1653,7 +1654,7 @@ public class HRegionServer extends HasThread implements
       .setStorefileSizeMB(storefileSizeMB)
       .setMemStoreSizeMB(memstoreSizeMB)
       .setStorefileIndexSizeKB(storefileIndexSizeKB)
-      .setRootIndexSizeKB(rootIndexSizeKB)
+      .setRootIndexSizeKB(rootLevelIndexSizeKB)
       .setTotalStaticIndexSizeKB(totalStaticIndexSizeKB)
       .setTotalStaticBloomSizeKB(totalStaticBloomSizeKB)
       .setReadRequestsCount(r.getReadRequestsCount())

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index a5d4b4d..95ca9dc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -2084,7 +2084,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
   }
 
   @Override
-  public long getStorefilesIndexSize() {
+  public long getStorefilesRootLevelIndexSize() {
     return getStoreFileFieldSize(StoreFileReader::indexSize);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
index b643ecf..eaaa4ae 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
@@ -791,7 +791,7 @@ class MetricsRegionServerWrapperImpl
               avgAgeNumerator += storeAvgStoreFileAge.getAsDouble() * storeHFiles;
             }
 
-            tempStorefileIndexSize += store.getStorefilesIndexSize();
+            tempStorefileIndexSize += store.getStorefilesRootLevelIndexSize();
             tempTotalStaticBloomSize += store.getTotalStaticBloomSize();
             tempTotalStaticIndexSize += store.getTotalStaticIndexSize();
             tempFlushedCellsCount += store.getFlushedCellsCount();

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
index d60de6b..042129f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
@@ -172,9 +172,9 @@ public interface Store {
   long getHFilesSize();
 
   /**
-   * @return The size of the store file indexes, in bytes.
+   * @return The size of the store file root-level indexes, in bytes.
    */
-  long getStorefilesIndexSize();
+  long getStorefilesRootLevelIndexSize();
 
   /**
    * Returns the total size of all index blocks in the data block indexes, including the root level,

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
index 969a757..5222019 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
@@ -47,13 +47,12 @@ import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.LongAdder;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
-
 import org.apache.commons.lang3.time.StopWatch;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.AuthUtil;
 import org.apache.hadoop.hbase.ChoreService;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.ClusterStatus;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -98,6 +97,7 @@ import org.apache.zookeeper.client.ConnectStringParser;
 import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index 5e7d728..c96e3e8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -58,7 +58,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
-
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.commons.lang3.StringUtils;
@@ -73,8 +72,8 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.ClusterStatus;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
@@ -569,7 +568,7 @@ public class HBaseFsck extends Configured implements Closeable {
     errors.print("Number of requests: " + status.getRequestCount());
     errors.print("Number of regions: " + status.getRegionsCount());
 
-    List<RegionState> rits = status.getRegionsInTransition();
+    List<RegionState> rits = status.getRegionStatesInTransition();
     errors.print("Number of regions in transition: " + rits.size());
     if (details) {
       for (RegionState state: rits) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
index b8811c7..64f2766 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
@@ -23,10 +23,9 @@ import java.util.Collection;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Random;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -120,7 +119,7 @@ public class HBaseFsckRepair {
       try {
         boolean inTransition = false;
         for (RegionState rs : admin.getClusterStatus(EnumSet.of(Option.REGIONS_IN_TRANSITION))
-                                   .getRegionsInTransition()) {
+                                   .getRegionStatesInTransition()) {
           if (RegionInfo.COMPARATOR.compare(rs.getRegion(), region) == 0) {
             inTransition = true;
             break;

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java
index 711507b..75c7dd5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java
@@ -41,10 +41,9 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
-
 import org.apache.commons.cli.CommandLine;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ServerName;

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java
index 5f480a5..ecc87fb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.util;
 
 import java.io.IOException;
 import java.math.BigInteger;
-
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.EnumSet;
@@ -42,10 +41,8 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.ClusterStatus;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -53,6 +50,8 @@ import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -434,7 +433,7 @@ public class RegionSplitter {
    * Alternative getCurrentNrHRS which is no longer available.
    * @param connection
    * @return Rough count of regionservers out on cluster.
-   * @throws IOException
+   * @throws IOException if a remote or network exception occurs
    */
   private static int getRegionServerCount(final Connection connection) throws IOException {
     try (Admin admin = connection.getAdmin()) {
@@ -785,7 +784,7 @@ public class RegionSplitter {
    * @param conf
    * @param tableName
    * @return A Pair where first item is table dir and second is the split file.
-   * @throws IOException
+   * @throws IOException if a remote or network exception occurs
    */
   private static Pair<Path, Path> getTableDirAndSplitFile(final Configuration conf,
       final TableName tableName)

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index e5cd13a..9e17a79 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -60,7 +60,7 @@ import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
 import org.apache.hadoop.hbase.Waiter.Predicate;
 import org.apache.hadoop.hbase.client.Admin;

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java
new file mode 100644
index 0000000..ac116d8
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java
@@ -0,0 +1,233 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Stream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
+import org.apache.hadoop.hbase.Waiter.Predicate;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.AsyncAdmin;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
+import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.MasterObserver;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
+import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestClientClusterMetrics {
+  private static HBaseTestingUtility UTIL;
+  private static Admin ADMIN;
+  private final static int SLAVES = 5;
+  private final static int MASTERS = 3;
+  private static MiniHBaseCluster CLUSTER;
+  private static HRegionServer DEAD;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, MyObserver.class.getName());
+    UTIL = new HBaseTestingUtility(conf);
+    UTIL.startMiniCluster(MASTERS, SLAVES);
+    CLUSTER = UTIL.getHBaseCluster();
+    CLUSTER.waitForActiveAndReadyMaster();
+    ADMIN = UTIL.getAdmin();
+    // Kill one region server
+    List<RegionServerThread> rsts = CLUSTER.getLiveRegionServerThreads();
+    RegionServerThread rst = rsts.get(rsts.size() - 1);
+    DEAD = rst.getRegionServer();
+    DEAD.stop("Test dead servers metrics");
+    while (rst.isAlive()) {
+      Thread.sleep(500);
+    }
+  }
+
+  @Test
+  public void testDefaults() throws Exception {
+    ClusterMetrics origin = ADMIN.getClusterStatus();
+    ClusterMetrics defaults = ADMIN.getClusterStatus(EnumSet.allOf(Option.class));
+    Assert.assertEquals(origin.getHBaseVersion(), defaults.getHBaseVersion());
+    Assert.assertEquals(origin.getClusterId(), defaults.getClusterId());
+    Assert.assertEquals(origin.getAverageLoad(), defaults.getAverageLoad(), 0);
+    Assert.assertEquals(origin.getBackupMasterNames().size(),
+        defaults.getBackupMasterNames().size());
+    Assert.assertEquals(origin.getDeadServerNames().size(), defaults.getDeadServerNames().size());
+    Assert.assertEquals(origin.getRegionCount(), defaults.getRegionCount());
+    Assert.assertEquals(origin.getLiveServerMetrics().size(),
+        defaults.getLiveServerMetrics().size());
+    Assert.assertEquals(origin.getMasterInfoPort(), defaults.getMasterInfoPort());
+  }
+
+  @Test
+  public void testAsyncClient() throws Exception {
+    try (AsyncConnection asyncConnect = ConnectionFactory.createAsyncConnection(
+      UTIL.getConfiguration()).get()) {
+      AsyncAdmin asyncAdmin = asyncConnect.getAdmin();
+      CompletableFuture<ClusterStatus> originFuture =
+        asyncAdmin.getClusterStatus();
+      CompletableFuture<ClusterStatus> defaultsFuture =
+        asyncAdmin.getClusterStatus(EnumSet.allOf(Option.class));
+      ClusterMetrics origin = originFuture.get();
+      ClusterMetrics defaults = defaultsFuture.get();
+      Assert.assertEquals(origin.getHBaseVersion(), defaults.getHBaseVersion());
+      Assert.assertEquals(origin.getClusterId(), defaults.getClusterId());
+      Assert.assertEquals(origin.getHBaseVersion(), defaults.getHBaseVersion());
+      Assert.assertEquals(origin.getClusterId(), defaults.getClusterId());
+      Assert.assertEquals(origin.getAverageLoad(), defaults.getAverageLoad(), 0);
+      Assert.assertEquals(origin.getBackupMasterNames().size(),
+        defaults.getBackupMasterNames().size());
+      Assert.assertEquals(origin.getDeadServerNames().size(), defaults.getDeadServerNames().size());
+      Assert.assertEquals(origin.getRegionCount(), defaults.getRegionCount());
+      Assert.assertEquals(origin.getLiveServerMetrics().size(),
+        defaults.getLiveServerMetrics().size());
+      Assert.assertEquals(origin.getMasterInfoPort(), defaults.getMasterInfoPort());
+    }
+  }
+
+  @Test
+  public void testLiveAndDeadServersStatus() throws Exception {
+    // Count the number of live regionservers
+    List<RegionServerThread> regionserverThreads = CLUSTER.getLiveRegionServerThreads();
+    int numRs = 0;
+    int len = regionserverThreads.size();
+    for (int i = 0; i < len; i++) {
+      if (regionserverThreads.get(i).isAlive()) {
+        numRs++;
+      }
+    }
+    // Depending on the (random) order of unit execution we may run this unit before the
+    // minicluster is fully up and recovered from the RS shutdown done during test init.
+    Waiter.waitFor(CLUSTER.getConfiguration(), 10 * 1000, 100, new Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        ClusterMetrics metrics = ADMIN.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
+        Assert.assertNotNull(metrics);
+        return metrics.getRegionCount() > 0;
+      }
+    });
+    // Retrieve live servers and dead servers info.
+    EnumSet<Option> options = EnumSet.of(Option.LIVE_SERVERS, Option.DEAD_SERVERS);
+    ClusterMetrics metrics = ADMIN.getClusterStatus(options);
+    Assert.assertNotNull(metrics);
+    // exclude a dead region server
+    Assert.assertEquals(SLAVES -1, numRs);
+    // live servers = nums of regionservers
+    // By default, HMaster don't carry any regions so it won't report its load.
+    // Hence, it won't be in the server list.
+    Assert.assertEquals(numRs, metrics.getLiveServerMetrics().size());
+    Assert.assertTrue(metrics.getRegionCount() > 0);
+    Assert.assertNotNull(metrics.getDeadServerNames());
+    Assert.assertEquals(1, metrics.getDeadServerNames().size());
+    ServerName deadServerName = metrics.getDeadServerNames().iterator().next();
+    Assert.assertEquals(DEAD.getServerName(), deadServerName);
+  }
+
+  @Test
+  public void testMasterAndBackupMastersStatus() throws Exception {
+    // get all the master threads
+    List<MasterThread> masterThreads = CLUSTER.getMasterThreads();
+    int numActive = 0;
+    int activeIndex = 0;
+    ServerName activeName = null;
+    HMaster active = null;
+    for (int i = 0; i < masterThreads.size(); i++) {
+      if (masterThreads.get(i).getMaster().isActiveMaster()) {
+        numActive++;
+        activeIndex = i;
+        active = masterThreads.get(activeIndex).getMaster();
+        activeName = active.getServerName();
+      }
+    }
+    Assert.assertNotNull(active);
+    Assert.assertEquals(1, numActive);
+    Assert.assertEquals(MASTERS, masterThreads.size());
+    // Retrieve master and backup masters infos only.
+    EnumSet<Option> options = EnumSet.of(Option.MASTER, Option.BACKUP_MASTERS);
+    ClusterMetrics metrics = ADMIN.getClusterStatus(options);
+    Assert.assertTrue(metrics.getMasterName().equals(activeName));
+    Assert.assertEquals(MASTERS - 1, metrics.getBackupMasterNames().size());
+  }
+
+  @Test
+  public void testOtherStatusInfos() throws Exception {
+    EnumSet<Option> options =
+        EnumSet.of(Option.MASTER_COPROCESSORS, Option.HBASE_VERSION,
+                   Option.CLUSTER_ID, Option.BALANCER_ON);
+    ClusterMetrics metrics = ADMIN.getClusterStatus(options);
+    Assert.assertEquals(1, metrics.getMasterCoprocessorNames().size());
+    Assert.assertNotNull(metrics.getHBaseVersion());
+    Assert.assertNotNull(metrics.getClusterId());
+    Assert.assertTrue(metrics.getAverageLoad() == 0.0);
+    Assert.assertNotNull(metrics.getBalancerOn());
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    if (ADMIN != null) {
+      ADMIN.close();
+    }
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testObserver() throws IOException {
+    int preCount = MyObserver.PRE_COUNT.get();
+    int postCount = MyObserver.POST_COUNT.get();
+    Assert.assertTrue(Stream.of(ADMIN.getClusterStatus().getMasterCoprocessors())
+        .anyMatch(s -> s.equals(MyObserver.class.getSimpleName())));
+    Assert.assertEquals(preCount + 1, MyObserver.PRE_COUNT.get());
+    Assert.assertEquals(postCount + 1, MyObserver.POST_COUNT.get());
+  }
+
+  public static class MyObserver implements MasterCoprocessor, MasterObserver {
+    private static final AtomicInteger PRE_COUNT = new AtomicInteger(0);
+    private static final AtomicInteger POST_COUNT = new AtomicInteger(0);
+
+    @Override public Optional<MasterObserver> getMasterObserver() {
+      return Optional.of(this);
+    }
+
+    @Override public void preGetClusterStatus(ObserverContext<MasterCoprocessorEnvironment> ctx)
+        throws IOException {
+      PRE_COUNT.incrementAndGet();
+    }
+
+    @Override public void postGetClusterStatus(ObserverContext<MasterCoprocessorEnvironment> ctx,
+        ClusterStatus metrics) throws IOException {
+      POST_COUNT.incrementAndGet();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java
new file mode 100644
index 0000000..245f3c9
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java
@@ -0,0 +1,258 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Stream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
+import org.apache.hadoop.hbase.Waiter.Predicate;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.AsyncAdmin;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
+import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.MasterObserver;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
+import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test the ClusterStatus.
+ */
+@Category(SmallTests.class)
+public class TestClientClusterStatus {
+  private static HBaseTestingUtility UTIL;
+  private static Admin ADMIN;
+  private final static int SLAVES = 5;
+  private final static int MASTERS = 3;
+  private static MiniHBaseCluster CLUSTER;
+  private static HRegionServer DEAD;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, MyObserver.class.getName());
+    UTIL = new HBaseTestingUtility(conf);
+    UTIL.startMiniCluster(MASTERS, SLAVES);
+    CLUSTER = UTIL.getHBaseCluster();
+    CLUSTER.waitForActiveAndReadyMaster();
+    ADMIN = UTIL.getAdmin();
+    // Kill one region server
+    List<RegionServerThread> rsts = CLUSTER.getLiveRegionServerThreads();
+    RegionServerThread rst = rsts.get(rsts.size() - 1);
+    DEAD = rst.getRegionServer();
+    DEAD.stop("Test dead servers status");
+    while (rst.isAlive()) {
+      Thread.sleep(500);
+    }
+  }
+
+  @Test
+  public void testDefaults() throws Exception {
+    ClusterStatus origin = ADMIN.getClusterStatus();
+    ClusterStatus defaults = ADMIN.getClusterStatus(EnumSet.allOf(Option.class));
+    checkPbObjectNotNull(origin);
+    checkPbObjectNotNull(defaults);
+    Assert.assertEquals(origin.getHBaseVersion(), defaults.getHBaseVersion());
+    Assert.assertEquals(origin.getClusterId(), defaults.getClusterId());
+    Assert.assertTrue(origin.getAverageLoad() == defaults.getAverageLoad());
+    Assert.assertTrue(origin.getBackupMastersSize() == defaults.getBackupMastersSize());
+    Assert.assertTrue(origin.getDeadServersSize() == defaults.getDeadServersSize());
+    Assert.assertTrue(origin.getRegionsCount() == defaults.getRegionsCount());
+    Assert.assertTrue(origin.getServersSize() == defaults.getServersSize());
+    Assert.assertTrue(origin.getMasterInfoPort() == defaults.getMasterInfoPort());
+    Assert.assertTrue(origin.equals(defaults));
+  }
+
+  @Test
+  public void testNone() throws Exception {
+    ClusterStatus status0 = ADMIN.getClusterStatus(EnumSet.allOf(Option.class));
+    ClusterStatus status1 = ADMIN.getClusterStatus(EnumSet.noneOf(Option.class));
+    Assert.assertEquals(status0, status1);
+    checkPbObjectNotNull(status0);
+    checkPbObjectNotNull(status1);
+  }
+
+  @Test
+  public void testAsyncClient() throws Exception {
+    try (AsyncConnection asyncConnect = ConnectionFactory.createAsyncConnection(
+      UTIL.getConfiguration()).get()) {
+      AsyncAdmin asyncAdmin = asyncConnect.getAdmin();
+      CompletableFuture<ClusterStatus> originFuture =
+        asyncAdmin.getClusterStatus();
+      CompletableFuture<ClusterStatus> defaultsFuture =
+        asyncAdmin.getClusterStatus(EnumSet.allOf(Option.class));
+      ClusterStatus origin = originFuture.get();
+      ClusterStatus defaults = defaultsFuture.get();
+      checkPbObjectNotNull(origin);
+      checkPbObjectNotNull(defaults);
+      Assert.assertEquals(origin.getHBaseVersion(), defaults.getHBaseVersion());
+      Assert.assertEquals(origin.getClusterId(), defaults.getClusterId());
+      Assert.assertTrue(origin.getAverageLoad() == defaults.getAverageLoad());
+      Assert.assertTrue(origin.getBackupMastersSize() == defaults.getBackupMastersSize());
+      Assert.assertTrue(origin.getDeadServersSize() == defaults.getDeadServersSize());
+      Assert.assertTrue(origin.getRegionsCount() == defaults.getRegionsCount());
+      Assert.assertTrue(origin.getServersSize() == defaults.getServersSize());
+    }
+  }
+
+  @Test
+  public void testLiveAndDeadServersStatus() throws Exception {
+    // Count the number of live regionservers
+    List<RegionServerThread> regionserverThreads = CLUSTER.getLiveRegionServerThreads();
+    int numRs = 0;
+    int len = regionserverThreads.size();
+    for (int i = 0; i < len; i++) {
+      if (regionserverThreads.get(i).isAlive()) {
+        numRs++;
+      }
+    }
+    // Depending on the (random) order of unit execution we may run this unit before the
+    // minicluster is fully up and recovered from the RS shutdown done during test init.
+    Waiter.waitFor(CLUSTER.getConfiguration(), 10 * 1000, 100, new Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        ClusterStatus status = ADMIN.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
+        Assert.assertNotNull(status);
+        return status.getRegionsCount() > 0;
+      }
+    });
+    // Retrieve live servers and dead servers info.
+    EnumSet<Option> options = EnumSet.of(Option.LIVE_SERVERS, Option.DEAD_SERVERS);
+    ClusterStatus status = ADMIN.getClusterStatus(options);
+    checkPbObjectNotNull(status);
+    Assert.assertNotNull(status);
+    Assert.assertNotNull(status.getServers());
+    // exclude a dead region server
+    Assert.assertEquals(SLAVES -1, numRs);
+    // live servers = nums of regionservers
+    // By default, HMaster don't carry any regions so it won't report its load.
+    // Hence, it won't be in the server list.
+    Assert.assertEquals(status.getServers().size(), numRs);
+    Assert.assertTrue(status.getRegionsCount() > 0);
+    Assert.assertNotNull(status.getDeadServerNames());
+    Assert.assertEquals(1, status.getDeadServersSize());
+    ServerName deadServerName = status.getDeadServerNames().iterator().next();
+    Assert.assertEquals(DEAD.getServerName(), deadServerName);
+  }
+
+  @Test
+  public void testMasterAndBackupMastersStatus() throws Exception {
+    // get all the master threads
+    List<MasterThread> masterThreads = CLUSTER.getMasterThreads();
+    int numActive = 0;
+    int activeIndex = 0;
+    ServerName activeName = null;
+    HMaster active = null;
+    for (int i = 0; i < masterThreads.size(); i++) {
+      if (masterThreads.get(i).getMaster().isActiveMaster()) {
+        numActive++;
+        activeIndex = i;
+        active = masterThreads.get(activeIndex).getMaster();
+        activeName = active.getServerName();
+      }
+    }
+    Assert.assertNotNull(active);
+    Assert.assertEquals(1, numActive);
+    Assert.assertEquals(MASTERS, masterThreads.size());
+    // Retrieve master and backup masters infos only.
+    EnumSet<Option> options = EnumSet.of(Option.MASTER, Option.BACKUP_MASTERS);
+    ClusterStatus status = ADMIN.getClusterStatus(options);
+    Assert.assertTrue(status.getMaster().equals(activeName));
+    Assert.assertEquals(MASTERS - 1, status.getBackupMastersSize());
+  }
+
+  @Test
+  public void testOtherStatusInfos() throws Exception {
+    EnumSet<Option> options =
+        EnumSet.of(Option.MASTER_COPROCESSORS, Option.HBASE_VERSION,
+                   Option.CLUSTER_ID, Option.BALANCER_ON);
+    ClusterStatus status = ADMIN.getClusterStatus(options);
+    Assert.assertTrue(status.getMasterCoprocessors().length == 1);
+    Assert.assertNotNull(status.getHBaseVersion());
+    Assert.assertNotNull(status.getClusterId());
+    Assert.assertTrue(status.getAverageLoad() == 0.0);
+    Assert.assertNotNull(status.getBalancerOn());
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    if (ADMIN != null) ADMIN.close();
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testObserver() throws IOException {
+    int preCount = MyObserver.PRE_COUNT.get();
+    int postCount = MyObserver.POST_COUNT.get();
+    Assert.assertTrue(Stream.of(ADMIN.getClusterStatus().getMasterCoprocessors())
+        .anyMatch(s -> s.equals(MyObserver.class.getSimpleName())));
+    Assert.assertEquals(preCount + 1, MyObserver.PRE_COUNT.get());
+    Assert.assertEquals(postCount + 1, MyObserver.POST_COUNT.get());
+  }
+
+  /**
+   * HBASE-19496 do the refactor for ServerLoad and RegionLoad so the inner pb object is useless
+   * now. However, they are Public classes, and consequently we must make sure the all pb objects
+   * have initialized.
+   */
+  private static void checkPbObjectNotNull(ClusterStatus status) {
+    for (ServerName name : status.getLiveServerMetrics().keySet()) {
+      ServerLoad load = status.getLoad(name);
+      Assert.assertNotNull(load.obtainServerLoadPB());
+      for (RegionLoad rl : load.getRegionsLoad().values()) {
+        Assert.assertNotNull(rl.regionLoadPB);
+      }
+    }
+  }
+
+  public static class MyObserver implements MasterCoprocessor, MasterObserver {
+    private static final AtomicInteger PRE_COUNT = new AtomicInteger(0);
+    private static final AtomicInteger POST_COUNT = new AtomicInteger(0);
+
+    @Override public Optional<MasterObserver> getMasterObserver() {
+      return Optional.of(this);
+    }
+
+    @Override public void preGetClusterStatus(ObserverContext<MasterCoprocessorEnvironment> ctx)
+        throws IOException {
+      PRE_COUNT.incrementAndGet();
+    }
+
+    @Override public void postGetClusterStatus(ObserverContext<MasterCoprocessorEnvironment> ctx,
+        ClusterStatus status) throws IOException {
+      POST_COUNT.incrementAndGet();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionLoad.java
index d482484..d454c50 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionLoad.java
@@ -19,11 +19,16 @@
  */
 package org.apache.hadoop.hbase;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -34,11 +39,8 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.io.IOException;
-import java.util.Collection;
-import java.util.EnumSet;
-import java.util.List;
-import java.util.Map;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
 
 @Category({MiscTests.class, MediumTests.class})
 public class TestRegionLoad {
@@ -124,6 +126,10 @@ public class TestRegionLoad {
   private void checkRegionsAndRegionLoads(Collection<HRegionInfo> regions,
       Collection<RegionLoad> regionLoads) {
 
+    for (RegionLoad load : regionLoads) {
+      assertNotNull(load.regionLoadPB);
+    }
+
     assertEquals("No of regions and regionloads doesn't match",
         regions.size(), regionLoads.size());
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionMetrics.java
new file mode 100644
index 0000000..e731807
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionMetrics.java
@@ -0,0 +1,130 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
+
+@Category({ MiscTests.class, MediumTests.class })
+public class TestRegionMetrics {
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+  private static Admin admin;
+
+  private static final TableName TABLE_1 = TableName.valueOf("table_1");
+  private static final TableName TABLE_2 = TableName.valueOf("table_2");
+  private static final TableName TABLE_3 = TableName.valueOf("table_3");
+  private static final TableName[] tables = new TableName[] { TABLE_1, TABLE_2, TABLE_3 };
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    UTIL.startMiniCluster(4);
+    admin = UTIL.getAdmin();
+    admin.balancerSwitch(false, true);
+    byte[] FAMILY = Bytes.toBytes("f");
+    for (TableName tableName : tables) {
+      Table table = UTIL.createMultiRegionTable(tableName, FAMILY, 16);
+      UTIL.waitTableAvailable(tableName);
+      UTIL.loadTable(table, FAMILY);
+    }
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    for (TableName table : tables) {
+      UTIL.deleteTableIfAny(table);
+    }
+    UTIL.shutdownMiniCluster();
+  }
+
+
+  @Test
+  public void testRegionMetrics() throws Exception {
+
+    // Check if regions match with the RegionMetrics from the server
+    for (ServerName serverName : admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS))
+        .getLiveServerMetrics().keySet()) {
+      List<RegionInfo> regions = admin.getRegions(serverName);
+      Collection<RegionMetrics> regionMetricsList =
+          admin.getRegionLoads(serverName).stream().collect(Collectors.toList());
+      checkRegionsAndRegionMetrics(regions, regionMetricsList);
+    }
+
+    // Check if regionMetrics matches the table's regions and nothing is missed
+    for (TableName table : new TableName[] { TABLE_1, TABLE_2, TABLE_3 }) {
+      List<RegionInfo> tableRegions = admin.getRegions(table);
+
+      List<RegionMetrics> regionMetrics = new ArrayList<>();
+      for (ServerName serverName : admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS))
+          .getLiveServerMetrics().keySet()) {
+        regionMetrics.addAll(admin.getRegionLoads(serverName, table));
+      }
+      checkRegionsAndRegionMetrics(tableRegions, regionMetrics);
+    }
+
+    // Check RegionMetrics matches the RegionMetrics from ClusterStatus
+    ClusterMetrics clusterStatus = admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
+    for (Map.Entry<ServerName, ServerMetrics> entry : clusterStatus.getLiveServerMetrics()
+        .entrySet()) {
+      ServerName serverName = entry.getKey();
+      ServerMetrics serverMetrics = entry.getValue();
+      List<RegionMetrics> regionMetrics =
+          admin.getRegionLoads(serverName).stream().collect(Collectors.toList());
+      assertEquals(serverMetrics.getRegionMetrics().size(), regionMetrics.size());
+    }
+  }
+
+  private void checkRegionsAndRegionMetrics(Collection<RegionInfo> regions,
+      Collection<RegionMetrics> regionMetrics) {
+
+    assertEquals("No of regions and regionMetrics doesn't match", regions.size(),
+        regionMetrics.size());
+
+    Map<byte[], RegionMetrics> regionMetricsMap = Maps.newTreeMap(Bytes.BYTES_COMPARATOR);
+    for (RegionMetrics r : regionMetrics) {
+      regionMetricsMap.put(r.getRegionName(), r);
+    }
+    for (RegionInfo info : regions) {
+      assertTrue("Region not in RegionMetricsMap region:"
+          + info.getRegionNameAsString() + " regionMap: "
+          + regionMetricsMap, regionMetricsMap.containsKey(info.getRegionName()));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerLoad.java
index 84d509b..b81a64c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerLoad.java
@@ -20,42 +20,44 @@
 
 package org.apache.hadoop.hbase;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 
-@Category({MiscTests.class, SmallTests.class})
+@Category({ MiscTests.class, SmallTests.class })
 public class TestServerLoad {
 
   @Test
   public void testRegionLoadAggregation() {
-    ServerLoad sl = new ServerLoad(createServerLoadProto());
+    ServerLoad sl = new ServerLoad(ServerName.valueOf("localhost,1,1"), createServerLoadProto());
     assertEquals(13, sl.getStores());
     assertEquals(114, sl.getStorefiles());
     assertEquals(129, sl.getStoreUncompressedSizeMB());
     assertEquals(504, sl.getRootIndexSizeKB());
     assertEquals(820, sl.getStorefileSizeMB());
     assertEquals(82, sl.getStorefileIndexSizeKB());
-    assertEquals(((long)Integer.MAX_VALUE)*2, sl.getReadRequestsCount());
+    assertEquals(((long) Integer.MAX_VALUE) * 2, sl.getReadRequestsCount());
     assertEquals(300, sl.getFilteredReadRequestsCount());
-    
   }
- 
+
   @Test
   public void testToString() {
-    ServerLoad sl = new ServerLoad(createServerLoadProto());
+    ServerLoad sl = new ServerLoad(ServerName.valueOf("localhost,1,1"), createServerLoadProto());
     String slToString = sl.toString();
+    assertNotNull(sl.obtainServerLoadPB());
     assertTrue(slToString.contains("numberOfStores=13"));
     assertTrue(slToString.contains("numberOfStorefiles=114"));
     assertTrue(slToString.contains("storefileUncompressedSizeMB=129"));
-    assertTrue(slToString.contains("storefileSizeMB=820")); 
+    assertTrue(slToString.contains("storefileSizeMB=820"));
     assertTrue(slToString.contains("rootIndexSizeKB=504"));
     assertTrue(slToString.contains("coprocessors=[]"));
     assertTrue(slToString.contains("filteredReadRequestsCount=300"));
@@ -63,36 +65,35 @@ public class TestServerLoad {
 
   @Test
   public void testRegionLoadWrapAroundAggregation() {
-	  ServerLoad sl = new ServerLoad(createServerLoadProto());
-	  long totalCount = ((long)Integer.MAX_VALUE)*2;
-	  assertEquals(totalCount, sl.getReadRequestsCount());
-	  assertEquals(totalCount, sl.getWriteRequestsCount());
+    ServerLoad sl = new ServerLoad(ServerName.valueOf("localhost,1,1"), createServerLoadProto());
+    assertNotNull(sl.obtainServerLoadPB());
+    long totalCount = ((long) Integer.MAX_VALUE) * 2;
+    assertEquals(totalCount, sl.getReadRequestsCount());
+    assertEquals(totalCount, sl.getWriteRequestsCount());
   }
 
   private ClusterStatusProtos.ServerLoad createServerLoadProto() {
-    HBaseProtos.RegionSpecifier rSpecOne =
-        HBaseProtos.RegionSpecifier.newBuilder()
-            .setType(HBaseProtos.RegionSpecifier.RegionSpecifierType.ENCODED_REGION_NAME)
-            .setValue(ByteString.copyFromUtf8("ASDFGQWERT")).build();
-    HBaseProtos.RegionSpecifier rSpecTwo =
-        HBaseProtos.RegionSpecifier.newBuilder()
-            .setType(HBaseProtos.RegionSpecifier.RegionSpecifierType.ENCODED_REGION_NAME)
-            .setValue(ByteString.copyFromUtf8("QWERTYUIOP")).build();
+    HBaseProtos.RegionSpecifier rSpecOne = HBaseProtos.RegionSpecifier.newBuilder()
+      .setType(HBaseProtos.RegionSpecifier.RegionSpecifierType.ENCODED_REGION_NAME)
+      .setValue(ByteString.copyFromUtf8("ASDFGQWERT")).build();
+    HBaseProtos.RegionSpecifier rSpecTwo = HBaseProtos.RegionSpecifier.newBuilder()
+      .setType(HBaseProtos.RegionSpecifier.RegionSpecifierType.ENCODED_REGION_NAME)
+      .setValue(ByteString.copyFromUtf8("QWERTYUIOP")).build();
 
     ClusterStatusProtos.RegionLoad rlOne =
-        ClusterStatusProtos.RegionLoad.newBuilder().setRegionSpecifier(rSpecOne).setStores(10)
-            .setStorefiles(101).setStoreUncompressedSizeMB(106).setStorefileSizeMB(520)
-            .setFilteredReadRequestsCount(100)
-            .setStorefileIndexSizeKB(42).setRootIndexSizeKB(201).setReadRequestsCount(Integer.MAX_VALUE).setWriteRequestsCount(Integer.MAX_VALUE).build();
+      ClusterStatusProtos.RegionLoad.newBuilder().setRegionSpecifier(rSpecOne).setStores(10)
+        .setStorefiles(101).setStoreUncompressedSizeMB(106).setStorefileSizeMB(520)
+        .setFilteredReadRequestsCount(100).setStorefileIndexSizeKB(42).setRootIndexSizeKB(201)
+        .setReadRequestsCount(Integer.MAX_VALUE).setWriteRequestsCount(Integer.MAX_VALUE).build();
     ClusterStatusProtos.RegionLoad rlTwo =
-        ClusterStatusProtos.RegionLoad.newBuilder().setRegionSpecifier(rSpecTwo).setStores(3)
-            .setStorefiles(13).setStoreUncompressedSizeMB(23).setStorefileSizeMB(300)
-            .setFilteredReadRequestsCount(200)
-            .setStorefileIndexSizeKB(40).setRootIndexSizeKB(303).setReadRequestsCount(Integer.MAX_VALUE).setWriteRequestsCount(Integer.MAX_VALUE).build();
+      ClusterStatusProtos.RegionLoad.newBuilder().setRegionSpecifier(rSpecTwo).setStores(3)
+        .setStorefiles(13).setStoreUncompressedSizeMB(23).setStorefileSizeMB(300)
+        .setFilteredReadRequestsCount(200).setStorefileIndexSizeKB(40).setRootIndexSizeKB(303)
+        .setReadRequestsCount(Integer.MAX_VALUE).setWriteRequestsCount(Integer.MAX_VALUE).build();
 
     ClusterStatusProtos.ServerLoad sl =
-        ClusterStatusProtos.ServerLoad.newBuilder().addRegionLoads(rlOne).
-          addRegionLoads(rlTwo).build();
+      ClusterStatusProtos.ServerLoad.newBuilder().addRegionLoads(rlOne).
+        addRegionLoads(rlTwo).build();
     return sl;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerMetrics.java
new file mode 100644
index 0000000..68a4360
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerMetrics.java
@@ -0,0 +1,114 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+
+@Category({ MiscTests.class, SmallTests.class })
+public class TestServerMetrics {
+
+  @Test
+  public void testRegionLoadAggregation() {
+    ServerMetrics metrics = ServerMetricsBuilder.toServerMetrics(
+        ServerName.valueOf("localhost,1,1"), createServerLoadProto());
+    assertEquals(13,
+        metrics.getRegionMetrics().values().stream().mapToInt(v -> v.getStoreCount()).sum());
+    assertEquals(114,
+        metrics.getRegionMetrics().values().stream().mapToInt(v -> v.getStoreFileCount()).sum());
+    assertEquals(129, metrics.getRegionMetrics().values().stream()
+        .mapToDouble(v -> v.getUncompressedStoreFileSize().get(Size.Unit.MEGABYTE)).sum(), 0);
+    assertEquals(504, metrics.getRegionMetrics().values().stream()
+        .mapToDouble(v -> v.getStoreFileRootLevelIndexSize().get(Size.Unit.KILOBYTE)).sum(), 0);
+    assertEquals(820, metrics.getRegionMetrics().values().stream()
+        .mapToDouble(v -> v.getStoreFileSize().get(Size.Unit.MEGABYTE)).sum(), 0);
+    assertEquals(82, metrics.getRegionMetrics().values().stream()
+        .mapToDouble(v -> v.getStoreFileIndexSize().get(Size.Unit.KILOBYTE)).sum(), 0);
+    assertEquals(((long) Integer.MAX_VALUE) * 2,
+        metrics.getRegionMetrics().values().stream().mapToLong(v -> v.getReadRequestCount()).sum());
+    assertEquals(300,
+        metrics.getRegionMetrics().values().stream().mapToLong(v -> v.getFilteredReadRequestCount())
+            .sum());
+  }
+
+  @Test
+  public void testToString() {
+    ServerMetrics metrics = ServerMetricsBuilder.toServerMetrics(
+        ServerName.valueOf("localhost,1,1"), createServerLoadProto());
+    String slToString = metrics.toString();
+    assertTrue(slToString.contains("numberOfStores=13"));
+    assertTrue(slToString.contains("numberOfStorefiles=114"));
+    assertTrue(slToString.contains("storefileUncompressedSizeMB=129"));
+    assertTrue(slToString.contains("storefileSizeMB=820"));
+    assertTrue(slToString.contains("rootIndexSizeKB=504"));
+    assertTrue(slToString.contains("coprocessors=[]"));
+    assertTrue(slToString.contains("filteredReadRequestsCount=300"));
+  }
+
+  @Test
+  public void testRegionLoadWrapAroundAggregation() {
+    ServerMetrics metrics = ServerMetricsBuilder.toServerMetrics(
+        ServerName.valueOf("localhost,1,1"), createServerLoadProto());
+    long totalCount = ((long) Integer.MAX_VALUE) * 2;
+    assertEquals(totalCount,
+        metrics.getRegionMetrics().values().stream().mapToLong(v -> v.getReadRequestCount()).sum());
+    assertEquals(totalCount,
+        metrics.getRegionMetrics().values().stream().mapToLong(v -> v.getWriteRequestCount())
+            .sum());
+  }
+
+  private ClusterStatusProtos.ServerLoad createServerLoadProto() {
+    HBaseProtos.RegionSpecifier rSpecOne = HBaseProtos.RegionSpecifier.newBuilder()
+        .setType(HBaseProtos.RegionSpecifier.RegionSpecifierType.ENCODED_REGION_NAME)
+        .setValue(ByteString.copyFromUtf8("ASDFGQWERT")).build();
+    HBaseProtos.RegionSpecifier rSpecTwo = HBaseProtos.RegionSpecifier.newBuilder()
+        .setType(HBaseProtos.RegionSpecifier.RegionSpecifierType.ENCODED_REGION_NAME)
+        .setValue(ByteString.copyFromUtf8("QWERTYUIOP")).build();
+
+    ClusterStatusProtos.RegionLoad rlOne =
+        ClusterStatusProtos.RegionLoad.newBuilder().setRegionSpecifier(rSpecOne).setStores(10)
+            .setStorefiles(101).setStoreUncompressedSizeMB(106).setStorefileSizeMB(520)
+            .setFilteredReadRequestsCount(100).setStorefileIndexSizeKB(42).setRootIndexSizeKB(201)
+            .setReadRequestsCount(Integer.MAX_VALUE).setWriteRequestsCount(Integer.MAX_VALUE)
+            .build();
+    ClusterStatusProtos.RegionLoad rlTwo =
+        ClusterStatusProtos.RegionLoad.newBuilder().setRegionSpecifier(rSpecTwo).setStores(3)
+            .setStorefiles(13).setStoreUncompressedSizeMB(23).setStorefileSizeMB(300)
+            .setFilteredReadRequestsCount(200).setStorefileIndexSizeKB(40).setRootIndexSizeKB(303)
+            .setReadRequestsCount(Integer.MAX_VALUE).setWriteRequestsCount(Integer.MAX_VALUE)
+            .build();
+
+    ClusterStatusProtos.ServerLoad sl =
+        ClusterStatusProtos.ServerLoad.newBuilder().addRegionLoads(rlOne).
+            addRegionLoads(rlTwo).build();
+    return sl;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSize.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSize.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSize.java
new file mode 100644
index 0000000..30d88bd
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSize.java
@@ -0,0 +1,84 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.Set;
+import java.util.TreeSet;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({MiscTests.class, SmallTests.class})
+public class TestSize {
+
+  @Test
+  public void testConversion() {
+    Size kbSize = new Size(1024D, Size.Unit.MEGABYTE);
+    assertEquals(1D, kbSize.get(Size.Unit.GIGABYTE), 0);
+    assertEquals(1024D, kbSize.get(), 0);
+    assertEquals(1024D * 1024D, kbSize.get(Size.Unit.KILOBYTE), 0);
+    assertEquals(1024D * 1024D * 1024D, kbSize.get(Size.Unit.BYTE), 0);
+  }
+
+  @Test
+  public void testCompare() {
+    Size size00 = new Size(100D, Size.Unit.GIGABYTE);
+    Size size01 = new Size(100D, Size.Unit.MEGABYTE);
+    Size size02 = new Size(100D, Size.Unit.BYTE);
+    Set<Size> sizes = new TreeSet<>();
+    sizes.add(size00);
+    sizes.add(size01);
+    sizes.add(size02);
+    int count = 0;
+    for (Size s : sizes) {
+      switch (count++) {
+        case 0:
+          assertEquals(size02, s);
+          break;
+        case 1:
+          assertEquals(size01, s);
+          break;
+        default:
+          assertEquals(size00, s);
+          break;
+      }
+    }
+    assertEquals(3, count);
+  }
+
+  @Test
+  public void testEqual() {
+    assertEquals(new Size(1024D, Size.Unit.TERABYTE),
+      new Size(1D, Size.Unit.PETABYTE));
+    assertEquals(new Size(1024D, Size.Unit.GIGABYTE),
+      new Size(1D, Size.Unit.TERABYTE));
+    assertEquals(new Size(1024D, Size.Unit.MEGABYTE),
+      new Size(1D, Size.Unit.GIGABYTE));
+    assertEquals(new Size(1024D, Size.Unit.KILOBYTE),
+      new Size(1D, Size.Unit.MEGABYTE));
+    assertEquals(new Size(1024D, Size.Unit.BYTE),
+      new Size(1D, Size.Unit.KILOBYTE));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
index fb5febc..d701a81 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
@@ -25,20 +25,14 @@ import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Random;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.stream.Collectors;
-
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -58,7 +52,6 @@ import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -68,13 +61,13 @@ import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java
index da77f29..26f45bc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java
@@ -21,6 +21,8 @@ import static org.apache.hadoop.hbase.client.AsyncProcess.START_LOG_ERRORS_AFTER
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 import java.io.IOException;
 import java.nio.file.FileSystems;
 import java.nio.file.Files;
@@ -31,10 +33,9 @@ import java.util.Collection;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
-
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.ClusterStatus;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.RegionLoad;
@@ -53,9 +54,6 @@ import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-
 @RunWith(Parameterized.class)
 @Category({ ClientTests.class, MediumTests.class })
 public class TestAsyncClusterAdminApi extends TestAsyncAdminBase {

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncDecommissionAdminApi.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncDecommissionAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncDecommissionAdminApi.java
index 856a31a..a3d7616 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncDecommissionAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncDecommissionAdminApi.java
@@ -25,9 +25,8 @@ import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.List;
 import java.util.stream.Collectors;
-
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.junit.Test;


[19/26] hbase git commit: HBASE-19524 Master side changes for moving peer modification from zk watcher to procedure

Posted by zh...@apache.org.
HBASE-19524 Master side changes for moving peer modification from zk watcher to procedure


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

Branch: refs/heads/HBASE-19397
Commit: 577c19aabe45524a99a395ec7850907285f4d476
Parents: ad6b8c6
Author: zhangduo <zh...@apache.org>
Authored: Mon Dec 18 15:22:36 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Fri Dec 22 21:41:28 2017 +0800

----------------------------------------------------------------------
 .../procedure2/RemoteProcedureDispatcher.java   |   3 +-
 .../src/main/protobuf/MasterProcedure.proto     |  21 +++-
 .../src/main/protobuf/RegionServerStatus.proto  |   3 +-
 .../src/main/protobuf/Replication.proto         |   5 +
 .../replication/ReplicationPeersZKImpl.java     |   4 +-
 .../org/apache/hadoop/hbase/master/HMaster.java | 100 ++++++++-----------
 .../hadoop/hbase/master/MasterRpcServices.java  |   4 +-
 .../hadoop/hbase/master/MasterServices.java     |  26 +++--
 .../assignment/RegionTransitionProcedure.java   |  11 +-
 .../master/procedure/MasterProcedureEnv.java    |   5 +
 .../master/procedure/ProcedurePrepareLatch.java |   2 +-
 .../master/replication/AddPeerProcedure.java    |  97 ++++++++++++++++++
 .../replication/DisablePeerProcedure.java       |  70 +++++++++++++
 .../master/replication/EnablePeerProcedure.java |  69 +++++++++++++
 .../master/replication/ModifyPeerProcedure.java |  97 +++++++++++++++---
 .../master/replication/RefreshPeerCallable.java |  67 -------------
 .../replication/RefreshPeerProcedure.java       |  28 ++++--
 .../master/replication/RemovePeerProcedure.java |  69 +++++++++++++
 .../master/replication/ReplicationManager.java  |  76 +++++++-------
 .../replication/UpdatePeerConfigProcedure.java  |  92 +++++++++++++++++
 .../hbase/regionserver/HRegionServer.java       |   6 +-
 .../regionserver/RefreshPeerCallable.java       |  70 +++++++++++++
 .../hbase/master/MockNoopMasterServices.java    |  23 +++--
 .../replication/DummyModifyPeerProcedure.java   |  13 ++-
 24 files changed, 736 insertions(+), 225 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/577c19aa/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
index e9a6906..1235b33 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
@@ -247,9 +247,8 @@ public abstract class RemoteProcedureDispatcher<TEnv, TRemote extends Comparable
     /**
      * Called when RS tells the remote procedure is failed through the {@code reportProcedureDone}
      * method.
-     * @param error the error message
      */
-    void remoteOperationFailed(TEnv env, String error);
+    void remoteOperationFailed(TEnv env, RemoteProcedureException error);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/577c19aa/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
index 0e2bdba..ae676ea 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -27,6 +27,7 @@ option optimize_for = SPEED;
 import "HBase.proto";
 import "RPC.proto";
 import "Snapshot.proto";
+import "Replication.proto";
 
 // ============================================================================
 //  WARNING - Compatibility rules
@@ -367,9 +368,10 @@ message GCMergedRegionsStateData {
 }
 
 enum PeerModificationState {
-  UPDATE_PEER_STORAGE = 1;
-  REFRESH_PEER_ON_RS = 2;
-  POST_PEER_MODIFICATION = 3;
+  PRE_PEER_MODIFICATION = 1;
+  UPDATE_PEER_STORAGE = 2;
+  REFRESH_PEER_ON_RS = 3;
+  POST_PEER_MODIFICATION = 4;
 }
 
 message PeerModificationStateData {
@@ -394,4 +396,17 @@ message RefreshPeerParameter {
   required string peer_id = 1;
   required PeerModificationType type = 2;
   required ServerName target_server = 3;
+}
+
+message ModifyPeerStateData {
+  required string peer_id = 1;
+}
+
+message AddPeerStateData {
+  required ReplicationPeer peer_config = 1;
+  required bool enabled = 2;
+}
+
+message UpdatePeerConfigStateData {
+  required ReplicationPeer peer_config = 1;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/577c19aa/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto b/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
index eb396ac..4f75941 100644
--- a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
@@ -28,6 +28,7 @@ option optimize_for = SPEED;
 
 import "HBase.proto";
 import "ClusterStatus.proto";
+import "ErrorHandling.proto";
 
 message RegionServerStartupRequest {
   /** Port number this regionserver is up on */
@@ -152,7 +153,7 @@ message ReportProcedureDoneRequest {
     ERROR = 2;
   }
   required Status status = 2;
-  optional string error = 3;
+  optional ForeignExceptionMessage error = 3;
 }
 
 message ReportProcedureDoneResponse {

http://git-wip-us.apache.org/repos/asf/hbase/blob/577c19aa/hbase-protocol-shaded/src/main/protobuf/Replication.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Replication.proto b/hbase-protocol-shaded/src/main/protobuf/Replication.proto
index 8657c25..9f7b4c2 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Replication.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Replication.proto
@@ -84,6 +84,7 @@ message AddReplicationPeerRequest {
 }
 
 message AddReplicationPeerResponse {
+  optional uint64 proc_id = 1;
 }
 
 message RemoveReplicationPeerRequest {
@@ -91,6 +92,7 @@ message RemoveReplicationPeerRequest {
 }
 
 message RemoveReplicationPeerResponse {
+  optional uint64 proc_id = 1;
 }
 
 message EnableReplicationPeerRequest {
@@ -98,6 +100,7 @@ message EnableReplicationPeerRequest {
 }
 
 message EnableReplicationPeerResponse {
+  optional uint64 proc_id = 1;
 }
 
 message DisableReplicationPeerRequest {
@@ -105,6 +108,7 @@ message DisableReplicationPeerRequest {
 }
 
 message DisableReplicationPeerResponse {
+  optional uint64 proc_id = 1;
 }
 
 message GetReplicationPeerConfigRequest {
@@ -122,6 +126,7 @@ message UpdateReplicationPeerConfigRequest {
 }
 
 message UpdateReplicationPeerConfigResponse {
+  optional uint64 proc_id = 1;
 }
 
 message ListReplicationPeersRequest {

http://git-wip-us.apache.org/repos/asf/hbase/blob/577c19aa/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
index 8f5e8d5..151b6b2 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
@@ -531,7 +531,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
         for (String queueId : queueIds) {
           ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
           if (queueInfo.getPeerId().equals(peerId)) {
-            throw new ReplicationException("undeleted queue for peerId: " + peerId
+            throw new IllegalArgumentException("undeleted queue for peerId: " + peerId
                 + ", replicator: " + replicator + ", queueId: " + queueId);
           }
         }
@@ -539,7 +539,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
       // Check for hfile-refs queue
       if (-1 != ZKUtil.checkExists(zookeeper, hfileRefsZNode)
           && queuesClient.getAllPeersFromHFileRefsQueue().contains(peerId)) {
-        throw new ReplicationException("Undeleted queue for peerId: " + peerId
+        throw new IllegalArgumentException("Undeleted queue for peerId: " + peerId
             + ", found in hfile-refs node path " + hfileRefsZNode);
       }
     } catch (KeeperException e) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/577c19aa/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 52b7b35..024a6ff 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.master;
 
 import com.google.protobuf.Descriptors;
 import com.google.protobuf.Service;
+
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.lang.reflect.Constructor;
@@ -49,10 +50,12 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Function;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
+
 import javax.servlet.ServletException;
 import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
@@ -127,7 +130,13 @@ import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
 import org.apache.hadoop.hbase.master.procedure.RecoverMetaProcedure;
 import org.apache.hadoop.hbase.master.procedure.TruncateTableProcedure;
+import org.apache.hadoop.hbase.master.replication.AddPeerProcedure;
+import org.apache.hadoop.hbase.master.replication.DisablePeerProcedure;
+import org.apache.hadoop.hbase.master.replication.EnablePeerProcedure;
+import org.apache.hadoop.hbase.master.replication.ModifyPeerProcedure;
+import org.apache.hadoop.hbase.master.replication.RemovePeerProcedure;
 import org.apache.hadoop.hbase.master.replication.ReplicationManager;
+import org.apache.hadoop.hbase.master.replication.UpdatePeerConfigProcedure;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.mob.MobConstants;
 import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
@@ -140,6 +149,7 @@ import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
 import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 import org.apache.hadoop.hbase.quotas.MasterSpaceQuotaObserver;
@@ -170,7 +180,6 @@ import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CompressionTest;
 import org.apache.hadoop.hbase.util.EncryptionTest;
-import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.apache.hadoop.hbase.util.HasThread;
 import org.apache.hadoop.hbase.util.IdLock;
@@ -331,15 +340,15 @@ public class HMaster extends HRegionServer implements MasterServices {
   private volatile boolean activeMaster = false;
 
   // flag set after we complete initialization once active
-  private final ProcedureEvent initialized = new ProcedureEvent("master initialized");
+  private final ProcedureEvent<?> initialized = new ProcedureEvent<>("master initialized");
 
   // flag set after master services are started,
   // initialization may have not completed yet.
   volatile boolean serviceStarted = false;
 
   // flag set after we complete assignMeta.
-  private final ProcedureEvent serverCrashProcessingEnabled =
-    new ProcedureEvent("server crash processing");
+  private final ProcedureEvent<?> serverCrashProcessingEnabled =
+    new ProcedureEvent<>("server crash processing");
 
   // Maximum time we should run balancer for
   private final int maxBlancingTime;
@@ -1198,7 +1207,6 @@ public class HMaster extends HRegionServer implements MasterServices {
 
   private void startProcedureExecutor() throws IOException {
     final MasterProcedureEnv procEnv = new MasterProcedureEnv(this);
-    final Path rootDir = FSUtils.getRootDir(conf);
 
     procedureStore = new WALProcedureStore(conf,
         new MasterProcedureEnv.WALStoreLeaseRecovery(this));
@@ -2318,11 +2326,8 @@ public class HMaster extends HRegionServer implements MasterServices {
             return true;
           }
           Pair<RegionInfo, ServerName> pair =
-              new Pair(MetaTableAccessor.getRegionInfo(data),
+              new Pair<>(MetaTableAccessor.getRegionInfo(data),
                   MetaTableAccessor.getServerName(data,0));
-          if (pair == null) {
-            return false;
-          }
           if (!pair.getFirst().getTable().equals(tableName)) {
             return false;
           }
@@ -2748,7 +2753,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   @Override
-  public ProcedureEvent getInitializedEvent() {
+  public ProcedureEvent<?> getInitializedEvent() {
     return initialized;
   }
 
@@ -2767,7 +2772,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     procedureExecutor.getEnvironment().setEventReady(serverCrashProcessingEnabled, b);
   }
 
-  public ProcedureEvent getServerCrashProcessingEnabledEvent() {
+  public ProcedureEvent<?> getServerCrashProcessingEnabledEvent() {
     return serverCrashProcessingEnabled;
   }
 
@@ -3318,54 +3323,36 @@ public class HMaster extends HRegionServer implements MasterServices {
     return favoredNodesManager;
   }
 
+  private long executePeerProcedure(ModifyPeerProcedure procedure) throws IOException {
+    long procId = procedureExecutor.submitProcedure(procedure);
+    procedure.getLatch().await();
+    return procId;
+  }
+
   @Override
-  public void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
+  public long addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
       throws ReplicationException, IOException {
-    if (cpHost != null) {
-      cpHost.preAddReplicationPeer(peerId, peerConfig);
-    }
-    LOG.info(getClientIdAuditPrefix() + " creating replication peer, id=" + peerId + ", config="
-        + peerConfig + ", state=" + (enabled ? "ENABLED" : "DISABLED"));
-    this.replicationManager.addReplicationPeer(peerId, peerConfig, enabled);
-    if (cpHost != null) {
-      cpHost.postAddReplicationPeer(peerId, peerConfig);
-    }
+    LOG.info(getClientIdAuditPrefix() + " creating replication peer, id=" + peerId + ", config=" +
+      peerConfig + ", state=" + (enabled ? "ENABLED" : "DISABLED"));
+    return executePeerProcedure(new AddPeerProcedure(peerId, peerConfig, enabled));
   }
 
   @Override
-  public void removeReplicationPeer(String peerId) throws ReplicationException, IOException {
-    if (cpHost != null) {
-      cpHost.preRemoveReplicationPeer(peerId);
-    }
+  public long removeReplicationPeer(String peerId) throws ReplicationException, IOException {
     LOG.info(getClientIdAuditPrefix() + " removing replication peer, id=" + peerId);
-    this.replicationManager.removeReplicationPeer(peerId);
-    if (cpHost != null) {
-      cpHost.postRemoveReplicationPeer(peerId);
-    }
+    return executePeerProcedure(new RemovePeerProcedure(peerId));
   }
 
   @Override
-  public void enableReplicationPeer(String peerId) throws ReplicationException, IOException {
-    if (cpHost != null) {
-      cpHost.preEnableReplicationPeer(peerId);
-    }
+  public long enableReplicationPeer(String peerId) throws ReplicationException, IOException {
     LOG.info(getClientIdAuditPrefix() + " enable replication peer, id=" + peerId);
-    this.replicationManager.enableReplicationPeer(peerId);
-    if (cpHost != null) {
-      cpHost.postEnableReplicationPeer(peerId);
-    }
+    return executePeerProcedure(new EnablePeerProcedure(peerId));
   }
 
   @Override
-  public void disableReplicationPeer(String peerId) throws ReplicationException, IOException {
-    if (cpHost != null) {
-      cpHost.preDisableReplicationPeer(peerId);
-    }
+  public long disableReplicationPeer(String peerId) throws ReplicationException, IOException {
     LOG.info(getClientIdAuditPrefix() + " disable replication peer, id=" + peerId);
-    this.replicationManager.disableReplicationPeer(peerId);
-    if (cpHost != null) {
-      cpHost.postDisableReplicationPeer(peerId);
-    }
+    return executePeerProcedure(new DisablePeerProcedure(peerId));
   }
 
   @Override
@@ -3384,17 +3371,11 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   @Override
-  public void updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
+  public long updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
       throws ReplicationException, IOException {
-    if (cpHost != null) {
-      cpHost.preUpdateReplicationPeerConfig(peerId, peerConfig);
-    }
-    LOG.info(getClientIdAuditPrefix() + " update replication peer config, id=" + peerId
-        + ", config=" + peerConfig);
-    this.replicationManager.updatePeerConfig(peerId, peerConfig);
-    if (cpHost != null) {
-      cpHost.postUpdateReplicationPeerConfig(peerId, peerConfig);
-    }
+    LOG.info(getClientIdAuditPrefix() + " update replication peer config, id=" + peerId +
+      ", config=" + peerConfig);
+    return executePeerProcedure(new UpdatePeerConfigProcedure(peerId, peerConfig));
   }
 
   @Override
@@ -3547,10 +3528,15 @@ public class HMaster extends HRegionServer implements MasterServices {
     }
   }
 
-  public void remoteProcedureFailed(long procId, String error) {
+  public void remoteProcedureFailed(long procId, RemoteProcedureException error) {
     RemoteProcedure<MasterProcedureEnv, ?> procedure = getRemoteProcedure(procId);
     if (procedure != null) {
       procedure.remoteOperationFailed(procedureExecutor.getEnvironment(), error);
     }
   }
-}
\ No newline at end of file
+
+  @Override
+  public ReplicationManager getReplicationManager() {
+    return replicationManager;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/577c19aa/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index f666a39..f214d83 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -71,6 +71,7 @@ import org.apache.hadoop.hbase.procedure2.LockType;
 import org.apache.hadoop.hbase.procedure2.LockedResource;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
@@ -2262,7 +2263,8 @@ public class MasterRpcServices extends RSRpcServices
     if (request.getStatus() == ReportProcedureDoneRequest.Status.SUCCESS) {
       master.remoteProcedureCompleted(request.getProcId());
     } else {
-      master.remoteProcedureFailed(request.getProcId(), request.getError());
+      master.remoteProcedureFailed(request.getProcId(),
+        RemoteProcedureException.fromProto(request.getError()));
     }
     return ReportProcedureDoneResponse.getDefaultInstance();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/577c19aa/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index 6b3c212..43df8b1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * 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
@@ -18,10 +17,11 @@
  */
 package org.apache.hadoop.hbase.master;
 
+import com.google.protobuf.Service;
+
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableDescriptors;
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.locking.LockManager;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.replication.ReplicationManager;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
 import org.apache.hadoop.hbase.procedure2.LockedResource;
@@ -52,8 +53,6 @@ import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 
-import com.google.protobuf.Service;
-
 /**
  * A curated subset of services provided by {@link HMaster}.
  * For use internally only. Passed to Managers, Services and Chores so can pass less-than-a
@@ -136,7 +135,7 @@ public interface MasterServices extends Server {
    * @return Tripped when Master has finished initialization.
    */
   @VisibleForTesting
-  public ProcedureEvent getInitializedEvent();
+  public ProcedureEvent<?> getInitializedEvent();
 
   /**
    * @return Master's instance of {@link MetricsMaster}
@@ -430,26 +429,26 @@ public interface MasterServices extends Server {
    * @param peerConfig configuration for the replication slave cluster
    * @param enabled peer state, true if ENABLED and false if DISABLED
    */
-  void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
+  long addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
       throws ReplicationException, IOException;
 
   /**
    * Removes a peer and stops the replication
    * @param peerId a short name that identifies the peer
    */
-  void removeReplicationPeer(String peerId) throws ReplicationException, IOException;
+  long removeReplicationPeer(String peerId) throws ReplicationException, IOException;
 
   /**
    * Restart the replication stream to the specified peer
    * @param peerId a short name that identifies the peer
    */
-  void enableReplicationPeer(String peerId) throws ReplicationException, IOException;
+  long enableReplicationPeer(String peerId) throws ReplicationException, IOException;
 
   /**
    * Stop the replication stream to the specified peer
    * @param peerId a short name that identifies the peer
    */
-  void disableReplicationPeer(String peerId) throws ReplicationException, IOException;
+  long disableReplicationPeer(String peerId) throws ReplicationException, IOException;
 
   /**
    * Returns the configured ReplicationPeerConfig for the specified peer
@@ -460,11 +459,16 @@ public interface MasterServices extends Server {
       IOException;
 
   /**
+   * Returns the {@link ReplicationManager}.
+   */
+  ReplicationManager getReplicationManager();
+
+  /**
    * Update the peerConfig for the specified peer
    * @param peerId a short name that identifies the peer
    * @param peerConfig new config for the peer
    */
-  void updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
+  long updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
       throws ReplicationException, IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/577c19aa/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
index 43e75ed..5714cae 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
@@ -16,7 +16,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.master.assignment;
 
 import java.io.IOException;
@@ -33,13 +32,15 @@ import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure;
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+
 /**
  * Base class for the Assign and Unassign Procedure.
  *
@@ -415,7 +416,7 @@ public abstract class RegionTransitionProcedure
   }
 
   @Override
-  public void remoteOperationFailed(MasterProcedureEnv env, String error) {
+  public void remoteOperationFailed(MasterProcedureEnv env, RemoteProcedureException error) {
     // should not be called for region operation until we modified the open/close region procedure
     throw new UnsupportedOperationException();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/577c19aa/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
index 0a4c97d..fa4d371 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
+import org.apache.hadoop.hbase.master.replication.ReplicationManager;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
@@ -137,6 +138,10 @@ public class MasterProcedureEnv implements ConfigurationObserver {
     return remoteDispatcher;
   }
 
+  public ReplicationManager getReplicationManager() {
+    return master.getReplicationManager();
+  }
+
   public boolean isRunning() {
     if (this.master == null || this.master.getMasterProcedureExecutor() == null) return false;
     return master.getMasterProcedureExecutor().isRunning();

http://git-wip-us.apache.org/repos/asf/hbase/blob/577c19aa/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java
index 09d05e6..dbea6fa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java
@@ -64,7 +64,7 @@ public abstract class ProcedurePrepareLatch {
   protected abstract void countDown(final Procedure proc);
   public abstract void await() throws IOException;
 
-  protected static void releaseLatch(final ProcedurePrepareLatch latch, final Procedure proc) {
+  public static void releaseLatch(final ProcedurePrepareLatch latch, final Procedure proc) {
     if (latch != null) {
       latch.countDown(proc);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/577c19aa/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
new file mode 100644
index 0000000..c3862d8
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
@@ -0,0 +1,97 @@
+/**
+ * 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.master.replication;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AddPeerStateData;
+
+/**
+ * The procedure for adding a new replication peer.
+ */
+@InterfaceAudience.Private
+public class AddPeerProcedure extends ModifyPeerProcedure {
+
+  private static final Log LOG = LogFactory.getLog(AddPeerProcedure.class);
+
+  private ReplicationPeerConfig peerConfig;
+
+  private boolean enabled;
+
+  public AddPeerProcedure() {
+  }
+
+  public AddPeerProcedure(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) {
+    super(peerId);
+    this.peerConfig = peerConfig;
+    this.enabled = enabled;
+  }
+
+  @Override
+  public PeerOperationType getPeerOperationType() {
+    return PeerOperationType.ADD;
+  }
+
+  @Override
+  protected void prePeerModification(MasterProcedureEnv env) throws IOException {
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.preAddReplicationPeer(peerId, peerConfig);
+    }
+  }
+
+  @Override
+  protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException {
+    env.getReplicationManager().addReplicationPeer(peerId, peerConfig, enabled);
+  }
+
+  @Override
+  protected void postPeerModification(MasterProcedureEnv env) throws IOException {
+    LOG.info("Successfully added " + (enabled ? "ENABLED" : "DISABLED") + " peer " + peerId +
+      ", config " + peerConfig);
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      env.getMasterCoprocessorHost().postAddReplicationPeer(peerId, peerConfig);
+    }
+  }
+
+  @Override
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.serializeStateData(serializer);
+    serializer.serialize(AddPeerStateData.newBuilder()
+        .setPeerConfig(ReplicationPeerConfigUtil.convert(peerConfig)).setEnabled(enabled).build());
+  }
+
+  @Override
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.deserializeStateData(serializer);
+    AddPeerStateData data = serializer.deserialize(AddPeerStateData.class);
+    peerConfig = ReplicationPeerConfigUtil.convert(data.getPeerConfig());
+    enabled = data.getEnabled();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/577c19aa/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
new file mode 100644
index 0000000..0b32db9
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
@@ -0,0 +1,70 @@
+/**
+ * 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.master.replication;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * The procedure for disabling a replication peer.
+ */
+@InterfaceAudience.Private
+public class DisablePeerProcedure extends ModifyPeerProcedure {
+
+  private static final Log LOG = LogFactory.getLog(DisablePeerProcedure.class);
+
+  public DisablePeerProcedure() {
+  }
+
+  public DisablePeerProcedure(String peerId) {
+    super(peerId);
+  }
+
+  @Override
+  public PeerOperationType getPeerOperationType() {
+    return PeerOperationType.DISABLE;
+  }
+
+  @Override
+  protected void prePeerModification(MasterProcedureEnv env) throws IOException {
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.preDisableReplicationPeer(peerId);
+    }
+  }
+
+  @Override
+  protected void updatePeerStorage(MasterProcedureEnv env)
+      throws IllegalArgumentException, Exception {
+    env.getReplicationManager().disableReplicationPeer(peerId);
+  }
+
+  @Override
+  protected void postPeerModification(MasterProcedureEnv env) throws IOException {
+    LOG.info("Successfully disabled peer " + peerId);
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.postDisableReplicationPeer(peerId);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/577c19aa/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
new file mode 100644
index 0000000..92ba000
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
@@ -0,0 +1,69 @@
+/**
+ * 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.master.replication;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * The procedure for enabling a replication peer.
+ */
+@InterfaceAudience.Private
+public class EnablePeerProcedure extends ModifyPeerProcedure {
+
+  private static final Log LOG = LogFactory.getLog(EnablePeerProcedure.class);
+
+  public EnablePeerProcedure() {
+  }
+
+  public EnablePeerProcedure(String peerId) {
+    super(peerId);
+  }
+
+  @Override
+  public PeerOperationType getPeerOperationType() {
+    return PeerOperationType.ENABLE;
+  }
+
+  @Override
+  protected void prePeerModification(MasterProcedureEnv env) throws IOException {
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.preEnableReplicationPeer(peerId);
+    }
+  }
+
+  @Override
+  protected void updatePeerStorage(MasterProcedureEnv env) throws Exception {
+    env.getReplicationManager().enableReplicationPeer(peerId);
+  }
+
+  @Override
+  protected void postPeerModification(MasterProcedureEnv env) throws IOException {
+    LOG.info("Successfully enabled peer " + peerId);
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.postEnableReplicationPeer(peerId);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/577c19aa/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
index fca05a7..7076bab 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
@@ -21,15 +21,22 @@ import java.io.IOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface;
+import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
 import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
 import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
 import org.apache.yetus.audience.InterfaceAudience;
 
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyPeerStateData;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationState;
 
+/**
+ * The base class for all replication peer related procedure.
+ */
 @InterfaceAudience.Private
 public abstract class ModifyPeerProcedure
     extends StateMachineProcedure<MasterProcedureEnv, PeerModificationState>
@@ -39,11 +46,21 @@ public abstract class ModifyPeerProcedure
 
   protected String peerId;
 
+  // used to keep compatible with old client where we can only returns after updateStorage.
+  protected ProcedurePrepareLatch latch;
+
   protected ModifyPeerProcedure() {
   }
 
   protected ModifyPeerProcedure(String peerId) {
     this.peerId = peerId;
+    // TODO: temporarily set a 4.0 here to always wait for the procedure exection completed. Change
+    // to 3.0 or 2.0 after the client modification is done.
+    this.latch = ProcedurePrepareLatch.createLatch(4, 0);
+  }
+
+  public ProcedurePrepareLatch getLatch() {
+    return latch;
   }
 
   @Override
@@ -52,28 +69,58 @@ public abstract class ModifyPeerProcedure
   }
 
   /**
-   * Return {@code false} means that the operation is invalid and we should give up, otherwise
-   * {@code true}.
+   * Called before we start the actual processing. If an exception is thrown then we will give up
+   * and mark the procedure as failed directly.
+   */
+  protected abstract void prePeerModification(MasterProcedureEnv env) throws IOException;
+
+  /**
+   * We will give up and mark the procedure as failure if {@link IllegalArgumentException} is
+   * thrown, for other type of Exception we will retry.
+   */
+  protected abstract void updatePeerStorage(MasterProcedureEnv env)
+      throws IllegalArgumentException, Exception;
+
+  /**
+   * Called before we finish the procedure. The implementation can do some logging work, and also
+   * call the coprocessor hook if any.
    * <p>
-   * You need to call {@link #setFailure(String, Throwable)} to give the detail failure information.
+   * Notice that, since we have already done the actual work, throwing exception here will not fail
+   * this procedure, we will just ignore it and finish the procedure as suceeded.
    */
-  protected abstract boolean updatePeerStorage() throws IOException;
+  protected abstract void postPeerModification(MasterProcedureEnv env) throws IOException;
 
-  protected void postPeerModification() {
+  private void releaseLatch() {
+    ProcedurePrepareLatch.releaseLatch(latch, this);
   }
 
   @Override
   protected Flow executeFromState(MasterProcedureEnv env, PeerModificationState state)
       throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
     switch (state) {
-      case UPDATE_PEER_STORAGE:
+      case PRE_PEER_MODIFICATION:
         try {
-          if (!updatePeerStorage()) {
-            assert isFailed() : "setFailure is not called";
-            return Flow.NO_MORE_STATE;
-          }
+          prePeerModification(env);
         } catch (IOException e) {
-          LOG.warn("update peer storage failed, retry", e);
+          LOG.warn(getClass().getName() + " failed to call prePeerModification for peer " + peerId +
+            ", mark the procedure as failure and give up", e);
+          setFailure("prePeerModification", e);
+          releaseLatch();
+          return Flow.NO_MORE_STATE;
+        }
+        setNextState(PeerModificationState.UPDATE_PEER_STORAGE);
+        return Flow.HAS_MORE_STATE;
+      case UPDATE_PEER_STORAGE:
+        try {
+          updatePeerStorage(env);
+        } catch (IllegalArgumentException e) {
+          setFailure("master-" + getPeerOperationType().name().toLowerCase() + "-peer",
+            new DoNotRetryIOException(e));
+          releaseLatch();
+          return Flow.NO_MORE_STATE;
+        } catch (Exception e) {
+          LOG.warn(
+            getClass().getName() + " update peer storage for peer " + peerId + " failed, retry", e);
           throw new ProcedureYieldException();
         }
         setNextState(PeerModificationState.REFRESH_PEER_ON_RS);
@@ -85,7 +132,13 @@ public abstract class ModifyPeerProcedure
         setNextState(PeerModificationState.POST_PEER_MODIFICATION);
         return Flow.HAS_MORE_STATE;
       case POST_PEER_MODIFICATION:
-        postPeerModification();
+        try {
+          postPeerModification(env);
+        } catch (IOException e) {
+          LOG.warn(getClass().getName() + " failed to call prePeerModification for peer " + peerId +
+            ", ignore since the procedure has already done", e);
+        }
+        releaseLatch();
         return Flow.NO_MORE_STATE;
       default:
         throw new UnsupportedOperationException("unhandled state=" + state);
@@ -107,6 +160,12 @@ public abstract class ModifyPeerProcedure
   @Override
   protected void rollbackState(MasterProcedureEnv env, PeerModificationState state)
       throws IOException, InterruptedException {
+    if (state == PeerModificationState.PRE_PEER_MODIFICATION ||
+      state == PeerModificationState.UPDATE_PEER_STORAGE) {
+      // actually the peer related operations has no rollback, but if we haven't done any
+      // modifications on the peer storage, we can just return.
+      return;
+    }
     throw new UnsupportedOperationException();
   }
 
@@ -122,6 +181,18 @@ public abstract class ModifyPeerProcedure
 
   @Override
   protected PeerModificationState getInitialState() {
-    return PeerModificationState.UPDATE_PEER_STORAGE;
+    return PeerModificationState.PRE_PEER_MODIFICATION;
+  }
+
+  @Override
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.serializeStateData(serializer);
+    serializer.serialize(ModifyPeerStateData.newBuilder().setPeerId(peerId).build());
+  }
+
+  @Override
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.deserializeStateData(serializer);
+    peerId = serializer.deserialize(ModifyPeerStateData.class).getPeerId();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/577c19aa/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerCallable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerCallable.java
deleted file mode 100644
index 4e09107..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerCallable.java
+++ /dev/null
@@ -1,67 +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.master.replication;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.executor.EventType;
-import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.yetus.audience.InterfaceAudience;
-
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RefreshPeerParameter;
-
-/**
- * The callable executed at RS side to refresh the peer config/state.
- * <p>
- * TODO: only a dummy implementation for verifying the framework, will add implementation later.
- */
-@InterfaceAudience.Private
-public class RefreshPeerCallable implements RSProcedureCallable {
-
-  private HRegionServer rs;
-
-  private String peerId;
-
-  private Exception initError;
-
-  @Override
-  public Void call() throws Exception {
-    if (initError != null) {
-      throw initError;
-    }
-    rs.getFileSystem().create(new Path("/" + peerId + "/" + rs.getServerName().toString())).close();
-    return null;
-  }
-
-  @Override
-  public void init(byte[] parameter, HRegionServer rs) {
-    this.rs = rs;
-    try {
-      this.peerId = RefreshPeerParameter.parseFrom(parameter).getPeerId();
-    } catch (InvalidProtocolBufferException e) {
-      initError = e;
-      return;
-    }
-  }
-
-  @Override
-  public EventType getEventType() {
-    return EventType.RS_REFRESH_PEER;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/577c19aa/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
index 18da487..ddc2401 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
@@ -32,6 +32,8 @@ import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
 import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
+import org.apache.hadoop.hbase.replication.regionserver.RefreshPeerCallable;
 import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -118,15 +120,22 @@ public class RefreshPeerProcedure extends Procedure<MasterProcedureEnv>
             .setTargetServer(ProtobufUtil.toServerName(remote)).build().toByteArray());
   }
 
-  private void complete(MasterProcedureEnv env, boolean succ) {
+  private void complete(MasterProcedureEnv env, Throwable error) {
     if (event == null) {
       LOG.warn("procedure event for " + getProcId() +
-          " is null, maybe the procedure is created when recovery", new Exception());
+          " is null, maybe the procedure is created when recovery",
+        new Exception());
       return;
     }
-    LOG.info("Refresh peer " + peerId + " for " + type + " on " + targetServer +
-        (succ ? " suceeded" : " failed"));
-    this.succ = succ;
+    if (error != null) {
+      LOG.warn("Refresh peer " + peerId + " for " + type + " on " + targetServer + " failed",
+        error);
+      this.succ = false;
+    } else {
+      LOG.info("Refresh peer " + peerId + " for " + type + " on " + targetServer + " suceeded");
+      this.succ = true;
+    }
+
     event.wake(env.getProcedureScheduler());
     event = null;
   }
@@ -134,17 +143,18 @@ public class RefreshPeerProcedure extends Procedure<MasterProcedureEnv>
   @Override
   public synchronized void remoteCallFailed(MasterProcedureEnv env, ServerName remote,
       IOException exception) {
-    complete(env, false);
+    complete(env, exception);
   }
 
   @Override
   public synchronized void remoteOperationCompleted(MasterProcedureEnv env) {
-    complete(env, true);
+    complete(env, null);
   }
 
   @Override
-  public synchronized void remoteOperationFailed(MasterProcedureEnv env, String error) {
-    complete(env, false);
+  public synchronized void remoteOperationFailed(MasterProcedureEnv env,
+      RemoteProcedureException error) {
+    complete(env, error);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/577c19aa/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
new file mode 100644
index 0000000..3daad6d
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
@@ -0,0 +1,69 @@
+/**
+ * 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.master.replication;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * The procedure for removing a replication peer.
+ */
+@InterfaceAudience.Private
+public class RemovePeerProcedure extends ModifyPeerProcedure {
+
+  private static final Log LOG = LogFactory.getLog(RemovePeerProcedure.class);
+
+  public RemovePeerProcedure() {
+  }
+
+  public RemovePeerProcedure(String peerId) {
+    super(peerId);
+  }
+
+  @Override
+  public PeerOperationType getPeerOperationType() {
+    return PeerOperationType.REMOVE;
+  }
+
+  @Override
+  protected void prePeerModification(MasterProcedureEnv env) throws IOException {
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.preRemoveReplicationPeer(peerId);
+    }
+  }
+
+  @Override
+  protected void updatePeerStorage(MasterProcedureEnv env) throws Exception {
+    env.getReplicationManager().removeReplicationPeer(peerId);
+  }
+
+  @Override
+  protected void postPeerModification(MasterProcedureEnv env) throws IOException {
+    LOG.info("Successfully removed peer " + peerId);
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.postRemoveReplicationPeer(peerId);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/577c19aa/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java
index f36b2e2..b6f8784 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java
@@ -27,10 +27,8 @@ import java.util.regex.Pattern;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.ReplicationPeerNotFoundException;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.hbase.replication.BaseReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
@@ -39,24 +37,21 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Manages and performs all replication admin operations.
+ * <p>
  * Used to add/remove a replication peer.
  */
 @InterfaceAudience.Private
 public class ReplicationManager {
-
-  private final Configuration conf;
-  private final ZKWatcher zkw;
   private final ReplicationQueuesClient replicationQueuesClient;
   private final ReplicationPeers replicationPeers;
 
   public ReplicationManager(Configuration conf, ZKWatcher zkw, Abortable abortable)
       throws IOException {
-    this.conf = conf;
-    this.zkw = zkw;
     try {
       this.replicationQueuesClient = ReplicationFactory
           .getReplicationQueuesClient(new ReplicationQueuesClientArguments(conf, abortable, zkw));
@@ -70,7 +65,7 @@ public class ReplicationManager {
   }
 
   public void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
-      throws ReplicationException, IOException {
+      throws ReplicationException {
     checkPeerConfig(peerConfig);
     replicationPeers.registerPeer(peerId, peerConfig, enabled);
     replicationPeers.peerConnected(peerId);
@@ -89,8 +84,8 @@ public class ReplicationManager {
     this.replicationPeers.disablePeer(peerId);
   }
 
-  public ReplicationPeerConfig getPeerConfig(String peerId) throws ReplicationException,
-      ReplicationPeerNotFoundException {
+  public ReplicationPeerConfig getPeerConfig(String peerId)
+      throws ReplicationException, ReplicationPeerNotFoundException {
     ReplicationPeerConfig peerConfig = replicationPeers.getReplicationPeerConfig(peerId);
     if (peerConfig == null) {
       throw new ReplicationPeerNotFoundException(peerId);
@@ -110,9 +105,9 @@ public class ReplicationManager {
     List<String> peerIds = replicationPeers.getAllPeerIds();
     for (String peerId : peerIds) {
       if (pattern == null || (pattern != null && pattern.matcher(peerId).matches())) {
-        peers.add(new ReplicationPeerDescription(peerId, replicationPeers
-            .getStatusOfPeerFromBackingStore(peerId), replicationPeers
-            .getReplicationPeerConfig(peerId)));
+        peers.add(new ReplicationPeerDescription(peerId,
+            replicationPeers.getStatusOfPeerFromBackingStore(peerId),
+            replicationPeers.getReplicationPeerConfig(peerId)));
       }
     }
     return peers;
@@ -126,13 +121,12 @@ public class ReplicationManager {
    * If replicate_all flag is false, it means all user tables can't be replicated to peer cluster.
    * Then allow to config namespaces or table-cfs which will be replicated to peer cluster.
    */
-  private void checkPeerConfig(ReplicationPeerConfig peerConfig)
-      throws ReplicationException, IOException {
+  private void checkPeerConfig(ReplicationPeerConfig peerConfig) {
     if (peerConfig.replicateAllUserTables()) {
-      if ((peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty())
-          || (peerConfig.getTableCFsMap() != null && !peerConfig.getTableCFsMap().isEmpty())) {
-        throw new ReplicationException("Need clean namespaces or table-cfs config firstly"
-            + " when replicate_all flag is true");
+      if ((peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty()) ||
+        (peerConfig.getTableCFsMap() != null && !peerConfig.getTableCFsMap().isEmpty())) {
+        throw new IllegalArgumentException("Need clean namespaces or table-cfs config firstly " +
+          "when you want replicate all cluster");
       }
       checkNamespacesAndTableCfsConfigConflict(peerConfig.getExcludeNamespaces(),
         peerConfig.getExcludeTableCFsMap());
@@ -141,7 +135,7 @@ public class ReplicationManager {
           && !peerConfig.getExcludeNamespaces().isEmpty())
           || (peerConfig.getExcludeTableCFsMap() != null
               && !peerConfig.getExcludeTableCFsMap().isEmpty())) {
-        throw new ReplicationException(
+        throw new IllegalArgumentException(
             "Need clean exclude-namespaces or exclude-table-cfs config firstly"
                 + " when replicate_all flag is false");
       }
@@ -154,20 +148,24 @@ public class ReplicationManager {
   /**
    * Set a namespace in the peer config means that all tables in this namespace will be replicated
    * to the peer cluster.
-   * 1. If peer config already has a namespace, then not allow set any table of this namespace
-   *    to the peer config.
-   * 2. If peer config already has a table, then not allow set this table's namespace to the peer
-   *    config.
-   *
+   * <ol>
+   * <li>If peer config already has a namespace, then not allow set any table of this namespace to
+   * the peer config.</li>
+   * <li>If peer config already has a table, then not allow set this table's namespace to the peer
+   * config.</li>
+   * </ol>
+   * <p>
    * Set a exclude namespace in the peer config means that all tables in this namespace can't be
    * replicated to the peer cluster.
-   * 1. If peer config already has a exclude namespace, then not allow set any exclude table of
-   *    this namespace to the peer config.
-   * 2. If peer config already has a exclude table, then not allow set this table's namespace
-   *    as a exclude namespace.
+   * <ol>
+   * <li>If peer config already has a exclude namespace, then not allow set any exclude table of
+   * this namespace to the peer config.</li>
+   * <li>If peer config already has a exclude table, then not allow set this table's namespace as a
+   * exclude namespace.</li>
+   * </ol>
    */
   private void checkNamespacesAndTableCfsConfigConflict(Set<String> namespaces,
-      Map<TableName, ? extends Collection<String>> tableCfs) throws ReplicationException {
+      Map<TableName, ? extends Collection<String>> tableCfs) {
     if (namespaces == null || namespaces.isEmpty()) {
       return;
     }
@@ -177,24 +175,22 @@ public class ReplicationManager {
     for (Map.Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
       TableName table = entry.getKey();
       if (namespaces.contains(table.getNamespaceAsString())) {
-        throw new ReplicationException("Table-cfs " + table + " is conflict with namespaces "
+        throw new IllegalArgumentException("Table-cfs " + table + " is conflict with namespaces "
             + table.getNamespaceAsString() + " in peer config");
       }
     }
   }
 
-  private void checkConfiguredWALEntryFilters(ReplicationPeerConfig peerConfig)
-      throws IOException {
-    String filterCSV = peerConfig.getConfiguration().
-        get(BaseReplicationEndpoint.REPLICATION_WALENTRYFILTER_CONFIG_KEY);
-    if (filterCSV != null && !filterCSV.isEmpty()){
-      String [] filters = filterCSV.split(",");
+  private void checkConfiguredWALEntryFilters(ReplicationPeerConfig peerConfig) {
+    String filterCSV = peerConfig.getConfiguration()
+        .get(BaseReplicationEndpoint.REPLICATION_WALENTRYFILTER_CONFIG_KEY);
+    if (filterCSV != null && !filterCSV.isEmpty()) {
+      String[] filters = filterCSV.split(",");
       for (String filter : filters) {
         try {
-          Class clazz = Class.forName(filter);
-          Object o = clazz.newInstance();
+          Class.forName(filter).newInstance();
         } catch (Exception e) {
-          throw new DoNotRetryIOException("Configured WALEntryFilter " + filter +
+          throw new IllegalArgumentException("Configured WALEntryFilter " + filter +
               " could not be created. Failing add/update " + "peer operation.", e);
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/577c19aa/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
new file mode 100644
index 0000000..435eefc
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
@@ -0,0 +1,92 @@
+/**
+ * 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.master.replication;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.UpdatePeerConfigStateData;
+
+/**
+ * The procedure for updating the config for a replication peer.
+ */
+@InterfaceAudience.Private
+public class UpdatePeerConfigProcedure extends ModifyPeerProcedure {
+
+  private static final Log LOG = LogFactory.getLog(UpdatePeerConfigProcedure.class);
+
+  private ReplicationPeerConfig peerConfig;
+
+  public UpdatePeerConfigProcedure() {
+  }
+
+  public UpdatePeerConfigProcedure(String peerId, ReplicationPeerConfig peerConfig) {
+    super(peerId);
+    this.peerConfig = peerConfig;
+  }
+
+  @Override
+  public PeerOperationType getPeerOperationType() {
+    return PeerOperationType.UPDATE_CONFIG;
+  }
+
+  @Override
+  protected void prePeerModification(MasterProcedureEnv env) throws IOException {
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.preUpdateReplicationPeerConfig(peerId, peerConfig);
+    }
+  }
+
+  @Override
+  protected void updatePeerStorage(MasterProcedureEnv env)
+      throws IllegalArgumentException, Exception {
+    env.getReplicationManager().updatePeerConfig(peerId, peerConfig);
+  }
+
+  @Override
+  protected void postPeerModification(MasterProcedureEnv env) throws IOException {
+    LOG.info("Successfully updated peer config of " + peerId + " to " + peerConfig);
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.postUpdateReplicationPeerConfig(peerId, peerConfig);
+    }
+  }
+
+  @Override
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.serializeStateData(serializer);
+    serializer.serialize(UpdatePeerConfigStateData.newBuilder()
+        .setPeerConfig(ReplicationPeerConfigUtil.convert(peerConfig)).build());
+  }
+
+  @Override
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.deserializeStateData(serializer);
+    peerConfig = ReplicationPeerConfigUtil
+        .convert(serializer.deserialize(UpdatePeerConfigStateData.class).getPeerConfig());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/577c19aa/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 0799ca6..49489d8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -148,6 +148,7 @@ import org.apache.hadoop.hbase.util.CompressionTest;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
 import org.apache.hadoop.hbase.util.HasThread;
 import org.apache.hadoop.hbase.util.JvmPauseMonitor;
 import org.apache.hadoop.hbase.util.NettyEventLoopGroupConfig;
@@ -178,7 +179,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
@@ -3742,7 +3742,7 @@ public class HRegionServer extends HasThread implements
       ReportProcedureDoneRequest.newBuilder().setProcId(procId);
     if (error != null) {
       builder.setStatus(ReportProcedureDoneRequest.Status.ERROR)
-          .setError(Throwables.getStackTraceAsString(error));
+          .setError(ForeignExceptionUtil.toProtoForeignException(serverName.toString(), error));
     } else {
       builder.setStatus(ReportProcedureDoneRequest.Status.SUCCESS);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/577c19aa/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java
new file mode 100644
index 0000000..a47a483
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java
@@ -0,0 +1,70 @@
+/**
+ * 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.replication.regionserver;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.executor.EventType;
+import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RefreshPeerParameter;
+
+/**
+ * The callable executed at RS side to refresh the peer config/state.
+ * <p>
+ * TODO: only a dummy implementation for verifying the framework, will add implementation later.
+ */
+@InterfaceAudience.Private
+public class RefreshPeerCallable implements RSProcedureCallable {
+
+  private HRegionServer rs;
+
+  private String peerId;
+
+  private Exception initError;
+
+  @Override
+  public Void call() throws Exception {
+    if (initError != null) {
+      throw initError;
+    }
+    Path dir = new Path("/" + peerId);
+    if (rs.getFileSystem().exists(dir)) {
+      rs.getFileSystem().create(new Path(dir, rs.getServerName().toString())).close();
+    }
+    return null;
+  }
+
+  @Override
+  public void init(byte[] parameter, HRegionServer rs) {
+    this.rs = rs;
+    try {
+      this.peerId = RefreshPeerParameter.parseFrom(parameter).getPeerId();
+    } catch (InvalidProtocolBufferException e) {
+      initError = e;
+      return;
+    }
+  }
+
+  @Override
+  public EventType getEventType() {
+    return EventType.RS_REFRESH_PEER;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/577c19aa/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
index 413abe3..540a67c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.locking.LockManager;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.replication.ReplicationManager;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
 import org.apache.hadoop.hbase.procedure2.LockedResource;
@@ -369,7 +370,6 @@ public class MockNoopMasterServices implements MasterServices, Server {
 
   @Override
   public ClusterConnection getClusterConnection() {
-    // TODO Auto-generated method stub
     return null;
   }
 
@@ -399,20 +399,24 @@ public class MockNoopMasterServices implements MasterServices, Server {
   }
 
   @Override
-  public void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
+  public long addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
       throws ReplicationException {
+    return 0;
   }
 
   @Override
-  public void removeReplicationPeer(String peerId) throws ReplicationException {
+  public long removeReplicationPeer(String peerId) throws ReplicationException {
+    return 0;
   }
 
   @Override
-  public void enableReplicationPeer(String peerId) throws ReplicationException, IOException {
+  public long enableReplicationPeer(String peerId) throws ReplicationException, IOException {
+    return 0;
   }
 
   @Override
-  public void disableReplicationPeer(String peerId) throws ReplicationException, IOException {
+  public long disableReplicationPeer(String peerId) throws ReplicationException, IOException {
+    return 0;
   }
 
   @Override
@@ -422,8 +426,9 @@ public class MockNoopMasterServices implements MasterServices, Server {
   }
 
   @Override
-  public void updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
+  public long updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
       throws ReplicationException, IOException {
+    return 0;
   }
 
   @Override
@@ -458,7 +463,6 @@ public class MockNoopMasterServices implements MasterServices, Server {
 
   @Override
   public ProcedureEvent getInitializedEvent() {
-    // TODO Auto-generated method stub
     return null;
   }
 
@@ -471,4 +475,9 @@ public class MockNoopMasterServices implements MasterServices, Server {
   public Connection createConnection(Configuration conf) throws IOException {
     return null;
   }
+
+  @Override
+  public ReplicationManager getReplicationManager() {
+    return null;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/577c19aa/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
index 44343d7..ed7c6fa 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hbase.master.replication;
 
-import java.io.IOException;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 
 public class DummyModifyPeerProcedure extends ModifyPeerProcedure {
 
@@ -34,8 +34,15 @@ public class DummyModifyPeerProcedure extends ModifyPeerProcedure {
   }
 
   @Override
-  protected boolean updatePeerStorage() throws IOException {
-    return true;
+  protected void prePeerModification(MasterProcedureEnv env) {
+  }
+
+  @Override
+  protected void updatePeerStorage(MasterProcedureEnv env) {
+  }
+
+  @Override
+  protected void postPeerModification(MasterProcedureEnv env) {
   }
 
 }


[17/26] hbase git commit: HBASE-19593 Possible NPE if wal is closed during waledit append.(Rajeshabbu)

Posted by zh...@apache.org.
HBASE-19593 Possible NPE if wal is closed during waledit append.(Rajeshabbu)


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

Branch: refs/heads/HBASE-19397
Commit: c0598dcb10f2185164548ad8a1fdf1867d6a0b34
Parents: 448ba3a
Author: Rajeshbabu Chintaguntla <ra...@apache.org>
Authored: Fri Dec 22 17:20:45 2017 +0530
Committer: Rajeshbabu Chintaguntla <ra...@apache.org>
Committed: Fri Dec 22 17:20:45 2017 +0530

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hbase/regionserver/HRegion.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c0598dcb/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index bb01fe8..6401a8b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -7585,7 +7585,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       }
       writeEntry = walKey.getWriteEntry();
     } catch (IOException ioe) {
-      if (walKey != null) {
+      if (walKey != null && walKey.getWriteEntry() != null) {
         mvcc.complete(walKey.getWriteEntry());
       }
       throw ioe;


[16/26] hbase git commit: HBASE-19496 Reusing the ByteBuffer in rpc layer corrupt the ServerLoad and RegionLoad

Posted by zh...@apache.org.
HBASE-19496 Reusing the ByteBuffer in rpc layer corrupt the ServerLoad and RegionLoad


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

Branch: refs/heads/HBASE-19397
Commit: 448ba3a78f50df2ffac874c3768e9f50d52b15f6
Parents: 7501e64
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Thu Dec 21 20:46:57 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Fri Dec 22 18:54:25 2017 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/ClusterMetrics.java | 198 ++++++++
 .../hadoop/hbase/ClusterMetricsBuilder.java     | 397 ++++++++++++++++
 .../org/apache/hadoop/hbase/ClusterStatus.java  | 363 ++++++---------
 .../org/apache/hadoop/hbase/RegionLoad.java     | 279 +++++++++---
 .../org/apache/hadoop/hbase/RegionMetrics.java  | 147 ++++++
 .../hadoop/hbase/RegionMetricsBuilder.java      | 437 ++++++++++++++++++
 .../org/apache/hadoop/hbase/ServerLoad.java     | 448 +++++++++++++------
 .../org/apache/hadoop/hbase/ServerMetrics.java  |  90 ++++
 .../hadoop/hbase/ServerMetricsBuilder.java      | 352 +++++++++++++++
 .../main/java/org/apache/hadoop/hbase/Size.java | 158 +++++++
 .../org/apache/hadoop/hbase/client/Admin.java   |   5 +-
 .../apache/hadoop/hbase/client/AsyncAdmin.java  |   4 +-
 .../hadoop/hbase/client/AsyncHBaseAdmin.java    |   4 +-
 .../hbase/client/ClusterStatusListener.java     |   2 +-
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |   7 +-
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java |   7 +-
 .../hbase/replication/ReplicationLoadSink.java  |   8 +-
 .../replication/ReplicationLoadSource.java      |  16 +-
 .../hbase/shaded/protobuf/ProtobufUtil.java     | 256 ++---------
 .../hbase/shaded/protobuf/RequestConverter.java |   5 +-
 .../chaos/actions/MoveRegionsOfTableAction.java |   3 +-
 .../mapreduce/IntegrationTestBulkLoad.java      |   2 +-
 .../test/IntegrationTestBigLinkedList.java      |   2 +-
 .../rest/StorageClusterStatusResource.java      |  12 +-
 .../rest/StorageClusterVersionResource.java     |   8 +-
 .../hadoop/hbase/rsgroup/TestRSGroupsBase.java  |  12 +-
 .../tmpl/master/BackupMasterStatusTmpl.jamon    |   4 +-
 .../hbase/master/ClusterStatusPublisher.java    |  19 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |  45 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |   7 +-
 .../hbase/master/RegionPlacementMaintainer.java |   4 +-
 .../hadoop/hbase/master/ServerManager.java      |   3 +-
 .../hbase/regionserver/HRegionServer.java       |   9 +-
 .../hadoop/hbase/regionserver/HStore.java       |   2 +-
 .../MetricsRegionServerWrapperImpl.java         |   2 +-
 .../apache/hadoop/hbase/regionserver/Store.java |   4 +-
 .../org/apache/hadoop/hbase/tool/Canary.java    |   4 +-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java |   5 +-
 .../hadoop/hbase/util/HBaseFsckRepair.java      |   5 +-
 .../apache/hadoop/hbase/util/RegionMover.java   |   3 +-
 .../hadoop/hbase/util/RegionSplitter.java       |  11 +-
 .../hadoop/hbase/HBaseTestingUtility.java       |   2 +-
 .../hadoop/hbase/TestClientClusterMetrics.java  | 233 ++++++++++
 .../hadoop/hbase/TestClientClusterStatus.java   | 258 +++++++++++
 .../org/apache/hadoop/hbase/TestRegionLoad.java |  24 +-
 .../apache/hadoop/hbase/TestRegionMetrics.java  | 130 ++++++
 .../org/apache/hadoop/hbase/TestServerLoad.java |  65 +--
 .../apache/hadoop/hbase/TestServerMetrics.java  | 114 +++++
 .../java/org/apache/hadoop/hbase/TestSize.java  |  84 ++++
 .../apache/hadoop/hbase/client/TestAdmin2.java  |  11 +-
 .../hbase/client/TestAsyncClusterAdminApi.java  |   8 +-
 .../client/TestAsyncDecommissionAdminApi.java   |   3 +-
 .../hbase/client/TestClientClusterStatus.java   | 242 ----------
 .../hadoop/hbase/client/TestFromClientSide.java |   2 +-
 .../hbase/client/TestMetaWithReplicas.java      |   2 +-
 .../hbase/client/TestServerLoadDurability.java  | 117 +++++
 .../hbase/master/TestAssignmentListener.java    |   5 +-
 .../hbase/master/TestMasterNoCluster.java       |   6 +-
 .../TestMasterOperationsForRegionReplicas.java  |   4 +-
 .../master/assignment/MockMasterServices.java   |   7 +-
 .../TestFavoredStochasticBalancerPickers.java   |  11 +-
 .../TestFavoredStochasticLoadBalancer.java      |   4 +-
 .../TestRegionServerReadRequestMetrics.java     |  25 +-
 .../replication/TestMasterReplication.java      |   5 +-
 .../replication/TestReplicationStatus.java      |   3 +-
 .../hadoop/hbase/util/BaseTestHBaseFsck.java    |   4 +-
 .../util/TestMiniClusterLoadSequential.java     |   5 +-
 67 files changed, 3608 insertions(+), 1115 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterMetrics.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterMetrics.java
new file mode 100644
index 0000000..103c107
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterMetrics.java
@@ -0,0 +1,198 @@
+/**
+ *
+ * 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;
+
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.util.List;
+import java.util.Map;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Metrics information on the HBase cluster.
+ * <p>
+ * <tt>ClusterMetrics</tt> provides clients with information such as:
+ * <ul>
+ * <li>The count and names of region servers in the cluster.</li>
+ * <li>The count and names of dead region servers in the cluster.</li>
+ * <li>The name of the active master for the cluster.</li>
+ * <li>The name(s) of the backup master(s) for the cluster, if they exist.</li>
+ * <li>The average cluster load.</li>
+ * <li>The number of regions deployed on the cluster.</li>
+ * <li>The number of requests since last report.</li>
+ * <li>Detailed region server loading and resource usage information,
+ *  per server and per region.</li>
+ * <li>Regions in transition at master</li>
+ * <li>The unique cluster ID</li>
+ * </ul>
+ * <tt>{@link Option}</tt> provides a way to get desired ClusterStatus information.
+ * The following codes will get all the cluster information.
+ * <pre>
+ * {@code
+ * // Original version still works
+ * Admin admin = connection.getAdmin();
+ * ClusterMetrics metrics = admin.getClusterStatus();
+ * // or below, a new version which has the same effects
+ * ClusterMetrics metrics = admin.getClusterStatus(EnumSet.allOf(Option.class));
+ * }
+ * </pre>
+ * If information about live servers is the only wanted.
+ * then codes in the following way:
+ * <pre>
+ * {@code
+ * Admin admin = connection.getAdmin();
+ * ClusterMetrics metrics = admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
+ * }
+ * </pre>
+ */
+@InterfaceAudience.Public
+public interface ClusterMetrics {
+
+  /**
+   * @return the HBase version string as reported by the HMaster
+   */
+  @Nullable
+  String getHBaseVersion();
+
+  /**
+   * @return the names of region servers on the dead list
+   */
+  List<ServerName> getDeadServerNames();
+
+  /**
+   * @return the names of region servers on the live list
+   */
+  Map<ServerName, ServerMetrics> getLiveServerMetrics();
+
+  /**
+   * @return the number of regions deployed on the cluster
+   */
+  default int getRegionCount() {
+    return getLiveServerMetrics().entrySet().stream()
+        .mapToInt(v -> v.getValue().getRegionMetrics().size()).sum();
+  }
+
+  /**
+   * @return the number of requests since last report
+   */
+  default long getRequestCount() {
+    return getLiveServerMetrics().entrySet().stream()
+        .flatMap(v -> v.getValue().getRegionMetrics().values().stream())
+        .mapToLong(RegionMetrics::getRequestCount).sum();
+  }
+
+  /**
+   * Returns detailed information about the current master {@link ServerName}.
+   * @return current master information if it exists
+   */
+  @Nullable
+  ServerName getMasterName();
+
+  /**
+   * @return the names of backup masters
+   */
+  List<ServerName> getBackupMasterNames();
+
+  @InterfaceAudience.Private
+  List<RegionState> getRegionStatesInTransition();
+
+  @Nullable
+  String getClusterId();
+
+  List<String> getMasterCoprocessorNames();
+
+  default long getLastMajorCompactionTimestamp(TableName table) {
+    return getLiveServerMetrics().values().stream()
+        .flatMap(s -> s.getRegionMetrics().values().stream())
+        .filter(r -> RegionInfo.getTable(r.getRegionName()).equals(table))
+        .mapToLong(RegionMetrics::getLastMajorCompactionTimestamp).min().orElse(0);
+  }
+
+  default long getLastMajorCompactionTimestamp(byte[] regionName) {
+    return getLiveServerMetrics().values().stream()
+        .filter(s -> s.getRegionMetrics().containsKey(regionName))
+        .findAny()
+        .map(s -> s.getRegionMetrics().get(regionName).getLastMajorCompactionTimestamp())
+        .orElse(0L);
+  }
+
+  @Nullable
+  Boolean getBalancerOn();
+
+  int getMasterInfoPort();
+
+  /**
+   * @return the average cluster load
+   */
+  default double getAverageLoad() {
+    int serverSize = getLiveServerMetrics().size();
+    if (serverSize == 0) {
+      return 0;
+    }
+    return (double)getRegionCount() / (double)serverSize;
+  }
+
+  /**
+   * Kinds of ClusterMetrics
+   */
+  enum Option {
+    /**
+     * metrics about hbase version
+     */
+    HBASE_VERSION,
+    /**
+     * metrics about cluster id
+     */
+    CLUSTER_ID,
+    /**
+     * metrics about balancer is on or not
+     */
+    BALANCER_ON,
+    /**
+     * metrics about live region servers
+     */
+    LIVE_SERVERS,
+    /**
+     * metrics about dead region servers
+     */
+    DEAD_SERVERS,
+    /**
+     * metrics about master name
+     */
+    MASTER,
+    /**
+     * metrics about backup masters name
+     */
+    BACKUP_MASTERS,
+    /**
+     * metrics about master coprocessors
+     */
+    MASTER_COPROCESSORS,
+    /**
+     * metrics about regions in transition
+     */
+    REGIONS_IN_TRANSITION,
+    /**
+     * metrics info port
+     */
+    MASTER_INFO_PORT
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterMetricsBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterMetricsBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterMetricsBuilder.java
new file mode 100644
index 0000000..ed669a5
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterMetricsBuilder.java
@@ -0,0 +1,397 @@
+/**
+ *
+ * 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;
+
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+
+@InterfaceAudience.Private
+public final class ClusterMetricsBuilder {
+
+  public static ClusterStatusProtos.ClusterStatus toClusterStatus(ClusterMetrics metrics) {
+    ClusterStatusProtos.ClusterStatus.Builder builder
+        = ClusterStatusProtos.ClusterStatus.newBuilder()
+        .addAllBackupMasters(metrics.getBackupMasterNames().stream()
+            .map(ProtobufUtil::toServerName).collect(Collectors.toList()))
+        .addAllDeadServers(metrics.getDeadServerNames().stream()
+            .map(ProtobufUtil::toServerName).collect(Collectors.toList()))
+        .addAllLiveServers(metrics.getLiveServerMetrics().entrySet().stream()
+            .map(s -> ClusterStatusProtos.LiveServerInfo
+                .newBuilder()
+                .setServer(ProtobufUtil.toServerName(s.getKey()))
+                .setServerLoad(ServerMetricsBuilder.toServerLoad(s.getValue()))
+                .build())
+            .collect(Collectors.toList()))
+        .addAllMasterCoprocessors(metrics.getMasterCoprocessorNames().stream()
+            .map(n -> HBaseProtos.Coprocessor.newBuilder().setName(n).build())
+            .collect(Collectors.toList()))
+        .addAllRegionsInTransition(metrics.getRegionStatesInTransition().stream()
+            .map(r -> ClusterStatusProtos.RegionInTransition
+                .newBuilder()
+                .setSpec(HBaseProtos.RegionSpecifier
+                    .newBuilder()
+                    .setType(HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME)
+                    .setValue(UnsafeByteOperations.unsafeWrap(r.getRegion().getRegionName()))
+                    .build())
+                .setRegionState(r.convert())
+                .build())
+            .collect(Collectors.toList()))
+        .setMasterInfoPort(metrics.getMasterInfoPort());
+    if (metrics.getMasterName() != null) {
+      builder.setMaster(ProtobufUtil.toServerName((metrics.getMasterName())));
+    }
+    if (metrics.getBalancerOn() != null) {
+      builder.setBalancerOn(metrics.getBalancerOn());
+    }
+    if (metrics.getClusterId() != null) {
+      builder.setClusterId(new ClusterId(metrics.getClusterId()).convert());
+    }
+    if (metrics.getHBaseVersion() != null) {
+      builder.setHbaseVersion(
+          FSProtos.HBaseVersionFileContent.newBuilder()
+              .setVersion(metrics.getHBaseVersion()));
+    }
+    return builder.build();
+  }
+
+  public static ClusterMetrics toClusterMetrics(
+      ClusterStatusProtos.ClusterStatus proto) {
+    ClusterMetricsBuilder builder = ClusterMetricsBuilder.newBuilder();
+    builder.setLiveServerMetrics(proto.getLiveServersList().stream()
+        .collect(Collectors.toMap(e -> ProtobufUtil.toServerName(e.getServer()),
+            ServerMetricsBuilder::toServerMetrics)))
+        .setDeadServerNames(proto.getDeadServersList().stream()
+            .map(ProtobufUtil::toServerName)
+            .collect(Collectors.toList()))
+        .setBackerMasterNames(proto.getBackupMastersList().stream()
+            .map(ProtobufUtil::toServerName)
+            .collect(Collectors.toList()))
+        .setRegionsInTransition(proto.getRegionsInTransitionList().stream()
+            .map(ClusterStatusProtos.RegionInTransition::getRegionState)
+            .map(RegionState::convert)
+            .collect(Collectors.toList()))
+        .setMasterCoprocessorNames(proto.getMasterCoprocessorsList().stream()
+            .map(HBaseProtos.Coprocessor::getName)
+            .collect(Collectors.toList()));
+    if (proto.hasClusterId()) {
+      builder.setClusterId(ClusterId.convert(proto.getClusterId()).toString());
+    }
+
+    if (proto.hasHbaseVersion()) {
+      builder.setHBaseVersion(proto.getHbaseVersion().getVersion());
+    }
+
+    if (proto.hasMaster()) {
+      builder.setMasterName(ProtobufUtil.toServerName(proto.getMaster()));
+    }
+
+    if (proto.hasBalancerOn()) {
+      builder.setBalancerOn(proto.getBalancerOn());
+    }
+
+    if (proto.hasMasterInfoPort()) {
+      builder.setMasterInfoPort(proto.getMasterInfoPort());
+    }
+    return builder.build();
+  }
+
+  /**
+   * Convert ClusterStatusProtos.Option to ClusterMetrics.Option
+   * @param option a ClusterStatusProtos.Option
+   * @return converted ClusterMetrics.Option
+   */
+  public static ClusterMetrics.Option toOption(ClusterStatusProtos.Option option) {
+    switch (option) {
+      case HBASE_VERSION: return ClusterMetrics.Option.HBASE_VERSION;
+      case LIVE_SERVERS: return ClusterMetrics.Option.LIVE_SERVERS;
+      case DEAD_SERVERS: return ClusterMetrics.Option.DEAD_SERVERS;
+      case REGIONS_IN_TRANSITION: return ClusterMetrics.Option.REGIONS_IN_TRANSITION;
+      case CLUSTER_ID: return ClusterMetrics.Option.CLUSTER_ID;
+      case MASTER_COPROCESSORS: return ClusterMetrics.Option.MASTER_COPROCESSORS;
+      case MASTER: return ClusterMetrics.Option.MASTER;
+      case BACKUP_MASTERS: return ClusterMetrics.Option.BACKUP_MASTERS;
+      case BALANCER_ON: return ClusterMetrics.Option.BALANCER_ON;
+      case MASTER_INFO_PORT: return ClusterMetrics.Option.MASTER_INFO_PORT;
+      // should not reach here
+      default: throw new IllegalArgumentException("Invalid option: " + option);
+    }
+  }
+
+  /**
+   * Convert ClusterMetrics.Option to ClusterStatusProtos.Option
+   * @param option a ClusterMetrics.Option
+   * @return converted ClusterStatusProtos.Option
+   */
+  public static ClusterStatusProtos.Option toOption(ClusterMetrics.Option option) {
+    switch (option) {
+      case HBASE_VERSION: return ClusterStatusProtos.Option.HBASE_VERSION;
+      case LIVE_SERVERS: return ClusterStatusProtos.Option.LIVE_SERVERS;
+      case DEAD_SERVERS: return ClusterStatusProtos.Option.DEAD_SERVERS;
+      case REGIONS_IN_TRANSITION: return ClusterStatusProtos.Option.REGIONS_IN_TRANSITION;
+      case CLUSTER_ID: return ClusterStatusProtos.Option.CLUSTER_ID;
+      case MASTER_COPROCESSORS: return ClusterStatusProtos.Option.MASTER_COPROCESSORS;
+      case MASTER: return ClusterStatusProtos.Option.MASTER;
+      case BACKUP_MASTERS: return ClusterStatusProtos.Option.BACKUP_MASTERS;
+      case BALANCER_ON: return ClusterStatusProtos.Option.BALANCER_ON;
+      case MASTER_INFO_PORT: return ClusterStatusProtos.Option.MASTER_INFO_PORT;
+      // should not reach here
+      default: throw new IllegalArgumentException("Invalid option: " + option);
+    }
+  }
+
+  /**
+   * Convert a list of ClusterStatusProtos.Option to an enum set of ClusterMetrics.Option
+   * @param options the pb options
+   * @return an enum set of ClusterMetrics.Option
+   */
+  public static EnumSet<ClusterMetrics.Option> toOptions(List<ClusterStatusProtos.Option> options) {
+    return options.stream().map(ClusterMetricsBuilder::toOption)
+        .collect(Collectors.toCollection(() -> EnumSet.noneOf(ClusterMetrics.Option.class)));
+  }
+
+  /**
+   * Convert an enum set of ClusterMetrics.Option to a list of ClusterStatusProtos.Option
+   * @param options the ClusterMetrics options
+   * @return a list of ClusterStatusProtos.Option
+   */
+  public static List<ClusterStatusProtos.Option> toOptions(EnumSet<ClusterMetrics.Option> options) {
+    return options.stream().map(ClusterMetricsBuilder::toOption).collect(Collectors.toList());
+  }
+
+  public static ClusterMetricsBuilder newBuilder() {
+    return new ClusterMetricsBuilder();
+  }
+  @Nullable
+  private String hbaseVersion;
+  private List<ServerName> deadServerNames = Collections.emptyList();
+  private Map<ServerName, ServerMetrics> liveServerMetrics = new TreeMap<>();
+  @Nullable
+  private ServerName masterName;
+  private List<ServerName> backupMasterNames = Collections.emptyList();
+  private List<RegionState> regionsInTransition = Collections.emptyList();
+  @Nullable
+  private String clusterId;
+  private List<String> masterCoprocessorNames = Collections.emptyList();
+  @Nullable
+  private Boolean balancerOn;
+  private int masterInfoPort;
+
+  private ClusterMetricsBuilder() {
+  }
+  public ClusterMetricsBuilder setHBaseVersion(String value) {
+    this.hbaseVersion = value;
+    return this;
+  }
+  public ClusterMetricsBuilder setDeadServerNames(List<ServerName> value) {
+    this.deadServerNames = value;
+    return this;
+  }
+
+  public ClusterMetricsBuilder setLiveServerMetrics(Map<ServerName, ServerMetrics> value) {
+    liveServerMetrics.putAll(value);
+    return this;
+  }
+
+  public ClusterMetricsBuilder setMasterName(ServerName value) {
+    this.masterName = value;
+    return this;
+  }
+  public ClusterMetricsBuilder setBackerMasterNames(List<ServerName> value) {
+    this.backupMasterNames = value;
+    return this;
+  }
+  public ClusterMetricsBuilder setRegionsInTransition(List<RegionState> value) {
+    this.regionsInTransition = value;
+    return this;
+  }
+  public ClusterMetricsBuilder setClusterId(String value) {
+    this.clusterId = value;
+    return this;
+  }
+  public ClusterMetricsBuilder setMasterCoprocessorNames(List<String> value) {
+    this.masterCoprocessorNames = value;
+    return this;
+  }
+  public ClusterMetricsBuilder setBalancerOn(@Nullable Boolean value) {
+    this.balancerOn = value;
+    return this;
+  }
+  public ClusterMetricsBuilder setMasterInfoPort(int value) {
+    this.masterInfoPort = value;
+    return this;
+  }
+  public ClusterMetrics build() {
+    return new ClusterMetricsImpl(
+        hbaseVersion,
+        deadServerNames,
+        liveServerMetrics,
+        masterName,
+        backupMasterNames,
+        regionsInTransition,
+        clusterId,
+        masterCoprocessorNames,
+        balancerOn,
+        masterInfoPort);
+  }
+  private static class ClusterMetricsImpl implements ClusterMetrics {
+    @Nullable
+    private final String hbaseVersion;
+    private final List<ServerName> deadServerNames;
+    private final Map<ServerName, ServerMetrics> liveServerMetrics;
+    @Nullable
+    private final ServerName masterName;
+    private final List<ServerName> backupMasterNames;
+    private final List<RegionState> regionsInTransition;
+    @Nullable
+    private final String clusterId;
+    private final List<String> masterCoprocessorNames;
+    @Nullable
+    private final Boolean balancerOn;
+    private final int masterInfoPort;
+
+    ClusterMetricsImpl(String hbaseVersion, List<ServerName> deadServerNames,
+        Map<ServerName, ServerMetrics> liveServerMetrics,
+        ServerName masterName,
+        List<ServerName> backupMasterNames,
+        List<RegionState> regionsInTransition,
+        String clusterId,
+        List<String> masterCoprocessorNames,
+        Boolean balancerOn,
+        int masterInfoPort) {
+      this.hbaseVersion = hbaseVersion;
+      this.deadServerNames = Preconditions.checkNotNull(deadServerNames);
+      this.liveServerMetrics = Preconditions.checkNotNull(liveServerMetrics);
+      this.masterName = masterName;
+      this.backupMasterNames = Preconditions.checkNotNull(backupMasterNames);
+      this.regionsInTransition = Preconditions.checkNotNull(regionsInTransition);
+      this.clusterId = clusterId;
+      this.masterCoprocessorNames = Preconditions.checkNotNull(masterCoprocessorNames);
+      this.balancerOn = balancerOn;
+      this.masterInfoPort = masterInfoPort;
+    }
+
+    @Override
+    public String getHBaseVersion() {
+      return hbaseVersion;
+    }
+
+    @Override
+    public List<ServerName> getDeadServerNames() {
+      return Collections.unmodifiableList(deadServerNames);
+    }
+
+    @Override
+    public Map<ServerName, ServerMetrics> getLiveServerMetrics() {
+      return Collections.unmodifiableMap(liveServerMetrics);
+    }
+
+    @Override
+    public ServerName getMasterName() {
+      return masterName;
+    }
+
+    @Override
+    public List<ServerName> getBackupMasterNames() {
+      return Collections.unmodifiableList(backupMasterNames);
+    }
+
+    @Override
+    public List<RegionState> getRegionStatesInTransition() {
+      return Collections.unmodifiableList(regionsInTransition);
+    }
+
+    @Override
+    public String getClusterId() {
+      return clusterId;
+    }
+
+    @Override
+    public List<String> getMasterCoprocessorNames() {
+      return Collections.unmodifiableList(masterCoprocessorNames);
+    }
+
+    @Override
+    public Boolean getBalancerOn() {
+      return balancerOn;
+    }
+
+    @Override
+    public int getMasterInfoPort() {
+      return masterInfoPort;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder(1024);
+      sb.append("Master: " + getMasterName());
+
+      int backupMastersSize = getBackupMasterNames().size();
+      sb.append("\nNumber of backup masters: " + backupMastersSize);
+      if (backupMastersSize > 0) {
+        for (ServerName serverName: getBackupMasterNames()) {
+          sb.append("\n  " + serverName);
+        }
+      }
+
+      int serversSize = getLiveServerMetrics().size();
+      sb.append("\nNumber of live region servers: " + serversSize);
+      if (serversSize > 0) {
+        for (ServerName serverName : getLiveServerMetrics().keySet()) {
+          sb.append("\n  " + serverName.getServerName());
+        }
+      }
+
+      int deadServerSize = getDeadServerNames().size();
+      sb.append("\nNumber of dead region servers: " + deadServerSize);
+      if (deadServerSize > 0) {
+        for (ServerName serverName : getDeadServerNames()) {
+          sb.append("\n  " + serverName);
+        }
+      }
+
+      sb.append("\nAverage load: " + getAverageLoad());
+      sb.append("\nNumber of requests: " + getRequestCount());
+      sb.append("\nNumber of regions: " + getRegionCount());
+
+      int ritSize = getRegionStatesInTransition().size();
+      sb.append("\nNumber of regions in transition: " + ritSize);
+      if (ritSize > 0) {
+        for (RegionState state : getRegionStatesInTransition()) {
+          sb.append("\n  " + state.toDescriptiveString());
+        }
+      }
+      return sb.toString();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
index 6b4c03a..8cbf302 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
@@ -19,16 +19,17 @@
 
 package org.apache.hadoop.hbase;
 
+import edu.umd.cs.findbugs.annotations.Nullable;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Objects;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.master.RegionState;
 
 /**
  * Status information on the HBase cluster.
@@ -47,7 +48,7 @@ import org.apache.hadoop.hbase.master.RegionState;
  * <li>Regions in transition at master</li>
  * <li>The unique cluster ID</li>
  * </ul>
- * <tt>{@link Option}</tt> provides a way to get desired ClusterStatus information.
+ * <tt>{@link ClusterMetrics.Option}</tt> provides a way to get desired ClusterStatus information.
  * The following codes will get all the cluster information.
  * <pre>
  * {@code
@@ -66,28 +67,20 @@ import org.apache.hadoop.hbase.master.RegionState;
  * ClusterStatus status = admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
  * }
  * </pre>
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+ *             Use {@link ClusterMetrics} instead.
  */
 @InterfaceAudience.Public
-public class ClusterStatus {
+@Deprecated
+public class ClusterStatus implements ClusterMetrics {
 
   // TODO: remove this in 3.0
   private static final byte VERSION = 2;
 
-  private String hbaseVersion;
-  private Map<ServerName, ServerLoad> liveServers;
-  private List<ServerName> deadServers;
-  private ServerName master;
-  private List<ServerName> backupMasters;
-  private List<RegionState> intransition;
-  private String clusterId;
-  private String[] masterCoprocessors;
-  private Boolean balancerOn;
-  private int masterInfoPort;
+  private final ClusterMetrics metrics;
 
   /**
-   * Use {@link ClusterStatus.Builder} to construct a ClusterStatus instead.
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-15511">HBASE-15511</a>).
    */
   @Deprecated
   public ClusterStatus(final String hbaseVersion, final String clusterid,
@@ -100,61 +93,52 @@ public class ClusterStatus {
       final Boolean balancerOn,
       final int masterInfoPort) {
     // TODO: make this constructor private
-    this.hbaseVersion = hbaseVersion;
-    this.liveServers = servers;
-    this.deadServers = new ArrayList<>(deadServers);
-    this.master = master;
-    this.backupMasters = new ArrayList<>(backupMasters);
-    this.intransition = rit;
-    this.clusterId = clusterid;
-    this.masterCoprocessors = masterCoprocessors;
-    this.balancerOn = balancerOn;
-    this.masterInfoPort = masterInfoPort;
+    this(ClusterMetricsBuilder.newBuilder().setHBaseVersion(hbaseVersion)
+      .setDeadServerNames(new ArrayList<>(deadServers))
+      .setLiveServerMetrics(servers.entrySet().stream()
+      .collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue())))
+      .setBackerMasterNames(new ArrayList<>(backupMasters)).setBalancerOn(balancerOn)
+      .setClusterId(clusterid)
+      .setMasterCoprocessorNames(Arrays.asList(masterCoprocessors))
+      .setMasterName(master)
+      .setMasterInfoPort(masterInfoPort)
+      .setRegionsInTransition(rit)
+      .build());
   }
 
-  private ClusterStatus(final String hbaseVersion, final String clusterid,
-      final Map<ServerName, ServerLoad> servers,
-      final List<ServerName> deadServers,
-      final ServerName master,
-      final List<ServerName> backupMasters,
-      final List<RegionState> rit,
-      final String[] masterCoprocessors,
-      final Boolean balancerOn,
-      final int masterInfoPort) {
-    this.hbaseVersion = hbaseVersion;
-    this.liveServers = servers;
-    this.deadServers = deadServers;
-    this.master = master;
-    this.backupMasters = backupMasters;
-    this.intransition = rit;
-    this.clusterId = clusterid;
-    this.masterCoprocessors = masterCoprocessors;
-    this.balancerOn = balancerOn;
-    this.masterInfoPort = masterInfoPort;
+  @InterfaceAudience.Private
+  public ClusterStatus(ClusterMetrics metrics) {
+    this.metrics = metrics;
   }
 
   /**
    * @return the names of region servers on the dead list
    */
+  @Override
   public List<ServerName> getDeadServerNames() {
-    if (deadServers == null) {
-      return Collections.emptyList();
-    }
-    return Collections.unmodifiableList(deadServers);
+    return metrics.getDeadServerNames();
+  }
+
+  @Override
+  public Map<ServerName, ServerMetrics> getLiveServerMetrics() {
+    return metrics.getLiveServerMetrics();
   }
 
   /**
-   * @return the number of region servers in the cluster
-   */
+  * @return the number of region servers in the cluster
+  * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+  *             Use {@link #getLiveServerMetrics()}.
+  */
+  @Deprecated
   public int getServersSize() {
-    return liveServers != null ? liveServers.size() : 0;
+    return metrics.getLiveServerMetrics().size();
   }
 
   /**
    * @return the number of dead region servers in the cluster
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
    *             (<a href="https://issues.apache.org/jira/browse/HBASE-13656">HBASE-13656</a>).
-   *             Use {@link #getDeadServersSize()}.
+   *             Use {@link #getDeadServerNames()}.
    */
   @Deprecated
   public int getDeadServers() {
@@ -163,32 +147,22 @@ public class ClusterStatus {
 
   /**
    * @return the number of dead region servers in the cluster
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getDeadServerNames()}.
    */
+  @Deprecated
   public int getDeadServersSize() {
-    return deadServers != null ? deadServers.size() : 0;
-  }
-
-
-  /**
-   * @return the average cluster load
-   */
-  public double getAverageLoad() {
-    int load = getRegionsCount();
-    int serverSize = getServersSize();
-    return serverSize != 0 ? (double)load / (double)serverSize : 0.0;
+    return metrics.getDeadServerNames().size();
   }
 
   /**
    * @return the number of regions deployed on the cluster
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getRegionCount()}.
    */
+  @Deprecated
   public int getRegionsCount() {
-    int count = 0;
-    if (liveServers != null && !liveServers.isEmpty()) {
-      for (Map.Entry<ServerName, ServerLoad> e: this.liveServers.entrySet()) {
-        count = count + e.getValue().getNumberOfRegions();
-      }
-    }
-    return count;
+    return getRegionCount();
   }
 
   /**
@@ -201,20 +175,32 @@ public class ClusterStatus {
     return (int) getRequestCount();
   }
 
-  public long getRequestCount() {
-    if (liveServers == null) {
-      return 0L;
-    }
-    return liveServers.values().stream()
-            .mapToLong(ServerLoad::getNumberOfRequests)
-            .sum();
+  @Nullable
+  @Override
+  public ServerName getMasterName() {
+    return metrics.getMasterName();
+  }
+
+  @Override
+  public List<ServerName> getBackupMasterNames() {
+    return metrics.getBackupMasterNames();
+  }
+
+  @Override
+  public List<RegionState> getRegionStatesInTransition() {
+    return metrics.getRegionStatesInTransition();
   }
 
   /**
    * @return the HBase version string as reported by the HMaster
    */
   public String getHBaseVersion() {
-    return hbaseVersion;
+    return metrics.getHBaseVersion();
+  }
+
+  private Map<ServerName, ServerLoad> getLiveServerLoads() {
+    return metrics.getLiveServerMetrics().entrySet().stream()
+      .collect(Collectors.toMap(e -> e.getKey(), e -> new ServerLoad(e.getValue())));
   }
 
   @Override
@@ -227,7 +213,7 @@ public class ClusterStatus {
     }
     ClusterStatus other = (ClusterStatus) o;
     return Objects.equal(getHBaseVersion(), other.getHBaseVersion()) &&
-      Objects.equal(this.liveServers, other.liveServers) &&
+      Objects.equal(getLiveServerLoads(), other.getLiveServerLoads()) &&
       getDeadServerNames().containsAll(other.getDeadServerNames()) &&
       Arrays.equals(getMasterCoprocessors(), other.getMasterCoprocessors()) &&
       Objects.equal(getMaster(), other.getMaster()) &&
@@ -238,8 +224,7 @@ public class ClusterStatus {
 
   @Override
   public int hashCode() {
-    return Objects.hashCode(hbaseVersion, liveServers, deadServers, master, backupMasters,
-      clusterId, masterInfoPort);
+    return metrics.hashCode();
   }
 
   /**
@@ -251,111 +236,123 @@ public class ClusterStatus {
     return VERSION;
   }
 
-  //
-  // Getters
-  //
-
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getLiveServerMetrics()} instead.
+   */
+  @Deprecated
   public Collection<ServerName> getServers() {
-    if (liveServers == null) {
-      return Collections.emptyList();
-    }
-    return Collections.unmodifiableCollection(this.liveServers.keySet());
+    return metrics.getLiveServerMetrics().keySet();
   }
 
   /**
    * Returns detailed information about the current master {@link ServerName}.
    * @return current master information if it exists
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getMasterName} instead.
    */
+  @Deprecated
   public ServerName getMaster() {
-    return this.master;
+    return metrics.getMasterName();
   }
 
   /**
    * @return the number of backup masters in the cluster
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getBackupMasterNames} instead.
    */
+  @Deprecated
   public int getBackupMastersSize() {
-    return backupMasters != null ? backupMasters.size() : 0;
+    return metrics.getBackupMasterNames().size();
   }
 
   /**
    * @return the names of backup masters
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getBackupMasterNames} instead.
    */
+  @Deprecated
   public List<ServerName> getBackupMasters() {
-    if (backupMasters == null) {
-      return Collections.emptyList();
-    }
-    return Collections.unmodifiableList(this.backupMasters);
+    return metrics.getBackupMasterNames();
   }
 
   /**
    * @param sn
    * @return Server's load or null if not found.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getLiveServerMetrics} instead.
    */
+  @Deprecated
   public ServerLoad getLoad(final ServerName sn) {
-    return liveServers != null ? liveServers.get(sn) : null;
+    ServerMetrics serverMetrics = metrics.getLiveServerMetrics().get(sn);
+    return serverMetrics == null ? null : new ServerLoad(serverMetrics);
   }
 
-  @InterfaceAudience.Private
-  public List<RegionState> getRegionsInTransition() {
-    if (intransition == null) {
-      return Collections.emptyList();
-    }
-    return Collections.unmodifiableList(intransition);
+  public String getClusterId() {
+    return metrics.getClusterId();
   }
 
-  public String getClusterId() {
-    return clusterId;
+  @Override
+  public List<String> getMasterCoprocessorNames() {
+    return metrics.getMasterCoprocessorNames();
   }
 
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getMasterCoprocessorNames} instead.
+   */
+  @Deprecated
   public String[] getMasterCoprocessors() {
-    return masterCoprocessors;
+    List<String> rval = metrics.getMasterCoprocessorNames();
+    return rval.toArray(new String[rval.size()]);
   }
 
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getLastMajorCompactionTimestamp(TableName)} instead.
+   */
+  @Deprecated
   public long getLastMajorCompactionTsForTable(TableName table) {
-    long result = Long.MAX_VALUE;
-    for (ServerName server : getServers()) {
-      ServerLoad load = getLoad(server);
-      for (RegionLoad rl : load.getRegionsLoad().values()) {
-        if (table.equals(HRegionInfo.getTable(rl.getName()))) {
-          result = Math.min(result, rl.getLastMajorCompactionTs());
-        }
-      }
-    }
-    return result == Long.MAX_VALUE ? 0 : result;
+    return metrics.getLastMajorCompactionTimestamp(table);
   }
 
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getLastMajorCompactionTimestamp(byte[])} instead.
+   */
+  @Deprecated
   public long getLastMajorCompactionTsForRegion(final byte[] region) {
-    for (ServerName server : getServers()) {
-      ServerLoad load = getLoad(server);
-      RegionLoad rl = load.getRegionsLoad().get(region);
-      if (rl != null) {
-        return rl.getLastMajorCompactionTs();
-      }
-    }
-    return 0;
+    return metrics.getLastMajorCompactionTimestamp(region);
   }
 
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             No flag in 2.0
+   */
+  @Deprecated
   public boolean isBalancerOn() {
-    return balancerOn != null && balancerOn;
+    return metrics.getBalancerOn() != null && metrics.getBalancerOn();
   }
 
+  @Override
   public Boolean getBalancerOn() {
-    return balancerOn;
+    return metrics.getBalancerOn();
   }
 
+  @Override
   public int getMasterInfoPort() {
-    return masterInfoPort;
+    return metrics.getMasterInfoPort();
   }
 
   @Override
   public String toString() {
     StringBuilder sb = new StringBuilder(1024);
-    sb.append("Master: " + master);
+    sb.append("Master: " + metrics.getMasterName());
 
     int backupMastersSize = getBackupMastersSize();
     sb.append("\nNumber of backup masters: " + backupMastersSize);
     if (backupMastersSize > 0) {
-      for (ServerName serverName: backupMasters) {
+      for (ServerName serverName: metrics.getBackupMasterNames()) {
         sb.append("\n  " + serverName);
       }
     }
@@ -363,15 +360,15 @@ public class ClusterStatus {
     int serversSize = getServersSize();
     sb.append("\nNumber of live region servers: " + serversSize);
     if (serversSize > 0) {
-      for (ServerName serverName: liveServers.keySet()) {
+      for (ServerName serverName : metrics.getLiveServerMetrics().keySet()) {
         sb.append("\n  " + serverName.getServerName());
       }
     }
 
-    int deadServerSize = getDeadServersSize();
+    int deadServerSize = metrics.getDeadServerNames().size();
     sb.append("\nNumber of dead region servers: " + deadServerSize);
     if (deadServerSize > 0) {
-      for (ServerName serverName: deadServers) {
+      for (ServerName serverName : metrics.getDeadServerNames()) {
         sb.append("\n  " + serverName);
       }
     }
@@ -380,109 +377,13 @@ public class ClusterStatus {
     sb.append("\nNumber of requests: " + getRequestCount());
     sb.append("\nNumber of regions: " + getRegionsCount());
 
-    int ritSize = (intransition != null) ? intransition.size() : 0;
+    int ritSize = metrics.getRegionStatesInTransition().size();
     sb.append("\nNumber of regions in transition: " + ritSize);
     if (ritSize > 0) {
-      for (RegionState state: intransition) {
+      for (RegionState state: metrics.getRegionStatesInTransition()) {
         sb.append("\n  " + state.toDescriptiveString());
       }
     }
     return sb.toString();
   }
-
-  @InterfaceAudience.Private
-  public static Builder newBuilder() {
-    return new Builder();
-  }
-
-  /**
-   * Builder for construct a ClusterStatus.
-   */
-  @InterfaceAudience.Private
-  public static class Builder {
-    private String hbaseVersion = null;
-    private Map<ServerName, ServerLoad> liveServers = null;
-    private List<ServerName> deadServers = null;
-    private ServerName master = null;
-    private List<ServerName> backupMasters = null;
-    private List<RegionState> intransition = null;
-    private String clusterId = null;
-    private String[] masterCoprocessors = null;
-    private Boolean balancerOn = null;
-    private int masterInfoPort = -1;
-
-    private Builder() {}
-
-    public Builder setHBaseVersion(String hbaseVersion) {
-      this.hbaseVersion = hbaseVersion;
-      return this;
-    }
-
-    public Builder setLiveServers(Map<ServerName, ServerLoad> liveServers) {
-      this.liveServers = liveServers;
-      return this;
-    }
-
-    public Builder setDeadServers(List<ServerName> deadServers) {
-      this.deadServers = deadServers;
-      return this;
-    }
-
-    public Builder setMaster(ServerName master) {
-      this.master = master;
-      return this;
-    }
-
-    public Builder setBackupMasters(List<ServerName> backupMasters) {
-      this.backupMasters = backupMasters;
-      return this;
-    }
-
-    public Builder setRegionState(List<RegionState> intransition) {
-      this.intransition = intransition;
-      return this;
-    }
-
-    public Builder setClusterId(String clusterId) {
-      this.clusterId = clusterId;
-      return this;
-    }
-
-    public Builder setMasterCoprocessors(String[] masterCoprocessors) {
-      this.masterCoprocessors = masterCoprocessors;
-      return this;
-    }
-
-    public Builder setBalancerOn(Boolean balancerOn) {
-      this.balancerOn = balancerOn;
-      return this;
-    }
-
-    public Builder setMasterInfoPort(int masterInfoPort) {
-      this.masterInfoPort = masterInfoPort;
-      return this;
-    }
-
-    public ClusterStatus build() {
-      return new ClusterStatus(hbaseVersion, clusterId, liveServers,
-          deadServers, master, backupMasters, intransition, masterCoprocessors,
-          balancerOn, masterInfoPort);
-    }
-  }
-
-  /**
-   * Kinds of ClusterStatus
-   */
-  public enum Option {
-    HBASE_VERSION, /** status about hbase version */
-    CLUSTER_ID, /** status about cluster id */
-    BALANCER_ON, /** status about balancer is on or not */
-    LIVE_SERVERS, /** status about live region servers */
-    DEAD_SERVERS, /** status about dead region servers */
-    MASTER, /** status about master */
-    BACKUP_MASTERS, /** status about backup masters */
-    MASTER_COPROCESSORS, /** status about master coprocessors */
-    REGIONS_IN_TRANSITION, /** status about regions in transition */
-    MASTER_INFO_PORT /** master info port **/
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLoad.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLoad.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLoad.java
index a2b993f..811f83c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLoad.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLoad.java
@@ -21,184 +21,341 @@
 package org.apache.hadoop.hbase;
 
 import java.util.List;
-
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hbase.util.Strings;
 import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Strings;
 
 /**
-  * Encapsulates per-region load metrics.
-  */
+ * Encapsulates per-region load metrics.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+ *             Use {@link RegionMetrics} instead.
+ */
 @InterfaceAudience.Public
-public class RegionLoad {
-
+@Deprecated
+public class RegionLoad implements RegionMetrics {
+  // DONT use this pb object since the byte array backed may be modified in rpc layer
+  // we keep this pb object for BC.
   protected ClusterStatusProtos.RegionLoad regionLoadPB;
+  private final RegionMetrics metrics;
 
-  @InterfaceAudience.Private
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="URF_UNREAD_PUBLIC_OR_PROTECTED_FIELD")
   public RegionLoad(ClusterStatusProtos.RegionLoad regionLoadPB) {
     this.regionLoadPB = regionLoadPB;
+    this.metrics = RegionMetricsBuilder.toRegionMetrics(regionLoadPB);
+  }
+
+  RegionLoad(RegionMetrics metrics) {
+    this.metrics = metrics;
+    this.regionLoadPB = RegionMetricsBuilder.toRegionLoad(metrics);
   }
 
   /**
    * @return the region name
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getRegionName} instead.
    */
+  @Deprecated
   public byte[] getName() {
-    return regionLoadPB.getRegionSpecifier().getValue().toByteArray();
+    return metrics.getRegionName();
   }
 
-  /**
-   * @return the region name as a string
-   */
-  public String getNameAsString() {
-    return Bytes.toStringBinary(getName());
+  @Override
+  public byte[] getRegionName() {
+    return metrics.getRegionName();
+  }
+
+  @Override
+  public int getStoreCount() {
+    return metrics.getStoreCount();
+  }
+
+  @Override
+  public int getStoreFileCount() {
+    return metrics.getStoreFileCount();
+  }
+
+  @Override
+  public Size getStoreFileSize() {
+    return metrics.getStoreFileSize();
+  }
+
+  @Override
+  public Size getMemStoreSize() {
+    return metrics.getMemStoreSize();
+  }
+
+  @Override
+  public long getReadRequestCount() {
+    return metrics.getReadRequestCount();
+  }
+
+  @Override
+  public long getFilteredReadRequestCount() {
+    return metrics.getFilteredReadRequestCount();
+  }
+
+  @Override
+  public Size getStoreFileIndexSize() {
+    return metrics.getStoreFileIndexSize();
+  }
+
+  @Override
+  public long getWriteRequestCount() {
+    return metrics.getWriteRequestCount();
+  }
+
+  @Override
+  public Size getStoreFileRootLevelIndexSize() {
+    return metrics.getStoreFileRootLevelIndexSize();
+  }
+
+  @Override
+  public Size getStoreFileUncompressedDataIndexSize() {
+    return metrics.getStoreFileUncompressedDataIndexSize();
+  }
+
+  @Override
+  public Size getBloomFilterSize() {
+    return metrics.getBloomFilterSize();
+  }
+
+  @Override
+  public long getCompactingCellCount() {
+    return metrics.getCompactingCellCount();
+  }
+
+  @Override
+  public long getCompactedCellCount() {
+    return metrics.getCompactedCellCount();
+  }
+
+  @Override
+  public long getCompletedSequenceId() {
+    return metrics.getCompletedSequenceId();
+  }
+
+  @Override
+  public Map<byte[], Long> getStoreSequenceId() {
+    return metrics.getStoreSequenceId();
+  }
+
+  @Override
+  public Size getUncompressedStoreFileSize() {
+    return metrics.getUncompressedStoreFileSize();
   }
 
   /**
    * @return the number of stores
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getStoreCount} instead.
    */
+  @Deprecated
   public int getStores() {
-    return regionLoadPB.getStores();
+    return metrics.getStoreCount();
   }
 
   /**
    * @return the number of storefiles
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getStoreFileCount} instead.
    */
+  @Deprecated
   public int getStorefiles() {
-    return regionLoadPB.getStorefiles();
+    return metrics.getStoreFileCount();
   }
 
   /**
    * @return the total size of the storefiles, in MB
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getStoreFileSize} instead.
    */
+  @Deprecated
   public int getStorefileSizeMB() {
-    return regionLoadPB.getStorefileSizeMB();
+    return (int) metrics.getStoreFileSize().get(Size.Unit.MEGABYTE);
   }
 
   /**
    * @return the memstore size, in MB
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getMemStoreSize} instead.
    */
+  @Deprecated
   public int getMemStoreSizeMB() {
-    return regionLoadPB.getMemStoreSizeMB();
+    return (int) metrics.getMemStoreSize().get(Size.Unit.MEGABYTE);
   }
 
   /**
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
    *             ((<a href="https://issues.apache.org/jira/browse/HBASE-3935">HBASE-3935</a>)).
-   *             Use {@link #getStorefileIndexSizeKB()} instead.
+   *             Use {@link #getStoreFileRootLevelIndexSize} instead.
    */
   @Deprecated
   public int getStorefileIndexSizeMB() {
     // Return value divided by 1024
-    return (int) (regionLoadPB.getStorefileIndexSizeKB() >> 10);
+    return (getRootIndexSizeKB() >> 10);
   }
 
-  public long getStorefileIndexSizeKB() {
-    return regionLoadPB.getStorefileIndexSizeKB();
+  /**
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getStoreFileRootLevelIndexSize()} instead.
+   */
+  @Deprecated
+  public int getStorefileIndexSizeKB() {
+    return getRootIndexSizeKB();
   }
 
   /**
    * @return the number of requests made to region
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getRequestCount()} instead.
    */
+  @Deprecated
   public long getRequestsCount() {
-    return getReadRequestsCount() + getWriteRequestsCount();
+    return metrics.getRequestCount();
   }
 
   /**
    * @return the number of read requests made to region
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getReadRequestCount} instead.
    */
+  @Deprecated
   public long getReadRequestsCount() {
-    return regionLoadPB.getReadRequestsCount();
+    return metrics.getReadRequestCount();
   }
 
   /**
    * @return the number of filtered read requests made to region
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getFilteredReadRequestCount} instead.
    */
+  @Deprecated
   public long getFilteredReadRequestsCount() {
-    return regionLoadPB.getFilteredReadRequestsCount();
+    return metrics.getFilteredReadRequestCount();
   }
 
   /**
    * @return the number of write requests made to region
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getWriteRequestCount} instead.
    */
+  @Deprecated
   public long getWriteRequestsCount() {
-    return regionLoadPB.getWriteRequestsCount();
+    return metrics.getWriteRequestCount();
   }
 
   /**
    * @return The current total size of root-level indexes for the region, in KB.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getStoreFileRootLevelIndexSize} instead.
    */
+  @Deprecated
   public int getRootIndexSizeKB() {
-    return regionLoadPB.getRootIndexSizeKB();
+    return (int) metrics.getStoreFileRootLevelIndexSize().get(Size.Unit.KILOBYTE);
   }
 
   /**
    * @return The total size of all index blocks, not just the root level, in KB.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getStoreFileUncompressedDataIndexSize} instead.
    */
+  @Deprecated
   public int getTotalStaticIndexSizeKB() {
-    return regionLoadPB.getTotalStaticIndexSizeKB();
+    return (int) metrics.getStoreFileUncompressedDataIndexSize().get(Size.Unit.KILOBYTE);
   }
 
   /**
    * @return The total size of all Bloom filter blocks, not just loaded into the
    * block cache, in KB.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getBloomFilterSize} instead.
    */
+  @Deprecated
   public int getTotalStaticBloomSizeKB() {
-    return regionLoadPB.getTotalStaticBloomSizeKB();
+    return (int) metrics.getBloomFilterSize().get(Size.Unit.KILOBYTE);
   }
 
   /**
    * @return the total number of kvs in current compaction
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getCompactingCellCount} instead.
    */
+  @Deprecated
   public long getTotalCompactingKVs() {
-    return regionLoadPB.getTotalCompactingKVs();
+    return metrics.getCompactingCellCount();
   }
 
   /**
    * @return the number of already compacted kvs in current compaction
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getCompactedCellCount} instead.
    */
+  @Deprecated
   public long getCurrentCompactedKVs() {
-    return regionLoadPB.getCurrentCompactedKVs();
+    return metrics.getCompactedCellCount();
   }
 
   /**
    * This does not really belong inside RegionLoad but its being done in the name of expediency.
    * @return the completed sequence Id for the region
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getCompletedSequenceId} instead.
    */
+  @Deprecated
   public long getCompleteSequenceId() {
-    return regionLoadPB.getCompleteSequenceId();
+    return metrics.getCompletedSequenceId();
   }
 
   /**
    * @return completed sequence id per store.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getStoreSequenceId} instead.
    */
-  public List<StoreSequenceId> getStoreCompleteSequenceId() {
-    return regionLoadPB.getStoreCompleteSequenceIdList();
+  @Deprecated
+  public List<ClusterStatusProtos.StoreSequenceId> getStoreCompleteSequenceId() {
+    return metrics.getStoreSequenceId().entrySet().stream()
+        .map(s -> ClusterStatusProtos.StoreSequenceId.newBuilder()
+                  .setFamilyName(UnsafeByteOperations.unsafeWrap(s.getKey()))
+                  .setSequenceId(s.getValue())
+                  .build())
+        .collect(Collectors.toList());
   }
 
   /**
    * @return the uncompressed size of the storefiles in MB.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getUncompressedStoreFileSize} instead.
    */
+  @Deprecated
   public int getStoreUncompressedSizeMB() {
-    return regionLoadPB.getStoreUncompressedSizeMB();
+    return (int) metrics.getUncompressedStoreFileSize().get(Size.Unit.KILOBYTE);
   }
 
   /**
    * @return the data locality of region in the regionserver.
    */
+  @Override
   public float getDataLocality() {
-    if (regionLoadPB.hasDataLocality()) {
-      return regionLoadPB.getDataLocality();
-    }
-    return 0.0f;
+    return metrics.getDataLocality();
+  }
+
+  @Override
+  public long getLastMajorCompactionTimestamp() {
+    return metrics.getLastMajorCompactionTimestamp();
   }
 
   /**
    * @return the timestamp of the oldest hfile for any store of this region.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+   *             Use {@link #getLastMajorCompactionTimestamp} instead.
    */
+  @Deprecated
   public long getLastMajorCompactionTs() {
-    return regionLoadPB.getLastMajorCompactionTs();
+    return metrics.getLastMajorCompactionTimestamp();
   }
 
   /**
@@ -208,47 +365,43 @@ public class RegionLoad {
   public String toString() {
     StringBuilder sb = Strings.appendKeyValue(new StringBuilder(), "numberOfStores",
         this.getStores());
-    sb = Strings.appendKeyValue(sb, "numberOfStorefiles",
-        this.getStorefiles());
-    sb = Strings.appendKeyValue(sb, "storefileUncompressedSizeMB",
-      this.getStoreUncompressedSizeMB());
-    sb = Strings.appendKeyValue(sb, "lastMajorCompactionTimestamp",
-      this.getLastMajorCompactionTs());
-    sb = Strings.appendKeyValue(sb, "storefileSizeMB",
-        this.getStorefileSizeMB());
+    Strings.appendKeyValue(sb, "numberOfStorefiles", this.getStorefiles());
+    Strings.appendKeyValue(sb, "storefileUncompressedSizeMB",
+        this.getStoreUncompressedSizeMB());
+    Strings.appendKeyValue(sb, "lastMajorCompactionTimestamp",
+        this.getLastMajorCompactionTs());
+    Strings.appendKeyValue(sb, "storefileSizeMB", this.getStorefileSizeMB());
     if (this.getStoreUncompressedSizeMB() != 0) {
-      sb = Strings.appendKeyValue(sb, "compressionRatio",
+      Strings.appendKeyValue(sb, "compressionRatio",
           String.format("%.4f", (float) this.getStorefileSizeMB() /
               (float) this.getStoreUncompressedSizeMB()));
     }
-    sb = Strings.appendKeyValue(sb, "memstoreSizeMB",
+    Strings.appendKeyValue(sb, "memstoreSizeMB",
         this.getMemStoreSizeMB());
-    sb = Strings.appendKeyValue(sb, "storefileIndexSizeKB",
-        this.getStorefileIndexSizeKB());
-    sb = Strings.appendKeyValue(sb, "readRequestsCount",
+    Strings.appendKeyValue(sb, "readRequestsCount",
         this.getReadRequestsCount());
-    sb = Strings.appendKeyValue(sb, "writeRequestsCount",
+    Strings.appendKeyValue(sb, "writeRequestsCount",
         this.getWriteRequestsCount());
-    sb = Strings.appendKeyValue(sb, "rootIndexSizeKB",
+    Strings.appendKeyValue(sb, "rootIndexSizeKB",
         this.getRootIndexSizeKB());
-    sb = Strings.appendKeyValue(sb, "totalStaticIndexSizeKB",
+    Strings.appendKeyValue(sb, "totalStaticIndexSizeKB",
         this.getTotalStaticIndexSizeKB());
-    sb = Strings.appendKeyValue(sb, "totalStaticBloomSizeKB",
+    Strings.appendKeyValue(sb, "totalStaticBloomSizeKB",
         this.getTotalStaticBloomSizeKB());
-    sb = Strings.appendKeyValue(sb, "totalCompactingKVs",
+    Strings.appendKeyValue(sb, "totalCompactingKVs",
         this.getTotalCompactingKVs());
-    sb = Strings.appendKeyValue(sb, "currentCompactedKVs",
+    Strings.appendKeyValue(sb, "currentCompactedKVs",
         this.getCurrentCompactedKVs());
     float compactionProgressPct = Float.NaN;
     if (this.getTotalCompactingKVs() > 0) {
       compactionProgressPct = ((float) this.getCurrentCompactedKVs() /
           (float) this.getTotalCompactingKVs());
     }
-    sb = Strings.appendKeyValue(sb, "compactionProgressPct",
+    Strings.appendKeyValue(sb, "compactionProgressPct",
         compactionProgressPct);
-    sb = Strings.appendKeyValue(sb, "completeSequenceId",
+    Strings.appendKeyValue(sb, "completeSequenceId",
         this.getCompleteSequenceId());
-    sb = Strings.appendKeyValue(sb, "dataLocality",
+    Strings.appendKeyValue(sb, "dataLocality",
         this.getDataLocality());
     return sb.toString();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetrics.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetrics.java
new file mode 100644
index 0000000..e73683f
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetrics.java
@@ -0,0 +1,147 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * 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;
+
+import java.util.Map;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+  * Encapsulates per-region load metrics.
+  */
+@InterfaceAudience.Public
+public interface RegionMetrics {
+
+  /**
+   * @return the region name
+   */
+  byte[] getRegionName();
+
+  /**
+   * @return the number of stores
+   */
+  int getStoreCount();
+
+  /**
+   * @return the number of storefiles
+   */
+  int getStoreFileCount();
+
+  /**
+   * @return the total size of the storefiles
+   */
+  Size getStoreFileSize();
+
+  /**
+   * @return the memstore size
+   */
+  Size getMemStoreSize();
+
+  /**
+   * @return the number of read requests made to region
+   */
+  long getReadRequestCount();
+
+  /**
+   * @return the number of write requests made to region
+   */
+  long getWriteRequestCount();
+
+  /**
+   * @return the number of write requests and read requests made to region
+   */
+  default long getRequestCount() {
+    return getReadRequestCount() + getWriteRequestCount();
+  }
+
+  /**
+   * @return the region name as a string
+   */
+  default String getNameAsString() {
+    return Bytes.toStringBinary(getRegionName());
+  }
+
+  /**
+   * @return the number of filtered read requests made to region
+   */
+  long getFilteredReadRequestCount();
+
+  /**
+   * TODO: why we pass the same value to different counters? Currently, the value from
+   * getStoreFileIndexSize() is same with getStoreFileRootLevelIndexSize()
+   * see HRegionServer#createRegionLoad.
+   * @return The current total size of root-level indexes for the region
+   */
+  Size getStoreFileIndexSize();
+
+  /**
+   * @return The current total size of root-level indexes for the region
+   */
+  Size getStoreFileRootLevelIndexSize();
+
+  /**
+   * @return The total size of all index blocks, not just the root level
+   */
+  Size getStoreFileUncompressedDataIndexSize();
+
+  /**
+   * @return The total size of all Bloom filter blocks, not just loaded into the block cache
+   */
+  Size getBloomFilterSize();
+
+  /**
+   * @return the total number of cells in current compaction
+   */
+  long getCompactingCellCount();
+
+  /**
+   * @return the number of already compacted kvs in current compaction
+   */
+  long getCompactedCellCount();
+
+  /**
+   * This does not really belong inside RegionLoad but its being done in the name of expediency.
+   * @return the completed sequence Id for the region
+   */
+  long getCompletedSequenceId();
+
+  /**
+   * @return completed sequence id per store.
+   */
+  Map<byte[], Long> getStoreSequenceId();
+
+
+  /**
+   * @return the uncompressed size of the storefiles
+   */
+  Size getUncompressedStoreFileSize();
+
+  /**
+   * @return the data locality of region in the regionserver.
+   */
+  float getDataLocality();
+
+  /**
+   * @return the timestamp of the oldest hfile for any store of this region.
+   */
+  long getLastMajorCompactionTimestamp();
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetricsBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetricsBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetricsBuilder.java
new file mode 100644
index 0000000..0fb6d3d
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionMetricsBuilder.java
@@ -0,0 +1,437 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * 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;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hbase.util.Strings;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+
+@InterfaceAudience.Private
+public final class RegionMetricsBuilder {
+
+  public static RegionMetrics toRegionMetrics(ClusterStatusProtos.RegionLoad regionLoadPB) {
+    return RegionMetricsBuilder
+        .newBuilder(regionLoadPB.getRegionSpecifier().getValue().toByteArray())
+        .setBloomFilterSize(new Size(regionLoadPB.getTotalStaticBloomSizeKB(), Size.Unit.KILOBYTE))
+        .setCompactedCellCount(regionLoadPB.getCurrentCompactedKVs())
+        .setCompactingCellCount(regionLoadPB.getTotalCompactingKVs())
+        .setCompletedSequenceId(regionLoadPB.getCompleteSequenceId())
+        .setDataLocality(regionLoadPB.hasDataLocality() ? regionLoadPB.getDataLocality() : 0.0f)
+        .setFilteredReadRequestCount(regionLoadPB.getFilteredReadRequestsCount())
+        .setStoreFileUncompressedDataIndexSize(new Size(regionLoadPB.getTotalStaticIndexSizeKB(),
+          Size.Unit.KILOBYTE))
+        .setLastMajorCompactionTimestamp(regionLoadPB.getLastMajorCompactionTs())
+        .setMemStoreSize(new Size(regionLoadPB.getMemStoreSizeMB(), Size.Unit.MEGABYTE))
+        .setReadRequestCount(regionLoadPB.getReadRequestsCount())
+        .setWriteRequestCount(regionLoadPB.getWriteRequestsCount())
+        .setStoreFileIndexSize(new Size(regionLoadPB.getStorefileIndexSizeKB(),
+          Size.Unit.KILOBYTE))
+        .setStoreFileRootLevelIndexSize(new Size(regionLoadPB.getRootIndexSizeKB(),
+          Size.Unit.KILOBYTE))
+        .setStoreCount(regionLoadPB.getStores())
+        .setStoreFileCount(regionLoadPB.getStorefiles())
+        .setStoreFileSize(new Size(regionLoadPB.getStorefileSizeMB(), Size.Unit.MEGABYTE))
+        .setStoreSequenceIds(regionLoadPB.getStoreCompleteSequenceIdList().stream()
+          .collect(Collectors.toMap(
+            (ClusterStatusProtos.StoreSequenceId s) -> s.getFamilyName().toByteArray(),
+              ClusterStatusProtos.StoreSequenceId::getSequenceId)))
+        .setUncompressedStoreFileSize(
+          new Size(regionLoadPB.getStoreUncompressedSizeMB(),Size.Unit.MEGABYTE))
+        .build();
+  }
+
+  private static List<ClusterStatusProtos.StoreSequenceId> toStoreSequenceId(
+      Map<byte[], Long> ids) {
+    return ids.entrySet().stream()
+        .map(e -> ClusterStatusProtos.StoreSequenceId.newBuilder()
+          .setFamilyName(UnsafeByteOperations.unsafeWrap(e.getKey()))
+          .setSequenceId(e.getValue())
+          .build())
+        .collect(Collectors.toList());
+  }
+
+  public static ClusterStatusProtos.RegionLoad toRegionLoad(RegionMetrics regionMetrics) {
+    return ClusterStatusProtos.RegionLoad.newBuilder()
+        .setRegionSpecifier(HBaseProtos.RegionSpecifier
+          .newBuilder().setType(HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME)
+          .setValue(UnsafeByteOperations.unsafeWrap(regionMetrics.getRegionName()))
+          .build())
+        .setTotalStaticBloomSizeKB((int) regionMetrics.getBloomFilterSize()
+          .get(Size.Unit.KILOBYTE))
+        .setCurrentCompactedKVs(regionMetrics.getCompactedCellCount())
+        .setTotalCompactingKVs(regionMetrics.getCompactingCellCount())
+        .setCompleteSequenceId(regionMetrics.getCompletedSequenceId())
+        .setDataLocality(regionMetrics.getDataLocality())
+        .setFilteredReadRequestsCount(regionMetrics.getFilteredReadRequestCount())
+        .setTotalStaticIndexSizeKB((int) regionMetrics.getStoreFileUncompressedDataIndexSize()
+          .get(Size.Unit.KILOBYTE))
+        .setLastMajorCompactionTs(regionMetrics.getLastMajorCompactionTimestamp())
+        .setMemStoreSizeMB((int) regionMetrics.getMemStoreSize().get(Size.Unit.MEGABYTE))
+        .setReadRequestsCount(regionMetrics.getReadRequestCount())
+        .setWriteRequestsCount(regionMetrics.getWriteRequestCount())
+        .setStorefileIndexSizeKB((long) regionMetrics.getStoreFileIndexSize()
+          .get(Size.Unit.KILOBYTE))
+        .setRootIndexSizeKB((int) regionMetrics.getStoreFileRootLevelIndexSize()
+          .get(Size.Unit.KILOBYTE))
+        .setStores(regionMetrics.getStoreCount())
+        .setStorefiles(regionMetrics.getStoreCount())
+        .setStorefileSizeMB((int) regionMetrics.getStoreFileSize().get(Size.Unit.MEGABYTE))
+        .addAllStoreCompleteSequenceId(toStoreSequenceId(regionMetrics.getStoreSequenceId()))
+        .setStoreUncompressedSizeMB(
+          (int) regionMetrics.getUncompressedStoreFileSize().get(Size.Unit.MEGABYTE))
+        .build();
+  }
+
+  public static RegionMetricsBuilder newBuilder(byte[] name) {
+    return new RegionMetricsBuilder(name);
+  }
+
+  private final byte[] name;
+  private int storeCount;
+  private int storeFileCount;
+  private long compactingCellCount;
+  private long compactedCellCount;
+  private Size storeFileSize = Size.ZERO;
+  private Size memStoreSize = Size.ZERO;
+  private Size indexSize = Size.ZERO;
+  private Size rootLevelIndexSize = Size.ZERO;
+  private Size uncompressedDataIndexSize = Size.ZERO;
+  private Size bloomFilterSize = Size.ZERO;
+  private Size uncompressedStoreFileSize = Size.ZERO;
+  private long writeRequestCount;
+  private long readRequestCount;
+  private long filteredReadRequestCount;
+  private long completedSequenceId;
+  private Map<byte[], Long> storeSequenceIds = Collections.emptyMap();
+  private float dataLocality;
+  private long lastMajorCompactionTimestamp;
+  private RegionMetricsBuilder(byte[] name) {
+    this.name = name;
+  }
+
+  public RegionMetricsBuilder setStoreCount(int value) {
+    this.storeCount = value;
+    return this;
+  }
+  public RegionMetricsBuilder setStoreFileCount(int value) {
+    this.storeFileCount = value;
+    return this;
+  }
+  public RegionMetricsBuilder setCompactingCellCount(long value) {
+    this.compactingCellCount = value;
+    return this;
+  }
+  public RegionMetricsBuilder setCompactedCellCount(long value) {
+    this.compactedCellCount = value;
+    return this;
+  }
+  public RegionMetricsBuilder setStoreFileSize(Size value) {
+    this.storeFileSize = value;
+    return this;
+  }
+  public RegionMetricsBuilder setMemStoreSize(Size value) {
+    this.memStoreSize = value;
+    return this;
+  }
+  public RegionMetricsBuilder setStoreFileIndexSize(Size value) {
+    this.indexSize = value;
+    return this;
+  }
+  public RegionMetricsBuilder setStoreFileRootLevelIndexSize(Size value) {
+    this.rootLevelIndexSize = value;
+    return this;
+  }
+  public RegionMetricsBuilder setStoreFileUncompressedDataIndexSize(Size value) {
+    this.uncompressedDataIndexSize = value;
+    return this;
+  }
+  public RegionMetricsBuilder setBloomFilterSize(Size value) {
+    this.bloomFilterSize = value;
+    return this;
+  }
+  public RegionMetricsBuilder setUncompressedStoreFileSize(Size value) {
+    this.uncompressedStoreFileSize = value;
+    return this;
+  }
+  public RegionMetricsBuilder setWriteRequestCount(long value) {
+    this.writeRequestCount = value;
+    return this;
+  }
+  public RegionMetricsBuilder setReadRequestCount(long value) {
+    this.readRequestCount = value;
+    return this;
+  }
+  public RegionMetricsBuilder setFilteredReadRequestCount(long value) {
+    this.filteredReadRequestCount = value;
+    return this;
+  }
+  public RegionMetricsBuilder setCompletedSequenceId(long value) {
+    this.completedSequenceId = value;
+    return this;
+  }
+  public RegionMetricsBuilder setStoreSequenceIds(Map<byte[], Long> value) {
+    this.storeSequenceIds = value;
+    return this;
+  }
+  public RegionMetricsBuilder setDataLocality(float value) {
+    this.dataLocality = value;
+    return this;
+  }
+  public RegionMetricsBuilder setLastMajorCompactionTimestamp(long value) {
+    this.lastMajorCompactionTimestamp = value;
+    return this;
+  }
+
+  public RegionMetrics build() {
+    return new RegionMetricsImpl(name,
+        storeCount,
+        storeFileCount,
+        compactingCellCount,
+        compactedCellCount,
+        storeFileSize,
+        memStoreSize,
+        indexSize,
+        rootLevelIndexSize,
+        uncompressedDataIndexSize,
+        bloomFilterSize,
+        uncompressedStoreFileSize,
+        writeRequestCount,
+        readRequestCount,
+        filteredReadRequestCount,
+        completedSequenceId,
+        storeSequenceIds,
+        dataLocality,
+        lastMajorCompactionTimestamp);
+  }
+
+  private static class RegionMetricsImpl implements RegionMetrics {
+    private final byte[] name;
+    private final int storeCount;
+    private final int storeFileCount;
+    private final long compactingCellCount;
+    private final long compactedCellCount;
+    private final Size storeFileSize;
+    private final Size memStoreSize;
+    private final Size indexSize;
+    private final Size rootLevelIndexSize;
+    private final Size uncompressedDataIndexSize;
+    private final Size bloomFilterSize;
+    private final Size uncompressedStoreFileSize;
+    private final long writeRequestCount;
+    private final long readRequestCount;
+    private final long filteredReadRequestCount;
+    private final long completedSequenceId;
+    private final Map<byte[], Long> storeSequenceIds;
+    private final float dataLocality;
+    private final long lastMajorCompactionTimestamp;
+    RegionMetricsImpl(byte[] name,
+        int storeCount,
+        int storeFileCount,
+        final long compactingCellCount,
+        long compactedCellCount,
+        Size storeFileSize,
+        Size memStoreSize,
+        Size indexSize,
+        Size rootLevelIndexSize,
+        Size uncompressedDataIndexSize,
+        Size bloomFilterSize,
+        Size uncompressedStoreFileSize,
+        long writeRequestCount,
+        long readRequestCount,
+        long filteredReadRequestCount,
+        long completedSequenceId,
+        Map<byte[], Long> storeSequenceIds,
+        float dataLocality,
+        long lastMajorCompactionTimestamp) {
+      this.name = Preconditions.checkNotNull(name);
+      this.storeCount = storeCount;
+      this.storeFileCount = storeFileCount;
+      this.compactingCellCount = compactingCellCount;
+      this.compactedCellCount = compactedCellCount;
+      this.storeFileSize = Preconditions.checkNotNull(storeFileSize);
+      this.memStoreSize = Preconditions.checkNotNull(memStoreSize);
+      this.indexSize = Preconditions.checkNotNull(indexSize);
+      this.rootLevelIndexSize = Preconditions.checkNotNull(rootLevelIndexSize);
+      this.uncompressedDataIndexSize = Preconditions.checkNotNull(uncompressedDataIndexSize);
+      this.bloomFilterSize = Preconditions.checkNotNull(bloomFilterSize);
+      this.uncompressedStoreFileSize = Preconditions.checkNotNull(uncompressedStoreFileSize);
+      this.writeRequestCount = writeRequestCount;
+      this.readRequestCount = readRequestCount;
+      this.filteredReadRequestCount = filteredReadRequestCount;
+      this.completedSequenceId = completedSequenceId;
+      this.storeSequenceIds = Preconditions.checkNotNull(storeSequenceIds);
+      this.dataLocality = dataLocality;
+      this.lastMajorCompactionTimestamp = lastMajorCompactionTimestamp;
+    }
+
+    @Override
+    public byte[] getRegionName() {
+      return name;
+    }
+
+    @Override
+    public int getStoreCount() {
+      return storeCount;
+    }
+
+    @Override
+    public int getStoreFileCount() {
+      return storeFileCount;
+    }
+
+    @Override
+    public Size getStoreFileSize() {
+      return storeFileSize;
+    }
+
+    @Override
+    public Size getMemStoreSize() {
+      return memStoreSize;
+    }
+
+    @Override
+    public long getReadRequestCount() {
+      return readRequestCount;
+    }
+
+    @Override
+    public long getFilteredReadRequestCount() {
+      return filteredReadRequestCount;
+    }
+
+    @Override
+    public long getWriteRequestCount() {
+      return writeRequestCount;
+    }
+
+    @Override
+    public Size getStoreFileIndexSize() {
+      return indexSize;
+    }
+
+    @Override
+    public Size getStoreFileRootLevelIndexSize() {
+      return rootLevelIndexSize;
+    }
+
+    @Override
+    public Size getStoreFileUncompressedDataIndexSize() {
+      return uncompressedDataIndexSize;
+    }
+
+    @Override
+    public Size getBloomFilterSize() {
+      return bloomFilterSize;
+    }
+
+    @Override
+    public long getCompactingCellCount() {
+      return compactingCellCount;
+    }
+
+    @Override
+    public long getCompactedCellCount() {
+      return compactedCellCount;
+    }
+
+    @Override
+    public long getCompletedSequenceId() {
+      return completedSequenceId;
+    }
+
+    @Override
+    public Map<byte[], Long> getStoreSequenceId() {
+      return Collections.unmodifiableMap(storeSequenceIds);
+    }
+
+    @Override
+    public Size getUncompressedStoreFileSize() {
+      return uncompressedStoreFileSize;
+    }
+
+    @Override
+    public float getDataLocality() {
+      return dataLocality;
+    }
+
+    @Override
+    public long getLastMajorCompactionTimestamp() {
+      return lastMajorCompactionTimestamp;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = Strings.appendKeyValue(new StringBuilder(), "storeCount",
+          this.getStoreCount());
+      Strings.appendKeyValue(sb, "storeFileCount",
+          this.getStoreFileCount());
+      Strings.appendKeyValue(sb, "uncompressedStoreFileSize",
+          this.getUncompressedStoreFileSize());
+      Strings.appendKeyValue(sb, "lastMajorCompactionTimestamp",
+          this.getLastMajorCompactionTimestamp());
+      Strings.appendKeyValue(sb, "storeFileSize",
+          this.getStoreFileSize());
+      if (this.getUncompressedStoreFileSize().get() != 0) {
+        Strings.appendKeyValue(sb, "compressionRatio",
+            String.format("%.4f",
+                (float) this.getStoreFileSize().get(Size.Unit.MEGABYTE) /
+                (float) this.getUncompressedStoreFileSize().get(Size.Unit.MEGABYTE)));
+      }
+      Strings.appendKeyValue(sb, "memStoreSize",
+          this.getMemStoreSize());
+      Strings.appendKeyValue(sb, "readRequestCount",
+          this.getReadRequestCount());
+      Strings.appendKeyValue(sb, "writeRequestCount",
+          this.getWriteRequestCount());
+      Strings.appendKeyValue(sb, "rootLevelIndexSize",
+          this.getStoreFileRootLevelIndexSize());
+      Strings.appendKeyValue(sb, "uncompressedDataIndexSize",
+          this.getStoreFileUncompressedDataIndexSize());
+      Strings.appendKeyValue(sb, "bloomFilterSize",
+          this.getBloomFilterSize());
+      Strings.appendKeyValue(sb, "compactingCellCount",
+          this.getCompactingCellCount());
+      Strings.appendKeyValue(sb, "compactedCellCount",
+          this.getCompactedCellCount());
+      float compactionProgressPct = Float.NaN;
+      if (this.getCompactingCellCount() > 0) {
+        compactionProgressPct = ((float) this.getCompactedCellCount() /
+            (float) this.getCompactingCellCount());
+      }
+      Strings.appendKeyValue(sb, "compactionProgressPct",
+          compactionProgressPct);
+      Strings.appendKeyValue(sb, "completedSequenceId",
+          this.getCompletedSequenceId());
+      Strings.appendKeyValue(sb, "dataLocality",
+          this.getDataLocality());
+      return sb.toString();
+    }
+  }
+
+}


[08/26] hbase git commit: HBASE--17248 fix javadoc in SimpleRegionNormalizer

Posted by zh...@apache.org.
HBASE--17248 fix javadoc in SimpleRegionNormalizer

Signed-off-by: Michael Stack <st...@apache.org>


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

Branch: refs/heads/HBASE-19397
Commit: 37bf54a516f038a86b818d872452fb7169dd5a42
Parents: 5f02bf5
Author: Daisuke <po...@gmail.com>
Authored: Sat Dec 3 19:03:38 2016 +0900
Committer: Michael Stack <st...@apache.org>
Committed: Thu Dec 21 21:35:08 2017 -0800

----------------------------------------------------------------------
 .../master/normalizer/SimpleRegionNormalizer.java     | 14 +++++++-------
 1 file changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/37bf54a5/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
index 767324a..38cf847 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.java
@@ -43,13 +43,13 @@ import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
  * Logic in use:
  *
  *  <ol>
- *  <li> get all regions of a given table
- *  <li> get avg size S of each region (by total size of store files reported in RegionLoad)
- *  <li> If biggest region is bigger than S * 2, it is kindly requested to split,
- *    and normalization stops
- *  <li> Otherwise, two smallest region R1 and its smallest neighbor R2 are kindly requested
- *    to merge, if R1 + R1 &lt;  S, and normalization stops
- *  <li> Otherwise, no action is performed
+ *  <li> Get all regions of a given table
+ *  <li> Get avg size S of each region (by total size of store files reported in RegionLoad)
+ *  <li> Seek every single region one by one. If a region R0 is bigger than S * 2, it is
+ *  kindly requested to split. Thereon evaluate the next region R1
+ *  <li> Otherwise, if R0 + R1 is smaller than S, R0 and R1 are kindly requested to merge.
+ *  Thereon evaluate the next region R2
+ *  <li> Otherwise, R1 is evaluated
  * </ol>
  * <p>
  * Region sizes are coarse and approximate on the order of megabytes. Additionally,


[12/26] hbase git commit: HBASE-19591 Cleanup the usage of ReplicationAdmin from hbase-shell

Posted by zh...@apache.org.
HBASE-19591 Cleanup the usage of ReplicationAdmin from hbase-shell


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

Branch: refs/heads/HBASE-19397
Commit: 7501e644267e063601578b293757415ac04360e1
Parents: 001bce5
Author: Guanghao Zhang <zg...@apache.org>
Authored: Fri Dec 22 14:14:10 2017 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Fri Dec 22 18:32:49 2017 +0800

----------------------------------------------------------------------
 hbase-shell/src/main/ruby/hbase/replication_admin.rb | 8 +++-----
 1 file changed, 3 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7501e644/hbase-shell/src/main/ruby/hbase/replication_admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/replication_admin.rb b/hbase-shell/src/main/ruby/hbase/replication_admin.rb
index 949bf68..f80c547 100644
--- a/hbase-shell/src/main/ruby/hbase/replication_admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/replication_admin.rb
@@ -19,21 +19,19 @@
 
 include Java
 
-java_import org.apache.hadoop.hbase.client.replication.ReplicationAdmin
 java_import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil
 java_import org.apache.hadoop.hbase.replication.ReplicationPeerConfig
 java_import org.apache.hadoop.hbase.util.Bytes
 java_import org.apache.hadoop.hbase.zookeeper.ZKConfig
 java_import org.apache.hadoop.hbase.TableName
 
-# Wrapper for org.apache.hadoop.hbase.client.replication.ReplicationAdmin
+# Used for replication administrative operations.
 
 module Hbase
   class RepAdmin
     include HBaseConstants
 
     def initialize(configuration)
-      @replication_admin = ReplicationAdmin.new(configuration)
       @configuration = configuration
       @admin = ConnectionFactory.createConnection(configuration).getAdmin
     end
@@ -272,10 +270,10 @@ module Hbase
     end
 
     def set_peer_replicate_all(id, replicate_all)
-      rpc = @replication_admin.getPeerConfig(id)
+      rpc = get_peer_config(id)
       return if rpc.nil?
       rpc.setReplicateAllUserTables(replicate_all)
-      @replication_admin.updatePeerConfig(id, rpc)
+      @admin.updateReplicationPeerConfig(id, rpc)
     end
 
     # Set exclude namespaces config for the specified peer


[11/26] hbase git commit: HBASE-19590 Remove the duplicate code in deprecated ReplicationAdmin

Posted by zh...@apache.org.
HBASE-19590 Remove the duplicate code in deprecated ReplicationAdmin


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

Branch: refs/heads/HBASE-19397
Commit: 001bce54e7835fd1877bec1976b227dfab45e2e3
Parents: 8ec0aa0
Author: Guanghao Zhang <zg...@apache.org>
Authored: Fri Dec 22 12:02:19 2017 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Fri Dec 22 18:32:15 2017 +0800

----------------------------------------------------------------------
 .../client/replication/ReplicationAdmin.java    | 103 +------------------
 .../replication/TestReplicationAdmin.java       |   2 +-
 2 files changed, 3 insertions(+), 102 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/001bce54/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
index a5081cb..78d4fbb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
@@ -23,10 +23,8 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 import java.util.TreeMap;
 import java.util.regex.Pattern;
 
@@ -44,7 +42,6 @@ import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 
 /**
  * <p>
@@ -131,8 +128,6 @@ public class ReplicationAdmin implements Closeable {
   @Deprecated
   public void addPeer(String id, ReplicationPeerConfig peerConfig) throws ReplicationException,
       IOException {
-    checkNamespacesAndTableCfsConfigConflict(peerConfig.getNamespaces(),
-      peerConfig.getTableCFsMap());
     this.admin.addReplicationPeer(id, peerConfig);
   }
 
@@ -256,36 +251,7 @@ public class ReplicationAdmin implements Closeable {
   @Deprecated
   public void appendPeerTableCFs(String id, Map<TableName, ? extends Collection<String>> tableCfs)
       throws ReplicationException, IOException {
-    if (tableCfs == null) {
-      throw new ReplicationException("tableCfs is null");
-    }
-    ReplicationPeerConfig peerConfig = admin.getReplicationPeerConfig(id);
-    Map<TableName, List<String>> preTableCfs = peerConfig.getTableCFsMap();
-    if (preTableCfs == null) {
-      setPeerTableCFs(id, tableCfs);
-      return;
-    }
-    for (Map.Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
-      TableName table = entry.getKey();
-      Collection<String> appendCfs = entry.getValue();
-      if (preTableCfs.containsKey(table)) {
-        List<String> cfs = preTableCfs.get(table);
-        if (cfs == null || appendCfs == null || appendCfs.isEmpty()) {
-          preTableCfs.put(table, null);
-        } else {
-          Set<String> cfSet = new HashSet<>(cfs);
-          cfSet.addAll(appendCfs);
-          preTableCfs.put(table, Lists.newArrayList(cfSet));
-        }
-      } else {
-        if (appendCfs == null || appendCfs.isEmpty()) {
-          preTableCfs.put(table, null);
-        } else {
-          preTableCfs.put(table, Lists.newArrayList(appendCfs));
-        }
-      }
-    }
-    updatePeerConfig(id, peerConfig);
+    this.admin.appendReplicationPeerTableCFs(id, tableCfs);
   }
 
   /**
@@ -313,42 +279,7 @@ public class ReplicationAdmin implements Closeable {
   @Deprecated
   public void removePeerTableCFs(String id, Map<TableName, ? extends Collection<String>> tableCfs)
       throws ReplicationException, IOException {
-    if (tableCfs == null) {
-      throw new ReplicationException("tableCfs is null");
-    }
-    ReplicationPeerConfig peerConfig = admin.getReplicationPeerConfig(id);
-    Map<TableName, List<String>> preTableCfs = peerConfig.getTableCFsMap();
-    if (preTableCfs == null) {
-      throw new ReplicationException("Table-Cfs for peer" + id + " is null");
-    }
-    for (Map.Entry<TableName, ? extends Collection<String>> entry: tableCfs.entrySet()) {
-
-      TableName table = entry.getKey();
-      Collection<String> removeCfs = entry.getValue();
-      if (preTableCfs.containsKey(table)) {
-        List<String> cfs = preTableCfs.get(table);
-        if (cfs == null && (removeCfs == null || removeCfs.isEmpty())) {
-          preTableCfs.remove(table);
-        } else if (cfs != null && (removeCfs != null && !removeCfs.isEmpty())) {
-          Set<String> cfSet = new HashSet<>(cfs);
-          cfSet.removeAll(removeCfs);
-          if (cfSet.isEmpty()) {
-            preTableCfs.remove(table);
-          } else {
-            preTableCfs.put(table, Lists.newArrayList(cfSet));
-          }
-        } else if (cfs == null && (removeCfs != null && !removeCfs.isEmpty())) {
-          throw new ReplicationException("Cannot remove cf of table: " + table
-              + " which doesn't specify cfs from table-cfs config in peer: " + id);
-        } else if (cfs != null && (removeCfs == null || removeCfs.isEmpty())) {
-          throw new ReplicationException("Cannot remove table: " + table
-              + " which has specified cfs from table-cfs config in peer: " + id);
-        }
-      } else {
-        throw new ReplicationException("No table: " + table + " in table-cfs config of peer: " + id);
-      }
-    }
-    updatePeerConfig(id, peerConfig);
+    this.admin.removeReplicationPeerTableCFs(id, tableCfs);
   }
 
   /**
@@ -455,34 +386,4 @@ public class ReplicationAdmin implements Closeable {
   List<ReplicationPeerDescription> listReplicationPeers() throws IOException {
     return admin.listReplicationPeers();
   }
-
-  /**
-   * Set a namespace in the peer config means that all tables in this namespace
-   * will be replicated to the peer cluster.
-   *
-   * 1. If you already have set a namespace in the peer config, then you can't set any table
-   *    of this namespace to the peer config.
-   * 2. If you already have set a table in the peer config, then you can't set this table's
-   *    namespace to the peer config.
-   *
-   * @param namespaces
-   * @param tableCfs
-   * @throws ReplicationException
-   */
-  private void checkNamespacesAndTableCfsConfigConflict(Set<String> namespaces,
-      Map<TableName, ? extends Collection<String>> tableCfs) throws ReplicationException {
-    if (namespaces == null || namespaces.isEmpty()) {
-      return;
-    }
-    if (tableCfs == null || tableCfs.isEmpty()) {
-      return;
-    }
-    for (Map.Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
-      TableName table = entry.getKey();
-      if (namespaces.contains(table.getNamespaceAsString())) {
-        throw new ReplicationException(
-            "Table-cfs config conflict with namespaces config in peer");
-      }
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/001bce54/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
index 67c635b..8770299 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
@@ -748,4 +748,4 @@ public class TestReplicationAdmin {
     assertEquals(2097152, admin.getPeerConfig(ID_ONE).getBandwidth());
     admin.removePeer(ID_ONE);
   }
-}
+}
\ No newline at end of file


[05/26] hbase git commit: HBASE-19148 Reevaluate default values of configurations; minor ADDENDUM

Posted by zh...@apache.org.
HBASE-19148 Reevaluate default values of configurations; minor ADDENDUM


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

Branch: refs/heads/HBASE-19397
Commit: bcaf2fd20c542137c1d11dd32f2702af93d5ce70
Parents: 4d6b928
Author: Michael Stack <st...@apache.org>
Authored: Thu Dec 21 14:13:23 2017 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Thu Dec 21 14:14:07 2017 -0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/master/HMaster.java      | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/bcaf2fd2/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index d0f435b..e31db82 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -1399,7 +1399,7 @@ public class HMaster extends HRegionServer implements MasterServices {
         }
       }
 
-      boolean isByTable = getConfiguration().getBoolean("hbase.master.loadbalance.bytable", true);
+      boolean isByTable = getConfiguration().getBoolean("hbase.master.loadbalance.bytable", false);
       Map<TableName, Map<ServerName, List<RegionInfo>>> assignmentsByTable =
         this.assignmentManager.getRegionStates().getAssignmentsByTable(!isByTable);
 


[10/26] hbase git commit: HBASE-19589 New regions should always be added with state CLOSED (followup of HBASE-19530)

Posted by zh...@apache.org.
HBASE-19589 New regions should always be added with state CLOSED (followup of HBASE-19530)


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

Branch: refs/heads/HBASE-19397
Commit: 8ec0aa0d709ced78331dd61d28c79f3433198227
Parents: 070c2ee
Author: Apekshit Sharma <ap...@apache.org>
Authored: Thu Dec 21 17:14:25 2017 -0800
Committer: Apekshit Sharma <ap...@apache.org>
Committed: Thu Dec 21 22:57:53 2017 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/MetaTableAccessor.java  | 247 +++++--------------
 .../procedure/CloneSnapshotProcedure.java       |   2 +
 .../hbase/snapshot/RestoreSnapshotHelper.java   |   2 +-
 .../hadoop/hbase/HBaseTestingUtility.java       |   2 +-
 .../hadoop/hbase/util/BaseTestHBaseFsck.java    |  17 --
 .../util/hbck/OfflineMetaRebuildTestCore.java   |   2 +-
 6 files changed, 73 insertions(+), 199 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8ec0aa0d/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
index 4f14192..f75472b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
@@ -1314,16 +1314,14 @@ public class MetaTableAccessor {
   /**
    * Generates and returns a Put containing the region into for the catalog table
    */
-  public static Put makePutFromRegionInfo(RegionInfo regionInfo)
-    throws IOException {
+  public static Put makePutFromRegionInfo(RegionInfo regionInfo) throws IOException {
     return makePutFromRegionInfo(regionInfo, EnvironmentEdgeManager.currentTime());
   }
 
   /**
    * Generates and returns a Put containing the region into for the catalog table
    */
-  public static Put makePutFromRegionInfo(RegionInfo regionInfo, long ts)
-    throws IOException {
+  public static Put makePutFromRegionInfo(RegionInfo regionInfo, long ts) throws IOException {
     Put put = new Put(regionInfo.getRegionName(), ts);
     addRegionInfo(put, regionInfo);
     return put;
@@ -1448,9 +1446,7 @@ public class MetaTableAccessor {
    */
   private static void put(final Table t, final List<Put> puts) throws IOException {
     try {
-      if (METALOG.isDebugEnabled()) {
-        METALOG.debug(mutationsToString(puts));
-      }
+      debugLogMutations(puts);
       t.put(puts);
     } finally {
       t.close();
@@ -1467,9 +1463,7 @@ public class MetaTableAccessor {
     throws IOException {
     Table t = getMetaHTable(connection);
     try {
-      if (METALOG.isDebugEnabled()) {
-        METALOG.debug(mutationsToString(ps));
-      }
+      debugLogMutations(ps);
       t.put(ps);
     } finally {
       t.close();
@@ -1499,9 +1493,7 @@ public class MetaTableAccessor {
     throws IOException {
     Table t = getMetaHTable(connection);
     try {
-      if (METALOG.isDebugEnabled()) {
-        METALOG.debug(mutationsToString(deletes));
-      }
+      debugLogMutations(deletes);
       t.delete(deletes);
     } finally {
       t.close();
@@ -1546,9 +1538,7 @@ public class MetaTableAccessor {
     throws IOException {
     Table t = getMetaHTable(connection);
     try {
-      if (METALOG.isDebugEnabled()) {
-        METALOG.debug(mutationsToString(mutations));
-      }
+      debugLogMutations(mutations);
       t.batch(mutations, null);
     } catch (InterruptedException e) {
       InterruptedIOException ie = new InterruptedIOException(e.getMessage());
@@ -1559,33 +1549,7 @@ public class MetaTableAccessor {
     }
   }
 
-  /**
-   * Adds a hbase:meta row for the specified new region.
-   * @param connection connection we're using
-   * @param regionInfo region information
-   * @throws IOException if problem connecting or updating meta
-   */
-  @VisibleForTesting
-  static void addRegionToMeta(Connection connection,
-                                     RegionInfo regionInfo)
-    throws IOException {
-    putToMetaTable(connection, makePutFromRegionInfo(regionInfo));
-    LOG.info("Added " + regionInfo.getRegionNameAsString());
-  }
-
-  /**
-   * Adds a hbase:meta row for the specified new region to the given catalog table. The
-   * Table is not flushed or closed.
-   * @param meta the Table for META
-   * @param regionInfo region information
-   * @throws IOException if problem connecting or updating meta
-   */
-  public static void addRegionToMeta(Table meta, RegionInfo regionInfo) throws IOException {
-    addRegionToMeta(meta, regionInfo, null, null);
-  }
-
-  static void addRegionStateToPut(Put put, RegionState.State  state)
-      throws IOException {
+  static void addRegionStateToPut(Put put, RegionState.State  state) throws IOException {
     put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
         .setRow(put.getRow())
         .setFamily(HConstants.CATALOG_FAMILY)
@@ -1597,79 +1561,71 @@ public class MetaTableAccessor {
   }
 
   /**
-   * Adds a (single) hbase:meta row for the specified new region and its daughters. Note that this
-   * does not add its daughter's as different rows, but adds information about the daughters
-   * in the same row as the parent. Use
+   * Adds daughter region infos to hbase:meta row for the specified region. Note that this does not
+   * add its daughter's as different rows, but adds information about the daughters in the same row
+   * as the parent. Use
    * {@link #splitRegion(Connection, RegionInfo, RegionInfo, RegionInfo, ServerName,int,boolean)}
    * if you want to do that.
-   * @param meta the Table for META
-   * @param regionInfo region information
+   * @param connection connection we're using
+   * @param regionInfo RegionInfo of parent region
    * @param splitA first split daughter of the parent regionInfo
    * @param splitB second split daughter of the parent regionInfo
    * @throws IOException if problem connecting or updating meta
    */
-  public static void addRegionToMeta(Table meta, RegionInfo regionInfo,
-                                     RegionInfo splitA, RegionInfo splitB) throws IOException {
-    Put put = makePutFromRegionInfo(regionInfo);
-    addRegionStateToPut(put, RegionState.State.CLOSED);
-    addDaughtersToPut(put, splitA, splitB);
-    meta.put(put);
-    if (METALOG.isDebugEnabled()) {
-      METALOG.debug(mutationToString(put));
-    }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Added " + regionInfo.getRegionNameAsString());
+  public static void addSpiltsToParent(Connection connection, RegionInfo regionInfo,
+      RegionInfo splitA, RegionInfo splitB) throws IOException {
+    Table meta = getMetaHTable(connection);
+    try {
+      Put put = makePutFromRegionInfo(regionInfo);
+      addDaughtersToPut(put, splitA, splitB);
+      meta.put(put);
+      debugLogMutation(put);
+      LOG.debug("Added region {}", regionInfo.getRegionNameAsString());
+    } finally {
+      meta.close();
     }
   }
 
   /**
-   * Adds a (single) hbase:meta row for the specified new region and its daughters. Note that this
-   * does not add its daughter's as different rows, but adds information about the daughters
-   * in the same row as the parent. Use
-   * {@link #splitRegion(Connection, RegionInfo, RegionInfo, RegionInfo, ServerName,int,boolean)}
-   * if you want to do that.
+   * Adds a hbase:meta row for the specified new region. Initial state of new region is CLOSED.
    * @param connection connection we're using
    * @param regionInfo region information
-   * @param splitA first split daughter of the parent regionInfo
-   * @param splitB second split daughter of the parent regionInfo
    * @throws IOException if problem connecting or updating meta
    */
-  public static void addRegionToMeta(Connection connection, RegionInfo regionInfo,
-                                     RegionInfo splitA, RegionInfo splitB) throws IOException {
-    Table meta = getMetaHTable(connection);
-    try {
-      addRegionToMeta(meta, regionInfo, splitA, splitB);
-    } finally {
-      meta.close();
-    }
+  @VisibleForTesting
+  public static void addRegionToMeta(Connection connection, RegionInfo regionInfo)
+      throws IOException {
+    addRegionsToMeta(connection, Collections.singletonList(regionInfo), 1);
   }
 
   /**
-   * Adds a hbase:meta row for each of the specified new regions.
+   * Adds a hbase:meta row for each of the specified new regions. Initial state for new regions
+   * is CLOSED.
    * @param connection connection we're using
    * @param regionInfos region information list
    * @throws IOException if problem connecting or updating meta
    */
-  public static void addRegionsToMeta(Connection connection,
-                                      List<RegionInfo> regionInfos, int regionReplication)
-    throws IOException {
+  public static void addRegionsToMeta(Connection connection, List<RegionInfo> regionInfos,
+      int regionReplication) throws IOException {
     addRegionsToMeta(connection, regionInfos, regionReplication, HConstants.LATEST_TIMESTAMP);
   }
   /**
-   * Adds a hbase:meta row for each of the specified new regions.
+   * Adds a hbase:meta row for each of the specified new regions. Initial state for new regions
+   * is CLOSED.
    * @param connection connection we're using
    * @param regionInfos region information list
    * @param regionReplication
    * @param ts desired timestamp
    * @throws IOException if problem connecting or updating meta
    */
-  public static void addRegionsToMeta(Connection connection,
-      List<RegionInfo> regionInfos, int regionReplication, long ts)
-          throws IOException {
+  public static void addRegionsToMeta(Connection connection, List<RegionInfo> regionInfos,
+      int regionReplication, long ts) throws IOException {
     List<Put> puts = new ArrayList<>();
     for (RegionInfo regionInfo : regionInfos) {
       if (RegionReplicaUtil.isDefaultReplica(regionInfo)) {
         Put put = makePutFromRegionInfo(regionInfo, ts);
+        // New regions are added with initial state of CLOSED.
+        addRegionStateToPut(put, RegionState.State.CLOSED);
         // Add empty locations for region replicas so that number of replicas can be cached
         // whenever the primary region is looked up from meta
         for (int i = 1; i < regionReplication; i++) {
@@ -1679,27 +1635,7 @@ public class MetaTableAccessor {
       }
     }
     putsToMetaTable(connection, puts);
-    LOG.info("Added " + puts.size());
-  }
-
-  /**
-   * Adds a daughter region entry to meta.
-   * @param regionInfo the region to put
-   * @param sn the location of the region
-   * @param openSeqNum the latest sequence number obtained when the region was open
-   */
-  public static void addDaughter(final Connection connection,
-      final RegionInfo regionInfo, final ServerName sn, final long openSeqNum)
-      throws NotAllMetaRegionsOnlineException, IOException {
-    long now = EnvironmentEdgeManager.currentTime();
-    Put put = new Put(regionInfo.getRegionName(), now);
-    addRegionInfo(put, regionInfo);
-    if (sn != null) {
-      addLocation(put, sn, openSeqNum, -1, regionInfo.getReplicaId());
-    }
-    putToMetaTable(connection, put);
-    LOG.info("Added daughter " + regionInfo.getEncodedName() +
-      (sn == null? ", serverName=null": ", serverName=" + sn.toString()));
+    LOG.info("Added {} regions to meta.", puts.size());
   }
 
   /**
@@ -1891,9 +1827,7 @@ public class MetaTableAccessor {
   public static void multiMutate(Connection connection, final Table table, byte[] row,
       final List<Mutation> mutations)
   throws IOException {
-    if (METALOG.isDebugEnabled()) {
-      METALOG.debug(mutationsToString(mutations));
-    }
+    debugLogMutations(mutations);
     // TODO: Need rollback!!!!
     // TODO: Need Retry!!!
     // TODO: What for a timeout? Default write timeout? GET FROM HTABLE?
@@ -2066,44 +2000,12 @@ public class MetaTableAccessor {
     }
     deleteFromMetaTable(connection, deletes);
     LOG.info("Deleted " + regionsInfo.size() + " regions from META");
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Deleted regions: " + regionsInfo);
-    }
+    LOG.debug("Deleted regions: {}", regionsInfo);
   }
 
   /**
-   * Adds and Removes the specified regions from hbase:meta
-   * @param connection connection we're using
-   * @param regionsToRemove list of regions to be deleted from META
-   * @param regionsToAdd list of regions to be added to META
-   * @throws IOException
-   */
-  public static void mutateRegions(Connection connection,
-                                   final List<RegionInfo> regionsToRemove,
-                                   final List<RegionInfo> regionsToAdd)
-    throws IOException {
-    List<Mutation> mutation = new ArrayList<>();
-    if (regionsToRemove != null) {
-      for (RegionInfo hri: regionsToRemove) {
-        mutation.add(makeDeleteFromRegionInfo(hri));
-      }
-    }
-    if (regionsToAdd != null) {
-      for (RegionInfo hri: regionsToAdd) {
-        mutation.add(makePutFromRegionInfo(hri));
-      }
-    }
-    mutateMetaTable(connection, mutation);
-    if (regionsToRemove != null && regionsToRemove.size() > 0) {
-      LOG.debug("Deleted " + RegionInfo.getShortNameToLog(regionsToRemove));
-    }
-    if (regionsToAdd != null && regionsToAdd.size() > 0) {
-      LOG.debug("Added " + RegionInfo.getShortNameToLog(regionsToAdd));
-    }
-  }
-
-  /**
-   * Overwrites the specified regions from hbase:meta
+   * Overwrites the specified regions from hbase:meta. Deletes old rows for the given regions and
+   * adds new ones. Regions added back have state CLOSED.
    * @param connection connection we're using
    * @param regionInfos list of regions to be added to META
    * @throws IOException
@@ -2121,9 +2023,7 @@ public class MetaTableAccessor {
     // HBASE-13875 uses master timestamp for the mutations. The 20ms sleep is not needed
     addRegionsToMeta(connection, regionInfos, regionReplication, now+1);
     LOG.info("Overwritten " + regionInfos.size() + " regions to Meta");
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Overwritten regions: " + regionInfos);
-    }
+    LOG.debug("Overwritten regions: {} ", regionInfos);
   }
 
   /**
@@ -2216,18 +2116,19 @@ public class MetaTableAccessor {
                 .build());
   }
 
-  private static String mutationsToString(List<? extends Mutation> mutations) throws IOException {
-    StringBuilder sb = new StringBuilder();
-    String prefix = "";
+  private static void debugLogMutations(List<? extends Mutation> mutations) throws IOException {
+    if (!METALOG.isDebugEnabled()) {
+      return;
+    }
+    // Logging each mutation in separate line makes it easier to see diff between them visually
+    // because of common starting indentation.
     for (Mutation mutation : mutations) {
-      sb.append(prefix).append(mutationToString(mutation));
-      prefix = ", ";
+      debugLogMutation(mutation);
     }
-    return sb.toString();
   }
 
-  private static String mutationToString(Mutation p) throws IOException {
-    return p.getClass().getSimpleName() + p.toJSON();
+  private static void debugLogMutation(Mutation p) throws IOException {
+    METALOG.debug("{} {}", p.getClass().getSimpleName(), p.toJSON());
   }
 
   public static Put addSequenceNum(final Put p, long openSeqNum, long time,
@@ -2328,57 +2229,45 @@ public class MetaTableAccessor {
     return map;
   }
 
+  private static String getSerialReplicationColumnValue(Connection connection,
+      byte[] encodedRegionName, byte[] columnQualifier) throws IOException {
+    Get get = new Get(encodedRegionName);
+    get.addColumn(HConstants.REPLICATION_META_FAMILY, columnQualifier);
+    Result result = get(getMetaHTable(connection), get);
+    if (!result.isEmpty()) {
+      Cell c = result.rawCells()[0];
+      return Bytes.toString(c.getValueArray(), c.getValueOffset(), c.getValueLength());
+    }
+    return null;
+  }
+
   /**
    * Get daughter region(s) for a region, only used in serial replication.
    * @param connection connection we're using
    * @param encodedName region's encoded name
-   * @throws IOException
    */
   public static String getSerialReplicationDaughterRegion(Connection connection, byte[] encodedName)
       throws IOException {
-    Get get = new Get(encodedName);
-    get.addColumn(HConstants.REPLICATION_META_FAMILY, daughterNameCq);
-    Result result = get(getMetaHTable(connection), get);
-    if (!result.isEmpty()) {
-      Cell c = result.rawCells()[0];
-      return Bytes.toString(c.getValueArray(), c.getValueOffset(), c.getValueLength());
-    }
-    return null;
+    return getSerialReplicationColumnValue(connection, encodedName, daughterNameCq);
   }
 
   /**
    * Get parent region(s) for a region, only used in serial replication.
    * @param connection connection we're using
    * @param encodedName region's encoded name
-   * @throws IOException
    */
   public static String getSerialReplicationParentRegion(Connection connection, byte[] encodedName)
       throws IOException {
-    Get get = new Get(encodedName);
-    get.addColumn(HConstants.REPLICATION_META_FAMILY, parentNameCq);
-    Result result = get(getMetaHTable(connection), get);
-    if (!result.isEmpty()) {
-      Cell c = result.rawCells()[0];
-      return Bytes.toString(c.getValueArray(), c.getValueOffset(), c.getValueLength());
-    }
-    return null;
+    return getSerialReplicationColumnValue(connection, encodedName, parentNameCq);
   }
 
   /**
    * Get the table name for a region, only used in serial replication.
    * @param connection connection we're using
    * @param encodedName region's encoded name
-   * @throws IOException
    */
   public static String getSerialReplicationTableName(Connection connection, byte[] encodedName)
       throws IOException {
-    Get get = new Get(encodedName);
-    get.addColumn(HConstants.REPLICATION_META_FAMILY, tableNameCq);
-    Result result = get(getMetaHTable(connection), get);
-    if (!result.isEmpty()) {
-      Cell c = result.rawCells()[0];
-      return Bytes.toString(c.getValueArray(), c.getValueOffset(), c.getValueLength());
-    }
-    return null;
+    return getSerialReplicationColumnValue(connection, encodedName, tableNameCq);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8ec0aa0d/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java
index a17108f..53533ea 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java
@@ -461,6 +461,8 @@ public class CloneSnapshotProcedure
   private void addRegionsToMeta(final MasterProcedureEnv env) throws IOException {
     newRegions = CreateTableProcedure.addTableToMeta(env, tableDescriptor, newRegions);
 
+    // TODO: parentsToChildrenPairMap is always empty, which makes updateMetaParentRegions()
+    // a no-op. This part seems unnecessary. Figure out. - Appy 12/21/17
     RestoreSnapshotHelper.RestoreMetaChanges metaChanges =
         new RestoreSnapshotHelper.RestoreMetaChanges(
                 tableDescriptor, parentsToChildrenPairMap);

http://git-wip-us.apache.org/repos/asf/hbase/blob/8ec0aa0d/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
index 99690de..009d7f8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
@@ -393,7 +393,7 @@ public class RestoreSnapshotHelper {
         }
 
         LOG.debug("Update splits parent " + regionInfo.getEncodedName() + " -> " + daughters);
-        MetaTableAccessor.addRegionToMeta(connection, regionInfo,
+        MetaTableAccessor.addSpiltsToParent(connection, regionInfo,
           regionsByName.get(daughters.getFirst()),
           regionsByName.get(daughters.getSecond()));
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8ec0aa0d/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 853e965..e5cd13a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -2293,7 +2293,7 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
           .setStartKey(startKeys[i])
           .setEndKey(startKeys[j])
           .build();
-      MetaTableAccessor.addRegionToMeta(meta, hri);
+      MetaTableAccessor.addRegionToMeta(getConnection(), hri);
       newRegions.add(hri);
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/8ec0aa0d/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
index fbc0096..611463d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
@@ -113,23 +113,6 @@ public class BaseTestHBaseFsck {
     Bytes.toBytes("00"), Bytes.toBytes("50"), Bytes.toBytes("A0"), Bytes.toBytes("A5"),
     Bytes.toBytes("B0"), Bytes.toBytes("B5"), Bytes.toBytes("C0"), Bytes.toBytes("C5") };
 
-
-  /**
-   * Create a new region in META.
-   */
-  protected RegionInfo createRegion(final HTableDescriptor
-      htd, byte[] startKey, byte[] endKey)
-      throws IOException {
-    Table meta = connection.getTable(TableName.META_TABLE_NAME, tableExecutorService);
-    RegionInfo hri = RegionInfoBuilder.newBuilder(htd.getTableName())
-        .setStartKey(startKey)
-        .setEndKey(endKey)
-        .build();
-    MetaTableAccessor.addRegionToMeta(meta, hri);
-    meta.close();
-    return hri;
-  }
-
   /**
    * Debugging method to dump the contents of meta.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/8ec0aa0d/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java
index 1085ce4..531d08c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/hbck/OfflineMetaRebuildTestCore.java
@@ -221,7 +221,7 @@ public class OfflineMetaRebuildTestCore {
     out.close();
 
     // add to meta.
-    MetaTableAccessor.addRegionToMeta(meta, hri);
+    MetaTableAccessor.addRegionToMeta(TEST_UTIL.getConnection(), hri);
     meta.close();
     return hri;
   }


[23/26] hbase git commit: HBASE-19580 Use slf4j instead of commons-logging in new, just-added Peer Procedure classes

Posted by zh...@apache.org.
HBASE-19580 Use slf4j instead of commons-logging in new, just-added Peer Procedure classes


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

Branch: refs/heads/HBASE-19397
Commit: 24c0ccc553d2d8669439f25121aa3740095845e8
Parents: bdc06c2
Author: zhangduo <zh...@apache.org>
Authored: Thu Dec 21 21:59:46 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Fri Dec 22 21:43:33 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/master/replication/AddPeerProcedure.java      | 6 +++---
 .../hadoop/hbase/master/replication/DisablePeerProcedure.java  | 6 +++---
 .../hadoop/hbase/master/replication/EnablePeerProcedure.java   | 6 +++---
 .../hadoop/hbase/master/replication/ModifyPeerProcedure.java   | 6 +++---
 .../hadoop/hbase/master/replication/RefreshPeerProcedure.java  | 6 +++---
 .../hadoop/hbase/master/replication/RemovePeerProcedure.java   | 6 +++---
 .../hbase/master/replication/UpdatePeerConfigProcedure.java    | 6 +++---
 7 files changed, 21 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/24c0ccc5/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
index c3862d8..066c3e7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
@@ -19,8 +19,6 @@ package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
@@ -28,6 +26,8 @@ import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AddPeerStateData;
 
@@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.A
 @InterfaceAudience.Private
 public class AddPeerProcedure extends ModifyPeerProcedure {
 
-  private static final Log LOG = LogFactory.getLog(AddPeerProcedure.class);
+  private static final Logger LOG = LoggerFactory.getLogger(AddPeerProcedure.class);
 
   private ReplicationPeerConfig peerConfig;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/24c0ccc5/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
index 0b32db9..9a28de6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
@@ -19,11 +19,11 @@ package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The procedure for disabling a replication peer.
@@ -31,7 +31,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 public class DisablePeerProcedure extends ModifyPeerProcedure {
 
-  private static final Log LOG = LogFactory.getLog(DisablePeerProcedure.class);
+  private static final Logger LOG = LoggerFactory.getLogger(DisablePeerProcedure.class);
 
   public DisablePeerProcedure() {
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/24c0ccc5/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
index 92ba000..4855901 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
@@ -19,11 +19,11 @@ package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The procedure for enabling a replication peer.
@@ -31,7 +31,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 public class EnablePeerProcedure extends ModifyPeerProcedure {
 
-  private static final Log LOG = LogFactory.getLog(EnablePeerProcedure.class);
+  private static final Logger LOG = LoggerFactory.getLogger(EnablePeerProcedure.class);
 
   public EnablePeerProcedure() {
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/24c0ccc5/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
index 23f6f87..c4552ed 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
@@ -19,8 +19,6 @@ package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface;
@@ -30,6 +28,8 @@ import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
 import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
 import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyPeerStateData;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationState;
@@ -42,7 +42,7 @@ public abstract class ModifyPeerProcedure
     extends StateMachineProcedure<MasterProcedureEnv, PeerModificationState>
     implements PeerProcedureInterface {
 
-  private static final Log LOG = LogFactory.getLog(ModifyPeerProcedure.class);
+  private static final Logger LOG = LoggerFactory.getLogger(ModifyPeerProcedure.class);
 
   protected String peerId;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/24c0ccc5/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
index ddc2401..ba4285f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
@@ -19,8 +19,6 @@ package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface;
@@ -35,6 +33,8 @@ import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProced
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
 import org.apache.hadoop.hbase.replication.regionserver.RefreshPeerCallable;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationType;
@@ -45,7 +45,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.R
 public class RefreshPeerProcedure extends Procedure<MasterProcedureEnv>
     implements PeerProcedureInterface, RemoteProcedure<MasterProcedureEnv, ServerName> {
 
-  private static final Log LOG = LogFactory.getLog(RefreshPeerProcedure.class);
+  private static final Logger LOG = LoggerFactory.getLogger(RefreshPeerProcedure.class);
 
   private String peerId;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/24c0ccc5/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
index 3daad6d..d40df02 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
@@ -19,11 +19,11 @@ package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The procedure for removing a replication peer.
@@ -31,7 +31,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 public class RemovePeerProcedure extends ModifyPeerProcedure {
 
-  private static final Log LOG = LogFactory.getLog(RemovePeerProcedure.class);
+  private static final Logger LOG = LoggerFactory.getLogger(RemovePeerProcedure.class);
 
   public RemovePeerProcedure() {
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/24c0ccc5/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
index 435eefc..d8154dc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
@@ -19,14 +19,14 @@ package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.UpdatePeerConfigStateData;
 
@@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.U
 @InterfaceAudience.Private
 public class UpdatePeerConfigProcedure extends ModifyPeerProcedure {
 
-  private static final Log LOG = LogFactory.getLog(UpdatePeerConfigProcedure.class);
+  private static final Logger LOG = LoggerFactory.getLogger(UpdatePeerConfigProcedure.class);
 
   private ReplicationPeerConfig peerConfig;
 


[03/26] hbase git commit: HBASE-19575 add copy constructor to Mutation

Posted by zh...@apache.org.
HBASE-19575 add copy constructor to Mutation


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

Branch: refs/heads/HBASE-19397
Commit: 59baf12c03303dab2a69352d5f35d38bd464d943
Parents: 6cefabe
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Fri Dec 22 03:13:03 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Fri Dec 22 03:24:43 2017 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Append.java  |  32 ++-
 .../org/apache/hadoop/hbase/client/Delete.java  |  32 ++-
 .../apache/hadoop/hbase/client/Increment.java   |  32 ++-
 .../apache/hadoop/hbase/client/Mutation.java    |  45 +++-
 .../hbase/client/OperationWithAttributes.java   |  21 +-
 .../org/apache/hadoop/hbase/client/Put.java     |  30 ++-
 .../hadoop/hbase/client/TestMutation.java       | 239 +++++++++++++++++++
 .../org/apache/hadoop/hbase/client/TestPut.java | 125 ----------
 .../hbase/regionserver/TestMetricsRegion.java   |   2 +-
 9 files changed, 387 insertions(+), 171 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/59baf12c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java
index 24e9512..0cb51a2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Append.java
@@ -112,17 +112,11 @@ public class Append extends Mutation {
   }
   /**
    * Copy constructor
-   * @param a
+   * @param appendToCopy append to copy
    */
-  public Append(Append a) {
-    this.row = a.getRow();
-    this.ts = a.getTimeStamp();
-    this.tr = a.getTimeRange();
-    this.familyMap.putAll(a.getFamilyCellMap());
-    for (Map.Entry<String, byte[]> entry : a.getAttributesMap().entrySet()) {
-      this.setAttribute(entry.getKey(), entry.getValue());
-    }
-    this.setPriority(a.getPriority());
+  public Append(Append appendToCopy) {
+    super(appendToCopy);
+    this.tr = appendToCopy.getTimeRange();
   }
 
   /** Create a Append operation for the specified row.
@@ -138,6 +132,18 @@ public class Append extends Mutation {
   }
 
   /**
+   * Construct the Append with user defined data. NOTED:
+   * 1) all cells in the familyMap must have the DataType.Put
+   * 2) the row of each cell must be same with passed row.
+   * @param row row. CAN'T be null
+   * @param ts timestamp
+   * @param familyMap the map to collect all cells internally. CAN'T be null
+   */
+  public Append(byte[] row, long ts, NavigableMap<byte [], List<Cell>> familyMap) {
+    super(row, ts, familyMap);
+  }
+
+  /**
    * Add the specified column and value to this Append operation.
    * @param family family name
    * @param qualifier column qualifier
@@ -202,6 +208,12 @@ public class Append extends Mutation {
     return (Append) super.setDurability(d);
   }
 
+  /**
+   * Method for setting the Append's familyMap
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
+   *             Use {@link Append#Append(byte[], long, NavigableMap)} instead
+   */
+  @Deprecated
   @Override
   public Append setFamilyCellMap(NavigableMap<byte[], List<Cell>> map) {
     return (Append) super.setFamilyCellMap(map);

http://git-wip-us.apache.org/repos/asf/hbase/blob/59baf12c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
index 2d296fb..57f5648 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Delete.java
@@ -135,17 +135,23 @@ public class Delete extends Mutation implements Comparable<Row> {
   }
 
   /**
-   * @param d Delete to clone.
+   * @param deleteToCopy delete to copy
    */
-  public Delete(final Delete d) {
-    this.row = d.getRow();
-    this.ts = d.getTimeStamp();
-    this.familyMap.putAll(d.getFamilyCellMap());
-    this.durability = d.durability;
-    for (Map.Entry<String, byte[]> entry : d.getAttributesMap().entrySet()) {
-      this.setAttribute(entry.getKey(), entry.getValue());
-    }
-    super.setPriority(d.getPriority());
+  public Delete(final Delete deleteToCopy) {
+    super(deleteToCopy);
+  }
+
+  /**
+   * Construct the Delete with user defined data. NOTED:
+   * 1) all cells in the familyMap must have the delete type.
+   * see {@link org.apache.hadoop.hbase.Cell.DataType}
+   * 2) the row of each cell must be same with passed row.
+   * @param row row. CAN'T be null
+   * @param ts timestamp
+   * @param familyMap the map to collect all cells internally. CAN'T be null
+   */
+  public Delete(byte[] row, long ts, NavigableMap<byte [], List<Cell>> familyMap) {
+    super(row, ts, familyMap);
   }
 
   /**
@@ -314,6 +320,12 @@ public class Delete extends Mutation implements Comparable<Row> {
     return (Delete) super.setDurability(d);
   }
 
+  /**
+   * Method for setting the Delete's familyMap
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
+   *             Use {@link Delete#Delete(byte[], long, NavigableMap)} instead
+   */
+  @Deprecated
   @Override
   public Delete setFamilyCellMap(NavigableMap<byte[], List<Cell>> map) {
     return (Delete) super.setFamilyCellMap(map);

http://git-wip-us.apache.org/repos/asf/hbase/blob/59baf12c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java
index 27cdafe..e9ae8fb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java
@@ -72,17 +72,23 @@ public class Increment extends Mutation implements Comparable<Row> {
   }
   /**
    * Copy constructor
-   * @param i
+   * @param incrementToCopy increment to copy
    */
-  public Increment(Increment i) {
-    this.row = i.getRow();
-    this.ts = i.getTimeStamp();
-    this.tr = i.getTimeRange();
-    this.familyMap.putAll(i.getFamilyCellMap());
-    for (Map.Entry<String, byte[]> entry : i.getAttributesMap().entrySet()) {
-      this.setAttribute(entry.getKey(), entry.getValue());
-    }
-    super.setPriority(i.getPriority());
+  public Increment(Increment incrementToCopy) {
+    super(incrementToCopy);
+    this.tr = incrementToCopy.getTimeRange();
+  }
+
+  /**
+   * Construct the Increment with user defined data. NOTED:
+   * 1) all cells in the familyMap must have the DataType.Put
+   * 2) the row of each cell must be same with passed row.
+   * @param row row. CAN'T be null
+   * @param ts timestamp
+   * @param familyMap the map to collect all cells internally. CAN'T be null
+   */
+  public Increment(byte[] row, long ts, NavigableMap<byte [], List<Cell>> familyMap) {
+    super(row, ts, familyMap);
   }
 
   /**
@@ -309,6 +315,12 @@ public class Increment extends Mutation implements Comparable<Row> {
     return (Increment) super.setDurability(d);
   }
 
+  /**
+   * Method for setting the Increment's familyMap
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
+   *             Use {@link Increment#Increment(byte[], long, NavigableMap)} instead
+   */
+  @Deprecated
   @Override
   public Increment setFamilyCellMap(NavigableMap<byte[], List<Cell>> map) {
     return (Increment) super.setFamilyCellMap(map);

http://git-wip-us.apache.org/repos/asf/hbase/blob/59baf12c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
index ba8f6cc..3983f35 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
@@ -28,6 +28,7 @@ import java.util.Map;
 import java.util.NavigableMap;
 import java.util.TreeMap;
 import java.util.UUID;
+import java.util.stream.Collectors;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScannable;
 import org.apache.hadoop.hbase.CellScanner;
@@ -49,6 +50,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.yetus.audience.InterfaceAudience;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.ArrayListMultimap;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.ListMultimap;
 import org.apache.hadoop.hbase.shaded.com.google.common.io.ByteArrayDataInput;
@@ -87,12 +89,48 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
 
   private static final String RETURN_RESULTS = "_rr_";
 
+  // TODO: row should be final
   protected byte [] row = null;
   protected long ts = HConstants.LATEST_TIMESTAMP;
   protected Durability durability = Durability.USE_DEFAULT;
 
+  // TODO: familyMap should be final
   // A Map sorted by column family.
-  protected NavigableMap<byte [], List<Cell>> familyMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+  protected NavigableMap<byte [], List<Cell>> familyMap;
+
+  /**
+   * empty construction.
+   * We need this empty construction to keep binary compatibility.
+   */
+  protected Mutation() {
+    this.familyMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+  }
+
+  protected Mutation(Mutation clone) {
+    super(clone);
+    this.row = clone.getRow();
+    this.ts = clone.getTimeStamp();
+    this.familyMap = clone.getFamilyCellMap().entrySet().stream()
+      .collect(Collectors.toMap(e -> e.getKey(), e -> new ArrayList<>(e.getValue()),
+        (k, v) -> {
+          throw new RuntimeException("collisions!!!");
+        }, () -> new TreeMap(Bytes.BYTES_COMPARATOR)));
+  }
+
+  /**
+   * Construct the mutation with user defined data.
+   * @param row row. CAN'T be null
+   * @param ts timestamp
+   * @param familyMap the map to collect all cells internally. CAN'T be null
+   */
+  protected Mutation(byte[] row, long ts, NavigableMap<byte [], List<Cell>> familyMap) {
+    this.row = Preconditions.checkNotNull(row);
+    if (row.length == 0) {
+      throw new IllegalArgumentException("Row can't be empty");
+    }
+    this.ts = ts;
+    this.familyMap = Preconditions.checkNotNull(familyMap);
+  }
 
   @Override
   public CellScanner cellScanner() {
@@ -260,8 +298,11 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
   }
 
   /**
-   * Method for setting the put's familyMap
+   * Method for setting the mutation's familyMap
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
+   *             Use {@link Mutation#Mutation(byte[], long, NavigableMap)} instead
    */
+  @Deprecated
   public Mutation setFamilyCellMap(NavigableMap<byte [], List<Cell>> map) {
     // TODO: Shut this down or move it up to be a Constructor.  Get new object rather than change
     // this internal data member.

http://git-wip-us.apache.org/repos/asf/hbase/blob/59baf12c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/OperationWithAttributes.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/OperationWithAttributes.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/OperationWithAttributes.java
index fc1f417..7342e65 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/OperationWithAttributes.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/OperationWithAttributes.java
@@ -22,11 +22,12 @@ package org.apache.hadoop.hbase.client;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
-
+import java.util.TreeMap;
+import java.util.stream.Collectors;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
+import org.apache.yetus.audience.InterfaceAudience;
 
 @InterfaceAudience.Public
 public abstract class OperationWithAttributes extends Operation implements Attributes {
@@ -37,6 +38,22 @@ public abstract class OperationWithAttributes extends Operation implements Attri
   public static final String ID_ATRIBUTE = "_operation.attributes.id";
   private int priority = HConstants.PRIORITY_UNSET;
 
+  /**
+   * empty construction.
+   * We need this empty construction to keep binary compatibility.
+   */
+  protected OperationWithAttributes() {
+  }
+
+  protected OperationWithAttributes(OperationWithAttributes clone) {
+    this.attributes = clone.getAttributesMap() == null ? null :
+      clone.getAttributesMap().entrySet().stream()
+        .collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue(), (k, v) -> {
+          throw new RuntimeException("collisions!!!");
+        }, () -> new TreeMap<>()));
+    this.priority = clone.getPriority();
+  }
+
   @Override
   public OperationWithAttributes setAttribute(String name, byte[] value) {
     if (attributes == null && value == null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/59baf12c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
index 73da4c8..1a1176f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
@@ -21,11 +21,9 @@ package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
-import java.util.TreeMap;
 import java.util.UUID;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
@@ -155,15 +153,19 @@ public class Put extends Mutation implements HeapSize, Comparable<Row> {
    * @param putToCopy put to copy
    */
   public Put(Put putToCopy) {
-    this(putToCopy.getRow(), putToCopy.ts);
-    this.familyMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-    for(Map.Entry<byte [], List<Cell>> entry: putToCopy.getFamilyCellMap().entrySet()) {
-      this.familyMap.put(entry.getKey(), new ArrayList<>(entry.getValue()));
-    }
-    this.durability = putToCopy.durability;
-    for (Map.Entry<String, byte[]> entry : putToCopy.getAttributesMap().entrySet()) {
-      this.setAttribute(entry.getKey(), entry.getValue());
-    }
+    super(putToCopy);
+  }
+
+  /**
+   * Construct the Put with user defined data. NOTED:
+   * 1) all cells in the familyMap must have the DataType.Put
+   * 2) the row of each cell must be same with passed row.
+   * @param row row. CAN'T be null
+   * @param ts timestamp
+   * @param familyMap the map to collect all cells internally. CAN'T be null
+   */
+  public Put(byte[] row, long ts, NavigableMap<byte [], List<Cell>> familyMap) {
+    super(row, ts, familyMap);
   }
 
   /**
@@ -321,6 +323,12 @@ public class Put extends Mutation implements HeapSize, Comparable<Row> {
     return (Put) super.setDurability(d);
   }
 
+  /**
+   * Method for setting the put's familyMap
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
+   *             Use {@link Put#Put(byte[], long, NavigableMap)} instead
+   */
+  @Deprecated
   @Override
   public Put setFamilyCellMap(NavigableMap<byte[], List<Cell>> map) {
     return (Put) super.setFamilyCellMap(map);

http://git-wip-us.apache.org/repos/asf/hbase/blob/59baf12c/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMutation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMutation.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMutation.java
new file mode 100644
index 0000000..111634a
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMutation.java
@@ -0,0 +1,239 @@
+/**
+ *
+ * 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.client;
+
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.Cell.DataType;
+import org.apache.hadoop.hbase.CellBuilderFactory;
+import org.apache.hadoop.hbase.CellBuilderType;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.io.TimeRange;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ SmallTests.class, ClientTests.class })
+public class TestMutation {
+
+  @Test
+  public void testAppendCopyConstructor() throws IOException {
+    Append origin = new Append(Bytes.toBytes("ROW-01"));
+    origin.setPriority(100);
+    byte[] family = Bytes.toBytes("CF-01");
+
+    origin.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
+      .setRow(origin.getRow())
+      .setFamily(family)
+      .setQualifier(Bytes.toBytes("q"))
+      .setType(DataType.Put)
+      .setValue(Bytes.toBytes(100))
+      .build());
+    origin.addColumn(family, Bytes.toBytes("q0"), Bytes.toBytes("value"));
+    origin.setTimeRange(100, 1000);
+    Append clone = new Append(origin);
+    assertEquals(origin, clone);
+    origin.addColumn(family, Bytes.toBytes("q1"), Bytes.toBytes("value"));
+
+    //They should have different cell lists
+    assertNotEquals(origin.getCellList(family), clone.getCellList(family));
+  }
+
+  @Test
+  public void testIncrementCopyConstructor() throws IOException {
+    Increment origin = new Increment(Bytes.toBytes("ROW-01"));
+    origin.setPriority(100);
+    byte[] family = Bytes.toBytes("CF-01");
+
+    origin.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
+      .setRow(origin.getRow())
+      .setFamily(family)
+      .setQualifier(Bytes.toBytes("q"))
+      .setType(DataType.Put)
+      .setValue(Bytes.toBytes(100))
+      .build());
+    origin.addColumn(family, Bytes.toBytes("q0"), 4);
+    origin.setTimeRange(100, 1000);
+    Increment clone = new Increment(origin);
+    assertEquals(origin, clone);
+    origin.addColumn(family, Bytes.toBytes("q1"), 3);
+
+    //They should have different cell lists
+    assertNotEquals(origin.getCellList(family), clone.getCellList(family));
+  }
+
+  @Test
+  public void testDeleteCopyConstructor() throws IOException {
+    Delete origin = new Delete(Bytes.toBytes("ROW-01"));
+    origin.setPriority(100);
+    byte[] family = Bytes.toBytes("CF-01");
+
+    origin.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
+      .setRow(origin.getRow())
+      .setFamily(family)
+      .setQualifier(Bytes.toBytes("q"))
+      .setType(DataType.Delete)
+      .build());
+    origin.addColumn(family, Bytes.toBytes("q0"));
+    origin.addColumns(family, Bytes.toBytes("q1"));
+    origin.addFamily(family);
+    origin.addColumns(family, Bytes.toBytes("q2"), 100);
+    origin.addFamilyVersion(family, 1000);
+    Delete clone = new Delete(origin);
+    assertEquals(origin, clone);
+    origin.addColumn(family, Bytes.toBytes("q3"));
+
+    //They should have different cell lists
+    assertNotEquals(origin.getCellList(family), clone.getCellList(family));
+  }
+
+  @Test
+  public void testPutCopyConstructor() throws IOException {
+    Put origin = new Put(Bytes.toBytes("ROW-01"));
+    origin.setPriority(100);
+    byte[] family = Bytes.toBytes("CF-01");
+
+    origin.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
+      .setRow(origin.getRow())
+      .setFamily(family)
+      .setQualifier(Bytes.toBytes("q"))
+      .setType(DataType.Put)
+      .setValue(Bytes.toBytes("value"))
+      .build());
+    origin.addColumn(family, Bytes.toBytes("q0"), Bytes.toBytes("V-01"));
+    origin.addColumn(family, Bytes.toBytes("q1"), 100, Bytes.toBytes("V-01"));
+    Put clone = new Put(origin);
+    assertEquals(origin, clone);
+    origin.addColumn(family, Bytes.toBytes("q2"), Bytes.toBytes("V-02"));
+
+    //They should have different cell lists
+    assertNotEquals(origin.getCellList(family), clone.getCellList(family));
+  }
+
+  private void assertEquals(Mutation origin, Mutation clone) {
+    Assert.assertEquals(origin.getFamilyCellMap().size(), clone.getFamilyCellMap().size());
+    for (byte[] family : origin.getFamilyCellMap().keySet()) {
+      List<Cell> originCells = origin.getCellList(family);
+      List<Cell> cloneCells = clone.getCellList(family);
+      Assert.assertEquals(originCells.size(), cloneCells.size());
+      for (int i = 0; i != cloneCells.size(); ++i) {
+        Cell originCell = originCells.get(i);
+        Cell cloneCell = cloneCells.get(i);
+        assertTrue(CellUtil.equals(originCell, cloneCell));
+        assertTrue(CellUtil.matchingValue(originCell, cloneCell));
+      }
+    }
+    Assert.assertEquals(origin.getAttributesMap().size(), clone.getAttributesMap().size());
+    for (String name : origin.getAttributesMap().keySet()) {
+      byte[] originValue = origin.getAttributesMap().get(name);
+      byte[] cloneValue = clone.getAttributesMap().get(name);
+      assertTrue(Bytes.equals(originValue, cloneValue));
+    }
+    Assert.assertEquals(origin.getTimeStamp(), clone.getTimeStamp());
+    Assert.assertEquals(origin.getPriority(), clone.getPriority());
+    if (origin instanceof Append) {
+      assertEquals(((Append)origin).getTimeRange(), ((Append)clone).getTimeRange());
+    }
+    if (origin instanceof Increment) {
+      assertEquals(((Increment)origin).getTimeRange(), ((Increment)clone).getTimeRange());
+    }
+  }
+
+  private static void assertEquals(TimeRange origin, TimeRange clone) {
+    Assert.assertEquals(origin.getMin(), clone.getMin());
+    Assert.assertEquals(origin.getMax(), clone.getMax());
+  }
+
+  // HBASE-14881
+  @Test
+  public void testRowIsImmutableOrNot() {
+    byte[] rowKey = Bytes.toBytes("immutable");
+
+    // Test when row key is immutable
+    Put putRowIsImmutable = new Put(rowKey, true);
+    assertTrue(rowKey == putRowIsImmutable.getRow());  // No local copy is made
+
+    // Test when row key is not immutable
+    Put putRowIsNotImmutable = new Put(rowKey, 1000L, false);
+    assertTrue(rowKey != putRowIsNotImmutable.getRow());  // A local copy is made
+  }
+
+  // HBASE-14882
+  @Test
+  public void testAddImmutableToPut() throws IOException {
+    byte[] row        = Bytes.toBytes("immutable-row");
+    byte[] family     = Bytes.toBytes("immutable-family");
+
+    byte[] qualifier0 = Bytes.toBytes("immutable-qualifier-0");
+    byte[] value0     = Bytes.toBytes("immutable-value-0");
+
+    byte[] qualifier1 = Bytes.toBytes("immutable-qualifier-1");
+    byte[] value1     = Bytes.toBytes("immutable-value-1");
+    long   ts1        = 5000L;
+
+    // "true" indicates that the input row is immutable
+    Put put = new Put(row, true);
+    put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
+            .setRow(row)
+            .setFamily(family)
+            .setQualifier(qualifier0)
+            .setTimestamp(put.getTimeStamp())
+            .setType(DataType.Put)
+            .setValue(value0)
+            .build())
+        .add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
+            .setRow(row)
+            .setFamily(family)
+            .setQualifier(qualifier1)
+            .setTimestamp(ts1)
+            .setType(DataType.Put)
+            .setValue(value1)
+            .build());
+
+    // Verify the cell of family:qualifier0
+    Cell cell0 = put.get(family, qualifier0).get(0);
+
+    // Verify no local copy is made for family, qualifier or value
+    assertTrue(cell0.getFamilyArray()    == family);
+    assertTrue(cell0.getQualifierArray() == qualifier0);
+    assertTrue(cell0.getValueArray()     == value0);
+
+    // Verify timestamp
+    assertTrue(cell0.getTimestamp()      == put.getTimeStamp());
+
+    // Verify the cell of family:qualifier1
+    Cell cell1 = put.get(family, qualifier1).get(0);
+
+    // Verify no local copy is made for family, qualifier or value
+    assertTrue(cell1.getFamilyArray()    == family);
+    assertTrue(cell1.getQualifierArray() == qualifier1);
+    assertTrue(cell1.getValueArray()     == value1);
+
+    // Verify timestamp
+    assertTrue(cell1.getTimestamp()      == ts1);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/59baf12c/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestPut.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestPut.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestPut.java
deleted file mode 100644
index 0ae2dfa..0000000
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestPut.java
+++ /dev/null
@@ -1,125 +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.client;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.Cell.DataType;
-import org.apache.hadoop.hbase.CellBuilderFactory;
-import org.apache.hadoop.hbase.CellBuilderType;
-import org.apache.hadoop.hbase.testclassification.ClientTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({ SmallTests.class, ClientTests.class })
-public class TestPut {
-  @Test
-  public void testCopyConstructor() {
-    Put origin = new Put(Bytes.toBytes("ROW-01"));
-    byte[] family = Bytes.toBytes("CF-01");
-    byte[] qualifier = Bytes.toBytes("Q-01");
-
-    origin.addColumn(family, qualifier, Bytes.toBytes("V-01"));
-    Put clone = new Put(origin);
-
-    assertEquals(origin.getCellList(family), clone.getCellList(family));
-    origin.addColumn(family, qualifier, Bytes.toBytes("V-02"));
-
-    //They should have different cell lists
-    assertNotEquals(origin.getCellList(family), clone.getCellList(family));
-
-  }
-
-  // HBASE-14881
-  @Test
-  public void testRowIsImmutableOrNot() {
-    byte[] rowKey = Bytes.toBytes("immutable");
-
-    // Test when row key is immutable
-    Put putRowIsImmutable = new Put(rowKey, true);
-    assertTrue(rowKey == putRowIsImmutable.getRow());  // No local copy is made
-
-    // Test when row key is not immutable
-    Put putRowIsNotImmutable = new Put(rowKey, 1000L, false);
-    assertTrue(rowKey != putRowIsNotImmutable.getRow());  // A local copy is made
-  }
-
-  // HBASE-14882
-  @Test
-  public void testAddImmutable() throws IOException {
-    byte[] row        = Bytes.toBytes("immutable-row");
-    byte[] family     = Bytes.toBytes("immutable-family");
-
-    byte[] qualifier0 = Bytes.toBytes("immutable-qualifier-0");
-    byte[] value0     = Bytes.toBytes("immutable-value-0");
-
-    byte[] qualifier1 = Bytes.toBytes("immutable-qualifier-1");
-    byte[] value1     = Bytes.toBytes("immutable-value-1");
-    long   ts1        = 5000L;
-
-    // "true" indicates that the input row is immutable
-    Put put = new Put(row, true);
-    put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
-            .setRow(row)
-            .setFamily(family)
-            .setQualifier(qualifier0)
-            .setTimestamp(put.getTimeStamp())
-            .setType(DataType.Put)
-            .setValue(value0)
-            .build())
-        .add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
-            .setRow(row)
-            .setFamily(family)
-            .setQualifier(qualifier1)
-            .setTimestamp(ts1)
-            .setType(DataType.Put)
-            .setValue(value1)
-            .build());
-
-    // Verify the cell of family:qualifier0
-    Cell cell0 = put.get(family, qualifier0).get(0);
-
-    // Verify no local copy is made for family, qualifier or value
-    assertTrue(cell0.getFamilyArray()    == family);
-    assertTrue(cell0.getQualifierArray() == qualifier0);
-    assertTrue(cell0.getValueArray()     == value0);
-
-    // Verify timestamp
-    assertTrue(cell0.getTimestamp()      == put.getTimeStamp());
-
-    // Verify the cell of family:qualifier1
-    Cell cell1 = put.get(family, qualifier1).get(0);
-
-    // Verify no local copy is made for family, qualifier or value
-    assertTrue(cell1.getFamilyArray()    == family);
-    assertTrue(cell1.getQualifierArray() == qualifier1);
-    assertTrue(cell1.getValueArray()     == value1);
-
-    // Verify timestamp
-    assertTrue(cell1.getTimestamp()      == ts1);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/59baf12c/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegion.java
index 6552808..e86f1e5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegion.java
@@ -83,7 +83,7 @@ public class TestMetricsRegion {
         "filteredReadRequestCount",
       107, agg);
     HELPER.assertCounter(
-      "namespace_TestNS_table_MetricsRegionWrapperStub_region_DEADBEEF001_metric_replicaid", 
+      "namespace_TestNS_table_MetricsRegionWrapperStub_region_DEADBEEF001_metric_replicaid",
       1, agg);
     HELPER.assertCounter(
       "namespace_TestNS_table_MetricsRegionWrapperStub_region_DEADBEEF001_metric_compactionsQueuedCount",


[13/26] hbase git commit: HBASE-19496 Reusing the ByteBuffer in rpc layer corrupt the ServerLoad and RegionLoad

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientClusterStatus.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientClusterStatus.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientClusterStatus.java
deleted file mode 100644
index 3e7d63a..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientClusterStatus.java
+++ /dev/null
@@ -1,242 +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.client;
-
-import java.io.IOException;
-import java.util.EnumSet;
-import java.util.List;
-import java.util.Optional;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.stream.Stream;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.ClusterStatus;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.MiniHBaseCluster;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.Waiter;
-import org.apache.hadoop.hbase.Waiter.Predicate;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
-import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
-import org.apache.hadoop.hbase.coprocessor.MasterObserver;
-import org.apache.hadoop.hbase.coprocessor.ObserverContext;
-import org.apache.hadoop.hbase.master.HMaster;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
-import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-/**
- * Test the ClusterStatus.
- */
-@Category(SmallTests.class)
-public class TestClientClusterStatus {
-  private static HBaseTestingUtility UTIL;
-  private static Admin ADMIN;
-  private final static int SLAVES = 5;
-  private final static int MASTERS = 3;
-  private static MiniHBaseCluster CLUSTER;
-  private static HRegionServer DEAD;
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    Configuration conf = HBaseConfiguration.create();
-    conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, MyObserver.class.getName());
-    UTIL = new HBaseTestingUtility(conf);
-    UTIL.startMiniCluster(MASTERS, SLAVES);
-    CLUSTER = UTIL.getHBaseCluster();
-    CLUSTER.waitForActiveAndReadyMaster();
-    ADMIN = UTIL.getAdmin();
-    // Kill one region server
-    List<RegionServerThread> rsts = CLUSTER.getLiveRegionServerThreads();
-    RegionServerThread rst = rsts.get(rsts.size() - 1);
-    DEAD = rst.getRegionServer();
-    DEAD.stop("Test dead servers status");
-    while (rst.isAlive()) {
-      Thread.sleep(500);
-    }
-  }
-
-  @Test
-  public void testDefaults() throws Exception {
-    ClusterStatus origin = ADMIN.getClusterStatus();
-    ClusterStatus defaults = ADMIN.getClusterStatus(EnumSet.allOf(Option.class));
-    Assert.assertEquals(origin.getHBaseVersion(), defaults.getHBaseVersion());
-    Assert.assertEquals(origin.getClusterId(), defaults.getClusterId());
-    Assert.assertTrue(origin.getAverageLoad() == defaults.getAverageLoad());
-    Assert.assertTrue(origin.getBackupMastersSize() == defaults.getBackupMastersSize());
-    Assert.assertTrue(origin.getDeadServersSize() == defaults.getDeadServersSize());
-    Assert.assertTrue(origin.getRegionsCount() == defaults.getRegionsCount());
-    Assert.assertTrue(origin.getServersSize() == defaults.getServersSize());
-    Assert.assertTrue(origin.getMasterInfoPort() == defaults.getMasterInfoPort());
-    Assert.assertTrue(origin.equals(defaults));
-  }
-
-  @Test
-  public void testNone() throws Exception {
-    ClusterStatus status0 = ADMIN.getClusterStatus(EnumSet.allOf(Option.class));
-    ClusterStatus status1 = ADMIN.getClusterStatus(EnumSet.noneOf(Option.class));
-    Assert.assertEquals(status0, status1);
-  }
-
-  @Test
-  public void testAsyncClient() throws Exception {
-    AsyncRegistry registry = AsyncRegistryFactory.getRegistry(UTIL.getConfiguration());
-    AsyncConnectionImpl asyncConnect = new AsyncConnectionImpl(UTIL.getConfiguration(), registry,
-      registry.getClusterId().get(), User.getCurrent());
-    AsyncAdmin asyncAdmin = asyncConnect.getAdmin();
-    CompletableFuture<ClusterStatus> originFuture =
-        asyncAdmin.getClusterStatus();
-    CompletableFuture<ClusterStatus> defaultsFuture =
-        asyncAdmin.getClusterStatus(EnumSet.allOf(Option.class));
-    ClusterStatus origin = originFuture.get();
-    ClusterStatus defaults = defaultsFuture.get();
-    Assert.assertEquals(origin.getHBaseVersion(), defaults.getHBaseVersion());
-    Assert.assertEquals(origin.getClusterId(), defaults.getClusterId());
-    Assert.assertTrue(origin.getAverageLoad() == defaults.getAverageLoad());
-    Assert.assertTrue(origin.getBackupMastersSize() == defaults.getBackupMastersSize());
-    Assert.assertTrue(origin.getDeadServersSize() == defaults.getDeadServersSize());
-    Assert.assertTrue(origin.getRegionsCount() == defaults.getRegionsCount());
-    Assert.assertTrue(origin.getServersSize() == defaults.getServersSize());
-    if (asyncConnect != null) {
-      asyncConnect.close();
-    }
-  }
-
-  @Test
-  public void testLiveAndDeadServersStatus() throws Exception {
-    // Count the number of live regionservers
-    List<RegionServerThread> regionserverThreads = CLUSTER.getLiveRegionServerThreads();
-    int numRs = 0;
-    int len = regionserverThreads.size();
-    for (int i = 0; i < len; i++) {
-      if (regionserverThreads.get(i).isAlive()) {
-        numRs++;
-      }
-    }
-    // Depending on the (random) order of unit execution we may run this unit before the
-    // minicluster is fully up and recovered from the RS shutdown done during test init.
-    Waiter.waitFor(CLUSTER.getConfiguration(), 10 * 1000, 100, new Predicate<Exception>() {
-      @Override
-      public boolean evaluate() throws Exception {
-        ClusterStatus status = ADMIN.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
-        Assert.assertNotNull(status);
-        return status.getRegionsCount() > 0;
-      }
-    });
-    // Retrieve live servers and dead servers info.
-    EnumSet<Option> options = EnumSet.of(Option.LIVE_SERVERS, Option.DEAD_SERVERS);
-    ClusterStatus status = ADMIN.getClusterStatus(options);
-    Assert.assertNotNull(status);
-    Assert.assertNotNull(status.getServers());
-    // exclude a dead region server
-    Assert.assertEquals(SLAVES -1, numRs);
-    // live servers = nums of regionservers
-    // By default, HMaster don't carry any regions so it won't report its load.
-    // Hence, it won't be in the server list.
-    Assert.assertEquals(status.getServers().size(), numRs);
-    Assert.assertTrue(status.getRegionsCount() > 0);
-    Assert.assertNotNull(status.getDeadServerNames());
-    Assert.assertEquals(1, status.getDeadServersSize());
-    ServerName deadServerName = status.getDeadServerNames().iterator().next();
-    Assert.assertEquals(DEAD.getServerName(), deadServerName);
-  }
-
-  @Test
-  public void testMasterAndBackupMastersStatus() throws Exception {
-    // get all the master threads
-    List<MasterThread> masterThreads = CLUSTER.getMasterThreads();
-    int numActive = 0;
-    int activeIndex = 0;
-    ServerName activeName = null;
-    HMaster active = null;
-    for (int i = 0; i < masterThreads.size(); i++) {
-      if (masterThreads.get(i).getMaster().isActiveMaster()) {
-        numActive++;
-        activeIndex = i;
-        active = masterThreads.get(activeIndex).getMaster();
-        activeName = active.getServerName();
-      }
-    }
-    Assert.assertNotNull(active);
-    Assert.assertEquals(1, numActive);
-    Assert.assertEquals(MASTERS, masterThreads.size());
-    // Retrieve master and backup masters infos only.
-    EnumSet<Option> options = EnumSet.of(Option.MASTER, Option.BACKUP_MASTERS);
-    ClusterStatus status = ADMIN.getClusterStatus(options);
-    Assert.assertTrue(status.getMaster().equals(activeName));
-    Assert.assertEquals(MASTERS - 1, status.getBackupMastersSize());
-  }
-
-  @Test
-  public void testOtherStatusInfos() throws Exception {
-    EnumSet<Option> options =
-        EnumSet.of(Option.MASTER_COPROCESSORS, Option.HBASE_VERSION,
-                   Option.CLUSTER_ID, Option.BALANCER_ON);
-    ClusterStatus status = ADMIN.getClusterStatus(options);
-    Assert.assertTrue(status.getMasterCoprocessors().length == 1);
-    Assert.assertNotNull(status.getHBaseVersion());
-    Assert.assertNotNull(status.getClusterId());
-    Assert.assertTrue(status.getAverageLoad() == 0.0);
-    Assert.assertNotNull(status.getBalancerOn());
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    if (ADMIN != null) ADMIN.close();
-    UTIL.shutdownMiniCluster();
-  }
-
-  @Test
-  public void testObserver() throws IOException {
-    int preCount = MyObserver.PRE_COUNT.get();
-    int postCount = MyObserver.POST_COUNT.get();
-    Assert.assertTrue(Stream.of(ADMIN.getClusterStatus().getMasterCoprocessors())
-        .anyMatch(s -> s.equals(MyObserver.class.getSimpleName())));
-    Assert.assertEquals(preCount + 1, MyObserver.PRE_COUNT.get());
-    Assert.assertEquals(postCount + 1, MyObserver.POST_COUNT.get());
-  }
-
-  public static class MyObserver implements MasterCoprocessor, MasterObserver {
-    private static final AtomicInteger PRE_COUNT = new AtomicInteger(0);
-    private static final AtomicInteger POST_COUNT = new AtomicInteger(0);
-
-    @Override public Optional<MasterObserver> getMasterObserver() {
-      return Optional.of(this);
-    }
-
-    @Override public void preGetClusterStatus(ObserverContext<MasterCoprocessorEnvironment> ctx)
-        throws IOException {
-      PRE_COUNT.incrementAndGet();
-    }
-
-    @Override public void postGetClusterStatus(ObserverContext<MasterCoprocessorEnvironment> ctx,
-        ClusterStatus status) throws IOException {
-      POST_COUNT.incrementAndGet();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
index c2a7f6b..e48667e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
@@ -50,7 +50,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
index 62a42ce..4497a529 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
@@ -33,7 +33,7 @@ import java.util.concurrent.ExecutorService;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.CategoryBasedTimeout;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestServerLoadDurability.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestServerLoadDurability.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestServerLoadDurability.java
new file mode 100644
index 0000000..ff0dcd6
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestServerLoadDurability.java
@@ -0,0 +1,117 @@
+/**
+ * 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.client;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.io.ByteBufferPool;
+import org.apache.hadoop.hbase.ipc.NettyRpcServer;
+import org.apache.hadoop.hbase.ipc.RpcServerFactory;
+import org.apache.hadoop.hbase.ipc.SimpleRpcServer;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+/**
+ * HBASE-19496 noticed that the RegionLoad/ServerLoad may be corrupted if rpc server
+ * reuses the bytebuffer backed, so this test call the Admin#getLastMajorCompactionTimestamp() to
+ * invoke HMaster to iterate all stored server/region loads.
+ */
+@RunWith(Parameterized.class)
+@Category({ MediumTests.class, ClientTests.class })
+public class TestServerLoadDurability {
+  private static final byte[] FAMILY = Bytes.toBytes("testFamily");
+
+  @Parameterized.Parameter
+  public Configuration conf;
+
+  @Parameterized.Parameters
+  public static final Collection<Object[]> parameters() {
+    List<Object[]> configurations = new ArrayList<>();
+    configurations.add(new Object[] { createConfigurationForSimpleRpcServer() });
+    configurations.add(new Object[] { createConfigurationForNettyRpcServer() });
+    return configurations;
+  }
+
+  private static Configuration createConfigurationForSimpleRpcServer() {
+    Configuration conf = HBaseConfiguration.create();
+    conf.set(RpcServerFactory.CUSTOM_RPC_SERVER_IMPL_CONF_KEY,
+        SimpleRpcServer.class.getName());
+    conf.setInt(ByteBufferPool.BUFFER_SIZE_KEY, 20);
+    return conf;
+  }
+
+  private static Configuration createConfigurationForNettyRpcServer() {
+    Configuration conf = HBaseConfiguration.create();
+    conf.set(RpcServerFactory.CUSTOM_RPC_SERVER_IMPL_CONF_KEY,
+        NettyRpcServer.class.getName());
+    return conf;
+  }
+
+  protected HBaseTestingUtility utility;
+  protected Connection conn;
+  protected Admin admin;
+
+  @Rule
+  public TestName testName = new TestName();
+  protected TableName tableName;
+
+  @Before
+  public void setUp() throws Exception {
+    utility = new HBaseTestingUtility(conf);
+    utility.startMiniCluster(2);
+    conn = ConnectionFactory.createConnection(utility.getConfiguration());
+    admin = conn.getAdmin();
+    String methodName = testName.getMethodName();
+    tableName = TableName.valueOf(methodName.substring(0, methodName.length() - 3));
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    utility.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testCompactionTimestamps() throws Exception {
+    createTableWithDefaultConf(tableName);
+    try (Table table = conn.getTable(tableName)) {
+      long ts = admin.getLastMajorCompactionTimestamp(tableName);
+    }
+  }
+
+  private void createTableWithDefaultConf(TableName tableName) throws IOException {
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
+    builder.addColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY));
+    admin.createTable(builder.build());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
index 34a1ccf..16f98a0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestAssignmentListener.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.ServerLoad;
+import org.apache.hadoop.hbase.ServerMetricsBuilder;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
@@ -341,8 +342,8 @@ public class TestAssignmentListener {
     // is started (just as we see when we failover to the Backup HMaster).
     // One of these will already be a draining server.
     HashMap<ServerName, ServerLoad> onlineServers = new HashMap<>();
-    onlineServers.put(SERVERNAME_A, ServerLoad.EMPTY_SERVERLOAD);
-    onlineServers.put(SERVERNAME_C, ServerLoad.EMPTY_SERVERLOAD);
+    onlineServers.put(SERVERNAME_A, new ServerLoad(ServerMetricsBuilder.of(SERVERNAME_A)));
+    onlineServers.put(SERVERNAME_C, new ServerLoad(ServerMetricsBuilder.of(SERVERNAME_C)));
 
     // Create draining znodes for the draining servers, which would have been
     // performed when the previous HMaster was running.

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
index 7a2817e..6119e43 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.MetaMockingUtil;
 import org.apache.hadoop.hbase.ServerLoad;
+import org.apache.hadoop.hbase.ServerMetricsBuilder;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
@@ -230,7 +231,7 @@ public class TestMasterNoCluster {
         RegionServerReportRequest.Builder request = RegionServerReportRequest.newBuilder();;
         ServerName sn = ServerName.parseVersionedServerName(sns[i].getVersionedBytes());
         request.setServer(ProtobufUtil.toServerName(sn));
-        request.setLoad(ServerLoad.EMPTY_SERVERLOAD.obtainServerLoadPB());
+        request.setLoad(ServerMetricsBuilder.toServerLoad(ServerMetricsBuilder.of(sn)));
         master.getMasterRpcServices().regionServerReport(null, request.build());
       }
        // Master should now come up.
@@ -272,7 +273,8 @@ public class TestMasterNoCluster {
           KeeperException, CoordinatedStateException {
         super.initializeZKBasedSystemTrackers();
         // Record a newer server in server manager at first
-        getServerManager().recordNewServerWithLock(newServer, ServerLoad.EMPTY_SERVERLOAD);
+        getServerManager().recordNewServerWithLock(newServer,
+          new ServerLoad(ServerMetricsBuilder.of(newServer)));
 
         List<ServerName> onlineServers = new ArrayList<>();
         onlineServers.add(deadServer);

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
index 43c258e..13d9841 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java
@@ -31,9 +31,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
-
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -52,7 +51,6 @@ import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
index 764aa4a..5a93300 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/MockMasterServices.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.ServerLoad;
+import org.apache.hadoop.hbase.ServerMetricsBuilder;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
@@ -168,8 +169,8 @@ public class MockMasterServices extends MockNoopMasterServices {
     startProcedureExecutor(remoteDispatcher);
     this.assignmentManager.start();
     for (int i = 0; i < numServes; ++i) {
-      serverManager.regionServerReport(
-        ServerName.valueOf("localhost", 100 + i, 1), ServerLoad.EMPTY_SERVERLOAD);
+      ServerName sn = ServerName.valueOf("localhost", 100 + i, 1);
+      serverManager.regionServerReport(sn, new ServerLoad(ServerMetricsBuilder.of(sn)));
     }
     this.procedureExecutor.getEnvironment().setEventReady(initialized, true);
   }
@@ -195,7 +196,7 @@ public class MockMasterServices extends MockNoopMasterServices {
       return;
     }
     ServerName sn = ServerName.valueOf(serverName.getAddress().toString(), startCode);
-    serverManager.regionServerReport(sn, ServerLoad.EMPTY_SERVERLOAD);
+    serverManager.regionServerReport(sn, new ServerLoad(ServerMetricsBuilder.of(sn)));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java
index fb0dea7..7865c2d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticBalancerPickers.java
@@ -27,10 +27,9 @@ import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
-
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.ClusterStatus;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
@@ -55,15 +54,15 @@ import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java
index 6541f15..ea0ca18 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java
@@ -30,9 +30,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
-
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -63,6 +62,7 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReadRequestMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReadRequestMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReadRequestMetrics.java
index 4ff9655..e7b35d9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReadRequestMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReadRequestMetrics.java
@@ -18,8 +18,19 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.RegionLoad;
 import org.apache.hadoop.hbase.ServerLoad;
@@ -57,18 +68,6 @@ import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.util.Collection;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
 @Category(MediumTests.class)
 public class TestRegionServerReadRequestMetrics {
   private static final Logger LOG =

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
index 3a5ca0e..df929c8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java
@@ -30,13 +30,12 @@ import java.util.List;
 import java.util.Optional;
 import java.util.Random;
 import java.util.concurrent.CountDownLatch;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.ClusterStatus;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
@@ -189,7 +188,7 @@ public class TestMasterReplication {
       @Override
       public boolean evaluate() throws Exception {
         ClusterStatus clusterStatus = utilities[0].getAdmin()
-            .getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
+            .getClusterStatus(EnumSet.of(ClusterMetrics.Option.LIVE_SERVERS));
         ServerLoad serverLoad = clusterStatus.getLoad(rsName);
         List<ReplicationLoadSource> replicationLoadSourceList =
             serverLoad.getReplicationLoadSourceList();

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatus.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatus.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatus.java
index 8532dff..f76be44 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatus.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStatus.java
@@ -22,11 +22,10 @@ import static org.junit.Assert.assertTrue;
 
 import java.util.EnumSet;
 import java.util.List;
-
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.testclassification.MediumTests;

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
index 611463d..9e4de7e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
@@ -33,13 +33,12 @@ import java.util.UUID;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.ClusterStatus;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -77,6 +76,7 @@ import org.apache.zookeeper.KeeperException;
 import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/448ba3a7/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java
index adc7567..0b9c4b5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestMiniClusterLoadSequential.java
@@ -23,15 +23,14 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.EnumSet;
 import java.util.List;
-
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;


[06/26] hbase git commit: HBASE-19570 Add hadoop3 tests to Nightly master/branch-2 runs.

Posted by zh...@apache.org.
HBASE-19570 Add hadoop3 tests to Nightly master/branch-2 runs.

Jenkins fails the whole build immediately if any stage fails. Hadoop2 tests run before Hadoop3 tests.
So Hadoop3 tests will run only if hadoop2 tests pass.


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

Branch: refs/heads/HBASE-19397
Commit: a4272a9f08f1005d1d3b2511dfbe8eca577f0d49
Parents: bcaf2fd
Author: Apekshit Sharma <ap...@apache.org>
Authored: Thu Dec 21 11:20:40 2017 -0800
Committer: Apekshit Sharma <ap...@apache.org>
Committed: Thu Dec 21 17:44:04 2017 -0800

----------------------------------------------------------------------
 dev-support/Jenkinsfile            | 73 ++++++++++++++++++++++++++++++---
 dev-support/hbase-personality.sh   |  8 ++++
 dev-support/hbase_nightly_yetus.sh |  5 +++
 3 files changed, 80 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a4272a9f/dev-support/Jenkinsfile
----------------------------------------------------------------------
diff --git a/dev-support/Jenkinsfile b/dev-support/Jenkinsfile
index 744e9b0..dcef649 100644
--- a/dev-support/Jenkinsfile
+++ b/dev-support/Jenkinsfile
@@ -35,7 +35,7 @@ pipeline {
     BASEDIR = "${env.WORKSPACE}/component"
     YETUS_RELEASE = '0.6.0'
     PROJECT = 'hbase'
-    PROJET_PERSONALITY = 'https://raw.githubusercontent.com/apache/hbase/master/dev-support/hbase-personality.sh'
+    PROJECT_PERSONALITY = 'https://raw.githubusercontent.com/apache/hbase/master/dev-support/hbase-personality.sh'
     // This section of the docs tells folks not to use the javadoc tag. older branches have our old version of the check for said tag.
     AUTHOR_IGNORE_LIST = 'src/main/asciidoc/_chapters/developer.adoc,dev-support/test-patch.sh'
     WHITESPACE_IGNORE_LIST = '.*/generated/.*'
@@ -102,7 +102,7 @@ fi
         dir ("${env.TOOLS}") {
           sh """#!/usr/bin/env bash
 echo "Downloading Project personality."
-curl -L  -o personality.sh "${env.PROJET_PERSONALITY}"
+curl -L  -o personality.sh "${env.PROJECT_PERSONALITY}"
           """
         }
         stash name: 'yetus', includes: "yetus-*/*,yetus-*/**/*,tools/personality.sh"
@@ -203,15 +203,15 @@ curl -L  -o personality.sh "${env.PROJET_PERSONALITY}"
         }
       }
     }
-    stage ('yetus jdk8 checks') {
+    stage ('yetus jdk8 hadoop2 checks') {
       when {
         not {
           branch 'branch-1.1*'
         }
       }
       environment {
-        TESTS = 'mvninstall,compile,javac,unit,findbugs,htmlout'
-        OUTPUT_DIR_RELATIVE = "output-jdk8"
+        TESTS = 'mvninstall,compile,javac,unit,htmlout'
+        OUTPUT_DIR_RELATIVE = "output-jdk8-hadoop2"
         OUTPUT_DIR = "${env.WORKSPACE}/${env.OUTPUT_DIR_RELATIVE}"
         // This isn't strictly needed on branches that only support jdk8, but doesn't hurt
         // and is needed on branches that do both jdk7 and jdk8
@@ -253,7 +253,68 @@ curl -L  -o personality.sh "${env.PROJET_PERSONALITY}"
             // Has to be relative to WORKSPACE.
             reportDir            : "${env.OUTPUT_DIR_RELATIVE}",
             reportFiles          : 'console-report.html',
-            reportName           : 'JDK8 Nightly Build Report'
+            reportName           : 'JDK8 Nightly Build Report (Hadoop2)'
+          ]
+        }
+      }
+    }
+    stage ('yetus jdk8 hadoop3 checks') {
+      when {
+        not {
+          branch 'branch-1*'
+        }
+      }
+      environment {
+        // Failure in any stage fails the build and consecutive stages are not built.
+        // Findbugs is part of this last yetus stage to prevent findbugs precluding hadoop3
+        // tests.
+        TESTS = 'mvninstall,compile,javac,unit,findbugs,htmlout'
+        OUTPUT_DIR_RELATIVE = "output-jdk8-hadoop3"
+        OUTPUT_DIR = "${env.WORKSPACE}/${env.OUTPUT_DIR_RELATIVE}"
+        // This isn't strictly needed on branches that only support jdk8, but doesn't hurt
+        // and is needed on branches that do both jdk7 and jdk8
+        SET_JAVA_HOME = '/usr/lib/jvm/java-8-openjdk-amd64'
+        // Activates hadoop 3.0 profile in maven runs.
+        HADOOP_PROFILE = '3.0'
+      }
+      steps {
+        unstash 'yetus'
+        sh '''#!/usr/bin/env bash
+          rm -rf "${OUTPUT_DIR}" && mkdir "${OUTPUT_DIR}"
+          rm -rf "${OUTPUT_DIR}/machine" && mkdir "${OUTPUT_DIR}/machine"
+          "${BASEDIR}/dev-support/gather_machine_environment.sh" "${OUTPUT_DIR_RELATIVE}/machine"
+'''
+        sh "${env.BASEDIR}/dev-support/hbase_nightly_yetus.sh"
+      }
+      post {
+        always {
+          // Not sure how two junit test reports will work. Disabling this for now.
+          // junit testResults: "${env.OUTPUT_DIR_RELATIVE}/**/target/**/TEST-*.xml", allowEmptyResults: true
+          // zip surefire reports.
+          sh '''#!/bin/bash -e
+            if [ -d "${OUTPUT_DIR}/archiver" ]; then
+              count=$(find "${OUTPUT_DIR}/archiver" -type f | wc -l)
+              if [[ 0 -ne ${count} ]]; then
+                echo "zipping ${count} archived files"
+                zip -q -m -r "${OUTPUT_DIR}/test_logs.zip" "${OUTPUT_DIR}/archiver"
+              else
+                echo "No archived files, skipping compressing."
+              fi
+            else
+              echo "No archiver directory, skipping compressing."
+            fi
+'''
+          // Has to be relative to WORKSPACE.
+          archive "${env.OUTPUT_DIR_RELATIVE}/*"
+          archive "${env.OUTPUT_DIR_RELATIVE}/**/*"
+          publishHTML target: [
+            allowMissing         : true,
+            keepAll              : true,
+            alwaysLinkToLastBuild: true,
+            // Has to be relative to WORKSPACE.
+            reportDir            : "${env.OUTPUT_DIR_RELATIVE}",
+            reportFiles          : 'console-report.html',
+            reportName           : 'JDK8 Nightly Build Report (Hadoop3)'
           ]
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a4272a9f/dev-support/hbase-personality.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase-personality.sh b/dev-support/hbase-personality.sh
index cf5eacd..e24930d 100755
--- a/dev-support/hbase-personality.sh
+++ b/dev-support/hbase-personality.sh
@@ -81,6 +81,9 @@ function personality_parse_args
       --include-tests-url=*)
         INCLUDE_TESTS_URL=${i#*=}
       ;;
+      --hadoop-profile=*)
+        HADOOP_PROFILE=${i#*=}
+      ;;
     esac
   done
 }
@@ -103,6 +106,10 @@ function personality_modules
 
   extra="-DHBasePatchProcess"
 
+  if [[ -n "${HADOOP_PROFILE}" ]]; then
+    extra="${extra} -Dhadoop.profile=${HADOOP_PROFILE}"
+  fi
+
   # BUILDMODE value is 'full' when there is no patch to be tested, and we are running checks on
   # full source code instead. In this case, do full compiles, tests, etc instead of per
   # module.
@@ -115,6 +122,7 @@ function personality_modules
   fi
 
   if [[ ${testtype} == mvninstall ]]; then
+    # shellcheck disable=SC2086
     personality_enqueue_module . ${extra}
     return
   fi

http://git-wip-us.apache.org/repos/asf/hbase/blob/a4272a9f/dev-support/hbase_nightly_yetus.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase_nightly_yetus.sh b/dev-support/hbase_nightly_yetus.sh
index d37292b..e1175d2 100755
--- a/dev-support/hbase_nightly_yetus.sh
+++ b/dev-support/hbase_nightly_yetus.sh
@@ -76,6 +76,11 @@ if [[ -n "${INCLUDE_TESTS_URL}" && "${BRANCH_NAME}" == "master" ]]; then
   YETUS_ARGS=("--include-tests-url=${INCLUDE_TESTS_URL}" "${YETUS_ARGS[@]}")
 fi
 
+# For testing with specific hadoop version. Activates corresponding profile in maven runs.
+if [[ -n "${HADOOP_PROFILE}" ]]; then
+  YETUS_ARGS=("--hadoop-profile=${HADOOP_PROFILE}" "${YETUS_ARGS[@]}")
+fi
+
 if [[ true == "${DEBUG}" ]]; then
   YETUS_ARGS=("--debug" "${YETUS_ARGS[@]}")
 fi


[24/26] hbase git commit: HBASE-19525 RS side changes for moving peer modification from zk watcher to procedure

Posted by zh...@apache.org.
HBASE-19525 RS side changes for moving peer modification from zk watcher to procedure


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

Branch: refs/heads/HBASE-19397
Commit: 72e30d87b33198bb0d0de3f2af8e13b6e38445ed
Parents: 24c0ccc
Author: huzheng <op...@gmail.com>
Authored: Wed Dec 20 10:47:18 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Fri Dec 22 21:43:44 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/protobuf/ProtobufUtil.java     |  11 +-
 .../hbase/shaded/protobuf/ProtobufUtil.java     |  13 +-
 .../hbase/replication/ReplicationListener.java  |  14 --
 .../hbase/replication/ReplicationPeer.java      |  28 ++-
 .../replication/ReplicationPeerZKImpl.java      | 180 ++++-----------
 .../replication/ReplicationPeersZKImpl.java     |  19 +-
 .../replication/ReplicationTrackerZKImpl.java   |  73 +-----
 .../regionserver/ReplicationSourceService.java  |   9 +-
 .../handler/RSProcedureHandler.java             |   3 +
 .../replication/BaseReplicationEndpoint.java    |   2 +-
 .../regionserver/PeerProcedureHandler.java      |  38 ++++
 .../regionserver/PeerProcedureHandlerImpl.java  |  81 +++++++
 .../regionserver/RefreshPeerCallable.java       |  39 +++-
 .../replication/regionserver/Replication.java   |  10 +
 .../regionserver/ReplicationSource.java         |   9 +-
 .../regionserver/ReplicationSourceManager.java  |  37 ++-
 .../replication/TestReplicationAdmin.java       |   2 +-
 .../TestReplicationAdminUsingProcedure.java     | 226 +++++++++++++++++++
 .../replication/DummyModifyPeerProcedure.java   |  48 ----
 .../TestDummyModifyPeerProcedure.java           |  80 -------
 .../TestReplicationTrackerZKImpl.java           |  51 -----
 .../TestReplicationSourceManager.java           |  32 ++-
 ...tTableBasedReplicationSourceManagerImpl.java |   7 +-
 23 files changed, 536 insertions(+), 476 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/72e30d87/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index 267dc7a..d5285dc 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hbase.protobuf;
 
+import static org.apache.hadoop.hbase.protobuf.ProtobufMagic.PB_MAGIC;
+
 import com.google.protobuf.ByteString;
 import com.google.protobuf.CodedInputStream;
 import com.google.protobuf.InvalidProtocolBufferException;
@@ -199,7 +201,7 @@ public final class ProtobufUtil {
    * byte array that is <code>bytes.length</code> plus {@link ProtobufMagic#PB_MAGIC}.length.
    */
   public static byte [] prependPBMagic(final byte [] bytes) {
-    return Bytes.add(ProtobufMagic.PB_MAGIC, bytes);
+    return Bytes.add(PB_MAGIC, bytes);
   }
 
   /**
@@ -224,10 +226,11 @@ public final class ProtobufUtil {
    * @param bytes bytes to check
    * @throws DeserializationException if we are missing the pb magic prefix
    */
-  public static void expectPBMagicPrefix(final byte [] bytes) throws DeserializationException {
+  public static void expectPBMagicPrefix(final byte[] bytes) throws DeserializationException {
     if (!isPBMagicPrefix(bytes)) {
-      throw new DeserializationException("Missing pb magic " +
-          Bytes.toString(ProtobufMagic.PB_MAGIC) + " prefix");
+      String bytesPrefix = bytes == null ? "null" : Bytes.toStringBinary(bytes, 0, PB_MAGIC.length);
+      throw new DeserializationException(
+          "Missing pb magic " + Bytes.toString(PB_MAGIC) + " prefix, bytes: " + bytesPrefix);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/72e30d87/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index 17b1141..8954d04 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hbase.shaded.protobuf;
 
+import static org.apache.hadoop.hbase.protobuf.ProtobufMagic.PB_MAGIC;
+
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
@@ -282,7 +284,7 @@ public final class ProtobufUtil {
    * byte array that is <code>bytes.length</code> plus {@link ProtobufMagic#PB_MAGIC}.length.
    */
   public static byte [] prependPBMagic(final byte [] bytes) {
-    return Bytes.add(ProtobufMagic.PB_MAGIC, bytes);
+    return Bytes.add(PB_MAGIC, bytes);
   }
 
   /**
@@ -307,10 +309,11 @@ public final class ProtobufUtil {
    * @param bytes bytes to check
    * @throws DeserializationException if we are missing the pb magic prefix
    */
-  public static void expectPBMagicPrefix(final byte [] bytes) throws DeserializationException {
+  public static void expectPBMagicPrefix(final byte[] bytes) throws DeserializationException {
     if (!isPBMagicPrefix(bytes)) {
-      throw new DeserializationException("Missing pb magic " +
-          Bytes.toString(ProtobufMagic.PB_MAGIC) + " prefix");
+      String bytesPrefix = bytes == null ? "null" : Bytes.toStringBinary(bytes, 0, PB_MAGIC.length);
+      throw new DeserializationException(
+          "Missing pb magic " + Bytes.toString(PB_MAGIC) + " prefix" + ", bytes: " + bytesPrefix);
     }
   }
 
@@ -1962,7 +1965,7 @@ public final class ProtobufUtil {
   public static byte [] toDelimitedByteArray(final Message m) throws IOException {
     // Allocate arbitrary big size so we avoid resizing.
     ByteArrayOutputStream baos = new ByteArrayOutputStream(4096);
-    baos.write(ProtobufMagic.PB_MAGIC);
+    baos.write(PB_MAGIC);
     m.writeDelimitedTo(baos);
     return baos.toByteArray();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/72e30d87/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationListener.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationListener.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationListener.java
index 3edfcf9..f040bf9 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationListener.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationListener.java
@@ -18,8 +18,6 @@
  */
 package org.apache.hadoop.hbase.replication;
 
-import java.util.List;
-
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -36,16 +34,4 @@ public interface ReplicationListener {
    * @param regionServer the removed region server
    */
   public void regionServerRemoved(String regionServer);
-
-  /**
-   * A peer cluster has been removed (i.e. unregistered) from replication.
-   * @param peerId The peer id of the cluster that has been removed
-   */
-  public void peerRemoved(String peerId);
-
-  /**
-   * The list of registered peer clusters has changed.
-   * @param peerIds A list of all currently registered peer clusters
-   */
-  public void peerListChanged(List<String> peerIds);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/72e30d87/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
index 97e2ddb..b66d76d 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
@@ -55,12 +55,27 @@ public interface ReplicationPeer {
   public ReplicationPeerConfig getPeerConfig();
 
   /**
-   * Returns the state of the peer
+   * Get the peer config object. if loadFromBackingStore is true, it will load from backing store
+   * directly and update its load peer config. otherwise, just return the local cached peer config.
+   * @return the ReplicationPeerConfig for this peer
+   */
+  public ReplicationPeerConfig getPeerConfig(boolean loadFromBackingStore)
+      throws ReplicationException;
+
+  /**
+   * Returns the state of the peer by reading local cache.
    * @return the enabled state
    */
   PeerState getPeerState();
 
   /**
+   * Returns the state of peer, if loadFromBackingStore is true, it will load from backing store
+   * directly and update its local peer state. otherwise, just return the local cached peer state.
+   * @return the enabled state
+   */
+  PeerState getPeerState(boolean loadFromBackingStore) throws ReplicationException;
+
+  /**
    * Get the configuration object required to communicate with this peer
    * @return configuration object
    */
@@ -84,6 +99,15 @@ public interface ReplicationPeer {
    */
   public long getPeerBandwidth();
 
-  void trackPeerConfigChanges(ReplicationPeerConfigListener listener);
+  /**
+   * Register a peer config listener to catch the peer config change event.
+   * @param listener listener to catch the peer config change event.
+   */
+  public void registerPeerConfigListener(ReplicationPeerConfigListener listener);
 
+  /**
+   * Notify all the registered ReplicationPeerConfigListener to update their peer config.
+   * @param newPeerConfig the new peer config.
+   */
+  public void triggerPeerConfigChange(ReplicationPeerConfig newPeerConfig);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/72e30d87/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
index 454d09c..5d051a0 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.replication;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -33,12 +34,10 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.log.HBaseMarkers;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
-import org.apache.hadoop.hbase.zookeeper.ZKNodeTracker;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.KeeperException.NodeExistsException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -47,14 +46,13 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase
     implements ReplicationPeer, Abortable, Closeable {
   private static final Logger LOG = LoggerFactory.getLogger(ReplicationPeerZKImpl.class);
 
-  private ReplicationPeerConfig peerConfig;
+  private volatile ReplicationPeerConfig peerConfig;
   private final String id;
   private volatile PeerState peerState;
   private volatile Map<TableName, List<String>> tableCFs = new HashMap<>();
   private final Configuration conf;
-  private PeerStateTracker peerStateTracker;
-  private PeerConfigTracker peerConfigTracker;
 
+  private final List<ReplicationPeerConfigListener> peerConfigListeners;
 
   /**
    * Constructor that takes all the objects required to communicate with the specified peer, except
@@ -63,62 +61,35 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase
    * @param id string representation of this peer's identifier
    * @param peerConfig configuration for the replication peer
    */
-  public ReplicationPeerZKImpl(ZKWatcher zkWatcher, Configuration conf,
-                               String id, ReplicationPeerConfig peerConfig,
-                               Abortable abortable)
-      throws ReplicationException {
+  public ReplicationPeerZKImpl(ZKWatcher zkWatcher, Configuration conf, String id,
+      ReplicationPeerConfig peerConfig, Abortable abortable) throws ReplicationException {
     super(zkWatcher, conf, abortable);
     this.conf = conf;
     this.peerConfig = peerConfig;
     this.id = id;
+    this.peerConfigListeners = new ArrayList<>();
   }
 
-  /**
-   * start a state tracker to check whether this peer is enabled or not
-   *
-   * @param peerStateNode path to zk node which stores peer state
-   * @throws KeeperException
-   */
-  public void startStateTracker(String peerStateNode)
-      throws KeeperException {
-    ensurePeerEnabled(peerStateNode);
-    this.peerStateTracker = new PeerStateTracker(peerStateNode, zookeeper, this);
-    this.peerStateTracker.start();
+  private PeerState readPeerState() throws ReplicationException {
     try {
-      this.readPeerStateZnode();
-    } catch (DeserializationException e) {
-      throw ZKUtil.convert(e);
+      byte[] data = ZKUtil.getData(zookeeper, this.getPeerStateNode(id));
+      this.peerState = isStateEnabled(data) ? PeerState.ENABLED : PeerState.DISABLED;
+    } catch (DeserializationException | KeeperException | InterruptedException e) {
+      throw new ReplicationException("Get and deserialize peer state data from zookeeper failed: ",
+          e);
     }
+    return this.peerState;
   }
 
-  private void readPeerStateZnode() throws DeserializationException {
-    this.peerState =
-        isStateEnabled(this.peerStateTracker.getData(false))
-          ? PeerState.ENABLED
-          : PeerState.DISABLED;
-  }
-
-  /**
-   * start a table-cfs tracker to listen the (table, cf-list) map change
-   * @param peerConfigNode path to zk node which stores table-cfs
-   * @throws KeeperException
-   */
-  public void startPeerConfigTracker(String peerConfigNode)
-    throws KeeperException {
-    this.peerConfigTracker = new PeerConfigTracker(peerConfigNode, zookeeper,
-        this);
-    this.peerConfigTracker.start();
-    this.readPeerConfig();
-  }
-
-  private ReplicationPeerConfig readPeerConfig() {
+  private ReplicationPeerConfig readPeerConfig() throws ReplicationException {
     try {
-      byte[] data = peerConfigTracker.getData(false);
+      byte[] data = ZKUtil.getData(zookeeper, this.getPeerNode(id));
       if (data != null) {
         this.peerConfig = ReplicationPeerConfigUtil.parsePeerFrom(data);
       }
-    } catch (DeserializationException e) {
-      LOG.error("", e);
+    } catch (DeserializationException | KeeperException | InterruptedException e) {
+      throw new ReplicationException("Get and deserialize peer config date from zookeeper failed: ",
+          e);
     }
     return this.peerConfig;
   }
@@ -128,6 +99,15 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase
     return peerState;
   }
 
+  @Override
+  public PeerState getPeerState(boolean loadFromBackingStore) throws ReplicationException {
+    if (loadFromBackingStore) {
+      return readPeerState();
+    } else {
+      return peerState;
+    }
+  }
+
   /**
    * Get the identifier of this peer
    * @return string representation of the id (short)
@@ -146,6 +126,16 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase
     return peerConfig;
   }
 
+  @Override
+  public ReplicationPeerConfig getPeerConfig(boolean loadFromBackingStore)
+      throws ReplicationException {
+    if (loadFromBackingStore) {
+      return readPeerConfig();
+    } else {
+      return peerConfig;
+    }
+  }
+
   /**
    * Get the configuration object required to communicate with this peer
    * @return configuration object
@@ -180,9 +170,14 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase
   }
 
   @Override
-  public void trackPeerConfigChanges(ReplicationPeerConfigListener listener) {
-    if (this.peerConfigTracker != null){
-      this.peerConfigTracker.setListener(listener);
+  public void registerPeerConfigListener(ReplicationPeerConfigListener listener) {
+    this.peerConfigListeners.add(listener);
+  }
+
+  @Override
+  public void triggerPeerConfigChange(ReplicationPeerConfig newPeerConfig) {
+    for (ReplicationPeerConfigListener listener : this.peerConfigListeners) {
+      listener.peerConfigUpdated(newPeerConfig);
     }
   }
 
@@ -223,97 +218,16 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase
   private static ReplicationProtos.ReplicationState.State parseStateFrom(final byte[] bytes)
       throws DeserializationException {
     ProtobufUtil.expectPBMagicPrefix(bytes);
-    int pblen = ProtobufUtil.lengthOfPBMagic();
+    int pbLen = ProtobufUtil.lengthOfPBMagic();
     ReplicationProtos.ReplicationState.Builder builder =
         ReplicationProtos.ReplicationState.newBuilder();
     ReplicationProtos.ReplicationState state;
     try {
-      ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
+      ProtobufUtil.mergeFrom(builder, bytes, pbLen, bytes.length - pbLen);
       state = builder.build();
       return state.getState();
     } catch (IOException e) {
       throw new DeserializationException(e);
     }
   }
-
-  /**
-   * Utility method to ensure an ENABLED znode is in place; if not present, we create it.
-   * @param path Path to znode to check
-   * @return True if we created the znode.
-   * @throws NodeExistsException
-   * @throws KeeperException
-   */
-  private boolean ensurePeerEnabled(final String path)
-      throws NodeExistsException, KeeperException {
-    if (ZKUtil.checkExists(zookeeper, path) == -1) {
-      // There is a race b/w PeerWatcher and ReplicationZookeeper#add method to create the
-      // peer-state znode. This happens while adding a peer.
-      // The peer state data is set as "ENABLED" by default.
-      ZKUtil.createNodeIfNotExistsAndWatch(zookeeper, path,
-        ReplicationStateZKBase.ENABLED_ZNODE_BYTES);
-      return true;
-    }
-    return false;
-  }
-
-  /**
-   * Tracker for state of this peer
-   */
-  public class PeerStateTracker extends ZKNodeTracker {
-
-    public PeerStateTracker(String peerStateZNode, ZKWatcher watcher,
-        Abortable abortable) {
-      super(watcher, peerStateZNode, abortable);
-    }
-
-    @Override
-    public synchronized void nodeDataChanged(String path) {
-      if (path.equals(node)) {
-        super.nodeDataChanged(path);
-        try {
-          readPeerStateZnode();
-        } catch (DeserializationException e) {
-          LOG.warn("Failed deserializing the content of " + path, e);
-        }
-      }
-    }
-  }
-
-  /**
-   * Tracker for PeerConfigNode of this peer
-   */
-  public class PeerConfigTracker extends ZKNodeTracker {
-
-    ReplicationPeerConfigListener listener;
-
-    public PeerConfigTracker(String peerConfigNode, ZKWatcher watcher,
-        Abortable abortable) {
-      super(watcher, peerConfigNode, abortable);
-    }
-
-    public synchronized void setListener(ReplicationPeerConfigListener listener){
-      this.listener = listener;
-    }
-
-    @Override
-    public synchronized void nodeCreated(String path) {
-      if (path.equals(node)) {
-        super.nodeCreated(path);
-        ReplicationPeerConfig config = readPeerConfig();
-        if (listener != null){
-          listener.peerConfigUpdated(config);
-        }
-      }
-    }
-
-    @Override
-    public synchronized void nodeDataChanged(String path) {
-      //superclass calls nodeCreated
-      if (path.equals(node)) {
-        super.nodeDataChanged(path);
-      }
-
-    }
-
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/72e30d87/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
index 151b6b2..13ec656 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
@@ -500,21 +500,12 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
     }
     Configuration peerConf = pair.getSecond();
 
-    ReplicationPeerZKImpl peer = new ReplicationPeerZKImpl(zookeeper,
-        peerConf, peerId, pair.getFirst(), abortable);
-    try {
-      peer.startStateTracker(this.getPeerStateNode(peerId));
-    } catch (KeeperException e) {
-      throw new ReplicationException("Error starting the peer state tracker for peerId=" +
-          peerId, e);
-    }
+    ReplicationPeerZKImpl peer =
+        new ReplicationPeerZKImpl(zookeeper, peerConf, peerId, pair.getFirst(), abortable);
 
-    try {
-      peer.startPeerConfigTracker(this.getPeerNode(peerId));
-    } catch (KeeperException e) {
-      throw new ReplicationException("Error starting the peer tableCFs tracker for peerId=" +
-          peerId, e);
-    }
+    // Load peer state and peer config by reading zookeeper directly.
+    peer.getPeerState(true);
+    peer.getPeerConfig(true);
 
     return peer;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/72e30d87/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java
index 9a1d9aa..2c522f6 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java
@@ -48,16 +48,12 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
   private final List<ReplicationListener> listeners = new CopyOnWriteArrayList<>();
   // List of all the other region servers in this cluster
   private final ArrayList<String> otherRegionServers = new ArrayList<>();
-  private final ReplicationPeers replicationPeers;
 
-  public ReplicationTrackerZKImpl(ZKWatcher zookeeper,
-      final ReplicationPeers replicationPeers, Configuration conf, Abortable abortable,
-      Stoppable stopper) {
+  public ReplicationTrackerZKImpl(ZKWatcher zookeeper, final ReplicationPeers replicationPeers,
+      Configuration conf, Abortable abortable, Stoppable stopper) {
     super(zookeeper, conf, abortable);
-    this.replicationPeers = replicationPeers;
     this.stopper = stopper;
     this.zookeeper.registerListener(new OtherRegionServerWatcher(this.zookeeper));
-    this.zookeeper.registerListener(new PeersWatcher(this.zookeeper));
   }
 
   @Override
@@ -146,71 +142,6 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
   }
 
   /**
-   * Watcher used to follow the creation and deletion of peer clusters.
-   */
-  public class PeersWatcher extends ZKListener {
-
-    /**
-     * Construct a ZooKeeper event listener.
-     */
-    public PeersWatcher(ZKWatcher watcher) {
-      super(watcher);
-    }
-
-    /**
-     * Called when a node has been deleted
-     * @param path full path of the deleted node
-     */
-    @Override
-    public void nodeDeleted(String path) {
-      List<String> peers = refreshPeersList(path);
-      if (peers == null) {
-        return;
-      }
-      if (isPeerPath(path)) {
-        String id = getZNodeName(path);
-        LOG.info(path + " znode expired, triggering peerRemoved event");
-        for (ReplicationListener rl : listeners) {
-          rl.peerRemoved(id);
-        }
-      }
-    }
-
-    /**
-     * Called when an existing node has a child node added or removed.
-     * @param path full path of the node whose children have changed
-     */
-    @Override
-    public void nodeChildrenChanged(String path) {
-      List<String> peers = refreshPeersList(path);
-      if (peers == null) {
-        return;
-      }
-      LOG.info(path + " znode expired, triggering peerListChanged event");
-      for (ReplicationListener rl : listeners) {
-        rl.peerListChanged(peers);
-      }
-    }
-  }
-
-  /**
-   * Verify if this event is meant for us, and if so then get the latest peers' list from ZK. Also
-   * reset the watches.
-   * @param path path to check against
-   * @return A list of peers' identifiers if the event concerns this watcher, else null.
-   */
-  private List<String> refreshPeersList(String path) {
-    if (!path.startsWith(getPeersZNode())) {
-      return null;
-    }
-    return this.replicationPeers.getAllPeerIds();
-  }
-
-  private String getPeersZNode() {
-    return this.peersZNode;
-  }
-
-  /**
    * Extracts the znode name of a peer cluster from a ZK path
    * @param fullPath Path to extract the id from
    * @return the id or an empty string if path is invalid

http://git-wip-us.apache.org/repos/asf/hbase/blob/72e30d87/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java
index 835ffbf..a82fa3d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import org.apache.hadoop.hbase.replication.regionserver.PeerProcedureHandler;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 
@@ -29,8 +30,14 @@ import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 @InterfaceAudience.Private
 public interface ReplicationSourceService extends ReplicationService {
   /**
-   * Returns a WALObserver for the service. This is needed to 
+   * Returns a WALObserver for the service. This is needed to
    * observe log rolls and log archival events.
    */
   WALActionsListener getWALActionsListener();
+
+
+  /**
+   * Returns a Handler to handle peer procedures.
+   */
+  PeerProcedureHandler getPeerProcedureHandler();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/72e30d87/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java
index 94bcfec..240b0a7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver.handler;
 import org.apache.hadoop.hbase.executor.EventHandler;
 import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.log4j.Logger;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -28,6 +29,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 public class RSProcedureHandler extends EventHandler {
 
+  private static final Logger LOG = Logger.getLogger(RSProcedureHandler.class);
   private final long procId;
 
   private final RSProcedureCallable callable;
@@ -44,6 +46,7 @@ public class RSProcedureHandler extends EventHandler {
     try {
       callable.call();
     } catch (Exception e) {
+      LOG.error("Catch exception when call RSProcedureCallable: ", e);
       error = e;
     }
     ((HRegionServer) server).reportProcedureDone(procId, error);

http://git-wip-us.apache.org/repos/asf/hbase/blob/72e30d87/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/BaseReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/BaseReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/BaseReplicationEndpoint.java
index c390d09..2057057 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/BaseReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/BaseReplicationEndpoint.java
@@ -49,7 +49,7 @@ public abstract class BaseReplicationEndpoint extends AbstractService
     if (this.ctx != null){
       ReplicationPeer peer = this.ctx.getReplicationPeer();
       if (peer != null){
-        peer.trackPeerConfigChanges(this);
+        peer.registerPeerConfigListener(this);
       } else {
         LOG.warn("Not tracking replication peer config changes for Peer Id " + this.ctx.getPeerId() +
             " because there's no such peer");

http://git-wip-us.apache.org/repos/asf/hbase/blob/72e30d87/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java
new file mode 100644
index 0000000..b392985
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java
@@ -0,0 +1,38 @@
+/**
+ * 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.replication.regionserver;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.replication.ReplicationException;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Private
+public interface PeerProcedureHandler {
+
+  public void addPeer(String peerId) throws ReplicationException, IOException;
+
+  public void removePeer(String peerId) throws ReplicationException, IOException;
+
+  public void disablePeer(String peerId) throws ReplicationException, IOException;
+
+  public void enablePeer(String peerId) throws ReplicationException, IOException;
+
+  public void updatePeerConfig(String peerId) throws ReplicationException, IOException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/72e30d87/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
new file mode 100644
index 0000000..9b493d9
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
@@ -0,0 +1,81 @@
+/**
+ * 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.replication.regionserver;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.ReplicationPeer;
+import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.log4j.Logger;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Private
+public class PeerProcedureHandlerImpl implements PeerProcedureHandler {
+  private static final Logger LOG = Logger.getLogger(PeerProcedureHandlerImpl.class);
+
+  private ReplicationSourceManager replicationSourceManager;
+
+  public PeerProcedureHandlerImpl(ReplicationSourceManager replicationSourceManager) {
+    this.replicationSourceManager = replicationSourceManager;
+  }
+
+  @Override
+  public void addPeer(String peerId) throws ReplicationException, IOException {
+    replicationSourceManager.addPeer(peerId);
+  }
+
+  @Override
+  public void removePeer(String peerId) throws ReplicationException, IOException {
+    replicationSourceManager.removePeer(peerId);
+  }
+
+  @Override
+  public void disablePeer(String peerId) throws ReplicationException, IOException {
+    ReplicationPeer peer = replicationSourceManager.getReplicationPeers().getConnectedPeer(peerId);
+    if (peer != null) {
+      PeerState peerState = peer.getPeerState(true);
+      LOG.info("disablePeer state, peer id: " + peerId + ", state: " + peerState);
+    } else {
+      throw new ReplicationException("No connected peer found, peerId=" + peerId);
+    }
+  }
+
+  @Override
+  public void enablePeer(String peerId) throws ReplicationException, IOException {
+    ReplicationPeer peer = replicationSourceManager.getReplicationPeers().getConnectedPeer(peerId);
+    if (peer != null) {
+      PeerState peerState = peer.getPeerState(true);
+      LOG.info("enablePeer state, peer id: " + peerId + ", state: " + peerState);
+    } else {
+      throw new ReplicationException("No connected peer found, peerId=" + peerId);
+    }
+  }
+
+  @Override
+  public void updatePeerConfig(String peerId) throws ReplicationException, IOException {
+    ReplicationPeer peer = replicationSourceManager.getReplicationPeers().getConnectedPeer(peerId);
+    if (peer == null) {
+      throw new ReplicationException("No connected peer found, peerId=" + peerId);
+    }
+    ReplicationPeerConfig rpc = peer.getPeerConfig(true);
+    peer.triggerPeerConfigChange(rpc);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/72e30d87/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java
index a47a483..c3f33aa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java
@@ -17,27 +17,29 @@
  */
 package org.apache.hadoop.hbase.replication.regionserver;
 
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.executor.EventType;
 import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationType;
+import org.apache.log4j.Logger;
 import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RefreshPeerParameter;
 
 /**
- * The callable executed at RS side to refresh the peer config/state.
- * <p>
- * TODO: only a dummy implementation for verifying the framework, will add implementation later.
+ * The callable executed at RS side to refresh the peer config/state. <br/>
  */
 @InterfaceAudience.Private
 public class RefreshPeerCallable implements RSProcedureCallable {
 
+  private static final Logger LOG = Logger.getLogger(RefreshPeerCallable.class);
   private HRegionServer rs;
 
   private String peerId;
 
+  private PeerModificationType type;
+
   private Exception initError;
 
   @Override
@@ -45,9 +47,27 @@ public class RefreshPeerCallable implements RSProcedureCallable {
     if (initError != null) {
       throw initError;
     }
-    Path dir = new Path("/" + peerId);
-    if (rs.getFileSystem().exists(dir)) {
-      rs.getFileSystem().create(new Path(dir, rs.getServerName().toString())).close();
+
+    LOG.info("Received a peer change event, peerId=" + peerId + ", type=" + type);
+    PeerProcedureHandler handler = rs.getReplicationSourceService().getPeerProcedureHandler();
+    switch (type) {
+      case ADD_PEER:
+        handler.addPeer(this.peerId);
+        break;
+      case REMOVE_PEER:
+        handler.removePeer(this.peerId);
+        break;
+      case ENABLE_PEER:
+        handler.enablePeer(this.peerId);
+        break;
+      case DISABLE_PEER:
+        handler.disablePeer(this.peerId);
+        break;
+      case UPDATE_PEER_CONFIG:
+        handler.updatePeerConfig(this.peerId);
+        break;
+      default:
+        throw new IllegalArgumentException("Unknown peer modification type: " + type);
     }
     return null;
   }
@@ -56,10 +76,11 @@ public class RefreshPeerCallable implements RSProcedureCallable {
   public void init(byte[] parameter, HRegionServer rs) {
     this.rs = rs;
     try {
-      this.peerId = RefreshPeerParameter.parseFrom(parameter).getPeerId();
+      RefreshPeerParameter param = RefreshPeerParameter.parseFrom(parameter);
+      this.peerId = param.getPeerId();
+      this.type = param.getType();
     } catch (InvalidProtocolBufferException e) {
       initError = e;
-      return;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/72e30d87/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
index d8212e9..571ee75 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
@@ -87,6 +87,8 @@ public class Replication implements
   // ReplicationLoad to access replication metrics
   private ReplicationLoad replicationLoad;
 
+  private PeerProcedureHandler peerProcedureHandler;
+
   /**
    * Instantiate the replication management (if rep is enabled).
    * @param server Hosting server
@@ -151,6 +153,8 @@ public class Replication implements
         this.conf.getInt("replication.stats.thread.period.seconds", 5 * 60);
     LOG.debug("ReplicationStatisticsThread " + this.statsThreadPeriod);
     this.replicationLoad = new ReplicationLoad();
+
+    this.peerProcedureHandler = new PeerProcedureHandlerImpl(replicationManager);
   }
 
   /**
@@ -168,6 +172,12 @@ public class Replication implements
   public WALActionsListener getWALActionsListener() {
     return this;
   }
+
+  @Override
+  public PeerProcedureHandler getPeerProcedureHandler() {
+    return peerProcedureHandler;
+  }
+
   /**
    * Stops replication service.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/72e30d87/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
index f4f35ae..19ea240 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -446,12 +445,10 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
 
   public void terminate(String reason, Exception cause, boolean join) {
     if (cause == null) {
-      LOG.info("Closing source "
-          + this.peerClusterZnode + " because: " + reason);
-
+      LOG.info("Closing source " + this.peerClusterZnode + " because: " + reason);
     } else {
-      LOG.error("Closing source " + this.peerClusterZnode
-          + " because an error occurred: " + reason, cause);
+      LOG.error("Closing source " + this.peerClusterZnode + " because an error occurred: " + reason,
+        cause);
     }
     this.sourceRunning = false;
     Collection<ReplicationSourceShipper> workers = workerThreads.values();

http://git-wip-us.apache.org/repos/asf/hbase/blob/72e30d87/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 07c53e1..a263fc3 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
@@ -564,6 +564,18 @@ public class ReplicationSourceManager implements ReplicationListener {
     this.walsById.remove(src.getPeerClusterZnode());
   }
 
+  public void addPeer(String id) throws ReplicationException, IOException {
+    LOG.info("Trying to add peer, peerId: " + id);
+    boolean added = this.replicationPeers.peerConnected(id);
+    if (added) {
+      LOG.info("Peer " + id + " connected success, trying to start the replication source thread.");
+      addSource(id);
+      if (replicationForBulkLoadDataEnabled) {
+        this.replicationQueues.addPeerToHFileRefs(id);
+      }
+    }
+  }
+
   /**
    * Thie method first deletes all the recovered sources for the specified
    * id, then deletes the normal source (deleting all related data in ZK).
@@ -611,6 +623,8 @@ public class ReplicationSourceManager implements ReplicationListener {
       }
       deleteSource(id, true);
     }
+    // Remove HFile Refs znode from zookeeper
+    this.replicationQueues.removePeerFromHFileRefs(id);
   }
 
   @Override
@@ -618,29 +632,6 @@ public class ReplicationSourceManager implements ReplicationListener {
     transferQueues(regionserver);
   }
 
-  @Override
-  public void peerRemoved(String peerId) {
-    removePeer(peerId);
-    this.replicationQueues.removePeerFromHFileRefs(peerId);
-  }
-
-  @Override
-  public void peerListChanged(List<String> peerIds) {
-    for (String id : peerIds) {
-      try {
-        boolean added = this.replicationPeers.peerConnected(id);
-        if (added) {
-          addSource(id);
-          if (replicationForBulkLoadDataEnabled) {
-            this.replicationQueues.addPeerToHFileRefs(id);
-          }
-        }
-      } catch (Exception e) {
-        LOG.error("Error while adding a new peer", e);
-      }
-    }
-  }
-
   /**
    * Class responsible to setup new ReplicationSources to take care of the
    * queues from dead region servers.

http://git-wip-us.apache.org/repos/asf/hbase/blob/72e30d87/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
index 8770299..67c635b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
@@ -748,4 +748,4 @@ public class TestReplicationAdmin {
     assertEquals(2097152, admin.getPeerConfig(ID_ONE).getBandwidth());
     admin.removePeer(ID_ONE);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/72e30d87/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminUsingProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminUsingProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminUsingProcedure.java
new file mode 100644
index 0000000..b09a8a7
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminUsingProcedure.java
@@ -0,0 +1,226 @@
+/**
+ * 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.client.replication;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.TestReplicationBase;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableMap;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.log4j.Logger;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MediumTests.class, ClientTests.class })
+public class TestReplicationAdminUsingProcedure extends TestReplicationBase {
+
+  private static final String PEER_ID = "2";
+  private static final Logger LOG = Logger.getLogger(TestReplicationAdminUsingProcedure.class);
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    conf1.setInt("hbase.multihconnection.threads.max", 10);
+
+    // Start the master & slave mini cluster.
+    TestReplicationBase.setUpBeforeClass();
+
+    // Remove the replication peer
+    hbaseAdmin.removeReplicationPeer(PEER_ID);
+  }
+
+  private void loadData(int startRowKey, int endRowKey) throws IOException {
+    for (int i = startRowKey; i < endRowKey; i++) {
+      byte[] rowKey = Bytes.add(row, Bytes.toBytes(i));
+      Put put = new Put(rowKey);
+      put.addColumn(famName, null, Bytes.toBytes(i));
+      htable1.put(put);
+    }
+  }
+
+  private void waitForReplication(int expectedRows, int retries)
+      throws IOException, InterruptedException {
+    Scan scan;
+    for (int i = 0; i < retries; i++) {
+      scan = new Scan();
+      if (i == retries - 1) {
+        throw new IOException("Waited too much time for normal batch replication");
+      }
+      try (ResultScanner scanner = htable2.getScanner(scan)) {
+        int count = 0;
+        for (Result res : scanner) {
+          count++;
+        }
+        if (count != expectedRows) {
+          LOG.info("Only got " + count + " rows,  expected rows: " + expectedRows);
+          Thread.sleep(SLEEP_TIME);
+        } else {
+          return;
+        }
+      }
+    }
+  }
+
+  @Before
+  public void setUp() throws IOException {
+    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(utility2.getClusterKey());
+    hbaseAdmin.addReplicationPeer(PEER_ID, rpc);
+
+    utility1.waitUntilAllRegionsAssigned(tableName);
+    utility2.waitUntilAllRegionsAssigned(tableName);
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    hbaseAdmin.removeReplicationPeer(PEER_ID);
+    truncateBoth();
+  }
+
+  private void truncateBoth() throws IOException {
+    utility1.deleteTableData(tableName);
+    utility2.deleteTableData(tableName);
+  }
+
+  @Test
+  public void testAddPeer() throws Exception {
+    // Load data
+    loadData(0, NB_ROWS_IN_BATCH);
+
+    // Wait the replication finished
+    waitForReplication(NB_ROWS_IN_BATCH, NB_RETRIES);
+  }
+
+  @Test
+  public void testRemovePeer() throws Exception {
+    // prev-check
+    waitForReplication(0, NB_RETRIES);
+
+    // Load data
+    loadData(0, NB_ROWS_IN_BATCH);
+
+    // Wait the replication finished
+    waitForReplication(NB_ROWS_IN_BATCH, NB_RETRIES);
+
+    // Remove the peer id
+    hbaseAdmin.removeReplicationPeer(PEER_ID);
+
+    // Load data again
+    loadData(NB_ROWS_IN_BATCH, 2 * NB_ROWS_IN_BATCH);
+
+    // Wait the replication again
+    boolean foundException = false;
+    try {
+      waitForReplication(NB_ROWS_IN_BATCH * 2, NB_RETRIES);
+    } catch (IOException e) {
+      foundException = true;
+    }
+    Assert.assertTrue(foundException);
+
+    // Truncate the table in source cluster
+    truncateBoth();
+
+    // Add peer again
+    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(utility2.getClusterKey());
+    hbaseAdmin.addReplicationPeer(PEER_ID, rpc);
+
+    // Load data again
+    loadData(0, NB_ROWS_IN_BATCH);
+
+    // Wait the replication finished
+    waitForReplication(NB_ROWS_IN_BATCH, NB_RETRIES);
+  }
+
+  @Test
+  public void testDisableAndEnablePeer() throws Exception {
+    // disable peer
+    hbaseAdmin.disableReplicationPeer(PEER_ID);
+
+    // Load data
+    loadData(0, NB_ROWS_IN_BATCH);
+
+    // Will failed to wait the replication.
+    boolean foundException = false;
+    try {
+      waitForReplication(NB_ROWS_IN_BATCH, NB_RETRIES);
+    } catch (IOException e) {
+      foundException = true;
+    }
+    Assert.assertTrue(foundException);
+
+    // Enable the peer
+    hbaseAdmin.enableReplicationPeer(PEER_ID);
+    waitForReplication(NB_ROWS_IN_BATCH, NB_RETRIES);
+
+    // Load more data
+    loadData(NB_ROWS_IN_BATCH, NB_ROWS_IN_BATCH * 2);
+
+    // Wait replication again.
+    waitForReplication(NB_ROWS_IN_BATCH * 2, NB_RETRIES);
+  }
+
+  @Test
+  public void testUpdatePeerConfig() throws Exception {
+    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(utility2.getClusterKey());
+    rpc.setExcludeTableCFsMap(
+      ImmutableMap.of(tableName, ImmutableList.of(Bytes.toString(famName))));
+
+    // Update the peer config to exclude the test table name.
+    hbaseAdmin.updateReplicationPeerConfig(PEER_ID, rpc);
+
+    // Load data
+    loadData(0, NB_ROWS_IN_BATCH);
+
+    // Will failed to wait the replication
+    boolean foundException = false;
+    try {
+      waitForReplication(NB_ROWS_IN_BATCH, NB_RETRIES);
+    } catch (IOException e) {
+      foundException = true;
+    }
+    Assert.assertTrue(foundException);
+
+    // Truncate the table in source cluster
+    truncateBoth();
+
+    // Update the peer config to include the test table name.
+    ReplicationPeerConfig rpc2 = new ReplicationPeerConfig();
+    rpc2.setClusterKey(utility2.getClusterKey());
+    hbaseAdmin.updateReplicationPeerConfig(PEER_ID, rpc2);
+
+    // Load data again
+    loadData(0, NB_ROWS_IN_BATCH);
+
+    // Wait the replication finished
+    waitForReplication(NB_ROWS_IN_BATCH, NB_RETRIES);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/72e30d87/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
deleted file mode 100644
index ed7c6fa..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
+++ /dev/null
@@ -1,48 +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.master.replication;
-
-import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-
-public class DummyModifyPeerProcedure extends ModifyPeerProcedure {
-
-  public DummyModifyPeerProcedure() {
-  }
-
-  public DummyModifyPeerProcedure(String peerId) {
-    super(peerId);
-  }
-
-  @Override
-  public PeerOperationType getPeerOperationType() {
-    return PeerOperationType.ADD;
-  }
-
-  @Override
-  protected void prePeerModification(MasterProcedureEnv env) {
-  }
-
-  @Override
-  protected void updatePeerStorage(MasterProcedureEnv env) {
-  }
-
-  @Override
-  protected void postPeerModification(MasterProcedureEnv env) {
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/72e30d87/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestDummyModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestDummyModifyPeerProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestDummyModifyPeerProcedure.java
deleted file mode 100644
index ec06306..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestDummyModifyPeerProcedure.java
+++ /dev/null
@@ -1,80 +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.master.replication;
-
-import static org.junit.Assert.assertTrue;
-
-import java.util.HashSet;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.Waiter;
-import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.testclassification.MasterTests;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({ MasterTests.class, LargeTests.class })
-public class TestDummyModifyPeerProcedure {
-
-  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
-
-  private static String PEER_ID;
-
-  private static Path DIR;
-
-  @BeforeClass
-  public static void setUp() throws Exception {
-    UTIL.startMiniCluster(3);
-    PEER_ID = "testPeer";
-    DIR = new Path("/" + PEER_ID);
-    UTIL.getTestFileSystem().mkdirs(DIR);
-  }
-
-  @AfterClass
-  public static void tearDown() throws Exception {
-    UTIL.shutdownMiniCluster();
-  }
-
-  @Test
-  public void test() throws Exception {
-    ProcedureExecutor<?> executor =
-        UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor();
-    long procId = executor.submitProcedure(new DummyModifyPeerProcedure(PEER_ID));
-    UTIL.waitFor(30000, new Waiter.Predicate<Exception>() {
-
-      @Override
-      public boolean evaluate() throws Exception {
-        return executor.isFinished(procId);
-      }
-    });
-    Set<String> serverNames = UTIL.getHBaseCluster().getRegionServerThreads().stream()
-        .map(t -> t.getRegionServer().getServerName().toString())
-        .collect(Collectors.toCollection(HashSet::new));
-    for (FileStatus s : UTIL.getTestFileSystem().listStatus(DIR)) {
-      assertTrue(serverNames.remove(s.getPath().getName()));
-    }
-    assertTrue(serverNames.isEmpty());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/72e30d87/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
index a04d524..f118ca3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase.replication;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
@@ -156,41 +155,6 @@ public class TestReplicationTrackerZKImpl {
   }
 
   @Test(timeout = 30000)
-  public void testPeerRemovedEvent() throws Exception {
-    rp.registerPeer("5", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey()));
-    rt.registerListener(new DummyReplicationListener());
-    rp.unregisterPeer("5");
-    // wait for event
-    while (peerRemovedCount.get() < 1) {
-      Thread.sleep(5);
-    }
-    assertEquals("5", peerRemovedData);
-  }
-
-  @Test(timeout = 30000)
-  public void testPeerListChangedEvent() throws Exception {
-    // add a peer
-    rp.registerPeer("5", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey()));
-    zkw.getRecoverableZooKeeper().getZooKeeper().getChildren("/hbase/replication/peers/5", true);
-    rt.registerListener(new DummyReplicationListener());
-    rp.disablePeer("5");
-    int tmp = plChangedCount.get();
-    LOG.info("Peer count=" + tmp);
-    ZKUtil.deleteNode(zkw, "/hbase/replication/peers/5/peer-state");
-    // wait for event
-    while (plChangedCount.get() <= tmp) {
-      Thread.sleep(100);
-      LOG.info("Peer count=" + tmp);
-    }
-    assertEquals(1, plChangedData.size());
-    assertTrue(plChangedData.contains("5"));
-
-    // clean up
-    //ZKUtil.deleteNode(zkw, "/hbase/replication/peers/5");
-    rp.unregisterPeer("5");
-  }
-
-  @Test(timeout = 30000)
   public void testPeerNameControl() throws Exception {
     int exists = 0;
     int hyphen = 0;
@@ -222,21 +186,6 @@ public class TestReplicationTrackerZKImpl {
       rsRemovedCount.getAndIncrement();
       LOG.debug("Received regionServerRemoved event: " + regionServer);
     }
-
-    @Override
-    public void peerRemoved(String peerId) {
-      peerRemovedData = peerId;
-      peerRemovedCount.getAndIncrement();
-      LOG.debug("Received peerDisconnected event: " + peerId);
-    }
-
-    @Override
-    public void peerListChanged(List<String> peerIds) {
-      plChangedData.clear();
-      plChangedData.addAll(peerIds);
-      int count = plChangedCount.getAndIncrement();
-      LOG.debug("Received peerListChanged event " + count);
-    }
   }
 
   private class DummyServer implements Server {

http://git-wip-us.apache.org/repos/asf/hbase/blob/72e30d87/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
index 307ea7f..9f234a8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
@@ -176,6 +176,12 @@ public abstract class TestReplicationSourceManager {
     replication = new Replication(new DummyServer(), fs, logDir, oldLogDir);
 
     managerOfCluster = getManagerFromCluster();
+    if (managerOfCluster != null) {
+      // After replication procedure, we need to add peer by hand (other than by receiving
+      // notification from zk)
+      managerOfCluster.addPeer(slaveId);
+    }
+
     manager = replication.getReplicationManager();
     manager.addSource(slaveId);
     if (managerOfCluster != null) {
@@ -535,18 +541,16 @@ public abstract class TestReplicationSourceManager {
       final int globalLogQueueSizeInitial = globalSource.getSizeOfLogQueue();
       final long sizeOfLatestPath = getSizeOfLatestPath();
       addPeerAndWait(peerId, peerConfig, true);
-      assertEquals(sizeOfLatestPath + globalLogQueueSizeInitial,
-          globalSource.getSizeOfLogQueue());
+      assertEquals(sizeOfLatestPath + globalLogQueueSizeInitial, globalSource.getSizeOfLogQueue());
       ReplicationSourceInterface source = manager.getSource(peerId);
       // Sanity check
       assertNotNull(source);
       final int sizeOfSingleLogQueue = source.getSourceMetrics().getSizeOfLogQueue();
       // Enqueue log and check if metrics updated
       source.enqueueLog(new Path("abc"));
-      assertEquals(1 + sizeOfSingleLogQueue,
-          source.getSourceMetrics().getSizeOfLogQueue());
-      assertEquals(source.getSourceMetrics().getSizeOfLogQueue()
-              + globalLogQueueSizeInitial, globalSource.getSizeOfLogQueue());
+      assertEquals(1 + sizeOfSingleLogQueue, source.getSourceMetrics().getSizeOfLogQueue());
+      assertEquals(source.getSourceMetrics().getSizeOfLogQueue() + globalLogQueueSizeInitial,
+        globalSource.getSizeOfLogQueue());
 
       // Removing the peer should reset the global metrics
       removePeerAndWait(peerId);
@@ -556,9 +560,8 @@ public abstract class TestReplicationSourceManager {
       addPeerAndWait(peerId, peerConfig, true);
       source = manager.getSource(peerId);
       assertNotNull(source);
-      assertEquals(sizeOfLatestPath, source.getSourceMetrics().getSizeOfLogQueue());
-      assertEquals(source.getSourceMetrics().getSizeOfLogQueue()
-          + globalLogQueueSizeInitial, globalSource.getSizeOfLogQueue());
+      assertEquals(source.getSourceMetrics().getSizeOfLogQueue() + globalLogQueueSizeInitial,
+        globalSource.getSizeOfLogQueue());
     } finally {
       removePeerAndWait(peerId);
     }
@@ -575,8 +578,14 @@ public abstract class TestReplicationSourceManager {
       final boolean waitForSource) throws Exception {
     final ReplicationPeers rp = manager.getReplicationPeers();
     rp.registerPeer(peerId, peerConfig);
+    try {
+      manager.addPeer(peerId);
+    } catch (Exception e) {
+      // ignore the failed exception, because we'll test both success & failed case.
+    }
     waitPeer(peerId, manager, waitForSource);
     if (managerOfCluster != null) {
+      managerOfCluster.addPeer(peerId);
       waitPeer(peerId, managerOfCluster, waitForSource);
     }
   }
@@ -609,6 +618,11 @@ public abstract class TestReplicationSourceManager {
     final ReplicationPeers rp = manager.getReplicationPeers();
     if (rp.getAllPeerIds().contains(peerId)) {
       rp.unregisterPeer(peerId);
+      try {
+        manager.removePeer(peerId);
+      } catch (Exception e) {
+        // ignore the failed exception and continue.
+      }
     }
     Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
       @Override public boolean evaluate() throws Exception {

http://git-wip-us.apache.org/repos/asf/hbase/blob/72e30d87/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestTableBasedReplicationSourceManagerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestTableBasedReplicationSourceManagerImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestTableBasedReplicationSourceManagerImpl.java
index 19457e2..9ff1865 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestTableBasedReplicationSourceManagerImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestTableBasedReplicationSourceManagerImpl.java
@@ -53,10 +53,9 @@ public class TestTableBasedReplicationSourceManagerImpl extends TestReplicationS
       TableBasedReplicationQueuesClientImpl.class, ReplicationQueuesClient.class);
     utility = new HBaseTestingUtility(conf);
     utility.startMiniCluster();
-    Waiter.waitFor(conf, 3 * 1000,
-      () -> utility.getMiniHBaseCluster().getMaster().isInitialized());
-    utility.waitUntilAllRegionsAssigned(TableName.valueOf(
-        NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "replication"));
+    Waiter.waitFor(conf, 3 * 1000, () -> utility.getMiniHBaseCluster().getMaster().isInitialized());
+    utility.waitUntilAllRegionsAssigned(
+      TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "replication"));
     setupZkAndReplication();
   }
 


[04/26] hbase git commit: HBASE-19148 Reevaluate default values of configurations

Posted by zh...@apache.org.
HBASE-19148 Reevaluate default values of configurations

Removed unused:
    <name>hbase.fs.tmp.dir</name>
Added hbase.master.loadbalance.bytable

Edit of description text. Moved stuff around  to put configs beside each
other.

M hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerCommandLine.java

 Emit some hbase configs in log on startup.

Signed-off-by: Michael Stack <st...@apache.org>


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

Branch: refs/heads/HBASE-19397
Commit: 4d6b928682cc2a17f3dfd0179fb3fd46fd9e0a1f
Parents: 59baf12
Author: Michael Stack <st...@apache.org>
Authored: Fri Dec 15 17:56:38 2017 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Thu Dec 21 14:07:16 2017 -0800

----------------------------------------------------------------------
 .../src/main/resources/hbase-default.xml        | 177 +++++++++++--------
 .../org/apache/hadoop/hbase/master/HMaster.java |   2 +-
 .../assignment/SplitTableRegionProcedure.java   |   7 +-
 .../hbase/regionserver/wal/AbstractFSWAL.java   |  14 +-
 .../hadoop/hbase/util/ServerCommandLine.java    |  28 ++-
 5 files changed, 140 insertions(+), 88 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4d6b9286/hbase-common/src/main/resources/hbase-default.xml
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml
index 7995c41..b42ce3f 100644
--- a/hbase-common/src/main/resources/hbase-default.xml
+++ b/hbase-common/src/main/resources/hbase-default.xml
@@ -63,13 +63,6 @@ possible configurations would overwhelm and obscure the important.
     machine restart.</description>
   </property>
   <property >
-    <name>hbase.fs.tmp.dir</name>
-    <value>/user/${user.name}/hbase-staging</value>
-    <description>A staging directory in default file system (HDFS)
-    for keeping temporary data.
-    </description>
-  </property>
-  <property >
     <name>hbase.cluster.distributed</name>
     <value>false</value>
     <description>The mode the cluster will be in. Possible values are
@@ -146,8 +139,8 @@ possible configurations would overwhelm and obscure the important.
   <property>
     <name>hbase.master.procedurewalcleaner.ttl</name>
     <value>604800000</value>
-    <description>How long a Procedure WAL stays will remain in the
-    {hbase.rootdir}/oldWALs/masterProcedureWALs directory, after which it will be cleaned
+    <description>How long a Procedure WAL will remain in the
+    {hbase.rootdir}/MasterProcedureWALs directory, after which it will be cleaned
     by a Master thread. The value is in milliseconds.</description>
   </property>
   <property>
@@ -166,7 +159,19 @@ possible configurations would overwhelm and obscure the important.
     <value>true</value>
     <description>Whether or not the Master listens to the Master web
       UI port (hbase.master.info.port) and redirects requests to the web
-      UI server shared by the Master and RegionServer.</description>
+      UI server shared by the Master and RegionServer. Config. makes
+      sense when Master is serving Regions (not the default).</description>
+  </property>
+  <property>
+    <name>hbase.master.fileSplitTimeout</name>
+    <value>600000</value>
+    <description>Splitting a region, how long to wait on the file-splitting
+      step before aborting the attempt. Default: 600000. This setting used
+      to be known as hbase.regionserver.fileSplitTimeout in hbase-1.x.
+      Split is now run master-side hence the rename (If a
+      'hbase.master.fileSplitTimeout' setting found, will use it to
+      prime the current 'hbase.master.fileSplitTimeout'
+      Configuration.</description>
   </property>
 
   <!--RegionServer configurations-->
@@ -198,7 +203,10 @@ possible configurations would overwhelm and obscure the important.
     <name>hbase.regionserver.handler.count</name>
     <value>30</value>
     <description>Count of RPC Listener instances spun up on RegionServers.
-    Same property is used by the Master for count of master handlers.</description>
+      Same property is used by the Master for count of master handlers.
+      Too many handlers can be counter-productive. Make it a multiple of
+      CPU count. If mostly read-only, handlers count close to cpu count
+      does well. Start with twice the CPU count and tune from there.</description>
   </property>
   <property>
     <name>hbase.ipc.server.callqueue.handler.factor</name>
@@ -292,31 +300,37 @@ possible configurations would overwhelm and obscure the important.
       Updates are blocked and flushes are forced until size of all memstores
       in a region server hits hbase.regionserver.global.memstore.size.lower.limit.
       The default value in this configuration has been intentionally left empty in order to
-      honor the old hbase.regionserver.global.memstore.upperLimit property if present.</description>
+      honor the old hbase.regionserver.global.memstore.upperLimit property if present.
+    </description>
   </property>
   <property>
     <name>hbase.regionserver.global.memstore.size.lower.limit</name>
     <value></value>
-    <description>Maximum size of all memstores in a region server before flushes are forced.
-      Defaults to 95% of hbase.regionserver.global.memstore.size (0.95).
-      A 100% value for this value causes the minimum possible flushing to occur when updates are
-      blocked due to memstore limiting.
-      The default value in this configuration has been intentionally left empty in order to
-      honor the old hbase.regionserver.global.memstore.lowerLimit property if present.</description>
+    <description>Maximum size of all memstores in a region server before flushes
+      are forced. Defaults to 95% of hbase.regionserver.global.memstore.size
+      (0.95). A 100% value for this value causes the minimum possible flushing
+      to occur when updates are blocked due to memstore limiting. The default
+      value in this configuration has been intentionally left empty in order to
+      honor the old hbase.regionserver.global.memstore.lowerLimit property if
+      present.
+    </description>
   </property>
   <property>
     <name>hbase.systemtables.compacting.memstore.type</name>
     <value>NONE</value>
-    <description>Determines the type of memstore to be used for system tables like META, namespace tables etc.
-    By default NONE is the type and hence we use the default memstore for all the system tables. If we
-    need to use compacting memstore for system tables then set this property to BASIC/EAGER</description>
+    <description>Determines the type of memstore to be used for system tables like
+      META, namespace tables etc. By default NONE is the type and hence we use the
+      default memstore for all the system tables. If we need to use compacting
+      memstore for system tables then set this property to BASIC/EAGER
+    </description>
   </property>
   <property>
     <name>hbase.regionserver.optionalcacheflushinterval</name>
     <value>3600000</value>
     <description>
     Maximum amount of time an edit lives in memory before being automatically flushed.
-    Default 1 hour. Set it to 0 to disable automatic flushing.</description>
+    Default 1 hour. Set it to 0 to disable automatic flushing.
+  </description>
   </property>
   <property>
     <name>hbase.regionserver.dns.interface</name>
@@ -335,19 +349,21 @@ possible configurations would overwhelm and obscure the important.
     <name>hbase.regionserver.region.split.policy</name>
     <value>org.apache.hadoop.hbase.regionserver.SteppingSplitPolicy</value>
     <description>
-      A split policy determines when a region should be split. The various other split policies that
-      are available currently are BusyRegionSplitPolicy, ConstantSizeRegionSplitPolicy, DisabledRegionSplitPolicy,
-      DelimitedKeyPrefixRegionSplitPolicy, KeyPrefixRegionSplitPolicy, and SteppingSplitPolicy.
-      DisabledRegionSplitPolicy blocks manual region splitting.
+      A split policy determines when a region should be split. The various
+      other split policies that are available currently are BusyRegionSplitPolicy,
+      ConstantSizeRegionSplitPolicy, DisabledRegionSplitPolicy,
+      DelimitedKeyPrefixRegionSplitPolicy, KeyPrefixRegionSplitPolicy, and
+      SteppingSplitPolicy. DisabledRegionSplitPolicy blocks manual region splitting.
     </description>
   </property>
   <property>
     <name>hbase.regionserver.regionSplitLimit</name>
     <value>1000</value>
     <description>
-      Limit for the number of regions after which no more region splitting should take place.
-      This is not hard limit for the number of regions but acts as a guideline for the regionserver
-      to stop splitting after a certain limit. Default is set to 1000.
+      Limit for the number of regions after which no more region splitting
+      should take place. This is not hard limit for the number of regions
+      but acts as a guideline for the regionserver to stop splitting after
+      a certain limit. Default is set to 1000.
     </description>
   </property>
 
@@ -357,14 +373,15 @@ possible configurations would overwhelm and obscure the important.
     <value>90000</value>
     <description>ZooKeeper session timeout in milliseconds. It is used in two different ways.
       First, this value is used in the ZK client that HBase uses to connect to the ensemble.
-      It is also used by HBase when it starts a ZK server and it is passed as the 'maxSessionTimeout'. See
-      http://hadoop.apache.org/zookeeper/docs/current/zookeeperProgrammers.html#ch_zkSessions.
+      It is also used by HBase when it starts a ZK server and it is passed as the 'maxSessionTimeout'.
+      See http://hadoop.apache.org/zookeeper/docs/current/zookeeperProgrammers.html#ch_zkSessions.
       For example, if an HBase region server connects to a ZK ensemble that's also managed
-      by HBase, then the
-      session timeout will be the one specified by this configuration. But, a region server that connects
-      to an ensemble managed with a different configuration will be subjected that ensemble's maxSessionTimeout. So,
-      even though HBase might propose using 90 seconds, the ensemble can have a max timeout lower than this and
-      it will take precedence. The current default that ZK ships with is 40 seconds, which is lower than HBase's.
+      by HBase, then the session timeout will be the one specified by this configuration.
+      But, a region server that connects to an ensemble managed with a different configuration
+      will be subjected that ensemble's maxSessionTimeout. So, even though HBase might propose
+      using 90 seconds, the ensemble can have a max timeout lower than this and it will take
+      precedence. The current default that ZK ships with is 40 seconds, which is lower than
+      HBase's.
     </description>
   </property>
   <property>
@@ -373,7 +390,8 @@ possible configurations would overwhelm and obscure the important.
     <description>Root ZNode for HBase in ZooKeeper. All of HBase's ZooKeeper
       files that are configured with a relative path will go under this node.
       By default, all of HBase's ZooKeeper file paths are configured with a
-      relative path, so they will all go under this directory unless changed.</description>
+      relative path, so they will all go under this directory unless changed.
+    </description>
   </property>
   <property>
     <name>zookeeper.znode.acl.parent</name>
@@ -1120,6 +1138,26 @@ possible configurations would overwhelm and obscure the important.
     HBase shell.</description>
   </property>
   <property>
+    <name>hbase.coprocessor.master.classes</name>
+    <value></value>
+    <description>A comma-separated list of
+    org.apache.hadoop.hbase.coprocessor.MasterObserver coprocessors that are
+    loaded by default on the active HMaster process. For any implemented
+    coprocessor methods, the listed classes will be called in order. After
+    implementing your own MasterObserver, just put it in HBase's classpath
+    and add the fully qualified class name here.</description>
+  </property>
+  <property>
+      <name>hbase.coprocessor.abortonerror</name>
+      <value>true</value>
+      <description>Set to true to cause the hosting server (master or regionserver)
+      to abort if a coprocessor fails to load, fails to initialize, or throws an
+      unexpected Throwable object. Setting this to false will allow the server to
+      continue execution but the system wide state of the coprocessor in question
+      will become inconsistent as it will be properly executing in only a subset
+      of servers, so this is most useful for debugging only.</description>
+  </property>
+  <property>
     <name>hbase.rest.port</name>
     <value>8080</value>
     <description>The port for the HBase REST server.</description>
@@ -1172,26 +1210,6 @@ possible configurations would overwhelm and obscure the important.
     version is X.X.X-SNAPSHOT"</description>
   </property>
   <property>
-    <name>hbase.coprocessor.master.classes</name>
-    <value></value>
-    <description>A comma-separated list of
-    org.apache.hadoop.hbase.coprocessor.MasterObserver coprocessors that are
-    loaded by default on the active HMaster process. For any implemented
-    coprocessor methods, the listed classes will be called in order. After
-    implementing your own MasterObserver, just put it in HBase's classpath
-    and add the fully qualified class name here.</description>
-  </property>
-  <property>
-      <name>hbase.coprocessor.abortonerror</name>
-      <value>true</value>
-      <description>Set to true to cause the hosting server (master or regionserver)
-      to abort if a coprocessor fails to load, fails to initialize, or throws an
-      unexpected Throwable object. Setting this to false will allow the server to
-      continue execution but the system wide state of the coprocessor in question
-      will become inconsistent as it will be properly executing in only a subset
-      of servers, so this is most useful for debugging only.</description>
-  </property>
-  <property>
     <name>hbase.table.lock.enable</name>
     <value>true</value>
     <description>Set to true to enable locking the table in zookeeper for schema change operations.
@@ -1388,7 +1406,6 @@ possible configurations would overwhelm and obscure the important.
       are NULL, CRC32, CRC32C.
     </description>
   </property>
-
   <property>
     <name>hbase.client.scanner.max.result.size</name>
     <value>2097152</value>
@@ -1398,7 +1415,6 @@ possible configurations would overwhelm and obscure the important.
     With faster and/or high latency networks this value should be increased.
     </description>
   </property>
-
   <property>
     <name>hbase.server.scanner.max.result.size</name>
     <value>104857600</value>
@@ -1408,7 +1424,6 @@ possible configurations would overwhelm and obscure the important.
     This is a safety setting to protect the server from OOM situations.
     </description>
   </property>
-
   <property>
     <name>hbase.status.published</name>
     <value>false</value>
@@ -1447,7 +1462,6 @@ possible configurations would overwhelm and obscure the important.
       Multicast port to use for the status publication by multicast.
     </description>
   </property>
-
   <property>
     <name>hbase.dynamic.jars.dir</name>
     <value>${hbase.rootdir}/lib</value>
@@ -1487,6 +1501,13 @@ possible configurations would overwhelm and obscure the important.
     </description>
   </property>
   <property>
+    <name>hbase.master.loadbalance.bytable</name>
+    <value>false</value>
+    <description>Factor Table name when the balancer runs.
+      Default: false.
+    </description>
+  </property>
+  <property>
     <name>hbase.master.normalizer.class</name>
     <value>org.apache.hadoop.hbase.master.normalizer.SimpleRegionNormalizer</value>
     <description>
@@ -1604,8 +1625,8 @@ possible configurations would overwhelm and obscure the important.
     <name>hbase.security.visibility.mutations.checkauths</name>
     <value>false</value>
     <description>
-      This property if enabled, will check whether the labels in the visibility expression are associated
-      with the user issuing the mutation
+      This property if enabled, will check whether the labels in the visibility
+      expression are associated with the user issuing the mutation
     </description>
   </property>
   <property>
@@ -1633,9 +1654,9 @@ possible configurations would overwhelm and obscure the important.
     <description>
         The maximum number of threads any replication source will use for
         shipping edits to the sinks in parallel. This also limits the number of
-        chunks each replication batch is broken into.
-        Larger values can improve the replication throughput between the master and
-        slave clusters. The default of 10 will rarely need to be changed.
+        chunks each replication batch is broken into. Larger values can improve
+        the replication throughput between the master and slave clusters. The
+        default of 10 will rarely need to be changed.
     </description>
   </property>
   <property>
@@ -1644,22 +1665,22 @@ possible configurations would overwhelm and obscure the important.
     <description>
       By default, in replication we can not make sure the order of operations in slave cluster is
       same as the order in master. If set REPLICATION_SCOPE to 2, we will push edits by the order
-      of written. This configure is to set how long (in ms) we will wait before next checking if a
-      log can not push right now because there are some logs written before it have not been pushed.
-      A larger waiting will decrease the number of queries on hbase:meta but will enlarge the delay
-      of replication. This feature relies on zk-less assignment, so users must set
+      of written. This configuration is to set how long (in ms) we will wait before next checking if
+      a log can NOT be pushed because there are some logs written before it that have yet to be
+      pushed. A larger waiting will decrease the number of queries on hbase:meta but will enlarge
+      the delay of replication. This feature relies on zk-less assignment, so users must set
       hbase.assignment.usezk to false to support it.
     </description>
   </property>
   <!-- Static Web User Filter properties. -->
   <property>
+    <name>hbase.http.staticuser.user</name>
+    <value>dr.stack</value>
     <description>
       The user name to filter as, on static web filters
       while rendering content. An example use is the HDFS
       web UI (user to be used for browsing files).
     </description>
-    <name>hbase.http.staticuser.user</name>
-    <value>dr.stack</value>
   </property>
   <property>
     <name>hbase.regionserver.handler.abort.on.error.percent</name>
@@ -1703,10 +1724,10 @@ possible configurations would overwhelm and obscure the important.
     <value>86400</value>
     <description>
       The period that ExpiredMobFileCleanerChore runs. The unit is second.
-      The default value is one day.
-      The MOB file name uses only the date part of the file creation time in it. We use this
-      time for deciding TTL expiry of the files. So the removal of TTL expired files might be
-      delayed. The max delay might be 24 hrs.
+      The default value is one day. The MOB file name uses only the date part of
+      the file creation time in it. We use this time for deciding TTL expiry of
+      the files. So the removal of TTL expired files might be delayed. The max
+      delay might be 24 hrs.
     </description>
   </property>
   <property>
@@ -1764,14 +1785,14 @@ possible configurations would overwhelm and obscure the important.
     <name>hbase.snapshot.master.timeout.millis</name>
     <value>300000</value>
     <description>
-       Timeout for master for the snapshot procedure execution
+       Timeout for master for the snapshot procedure execution.
     </description>
    </property>
      <property>
     <name>hbase.snapshot.region.timeout</name>
     <value>300000</value>
     <description>
-       Timeout for regionservers to keep threads in snapshot request pool waiting
+       Timeout for regionservers to keep threads in snapshot request pool waiting.
     </description>
    </property>
    <property>

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d6b9286/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index e31db82..d0f435b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -1399,7 +1399,7 @@ public class HMaster extends HRegionServer implements MasterServices {
         }
       }
 
-      boolean isByTable = getConfiguration().getBoolean("hbase.master.loadbalance.bytable", false);
+      boolean isByTable = getConfiguration().getBoolean("hbase.master.loadbalance.bytable", true);
       Map<TableName, Map<ServerName, List<RegionInfo>>> assignmentsByTable =
         this.assignmentManager.getRegionStates().getAssignmentsByTable(!isByTable);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d6b9286/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
index 809d6f3..fd78604 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
@@ -609,8 +609,11 @@ public class SplitTableRegionProcedure
     // Shutdown the pool
     threadPool.shutdown();
 
-    // Wait for all the tasks to finish
-    long fileSplitTimeout = conf.getLong("hbase.master.fileSplitTimeout", 30000);
+    // Wait for all the tasks to finish.
+    // When splits ran on the RegionServer, how-long-to-wait-configuration was named
+    // hbase.regionserver.fileSplitTimeout. If set, use its value.
+    long fileSplitTimeout = conf.getLong("hbase.master.fileSplitTimeout",
+        conf.getLong("hbase.regionserver.fileSplitTimeout", 600000));
     try {
       boolean stillRunning = !threadPool.awaitTermination(fileSplitTimeout, TimeUnit.MILLISECONDS);
       if (stillRunning) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d6b9286/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
index baa7590..d0fe49b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
@@ -402,12 +402,18 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
     }
     this.coprocessorHost = new WALCoprocessorHost(this, conf);
 
-    // Get size to roll log at. Roll at 95% of HDFS block size so we avoid crossing HDFS blocks
-    // (it costs a little x'ing bocks)
+    // Schedule a WAL roll when the WAL is 50% of the HDFS block size. Scheduling at 50% of block
+    // size should make it so WAL rolls before we get to the end-of-block (Block transitions cost
+    // some latency). In hbase-1 we did this differently. We scheduled a roll when we hit 95% of
+    // the block size but experience from the field has it that this was not enough time for the
+    // roll to happen before end-of-block. So the new accounting makes WALs of about the same
+    // size as those made in hbase-1 (to prevent surprise), we now have default block size as
+    // 2 times the DFS default: i.e. 2 * DFS default block size rolling at 50% full will generally
+    // make similar size logs to 1 * DFS default block size rolling at 95% full. See HBASE-19148.
     final long blocksize = this.conf.getLong("hbase.regionserver.hlog.blocksize",
-      CommonFSUtils.getDefaultBlockSize(this.fs, this.walDir));
+      CommonFSUtils.getDefaultBlockSize(this.fs, this.walDir) * 2);
     this.logrollsize =
-      (long) (blocksize * conf.getFloat("hbase.regionserver.logroll.multiplier", 0.95f));
+      (long) (blocksize * conf.getFloat("hbase.regionserver.logroll.multiplier", 0.5f));
 
     boolean maxLogsDefined = conf.get("hbase.regionserver.maxlogs") != null;
     if (maxLogsDefined) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/4d6b9286/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerCommandLine.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerCommandLine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerCommandLine.java
index 83ec5ff..f99a090 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerCommandLine.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerCommandLine.java
@@ -26,14 +26,16 @@ import java.util.Locale;
 import java.util.Map.Entry;
 import java.util.Set;
 
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Base class for command lines that start up various HBase daemons.
@@ -84,6 +86,23 @@ public abstract class ServerCommandLine extends Configured implements Tool {
   }
 
   /**
+   * Print into log some of the important hbase attributes.
+   */
+  private static void logHBaseConfigs(Configuration conf) {
+    final String [] keys = new String [] {
+      // Expand this list as you see fit.
+      "hbase.tmp.dir",
+      HConstants.HBASE_DIR,
+      HConstants.CLUSTER_DISTRIBUTED,
+      HConstants.ZOOKEEPER_QUORUM,
+
+    };
+    for (String key: keys) {
+      LOG.info(key + ": " + conf.get(key));
+    }
+  }
+
+  /**
    * Logs information about the currently running JVM process including
    * the environment variables. Logging of env vars can be disabled by
    * setting {@code "hbase.envvars.logging.disabled"} to {@code "true"}.
@@ -92,6 +111,8 @@ public abstract class ServerCommandLine extends Configured implements Tool {
    * to comma separated list of such substrings.
    */
   public static void logProcessInfo(Configuration conf) {
+    logHBaseConfigs(conf);
+
     // log environment variables unless asked not to
     if (conf == null || !conf.getBoolean("hbase.envvars.logging.disabled", false)) {
       Set<String> skipWords = new HashSet<>(DEFAULT_SKIP_WORDS);
@@ -114,6 +135,7 @@ public abstract class ServerCommandLine extends Configured implements Tool {
         LOG.info("env:"+entry);
       }
     }
+
     // and JVM info
     logJVMInfo();
   }


[07/26] hbase git commit: HBASE-10092 Addendum. Move to slf4j. Few changes in bin/ scripts.

Posted by zh...@apache.org.
HBASE-10092 Addendum. Move to slf4j. Few changes in bin/ scripts.


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

Branch: refs/heads/HBASE-19397
Commit: 5f02bf5b7b471e093a546ea494b39216570105e7
Parents: a4272a9
Author: Apekshit Sharma <ap...@apache.org>
Authored: Thu Dec 21 19:20:29 2017 -0800
Committer: Apekshit Sharma <ap...@apache.org>
Committed: Thu Dec 21 19:22:00 2017 -0800

----------------------------------------------------------------------
 bin/draining_servers.rb             | 22 ++--------------------
 bin/replication/copy_tables_desc.rb |  4 ++--
 2 files changed, 4 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5f02bf5b/bin/draining_servers.rb
----------------------------------------------------------------------
diff --git a/bin/draining_servers.rb b/bin/draining_servers.rb
index 588bac4..27cc0fa 100644
--- a/bin/draining_servers.rb
+++ b/bin/draining_servers.rb
@@ -27,8 +27,7 @@ java_import org.apache.hadoop.hbase.HBaseConfiguration
 java_import org.apache.hadoop.hbase.client.ConnectionFactory
 java_import org.apache.hadoop.hbase.client.HBaseAdmin
 java_import org.apache.hadoop.hbase.zookeeper.ZKUtil
-java_import org.apache.commons.logging.Log
-java_import org.apache.commons.logging.LogFactory
+java_import org.slf4j.LoggerFactory
 
 # Name of this script
 NAME = 'draining_servers'.freeze
@@ -43,10 +42,6 @@ optparse = OptionParser.new do |opts|
     puts opts
     exit
   end
-  options[:debug] = false
-  opts.on('-d', '--debug', 'Display extra debug logging') do
-    options[:debug] = true
-  end
 end
 optparse.parse!
 
@@ -133,21 +128,8 @@ end
 
 hostOrServers = ARGV[1..ARGV.size]
 
-# Create a logger and disable the DEBUG-level annoying client logging
-def configureLogging(options)
-  apacheLogger = LogFactory.getLog(NAME)
-  # Configure log4j to not spew so much
-  unless options[:debug]
-    logger = org.apache.log4j.Logger.getLogger('org.apache.hadoop.hbase')
-    logger.setLevel(org.apache.log4j.Level::WARN)
-    logger = org.apache.log4j.Logger.getLogger('org.apache.zookeeper')
-    logger.setLevel(org.apache.log4j.Level::WARN)
-  end
-  apacheLogger
-end
-
 # Create a logger and save it to ruby global
-$LOG = configureLogging(options)
+$LOG = LoggerFactory.getLogger(NAME)
 case ARGV[0]
 when 'add'
   if ARGV.length < 2

http://git-wip-us.apache.org/repos/asf/hbase/blob/5f02bf5b/bin/replication/copy_tables_desc.rb
----------------------------------------------------------------------
diff --git a/bin/replication/copy_tables_desc.rb b/bin/replication/copy_tables_desc.rb
index c494765..44a24f9 100644
--- a/bin/replication/copy_tables_desc.rb
+++ b/bin/replication/copy_tables_desc.rb
@@ -24,7 +24,6 @@
 #
 
 include Java
-java_import org.apache.commons.logging.LogFactory
 java_import org.apache.hadoop.conf.Configuration
 java_import org.apache.hadoop.hbase.HBaseConfiguration
 java_import org.apache.hadoop.hbase.HConstants
@@ -32,6 +31,7 @@ java_import org.apache.hadoop.hbase.HTableDescriptor
 java_import org.apache.hadoop.hbase.TableName
 java_import org.apache.hadoop.hbase.client.ConnectionFactory
 java_import org.apache.hadoop.hbase.client.HBaseAdmin
+java_import org.slf4j.LoggerFactory
 
 # Name of this script
 NAME = 'copy_tables_desc'.freeze
@@ -64,7 +64,7 @@ end
 
 usage if ARGV.size < 2 || ARGV.size > 3
 
-LOG = LogFactory.getLog(NAME)
+LOG = LoggerFactory.getLogger(NAME)
 
 parts1 = ARGV[0].split(':')
 


[18/26] hbase git commit: HBASE-19216 Implement a general framework to execute remote procedure on RS

Posted by zh...@apache.org.
HBASE-19216 Implement a general framework to execute remote procedure on RS


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

Branch: refs/heads/HBASE-19397
Commit: ad6b8c69a8d4babecaffc6bf9ab10c102a452398
Parents: c0598dc
Author: zhangduo <zh...@apache.org>
Authored: Fri Dec 15 21:06:44 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Fri Dec 22 21:39:59 2017 +0800

----------------------------------------------------------------------
 .../hbase/procedure2/LockedResourceType.java    |   4 +-
 .../procedure2/RemoteProcedureDispatcher.java   |  23 +-
 .../src/main/protobuf/Admin.proto               |   9 +-
 .../src/main/protobuf/MasterProcedure.proto     |  30 +++
 .../src/main/protobuf/RegionServerStatus.proto  |  15 ++
 .../apache/hadoop/hbase/executor/EventType.java |  26 ++-
 .../hadoop/hbase/executor/ExecutorType.java     |   3 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |  33 ++-
 .../hadoop/hbase/master/MasterRpcServices.java  |  13 ++
 .../assignment/RegionTransitionProcedure.java   |  18 +-
 .../procedure/MasterProcedureScheduler.java     | 224 +++++++++++++------
 .../procedure/PeerProcedureInterface.java       |  34 +++
 .../master/procedure/RSProcedureDispatcher.java |  90 ++++----
 .../master/replication/ModifyPeerProcedure.java | 127 +++++++++++
 .../master/replication/RefreshPeerCallable.java |  67 ++++++
 .../replication/RefreshPeerProcedure.java       | 197 ++++++++++++++++
 .../hbase/procedure2/RSProcedureCallable.java   |  43 ++++
 .../hbase/regionserver/HRegionServer.java       |  69 +++++-
 .../hbase/regionserver/RSRpcServices.java       |  56 +++--
 .../handler/RSProcedureHandler.java             |  51 +++++
 .../assignment/TestAssignmentManager.java       |  20 +-
 .../replication/DummyModifyPeerProcedure.java   |  41 ++++
 .../TestDummyModifyPeerProcedure.java           |  80 +++++++
 .../security/access/TestAccessController.java   |   6 +-
 24 files changed, 1109 insertions(+), 170 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ad6b8c69/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResourceType.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResourceType.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResourceType.java
index c5fe62b..dc9b5d4 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResourceType.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResourceType.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -22,5 +22,5 @@ import org.apache.yetus.audience.InterfaceAudience;
 
 @InterfaceAudience.Private
 public enum LockedResourceType {
-  SERVER, NAMESPACE, TABLE, REGION
+  SERVER, NAMESPACE, TABLE, REGION, PEER
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad6b8c69/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
index 54f2b08..e9a6906 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
@@ -226,13 +226,30 @@ public abstract class RemoteProcedureDispatcher<TEnv, TRemote extends Comparable
 
   /**
    * Remote procedure reference.
-   * @param <TEnv>
-   * @param <TRemote>
    */
   public interface RemoteProcedure<TEnv, TRemote> {
+    /**
+     * For building the remote operation.
+     */
     RemoteOperation remoteCallBuild(TEnv env, TRemote remote);
-    void remoteCallCompleted(TEnv env, TRemote remote, RemoteOperation response);
+
+    /**
+     * Called when the executeProcedure call is failed.
+     */
     void remoteCallFailed(TEnv env, TRemote remote, IOException exception);
+
+    /**
+     * Called when RS tells the remote procedure is succeeded through the
+     * {@code reportProcedureDone} method.
+     */
+    void remoteOperationCompleted(TEnv env);
+
+    /**
+     * Called when RS tells the remote procedure is failed through the {@code reportProcedureDone}
+     * method.
+     * @param error the error message
+     */
+    void remoteOperationFailed(TEnv env, String error);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad6b8c69/hbase-protocol-shaded/src/main/protobuf/Admin.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Admin.proto b/hbase-protocol-shaded/src/main/protobuf/Admin.proto
index 118c79b..ddcc266 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Admin.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Admin.proto
@@ -256,14 +256,19 @@ message ClearRegionBlockCacheResponse {
   required CacheEvictionStats stats = 1;
 }
 
+message RemoteProcedureRequest {
+  required uint64 proc_id = 1;
+  required string proc_class = 2;
+  optional bytes proc_data = 3;
+}
+
 message ExecuteProceduresRequest {
   repeated OpenRegionRequest open_region = 1;
   repeated CloseRegionRequest close_region = 2;
+  repeated RemoteProcedureRequest proc = 3;
 }
 
 message ExecuteProceduresResponse {
-  repeated OpenRegionResponse open_region = 1;
-  repeated CloseRegionResponse close_region = 2;
 }
 
 service AdminService {

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad6b8c69/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
index f9b8807..0e2bdba 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -365,3 +365,33 @@ message GCMergedRegionsStateData {
   required RegionInfo parent_b = 2;
   required RegionInfo merged_child = 3;
 }
+
+enum PeerModificationState {
+  UPDATE_PEER_STORAGE = 1;
+  REFRESH_PEER_ON_RS = 2;
+  POST_PEER_MODIFICATION = 3;
+}
+
+message PeerModificationStateData {
+  required string peer_id = 1;
+}
+
+enum PeerModificationType {
+  ADD_PEER = 1;
+  REMOVE_PEER = 2;
+  ENABLE_PEER = 3;
+  DISABLE_PEER = 4;
+  UPDATE_PEER_CONFIG = 5;
+}
+
+message RefreshPeerStateData {
+  required string peer_id = 1;
+  required PeerModificationType type = 2;
+  required ServerName target_server = 3;
+}
+
+message RefreshPeerParameter {
+  required string peer_id = 1;
+  required PeerModificationType type = 2;
+  required ServerName target_server = 3;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad6b8c69/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto b/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
index f83bb20..eb396ac 100644
--- a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
@@ -143,7 +143,19 @@ message RegionSpaceUseReportRequest {
 }
 
 message RegionSpaceUseReportResponse {
+}
 
+message ReportProcedureDoneRequest {
+  required uint64 proc_id = 1;
+  enum Status {
+    SUCCESS = 1;
+    ERROR = 2;
+  }
+  required Status status = 2;
+  optional string error = 3;
+}
+
+message ReportProcedureDoneResponse {
 }
 
 service RegionServerStatusService {
@@ -181,4 +193,7 @@ service RegionServerStatusService {
    */
   rpc ReportRegionSpaceUse(RegionSpaceUseReportRequest)
     returns(RegionSpaceUseReportResponse);
+
+  rpc ReportProcedureDone(ReportProcedureDoneRequest)
+    returns(ReportProcedureDoneResponse);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad6b8c69/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java
index 26fb63a..922deb8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java
@@ -20,15 +20,14 @@ package org.apache.hadoop.hbase.executor;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
- * List of all HBase event handler types.  Event types are named by a
- * convention: event type names specify the component from which the event
- * originated and then where its destined -- e.g. RS2ZK_ prefix means the
- * event came from a regionserver destined for zookeeper -- and then what
- * the even is; e.g. REGION_OPENING.
- *
- * <p>We give the enums indices so we can add types later and keep them
- * grouped together rather than have to add them always to the end as we
- * would have to if we used raw enum ordinals.
+ * List of all HBase event handler types.
+ * <p>
+ * Event types are named by a convention: event type names specify the component from which the
+ * event originated and then where its destined -- e.g. RS_ZK_ prefix means the event came from a
+ * regionserver destined for zookeeper -- and then what the even is; e.g. REGION_OPENING.
+ * <p>
+ * We give the enums indices so we can add types later and keep them grouped together rather than
+ * have to add them always to the end as we would have to if we used raw enum ordinals.
  */
 @InterfaceAudience.Private
 public enum EventType {
@@ -275,7 +274,14 @@ public enum EventType {
    *
    * RS_COMPACTED_FILES_DISCHARGER
    */
-  RS_COMPACTED_FILES_DISCHARGER (83, ExecutorType.RS_COMPACTED_FILES_DISCHARGER);
+  RS_COMPACTED_FILES_DISCHARGER (83, ExecutorType.RS_COMPACTED_FILES_DISCHARGER),
+
+  /**
+   * RS refresh peer.<br>
+   *
+   * RS_REFRESH_PEER
+   */
+  RS_REFRESH_PEER (84, ExecutorType.RS_REFRESH_PEER);
 
   private final int code;
   private final ExecutorType executor;

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad6b8c69/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java
index c75a0a9..7f130d1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java
@@ -46,7 +46,8 @@ public enum ExecutorType {
   RS_LOG_REPLAY_OPS          (27),
   RS_REGION_REPLICA_FLUSH_OPS  (28),
   RS_COMPACTED_FILES_DISCHARGER (29),
-  RS_OPEN_PRIORITY_REGION    (30);
+  RS_OPEN_PRIORITY_REGION    (30),
+  RS_REFRESH_PEER               (31);
 
   ExecutorType(int value) {
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad6b8c69/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index cf95030..52b7b35 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * 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
@@ -140,6 +139,7 @@ import org.apache.hadoop.hbase.procedure2.LockedResource;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure;
 import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 import org.apache.hadoop.hbase.quotas.MasterSpaceQuotaObserver;
@@ -330,8 +330,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   // flag set after we become the active master (used for testing)
   private volatile boolean activeMaster = false;
 
-  // flag set after we complete initialization once active,
-  // it is not private since it's used in unit tests
+  // flag set after we complete initialization once active
   private final ProcedureEvent initialized = new ProcedureEvent("master initialized");
 
   // flag set after master services are started,
@@ -3530,4 +3529,28 @@ public class HMaster extends HRegionServer implements MasterServices {
   public SpaceQuotaSnapshotNotifier getSpaceQuotaSnapshotNotifier() {
     return this.spaceQuotaSnapshotNotifier;
   }
-}
+
+  @SuppressWarnings("unchecked")
+  private RemoteProcedure<MasterProcedureEnv, ?> getRemoteProcedure(long procId) {
+    Procedure<?> procedure = procedureExecutor.getProcedure(procId);
+    if (procedure == null) {
+      return null;
+    }
+    assert procedure instanceof RemoteProcedure;
+    return (RemoteProcedure<MasterProcedureEnv, ?>) procedure;
+  }
+
+  public void remoteProcedureCompleted(long procId) {
+    RemoteProcedure<MasterProcedureEnv, ?> procedure = getRemoteProcedure(procId);
+    if (procedure != null) {
+      procedure.remoteOperationCompleted(procedureExecutor.getEnvironment());
+    }
+  }
+
+  public void remoteProcedureFailed(long procId, String error) {
+    RemoteProcedure<MasterProcedureEnv, ?> procedure = getRemoteProcedure(procId);
+    if (procedure != null) {
+      procedure.remoteOperationFailed(procedureExecutor.getEnvironment(), error);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad6b8c69/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index f0f2e10..f666a39 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -263,6 +263,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportProcedureDoneRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportProcedureDoneResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
@@ -2253,4 +2255,15 @@ public class MasterRpcServices extends RSRpcServices
     }
     return response.build();
   }
+
+  @Override
+  public ReportProcedureDoneResponse reportProcedureDone(RpcController controller,
+      ReportProcedureDoneRequest request) throws ServiceException {
+    if (request.getStatus() == ReportProcedureDoneRequest.Status.SUCCESS) {
+      master.remoteProcedureCompleted(request.getProcId());
+    } else {
+      master.remoteProcedureFailed(request.getProcId(), request.getError());
+    }
+    return ReportProcedureDoneResponse.getDefaultInstance();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad6b8c69/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
index 6e671c8..43e75ed 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
@@ -173,12 +173,6 @@ public abstract class RegionTransitionProcedure
       RegionStateNode regionNode, IOException exception);
 
   @Override
-  public void remoteCallCompleted(final MasterProcedureEnv env,
-      final ServerName serverName, final RemoteOperation response) {
-    // Ignore the response? reportTransition() is the one that count?
-  }
-
-  @Override
   public void remoteCallFailed(final MasterProcedureEnv env,
       final ServerName serverName, final IOException exception) {
     final RegionStateNode regionNode = getRegionState(env);
@@ -413,4 +407,16 @@ public abstract class RegionTransitionProcedure
    * @return ServerName the Assign or Unassign is going against.
    */
   public abstract ServerName getServer(final MasterProcedureEnv env);
+
+  @Override
+  public void remoteOperationCompleted(MasterProcedureEnv env) {
+    // should not be called for region operation until we modified the open/close region procedure
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void remoteOperationFailed(MasterProcedureEnv env, String error) {
+    // should not be called for region operation until we modified the open/close region procedure
+    throw new UnsupportedOperationException();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad6b8c69/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
index c60de5c..dd85f5c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
@@ -24,7 +24,7 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
+import java.util.function.Function;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ServerName;
@@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.master.locking.LockProcedure;
+import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface.PeerOperationType;
 import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface.TableOperationType;
 import org.apache.hadoop.hbase.procedure2.AbstractProcedureScheduler;
 import org.apache.hadoop.hbase.procedure2.LockAndQueue;
@@ -109,12 +110,17 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       new ServerQueueKeyComparator();
   private final static TableQueueKeyComparator TABLE_QUEUE_KEY_COMPARATOR =
       new TableQueueKeyComparator();
+  private final static PeerQueueKeyComparator PEER_QUEUE_KEY_COMPARATOR =
+      new PeerQueueKeyComparator();
 
   private final FairQueue<ServerName> serverRunQueue = new FairQueue<>();
   private final FairQueue<TableName> tableRunQueue = new FairQueue<>();
+  private final FairQueue<String> peerRunQueue = new FairQueue<>();
 
   private final ServerQueue[] serverBuckets = new ServerQueue[128];
   private TableQueue tableMap = null;
+  private PeerQueue peerMap = null;
+
   private final SchemaLocking locking = new SchemaLocking();
 
   /**
@@ -161,6 +167,8 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       doAdd(tableRunQueue, getTableQueue(getTableName(proc)), proc, addFront);
     } else if (isServerProcedure(proc)) {
       doAdd(serverRunQueue, getServerQueue(getServerName(proc)), proc, addFront);
+    } else if (isPeerProcedure(proc)) {
+      doAdd(peerRunQueue, getPeerQueue(getPeerId(proc)), proc, addFront);
     } else {
       // TODO: at the moment we only have Table and Server procedures
       // if you are implementing a non-table/non-server procedure, you have two options: create
@@ -172,7 +180,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
   }
 
   private <T extends Comparable<T>> void doAdd(final FairQueue<T> fairq,
-      final Queue<T> queue, final Procedure proc, final boolean addFront) {
+      final Queue<T> queue, final Procedure<?> proc, final boolean addFront) {
     queue.add(proc, addFront);
     if (!queue.getLockStatus().hasExclusiveLock() || queue.getLockStatus().isLockOwner(proc.getProcId())) {
       // if the queue was not remove for an xlock execution
@@ -189,7 +197,8 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
 
   @Override
   protected boolean queueHasRunnables() {
-    return tableRunQueue.hasRunnables() || serverRunQueue.hasRunnables();
+    return tableRunQueue.hasRunnables() || serverRunQueue.hasRunnables() ||
+        peerRunQueue.hasRunnables();
   }
 
   @Override
@@ -197,7 +206,10 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
     // For now, let server handling have precedence over table handling; presumption is that it
     // is more important handling crashed servers than it is running the
     // enabling/disabling tables, etc.
-    Procedure pollResult = doPoll(serverRunQueue);
+    Procedure<?> pollResult = doPoll(serverRunQueue);
+    if (pollResult == null) {
+      pollResult = doPoll(peerRunQueue);
+    }
     if (pollResult == null) {
       pollResult = doPoll(tableRunQueue);
     }
@@ -267,60 +279,30 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
         exclusiveLockOwnerProcedure, sharedLockCount, waitingProcedures);
   }
 
+  private <T> void addToLockedResources(List<LockedResource> lockedResources,
+      Map<T, LockAndQueue> locks, Function<T, String> keyTransformer,
+      LockedResourceType resourcesType) {
+    locks.entrySet().stream().filter(e -> e.getValue().isLocked())
+        .map(
+          e -> createLockedResource(resourcesType, keyTransformer.apply(e.getKey()), e.getValue()))
+        .forEachOrdered(lockedResources::add);
+  }
+
   @Override
   public List<LockedResource> getLocks() {
     schedLock();
-
     try {
       List<LockedResource> lockedResources = new ArrayList<>();
-
-      for (Entry<ServerName, LockAndQueue> entry : locking.serverLocks
-          .entrySet()) {
-        String serverName = entry.getKey().getServerName();
-        LockAndQueue queue = entry.getValue();
-
-        if (queue.isLocked()) {
-          LockedResource lockedResource =
-            createLockedResource(LockedResourceType.SERVER, serverName, queue);
-          lockedResources.add(lockedResource);
-        }
-      }
-
-      for (Entry<String, LockAndQueue> entry : locking.namespaceLocks
-          .entrySet()) {
-        String namespaceName = entry.getKey();
-        LockAndQueue queue = entry.getValue();
-
-        if (queue.isLocked()) {
-          LockedResource lockedResource =
-            createLockedResource(LockedResourceType.NAMESPACE, namespaceName, queue);
-          lockedResources.add(lockedResource);
-        }
-      }
-
-      for (Entry<TableName, LockAndQueue> entry : locking.tableLocks
-          .entrySet()) {
-        String tableName = entry.getKey().getNameAsString();
-        LockAndQueue queue = entry.getValue();
-
-        if (queue.isLocked()) {
-          LockedResource lockedResource =
-            createLockedResource(LockedResourceType.TABLE, tableName, queue);
-          lockedResources.add(lockedResource);
-        }
-      }
-
-      for (Entry<String, LockAndQueue> entry : locking.regionLocks.entrySet()) {
-        String regionName = entry.getKey();
-        LockAndQueue queue = entry.getValue();
-
-        if (queue.isLocked()) {
-          LockedResource lockedResource =
-            createLockedResource(LockedResourceType.REGION, regionName, queue);
-          lockedResources.add(lockedResource);
-        }
-      }
-
+      addToLockedResources(lockedResources, locking.serverLocks, sn -> sn.getServerName(),
+        LockedResourceType.SERVER);
+      addToLockedResources(lockedResources, locking.namespaceLocks, Function.identity(),
+        LockedResourceType.NAMESPACE);
+      addToLockedResources(lockedResources, locking.tableLocks, tn -> tn.getNameAsString(),
+        LockedResourceType.TABLE);
+      addToLockedResources(lockedResources, locking.regionLocks, Function.identity(),
+        LockedResourceType.REGION);
+      addToLockedResources(lockedResources, locking.peerLocks, Function.identity(),
+        LockedResourceType.PEER);
       return lockedResources;
     } finally {
       schedUnlock();
@@ -328,8 +310,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
   }
 
   @Override
-  public LockedResource getLockResource(LockedResourceType resourceType,
-      String resourceName) {
+  public LockedResource getLockResource(LockedResourceType resourceType, String resourceName) {
     LockAndQueue queue = null;
     schedLock();
     try {
@@ -346,8 +327,10 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
         case REGION:
           queue = locking.regionLocks.get(resourceName);
           break;
+        case PEER:
+          queue = locking.peerLocks.get(resourceName);
+          break;
       }
-
       return queue != null ? createLockedResource(resourceType, resourceName, queue) : null;
     } finally {
       schedUnlock();
@@ -431,6 +414,11 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
         markTableAsDeleted(iProcTable.getTableName(), proc);
         return;
       }
+    } else if (proc instanceof PeerProcedureInterface) {
+      PeerProcedureInterface iProcPeer = (PeerProcedureInterface) proc;
+      if (iProcPeer.getPeerOperationType() == PeerOperationType.REMOVE) {
+        removePeerQueue(iProcPeer.getPeerId());
+      }
     } else {
       // No cleanup for ServerProcedureInterface types, yet.
       return;
@@ -468,12 +456,11 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
     locking.removeTableLock(tableName);
   }
 
-
-  private static boolean isTableProcedure(Procedure proc) {
+  private static boolean isTableProcedure(Procedure<?> proc) {
     return proc instanceof TableProcedureInterface;
   }
 
-  private static TableName getTableName(Procedure proc) {
+  private static TableName getTableName(Procedure<?> proc) {
     return ((TableProcedureInterface)proc).getTableName();
   }
 
@@ -494,15 +481,42 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
     return Math.abs(hashCode) % buckets.length;
   }
 
-  private static boolean isServerProcedure(Procedure proc) {
+  private static boolean isServerProcedure(Procedure<?> proc) {
     return proc instanceof ServerProcedureInterface;
   }
 
-  private static ServerName getServerName(Procedure proc) {
+  private static ServerName getServerName(Procedure<?> proc) {
     return ((ServerProcedureInterface)proc).getServerName();
   }
 
   // ============================================================================
+  //  Peer Queue Lookup Helpers
+  // ============================================================================
+  private PeerQueue getPeerQueue(String peerId) {
+    PeerQueue node = AvlTree.get(peerMap, peerId, PEER_QUEUE_KEY_COMPARATOR);
+    if (node != null) {
+      return node;
+    }
+    node = new PeerQueue(peerId, locking.getPeerLock(peerId));
+    peerMap = AvlTree.insert(peerMap, node);
+    return node;
+  }
+
+  private void removePeerQueue(String peerId) {
+    peerMap = AvlTree.remove(peerMap, peerId, PEER_QUEUE_KEY_COMPARATOR);
+    locking.removePeerLock(peerId);
+  }
+
+
+  private static boolean isPeerProcedure(Procedure<?> proc) {
+    return proc instanceof PeerProcedureInterface;
+  }
+
+  private static String getPeerId(Procedure<?> proc) {
+    return ((PeerProcedureInterface) proc).getPeerId();
+  }
+
+  // ============================================================================
   //  Table and Server Queue Implementation
   // ============================================================================
   private static class ServerQueueKeyComparator implements AvlKeyComparator<ServerQueue> {
@@ -571,6 +585,26 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
     }
   }
 
+  private static class PeerQueueKeyComparator implements AvlKeyComparator<PeerQueue> {
+
+    @Override
+    public int compareKey(PeerQueue node, Object key) {
+      return node.compareKey((String) key);
+    }
+  }
+
+  public static class PeerQueue extends Queue<String> {
+
+    public PeerQueue(String peerId, LockStatus lockStatus) {
+      super(peerId, lockStatus);
+    }
+
+    @Override
+    public boolean requireExclusiveLock(Procedure proc) {
+      return requirePeerExclusiveLock((PeerProcedureInterface) proc);
+    }
+  }
+
   // ============================================================================
   //  Table Locking Helpers
   // ============================================================================
@@ -958,7 +992,8 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
    * @param serverName Server to lock
    * @return true if the procedure has to wait for the server to be available
    */
-  public boolean waitServerExclusiveLock(final Procedure procedure, final ServerName serverName) {
+  public boolean waitServerExclusiveLock(final Procedure<?> procedure,
+      final ServerName serverName) {
     schedLock();
     try {
       final LockAndQueue lock = locking.getServerLock(serverName);
@@ -980,7 +1015,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
    * @param procedure the procedure releasing the lock
    * @param serverName the server that has the exclusive lock
    */
-  public void wakeServerExclusiveLock(final Procedure procedure, final ServerName serverName) {
+  public void wakeServerExclusiveLock(final Procedure<?> procedure, final ServerName serverName) {
     schedLock();
     try {
       final LockAndQueue lock = locking.getServerLock(serverName);
@@ -994,6 +1029,56 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
   }
 
   // ============================================================================
+  //  Peer Locking Helpers
+  // ============================================================================
+
+  private static boolean requirePeerExclusiveLock(PeerProcedureInterface proc) {
+    return proc.getPeerOperationType() != PeerOperationType.REFRESH;
+  }
+
+  /**
+   * Try to acquire the exclusive lock on the specified peer.
+   * @see #wakePeerExclusiveLock(Procedure, String)
+   * @param procedure the procedure trying to acquire the lock
+   * @param peerId peer to lock
+   * @return true if the procedure has to wait for the per to be available
+   */
+  public boolean waitPeerExclusiveLock(Procedure<?> procedure, String peerId) {
+    schedLock();
+    try {
+      final LockAndQueue lock = locking.getPeerLock(peerId);
+      if (lock.tryExclusiveLock(procedure)) {
+        removeFromRunQueue(peerRunQueue, getPeerQueue(peerId));
+        return false;
+      }
+      waitProcedure(lock, procedure);
+      logLockedResource(LockedResourceType.PEER, peerId);
+      return true;
+    } finally {
+      schedUnlock();
+    }
+  }
+
+  /**
+   * Wake the procedures waiting for the specified peer
+   * @see #waitPeerExclusiveLock(Procedure, String)
+   * @param procedure the procedure releasing the lock
+   * @param peerId the peer that has the exclusive lock
+   */
+  public void wakePeerExclusiveLock(Procedure<?> procedure, String peerId) {
+    schedLock();
+    try {
+      final LockAndQueue lock = locking.getPeerLock(peerId);
+      lock.releaseExclusiveLock(procedure);
+      addToRunQueue(peerRunQueue, getPeerQueue(peerId));
+      int waitingCount = wakeWaitingProcedures(lock);
+      wakePollIfNeeded(waitingCount);
+    } finally {
+      schedUnlock();
+    }
+  }
+
+  // ============================================================================
   //  Generic Helpers
   // ============================================================================
   private static abstract class Queue<TKey extends Comparable<TKey>>
@@ -1098,6 +1183,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
     final Map<TableName, LockAndQueue> tableLocks = new HashMap<>();
     // Single map for all regions irrespective of tables. Key is encoded region name.
     final Map<String, LockAndQueue> regionLocks = new HashMap<>();
+    final Map<String, LockAndQueue> peerLocks = new HashMap<>();
 
     private <T> LockAndQueue getLock(Map<T, LockAndQueue> map, T key) {
       LockAndQueue lock = map.get(key);
@@ -1132,6 +1218,14 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       return getLock(serverLocks, serverName);
     }
 
+    LockAndQueue getPeerLock(String peerId) {
+      return getLock(peerLocks, peerId);
+    }
+
+    LockAndQueue removePeerLock(String peerId) {
+      return peerLocks.remove(peerId);
+    }
+
     /**
      * Removes all locks by clearing the maps.
      * Used when procedure executor is stopped for failure and recovery testing.
@@ -1142,6 +1236,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       namespaceLocks.clear();
       tableLocks.clear();
       regionLocks.clear();
+      peerLocks.clear();
     }
 
     @Override
@@ -1149,7 +1244,8 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       return "serverLocks=" + filterUnlocked(this.serverLocks) +
         ", namespaceLocks=" + filterUnlocked(this.namespaceLocks) +
         ", tableLocks=" + filterUnlocked(this.tableLocks) +
-        ", regionLocks=" + filterUnlocked(this.regionLocks);
+        ", regionLocks=" + filterUnlocked(this.regionLocks) +
+        ", peerLocks=" + filterUnlocked(this.peerLocks);
     }
 
     private String filterUnlocked(Map<?, LockAndQueue> locks) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad6b8c69/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerProcedureInterface.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerProcedureInterface.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerProcedureInterface.java
new file mode 100644
index 0000000..4abc9ad
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerProcedureInterface.java
@@ -0,0 +1,34 @@
+/**
+ * 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.master.procedure;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface PeerProcedureInterface {
+
+  enum PeerOperationType {
+    ADD, REMOVE, ENABLE, DISABLE, UPDATE_CONFIG, REFRESH
+  }
+
+  String getPeerId();
+
+  PeerOperationType getPeerOperationType();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad6b8c69/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
index 72e0846..22b70f1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.ipc.RemoteException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.ArrayListMultimap;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
@@ -49,6 +50,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProc
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RemoteProcedureRequest;
 
 /**
  * A remote procecdure dispatcher for regionservers.
@@ -222,7 +224,10 @@ public class RSProcedureDispatcher
 
   private interface RemoteProcedureResolver {
     void dispatchOpenRequests(MasterProcedureEnv env, List<RegionOpenOperation> operations);
+
     void dispatchCloseRequests(MasterProcedureEnv env, List<RegionCloseOperation> operations);
+
+    void dispatchServerOperations(MasterProcedureEnv env, List<ServerOperation> operations);
   }
 
   /**
@@ -231,22 +236,28 @@ public class RSProcedureDispatcher
    * Then {@code resolver} is used to dispatch {@link RegionOpenOperation}s and
    * {@link RegionCloseOperation}s.
    * @param serverName RegionServer to which the remote operations are sent
-   * @param remoteProcedures Remote procedures which are dispatched to the given server
+   * @param operations Remote procedures which are dispatched to the given server
    * @param resolver Used to dispatch remote procedures to given server.
    */
-  public void splitAndResolveOperation(final ServerName serverName,
-      final Set<RemoteProcedure> remoteProcedures, final RemoteProcedureResolver resolver) {
-    final ArrayListMultimap<Class<?>, RemoteOperation> reqsByType =
-      buildAndGroupRequestByType(procedureEnv, serverName, remoteProcedures);
+  public void splitAndResolveOperation(ServerName serverName, Set<RemoteProcedure> operations,
+      RemoteProcedureResolver resolver) {
+    MasterProcedureEnv env = master.getMasterProcedureExecutor().getEnvironment();
+    ArrayListMultimap<Class<?>, RemoteOperation> reqsByType =
+      buildAndGroupRequestByType(env, serverName, operations);
 
-    final List<RegionOpenOperation> openOps = fetchType(reqsByType, RegionOpenOperation.class);
+    List<RegionOpenOperation> openOps = fetchType(reqsByType, RegionOpenOperation.class);
     if (!openOps.isEmpty()) {
-      resolver.dispatchOpenRequests(procedureEnv, openOps);
+      resolver.dispatchOpenRequests(env, openOps);
     }
 
-    final List<RegionCloseOperation> closeOps = fetchType(reqsByType, RegionCloseOperation.class);
+    List<RegionCloseOperation> closeOps = fetchType(reqsByType, RegionCloseOperation.class);
     if (!closeOps.isEmpty()) {
-      resolver.dispatchCloseRequests(procedureEnv, closeOps);
+      resolver.dispatchCloseRequests(env, closeOps);
+    }
+
+    List<ServerOperation> refreshOps = fetchType(reqsByType, ServerOperation.class);
+    if (!refreshOps.isEmpty()) {
+      resolver.dispatchServerOperations(env, refreshOps);
     }
 
     if (!reqsByType.isEmpty()) {
@@ -277,8 +288,7 @@ public class RSProcedureDispatcher
       splitAndResolveOperation(getServerName(), remoteProcedures, this);
 
       try {
-        final ExecuteProceduresResponse response = sendRequest(getServerName(), request.build());
-        remoteCallCompleted(procedureEnv, response);
+        sendRequest(getServerName(), request.build());
       } catch (IOException e) {
         e = unwrapException(e);
         // TODO: In the future some operation may want to bail out early.
@@ -302,6 +312,11 @@ public class RSProcedureDispatcher
       }
     }
 
+    @Override
+    public void dispatchServerOperations(MasterProcedureEnv env, List<ServerOperation> operations) {
+      operations.stream().map(o -> o.buildRequest()).forEachOrdered(request::addProc);
+    }
+
     protected ExecuteProceduresResponse sendRequest(final ServerName serverName,
         final ExecuteProceduresRequest request) throws IOException {
       try {
@@ -311,17 +326,8 @@ public class RSProcedureDispatcher
       }
     }
 
-
-    private void remoteCallCompleted(final MasterProcedureEnv env,
-        final ExecuteProceduresResponse response) {
-      /*
-      for (RemoteProcedure proc: operations) {
-        proc.remoteCallCompleted(env, getServerName(), response);
-      }*/
-    }
-
     private void remoteCallFailed(final MasterProcedureEnv env, final IOException e) {
-      for (RemoteProcedure proc: remoteProcedures) {
+      for (RemoteProcedure proc : remoteProcedures) {
         proc.remoteCallFailed(env, getServerName(), e);
       }
     }
@@ -362,8 +368,7 @@ public class RSProcedureDispatcher
           buildOpenRegionRequest(procedureEnv, getServerName(), operations);
 
       try {
-        OpenRegionResponse response = sendRequest(getServerName(), request);
-        remoteCallCompleted(procedureEnv, response);
+        sendRequest(getServerName(), request);
       } catch (IOException e) {
         e = unwrapException(e);
         // TODO: In the future some operation may want to bail out early.
@@ -384,16 +389,6 @@ public class RSProcedureDispatcher
       }
     }
 
-    private void remoteCallCompleted(final MasterProcedureEnv env,
-        final OpenRegionResponse response) {
-      int index = 0;
-      for (RegionOpenOperation op: operations) {
-        OpenRegionResponse.RegionOpeningState state = response.getOpeningState(index++);
-        op.setFailedOpen(state == OpenRegionResponse.RegionOpeningState.FAILED_OPENING);
-        op.getRemoteProcedure().remoteCallCompleted(env, getServerName(), op);
-      }
-    }
-
     private void remoteCallFailed(final MasterProcedureEnv env, final IOException e) {
       for (RegionOpenOperation op: operations) {
         op.getRemoteProcedure().remoteCallFailed(env, getServerName(), e);
@@ -443,7 +438,6 @@ public class RSProcedureDispatcher
     private void remoteCallCompleted(final MasterProcedureEnv env,
         final CloseRegionResponse response) {
       operation.setClosed(response.getClosed());
-      operation.getRemoteProcedure().remoteCallCompleted(env, getServerName(), operation);
     }
 
     private void remoteCallFailed(final MasterProcedureEnv env, final IOException e) {
@@ -482,6 +476,11 @@ public class RSProcedureDispatcher
         submitTask(new CloseRegionRemoteCall(serverName, op));
       }
     }
+
+    @Override
+    public void dispatchServerOperations(MasterProcedureEnv env, List<ServerOperation> operations) {
+      throw new UnsupportedOperationException();
+    }
   }
 
   // ==========================================================================
@@ -489,13 +488,28 @@ public class RSProcedureDispatcher
   //  - ServerOperation: refreshConfig, grant, revoke, ... (TODO)
   //  - RegionOperation: open, close, flush, snapshot, ...
   // ==========================================================================
-  /* Currently unused
-  public static abstract class ServerOperation extends RemoteOperation {
-    protected ServerOperation(final RemoteProcedure remoteProcedure) {
+
+  public static final class ServerOperation extends RemoteOperation {
+
+    private final long procId;
+
+    private final Class<?> rsProcClass;
+
+    private final byte[] rsProcData;
+
+    public ServerOperation(RemoteProcedure remoteProcedure, long procId, Class<?> rsProcClass,
+        byte[] rsProcData) {
       super(remoteProcedure);
+      this.procId = procId;
+      this.rsProcClass = rsProcClass;
+      this.rsProcData = rsProcData;
+    }
+
+    public RemoteProcedureRequest buildRequest() {
+      return RemoteProcedureRequest.newBuilder().setProcId(procId)
+          .setProcClass(rsProcClass.getName()).setProcData(ByteString.copyFrom(rsProcData)).build();
     }
   }
-  */
 
   public static abstract class RegionOperation extends RemoteOperation {
     private final RegionInfo regionInfo;

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad6b8c69/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
new file mode 100644
index 0000000..fca05a7
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
@@ -0,0 +1,127 @@
+/**
+ * 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.master.replication;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationState;
+
+@InterfaceAudience.Private
+public abstract class ModifyPeerProcedure
+    extends StateMachineProcedure<MasterProcedureEnv, PeerModificationState>
+    implements PeerProcedureInterface {
+
+  private static final Log LOG = LogFactory.getLog(ModifyPeerProcedure.class);
+
+  protected String peerId;
+
+  protected ModifyPeerProcedure() {
+  }
+
+  protected ModifyPeerProcedure(String peerId) {
+    this.peerId = peerId;
+  }
+
+  @Override
+  public String getPeerId() {
+    return peerId;
+  }
+
+  /**
+   * Return {@code false} means that the operation is invalid and we should give up, otherwise
+   * {@code true}.
+   * <p>
+   * You need to call {@link #setFailure(String, Throwable)} to give the detail failure information.
+   */
+  protected abstract boolean updatePeerStorage() throws IOException;
+
+  protected void postPeerModification() {
+  }
+
+  @Override
+  protected Flow executeFromState(MasterProcedureEnv env, PeerModificationState state)
+      throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
+    switch (state) {
+      case UPDATE_PEER_STORAGE:
+        try {
+          if (!updatePeerStorage()) {
+            assert isFailed() : "setFailure is not called";
+            return Flow.NO_MORE_STATE;
+          }
+        } catch (IOException e) {
+          LOG.warn("update peer storage failed, retry", e);
+          throw new ProcedureYieldException();
+        }
+        setNextState(PeerModificationState.REFRESH_PEER_ON_RS);
+        return Flow.HAS_MORE_STATE;
+      case REFRESH_PEER_ON_RS:
+        addChildProcedure(env.getMasterServices().getServerManager().getOnlineServersList().stream()
+            .map(sn -> new RefreshPeerProcedure(peerId, getPeerOperationType(), sn))
+            .toArray(RefreshPeerProcedure[]::new));
+        setNextState(PeerModificationState.POST_PEER_MODIFICATION);
+        return Flow.HAS_MORE_STATE;
+      case POST_PEER_MODIFICATION:
+        postPeerModification();
+        return Flow.NO_MORE_STATE;
+      default:
+        throw new UnsupportedOperationException("unhandled state=" + state);
+    }
+  }
+
+  @Override
+  protected LockState acquireLock(MasterProcedureEnv env) {
+    return env.getProcedureScheduler().waitPeerExclusiveLock(this, peerId)
+      ? LockState.LOCK_EVENT_WAIT
+      : LockState.LOCK_ACQUIRED;
+  }
+
+  @Override
+  protected void releaseLock(MasterProcedureEnv env) {
+    env.getProcedureScheduler().wakePeerExclusiveLock(this, peerId);
+  }
+
+  @Override
+  protected void rollbackState(MasterProcedureEnv env, PeerModificationState state)
+      throws IOException, InterruptedException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  protected PeerModificationState getState(int stateId) {
+    return PeerModificationState.forNumber(stateId);
+  }
+
+  @Override
+  protected int getStateId(PeerModificationState state) {
+    return state.getNumber();
+  }
+
+  @Override
+  protected PeerModificationState getInitialState() {
+    return PeerModificationState.UPDATE_PEER_STORAGE;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad6b8c69/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerCallable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerCallable.java
new file mode 100644
index 0000000..4e09107
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerCallable.java
@@ -0,0 +1,67 @@
+/**
+ * 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.master.replication;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.executor.EventType;
+import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RefreshPeerParameter;
+
+/**
+ * The callable executed at RS side to refresh the peer config/state.
+ * <p>
+ * TODO: only a dummy implementation for verifying the framework, will add implementation later.
+ */
+@InterfaceAudience.Private
+public class RefreshPeerCallable implements RSProcedureCallable {
+
+  private HRegionServer rs;
+
+  private String peerId;
+
+  private Exception initError;
+
+  @Override
+  public Void call() throws Exception {
+    if (initError != null) {
+      throw initError;
+    }
+    rs.getFileSystem().create(new Path("/" + peerId + "/" + rs.getServerName().toString())).close();
+    return null;
+  }
+
+  @Override
+  public void init(byte[] parameter, HRegionServer rs) {
+    this.rs = rs;
+    try {
+      this.peerId = RefreshPeerParameter.parseFrom(parameter).getPeerId();
+    } catch (InvalidProtocolBufferException e) {
+      initError = e;
+      return;
+    }
+  }
+
+  @Override
+  public EventType getEventType() {
+    return EventType.RS_REFRESH_PEER;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad6b8c69/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
new file mode 100644
index 0000000..18da487
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
@@ -0,0 +1,197 @@
+/**
+ * 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.master.replication;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface;
+import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher.ServerOperation;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationType;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RefreshPeerParameter;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RefreshPeerStateData;
+
+@InterfaceAudience.Private
+public class RefreshPeerProcedure extends Procedure<MasterProcedureEnv>
+    implements PeerProcedureInterface, RemoteProcedure<MasterProcedureEnv, ServerName> {
+
+  private static final Log LOG = LogFactory.getLog(RefreshPeerProcedure.class);
+
+  private String peerId;
+
+  private PeerOperationType type;
+
+  private ServerName targetServer;
+
+  private boolean dispatched;
+
+  private ProcedureEvent<?> event;
+
+  private boolean succ;
+
+  public RefreshPeerProcedure() {
+  }
+
+  public RefreshPeerProcedure(String peerId, PeerOperationType type, ServerName targetServer) {
+    this.peerId = peerId;
+    this.type = type;
+    this.targetServer = targetServer;
+  }
+
+  @Override
+  public String getPeerId() {
+    return peerId;
+  }
+
+  @Override
+  public PeerOperationType getPeerOperationType() {
+    return PeerOperationType.REFRESH;
+  }
+
+  private static PeerModificationType toPeerModificationType(PeerOperationType type) {
+    switch (type) {
+      case ADD:
+        return PeerModificationType.ADD_PEER;
+      case REMOVE:
+        return PeerModificationType.REMOVE_PEER;
+      case ENABLE:
+        return PeerModificationType.ENABLE_PEER;
+      case DISABLE:
+        return PeerModificationType.DISABLE_PEER;
+      case UPDATE_CONFIG:
+        return PeerModificationType.UPDATE_PEER_CONFIG;
+      default:
+        throw new IllegalArgumentException("Unknown type: " + type);
+    }
+  }
+
+  private static PeerOperationType toPeerOperationType(PeerModificationType type) {
+    switch (type) {
+      case ADD_PEER:
+        return PeerOperationType.ADD;
+      case REMOVE_PEER:
+        return PeerOperationType.REMOVE;
+      case ENABLE_PEER:
+        return PeerOperationType.ENABLE;
+      case DISABLE_PEER:
+        return PeerOperationType.DISABLE;
+      case UPDATE_PEER_CONFIG:
+        return PeerOperationType.UPDATE_CONFIG;
+      default:
+        throw new IllegalArgumentException("Unknown type: " + type);
+    }
+  }
+
+  @Override
+  public RemoteOperation remoteCallBuild(MasterProcedureEnv env, ServerName remote) {
+    assert targetServer.equals(remote);
+    return new ServerOperation(this, getProcId(), RefreshPeerCallable.class,
+        RefreshPeerParameter.newBuilder().setPeerId(peerId).setType(toPeerModificationType(type))
+            .setTargetServer(ProtobufUtil.toServerName(remote)).build().toByteArray());
+  }
+
+  private void complete(MasterProcedureEnv env, boolean succ) {
+    if (event == null) {
+      LOG.warn("procedure event for " + getProcId() +
+          " is null, maybe the procedure is created when recovery", new Exception());
+      return;
+    }
+    LOG.info("Refresh peer " + peerId + " for " + type + " on " + targetServer +
+        (succ ? " suceeded" : " failed"));
+    this.succ = succ;
+    event.wake(env.getProcedureScheduler());
+    event = null;
+  }
+
+  @Override
+  public synchronized void remoteCallFailed(MasterProcedureEnv env, ServerName remote,
+      IOException exception) {
+    complete(env, false);
+  }
+
+  @Override
+  public synchronized void remoteOperationCompleted(MasterProcedureEnv env) {
+    complete(env, true);
+  }
+
+  @Override
+  public synchronized void remoteOperationFailed(MasterProcedureEnv env, String error) {
+    complete(env, false);
+  }
+
+  @Override
+  protected synchronized Procedure<MasterProcedureEnv>[] execute(MasterProcedureEnv env)
+      throws ProcedureYieldException, ProcedureSuspendedException, InterruptedException {
+    if (dispatched) {
+      if (succ) {
+        return null;
+      }
+      // retry
+      dispatched = false;
+    }
+    if (!env.getRemoteDispatcher().addOperationToNode(targetServer, this)) {
+      LOG.info("Can not add remote operation for refreshing peer " + peerId + " for " + type +
+          " to " + targetServer + ", this usually because the server is already dead," +
+          " give up and mark the procedure as complete");
+      return null;
+    }
+    dispatched = true;
+    event = new ProcedureEvent<>(this);
+    event.suspendIfNotReady(this);
+    throw new ProcedureSuspendedException();
+  }
+
+  @Override
+  protected void rollback(MasterProcedureEnv env) throws IOException, InterruptedException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  protected boolean abort(MasterProcedureEnv env) {
+    // TODO: no correctness problem if we just ignore this, implement later.
+    return false;
+  }
+
+  @Override
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    serializer.serialize(
+      RefreshPeerStateData.newBuilder().setPeerId(peerId).setType(toPeerModificationType(type))
+          .setTargetServer(ProtobufUtil.toServerName(targetServer)).build());
+  }
+
+  @Override
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    RefreshPeerStateData data = serializer.deserialize(RefreshPeerStateData.class);
+    peerId = data.getPeerId();
+    type = toPeerOperationType(data.getType());
+    targetServer = ProtobufUtil.toServerName(data.getTargetServer());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad6b8c69/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/RSProcedureCallable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/RSProcedureCallable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/RSProcedureCallable.java
new file mode 100644
index 0000000..62c2e36
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/RSProcedureCallable.java
@@ -0,0 +1,43 @@
+/**
+ * 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.procedure2;
+
+import java.util.concurrent.Callable;
+
+import org.apache.hadoop.hbase.executor.EventType;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A general interface for a sub procedure runs at RS side.
+ */
+@InterfaceAudience.Private
+public interface RSProcedureCallable extends Callable<Void> {
+
+  /**
+   * Initialize the callable
+   * @param parameter the parameter passed from master.
+   * @param rs the regionserver instance
+   */
+  void init(byte[] parameter, HRegionServer rs);
+
+  /**
+   * Event type used to select thread pool.
+   */
+  EventType getEventType();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad6b8c69/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 8ead08c..0799ca6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -17,9 +17,6 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
-import javax.servlet.http.HttpServlet;
 import java.io.IOException;
 import java.lang.Thread.UncaughtExceptionHandler;
 import java.lang.management.MemoryType;
@@ -51,6 +48,10 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.function.Function;
 
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import javax.servlet.http.HttpServlet;
+
 import org.apache.commons.lang3.RandomUtils;
 import org.apache.commons.lang3.SystemUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -117,6 +118,7 @@ import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.master.RegionState.State;
 import org.apache.hadoop.hbase.mob.MobCacheConfig;
 import org.apache.hadoop.hbase.procedure.RegionServerProcedureManagerHost;
+import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
 import org.apache.hadoop.hbase.quotas.FileSystemUtilizationChore;
 import org.apache.hadoop.hbase.quotas.QuotaUtil;
 import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
@@ -127,6 +129,7 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequester;
 import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler;
 import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler;
+import org.apache.hadoop.hbase.regionserver.handler.RSProcedureHandler;
 import org.apache.hadoop.hbase.regionserver.handler.RegionReplicaFlushHandler;
 import org.apache.hadoop.hbase.regionserver.throttle.FlushThroughputControllerFactory;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
@@ -175,6 +178,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
@@ -206,6 +210,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportProcedureDoneRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
@@ -1932,6 +1937,8 @@ public class HRegionServer extends HasThread implements
           conf.getInt("hbase.regionserver.region.replica.flusher.threads",
               conf.getInt("hbase.regionserver.executor.openregion.threads", 3)));
     }
+    this.executorService.startExecutorService(ExecutorType.RS_REFRESH_PEER,
+      conf.getInt("hbase.regionserver.executor.refresh.peer.threads", 2));
 
     Threads.setDaemonThreadRunning(this.walRoller.getThread(), getName() + ".logRoller",
     uncaughtExceptionHandler);
@@ -3725,4 +3732,60 @@ public class HRegionServer extends HasThread implements
     return ConnectionUtils.createShortCircuitConnection(conf, null, user, this.serverName,
         this.rpcServices, this.rpcServices);
   }
+
+  public void executeProcedure(long procId, RSProcedureCallable callable) {
+    executorService.submit(new RSProcedureHandler(this, procId, callable));
+  }
+
+  public void reportProcedureDone(long procId, Throwable error) {
+    ReportProcedureDoneRequest.Builder builder =
+      ReportProcedureDoneRequest.newBuilder().setProcId(procId);
+    if (error != null) {
+      builder.setStatus(ReportProcedureDoneRequest.Status.ERROR)
+          .setError(Throwables.getStackTraceAsString(error));
+    } else {
+      builder.setStatus(ReportProcedureDoneRequest.Status.SUCCESS);
+    }
+    ReportProcedureDoneRequest request = builder.build();
+    int tries = 0;
+    long pauseTime = INIT_PAUSE_TIME_MS;
+    while (keepLooping()) {
+      RegionServerStatusService.BlockingInterface rss = rssStub;
+      try {
+        if (rss == null) {
+          createRegionServerStatusStub();
+          continue;
+        }
+        rss.reportProcedureDone(null, request);
+        // Log if we had to retry else don't log unless TRACE. We want to
+        // know if were successful after an attempt showed in logs as failed.
+        if (tries > 0 || LOG.isTraceEnabled()) {
+          LOG.info("PROCEDURE REPORTED " + request);
+        }
+        return;
+      } catch (ServiceException se) {
+        IOException ioe = ProtobufUtil.getRemoteException(se);
+        boolean pause =
+          ioe instanceof ServerNotRunningYetException || ioe instanceof PleaseHoldException;
+        if (pause) {
+          // Do backoff else we flood the Master with requests.
+          pauseTime = ConnectionUtils.getPauseTime(INIT_PAUSE_TIME_MS, tries);
+        } else {
+          pauseTime = INIT_PAUSE_TIME_MS; // Reset.
+        }
+        LOG.info(
+          "Failed to report transition " + TextFormat.shortDebugString(request) + "; retry (#" +
+            tries + ")" + (pause ? " after " + pauseTime + "ms delay (Master is coming online...)."
+              : " immediately."),
+          ioe);
+        if (pause) {
+          Threads.sleep(pauseTime);
+        }
+        tries++;
+        if (rssStub == rss) {
+          rssStub = null;
+        }
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad6b8c69/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 584d0a2..2ad8fc8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * 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
@@ -99,6 +98,7 @@ import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
 import org.apache.hadoop.hbase.log.HBaseMarkers;
 import org.apache.hadoop.hbase.master.MasterRpcServices;
+import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
 import org.apache.hadoop.hbase.quotas.ActivePolicyEnforcement;
 import org.apache.hadoop.hbase.quotas.OperationQuota;
 import org.apache.hadoop.hbase.quotas.QuotaUtil;
@@ -171,6 +171,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionR
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse.RegionOpeningState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RemoteProcedureRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest;
@@ -3434,23 +3435,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
   }
 
   @Override
-  public ExecuteProceduresResponse executeProcedures(RpcController controller,
-       ExecuteProceduresRequest request) throws ServiceException {
-    ExecuteProceduresResponse.Builder builder = ExecuteProceduresResponse.newBuilder();
-    if (request.getOpenRegionCount() > 0) {
-      for (OpenRegionRequest req : request.getOpenRegionList()) {
-        builder.addOpenRegion(openRegion(controller, req));
-      }
-    }
-    if (request.getCloseRegionCount() > 0) {
-      for (CloseRegionRequest req : request.getCloseRegionList()) {
-        builder.addCloseRegion(closeRegion(controller, req));
-      }
-    }
-    return builder.build();
-  }
-
-  @Override
   public ClearRegionBlockCacheResponse clearRegionBlockCache(RpcController controller,
       ClearRegionBlockCacheRequest request) {
     ClearRegionBlockCacheResponse.Builder builder =
@@ -3467,4 +3451,38 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     stats.withMaxCacheSize(regionServer.getCacheConfig().getBlockCache().getMaxSize());
     return builder.setStats(ProtobufUtil.toCacheEvictionStats(stats.build())).build();
   }
+
+  @Override
+  public ExecuteProceduresResponse executeProcedures(RpcController controller,
+      ExecuteProceduresRequest request) throws ServiceException {
+    if (request.getOpenRegionCount() > 0) {
+      for (OpenRegionRequest req : request.getOpenRegionList()) {
+        openRegion(controller, req);
+      }
+    }
+    if (request.getCloseRegionCount() > 0) {
+      for (CloseRegionRequest req : request.getCloseRegionList()) {
+        closeRegion(controller, req);
+      }
+    }
+    if (request.getProcCount() > 0) {
+      for (RemoteProcedureRequest req : request.getProcList()) {
+        RSProcedureCallable callable;
+        try {
+          callable =
+            Class.forName(req.getProcClass()).asSubclass(RSProcedureCallable.class).newInstance();
+        } catch (Exception e) {
+          // here we just ignore the error as this should not happen and we do not provide a general
+          // way to report errors for all types of remote procedure. The procedure will hang at
+          // master side but after you solve the problem and restart master it will be executed
+          // again and pass.
+          LOG.warn("create procedure of type " + req.getProcClass() + " failed, give up", e);
+          continue;
+        }
+        callable.init(req.getProcData().toByteArray(), regionServer);
+        regionServer.executeProcedure(req.getProcId(), callable);
+      }
+    }
+    return ExecuteProceduresResponse.getDefaultInstance();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad6b8c69/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java
new file mode 100644
index 0000000..94bcfec
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java
@@ -0,0 +1,51 @@
+/**
+ * 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.regionserver.handler;
+
+import org.apache.hadoop.hbase.executor.EventHandler;
+import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A event handler for running procedure.
+ */
+@InterfaceAudience.Private
+public class RSProcedureHandler extends EventHandler {
+
+  private final long procId;
+
+  private final RSProcedureCallable callable;
+
+  public RSProcedureHandler(HRegionServer rs, long procId, RSProcedureCallable callable) {
+    super(rs, callable.getEventType());
+    this.procId = procId;
+    this.callable = callable;
+  }
+
+  @Override
+  public void process() {
+    Exception error = null;
+    try {
+      callable.call();
+    } catch (Exception e) {
+      error = e;
+    }
+    ((HRegionServer) server).reportProcedureDone(procId, error);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad6b8c69/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
index 3c453bc..3ab915b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
@@ -531,26 +531,16 @@ public class TestAssignmentManager {
     @Override
     public ExecuteProceduresResponse sendRequest(ServerName server,
         ExecuteProceduresRequest request) throws IOException {
-      ExecuteProceduresResponse.Builder builder = ExecuteProceduresResponse.newBuilder();
       if (request.getOpenRegionCount() > 0) {
-        for (OpenRegionRequest req: request.getOpenRegionList()) {
-          OpenRegionResponse.Builder resp = OpenRegionResponse.newBuilder();
-          for (RegionOpenInfo openReq: req.getOpenInfoList()) {
-            RegionOpeningState state = execOpenRegion(server, openReq);
-            if (state != null) {
-              resp.addOpeningState(state);
-            }
+        for (OpenRegionRequest req : request.getOpenRegionList()) {
+          for (RegionOpenInfo openReq : req.getOpenInfoList()) {
+            execOpenRegion(server, openReq);
           }
-          builder.addOpenRegion(resp.build());
         }
       }
       if (request.getCloseRegionCount() > 0) {
-        for (CloseRegionRequest req: request.getCloseRegionList()) {
-          CloseRegionResponse resp = execCloseRegion(server,
-              req.getRegion().getValue().toByteArray());
-          if (resp != null) {
-            builder.addCloseRegion(resp);
-          }
+        for (CloseRegionRequest req : request.getCloseRegionList()) {
+          execCloseRegion(server, req.getRegion().getValue().toByteArray());
         }
       }
       return ExecuteProceduresResponse.newBuilder().build();

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad6b8c69/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
new file mode 100644
index 0000000..44343d7
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
@@ -0,0 +1,41 @@
+/**
+ * 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.master.replication;
+
+import java.io.IOException;
+
+public class DummyModifyPeerProcedure extends ModifyPeerProcedure {
+
+  public DummyModifyPeerProcedure() {
+  }
+
+  public DummyModifyPeerProcedure(String peerId) {
+    super(peerId);
+  }
+
+  @Override
+  public PeerOperationType getPeerOperationType() {
+    return PeerOperationType.ADD;
+  }
+
+  @Override
+  protected boolean updatePeerStorage() throws IOException {
+    return true;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad6b8c69/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestDummyModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestDummyModifyPeerProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestDummyModifyPeerProcedure.java
new file mode 100644
index 0000000..ec06306
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestDummyModifyPeerProcedure.java
@@ -0,0 +1,80 @@
+/**
+ * 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.master.replication;
+
+import static org.junit.Assert.assertTrue;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MasterTests.class, LargeTests.class })
+public class TestDummyModifyPeerProcedure {
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static String PEER_ID;
+
+  private static Path DIR;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    UTIL.startMiniCluster(3);
+    PEER_ID = "testPeer";
+    DIR = new Path("/" + PEER_ID);
+    UTIL.getTestFileSystem().mkdirs(DIR);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void test() throws Exception {
+    ProcedureExecutor<?> executor =
+        UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor();
+    long procId = executor.submitProcedure(new DummyModifyPeerProcedure(PEER_ID));
+    UTIL.waitFor(30000, new Waiter.Predicate<Exception>() {
+
+      @Override
+      public boolean evaluate() throws Exception {
+        return executor.isFinished(procId);
+      }
+    });
+    Set<String> serverNames = UTIL.getHBaseCluster().getRegionServerThreads().stream()
+        .map(t -> t.getRegionServer().getServerName().toString())
+        .collect(Collectors.toCollection(HashSet::new));
+    for (FileStatus s : UTIL.getTestFileSystem().listStatus(DIR)) {
+      assertTrue(serverNames.remove(s.getPath().getName()));
+    }
+    assertTrue(serverNames.isEmpty());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ad6b8c69/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index 138a40e..cade419 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -30,6 +30,7 @@ import com.google.protobuf.RpcCallback;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.Service;
 import com.google.protobuf.ServiceException;
+
 import java.io.IOException;
 import java.security.PrivilegedAction;
 import java.util.ArrayList;
@@ -119,8 +120,6 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.Permission.Action;
-import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.SecurityTests;
 import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
@@ -136,6 +135,9 @@ import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
+
 /**
  * Performs authorization checks for common operations, according to different
  * levels of authorized users.


[21/26] hbase git commit: HBASE-19564 Procedure id is missing in the response of peer related operations

Posted by zh...@apache.org.
HBASE-19564 Procedure id is missing in the response of peer related operations


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

Branch: refs/heads/HBASE-19397
Commit: 377d776b894e4dc2eccca46f03171d8dbbf11838
Parents: f2077c9
Author: zhangduo <zh...@apache.org>
Authored: Wed Dec 20 20:57:37 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Fri Dec 22 21:41:28 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/master/MasterRpcServices.java  | 24 ++++++++++----------
 .../master/replication/ModifyPeerProcedure.java |  4 +---
 2 files changed, 13 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/377d776b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index f214d83..4757709 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -1892,10 +1892,10 @@ public class MasterRpcServices extends RSRpcServices
   public AddReplicationPeerResponse addReplicationPeer(RpcController controller,
       AddReplicationPeerRequest request) throws ServiceException {
     try {
-      master.addReplicationPeer(request.getPeerId(),
-        ReplicationPeerConfigUtil.convert(request.getPeerConfig()), request.getPeerState()
-            .getState().equals(ReplicationState.State.ENABLED));
-      return AddReplicationPeerResponse.newBuilder().build();
+      long procId = master.addReplicationPeer(request.getPeerId(),
+        ReplicationPeerConfigUtil.convert(request.getPeerConfig()),
+        request.getPeerState().getState().equals(ReplicationState.State.ENABLED));
+      return AddReplicationPeerResponse.newBuilder().setProcId(procId).build();
     } catch (ReplicationException | IOException e) {
       throw new ServiceException(e);
     }
@@ -1905,8 +1905,8 @@ public class MasterRpcServices extends RSRpcServices
   public RemoveReplicationPeerResponse removeReplicationPeer(RpcController controller,
       RemoveReplicationPeerRequest request) throws ServiceException {
     try {
-      master.removeReplicationPeer(request.getPeerId());
-      return RemoveReplicationPeerResponse.newBuilder().build();
+      long procId = master.removeReplicationPeer(request.getPeerId());
+      return RemoveReplicationPeerResponse.newBuilder().setProcId(procId).build();
     } catch (ReplicationException | IOException e) {
       throw new ServiceException(e);
     }
@@ -1916,8 +1916,8 @@ public class MasterRpcServices extends RSRpcServices
   public EnableReplicationPeerResponse enableReplicationPeer(RpcController controller,
       EnableReplicationPeerRequest request) throws ServiceException {
     try {
-      master.enableReplicationPeer(request.getPeerId());
-      return EnableReplicationPeerResponse.newBuilder().build();
+      long procId = master.enableReplicationPeer(request.getPeerId());
+      return EnableReplicationPeerResponse.newBuilder().setProcId(procId).build();
     } catch (ReplicationException | IOException e) {
       throw new ServiceException(e);
     }
@@ -1927,8 +1927,8 @@ public class MasterRpcServices extends RSRpcServices
   public DisableReplicationPeerResponse disableReplicationPeer(RpcController controller,
       DisableReplicationPeerRequest request) throws ServiceException {
     try {
-      master.disableReplicationPeer(request.getPeerId());
-      return DisableReplicationPeerResponse.newBuilder().build();
+      long procId = master.disableReplicationPeer(request.getPeerId());
+      return DisableReplicationPeerResponse.newBuilder().setProcId(procId).build();
     } catch (ReplicationException | IOException e) {
       throw new ServiceException(e);
     }
@@ -1954,9 +1954,9 @@ public class MasterRpcServices extends RSRpcServices
   public UpdateReplicationPeerConfigResponse updateReplicationPeerConfig(RpcController controller,
       UpdateReplicationPeerConfigRequest request) throws ServiceException {
     try {
-      master.updateReplicationPeerConfig(request.getPeerId(),
+      long procId = master.updateReplicationPeerConfig(request.getPeerId(),
         ReplicationPeerConfigUtil.convert(request.getPeerConfig()));
-      return UpdateReplicationPeerConfigResponse.newBuilder().build();
+      return UpdateReplicationPeerConfigResponse.newBuilder().setProcId(procId).build();
     } catch (ReplicationException | IOException e) {
       throw new ServiceException(e);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/377d776b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
index 7076bab..23f6f87 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
@@ -54,9 +54,7 @@ public abstract class ModifyPeerProcedure
 
   protected ModifyPeerProcedure(String peerId) {
     this.peerId = peerId;
-    // TODO: temporarily set a 4.0 here to always wait for the procedure exection completed. Change
-    // to 3.0 or 2.0 after the client modification is done.
-    this.latch = ProcedurePrepareLatch.createLatch(4, 0);
+    this.latch = ProcedurePrepareLatch.createLatch(2, 0);
   }
 
   public ProcedurePrepareLatch getLatch() {


[09/26] hbase git commit: HBASE-15124 Document the new 'normalization' feature in refguid

Posted by zh...@apache.org.
HBASE-15124 Document the new 'normalization' feature in refguid


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

Branch: refs/heads/HBASE-19397
Commit: 070c2ee7dd371157c1c242fe28b91f76ba47c5c5
Parents: 37bf54a
Author: Michael Stack <st...@apache.org>
Authored: Thu Dec 21 22:11:12 2017 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Thu Dec 21 22:47:25 2017 -0800

----------------------------------------------------------------------
 src/main/asciidoc/_chapters/ops_mgt.adoc | 136 ++++++++++++++++++++++++++
 1 file changed, 136 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/070c2ee7/src/main/asciidoc/_chapters/ops_mgt.adoc
----------------------------------------------------------------------
diff --git a/src/main/asciidoc/_chapters/ops_mgt.adoc b/src/main/asciidoc/_chapters/ops_mgt.adoc
index 7b0f89b..f5fa456 100644
--- a/src/main/asciidoc/_chapters/ops_mgt.adoc
+++ b/src/main/asciidoc/_chapters/ops_mgt.adoc
@@ -2704,4 +2704,140 @@ Viewing the Master log will give you insight on rsgroup operation.
 If it appears stuck, restart the Master process.
 
 
+[[normalizer]]
+== Region Normalizer
+
+The Region Normalizer tries to make Regions all in a table about the same in size.
+It does this by finding a rough average. Any region that is larger than twice this
+size is split. Any region that is much smaller is merged into an adjacent region.
+It is good to run the Normalizer on occasion on a down time after the cluster has
+been running a while or say after a burst of activity such as a large delete.
+
+(The bulk of the below detail was copied wholesale from the blog by Romil Choksi at
+link:https://community.hortonworks.com/articles/54987/hbase-region-normalizer.html[HBase Region Normalizer])
+
+The Region Normalizer is feature available since HBase-1.2. It runs a set of
+pre-calculated merge/split actions to resize regions that are either too
+large or too small compared to the average region size for a given table. Region
+Normalizer when invoked computes a normalization 'plan' for all of the tables in
+HBase. System tables (such as hbase:meta, hbase:namespace, Phoenix system tables
+etc) and user tables with normalization disabled are ignored while computing the
+plan. For normalization enabled tables, normalization plan is carried out in
+parallel across multiple tables.
+
+Normalizer can be enabled or disabled globally for the entire cluster using the
+‘normalizer_switch’ command in the HBase shell. Normalization can also be
+controlled on a per table basis, which is disabled by default when a table is
+created. Normalization for a table can be enabled or disabled by setting the
+NORMALIZATION_ENABLED table attribute to true or false.
+
+To check normalizer status and enable/disable normalizer
 
+[source,bash]
+----
+hbase(main):001:0> normalizer_enabled
+true 
+0 row(s) in 0.4870 seconds
+ 
+hbase(main):002:0> normalizer_switch false
+true 
+0 row(s) in 0.0640 seconds
+ 
+hbase(main):003:0> normalizer_enabled
+false 
+0 row(s) in 0.0120 seconds
+ 
+hbase(main):004:0> normalizer_switch true
+false
+0 row(s) in 0.0200 seconds
+ 
+hbase(main):005:0> normalizer_enabled
+true
+0 row(s) in 0.0090 seconds
+----
+
+When enabled, Normalizer is invoked in the background every 5 mins (by default),
+which can be configured using `hbase.normalization.period` in `hbase-site.xml`.
+Normalizer can also be invoked manually/programmatically at will using HBase shell’s
+`normalize` command. HBase by default uses `SimpleRegionNormalizer`, but users can
+design their own normalizer as long as they implement the RegionNormalizer Interface.
+Details about the logic used by `SimpleRegionNormalizer` to compute its normalization
+plan can be found link:https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.html[here].
+
+The below example shows a normalization plan being computed for an user table, and
+merge action being taken as a result of the normalization plan computed by SimpleRegionNormalizer.
+
+Consider an user table with some pre-split regions having 3 equally large regions
+(about 100K rows) and 1 relatively small region (about 25K rows). Following is the
+snippet from an hbase meta table scan showing each of the pre-split regions for 
+the user table.
+
+----
+table_p8ddpd6q5z,,1469494305548.68b9892220865cb6048 column=info:regioninfo, timestamp=1469494306375, value={ENCODED => 68b9892220865cb604809c950d1adf48, NAME => 'table_p8ddpd6q5z,,1469494305548.68b989222 09c950d1adf48.   0865cb604809c950d1adf48.', STARTKEY => '', ENDKEY => '1'} 
+.... 
+table_p8ddpd6q5z,1,1469494317178.867b77333bdc75a028 column=info:regioninfo, timestamp=1469494317848, value={ENCODED => 867b77333bdc75a028bb4c5e4b235f48, NAME => 'table_p8ddpd6q5z,1,1469494317178.867b7733 bb4c5e4b235f48.  3bdc75a028bb4c5e4b235f48.', STARTKEY => '1', ENDKEY => '3'} 
+.... 
+table_p8ddpd6q5z,3,1469494328323.98f019a753425e7977 column=info:regioninfo, timestamp=1469494328486, value={ENCODED => 98f019a753425e7977ab8636e32deeeb, NAME => 'table_p8ddpd6q5z,3,1469494328323.98f019a7 ab8636e32deeeb.  53425e7977ab8636e32deeeb.', STARTKEY => '3', ENDKEY => '7'} 
+.... 
+table_p8ddpd6q5z,7,1469494339662.94c64e748979ecbb16 column=info:regioninfo, timestamp=1469494339859, value={ENCODED => 94c64e748979ecbb166f6cc6550e25c6, NAME => 'table_p8ddpd6q5z,7,1469494339662.94c64e74 6f6cc6550e25c6.   8979ecbb166f6cc6550e25c6.', STARTKEY => '7', ENDKEY => '8'} 
+.... 
+table_p8ddpd6q5z,8,1469494339662.6d2b3f5fd1595ab8e7 column=info:regioninfo, timestamp=1469494339859, value={ENCODED => 6d2b3f5fd1595ab8e7c031876057b1ee, NAME => 'table_p8ddpd6q5z,8,1469494339662.6d2b3f5f c031876057b1ee.   d1595ab8e7c031876057b1ee.', STARTKEY => '8', ENDKEY => ''}  
+----
+Invoking the normalizer using ‘normalize’ int the HBase shell, the below log snippet
+from HMaster log shows the normalization plan computed as per the logic defined for
+SimpleRegionNormalizer. Since the total region size (in MB) for the adjacent smallest
+regions in the table is less than the average region size, the normalizer computes a
+plan to merge these two regions.
+
+----
+2016-07-26 07:08:26,928 DEBUG [B.fifo.QRpcServer.handler=20,queue=2,port=20000] master.HMaster: Skipping normalization for table: hbase:namespace, as it's either system table or doesn't have auto
+normalization turned on
+2016-07-26 07:08:26,928 DEBUG [B.fifo.QRpcServer.handler=20,queue=2,port=20000] master.HMaster: Skipping normalization for table: hbase:backup, as it's either system table or doesn't have auto normalization turned on
+2016-07-26 07:08:26,928 DEBUG [B.fifo.QRpcServer.handler=20,queue=2,port=20000] master.HMaster: Skipping normalization for table: hbase:meta, as it's either system table or doesn't have auto normalization turned on
+2016-07-26 07:08:26,928 DEBUG [B.fifo.QRpcServer.handler=20,queue=2,port=20000] master.HMaster: Skipping normalization for table: table_h2osxu3wat, as it's either system table or doesn't have autonormalization turned on
+2016-07-26 07:08:26,928 DEBUG [B.fifo.QRpcServer.handler=20,queue=2,port=20000] normalizer.SimpleRegionNormalizer: Computing normalization plan for table: table_p8ddpd6q5z, number of regions: 5
+2016-07-26 07:08:26,929 DEBUG [B.fifo.QRpcServer.handler=20,queue=2,port=20000] normalizer.SimpleRegionNormalizer: Table table_p8ddpd6q5z, total aggregated regions size: 12
+2016-07-26 07:08:26,929 DEBUG [B.fifo.QRpcServer.handler=20,queue=2,port=20000] normalizer.SimpleRegionNormalizer: Table table_p8ddpd6q5z, average region size: 2.4
+2016-07-26 07:08:26,929 INFO  [B.fifo.QRpcServer.handler=20,queue=2,port=20000] normalizer.SimpleRegionNormalizer: Table table_p8ddpd6q5z, small region size: 0 plus its neighbor size: 0, less thanthe avg size 2.4, merging them
+2016-07-26 07:08:26,971 INFO  [B.fifo.QRpcServer.handler=20,queue=2,port=20000] normalizer.MergeNormalizationPlan: Executing merging normalization plan: MergeNormalizationPlan{firstRegion={ENCODED=> d51df2c58e9b525206b1325fd925a971, NAME => 'table_p8ddpd6q5z,,1469514755237.d51df2c58e9b525206b1325fd925a971.', STARTKEY => '', ENDKEY => '1'}, secondRegion={ENCODED => e69c6b25c7b9562d078d9ad3994f5330, NAME => 'table_p8ddpd6q5z,1,1469514767669.e69c6b25c7b9562d078d9ad3994f5330.',
+STARTKEY => '1', ENDKEY => '3'}}
+----
+Region normalizer as per it’s computed plan, merged the region with start key as ‘’
+and end key as ‘1’, with another region having start key as ‘1’ and end key as ‘3’.
+Now, that these regions have been merged we see a single new region with start key
+as ‘’ and end key as ‘3’
+----
+table_p8ddpd6q5z,,1469516907210.e06c9b83c4a252b130e column=info:mergeA, timestamp=1469516907431, 
+value=PBUF\x08\xA5\xD9\x9E\xAF\xE2*\x12\x1B\x0A\x07default\x12\x10table_p8ddpd6q5z\x1A\x00"\x011(\x000\x00 ea74d246741ba.   8\x00 
+table_p8ddpd6q5z,,1469516907210.e06c9b83c4a252b130e column=info:mergeB, timestamp=1469516907431,
+value=PBUF\x08\xB5\xBA\x9F\xAF\xE2*\x12\x1B\x0A\x07default\x12\x10table_p8ddpd6q5z\x1A\x011"\x013(\x000\x0 ea74d246741ba.   08\x00 
+table_p8ddpd6q5z,,1469516907210.e06c9b83c4a252b130e column=info:regioninfo, timestamp=1469516907431, value={ENCODED => e06c9b83c4a252b130eea74d246741ba, NAME => 'table_p8ddpd6q5z,,1469516907210.e06c9b83c ea74d246741ba.   4a252b130eea74d246741ba.', STARTKEY => '', ENDKEY => '3'}
+.... 
+table_p8ddpd6q5z,3,1469514778736.bf024670a847c0adff column=info:regioninfo, timestamp=1469514779417, value={ENCODED => bf024670a847c0adffb74b2e13408b32, NAME => 'table_p8ddpd6q5z,3,1469514778736.bf024670 b74b2e13408b32.  a847c0adffb74b2e13408b32.' STARTKEY => '3', ENDKEY => '7'} 
+.... 
+table_p8ddpd6q5z,7,1469514790152.7c5a67bc755e649db2 column=info:regioninfo, timestamp=1469514790312, value={ENCODED => 7c5a67bc755e649db22f49af6270f1e1, NAME => 'table_p8ddpd6q5z,7,1469514790152.7c5a67bc 2f49af6270f1e1.  755e649db22f49af6270f1e1.', STARTKEY => '7', ENDKEY => '8'} 
+....
+table_p8ddpd6q5z,8,1469514790152.58e7503cda69f98f47 column=info:regioninfo, timestamp=1469514790312, value={ENCODED => 58e7503cda69f98f4755178e74288c3a, NAME => 'table_p8ddpd6q5z,8,1469514790152.58e7503c 55178e74288c3a.  da69f98f4755178e74288c3a.', STARTKEY => '8', ENDKEY => ''}
+----
+
+A similar example can be seen for an user table with 3 smaller regions and 1
+relatively large region. For this example, we have an user table with 1 large region containing 100K rows, and 3 relatively smaller regions with about 33K rows each. As seen from the normalization plan, since the larger region is more than twice the average region size it ends being split into two regions – one with start key as ‘1’ and end key as ‘154717’ and the other region with start key as '154717' and end key as ‘3’
+----
+2016-07-26 07:39:45,636 DEBUG [B.fifo.QRpcServer.handler=7,queue=1,port=20000] master.HMaster: Skipping normalization for table: hbase:backup, as it's either system table or doesn't have auto normalization turned on
+2016-07-26 07:39:45,636 DEBUG [B.fifo.QRpcServer.handler=7,queue=1,port=20000] normalizer.SimpleRegionNormalizer: Computing normalization plan for table: table_p8ddpd6q5z, number of regions: 4
+2016-07-26 07:39:45,636 DEBUG [B.fifo.QRpcServer.handler=7,queue=1,port=20000] normalizer.SimpleRegionNormalizer: Table table_p8ddpd6q5z, total aggregated regions size: 12
+2016-07-26 07:39:45,636 DEBUG [B.fifo.QRpcServer.handler=7,queue=1,port=20000] normalizer.SimpleRegionNormalizer: Table table_p8ddpd6q5z, average region size: 3.0
+2016-07-26 07:39:45,636 DEBUG [B.fifo.QRpcServer.handler=7,queue=1,port=20000] normalizer.SimpleRegionNormalizer: No normalization needed, regions look good for table: table_p8ddpd6q5z
+2016-07-26 07:39:45,636 DEBUG [B.fifo.QRpcServer.handler=7,queue=1,port=20000] normalizer.SimpleRegionNormalizer: Computing normalization plan for table: table_h2osxu3wat, number of regions: 5
+2016-07-26 07:39:45,636 DEBUG [B.fifo.QRpcServer.handler=7,queue=1,port=20000] normalizer.SimpleRegionNormalizer: Table table_h2osxu3wat, total aggregated regions size: 7
+2016-07-26 07:39:45,636 DEBUG [B.fifo.QRpcServer.handler=7,queue=1,port=20000] normalizer.SimpleRegionNormalizer: Table table_h2osxu3wat, average region size: 1.4
+2016-07-26 07:39:45,636 INFO  [B.fifo.QRpcServer.handler=7,queue=1,port=20000] normalizer.SimpleRegionNormalizer: Table table_h2osxu3wat, large region table_h2osxu3wat,1,1469515926544.27f2fdbb2b6612ea163eb6b40753c3db. has size 4, more than twice avg size, splitting
+2016-07-26 07:39:45,640 INFO [B.fifo.QRpcServer.handler=7,queue=1,port=20000] normalizer.SplitNormalizationPlan: Executing splitting normalization plan: SplitNormalizationPlan{regionInfo={ENCODED => 27f2fdbb2b6612ea163eb6b40753c3db, NAME => 'table_h2osxu3wat,1,1469515926544.27f2fdbb2b6612ea163eb6b40753c3db.', STARTKEY => '1', ENDKEY => '3'}, splitPoint=null}
+2016-07-26 07:39:45,656 DEBUG [B.fifo.QRpcServer.handler=7,queue=1,port=20000] master.HMaster: Skipping normalization for table: hbase:namespace, as it's either system table or doesn't have auto normalization turned on
+2016-07-26 07:39:45,656 DEBUG [B.fifo.QRpcServer.handler=7,queue=1,port=20000] master.HMaster: Skipping normalization for table: hbase:meta, as it's either system table or doesn't
+have auto normalization turned on …..…..….
+2016-07-26 07:39:46,246 DEBUG [AM.ZK.Worker-pool2-t278] master.RegionStates: Onlined 54de97dae764b864504704c1c8d3674a on hbase-test-rc-5.openstacklocal,16020,1469419333913 {ENCODED => 54de97dae764b864504704c1c8d3674a, NAME => 'table_h2osxu3wat,1,1469518785661.54de97dae764b864504704c1c8d3674a.', STARTKEY => '1', ENDKEY => '154717'}
+2016-07-26 07:39:46,246 INFO  [AM.ZK.Worker-pool2-t278] master.RegionStates: Transition {d6b5625df331cfec84dce4f1122c567f state=SPLITTING_NEW, ts=1469518786246, server=hbase-test-rc-5.openstacklocal,16020,1469419333913} to {d6b5625df331cfec84dce4f1122c567f state=OPEN, ts=1469518786246,
+server=hbase-test-rc-5.openstacklocal,16020,1469419333913}
+2016-07-26 07:39:46,246 DEBUG [AM.ZK.Worker-pool2-t278] master.RegionStates: Onlined d6b5625df331cfec84dce4f1122c567f on hbase-test-rc-5.openstacklocal,16020,1469419333913 {ENCODED => d6b5625df331cfec84dce4f1122c567f, NAME => 'table_h2osxu3wat,154717,1469518785661.d6b5625df331cfec84dce4f1122c567f.', STARTKEY => '154717', ENDKEY => '3'}
+----


[22/26] hbase git commit: HBASE-19520 Add UTs for the new lock type PEER

Posted by zh...@apache.org.
HBASE-19520 Add UTs for the new lock type PEER

Signed-off-by: zhangduo <zh...@apache.org>


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

Branch: refs/heads/HBASE-19397
Commit: bdc06c2e089ed997c96e3ebb908668ba6721ac38
Parents: 377d776
Author: Guanghao Zhang <zg...@apache.org>
Authored: Wed Dec 20 16:43:38 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Fri Dec 22 21:41:28 2017 +0800

----------------------------------------------------------------------
 .../procedure/MasterProcedureScheduler.java     |   9 +-
 .../procedure/TestMasterProcedureScheduler.java |  65 ++++++++-
 ...TestMasterProcedureSchedulerConcurrency.java | 135 +++++++++++++++++++
 3 files changed, 201 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/bdc06c2e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
index dd85f5c..5f4665c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
@@ -389,6 +389,13 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
     while (tableIter.hasNext()) {
       count += tableIter.next().size();
     }
+
+    // Peer queues
+    final AvlTreeIterator<PeerQueue> peerIter = new AvlTreeIterator<>(peerMap);
+    while (peerIter.hasNext()) {
+      count += peerIter.next().size();
+    }
+
     return count;
   }
 
@@ -1041,7 +1048,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
    * @see #wakePeerExclusiveLock(Procedure, String)
    * @param procedure the procedure trying to acquire the lock
    * @param peerId peer to lock
-   * @return true if the procedure has to wait for the per to be available
+   * @return true if the procedure has to wait for the peer to be available
    */
   public boolean waitPeerExclusiveLock(Procedure<?> procedure, String peerId) {
     schedLock();

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdc06c2e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java
index 0291165..fd77e1f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java
@@ -905,6 +905,27 @@ public class TestMasterProcedureScheduler {
     }
   }
 
+  public static class TestPeerProcedure extends TestProcedure implements PeerProcedureInterface {
+    private final String peerId;
+    private final PeerOperationType opType;
+
+    public TestPeerProcedure(long procId, String peerId, PeerOperationType opType) {
+      super(procId);
+      this.peerId = peerId;
+      this.opType = opType;
+    }
+
+    @Override
+    public String getPeerId() {
+      return peerId;
+    }
+
+    @Override
+    public PeerOperationType getPeerOperationType() {
+      return opType;
+    }
+  }
+
   private static LockProcedure createLockProcedure(LockType lockType, long procId) throws Exception {
     LockProcedure procedure = new LockProcedure();
 
@@ -927,22 +948,19 @@ public class TestMasterProcedureScheduler {
     return createLockProcedure(LockType.SHARED, procId);
   }
 
-  private static void assertLockResource(LockedResource resource,
-      LockedResourceType resourceType, String resourceName)
-  {
+  private static void assertLockResource(LockedResource resource, LockedResourceType resourceType,
+      String resourceName) {
     assertEquals(resourceType, resource.getResourceType());
     assertEquals(resourceName, resource.getResourceName());
   }
 
-  private static void assertExclusiveLock(LockedResource resource, Procedure<?> procedure)
-  {
+  private static void assertExclusiveLock(LockedResource resource, Procedure<?> procedure) {
     assertEquals(LockType.EXCLUSIVE, resource.getLockType());
     assertEquals(procedure, resource.getExclusiveLockOwnerProcedure());
     assertEquals(0, resource.getSharedLockCount());
   }
 
-  private static void assertSharedLock(LockedResource resource, int lockCount)
-  {
+  private static void assertSharedLock(LockedResource resource, int lockCount) {
     assertEquals(LockType.SHARED, resource.getLockType());
     assertEquals(lockCount, resource.getSharedLockCount());
   }
@@ -1027,6 +1045,39 @@ public class TestMasterProcedureScheduler {
   }
 
   @Test
+  public void testListLocksPeer() throws Exception {
+    String peerId = "1";
+    LockProcedure procedure = createExclusiveLockProcedure(4);
+    queue.waitPeerExclusiveLock(procedure, peerId);
+
+    List<LockedResource> locks = queue.getLocks();
+    assertEquals(1, locks.size());
+
+    LockedResource resource = locks.get(0);
+    assertLockResource(resource, LockedResourceType.PEER, peerId);
+    assertExclusiveLock(resource, procedure);
+    assertTrue(resource.getWaitingProcedures().isEmpty());
+
+    // Try to acquire the exclusive lock again with same procedure
+    assertFalse(queue.waitPeerExclusiveLock(procedure, peerId));
+
+    // Try to acquire the exclusive lock again with new procedure
+    LockProcedure procedure2 = createExclusiveLockProcedure(5);
+    assertTrue(queue.waitPeerExclusiveLock(procedure2, peerId));
+
+    // Same peerId, still only has 1 LockedResource
+    locks = queue.getLocks();
+    assertEquals(1, locks.size());
+
+    resource = locks.get(0);
+    assertLockResource(resource, LockedResourceType.PEER, peerId);
+    // LockedResource owner still is the origin procedure
+    assertExclusiveLock(resource, procedure);
+    // The new procedure should in the waiting list
+    assertEquals(1, resource.getWaitingProcedures().size());
+  }
+
+  @Test
   public void testListLocksWaiting() throws Exception {
     LockProcedure procedure1 = createExclusiveLockProcedure(1);
     queue.waitTableExclusiveLock(procedure1, TableName.valueOf("ns4", "table4"));

http://git-wip-us.apache.org/repos/asf/hbase/blob/bdc06c2e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureSchedulerConcurrency.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureSchedulerConcurrency.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureSchedulerConcurrency.java
index 2e8e52a..4e67a63 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureSchedulerConcurrency.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureSchedulerConcurrency.java
@@ -26,6 +26,8 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface.PeerOperationType;
+import org.apache.hadoop.hbase.master.procedure.TestMasterProcedureScheduler.TestPeerProcedure;
 import org.apache.hadoop.hbase.master.procedure.TestMasterProcedureScheduler.TestTableProcedure;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -63,6 +65,85 @@ public class TestMasterProcedureSchedulerConcurrency {
     queue.clear();
   }
 
+  @Test
+  public void testConcurrentPeerOperations() throws Exception {
+    TestPeerProcedureSet procSet = new TestPeerProcedureSet(queue);
+
+    int NUM_ITEMS = 10;
+    int NUM_PEERS = 5;
+    AtomicInteger opsCount = new AtomicInteger(0);
+    for (int i = 0; i < NUM_PEERS; ++i) {
+      String peerId = String.format("test-peer-%04d", i);
+      for (int j = 1; j < NUM_ITEMS; ++j) {
+        procSet.addBack(new TestPeerProcedure(i * 100 + j, peerId, PeerOperationType.ADD));
+        opsCount.incrementAndGet();
+      }
+    }
+    assertEquals(opsCount.get(), queue.size());
+
+    Thread[] threads = new Thread[NUM_PEERS * 2];
+    HashSet<String> concurrentPeers = new HashSet<>();
+    ArrayList<String> failures = new ArrayList<>();
+    AtomicInteger concurrentCount = new AtomicInteger(0);
+    for (int i = 0; i < threads.length; ++i) {
+      threads[i] = new Thread() {
+        @Override
+        public void run() {
+          while (opsCount.get() > 0) {
+            try {
+              TestPeerProcedure proc = procSet.acquire();
+              if (proc == null) {
+                queue.signalAll();
+                if (opsCount.get() > 0) {
+                  continue;
+                }
+                break;
+              }
+
+              String peerId = proc.getPeerId();
+              synchronized (concurrentPeers) {
+                assertTrue("unexpected concurrency on " + peerId, concurrentPeers.add(peerId));
+              }
+              assertTrue(opsCount.decrementAndGet() >= 0);
+
+              try {
+                long procId = proc.getProcId();
+                int concurrent = concurrentCount.incrementAndGet();
+                assertTrue("inc-concurrent="+ concurrent +" 1 <= concurrent <= "+ NUM_PEERS,
+                  concurrent >= 1 && concurrent <= NUM_PEERS);
+                LOG.debug("[S] peerId="+ peerId +" procId="+ procId +" concurrent="+ concurrent);
+                Thread.sleep(2000);
+                concurrent = concurrentCount.decrementAndGet();
+                LOG.debug("[E] peerId="+ peerId +" procId="+ procId +" concurrent="+ concurrent);
+                assertTrue("dec-concurrent=" + concurrent, concurrent < NUM_PEERS);
+              } finally {
+                synchronized (concurrentPeers) {
+                  assertTrue(concurrentPeers.remove(peerId));
+                }
+                procSet.release(proc);
+              }
+            } catch (Throwable e) {
+              LOG.error("Failed " + e.getMessage(), e);
+              synchronized (failures) {
+                failures.add(e.getMessage());
+              }
+            } finally {
+              queue.signalAll();
+            }
+          }
+        }
+      };
+      threads[i].start();
+    }
+
+    for (int i = 0; i < threads.length; ++i) {
+      threads[i].join();
+    }
+    assertTrue(failures.toString(), failures.isEmpty());
+    assertEquals(0, opsCount.get());
+    assertEquals(0, queue.size());
+  }
+
   /**
    * Verify that "write" operations for a single table are serialized,
    * but different tables can be executed in parallel.
@@ -221,4 +302,58 @@ public class TestMasterProcedureSchedulerConcurrency {
       return ((TableProcedureInterface)proc).getTableOperationType();
     }
   }
+
+  public static class TestPeerProcedureSet {
+    private final MasterProcedureScheduler queue;
+
+    public TestPeerProcedureSet(final MasterProcedureScheduler queue) {
+      this.queue = queue;
+    }
+
+    public void addBack(TestPeerProcedure proc) {
+      queue.addBack(proc);
+    }
+
+    public TestPeerProcedure acquire() {
+      TestPeerProcedure proc = null;
+      boolean waiting = true;
+      while (waiting && queue.size() > 0) {
+        proc = (TestPeerProcedure) queue.poll(100000000L);
+        if (proc == null) {
+          continue;
+        }
+        switch (proc.getPeerOperationType()) {
+          case ADD:
+          case REMOVE:
+          case ENABLE:
+          case DISABLE:
+          case UPDATE_CONFIG:
+            waiting = queue.waitPeerExclusiveLock(proc, proc.getPeerId());
+            break;
+          case REFRESH:
+            waiting = false;
+            break;
+          default:
+            throw new UnsupportedOperationException();
+        }
+      }
+      return proc;
+    }
+
+    public void release(TestPeerProcedure proc) {
+      switch (proc.getPeerOperationType()) {
+        case ADD:
+        case REMOVE:
+        case ENABLE:
+        case DISABLE:
+        case UPDATE_CONFIG:
+          queue.wakePeerExclusiveLock(proc, proc.getPeerId());
+          break;
+        case REFRESH:
+          break;
+        default:
+          throw new UnsupportedOperationException();
+      }
+    }
+  }
 }


[02/26] hbase git commit: HBASE-19571 Renames OUTPUTDIR to OUTPUT_DIR and OUTPUT_RELATIVE to OUTPUT_DIR_RELATIVE.

Posted by zh...@apache.org.
HBASE-19571 Renames OUTPUTDIR to OUTPUT_DIR and OUTPUT_RELATIVE to OUTPUT_DIR_RELATIVE.


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

Branch: refs/heads/HBASE-19397
Commit: 6cefabee268f8509d7a50c632ff0ff7effcaf685
Parents: 35728ac
Author: Apekshit Sharma <ap...@apache.org>
Authored: Wed Dec 20 13:27:25 2017 -0800
Committer: Apekshit Sharma <ap...@apache.org>
Committed: Thu Dec 21 10:46:43 2017 -0800

----------------------------------------------------------------------
 dev-support/Jenkinsfile            | 99 +++++++++++++++------------------
 dev-support/hbase_nightly_yetus.sh | 17 +++---
 2 files changed, 55 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6cefabee/dev-support/Jenkinsfile
----------------------------------------------------------------------
diff --git a/dev-support/Jenkinsfile b/dev-support/Jenkinsfile
index fda3299..744e9b0 100644
--- a/dev-support/Jenkinsfile
+++ b/dev-support/Jenkinsfile
@@ -34,13 +34,6 @@ pipeline {
     // where we check out to across stages
     BASEDIR = "${env.WORKSPACE}/component"
     YETUS_RELEASE = '0.6.0'
-    // where we'll write everything from different steps.
-    OUTPUT_RELATIVE_GENERAL = 'output-general'
-    OUTPUTDIR_GENERAL = "${env.WORKSPACE}/output-general"
-    OUTPUT_RELATIVE_JDK7 = 'output-jdk7'
-    OUTPUTDIR_JDK7 = "${env.WORKSPACE}/output-jdk7"
-    OUTPUT_RELATIVE_JDK8 = 'output-jdk8'
-    OUTPUTDIR_JDK8 = "${env.WORKSPACE}/output-jdk8"
     PROJECT = 'hbase'
     PROJET_PERSONALITY = 'https://raw.githubusercontent.com/apache/hbase/master/dev-support/hbase-personality.sh'
     // This section of the docs tells folks not to use the javadoc tag. older branches have our old version of the check for said tag.
@@ -123,30 +116,30 @@ curl -L  -o personality.sh "${env.PROJET_PERSONALITY}"
         // on branches that don't support jdk7, this will already be JAVA_HOME, so we'll end up not
         // doing multijdk there.
         MULTIJDK = '/usr/lib/jvm/java-8-openjdk-amd64'
-        OUTPUT_RELATIVE = "${env.OUTPUT_RELATIVE_GENERAL}"
-        OUTPUTDIR = "${env.OUTPUTDIR_GENERAL}"
+        OUTPUT_DIR_RELATIVE = "output-general"
+        OUTPUT_DIR = "${env.WORKSPACE}/${env.OUTPUT_DIR_RELATIVE}"
       }
       steps {
         unstash 'yetus'
         sh '''#!/usr/bin/env bash
-          rm -rf "${OUTPUTDIR}" && mkdir "${OUTPUTDIR}"
-          rm -rf "${OUTPUTDIR}/machine" && mkdir "${OUTPUTDIR}/machine"
-          "${BASEDIR}/dev-support/gather_machine_environment.sh" "${OUTPUT_RELATIVE}/machine"
+          rm -rf "${OUTPUT_DIR}" && mkdir "${OUTPUT_DIR}"
+          rm -rf "${OUTPUT_DIR}/machine" && mkdir "${OUTPUT_DIR}/machine"
+          "${BASEDIR}/dev-support/gather_machine_environment.sh" "${OUTPUT_DIR_RELATIVE}/machine"
 '''
         // TODO should this be a download from master, similar to how the personality is?
         sh "${env.BASEDIR}/dev-support/hbase_nightly_yetus.sh"
       }
       post {
         always {
-          // env variables don't work in archive? or this has to be relative to WORKSPACE. :(
-          archive 'output-general/*'
-          archive 'output-general/**/*'
+          // Has to be relative to WORKSPACE.
+          archive "${env.OUTPUT_DIR_RELATIVE}/*"
+          archive "${env.OUTPUT_DIR_RELATIVE}/**/*"
           publishHTML target: [
             allowMissing: true,
             keepAll: true,
             alwaysLinkToLastBuild: true,
-            // has to be relative to WORKSPACE :(
-            reportDir: 'output-general',
+            // Has to be relative to WORKSPACE
+            reportDir: "${env.OUTPUT_DIR_RELATIVE}",
             reportFiles: 'console-report.html',
             reportName: 'General Nightly Build Report'
           ]
@@ -159,16 +152,16 @@ curl -L  -o personality.sh "${env.PROJET_PERSONALITY}"
       }
       environment {
         TESTS = 'mvninstall,compile,javac,unit,htmlout'
-        OUTPUT_RELATIVE = "${env.OUTPUT_RELATIVE_JDK7}"
-        OUTPUTDIR = "${env.OUTPUTDIR_JDK7}"
+        OUTPUT_DIR_RELATIVE = "output-jdk7"
+        OUTPUT_DIR = "${env.WORKSPACE}/${env.OUTPUT_DIR_RELATIVE}"
         // On branches where we do jdk7 checks, jdk7 will be JAVA_HOME already.
       }
       steps {
         unstash 'yetus'
         sh '''#!/usr/bin/env bash
-          rm -rf "${OUTPUTDIR}" && mkdir "${OUTPUTDIR}"
-          rm -rf "${OUTPUTDIR}/machine" && mkdir "${OUTPUTDIR}/machine"
-          "${BASEDIR}/dev-support/gather_machine_environment.sh" "${OUTPUT_RELATIVE}/machine"
+          rm -rf "${OUTPUT_DIR}" && mkdir "${OUTPUT_DIR}"
+          rm -rf "${OUTPUT_DIR}/machine" && mkdir "${OUTPUT_DIR}/machine"
+          "${BASEDIR}/dev-support/gather_machine_environment.sh" "${OUTPUT_DIR_RELATIVE}/machine"
 '''
         sh """#!/usr/bin/env bash
           # for branch-1.1 we don't do jdk8 findbugs, so do it here
@@ -180,14 +173,14 @@ curl -L  -o personality.sh "${env.PROJET_PERSONALITY}"
       }
       post {
         always {
-          junit testResults: 'output-jdk7/**/target/**/TEST-*.xml', allowEmptyResults: true
+          junit testResults: "${env.OUTPUT_DIR_RELATIVE}/**/target/**/TEST-*.xml", allowEmptyResults: true
           // zip surefire reports.
           sh '''#!/bin/bash -e
-            if [ -d "${OUTPUTDIR}/archiver" ]; then
-              count=$(find "${OUTPUTDIR}/archiver" -type f | wc -l)
+            if [ -d "${OUTPUT_DIR}/archiver" ]; then
+              count=$(find "${OUTPUT_DIR}/archiver" -type f | wc -l)
               if [[ 0 -ne ${count} ]]; then
                 echo "zipping ${count} archived files"
-                zip -q -m -r "${OUTPUTDIR}/test_logs.zip" "${OUTPUTDIR}/archiver"
+                zip -q -m -r "${OUTPUT_DIR}/test_logs.zip" "${OUTPUT_DIR}/archiver"
               else
                 echo "No archived files, skipping compressing."
               fi
@@ -195,17 +188,17 @@ curl -L  -o personality.sh "${env.PROJET_PERSONALITY}"
               echo "No archiver directory, skipping compressing."
             fi
 '''
-          // env variables don't work in archive? or this has to be relative to WORKSPACE. :(
-          archive 'output-jdk7/*'
-          archive 'output-jdk7/**/*'
+          // Has to be relative to WORKSPACE.
+          archive "${env.OUTPUT_DIR_RELATIVE}/*"
+          archive "${env.OUTPUT_DIR_RELATIVE}/**/*"
           publishHTML target: [
-            allowMissing: true,
-            keepAll: true,
+            allowMissing         : true,
+            keepAll              : true,
             alwaysLinkToLastBuild: true,
-            // has to be relative to WORKSPACE :(
-            reportDir: 'output-jdk7',
-            reportFiles: 'console-report.html',
-            reportName: 'JDK7 Nightly Build Report'
+            // Has to be relative to WORKSPACE.
+            reportDir            : "${env.OUTPUT_DIR_RELATIVE}",
+            reportFiles          : 'console-report.html',
+            reportName           : 'JDK7 Nightly Build Report'
           ]
         }
       }
@@ -218,8 +211,8 @@ curl -L  -o personality.sh "${env.PROJET_PERSONALITY}"
       }
       environment {
         TESTS = 'mvninstall,compile,javac,unit,findbugs,htmlout'
-        OUTPUT_RELATIVE = "${env.OUTPUT_RELATIVE_JDK8}"
-        OUTPUTDIR = "${env.OUTPUTDIR_JDK8}"
+        OUTPUT_DIR_RELATIVE = "output-jdk8"
+        OUTPUT_DIR = "${env.WORKSPACE}/${env.OUTPUT_DIR_RELATIVE}"
         // This isn't strictly needed on branches that only support jdk8, but doesn't hurt
         // and is needed on branches that do both jdk7 and jdk8
         SET_JAVA_HOME = '/usr/lib/jvm/java-8-openjdk-amd64'
@@ -227,22 +220,22 @@ curl -L  -o personality.sh "${env.PROJET_PERSONALITY}"
       steps {
         unstash 'yetus'
         sh '''#!/usr/bin/env bash
-          rm -rf "${OUTPUTDIR}" && mkdir "${OUTPUTDIR}"
-          rm -rf "${OUTPUTDIR}/machine" && mkdir "${OUTPUTDIR}/machine"
-          "${BASEDIR}/dev-support/gather_machine_environment.sh" "${OUTPUT_RELATIVE}/machine"
+          rm -rf "${OUTPUT_DIR}" && mkdir "${OUTPUT_DIR}"
+          rm -rf "${OUTPUT_DIR}/machine" && mkdir "${OUTPUT_DIR}/machine"
+          "${BASEDIR}/dev-support/gather_machine_environment.sh" "${OUTPUT_DIR_RELATIVE}/machine"
 '''
         sh "${env.BASEDIR}/dev-support/hbase_nightly_yetus.sh"
       }
       post {
         always {
-          junit testResults: 'output-jdk8/**/target/**/TEST-*.xml', allowEmptyResults: true
+          junit testResults: "${env.OUTPUT_DIR_RELATIVE}/**/target/**/TEST-*.xml", allowEmptyResults: true
           // zip surefire reports.
           sh '''#!/bin/bash -e
-            if [ -d "${OUTPUTDIR}/archiver" ]; then
-              count=$(find "${OUTPUTDIR}/archiver" -type f | wc -l)
+            if [ -d "${OUTPUT_DIR}/archiver" ]; then
+              count=$(find "${OUTPUT_DIR}/archiver" -type f | wc -l)
               if [[ 0 -ne ${count} ]]; then
                 echo "zipping ${count} archived files"
-                zip -q -m -r "${OUTPUTDIR}/test_logs.zip" "${OUTPUTDIR}/archiver"
+                zip -q -m -r "${OUTPUT_DIR}/test_logs.zip" "${OUTPUT_DIR}/archiver"
               else
                 echo "No archived files, skipping compressing."
               fi
@@ -250,17 +243,17 @@ curl -L  -o personality.sh "${env.PROJET_PERSONALITY}"
               echo "No archiver directory, skipping compressing."
             fi
 '''
-          // env variables don't work in archive? or this has to be relative to WORKSPACE. :(
-          archive 'output-jdk8/*'
-          archive 'output-jdk8/**/*'
+          // Has to be relative to WORKSPACE.
+          archive "${env.OUTPUT_DIR_RELATIVE}/*"
+          archive "${env.OUTPUT_DIR_RELATIVE}/**/*"
           publishHTML target: [
-            allowMissing: true,
-            keepAll: true,
+            allowMissing         : true,
+            keepAll              : true,
             alwaysLinkToLastBuild: true,
-            // has to be relative to WORKSPACE :(
-            reportDir: 'output-jdk8',
-            reportFiles: 'console-report.html',
-            reportName: 'JDK8 Nightly Build Report'
+            // Has to be relative to WORKSPACE.
+            reportDir            : "${env.OUTPUT_DIR_RELATIVE}",
+            reportFiles          : 'console-report.html',
+            reportName           : 'JDK8 Nightly Build Report'
           ]
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6cefabee/dev-support/hbase_nightly_yetus.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase_nightly_yetus.sh b/dev-support/hbase_nightly_yetus.sh
index da94c32..d37292b 100755
--- a/dev-support/hbase_nightly_yetus.sh
+++ b/dev-support/hbase_nightly_yetus.sh
@@ -18,8 +18,8 @@
 
 declare -i missing_env=0
 # Validate params
-for required_env in "TESTS" "TOOLS" "BASEDIR" "ARCHIVE_PATTERN_LIST" "OUTPUT_RELATIVE" \
-                    "BRANCH_SPECIFIC_DOCKERFILE" "OUTPUTDIR" "PROJECT" "AUTHOR_IGNORE_LIST" \
+for required_env in "TESTS" "TOOLS" "BASEDIR" "ARCHIVE_PATTERN_LIST" "OUTPUT_DIR_RELATIVE" \
+                    "BRANCH_SPECIFIC_DOCKERFILE" "OUTPUT_DIR" "PROJECT" "AUTHOR_IGNORE_LIST" \
                     "WHITESPACE_IGNORE_LIST" "BRANCH_NAME" "TESTS_FILTER" "DEBUG" \
                     "USE_YETUS_PRERELEASE" "WORKSPACE" "YETUS_RELEASE"; do
   if [ -z "${!required_env}" ]; then
@@ -49,15 +49,16 @@ YETUS_ARGS=("--basedir=${BASEDIR}" "${YETUS_ARGS[@]}")
 YETUS_ARGS=("--archive-list=${ARCHIVE_PATTERN_LIST}" "${YETUS_ARGS[@]}")
 YETUS_ARGS=("--console-urls" "${YETUS_ARGS[@]}")
 # YETUS-532, repeat this twice in case the fix is to update args rather than docs
-YETUS_ARGS=("--build-url-patchdir=artifact/${OUTPUT_RELATIVE}" "${YETUS_ARGS[@]}")
-YETUS_ARGS=("--build-url-artifacts=artifact/${OUTPUT_RELATIVE}" "${YETUS_ARGS[@]}")
+YETUS_ARGS=("--build-url-patchdir=artifact/${OUTPUT_DIR_RELATIVE}" "${YETUS_ARGS[@]}")
+YETUS_ARGS=("--build-url-artifacts=artifact/${OUTPUT_DIR_RELATIVE}" "${YETUS_ARGS[@]}")
 YETUS_ARGS=("--docker" "${YETUS_ARGS[@]}")
 YETUS_ARGS=("--dockerfile=${BRANCH_SPECIFIC_DOCKERFILE}" "${YETUS_ARGS[@]}")
+# Yetus sets BUILDMODE env variable to "full" if this arg is passed.
 YETUS_ARGS=("--empty-patch" "${YETUS_ARGS[@]}")
-YETUS_ARGS=("--html-report-file=${OUTPUTDIR}/console-report.html" "${YETUS_ARGS[@]}")
+YETUS_ARGS=("--html-report-file=${OUTPUT_DIR}/console-report.html" "${YETUS_ARGS[@]}")
 YETUS_ARGS=("--jenkins" "${YETUS_ARGS[@]}")
 YETUS_ARGS=("--mvn-custom-repos" "${YETUS_ARGS[@]}")
-YETUS_ARGS=("--patch-dir=${OUTPUTDIR}" "${YETUS_ARGS[@]}")
+YETUS_ARGS=("--patch-dir=${OUTPUT_DIR}" "${YETUS_ARGS[@]}")
 YETUS_ARGS=("--project=${PROJECT}" "${YETUS_ARGS[@]}")
 YETUS_ARGS=("--resetrepo" "${YETUS_ARGS[@]}")
 YETUS_ARGS=("--author-ignore-list=${AUTHOR_IGNORE_LIST}" "${YETUS_ARGS[@]}")
@@ -79,8 +80,8 @@ if [[ true == "${DEBUG}" ]]; then
   YETUS_ARGS=("--debug" "${YETUS_ARGS[@]}")
 fi
 
-rm -rf "${OUTPUTDIR}"
-mkdir -p "${OUTPUTDIR}"
+rm -rf "${OUTPUT_DIR}"
+mkdir -p "${OUTPUT_DIR}"
 if [[ true !=  "${USE_YETUS_PRERELEASE}" ]]; then
   YETUS_ARGS=("--shelldocs=${WORKSPACE}/yetus-${YETUS_RELEASE}/bin/shelldocs" "${YETUS_ARGS[@]}")
   TESTPATCHBIN="${WORKSPACE}/yetus-${YETUS_RELEASE}/bin/test-patch"