[09/38] hbase git commit: HBASE-18452 VerifyReplication by Snapshot should cache HDFS token before submit job for kerberos env

2018-01-04 Thread zhangduo
HBASE-18452 VerifyReplication by Snapshot should cache HDFS token before submit 
job for kerberos env


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

Branch: refs/heads/HBASE-19397
Commit: 51954359416b107ce5eda6cb710449edc98ab0e6
Parents: 8119acf
Author: huzheng 
Authored: Thu Jan 4 11:47:20 2018 +0800
Committer: huzheng 
Committed: Thu Jan 4 13:55:27 2018 +0800

--
 .../hadoop/hbase/mapreduce/replication/VerifyReplication.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/51954359/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
--
diff --git 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
index 035086e..9065f4e 100644
--- 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
+++ 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -428,7 +429,7 @@ public class VerifyReplication extends Configured 
implements Tool {
   conf.set(NAME + ".peerHBaseRootAddress", peerHBaseRootAddress);
 
   // This is to create HDFS delegation token for peer cluster in case of 
secured
-  conf.setStrings(MRJobConfig.JOB_NAMENODES, peerFSAddress);
+  conf.setStrings(MRJobConfig.JOB_NAMENODES, peerFSAddress, 
conf.get(HConstants.HBASE_DIR));
 }
 
 Job job = Job.getInstance(conf, conf.get(JOB_NAME_CONF_KEY, NAME + "_" + 
tableName));



[20/38] hbase git commit: HBASE-19617 Remove ReplicationQueues, use ReplicationQueueStorage directly

2018-01-04 Thread zhangduo
http://git-wip-us.apache.org/repos/asf/hbase/blob/72f1e971/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
index b6cf54d..4b9ed74 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.util.List;
 import java.util.UUID;
 
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -32,9 +31,10 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Interface that defines a replication source
@@ -47,15 +47,10 @@ public interface ReplicationSourceInterface {
* @param conf the configuration to use
* @param fs the file system to use
* @param manager the manager to use
-   * @param replicationQueues
-   * @param replicationPeers
* @param server the server for this region server
-   * @param peerClusterZnode
-   * @param clusterId
-   * @throws IOException
*/
   void init(Configuration conf, FileSystem fs, ReplicationSourceManager 
manager,
-  ReplicationQueues replicationQueues, ReplicationPeers replicationPeers, 
Server server,
+  ReplicationQueueStorage queueStorage, ReplicationPeers replicationPeers, 
Server server,
   String peerClusterZnode, UUID clusterId, ReplicationEndpoint 
replicationEndpoint,
   WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) 
throws IOException;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/72f1e971/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 b1d82c8..853bafb 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
@@ -34,19 +34,21 @@ import java.util.TreeSet;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.Future;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.RejectedExecutionException;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
-
+import java.util.stream.Collectors;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Connection;
@@ -60,7 +62,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeer;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationTracker;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
@@ -68,6 +70,7 @@ import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import 
org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import 
org.apache.hbase

[15/38] hbase git commit: HBASE-19622 Reimplement ReplicationPeers with the new replication storage interface

2018-01-04 Thread zhangduo
HBASE-19622 Reimplement ReplicationPeers with the new replication storage 
interface


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

Branch: refs/heads/HBASE-19397
Commit: aed5fe35dfa0a32706efa837eab1a24b744b0a0f
Parents: 1821fbc
Author: huzheng 
Authored: Tue Dec 26 16:46:10 2017 +0800
Committer: zhangduo 
Committed: Thu Jan 4 14:50:49 2018 +0800

--
 .../replication/ReplicationPeerConfigUtil.java  |  10 +-
 .../replication/VerifyReplication.java  |   9 +-
 .../hbase/replication/ReplicationFactory.java   |  10 +-
 .../hbase/replication/ReplicationPeerImpl.java  |  60 +-
 .../replication/ReplicationPeerStorage.java |   3 +-
 .../hbase/replication/ReplicationPeers.java | 235 
 .../replication/ReplicationPeersZKImpl.java | 543 ---
 .../replication/ZKReplicationPeerStorage.java   |  12 +-
 .../replication/ZKReplicationStorageBase.java   |   3 +-
 .../replication/TestReplicationStateBasic.java  | 125 ++---
 .../replication/TestReplicationStateZKImpl.java |   2 +-
 .../TestZKReplicationPeerStorage.java   |  12 +-
 .../cleaner/ReplicationZKNodeCleaner.java   |  57 +-
 .../replication/ReplicationPeerManager.java |   6 +-
 .../regionserver/DumpReplicationQueues.java |   2 +-
 .../regionserver/PeerProcedureHandlerImpl.java  |  49 +-
 .../replication/regionserver/Replication.java   |   2 +-
 .../regionserver/ReplicationSource.java |   6 +-
 .../regionserver/ReplicationSourceManager.java  |  45 +-
 .../cleaner/TestReplicationHFileCleaner.java|   7 +-
 .../replication/TestMultiSlaveReplication.java  |   2 -
 .../TestReplicationTrackerZKImpl.java   |  36 +-
 .../TestReplicationSourceManager.java   |  17 +-
 .../hadoop/hbase/HBaseZKTestingUtility.java |   3 +-
 24 files changed, 304 insertions(+), 952 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/aed5fe35/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
--
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
index 022bf64..a234a9b 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
@@ -247,22 +247,22 @@ public final class ReplicationPeerConfigUtil {
   public static ReplicationPeerConfig parsePeerFrom(final byte[] bytes)
   throws DeserializationException {
 if (ProtobufUtil.isPBMagicPrefix(bytes)) {
-  int pblen = ProtobufUtil.lengthOfPBMagic();
+  int pbLen = ProtobufUtil.lengthOfPBMagic();
   ReplicationProtos.ReplicationPeer.Builder builder =
   ReplicationProtos.ReplicationPeer.newBuilder();
   ReplicationProtos.ReplicationPeer peer;
   try {
-ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
+ProtobufUtil.mergeFrom(builder, bytes, pbLen, bytes.length - pbLen);
 peer = builder.build();
   } catch (IOException e) {
 throw new DeserializationException(e);
   }
   return convert(peer);
 } else {
-  if (bytes.length > 0) {
-return 
ReplicationPeerConfig.newBuilder().setClusterKey(Bytes.toString(bytes)).build();
+  if (bytes == null || bytes.length <= 0) {
+throw new DeserializationException("Bytes to deserialize should not be 
empty.");
   }
-  return ReplicationPeerConfig.newBuilder().setClusterKey("").build();
+  return 
ReplicationPeerConfig.newBuilder().setClusterKey(Bytes.toString(bytes)).build();
 }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/aed5fe35/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
--
diff --git 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
index 09d4b4b..f0070f0 100644
--- 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
+++ 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
@@ -339,15 +339,10 @@ public class VerifyReplication extends Configured 
implements Tool {
 @Override public boolean isAborte

[05/38] hbase git commit: HBASE-19473 Miscellaneous changes to ClientScanner

2018-01-04 Thread zhangduo
HBASE-19473 Miscellaneous changes to ClientScanner

- Remove superfluous logging code guard
- Simplify some of the code
- Use ArrayDeque instead of LinkedList for queue implementation


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

Branch: refs/heads/HBASE-19397
Commit: 2bd259b445971131526a5e6580363c92dc597b10
Parents: 4ba7416
Author: BELUGA BEHR 
Authored: Wed Jan 3 19:59:45 2018 -0800
Committer: Apekshit Sharma 
Committed: Wed Jan 3 20:25:21 2018 -0800

--
 .../hadoop/hbase/client/ClientScanner.java  | 99 ++--
 1 file changed, 49 insertions(+), 50 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/2bd259b4/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
--
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
index 42597ff..0c6dc16 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java
@@ -25,29 +25,30 @@ import 
org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
 
 import java.io.IOException;
 import java.io.InterruptedIOException;
-import java.util.LinkedList;
+import java.util.ArrayDeque;
 import java.util.Queue;
 import java.util.concurrent.ExecutorService;
 
 import org.apache.commons.lang3.mutable.MutableBoolean;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.ScannerCallable.MoreResults;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
+import org.apache.hadoop.hbase.exceptions.ScannerResetException;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.NotServingRegionException;
+import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.UnknownScannerException;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.client.ScannerCallable.MoreResults;
-import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
-import org.apache.hadoop.hbase.exceptions.ScannerResetException;
-import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.util.Bytes;
 
 /**
  * Implements the scanner interface for the HBase client. If there are 
multiple regions in a table,
@@ -294,25 +295,30 @@ public abstract class ClientScanner extends 
AbstractClientScanner {
   }
 
   protected void initSyncCache() {
-cache = new LinkedList<>();
+cache = new ArrayDeque<>();
   }
 
   protected Result nextWithSyncCache() throws IOException {
-// If the scanner is closed and there's nothing left in the cache, next is 
a no-op.
-if (cache.isEmpty() && this.closed) {
-  return null;
+Result result = cache.poll();
+if (result != null) {
+  return result;
 }
-if (cache.isEmpty()) {
-  loadCache();
+// If there is nothing left in the cache and the scanner is closed,
+// return a no-op
+if (this.closed) {
+  return null;
 }
 
-if (cache.size() > 0) {
-  return cache.poll();
-}
+loadCache();
+
+// try again to load from cache
+result = cache.poll();
 
 // if we exhausted this scanner before calling close, write out the scan 
metrics
-writeScanMetrics();
-return null;
+if (result == null) {
+  writeScanMetrics();
+}
+return result;
   }
 
   @VisibleForTesting
@@ -410,11 +416,9 @@ public abstract class ClientScanner extends 
AbstractClientScanner {
 long remainingResultSize = maxScannerResultSize;
 int countdown = this.caching;
 // This is possible if we just stopped at the boundary of a region in the 
previous call.
-if (callable == null) {
-  if (!moveToNextRegion()) {
-closed = true;
-return;
-  }
+if (callable == null && !moveToNextRegion()) {
+  closed = true;
+  return;
 }
 // This flag is set when we want to skip the result returned. We do
   

[08/38] hbase git commit: HBASE-19596 RegionMetrics/ServerMetrics/ClusterMetrics should apply to all public classes

2018-01-04 Thread zhangduo
HBASE-19596 RegionMetrics/ServerMetrics/ClusterMetrics should apply to all 
public classes


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

Branch: refs/heads/HBASE-19397
Commit: 8119acfca7e35cd7c4c203a397b970a50d7d7574
Parents: 2bd259b
Author: Chia-Ping Tsai 
Authored: Thu Jan 4 12:40:09 2018 +0800
Committer: Chia-Ping Tsai 
Committed: Thu Jan 4 13:13:34 2018 +0800

--
 .../hadoop/hbase/RegionMetricsBuilder.java  |   8 +
 .../org/apache/hadoop/hbase/client/Admin.java   | 119 ++---
 .../apache/hadoop/hbase/client/AsyncAdmin.java  |  44 +-
 .../hadoop/hbase/client/AsyncHBaseAdmin.java|  20 +-
 .../hbase/client/ClusterStatusListener.java |  13 +-
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |  48 +-
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java |  49 +-
 .../hbase/shaded/protobuf/ProtobufUtil.java |  27 --
 .../hadoop/hbase/client/TestInterfaceAlign.java |   3 +-
 .../hadoop/hbase/DistributedHBaseCluster.java   |  56 +--
 .../hadoop/hbase/IntegrationTestIngest.java |   6 +-
 .../hbase/IntegrationTestLazyCfLoading.java |  10 +-
 ...IntegrationTestRegionReplicaReplication.java |   7 +-
 .../hadoop/hbase/IntegrationTestingUtility.java |   7 +-
 .../StripeCompactionsPerformanceEvaluation.java |  10 +-
 .../hadoop/hbase/chaos/actions/Action.java  |  38 +-
 .../chaos/actions/BatchRestartRsAction.java |   5 +-
 .../chaos/actions/DumpClusterStatusAction.java  |   2 +-
 .../chaos/actions/MoveRegionsOfTableAction.java |   2 +-
 .../actions/RestartActiveMasterAction.java  |   2 +-
 .../actions/RestartRsHoldingMetaAction.java |   6 +-
 .../UnbalanceKillAndRebalanceAction.java|   7 +-
 .../chaos/actions/UnbalanceRegionsAction.java   |   7 +-
 .../mapreduce/IntegrationTestBulkLoad.java  |   5 +-
 .../hadoop/hbase/mttr/IntegrationTestMTTR.java  |   4 +-
 .../test/IntegrationTestBigLinkedList.java  |   7 +-
 ...stTimeBoundedRequestsWithRegionReplicas.java |   5 +-
 .../hbase/mapreduce/RegionSizeCalculator.java   |  15 +-
 .../mapreduce/TestRegionSizeCalculator.java |  42 +-
 .../rest/StorageClusterStatusResource.java  |  48 +-
 .../rest/StorageClusterVersionResource.java |   2 +-
 .../hbase/rsgroup/RSGroupBasedLoadBalancer.java |  11 +-
 .../hadoop/hbase/rsgroup/TestRSGroupsBase.java  |  57 +--
 .../tmpl/master/BackupMasterStatusTmpl.jamon|   4 +-
 .../hbase/coprocessor/MasterObserver.java   |   9 +-
 .../hbase/master/ClusterStatusPublisher.java|  33 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |  36 +-
 .../hadoop/hbase/master/LoadBalancer.java   |   8 +-
 .../hbase/master/MasterCoprocessorHost.java |  14 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |   3 +-
 .../hbase/master/RegionPlacementMaintainer.java | 451 ++-
 .../master/balancer/BalancerRegionLoad.java |  17 +-
 .../hbase/master/balancer/BaseLoadBalancer.java |   8 +-
 .../master/balancer/ClusterStatusChore.java |   2 +-
 .../master/balancer/RegionLocationFinder.java   |  10 +-
 .../master/balancer/StochasticLoadBalancer.java |  35 +-
 .../hbase/security/access/AccessController.java |   2 -
 .../org/apache/hadoop/hbase/tool/Canary.java|  17 +-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java |  20 +-
 .../hadoop/hbase/util/HBaseFsckRepair.java  |   2 +-
 .../apache/hadoop/hbase/util/RegionMover.java   |   4 +-
 .../hadoop/hbase/util/RegionSplitter.java   |   6 +-
 .../org/apache/hadoop/hbase/HBaseCluster.java   |  24 +-
 .../hadoop/hbase/HBaseTestingUtility.java   |   5 +-
 .../apache/hadoop/hbase/MiniHBaseCluster.java   |  31 +-
 .../hadoop/hbase/TestClientClusterMetrics.java  |  29 +-
 .../hadoop/hbase/TestClientClusterStatus.java   |  51 +--
 .../org/apache/hadoop/hbase/TestRegionLoad.java |  21 +-
 .../apache/hadoop/hbase/TestRegionMetrics.java  |  14 +-
 .../hbase/TestStochasticBalancerJmxMetrics.java |  17 +-
 .../apache/hadoop/hbase/client/TestAdmin2.java  |   3 +-
 .../hbase/client/TestAsyncClusterAdminApi.java  |  45 +-
 .../client/TestAsyncDecommissionAdminApi.java   |   3 +-
 .../hadoop/hbase/client/TestFromClientSide.java |   7 +-
 .../hbase/client/TestMetaWithReplicas.java  |  10 +-
 .../hadoop/hbase/client/TestMultiParallel.java  |   2 +-
 .../hbase/coprocessor/TestMasterObserver.java   |   4 +-
 .../hadoop/hbase/master/TestMasterFailover.java |  29 +-
 .../TestMasterFailoverBalancerPersistence.java  |  25 +-
 .../TestMasterOperationsForRegionReplicas.java  |   6 +-
 .../hadoop/hbase/master/TestMasterShutdown.java |   8 +-
 .../TestFavoredStochasticBalancerPickers.java   |   8 +-
 .../TestFavoredStochasticLoadBalancer.java  |  27 +-
 .../balancer/TestRegionLocationFinder.java  |   3 +-
 .../balancer/TestStochasticLoadB

[11/38] hbase git commit: HBASE-19630 Add peer cluster key check when add new replication peer

2018-01-04 Thread zhangduo
HBASE-19630 Add peer cluster key check when add new replication peer

Signed-off-by: zhangduo 


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

Branch: refs/heads/HBASE-19397
Commit: 3fc125c82a4b7d3910a22c9f1845642af765df7b
Parents: c2464ae
Author: Guanghao Zhang 
Authored: Tue Dec 26 21:10:00 2017 +0800
Committer: zhangduo 
Committed: Thu Jan 4 14:50:49 2018 +0800

--
 .../replication/ReplicationPeerManager.java | 54 
 .../replication/TestReplicationAdmin.java   | 22 
 2 files changed, 54 insertions(+), 22 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/3fc125c8/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
index 84abfeb..b78cbce 100644
--- 
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
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.master.replication;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
@@ -42,6 +43,7 @@ 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.ZKConfig;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 
@@ -216,36 +218,36 @@ public final class ReplicationPeerManager {
 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.
-   * 
-   * 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 {
+  private void checkPeerConfig(ReplicationPeerConfig peerConfig) throws 
DoNotRetryIOException {
+checkClusterKey(peerConfig.getClusterKey());
+
 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");
+  // 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 ((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())) {
+  // 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.
+  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" +
-  

[37/38] hbase git commit: HBASE-19687 Move the logic in ReplicationZKNodeCleaner to ReplicationChecker and remove ReplicationZKNodeCleanerChore

2018-01-04 Thread zhangduo
HBASE-19687 Move the logic in ReplicationZKNodeCleaner to ReplicationChecker 
and remove ReplicationZKNodeCleanerChore


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

Branch: refs/heads/HBASE-19397
Commit: 7b0637c503b14062dd4a9c0279793bca49a4aac4
Parents: 421099e
Author: zhangduo 
Authored: Wed Jan 3 09:39:44 2018 +0800
Committer: zhangduo 
Committed: Thu Jan 4 14:52:54 2018 +0800

--
 .../replication/VerifyReplication.java  |   6 +-
 .../hbase/replication/ReplicationPeers.java |  26 +--
 .../hbase/replication/ReplicationUtils.java |  69 +++
 .../replication/TestReplicationStateBasic.java  |   2 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |  13 --
 .../cleaner/ReplicationZKNodeCleaner.java   | 192 ---
 .../cleaner/ReplicationZKNodeCleanerChore.java  |  54 --
 .../replication/ReplicationPeerManager.java |  18 +-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java |  13 +-
 .../hbase/util/hbck/ReplicationChecker.java | 109 +++
 .../cleaner/TestReplicationZKNodeCleaner.java   | 109 ---
 .../hbase/util/TestHBaseFsckReplication.java| 101 ++
 .../hadoop/hbase/util/hbck/HbckTestingUtil.java |   6 +-
 13 files changed, 259 insertions(+), 459 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/7b0637c5/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
--
diff --git 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
index fe45762..fac4875 100644
--- 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
+++ 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
@@ -50,8 +50,8 @@ import org.apache.hadoop.hbase.mapreduce.TableSplit;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerStorage;
-import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
+import org.apache.hadoop.hbase.replication.ReplicationUtils;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -345,10 +345,10 @@ public class VerifyReplication extends Configured 
implements Tool {
 }
   });
   ReplicationPeerStorage storage =
-  ReplicationStorageFactory.getReplicationPeerStorage(localZKW, conf);
+ReplicationStorageFactory.getReplicationPeerStorage(localZKW, conf);
   ReplicationPeerConfig peerConfig = storage.getPeerConfig(peerId);
   return Pair.newPair(peerConfig,
-ReplicationPeers.getPeerClusterConfiguration(peerConfig, conf));
+ReplicationUtils.getPeerClusterConfiguration(peerConfig, conf));
 } catch (ReplicationException e) {
   throw new IOException("An error occurred while trying to connect to the 
remove peer cluster",
   e);

http://git-wip-us.apache.org/repos/asf/hbase/blob/7b0637c5/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
--
diff --git 
a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
 
b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
index ad3fee7..95192b8 100644
--- 
a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
+++ 
b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
@@ -17,14 +17,11 @@
  */
 package org.apache.hadoop.hbase.replication;
 
-import java.io.IOException;
 import java.util.Collections;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.CompoundConfiguration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -106,25 +103,6 @@ public class ReplicationPeers {
 return Collections.unmodifiableSet(peerCache.keySet());
   }
 
-  public static 

[38/38] hbase git commit: HBASE-19634 Add permission check for executeProcedures in AccessController

2018-01-04 Thread zhangduo
HBASE-19634 Add permission check for executeProcedures in AccessController


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

Branch: refs/heads/HBASE-19397
Commit: f27b9d4d7a9d9271d454f20c16ba7a8589f3b8a5
Parents: 8f1656d
Author: zhangduo 
Authored: Thu Jan 4 16:18:21 2018 +0800
Committer: zhangduo 
Committed: Thu Jan 4 16:18:21 2018 +0800

--
 .../hbase/coprocessor/RegionServerObserver.java | 14 +
 .../hbase/regionserver/RSRpcServices.java   | 54 +++-
 .../RegionServerCoprocessorHost.java| 18 +++
 .../hbase/security/access/AccessController.java | 24 +
 .../hadoop/hbase/TestJMXConnectorServer.java|  7 +++
 .../security/access/TestAccessController.java   | 18 +--
 6 files changed, 98 insertions(+), 37 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/f27b9d4d/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java
index c1af3fb..5b751df 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java
@@ -126,4 +126,18 @@ public interface RegionServerObserver {
   default void postClearCompactionQueues(
   final ObserverContext ctx)
   throws IOException {}
+
+  /**
+   * This will be called before executing procedures
+   * @param ctx the environment to interact with the framework and region 
server.
+   */
+  default void 
preExecuteProcedures(ObserverContext ctx)
+  throws IOException {}
+
+  /**
+   * This will be called after executing procedures
+   * @param ctx the environment to interact with the framework and region 
server.
+   */
+  default void 
postExecuteProcedures(ObserverContext ctx)
+  throws IOException {}
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f27b9d4d/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 e88f70e..695b859 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
@@ -41,7 +41,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.LongAdder;
-
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -142,6 +141,7 @@ import 
org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
 import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
 import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
@@ -3454,36 +3454,40 @@ public class RSRpcServices implements 
HBaseRPCErrorHandler,
   }
 
   @Override
+  @QosPriority(priority = HConstants.ADMIN_QOS)
   public ExecuteProceduresResponse executeProcedures(RpcController controller,
   ExecuteProceduresRequest request) throws ServiceException {
-if (request.getOpenRegionCount() > 0) {
-  for (OpenRegionRequest req : request.getOpenRegionList()) {
-openRegion(controller, req);
+try {
+  checkOpen();
+  regionServer.getRegionServerCoprocessorHost().preExecuteProcedures();
+  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.getCloseRegionCount() > 0) {
+for (CloseRegionRequest req : request.getCloseRegionList()) {
+  closeRegion(controller, req);
+}
   }
-}
-if (request.getProcCount() > 0) 

[07/38] hbase git commit: HBASE-19596 RegionMetrics/ServerMetrics/ClusterMetrics should apply to all public classes

2018-01-04 Thread zhangduo
http://git-wip-us.apache.org/repos/asf/hbase/blob/8119acfc/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRegionSizeCalculator.java
--
diff --git 
a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRegionSizeCalculator.java
 
b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRegionSizeCalculator.java
index 301cfef..1227595 100644
--- 
a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRegionSizeCalculator.java
+++ 
b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestRegionSizeCalculator.java
@@ -17,31 +17,28 @@
  */
 package org.apache.hadoop.hbase.mapreduce;
 
+import static org.apache.hadoop.hbase.HConstants.DEFAULT_REGIONSERVER_PORT;
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.RegionLoad;
+import org.apache.hadoop.hbase.RegionMetrics;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.Size;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.client.RegionLocator;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-
-import static org.apache.hadoop.hbase.HConstants.DEFAULT_REGIONSERVER_PORT;
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Mockito.when;
-
 @Category({MiscTests.class, SmallTests.class})
 public class TestRegionSizeCalculator {
 
@@ -134,14 +131,15 @@ public class TestRegionSizeCalculator {
   /**
* Creates mock returning RegionLoad info about given servers.
   */
-  private Admin mockAdmin(RegionLoad... regionLoadArray) throws Exception {
+  private Admin mockAdmin(RegionMetrics... regionLoadArray) throws Exception {
 Admin mockAdmin = Mockito.mock(Admin.class);
-Map regionLoads = new 
TreeMap<>(Bytes.BYTES_COMPARATOR);
-for (RegionLoad regionLoad : regionLoadArray) {
-  regionLoads.put(regionLoad.getName(), regionLoad);
+List regionLoads = new ArrayList<>();
+for (RegionMetrics regionLoad : regionLoadArray) {
+  regionLoads.add(regionLoad);
 }
 when(mockAdmin.getConfiguration()).thenReturn(configuration);
-when(mockAdmin.getRegionLoad(sn, 
TableName.valueOf("sizeTestTable"))).thenReturn(regionLoads);
+when(mockAdmin.getRegionMetrics(sn, TableName.valueOf("sizeTestTable")))
+.thenReturn(regionLoads);
 return mockAdmin;
   }
 
@@ -150,11 +148,11 @@ public class TestRegionSizeCalculator {
*
* @param  fileSizeMb number of megabytes occupied by region in file store 
in megabytes
* */
-  private RegionLoad mockRegion(String regionName, int fileSizeMb) {
-RegionLoad region = Mockito.mock(RegionLoad.class);
-when(region.getName()).thenReturn(regionName.getBytes());
+  private RegionMetrics mockRegion(String regionName, int fileSizeMb) {
+RegionMetrics region = Mockito.mock(RegionMetrics.class);
+when(region.getRegionName()).thenReturn(regionName.getBytes());
 when(region.getNameAsString()).thenReturn(regionName);
-when(region.getStorefileSizeMB()).thenReturn(fileSizeMb);
+when(region.getStoreFileSize()).thenReturn(new Size(fileSizeMb, 
Size.Unit.MEGABYTE));
 return region;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8119acfc/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 7ee1065..2323bf3 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,6 +21,7 @@ package org.apache.hadoop.hbase.rest;
 
 import java.io.IOException;
 import java.util.EnumSet;
+import java.util.Map;
 import javax.ws.rs.GET;
 import javax.ws.rs.Produces;
 import javax.ws.rs.core.CacheControl;
@@ -28,11 +29,12 @@ 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.had

[04/38] hbase git commit: HBASE-19588 Additional jar dependencies needed for mapreduce PerformanceEvaluation

2018-01-04 Thread zhangduo
HBASE-19588 Additional jar dependencies needed for mapreduce
PerformanceEvaluation

Signed-off-by: Albert Chu 


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

Branch: refs/heads/HBASE-19397
Commit: 4ba741674d623309e0ff6cd37d2b53ab7c6d7398
Parents: 338a74e
Author: Michael Stack 
Authored: Wed Jan 3 21:32:16 2018 -0600
Committer: Michael Stack 
Committed: Wed Jan 3 21:39:37 2018 -0600

--
 .../java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/4ba74167/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
--
diff --git 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
index 83895fd..df8ea76 100644
--- 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
+++ 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
@@ -808,7 +808,7 @@ public class TableMapReduceUtil {
   org.apache.hadoop.hbase.ipc.RpcServer.class,   // 
hbase-server
   org.apache.hadoop.hbase.CompatibilityFactory.class,// 
hbase-hadoop-compat
   org.apache.hadoop.hbase.mapreduce.JobUtil.class,   // 
hbase-hadoop2-compat
-  org.apache.hadoop.hbase.mapreduce.TableMapper.class,   // 
hbase-mapreduce
+  org.apache.hadoop.hbase.mapreduce.TableMapper.class,   // 
hbase-server
   org.apache.hadoop.hbase.metrics.impl.FastLongHistogram.class,  // 
hbase-metrics
   org.apache.hadoop.hbase.metrics.Snapshot.class,// 
hbase-metrics-api
   org.apache.zookeeper.ZooKeeper.class,



[21/38] hbase git commit: HBASE-19617 Remove ReplicationQueues, use ReplicationQueueStorage directly

2018-01-04 Thread zhangduo
HBASE-19617 Remove ReplicationQueues, use ReplicationQueueStorage directly


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

Branch: refs/heads/HBASE-19397
Commit: 72f1e971d4565607c58e4b1c107f1ba319b59a5e
Parents: 826de63
Author: zhangduo 
Authored: Wed Dec 27 22:03:51 2017 +0800
Committer: zhangduo 
Committed: Thu Jan 4 14:50:49 2018 +0800

--
 .../hbase/replication/ReplicationFactory.java   |   9 +-
 .../hbase/replication/ReplicationQueues.java| 160 ---
 .../replication/ReplicationQueuesArguments.java |  70 ---
 .../replication/ReplicationQueuesZKImpl.java| 407 -
 .../hbase/replication/ReplicationTableBase.java | 442 ---
 .../replication/ReplicationTrackerZKImpl.java   |  21 +-
 .../replication/ZKReplicationQueueStorage.java  |  22 +
 .../replication/TestReplicationStateBasic.java  | 131 +++---
 .../replication/TestReplicationStateZKImpl.java |  41 +-
 .../regionserver/DumpReplicationQueues.java |  15 +-
 .../RecoveredReplicationSource.java |  17 +-
 .../RecoveredReplicationSourceShipper.java  |  22 +-
 .../replication/regionserver/Replication.java   |  41 +-
 .../regionserver/ReplicationSource.java |  23 +-
 .../ReplicationSourceInterface.java |  11 +-
 .../regionserver/ReplicationSourceManager.java  | 261 ++-
 .../regionserver/ReplicationSyncUp.java |  29 +-
 .../hbase/master/cleaner/TestLogsCleaner.java   |  12 +-
 .../cleaner/TestReplicationHFileCleaner.java|  26 +-
 .../cleaner/TestReplicationZKNodeCleaner.java   |  22 +-
 .../replication/ReplicationSourceDummy.java |   6 +-
 .../replication/TestReplicationSyncUpTool.java  |   6 +-
 .../TestReplicationSourceManager.java   | 104 ++---
 .../TestReplicationSourceManagerZkImpl.java |  58 +--
 24 files changed, 385 insertions(+), 1571 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/72f1e971/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
--
diff --git 
a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
 
b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
index 6c1c213..5e70e57 100644
--- 
a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
+++ 
b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
@@ -17,12 +17,11 @@
  */
 package org.apache.hadoop.hbase.replication;
 
-import org.apache.commons.lang3.reflect.ConstructorUtils;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * A factory class for instantiating replication objects that deal with 
replication state.
@@ -30,12 +29,6 @@ import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 @InterfaceAudience.Private
 public class ReplicationFactory {
 
-  public static ReplicationQueues 
getReplicationQueues(ReplicationQueuesArguments args)
-  throws Exception {
-return (ReplicationQueues) 
ConstructorUtils.invokeConstructor(ReplicationQueuesZKImpl.class,
-  args);
-  }
-
   public static ReplicationPeers getReplicationPeers(ZKWatcher zk, 
Configuration conf,
   Abortable abortable) {
 return getReplicationPeers(zk, conf, null, abortable);

http://git-wip-us.apache.org/repos/asf/hbase/blob/72f1e971/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
--
diff --git 
a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
 
b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
deleted file mode 100644
index 7f440b1..000
--- 
a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
+++ /dev/null
@@ -1,160 +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/l

[32/38] hbase git commit: HBASE-19686 Use KeyLocker instead of ReentrantLock in PeerProcedureHandlerImpl

2018-01-04 Thread zhangduo
HBASE-19686 Use KeyLocker instead of ReentrantLock in PeerProcedureHandlerImpl


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

Branch: refs/heads/HBASE-19397
Commit: 018f1885b6f9495ea43ec04aaabdaa2da70b6f7a
Parents: c5e55d7
Author: zhangduo 
Authored: Tue Jan 2 16:13:55 2018 +0800
Committer: zhangduo 
Committed: Thu Jan 4 14:52:27 2018 +0800

--
 .../regionserver/PeerProcedureHandlerImpl.java  | 41 ++--
 1 file changed, 29 insertions(+), 12 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/018f1885/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
index 1efe180..c09c6a0 100644
--- 
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
@@ -19,10 +19,10 @@
 package org.apache.hadoop.hbase.replication.regionserver;
 
 import java.io.IOException;
-import java.util.concurrent.locks.ReentrantLock;
-
+import java.util.concurrent.locks.Lock;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
+import org.apache.hadoop.hbase.util.KeyLocker;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -32,7 +32,7 @@ public class PeerProcedureHandlerImpl implements 
PeerProcedureHandler {
   private static final Logger LOG = 
LoggerFactory.getLogger(PeerProcedureHandlerImpl.class);
 
   private final ReplicationSourceManager replicationSourceManager;
-  private final ReentrantLock peersLock = new ReentrantLock();
+  private final KeyLocker peersLock = new KeyLocker<>();
 
   public PeerProcedureHandlerImpl(ReplicationSourceManager 
replicationSourceManager) {
 this.replicationSourceManager = replicationSourceManager;
@@ -40,40 +40,57 @@ public class PeerProcedureHandlerImpl implements 
PeerProcedureHandler {
 
   @Override
   public void addPeer(String peerId) throws ReplicationException, IOException {
-peersLock.lock();
+Lock peerLock = peersLock.acquireLock(peerId);
 try {
   replicationSourceManager.addPeer(peerId);
 } finally {
-  peersLock.unlock();
+  peerLock.unlock();
 }
   }
 
   @Override
   public void removePeer(String peerId) throws ReplicationException, 
IOException {
-peersLock.lock();
+Lock peerLock = peersLock.acquireLock(peerId);
 try {
   if (replicationSourceManager.getReplicationPeers().getPeer(peerId) != 
null) {
 replicationSourceManager.removePeer(peerId);
   }
 } finally {
-  peersLock.unlock();
+  peerLock.unlock();
 }
   }
 
   @Override
   public void disablePeer(String peerId) throws ReplicationException, 
IOException {
-PeerState newState = 
replicationSourceManager.getReplicationPeers().refreshPeerState(peerId);
-LOG.info("disable replication peer, id: " + peerId + ", new state: " + 
newState);
+PeerState newState;
+Lock peerLock = peersLock.acquireLock(peerId);
+try {
+  newState = 
replicationSourceManager.getReplicationPeers().refreshPeerState(peerId);
+} finally {
+  peerLock.unlock();
+}
+LOG.info("disable replication peer, id: {}, new state: {}", peerId, 
newState);
   }
 
   @Override
   public void enablePeer(String peerId) throws ReplicationException, 
IOException {
-PeerState newState = 
replicationSourceManager.getReplicationPeers().refreshPeerState(peerId);
-LOG.info("enable replication peer, id: " + peerId + ", new state: " + 
newState);
+PeerState newState;
+Lock peerLock = peersLock.acquireLock(peerId);
+try {
+  newState = 
replicationSourceManager.getReplicationPeers().refreshPeerState(peerId);
+} finally {
+  peerLock.unlock();
+}
+LOG.info("enable replication peer, id: {}, new state: {}", peerId, 
newState);
   }
 
   @Override
   public void updatePeerConfig(String peerId) throws ReplicationException, 
IOException {
-replicationSourceManager.getReplicationPeers().refreshPeerConfig(peerId);
+Lock peerLock = peersLock.acquireLock(peerId);
+try {
+  replicationSourceManager.getReplicationPeers().refreshPeerConfig(peerId);
+} finally {
+  peerLock.unlock();
+}

[25/38] hbase git commit: HBASE-19599 Remove ReplicationQueuesClient, use ReplicationQueueStorage directly

2018-01-04 Thread zhangduo
http://git-wip-us.apache.org/repos/asf/hbase/blob/a7c2428b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
index 6e27a21..d8f9625 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
@@ -21,13 +21,13 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Queue;
 import java.util.Set;
 import java.util.stream.Collectors;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileStatus;
@@ -48,17 +48,18 @@ 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.ReplicationQueueInfo;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationQueues;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
+import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.replication.ReplicationTracker;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import 
org.apache.hbase.thirdparty.com.google.common.util.concurrent.AtomicLongMap;
 
 /**
@@ -303,57 +304,53 @@ public class DumpReplicationQueues extends Configured 
implements Tool {
   }
 
   public String dumpQueues(ClusterConnection connection, ZKWatcher zkw, 
Set peerIds,
-   boolean hdfs) throws Exception {
-ReplicationQueuesClient queuesClient;
+  boolean hdfs) throws Exception {
+ReplicationQueueStorage queueStorage;
 ReplicationPeers replicationPeers;
 ReplicationQueues replicationQueues;
 ReplicationTracker replicationTracker;
-ReplicationQueuesClientArguments replicationArgs =
-new ReplicationQueuesClientArguments(getConf(), new 
WarnOnlyAbortable(), zkw);
+ReplicationQueuesArguments replicationArgs =
+new ReplicationQueuesArguments(getConf(), new WarnOnlyAbortable(), 
zkw);
 StringBuilder sb = new StringBuilder();
 
-queuesClient = 
ReplicationFactory.getReplicationQueuesClient(replicationArgs);
-queuesClient.init();
+queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zkw, 
getConf());
 replicationQueues = 
ReplicationFactory.getReplicationQueues(replicationArgs);
-replicationPeers = ReplicationFactory.getReplicationPeers(zkw, getConf(), 
queuesClient, connection);
+replicationPeers =
+ReplicationFactory.getReplicationPeers(zkw, getConf(), queueStorage, 
connection);
 replicationTracker = ReplicationFactory.getReplicationTracker(zkw, 
replicationPeers, getConf(),
   new WarnOnlyAbortable(), new WarnOnlyStoppable());
-List liveRegionServers = 
replicationTracker.getListOfRegionServers();
+Set liveRegionServers = new 
HashSet<>(replicationTracker.getListOfRegionServers());
 
 // Loops each peer on each RS and dumps the queues
-try {
-  List regionservers = queuesClient.getListOfReplicators();
-  if (regionservers == null || regionservers.isEmpty()) {
-return sb.toString();
+List regionservers = queueStorage.getListOfReplicators();
+if (regionservers == null || regionservers.isEmpty()) {
+  return sb.toString();
+}
+for (ServerName regionserver : regionservers) {
+  List queueIds = queueStorage.getAllQueues(regionserver);
+  replicationQueues.init(regionserver.getServerName());
+  if (!liveRegionServers.contains(regionserver.getServerName())) {
+deadRegionServers.add(regionserver.getServerName());
   }
-  for (String regionserver : regionservers) {
-List queueIds = queuesClient.getAllQueues(regionserver);
-replicationQueues.init(regionserver);
-if (!liveRegionServers.contains(regionserver)) {
-  deadRegionServers.add(regionserver);
-}
-for (String 

[36/38] hbase git commit: HBASE-19661 Replace ReplicationStateZKBase with ZKReplicationStorageBase

2018-01-04 Thread zhangduo
HBASE-19661 Replace ReplicationStateZKBase with ZKReplicationStorageBase


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

Branch: refs/heads/HBASE-19397
Commit: ada969af31162247fd5eba7ba38cdbf7ca99c1ce
Parents: 7b0637c
Author: huzheng 
Authored: Fri Dec 29 15:55:28 2017 +0800
Committer: zhangduo 
Committed: Thu Jan 4 14:52:54 2018 +0800

--
 .../hbase/replication/ReplicationFactory.java   |   5 +-
 .../replication/ReplicationStateZKBase.java | 153 ---
 .../replication/ReplicationTrackerZKImpl.java   |  18 ++-
 .../replication/ZKReplicationPeerStorage.java   |  24 ++-
 .../replication/ZKReplicationStorageBase.java   |  13 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |   4 +-
 .../master/ReplicationPeerConfigUpgrader.java   | 128 
 .../regionserver/DumpReplicationQueues.java |  18 +--
 .../replication/regionserver/Replication.java   |   3 +-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java |   3 +-
 .../TestReplicationTrackerZKImpl.java   |   3 +-
 .../replication/master/TestTableCFsUpdater.java |  41 ++---
 .../TestReplicationSourceManager.java   |   6 +-
 13 files changed, 135 insertions(+), 284 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/ada969af/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
--
diff --git 
a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
 
b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
index 6c66aff..2a970ba 100644
--- 
a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
+++ 
b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
@@ -33,9 +33,8 @@ public class ReplicationFactory {
 return new ReplicationPeers(zk, conf);
   }
 
-  public static ReplicationTracker getReplicationTracker(ZKWatcher zookeeper,
-  final ReplicationPeers replicationPeers, Configuration conf, Abortable 
abortable,
+  public static ReplicationTracker getReplicationTracker(ZKWatcher zookeeper, 
Abortable abortable,
   Stoppable stopper) {
-return new ReplicationTrackerZKImpl(zookeeper, replicationPeers, conf, 
abortable, stopper);
+return new ReplicationTrackerZKImpl(zookeeper, abortable, stopper);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ada969af/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
deleted file mode 100644
index f49537c..000
--- 
a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
+++ /dev/null
@@ -1,153 +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.replication;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import 
org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.hbase.thirdparty.com.google.protobuf.CodedOutputStream;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
-import org.apache.hadoop.hbase.zookeeper.ZKConfig;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.yetus.audience.Inte

[27/38] hbase git commit: HBASE-19635 Introduce a thread at RS side to call reportProcedureDone

2018-01-04 Thread zhangduo
HBASE-19635 Introduce a thread at RS side to call reportProcedureDone


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

Branch: refs/heads/HBASE-19397
Commit: 1821fbcf6f1ee6679e12e707652081205a46ddce
Parents: 72f1e97
Author: zhangduo 
Authored: Wed Dec 27 20:13:42 2017 +0800
Committer: zhangduo 
Committed: Thu Jan 4 14:50:49 2018 +0800

--
 .../src/main/protobuf/RegionServerStatus.proto  |   5 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |  15 ++-
 .../hbase/regionserver/HRegionServer.java   |  72 
 .../RemoteProcedureResultReporter.java  | 111 +++
 .../handler/RSProcedureHandler.java |   2 +-
 5 files changed, 149 insertions(+), 56 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/1821fbcf/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 4f75941..3f836cd 100644
--- a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
@@ -146,7 +146,7 @@ message RegionSpaceUseReportRequest {
 message RegionSpaceUseReportResponse {
 }
 
-message ReportProcedureDoneRequest {
+message RemoteProcedureResult {
   required uint64 proc_id = 1;
   enum Status {
 SUCCESS = 1;
@@ -155,6 +155,9 @@ message ReportProcedureDoneRequest {
   required Status status = 2;
   optional ForeignExceptionMessage error = 3;
 }
+message ReportProcedureDoneRequest {
+  repeated RemoteProcedureResult result = 1;
+}
 
 message ReportProcedureDoneResponse {
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1821fbcf/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 c3dbda8..d822aee 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
@@ -264,6 +264,7 @@ 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.RemoteProcedureResult;
 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;
@@ -2253,12 +2254,14 @@ public class MasterRpcServices extends RSRpcServices
   @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(),
-RemoteProcedureException.fromProto(request.getError()));
-}
+request.getResultList().forEach(result -> {
+  if (result.getStatus() == RemoteProcedureResult.Status.SUCCESS) {
+master.remoteProcedureCompleted(result.getProcId());
+  } else {
+master.remoteProcedureFailed(result.getProcId(),
+  RemoteProcedureException.fromProto(result.getError()));
+  }
+});
 return ReportProcedureDoneResponse.getDefaultInstance();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1821fbcf/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 d413d9e..246eecc 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRe

[30/38] hbase git commit: HBASE-19579 Add peer lock test for shell command list_locks

2018-01-04 Thread zhangduo
HBASE-19579 Add peer lock test for shell command list_locks

Signed-off-by: zhangduo 


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

Branch: refs/heads/HBASE-19397
Commit: fc7cff3a35972cc368524b4100e14169aab71292
Parents: a7c2428
Author: Guanghao Zhang 
Authored: Sat Dec 23 21:04:27 2017 +0800
Committer: zhangduo 
Committed: Thu Jan 4 14:50:49 2018 +0800

--
 .../src/main/protobuf/LockService.proto  |  1 +
 .../src/test/ruby/shell/list_locks_test.rb   | 19 +++
 2 files changed, 20 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/fc7cff3a/hbase-protocol-shaded/src/main/protobuf/LockService.proto
--
diff --git a/hbase-protocol-shaded/src/main/protobuf/LockService.proto 
b/hbase-protocol-shaded/src/main/protobuf/LockService.proto
index b8d180c..0675070 100644
--- a/hbase-protocol-shaded/src/main/protobuf/LockService.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/LockService.proto
@@ -77,6 +77,7 @@ enum LockedResourceType {
   NAMESPACE = 2;
   TABLE = 3;
   REGION = 4;
+  PEER = 5;
 }
 
 message LockedResource {

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc7cff3a/hbase-shell/src/test/ruby/shell/list_locks_test.rb
--
diff --git a/hbase-shell/src/test/ruby/shell/list_locks_test.rb 
b/hbase-shell/src/test/ruby/shell/list_locks_test.rb
index f465a6b..ef1c0ce 100644
--- a/hbase-shell/src/test/ruby/shell/list_locks_test.rb
+++ b/hbase-shell/src/test/ruby/shell/list_locks_test.rb
@@ -67,6 +67,25 @@ module Hbase
 proc_id)
 end
 
+define_test 'list peer locks' do
+  lock = create_exclusive_lock(0)
+  peer_id = '1'
+
+  @scheduler.waitPeerExclusiveLock(lock, peer_id)
+  output = capture_stdout { @list_locks.command }
+  @scheduler.wakePeerExclusiveLock(lock, peer_id)
+
+  assert_equal(
+"PEER(1)\n" \
+"Lock type: EXCLUSIVE, procedure: {" \
+  
"\"className\"=>\"org.apache.hadoop.hbase.master.locking.LockProcedure\", " \
+  "\"procId\"=>\"0\", \"submittedTime\"=>\"0\", 
\"state\"=>\"RUNNABLE\", " \
+  "\"lastUpdate\"=>\"0\", " \
+  "\"stateMessage\"=>[{\"lockType\"=>\"EXCLUSIVE\", 
\"description\"=>\"description\"}]" \
+"}\n\n",
+output)
+end
+
 define_test 'list server locks' do
   lock = create_exclusive_lock(0)
 



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

2018-01-04 Thread zhangduo
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/5d32c8c4
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/5d32c8c4
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/5d32c8c4

Branch: refs/heads/HBASE-19397
Commit: 5d32c8c466a1b2e5487c1f65e36353054c4abcfc
Parents: cfdc501
Author: zhangduo 
Authored: Wed Dec 20 20:57:37 2017 +0800
Committer: zhangduo 
Committed: Thu Jan 4 14:50:49 2018 +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/5d32c8c4/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 f2a868b..c3dbda8 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
@@ -1885,10 +1885,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);
 }
@@ -1898,8 +1898,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);
 }
@@ -1909,8 +1909,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);
 }
@@ -1920,8 +1920,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);
 }
@@ -1947,9 +1947,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 Ser

[31/38] hbase git commit: HBASE-19623 Create replication endpoint asynchronously when adding a replication source

2018-01-04 Thread zhangduo
HBASE-19623 Create replication endpoint asynchronously when adding a 
replication source


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

Branch: refs/heads/HBASE-19397
Commit: c5e55d708668ba87424b9c73e0a6141f41d3128d
Parents: fc7a3ed
Author: zhangduo 
Authored: Tue Jan 2 13:25:58 2018 +0800
Committer: zhangduo 
Committed: Thu Jan 4 14:52:27 2018 +0800

--
 .../hbase/replication/ReplicationPeer.java  |   8 ++
 .../hbase/replication/ReplicationPeers.java |  19 +---
 .../replication/ZKReplicationPeerStorage.java   |   7 +-
 .../replication/TestReplicationStateBasic.java  |  20 +---
 .../TestZKReplicationPeerStorage.java   |  14 +--
 .../HBaseInterClusterReplicationEndpoint.java   |  17 ++-
 .../RecoveredReplicationSource.java |  13 +--
 .../regionserver/ReplicationSource.java | 110 +++
 .../ReplicationSourceInterface.java |   8 +-
 .../regionserver/ReplicationSourceManager.java  |  47 +---
 .../client/TestAsyncReplicationAdminApi.java|   2 -
 .../replication/TestReplicationAdmin.java   |   2 -
 .../replication/ReplicationSourceDummy.java |   7 +-
 .../replication/TestReplicationSource.java  |  27 +++--
 .../TestReplicationSourceManager.java   |   8 +-
 15 files changed, 127 insertions(+), 182 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/c5e55d70/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 4846018..2da3cce 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
@@ -54,6 +54,14 @@ public interface ReplicationPeer {
   PeerState getPeerState();
 
   /**
+   * Test whether the peer is enabled.
+   * @return {@code true} if enabled, otherwise {@code false}.
+   */
+  default boolean isPeerEnabled() {
+return getPeerState() == PeerState.ENABLED;
+  }
+
+  /**
* Get the peer config object
* @return the ReplicationPeerConfig for this peer
*/

http://git-wip-us.apache.org/repos/asf/hbase/blob/c5e55d70/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
--
diff --git 
a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
 
b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
index 3d18091..ad3fee7 100644
--- 
a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
+++ 
b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.replication;
 
 import java.io.IOException;
+import java.util.Collections;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
@@ -86,21 +87,6 @@ public class ReplicationPeers {
   }
 
   /**
-   * Get the peer state for the specified connected remote slave cluster. The 
value might be read
-   * from cache, so it is recommended to use {@link #peerStorage } to read 
storage directly if
-   * reading the state after enabling or disabling it.
-   * @param peerId a short that identifies the cluster
-   * @return true if replication is enabled, false otherwise.
-   */
-  public boolean isPeerEnabled(String peerId) {
-ReplicationPeer replicationPeer = this.peerCache.get(peerId);
-if (replicationPeer == null) {
-  throw new IllegalArgumentException("Peer with id= " + peerId + " is not 
cached");
-}
-return replicationPeer.getPeerState() == PeerState.ENABLED;
-  }
-
-  /**
* Returns the ReplicationPeerImpl for the specified cached peer. This 
ReplicationPeer will
* continue to track changes to the Peer's state and config. This method 
returns null if no peer
* has been cached with the given peerId.
@@ -117,7 +103,7 @@ public class ReplicationPeers {
* @return a Set of Strings for peerIds
*/
   public Set getAllPeerIds() {
-return peerCache.keySet();
+return Collections.unmodifiableSet(peerCache.keySet());
   }
 
   public static Configuration 
getPeerClusterConfiguration(ReplicationPeerConfig peerConfig,
@@ -161,7 +147,6 @@ public class ReplicationPeers {
* Helper method

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

2018-01-04 Thread zhangduo
HBASE-19520 Add UTs for the new lock type PEER

Signed-off-by: zhangduo 


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

Branch: refs/heads/HBASE-19397
Commit: 8f6dc6087689849bbc81fd15896e69cd144f4a8a
Parents: 5d32c8c
Author: Guanghao Zhang 
Authored: Wed Dec 20 16:43:38 2017 +0800
Committer: zhangduo 
Committed: Thu Jan 4 14:50:49 2018 +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/8f6dc608/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 8ff2d12..a25217c 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 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/8f6dc608/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)
-  {
+  pr

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

2018-01-04 Thread zhangduo
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/f9123062
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/f9123062
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/f9123062

Branch: refs/heads/HBASE-19397
Commit: f9123062a35d2a942f1c35e636c8ec6b6f22c8df
Parents: 0225340
Author: huzheng 
Authored: Wed Dec 20 10:47:18 2017 +0800
Committer: zhangduo 
Committed: Thu Jan 4 14:50:49 2018 +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 ++-
 .../TestReplicationAdminUsingProcedure.java | 226 +++
 .../replication/DummyModifyPeerProcedure.java   |  48 
 .../TestDummyModifyPeerProcedure.java   |  80 ---
 .../TestReplicationTrackerZKImpl.java   |  51 -
 .../TestReplicationSourceManager.java   |  32 ++-
 21 files changed, 532 insertions(+), 471 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/f9123062/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 d549607..2f2dc86 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 bytes.length 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/f9123062/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 0706129..5b5d500 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;
@@ -279,7 +281,7 @@ public final class ProtobufUtil {
* byte array that is bytes.length plus {@link 
ProtobufMagic#PB_MAG

[01/38] hbase git commit: HBASE-18806 VerifyRep by snapshot need not to restore snapshot for each mapper [Forced Update!]

2018-01-04 Thread zhangduo
Repository: hbase
Updated Branches:
  refs/heads/HBASE-19397 e424657c9 -> f27b9d4d7 (forced update)


HBASE-18806 VerifyRep by snapshot need not to restore snapshot for each mapper


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

Branch: refs/heads/HBASE-19397
Commit: 6e136f26bf0761797716b532b1a8c4984bf80c58
Parents: 9a98bb4
Author: huzheng 
Authored: Thu Sep 14 17:08:16 2017 +0800
Committer: huzheng 
Committed: Thu Jan 4 10:10:03 2018 +0800

--
 .../hbase/mapreduce/TableMapReduceUtil.java |  32 +++---
 .../mapreduce/TableSnapshotInputFormatImpl.java |   6 +-
 .../replication/VerifyReplication.java  |  24 +++-
 .../replication/TestReplicationSmallTests.java  |  19 
 .../hbase/client/TableSnapshotScanner.java  | 112 +--
 .../hbase/client/TestTableSnapshotScanner.java  |  47 
 6 files changed, 185 insertions(+), 55 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/6e136f26/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
--
diff --git 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
index d1101c5..83895fd 100644
--- 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
+++ 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
@@ -345,22 +345,20 @@ public class TableMapReduceUtil {
   }
 
   /**
-   * Sets up the job for reading from a table snapshot. It bypasses hbase 
servers
-   * and read directly from snapshot files.
-   *
+   * Sets up the job for reading from a table snapshot. It bypasses hbase 
servers and read directly
+   * from snapshot files.
* @param snapshotName The name of the snapshot (of a table) to read from.
-   * @param scan  The scan instance with the columns, time range etc.
-   * @param mapper  The mapper class to use.
-   * @param outputKeyClass  The class of the output key.
-   * @param outputValueClass  The class of the output value.
-   * @param job  The current job to adjust.  Make sure the passed job is
-   * carrying all necessary HBase configuration.
-   * @param addDependencyJars upload HBase jars and jars for any of the 
configured
-   *   job classes via the distributed cache (tmpjars).
-   *
+   * @param scan The scan instance with the columns, time range etc.
+   * @param mapper The mapper class to use.
+   * @param outputKeyClass The class of the output key.
+   * @param outputValueClass The class of the output value.
+   * @param job The current job to adjust. Make sure the passed job is 
carrying all necessary HBase
+   *  configuration.
+   * @param addDependencyJars upload HBase jars and jars for any of the 
configured job classes via
+   *  the distributed cache (tmpjars).
* @param tmpRestoreDir a temporary directory to copy the snapshot files 
into. Current user should
-   * have write permissions to this directory, and this should not be a 
subdirectory of rootdir.
-   * After the job is finished, restore directory can be deleted.
+   *  have write permissions to this directory, and this should not be 
a subdirectory of
+   *  rootdir. After the job is finished, restore directory can be 
deleted.
* @throws IOException When setting up the details fails.
* @see TableSnapshotInputFormat
*/
@@ -369,10 +367,10 @@ public class TableMapReduceUtil {
   Class outputKeyClass,
   Class outputValueClass, Job job,
   boolean addDependencyJars, Path tmpRestoreDir)
-  throws IOException {
+  throws IOException {
 TableSnapshotInputFormat.setInput(job, snapshotName, tmpRestoreDir);
-initTableMapperJob(snapshotName, scan, mapper, outputKeyClass,
-outputValueClass, job, addDependencyJars, false, 
TableSnapshotInputFormat.class);
+initTableMapperJob(snapshotName, scan, mapper, outputKeyClass, 
outputValueClass, job,
+  addDependencyJars, false, TableSnapshotInputFormat.class);
 resetCacheConfig(job.getConfiguration());
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6e136f26/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatImpl.java
--
diff --git 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatImpl.java
 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatImpl.java
index 904ed

[12/38] hbase git commit: HBASE-19592 Add UTs to test retry on update zk failure

2018-01-04 Thread zhangduo
HBASE-19592 Add UTs to test retry on update zk failure


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

Branch: refs/heads/HBASE-19397
Commit: 8f717863263812454745ccf4e0b43e0fb7750f1f
Parents: 3fc125c
Author: zhangduo 
Authored: Tue Dec 26 20:39:00 2017 +0800
Committer: zhangduo 
Committed: Thu Jan 4 14:50:49 2018 +0800

--
 .../replication/ReplicationPeerManager.java |   5 +-
 .../TestReplicationProcedureRetry.java  | 200 +++
 2 files changed, 202 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/8f717863/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
index b78cbce..f4ccce8 100644
--- 
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
@@ -53,7 +53,7 @@ import org.apache.yetus.audience.InterfaceAudience;
  * Used to add/remove a replication peer.
  */
 @InterfaceAudience.Private
-public final class ReplicationPeerManager {
+public class ReplicationPeerManager {
 
   private final ReplicationPeerStorage peerStorage;
 
@@ -61,8 +61,7 @@ public final class ReplicationPeerManager {
 
   private final ConcurrentMap peers;
 
-  private ReplicationPeerManager(ReplicationPeerStorage peerStorage,
-  ReplicationQueueStorage queueStorage,
+  ReplicationPeerManager(ReplicationPeerStorage peerStorage, 
ReplicationQueueStorage queueStorage,
   ConcurrentMap peers) {
 this.peerStorage = peerStorage;
 this.queueStorage = queueStorage;

http://git-wip-us.apache.org/repos/asf/hbase/blob/8f717863/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java
--
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java
new file mode 100644
index 000..ab35b46
--- /dev/null
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java
@@ -0,0 +1,200 @@
+/**
+ * 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.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.spy;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.zookeeper.KeeperException;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.invocation.InvocationOnMock;
+
+/**
+ * All the modification method will fail once in the test and should finally 
succeed.
+ */
+@Category({ ReplicationTests

[34/38] hbase git commit: HBASE-19633 Clean up the replication queues in the postPeerModification stage when removing a peer

2018-01-04 Thread zhangduo
HBASE-19633 Clean up the replication queues in the postPeerModification stage 
when removing a peer


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

Branch: refs/heads/HBASE-19397
Commit: fc7a3ed3e55308e3957a377eb9ef1db0b0734959
Parents: aed5fe3
Author: zhangduo 
Authored: Tue Jan 2 09:57:23 2018 +0800
Committer: zhangduo 
Committed: Thu Jan 4 14:52:27 2018 +0800

--
 .../replication/ReplicationPeerConfig.java  |  2 +-
 .../replication/VerifyReplication.java  | 34 ++---
 .../hbase/replication/ReplicationPeers.java | 35 +++---
 .../replication/ZKReplicationQueueStorage.java  |  3 +-
 .../replication/ZKReplicationStorageBase.java   |  4 +-
 .../replication/TestReplicationStateBasic.java  | 10 +
 .../org/apache/hadoop/hbase/master/HMaster.java |  4 +-
 .../master/replication/AddPeerProcedure.java|  5 +--
 .../replication/DisablePeerProcedure.java   |  3 +-
 .../master/replication/EnablePeerProcedure.java |  3 +-
 .../master/replication/ModifyPeerProcedure.java | 34 +
 .../replication/RefreshPeerProcedure.java   | 17 -
 .../master/replication/RemovePeerProcedure.java |  7 ++--
 .../replication/ReplicationPeerManager.java | 31 +++-
 .../replication/UpdatePeerConfigProcedure.java  |  3 +-
 .../hbase/regionserver/HRegionServer.java   | 18 -
 .../RemoteProcedureResultReporter.java  |  3 +-
 .../regionserver/RefreshPeerCallable.java   |  5 ++-
 .../regionserver/ReplicationSourceManager.java  | 39 +++-
 .../TestReplicationAdminUsingProcedure.java |  7 ++--
 20 files changed, 135 insertions(+), 132 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/fc7a3ed3/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
--
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
index ab75dff..c6d0fae 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
@@ -27,8 +27,8 @@ import java.util.Set;
 import java.util.TreeMap;
 
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * A configuration for the replication peer cluster.

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc7a3ed3/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
--
diff --git 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
index f0070f0..fe45762 100644
--- 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
+++ 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.mapreduce.replication;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.UUID;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
@@ -45,13 +44,14 @@ import org.apache.hadoop.hbase.filter.PrefixFilter;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.mapreduce.TableInputFormat;
 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
-import org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormat;
 import org.apache.hadoop.hbase.mapreduce.TableMapper;
+import org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormat;
 import org.apache.hadoop.hbase.mapreduce.TableSplit;
 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.ReplicationPeerStorage;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FS

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

2018-01-04 Thread zhangduo
http://git-wip-us.apache.org/repos/asf/hbase/blob/9ff35d31/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..000
--- 
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.
- * 
- * 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 listReplicationPeers(Pattern pattern)
-  throws ReplicationException {
-List peers = new ArrayList<>();
-List peerIds = replicationPeers.getAllPeerIds();
-for (String peerId : peer

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

2018-01-04 Thread zhangduo
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/02253407
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/02253407
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/02253407

Branch: refs/heads/HBASE-19397
Commit: 0225340701e724fe781914460814c4c9d36700a7
Parents: 8f6dc60
Author: zhangduo 
Authored: Thu Dec 21 21:59:46 2017 +0800
Committer: zhangduo 
Committed: Thu Jan 4 14:50:49 2018 +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/02253407/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/02253407/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/02253407/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
--
diff --git 
a/hbase-server/src/main/java/org

[16/38] hbase git commit: HBASE-19642 Fix locking for peer modification procedure

2018-01-04 Thread zhangduo
HBASE-19642 Fix locking for peer modification procedure


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

Branch: refs/heads/HBASE-19397
Commit: 826de635cb2dbe923f1c583e4b4817dc17b075a1
Parents: 8f71786
Author: zhangduo 
Authored: Wed Dec 27 18:27:13 2017 +0800
Committer: zhangduo 
Committed: Thu Jan 4 14:50:49 2018 +0800

--
 .../procedure/MasterProcedureScheduler.java | 14 +
 .../master/replication/ModifyPeerProcedure.java | 21 +---
 2 files changed, 32 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/826de635/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 a25217c..4ecb3b1 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
@@ -610,6 +610,20 @@ public class MasterProcedureScheduler extends 
AbstractProcedureScheduler {
 public boolean requireExclusiveLock(Procedure proc) {
   return requirePeerExclusiveLock((PeerProcedureInterface) proc);
 }
+
+@Override
+public boolean isAvailable() {
+  if (isEmpty()) {
+return false;
+  }
+  if (getLockStatus().hasExclusiveLock()) {
+// if we have an exclusive lock already taken
+// only child of the lock owner can be executed
+Procedure nextProc = peek();
+return nextProc != null && getLockStatus().hasLockAccess(nextProc);
+  }
+  return true;
+}
   }
 
   // 


http://git-wip-us.apache.org/repos/asf/hbase/blob/826de635/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 279fbc7..a682606 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
@@ -46,6 +46,8 @@ public abstract class ModifyPeerProcedure
 
   protected String peerId;
 
+  private volatile boolean locked;
+
   // used to keep compatible with old client where we can only returns after 
updateStorage.
   protected ProcedurePrepareLatch latch;
 
@@ -145,17 +147,30 @@ public abstract class ModifyPeerProcedure
 
   @Override
   protected LockState acquireLock(MasterProcedureEnv env) {
-return env.getProcedureScheduler().waitPeerExclusiveLock(this, peerId)
-  ? LockState.LOCK_EVENT_WAIT
-  : LockState.LOCK_ACQUIRED;
+if (env.getProcedureScheduler().waitPeerExclusiveLock(this, peerId)) {
+  return  LockState.LOCK_EVENT_WAIT;
+}
+locked = true;
+return LockState.LOCK_ACQUIRED;
   }
 
   @Override
   protected void releaseLock(MasterProcedureEnv env) {
+locked = false;
 env.getProcedureScheduler().wakePeerExclusiveLock(this, peerId);
   }
 
   @Override
+  protected boolean holdLock(MasterProcedureEnv env) {
+return true;
+  }
+
+  @Override
+  protected boolean hasLock(MasterProcedureEnv env) {
+return locked;
+  }
+
+  @Override
   protected void rollbackState(MasterProcedureEnv env, PeerModificationState 
state)
   throws IOException, InterruptedException {
 if (state == PeerModificationState.PRE_PEER_MODIFICATION) {



[02/38] hbase git commit: HBASE-19613 Miscellaneous changes to WALSplitter.

2018-01-04 Thread zhangduo
HBASE-19613 Miscellaneous changes to WALSplitter.

* Use ArrayList instead LinkedList
* Use Apache Commons where appropriate
* Parameterize and improve logging


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

Branch: refs/heads/HBASE-19397
Commit: 301062566ac6e32d5bc3c6dbfd819b5e62742e8c
Parents: 6e136f2
Author: BELUGA BEHR 
Authored: Wed Jan 3 18:29:09 2018 -0800
Committer: Apekshit Sharma 
Committed: Wed Jan 3 18:30:10 2018 -0800

--
 .../apache/hadoop/hbase/wal/WALSplitter.java| 163 +--
 1 file changed, 75 insertions(+), 88 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/30106256/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
index 328390e..2aad203 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
@@ -24,9 +24,9 @@ import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.text.ParseException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableSet;
@@ -48,6 +48,9 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.collections.MapUtils;
+import org.apache.commons.lang3.ArrayUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileStatus;
@@ -86,14 +89,14 @@ import org.apache.hadoop.hbase.wal.WALProvider.Writer;
 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
 import org.apache.hadoop.io.MultipleIOException;
 import org.apache.hadoop.ipc.RemoteException;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import 
org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry;
 import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType;
 import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
@@ -203,7 +206,7 @@ public class WALSplitter {
 final FileStatus[] logfiles = SplitLogManager.getFileList(conf,
 Collections.singletonList(logDir), null);
 List splits = new ArrayList<>();
-if (logfiles != null && logfiles.length > 0) {
+if (ArrayUtils.isNotEmpty(logfiles)) {
   for (FileStatus logfile: logfiles) {
 WALSplitter s = new WALSplitter(factory, conf, rootDir, fs, null, 
null);
 if (s.splitLogFile(logfile, null)) {
@@ -245,7 +248,7 @@ public class WALSplitter {
 this.fileBeingSplit = logfile;
 try {
   long logLength = logfile.getLen();
-  LOG.info("Splitting WAL=" + logPath + ", length=" + logLength);
+  LOG.info("Splitting WAL={}, length={}", logPath, logLength);
   status.setStatus("Opening log file");
   if (reporter != null && !reporter.progress()) {
 progress_failed = true;
@@ -253,7 +256,7 @@ public class WALSplitter {
   }
   logFileReader = getReader(logfile, skipErrors, reporter);
   if (logFileReader == null) {
-LOG.warn("Nothing to split in WAL=" + logPath);
+LOG.warn("Nothing to split in WAL={}", logPath);
 return true;
   }
   int numOpenedFilesBeforeReporting = 
conf.getInt("hbase.splitlog.report.openedfiles", 3);
@@ -317,7 +320,7 @@ public class WALSplitter {
   iie.initCause(ie);
   throw iie;
 } catch (CorruptedLogFileException e) {
-  LOG.warn("Could not parse, corrupted WAL=" + logPath, e);
+  LOG.warn("Could not parse, corrupted WAL={}", logPath, e);
   if (splitLogWorkerCoordination != null) {
 // Some tests pass in a csm of null.
 splitLogWorkerCoordination.markCorrupted(rootDir, 
logfile.getPath().g

[33/38] hbase git commit: HBASE-19544 Add UTs for testing concurrent modifications on replication peer

2018-01-04 Thread zhangduo
HBASE-19544 Add UTs for testing concurrent modifications on replication peer

Signed-off-by: zhangduo 


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

Branch: refs/heads/HBASE-19397
Commit: 421099e3a806530cf38997f8cbacef4d6d1d2402
Parents: 018f188
Author: Guanghao Zhang 
Authored: Tue Jan 2 17:07:41 2018 +0800
Committer: zhangduo 
Committed: Thu Jan 4 14:52:27 2018 +0800

--
 .../replication/TestReplicationAdmin.java   | 69 
 1 file changed, 69 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/421099e3/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 9b71595..89cf393 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
@@ -31,6 +31,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.regex.Pattern;
 
 import org.apache.hadoop.conf.Configuration;
@@ -55,6 +56,8 @@ 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;
 
 /**
  * Unit testing of ReplicationAdmin
@@ -62,6 +65,8 @@ import org.junit.rules.TestName;
 @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();
 
@@ -111,6 +116,70 @@ public class TestReplicationAdmin {
   }
 
   @Test
+  public void testConcurrentPeerOperations() throws Exception {
+int threadNum = 5;
+AtomicLong successCount = new AtomicLong(0);
+
+// Test concurrent add peer operation
+Thread[] addPeers = new Thread[threadNum];
+for (int i = 0; i < threadNum; i++) {
+  addPeers[i] = new Thread(() -> {
+try {
+  hbaseAdmin.addReplicationPeer(ID_ONE,
+ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build());
+  successCount.incrementAndGet();
+} catch (Exception e) {
+  LOG.debug("Got exception when add replication peer", e);
+}
+  });
+  addPeers[i].start();
+}
+for (Thread addPeer : addPeers) {
+  addPeer.join();
+}
+assertEquals(1, successCount.get());
+
+// Test concurrent remove peer operation
+successCount.set(0);
+Thread[] removePeers = new Thread[threadNum];
+for (int i = 0; i < threadNum; i++) {
+  removePeers[i] = new Thread(() -> {
+try {
+  hbaseAdmin.removeReplicationPeer(ID_ONE);
+  successCount.incrementAndGet();
+} catch (Exception e) {
+  LOG.debug("Got exception when remove replication peer", e);
+}
+  });
+  removePeers[i].start();
+}
+for (Thread removePeer : removePeers) {
+  removePeer.join();
+}
+assertEquals(1, successCount.get());
+
+// Test concurrent add peer operation again
+successCount.set(0);
+addPeers = new Thread[threadNum];
+for (int i = 0; i < threadNum; i++) {
+  addPeers[i] = new Thread(() -> {
+try {
+  hbaseAdmin.addReplicationPeer(ID_ONE,
+ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build());
+  successCount.incrementAndGet();
+} catch (Exception e) {
+  LOG.debug("Got exception when add replication peer", e);
+}
+  });
+  addPeers[i].start();
+}
+for (Thread addPeer : addPeers) {
+  addPeer.join();
+}
+assertEquals(1, successCount.get());
+  }
+
+  @Test
   public void testAddInvalidPeer() {
 ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder();
 builder.setClusterKey(KEY_ONE);



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

2018-01-04 Thread zhangduo
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/e11fea13
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/e11fea13
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/e11fea13

Branch: refs/heads/HBASE-19397
Commit: e11fea13b99a4a2edd1058eedde82bb0fdecb787
Parents: 5195435
Author: zhangduo 
Authored: Fri Dec 15 21:06:44 2017 +0800
Committer: zhangduo 
Committed: Thu Jan 4 14:50:49 2018 +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 | 101 +
 .../master/replication/ModifyPeerProcedure.java | 127 +++
 .../master/replication/RefreshPeerCallable.java |  67 ++
 .../replication/RefreshPeerProcedure.java   | 197 
 .../hbase/procedure2/RSProcedureCallable.java   |  43 
 .../hbase/regionserver/HRegionServer.java   |  90 ++--
 .../hbase/regionserver/RSRpcServices.java   |  56 +++--
 .../handler/RSProcedureHandler.java |  51 +
 .../assignment/TestAssignmentManager.java   |  20 +-
 .../replication/DummyModifyPeerProcedure.java   |  41 
 .../TestDummyModifyPeerProcedure.java   |  80 +++
 .../security/access/TestAccessController.java   |   1 +
 24 files changed, 1122 insertions(+), 184 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/e11fea13/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/e11fea13/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 8bbfcec..02676a8 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
-   * @param 
*/
   public interface RemoteProcedure {
+/**
+ * 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/e11fea13/hbase-protocol-shaded/src/main/protobuf/Admin.proto

[03/38] hbase git commit: HBASE-19490 Rare failure in TestRateLimiter

2018-01-04 Thread zhangduo
HBASE-19490 Rare failure in TestRateLimiter


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

Branch: refs/heads/HBASE-19397
Commit: 338a74e73705fd7c80111ade47345b2a6efe11e7
Parents: 3010625
Author: Chia-Ping Tsai 
Authored: Wed Jan 3 03:19:07 2018 +0800
Committer: Chia-Ping Tsai 
Committed: Thu Jan 4 10:46:43 2018 +0800

--
 .../apache/hadoop/hbase/quotas/TestRateLimiter.java  | 15 +--
 1 file changed, 13 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/338a74e7/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRateLimiter.java
--
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRateLimiter.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRateLimiter.java
index e205f9b..567577b 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRateLimiter.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestRateLimiter.java
@@ -23,10 +23,10 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.util.concurrent.TimeUnit;
-
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.EnvironmentEdge;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -122,6 +122,16 @@ public class TestRateLimiter {
 RateLimiter limiter = new FixedIntervalRateLimiter();
 limiter.set(10, TimeUnit.SECONDS);
 
+// fix the current time in order to get the precise value of interval
+EnvironmentEdge edge = new EnvironmentEdge() {
+  private final long ts = System.currentTimeMillis();
+
+  @Override
+  public long currentTime() {
+return ts;
+  }
+};
+EnvironmentEdgeManager.injectEdge(edge);
 // 10 resources are available, but we need to consume 20 resources
 // Verify that we have to wait at least 1.1sec to have 1 resource available
 assertTrue(limiter.canExecute());
@@ -130,6 +140,7 @@ public class TestRateLimiter {
 assertEquals(1000, limiter.waitInterval(1));
 // To consume 10 resource wait for 100ms
 assertEquals(1000, limiter.waitInterval(10));
+EnvironmentEdgeManager.reset();
 
 limiter.setNextRefillTime(limiter.getNextRefillTime() - 900);
 // Verify that after 1sec also no resource should be available



[35/38] hbase git commit: HBASE-19697 Remove TestReplicationAdminUsingProcedure

2018-01-04 Thread zhangduo
HBASE-19697 Remove TestReplicationAdminUsingProcedure


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

Branch: refs/heads/HBASE-19397
Commit: 8f1656d8ddf8dd31509173ad748c0bc6701b8fb9
Parents: ada969a
Author: zhangduo 
Authored: Wed Jan 3 21:13:57 2018 +0800
Committer: zhangduo 
Committed: Thu Jan 4 14:52:54 2018 +0800

--
 .../TestReplicationAdminUsingProcedure.java | 225 ---
 1 file changed, 225 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/8f1656d8/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
deleted file mode 100644
index 1300376..000
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminUsingProcedure.java
+++ /dev/null
@@ -1,225 +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.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.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;
-
-import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
-import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
-
-@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.se

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

2018-01-04 Thread zhangduo
HBASE-19536 Client side changes for moving peer modification from zk watcher to 
procedure

Signed-off-by: zhangduo 


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

Branch: refs/heads/HBASE-19397
Commit: cfdc501e2038c62e9eb5a63c37f8ccbd223791dd
Parents: de3a838
Author: Guanghao Zhang 
Authored: Tue Dec 19 15:50:57 2017 +0800
Committer: zhangduo 
Committed: Thu Jan 4 14:50:49 2018 +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/cfdc501e/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 f61b32e..6729473 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
@@ -2466,7 +2466,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)
@@ -2477,7 +2477,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
*/
@@ -2485,6 +2485,37 @@ public interface Admin extends Abortable, Closeable {
   throws IOException;
 
   /**
+   * Add a new replication peer but does not block and wait for it.
+   * 
+   * 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 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.
+   * 
+   * 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 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
@@ -2492,6 +2523,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.
+   * 
+   * 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 tha

[10/38] hbase git commit: HBASE-19573 Rewrite ReplicationPeer with the new replication storage interface

2018-01-04 Thread zhangduo
HBASE-19573 Rewrite ReplicationPeer with the new replication storage interface


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

Branch: refs/heads/HBASE-19397
Commit: c2464ae58b2770878c1230e2d7c4cf5c862dacfa
Parents: fc7cff3
Author: Guanghao Zhang 
Authored: Tue Dec 26 11:39:34 2017 +0800
Committer: zhangduo 
Committed: Thu Jan 4 14:50:49 2018 +0800

--
 .../replication/VerifyReplication.java  |   5 -
 .../hbase/replication/ReplicationPeer.java  |  42 ++--
 .../hbase/replication/ReplicationPeerImpl.java  | 170 ++
 .../replication/ReplicationPeerZKImpl.java  | 233 ---
 .../hbase/replication/ReplicationPeers.java |   4 +-
 .../replication/ReplicationPeersZKImpl.java |  23 +-
 .../replication/TestReplicationStateBasic.java  |   7 +-
 .../regionserver/PeerProcedureHandlerImpl.java  |  29 +--
 8 files changed, 217 insertions(+), 296 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/c2464ae5/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
--
diff --git 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
index 9065f4e..09d4b4b 100644
--- 
a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
+++ 
b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
@@ -51,7 +51,6 @@ import org.apache.hadoop.hbase.mapreduce.TableSplit;
 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.ReplicationPeerZKImpl;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -333,7 +332,6 @@ public class VerifyReplication extends Configured 
implements Tool {
   private static Pair 
getPeerQuorumConfig(
   final Configuration conf, String peerId) throws IOException {
 ZKWatcher localZKW = null;
-ReplicationPeerZKImpl peer = null;
 try {
   localZKW = new ZKWatcher(conf, "VerifyReplication",
   new Abortable() {
@@ -354,9 +352,6 @@ public class VerifyReplication extends Configured 
implements Tool {
   throw new IOException(
   "An error occurred while trying to connect to the remove peer 
cluster", e);
 } finally {
-  if (peer != null) {
-peer.close();
-  }
   if (localZKW != null) {
 localZKW.close();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c2464ae5/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 b66d76d..4846018 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
@@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.yetus.audience.InterfaceAudience;
 
-
 /**
  * ReplicationPeer manages enabled / disabled state for the peer.
  */
@@ -49,65 +48,52 @@ public interface ReplicationPeer {
   String getId();
 
   /**
-   * Get the peer config object
-   * @return the ReplicationPeerConfig for this peer
-   */
-  public ReplicationPeerConfig getPeerConfig();
-
-  /**
-   * 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 

[06/38] hbase git commit: HBASE-19596 RegionMetrics/ServerMetrics/ClusterMetrics should apply to all public classes

2018-01-04 Thread zhangduo
http://git-wip-us.apache.org/repos/asf/hbase/blob/8119acfc/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java
--
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java
index 58ae059..59a0059 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java
@@ -19,16 +19,16 @@ package org.apache.hadoop.hbase;
 
 import java.io.Closeable;
 import java.io.IOException;
-
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.util.Threads;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
+
 import 
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
 import 
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
 import 
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
-import org.apache.hadoop.hbase.util.Threads;
 
 /**
  * This class defines methods that can help with managing HBase clusters
@@ -61,7 +61,7 @@ public abstract class HBaseCluster implements Closeable, 
Configurable {
   protected Configuration conf;
 
   /** the status of the cluster before we begin */
-  protected ClusterStatus initialClusterStatus;
+  protected ClusterMetrics initialClusterStatus;
 
   /**
* Construct an HBaseCluster
@@ -82,16 +82,16 @@ public abstract class HBaseCluster implements Closeable, 
Configurable {
   }
 
   /**
-   * Returns a ClusterStatus for this HBase cluster.
-   * @see #getInitialClusterStatus()
+   * Returns a ClusterMetrics for this HBase cluster.
+   * @see #getInitialClusterMetrics()
*/
-  public abstract ClusterStatus getClusterStatus() throws IOException;
+  public abstract ClusterMetrics getClusterMetrics() throws IOException;
 
   /**
* Returns a ClusterStatus for this HBase cluster as observed at the
* starting of the HBaseCluster
*/
-  public ClusterStatus getInitialClusterStatus() throws IOException {
+  public ClusterMetrics getInitialClusterMetrics() throws IOException {
 return initialClusterStatus;
   }
 
@@ -153,7 +153,7 @@ public abstract class HBaseCluster implements Closeable, 
Configurable {
   throws IOException {
 long start = System.currentTimeMillis();
 while ((System.currentTimeMillis() - start) < timeout) {
-  for (ServerName server : getClusterStatus().getServers()) {
+  for (ServerName server : 
getClusterMetrics().getLiveServerMetrics().keySet()) {
 if (server.getHostname().equals(hostname) && server.getPort() == port) 
{
   return;
 }
@@ -317,7 +317,7 @@ public abstract class HBaseCluster implements Closeable, 
Configurable {
* @return whether restoration is complete
*/
   public boolean restoreInitialStatus() throws IOException {
-return restoreClusterStatus(getInitialClusterStatus());
+return restoreClusterMetrics(getInitialClusterMetrics());
   }
 
   /**
@@ -327,7 +327,7 @@ public abstract class HBaseCluster implements Closeable, 
Configurable {
* permissions, etc. restoration might be partial.
* @return whether restoration is complete
*/
-  public boolean restoreClusterStatus(ClusterStatus desiredStatus) throws 
IOException {
+  public boolean restoreClusterMetrics(ClusterMetrics desiredStatus) throws 
IOException {
 return true;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/8119acfc/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 9e17a79..304b3cb 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
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import edu.umd.cs.findbugs.annotations.Nullable;
-
 import java.io.File;
 import java.io.IOException;
 import java.io.OutputStream;
@@ -52,7 +51,6 @@ import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.stream.Collectors;
-
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.commons.logging.impl.Jdk14Logger;
@@ -151,6 +149,7 @@ import org.apache.zookeeper.ZooKeeper.States;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.impl.Log4jLoggerAdapter;
+
 import 

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

2018-01-04 Thread zhangduo
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/de3a838f
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/de3a838f
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/de3a838f

Branch: refs/heads/HBASE-19397
Commit: de3a838ff5014a8f2c079691c98dbe83ab8edf14
Parents: e11fea1
Author: zhangduo 
Authored: Mon Dec 18 15:22:36 2017 +0800
Committer: zhangduo 
Committed: Thu Jan 4 14:50:49 2018 +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   |  13 +--
 .../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, 737 insertions(+), 226 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/de3a838f/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 02676a8..bdff1ca 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 RemoteProcedureDispatcherhttp://git-wip-us.apache.org/repos/asf/hbase/blob/de3a838f/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/de3a838f/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-sh

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

2018-01-04 Thread zhangduo
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/9ff35d31
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/9ff35d31
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/9ff35d31

Branch: refs/heads/HBASE-19397
Commit: 9ff35d31b2729444e9e2eec2a473ec15c8bb9831
Parents: f912306
Author: zhangduo 
Authored: Fri Dec 22 14:37:28 2017 +0800
Committer: zhangduo 
Committed: Thu Jan 4 14:50:49 2018 +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   |  62 ++-
 .../hbase/master/MockNoopMasterServices.java|  12 +-
 .../hbase/master/TestMasterNoCluster.java   |   4 +-
 .../TestReplicationDisableInactivePeer.java |   6 +-
 26 files changed, 1750 insertions(+), 312 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/9ff35d31/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  List nullToEmpty(List 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/9ff35d31/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 @@
   org.apache.hbase
   hbase-zookeeper
 
+
+  org.apache.hbase
+  hbase-common
+  test-jar
+  test
+
+
+  org.apache.hbase
+  hbase-zookeeper
+  test-jar
+  test
+
 
 
   org.apache.commons

http://git-wip-us.apache.org/repos/asf/hbase/blob/9ff35d31/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 000..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:/

[26/38] hbase git commit: HBASE-19599 Remove ReplicationQueuesClient, use ReplicationQueueStorage directly

2018-01-04 Thread zhangduo
HBASE-19599 Remove ReplicationQueuesClient, use ReplicationQueueStorage directly


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

Branch: refs/heads/HBASE-19397
Commit: a7c2428be1f704b503cf15b5b6b045c9eb213a40
Parents: 9ff35d3
Author: zhangduo 
Authored: Mon Dec 25 18:49:56 2017 +0800
Committer: zhangduo 
Committed: Thu Jan 4 14:50:49 2018 +0800

--
 .../hbase/replication/ReplicationFactory.java   |  19 +-
 .../replication/ReplicationPeersZKImpl.java |  24 +-
 .../replication/ReplicationQueueStorage.java|  26 +-
 .../replication/ReplicationQueuesClient.java|  93 -
 .../ReplicationQueuesClientArguments.java   |  40 --
 .../ReplicationQueuesClientZKImpl.java  | 176 -
 .../replication/ZKReplicationQueueStorage.java  |  90 -
 .../replication/TestReplicationStateBasic.java  | 378 +++
 .../replication/TestReplicationStateZKImpl.java | 148 
 .../TestZKReplicationQueueStorage.java  |  74 
 .../cleaner/ReplicationZKNodeCleaner.java   |  71 ++--
 .../cleaner/ReplicationZKNodeCleanerChore.java  |   5 +-
 .../replication/ReplicationPeerManager.java |  31 +-
 .../master/ReplicationHFileCleaner.java | 109 ++
 .../master/ReplicationLogCleaner.java   |  35 +-
 .../regionserver/DumpReplicationQueues.java |  78 ++--
 .../hbase/util/hbck/ReplicationChecker.java |  14 +-
 .../client/TestAsyncReplicationAdminApi.java|  31 +-
 .../replication/TestReplicationAdmin.java   |   2 +
 .../hbase/master/cleaner/TestLogsCleaner.java   |  30 +-
 .../cleaner/TestReplicationHFileCleaner.java|  59 +--
 .../cleaner/TestReplicationZKNodeCleaner.java   |  12 +-
 .../replication/TestReplicationStateBasic.java  | 378 ---
 .../replication/TestReplicationStateZKImpl.java | 227 ---
 .../TestReplicationSourceManagerZkImpl.java |  84 ++---
 25 files changed, 907 insertions(+), 1327 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/a7c2428b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
--
diff --git 
a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
 
b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
index 9f4ad18..6c1c213 100644
--- 
a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
+++ 
b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.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
@@ -37,20 +36,14 @@ public class ReplicationFactory {
   args);
   }
 
-  public static ReplicationQueuesClient
-  getReplicationQueuesClient(ReplicationQueuesClientArguments args) throws 
Exception {
-return (ReplicationQueuesClient) ConstructorUtils
-.invokeConstructor(ReplicationQueuesClientZKImpl.class, args);
-  }
-
-  public static ReplicationPeers getReplicationPeers(final ZKWatcher zk, 
Configuration conf,
- Abortable abortable) {
+  public static ReplicationPeers getReplicationPeers(ZKWatcher zk, 
Configuration conf,
+  Abortable abortable) {
 return getReplicationPeers(zk, conf, null, abortable);
   }
 
-  public static ReplicationPeers getReplicationPeers(final ZKWatcher zk, 
Configuration conf,
- final 
ReplicationQueuesClient queuesClient, Abortable abortable) {
-return new ReplicationPeersZKImpl(zk, conf, queuesClient, abortable);
+  public static ReplicationPeers getReplicationPeers(ZKWatcher zk, 
Configuration conf,
+  ReplicationQueueStorage queueStorage, Abortable abortable) {
+return new ReplicationPeersZKImpl(zk, conf, queueStorage, abortable);
   }
 
   public static ReplicationTracker getReplicationTracker(ZKWatcher zookeeper,

http://git-wip-us.apache.org/repos/asf/hbase/blob/a7c2428b/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 8e2c5f4..f2e5647 100644
--- 
a/hbase-replication/src/main/java/org/apache/hadoop/hbase/repli

[14/38] hbase git commit: HBASE-19622 Reimplement ReplicationPeers with the new replication storage interface

2018-01-04 Thread zhangduo
http://git-wip-us.apache.org/repos/asf/hbase/blob/aed5fe35/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 853bafb..24a4f30 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
@@ -166,7 +166,6 @@ public class ReplicationSourceManager implements 
ReplicationListener {
 this.clusterId = clusterId;
 this.walFileLengthProvider = walFileLengthProvider;
 this.replicationTracker.registerListener(this);
-this.replicationPeers.getAllPeerIds();
 // It's preferable to failover 1 RS at a time, but with good zk servers
 // more could be processed at the same time.
 int nbWorkers = conf.getInt("replication.executor.workers", 1);
@@ -270,8 +269,8 @@ public class ReplicationSourceManager implements 
ReplicationListener {
 }
 List otherRegionServers = 
replicationTracker.getListOfRegionServers().stream()
 .map(ServerName::valueOf).collect(Collectors.toList());
-LOG.info(
-  "Current list of replicators: " + currentReplicators + " other RSs: " + 
otherRegionServers);
+LOG.info("Current list of replicators: " + currentReplicators + " other 
RSs: "
++ otherRegionServers);
 
 // Look if there's anything to process after a restart
 for (ServerName rs : currentReplicators) {
@@ -288,7 +287,7 @@ public class ReplicationSourceManager implements 
ReplicationListener {
* The returned future is for adoptAbandonedQueues task.
*/
   Future init() throws IOException, ReplicationException {
-for (String id : this.replicationPeers.getConnectedPeerIds()) {
+for (String id : this.replicationPeers.getAllPeerIds()) {
   addSource(id);
   if (replicationForBulkLoadDataEnabled) {
 // Check if peer exists in hfile-refs queue, if not add it. This can 
happen in the case
@@ -307,8 +306,8 @@ public class ReplicationSourceManager implements 
ReplicationListener {
*/
   @VisibleForTesting
   ReplicationSourceInterface addSource(String id) throws IOException, 
ReplicationException {
-ReplicationPeerConfig peerConfig = 
replicationPeers.getReplicationPeerConfig(id);
-ReplicationPeer peer = replicationPeers.getConnectedPeer(id);
+ReplicationPeerConfig peerConfig = replicationPeers.getPeerConfig(id);
+ReplicationPeer peer = replicationPeers.getPeer(id);
 ReplicationSourceInterface src = getReplicationSource(id, peerConfig, 
peer);
 synchronized (this.walsById) {
   this.sources.add(src);
@@ -354,7 +353,7 @@ public class ReplicationSourceManager implements 
ReplicationListener {
   public void deleteSource(String peerId, boolean closeConnection) {
 abortWhenFail(() -> this.queueStorage.removeQueue(server.getServerName(), 
peerId));
 if (closeConnection) {
-  this.replicationPeers.peerDisconnected(peerId);
+  this.replicationPeers.removePeer(peerId);
 }
   }
 
@@ -445,12 +444,12 @@ public class ReplicationSourceManager implements 
ReplicationListener {
 // update replication queues on ZK
 // synchronize on replicationPeers to avoid adding source for the 
to-be-removed peer
 synchronized (replicationPeers) {
-  for (String id : replicationPeers.getConnectedPeerIds()) {
+  for (String id : replicationPeers.getAllPeerIds()) {
 try {
   this.queueStorage.addWAL(server.getServerName(), id, logName);
 } catch (ReplicationException e) {
-  throw new IOException("Cannot add log to replication queue" +
-" when creating a new source, queueId=" + id + ", filename=" + 
logName, e);
+  throw new IOException("Cannot add log to replication queue"
+  + " when creating a new source, queueId=" + id + ", filename=" + 
logName, e);
 }
   }
 }
@@ -593,7 +592,7 @@ public class ReplicationSourceManager implements 
ReplicationListener {
 
   public void addPeer(String id) throws ReplicationException, IOException {
 LOG.info("Trying to add peer, peerId: " + id);
-boolean added = this.replicationPeers.peerConnected(id);
+boolean added = this.replicationPeers.addPeer(id);
 if (added) {
   LOG.info("Peer " + id + " connected success, trying to start the 
replication source thread.");
   addSource(id);
@@ -729,19 +728,25 @@ public class ReplicationSourceManager implements 
ReplicationListener {
   // there is not an actual peer defined corresponding to peerId for 
the failover.
   ReplicationQueueInfo replicationQueueInfo = new 
ReplicationQueueInfo(peerId);

[36/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/class-use/RegionMetrics.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/class-use/RegionMetrics.html 
b/devapidocs/org/apache/hadoop/hbase/class-use/RegionMetrics.html
index 119b258..bfbc974 100644
--- a/devapidocs/org/apache/hadoop/hbase/class-use/RegionMetrics.html
+++ b/devapidocs/org/apache/hadoop/hbase/class-use/RegionMetrics.html
@@ -86,6 +86,16 @@
 org.apache.hadoop.hbase
  
 
+
+org.apache.hadoop.hbase.client
+
+Provides HBase Client
+
+
+
+org.apache.hadoop.hbase.master.balancer
+ 
+
 
 
 
@@ -187,6 +197,10 @@
 http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">Map
 ServerMetricsBuilder.ServerMetricsImpl.getRegionMetrics() 
 
+
+static http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List
+RegionMetricsBuilder.toRegionMetrics(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadResponse regionLoadResponse) 
+
 
 
 
@@ -251,6 +265,90 @@
 
 
 
+
+
+
+Uses of RegionMetrics in org.apache.hadoop.hbase.client
+
+Methods in org.apache.hadoop.hbase.client
 that return types with arguments of type RegionMetrics 
+
+Modifier and Type
+Method and Description
+
+
+
+private http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
+RawAsyncHBaseAdmin.getRegionMetrics(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadRequest request,
+ServerName serverName) 
+
+
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
+AsyncHBaseAdmin.getRegionMetrics(ServerName serverName) 
+
+
+default http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List
+Admin.getRegionMetrics(ServerName serverName)
+Get RegionMetrics of 
all regions hosted on a regionserver.
+
+
+
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
+AsyncAdmin.getRegionMetrics(ServerName serverName)
+Get a list of RegionMetrics of 
all regions hosted on a region seerver.
+
+
+
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
+RawAsyncHBaseAdmin.getRegionMetrics(ServerName serverName) 
+
+
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
+AsyncHBaseAdmin.getRegionMetrics(ServerName serverName,
+TableName tableName) 
+
+
+http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List
+Admin.getRegionMetrics(ServerName serverName,
+TableName tableName)
+Get RegionMetrics of 
all regions hosted on a regionserver for a table.
+
+
+
+http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List
+HBaseAdmin.getRegionMetrics(ServerName serverName,
+TableName tableName) 
+
+
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
+AsyncAdmin.getRegionMetrics(ServerName serverName,
+TableName tableName)
+Get a list of RegionMetrics of 
all regions hosted on a region seerver for a table.
+
+
+
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="cl

[22/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/coprocessor/example/ExampleMasterObserverWithMetrics.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/coprocessor/example/ExampleMasterObserverWithMetrics.html
 
b/devapidocs/org/apache/hadoop/hbase/coprocessor/example/ExampleMasterObserverWithMetrics.html
index ad79e6d..8bee2e4 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/coprocessor/example/ExampleMasterObserverWithMetrics.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/coprocessor/example/ExampleMasterObserverWithMetrics.html
@@ -287,7 +287,7 @@ implements MasterObserver
-postAbortProcedure,
 postAddReplicationPeer,
 postAddRSGroup,
 postAssign,
 postBalance, postBalanceRSGroup,
 postBalanceSwitch,
 postClearDeadServers,
 postCloneSnapshot,
 postCompletedCreateTableAction,
 postCompletedDeleteTableAction,
 postCompletedDisableTableAction,
 postCompletedEnableTableAction,
 postCompletedMergeRegionsAction,
 postCompletedModifyTableAction,
 postCompletedSplitRegionAction,
 postCompletedTruncateTableAction, postCreateNamespace,
 postDecommissionRegionServers,
 postDeleteNamespace,
 postDeleteSnapshot,
 postDeleteTable,
 postDisableReplicationPeer,
 postDisableTable,
 postEnableReplicationPeer,
 postEnableTable,
 postGetClusterStatus,
 postGetLocks,
 postGetNamespaceDescriptor,
 postGetProcedures,
 postGetReplicationPeerCon
 fig, postGetTableDescriptors,
 postGetTableNames,
 postListDecommissionedRegionServers,
 postListNamespaceDescriptors,
 postListReplicationPeers,
 postListSnapshot,
 postLockHeartbeat,
 postMergeRegions,
 postMergeRegionsCommitA
 ction, postModifyNamespace,
 postModifyTable,
 postMove,
 postMoveServers,
 postMoveServersAndTables,
 postMoveTables,
 postRecommissionRegionServer,
 postRegionOffline,
 postRemoveReplicationPeer,
 postRemoveRSGroup,
 postRemoveServers,
 postRequestLock,
 postRestoreSnapshot, postRollBackMergeRegionsAction,
 postRollBackSplitRegionAction,
 postSetNamespaceQuota,
 postSetSplitOrMergeEn
 abled, postSetTableQuota,
 postSetUserQuota,
 postSetUserQuota,
 postSetUserQuota, postSnapshot,
 postStartMaster,
 postTableFlush,
 postTruncateTable,
 postUnassign,
 postUpdateReplicationPeerConfig,
 preAbortProcedure,
 preAddReplicationPeer,
 preAddRSGroup, preAssign,
 preBalance,
 preBalanceRSGroup,
 preBalanceSwitch,
 preClearDeadServers,
 preCloneSnapshot,
 preCreateNamespace,
 preCreateTableAction,
 preDecommissionRegionServers,
 preDeleteNamespace,
 preDeleteSnapshot,
 preDeleteTable,
 preDeleteTableAction,
 preDisableReplicationPeer, preDisableTableAction,
 preEnableReplicationPeer,
 preEnableTable,
 preEnableTableAction,
 preGetClusterStatus,
 preGetLocks,
 preGetNamespaceDescriptor,
 preGetProcedures,
 preGetReplicationPeerConfig,
 preGetTableDescriptors,
 preGetTableNames,
 preListDecommissionedRegionServers,
 preListNamespaceDescriptors,
 preListReplicationPeers,
 preListSnapshot,
 preLockHeartbeat,
 preMasterInitialization,
 preMergeRegions,
 preMergeRegionsAction, preMergeRegionsCommitAction,
 preModifyNamespace,
 preModifyTable,
 preModifyTableAction, preMove,
 preMoveServers,
 preMoveServersAndTables,
 preMoveTables,
 preRecommissionRegionServer,
 preRegionOffline,
 preRemoveReplicationPeer,
 preRemoveRSGroup,
 preRemoveServers, preRequestLock,
 preRestoreSnapshot,
 preSetNamespaceQuota,
 preSetSplitOrMergeEnabled, preSetTableQuota,
 preSetUserQuota,
 preSetUserQuota,
 preSetUserQuota,
 preShutdown,
 preSnapshot,
 preSplitRegion,
 preSplitRegionAction,
 preSplitRegionAfterMETAAction,
 preSplitRegionBeforeMETAAction,
 preStopMaster,
 preTableFlush,
 preTruncateTable,
 preTruncateTableAction,
 preUnassign,
 preUpdateReplicationPeerConfig
+postAbortProcedure,
 postAddReplicationPeer,
 postAddRSGroup,
 postAssign,
 postBalance, postBalanceRSGroup,
 postBalanceSwitch,
 postClearDeadServers,
 postCloneSnapshot,
 postCompletedCreateTableAction,
 postCompletedDeleteTableAction,
 postCompletedDisableTableAction,
 postCompletedEnableTableAction,
 postCompletedMergeRegionsAction,
 postCompletedModifyTableAction,
 postCompletedSplitRegionAction,
 postCompletedTruncateTableAction, postCreateNamespace,
 postDecommissionRegionServers,
 postDeleteNamespace,
 postDeleteSnapshot,
 postDeleteTable,
 postDisableReplicationPeer,
 postDisableTable,
 postEnableReplicationPeer,
 postEnableTable,
 postGetClusterMetrics,
 postGetLocks,
 postGetNamespaceDescriptor,
 postGetProcedures,
 postGetReplicationPeer
 Config, postGetTableDescriptors,
 postGetTableNames,
 postListDecommissionedRegionServers,
 postListNamespaceDescriptors,
 postListReplicationPeers,
 postListSnapshot,
 postLockHeartbeat,
 postMergeRegions,
 postMergeRegionsComm
 itAction, postModifyNamespace,
 postModifyTable,
 postMove,
 postMoveServers,
 postMoveServersAndTables,
 postMoveTables,
 postRecommissionRegionServer,
 postRegionOffline,
 postRemoveReplicationPeer,
 postRemoveRSGroup,
 postRemoveSe

[35/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/class-use/TableName.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/class-use/TableName.html 
b/devapidocs/org/apache/hadoop/hbase/class-use/TableName.html
index e479124..898a1d0 100644
--- a/devapidocs/org/apache/hadoop/hbase/class-use/TableName.html
+++ b/devapidocs/org/apache/hadoop/hbase/class-use/TableName.html
@@ -3400,45 +3400,6 @@ service.
 ConnectionImplementation.getNumberOfCachedRegionLocations(TableName tableName) 
 
 
-default http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">Map
-Admin.getRegionLoad(ServerName serverName,
- TableName tableName)
-Deprecated. 
-since 2.0 version and will 
be removed in 3.0 version.
- use Admin.getRegionLoads(ServerName,
 TableName)
-
-
-
-
-http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
-AsyncHBaseAdmin.getRegionLoads(ServerName serverName,
-  TableName tableName) 
-
-
-http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List
-Admin.getRegionLoads(ServerName serverName,
-  TableName tableName)
-Get RegionLoad of all regions hosted on a 
regionserver for a table.
-
-
-
-http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List
-HBaseAdmin.getRegionLoads(ServerName serverName,
-  TableName tableName) 
-
-
-http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
-AsyncAdmin.getRegionLoads(ServerName serverName,
-  TableName tableName)
-Get a list of RegionLoad of all regions hosted on a 
region seerver for a table.
-
-
-
-http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
-RawAsyncHBaseAdmin.getRegionLoads(ServerName serverName,
-  TableName tableName) 
-
-
 HRegionLocation
 ConnectionImplementation.getRegionLocation(TableName tableName,
  byte[] row,
@@ -3518,98 +3479,127 @@ service.
 
 
 
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
+AsyncHBaseAdmin.getRegionMetrics(ServerName serverName,
+TableName tableName) 
+
+
+http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List
+Admin.getRegionMetrics(ServerName serverName,
+TableName tableName)
+Get RegionMetrics of 
all regions hosted on a regionserver for a table.
+
+
+
+http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List
+HBaseAdmin.getRegionMetrics(ServerName serverName,
+TableName tableName) 
+
+
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
+AsyncAdmin.getRegionMetrics(ServerName serverName,
+TableName tableName)
+Get a list of RegionMetrics of 
all regions hosted on a region seerver for a table.
+
+
+
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
+RawAsyncHBaseAdmin.getRegionMetrics(ServerName serverName,
+TableName tableName) 
+
+
 http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concu

[27/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.html 
b/devapidocs/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.html
index 19aa0b9..aedcf26 100644
--- a/devapidocs/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.html
+++ b/devapidocs/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.html
@@ -114,7 +114,7 @@ var activeTableTab = "activeTableTab";
 
 
 @InterfaceAudience.Private
-class RawAsyncHBaseAdmin
+class RawAsyncHBaseAdmin
 extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object
 implements AsyncAdmin
 The implementation of AsyncAdmin.
@@ -704,12 +704,12 @@ implements 
-http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFuture
-getClusterStatus() 
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFuture
+getClusterMetrics() 
 
 
-http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFuture
-getClusterStatus(http://docs.oracle.com/javase/8/docs/api/java/util/EnumSet.html?is-external=true";
 title="class or interface in java.util">EnumSet options) 
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFuture
+getClusterMetrics(http://docs.oracle.com/javase/8/docs/api/java/util/EnumSet.html?is-external=true";
 title="class or interface in java.util">EnumSet options) 
 
 
 http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFuture
@@ -788,27 +788,27 @@ implements 
-private http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
-getRegionLoad(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadRequest request,
- ServerName serverName) 
+(package private) http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFuture
+getRegionLocation(byte[] regionNameOrEncodedRegionName)
+Get the region location for the passed region name.
+
 
 
-http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
-getRegionLoads(ServerName serverName)
-Get a list of RegionLoad of all regions hosted on a 
region seerver.
-
+private http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
+getRegionMetrics(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadRequest request,
+ServerName serverName) 
 
 
-http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
-getRegionLoads(ServerName serverName,
-  TableName tableName)
-Get a list of RegionLoad of all regions hosted on a 
region seerver for a table.
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
+getRegionMetrics(ServerName serverName)
+Get a list of RegionMetrics of 
all regions hosted on a region seerver.
 
 
 
-(package private) http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/Completab

[05/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/security/visibility/VisibilityController.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/security/visibility/VisibilityController.html
 
b/devapidocs/org/apache/hadoop/hbase/security/visibility/VisibilityController.html
index 7715243..398a35e 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/security/visibility/VisibilityController.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/security/visibility/VisibilityController.html
@@ -551,7 +551,7 @@ implements MasterObserver
-postAbortProcedure,
 postAddReplicationPeer,
 postAddRSGroup,
 postAssign,
 postBalance, postBalanceRSGroup,
 postBalanceSwitch,
 postClearDeadServers,
 postCloneSnapshot,
 postCompletedCreateTableAction,
 postCompletedDeleteTableAction,
 postCompletedDisableTableAction,
 postCompletedEnableTableAction,
 postCompletedMergeRegionsAction,
 postCompletedModifyTableAction,
 postCompletedSplitRegionAction,
 postCompletedTruncateTableAction, postCreateNamespace,
 postCreateTable,
 postDecommissionRegionServers,
 postDeleteNamespace,
 postDeleteSnapshot,
 postDeleteTable,
 postDisableReplicationPeer,
 postDisableTable,
 postEnableReplicationPeer, postEnableTable,
 postGetClusterStatus,
 postGetLocks,
 postGetNamespaceDescriptor,
 postGetProcedures, postGetReplicationPeerConfig,
 postGetTableDescriptors,
 postGetTableNames,
 postListDecommissionedRegionServers,
 postListNamespaceDescriptors,
 postListReplicationPeers,
 postListSnapshot,
 postLockHeartbeat,
 postMergeRegions,
 postMergeRegionsCommitAction,
 postModifyNamespace,
 postModifyTable,
 postMove, postMoveServers,
 postMoveServersAndTables,
 postMoveTables,
 postRecommissionRegionServer,
 postRegionOffline,
 postRemoveReplicationPeer,
 postRemoveRSGroup,
 postRemoveServers,
 postRequestLock, postRestoreSnapshot,
 postRollBackMergeRegionsAction,
 postRollBackSplitRegionAction,
 pos
 tSetNamespaceQuota, postSetTableQuota,
 postSetUserQuota,
 postSetUserQuota,
 postSetUserQuota, postSnapshot,
 postTableFlush,
 postTruncateTable,
 postUnassign,
 postUpdateReplicationPeerConfig,
 preAbortProcedure,
 preAddReplicationPeer,
 preAddRSGroup,
 preAssign,
 preBalance,
 preBalanceRSGroup,
 preBalanceSwitch,
 preClearDeadServers,
 preCloneSnapshot,
 preCreateNamespace,
 preCreateTable,
 preCreateTableAction,
 preDecommissionRegionServers,
 preDeleteNamespace,
 preDeleteSnapshot,
 preDeleteTable,
 preDeleteTableAction,
 preDisableReplicationPeer,
 preDisableTableAction,
 preEnableReplicationPeer,
 preEnableTable,
 preEnableTableActio
 n, preGetClusterStatus,
 preGetLocks,
 preGetNamespaceDescriptor,
 preGetProcedures,
 preGetReplicationPeerConfig,
 preGetTableDescriptors,
 preGetTableNames,
 preListDecommissionedRegionServers,
 preListNamespaceDescriptors,
 preListReplicationPeers, preListSnapshot,
 preLockHeartbeat,
 preMasterInitialization,
 preMergeRegions,
 preMergeRegionsAction,
 preMergeRegionsCommitAction,
 preModifyNamespace,
 preModifyTableAction,
 preMove,
 preMoveServers,
 preMoveServersAndTables,
 preMoveTables,
 preRecommissionRegionServer, preRegionOffline,
 preRemoveReplicationPeer,
 preRemoveRSGroup,
 preRemoveServers,
 preRequestLock,
 preRestoreSnapshot,
 preSetNamespaceQuota,
 preSetSplitOrMergeEnabled,
 preSetTableQuota,
 preSetUserQuota,
 preSetUserQuota,
 preSetUserQuota,
 preShutdown,
 preSnapshot,
 preSplitRegion,
 preSplitRegionAction,
 p
 reSplitRegionAfterMETAAction, preSplitRegionBeforeMETAAction,
 preStopMaster,
 preTableFlush,
 preTruncateTable,
 preTruncateTableAction, preUnassign,
 preUpdateReplicationPeerConfig
+postAbortProcedure,
 postAddReplicationPeer,
 postAddRSGroup,
 postAssign,
 postBalance, postBalanceRSGroup,
 postBalanceSwitch,
 postClearDeadServers,
 postCloneSnapshot,
 postCompletedCreateTableAction,
 postCompletedDeleteTableAction,
 postCompletedDisableTableAction,
 postCompletedEnableTableAction,
 postCompletedMergeRegionsAction,
 postCompletedModifyTableAction,
 postCompletedSplitRegionAction,
 postCompletedTruncateTableAction, postCreateNamespace,
 postCreateTable,
 postDecommissionRegionServers,
 postDeleteNamespace,
 postDeleteSnapshot,
 postDeleteTable,
 postDisableReplicationPeer,
 postDisableTable,
 postEnableReplicationPeer, postEnableTable,
 postGetClusterMetrics,
 postGetLocks,
 postGetNamespaceDescriptor,
 postGetProcedures, postGetReplicationPeerConfig,
 postGetTableDescriptors,
 postGetTableNames,
 postListDecommissionedRegionServers,
 postListNamespaceDescriptors,
 postListReplicationPeers,
 postListSnapshot,
 postLockHeartbeat,
 postMergeRegions,
 postMergeRegionsCommitAction,
 postModifyNamespace,
 postModifyTable,
 postMove, postMoveServers,
 postMoveServersAndTables,
 postMoveTables,
 postRecommissionRegionServer,
 postRegionOffline,
 postRemoveReplicationPeer,
 postRemoveRSGroup,
 postRemoveServers,
 postRequestLock, postRestoreSnapshot,
 postRollBackMergeRegionsActio

[21/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.html 
b/devapidocs/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.html
index 5565471..47eff2e 100644
--- a/devapidocs/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.html
+++ b/devapidocs/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.html
@@ -870,7 +870,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 initTableSnapshotMapperJob
-public static void initTableSnapshotMapperJob(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String snapshotName,
+public static void initTableSnapshotMapperJob(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String snapshotName,
   Scan scan,
   http://docs.oracle.com/javase/8/docs/api/java/lang/Class.html?is-external=true";
 title="class or interface in java.lang">Class mapper,
   http://docs.oracle.com/javase/8/docs/api/java/lang/Class.html?is-external=true";
 title="class or interface in java.lang">Class outputKeyClass,
@@ -879,8 +879,8 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
   boolean addDependencyJars,
   
org.apache.hadoop.fs.Path tmpRestoreDir)
throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
-Sets up the job for reading from a table snapshot. It 
bypasses hbase servers
- and read directly from snapshot files.
+Sets up the job for reading from a table snapshot. It 
bypasses hbase servers and read directly
+ from snapshot files.
 
 Parameters:
 snapshotName - The name of the snapshot (of a table) to read 
from.
@@ -888,13 +888,13 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 mapper - The mapper class to use.
 outputKeyClass - The class of the output key.
 outputValueClass - The class of the output value.
-job - The current job to adjust.  Make sure the passed job is
- carrying all necessary HBase configuration.
-addDependencyJars - upload HBase jars and jars for any of the 
configured
-   job classes via the distributed cache (tmpjars).
+job - The current job to adjust. Make sure the passed job is 
carrying all necessary HBase
+  configuration.
+addDependencyJars - upload HBase jars and jars for any of the 
configured job classes via
+  the distributed cache (tmpjars).
 tmpRestoreDir - a temporary directory to copy the snapshot 
files into. Current user should
- have write permissions to this directory, and this should not be a 
subdirectory of rootdir.
- After the job is finished, restore directory can be deleted.
+  have write permissions to this directory, and this should not be a 
subdirectory of
+  rootdir. After the job is finished, restore directory can be 
deleted.
 Throws:
 http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException - When setting up 
the details fails.
 See Also:
@@ -908,7 +908,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 initTableSnapshotMapperJob
-public static void initTableSnapshotMapperJob(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String snapshotName,
+public static void initTableSnapshotMapperJob(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String snapshotName,
   Scan scan,
   http://docs.oracle.com/javase/8/docs/api/java/lang/Class.html?is-external=true";
 title="class or interface in java.lang">Class mapper,
   http://docs.oracle.com/javase/8/docs/api/java/lang/Class.html?is-external=true";
 title="class or interface in java.lang">Class outputKeyClass,
@@ -950,7 +950,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 initTableMapperJob
-public static void initTableMapperJob(http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List scans,
+public static void initTableMapperJob(http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="c

[11/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.html
 
b/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.html
index 6077486..dc5f1b8 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.html
@@ -123,7 +123,7 @@ var activeTableTab = "activeTableTab";
 
 
 @InterfaceAudience.LimitedPrivate(value="Configuration")
-public class StochasticLoadBalancer
+public class StochasticLoadBalancer
 extends BaseLoadBalancer
 This is a best effort load balancer. Given a Cost 
function F(C) => x It will
  randomly try and mutate the cluster to Cprime. If F(Cprime) < F(C) then the
@@ -579,7 +579,7 @@ extends 
 void
-setClusterStatus(ClusterStatus st)
+setClusterMetrics(ClusterMetrics st)
 Set the current cluster status.
 
 
@@ -664,7 +664,7 @@ extends 
 
 STEPS_PER_REGION_KEY
-protected static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String STEPS_PER_REGION_KEY
+protected static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String STEPS_PER_REGION_KEY
 
 See Also:
 Constant
 Field Values
@@ -677,7 +677,7 @@ extends 
 
 MAX_STEPS_KEY
-protected static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String MAX_STEPS_KEY
+protected static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String MAX_STEPS_KEY
 
 See Also:
 Constant
 Field Values
@@ -690,7 +690,7 @@ extends 
 
 RUN_MAX_STEPS_KEY
-protected static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String RUN_MAX_STEPS_KEY
+protected static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String RUN_MAX_STEPS_KEY
 
 See Also:
 Constant
 Field Values
@@ -703,7 +703,7 @@ extends 
 
 MAX_RUNNING_TIME_KEY
-protected static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String MAX_RUNNING_TIME_KEY
+protected static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String MAX_RUNNING_TIME_KEY
 
 See Also:
 Constant
 Field Values
@@ -716,7 +716,7 @@ extends 
 
 KEEP_REGION_LOADS
-protected static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String KEEP_REGION_LOADS
+protected static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String KEEP_REGION_LOADS
 
 See Also:
 Constant
 Field Values
@@ -729,7 +729,7 @@ extends 
 
 TABLE_FUNCTION_SEP
-private static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String TABLE_FUNCTION_SEP
+private static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String TABLE_FUNCTION_SEP
 
 See Also:
 Constant
 Field Values
@@ -742,7 +742,7 @@ extends 
 
 MIN_COST_NEED_BALANCE_KEY
-protected static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String MIN_COST_NEED_BALANCE_KEY
+protected static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String MIN_COST_NEED_BALANCE_KEY
 
 See Also:
 Constant
 Field Values
@@ -755,7 +755,7 @@ extends 
 
 RANDOM
-protected static final http://docs.oracle.com/javase/8/docs/api/java/util/Random.html?is-external=true";
 title="class or interface in java.util">Random RANDOM
+protected static final http://docs.oracle.com/javase/8/docs/api/java/util/Random.html?is-external=true";
 title="class or interface in java.util">Random RANDOM
 
 
 
@@ -764,7 +764,7 @@ extends 
 
 LOG
-private static final org.slf4j.Logger LOG
+private static final org.slf4j.Logger LOG
 
 
 
@@ -773,7 +773,7 @@ extends 
 
 loads
-http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">MapString,http://docs.oracle.com/javase/8/docs/api/java/uti

[16/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/master/MasterRpcServices.BalanceSwitchMode.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/master/MasterRpcServices.BalanceSwitchMode.html
 
b/devapidocs/org/apache/hadoop/hbase/master/MasterRpcServices.BalanceSwitchMode.html
index 3f627a4..ca3f255 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/master/MasterRpcServices.BalanceSwitchMode.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/master/MasterRpcServices.BalanceSwitchMode.html
@@ -122,7 +122,7 @@ var activeTableTab = "activeTableTab";
 
 
 
-static enum MasterRpcServices.BalanceSwitchMode
+static enum MasterRpcServices.BalanceSwitchMode
 extends http://docs.oracle.com/javase/8/docs/api/java/lang/Enum.html?is-external=true";
 title="class or interface in java.lang">Enum
 
 
@@ -210,7 +210,7 @@ the order they are declared.
 
 
 SYNC
-public static final MasterRpcServices.BalanceSwitchMode SYNC
+public static final MasterRpcServices.BalanceSwitchMode SYNC
 
 
 
@@ -219,7 +219,7 @@ the order they are declared.
 
 
 ASYNC
-public static final MasterRpcServices.BalanceSwitchMode ASYNC
+public static final MasterRpcServices.BalanceSwitchMode ASYNC
 
 
 
@@ -236,7 +236,7 @@ the order they are declared.
 
 
 values
-public static MasterRpcServices.BalanceSwitchMode[] values()
+public static MasterRpcServices.BalanceSwitchMode[] values()
 Returns an array containing the constants of this enum 
type, in
 the order they are declared.  This method may be used to iterate
 over the constants as follows:
@@ -256,7 +256,7 @@ for (MasterRpcServices.BalanceSwitchMode c : 
MasterRpcServices.BalanceSwitchMode
 
 
 valueOf
-public static MasterRpcServices.BalanceSwitchMode valueOf(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String name)
+public static MasterRpcServices.BalanceSwitchMode valueOf(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String name)
 Returns the enum constant of this type with the specified 
name.
 The string must match exactly an identifier used to declare an
 enum constant in this type.  (Extraneous whitespace characters are 



[51/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
Published site at .


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

Branch: refs/heads/asf-site
Commit: c7c40c622b42be870c95ed287cd17f15b0d316e4
Parents: 1f4f0ee
Author: jenkins 
Authored: Thu Jan 4 15:19:17 2018 +
Committer: jenkins 
Committed: Thu Jan 4 15:19:17 2018 +

--
 acid-semantics.html | 4 +-
 apache_hbase_reference_guide.pdf|  1908 +-
 apidocs/deprecated-list.html|70 +-
 apidocs/index-all.html  |76 +-
 .../hadoop/hbase/class-use/ClusterMetrics.html  |49 +
 .../hadoop/hbase/class-use/ClusterStatus.html   |30 +-
 .../hadoop/hbase/class-use/RegionLoad.html  |65 -
 .../hadoop/hbase/class-use/RegionMetrics.html   |46 +
 .../hadoop/hbase/class-use/ServerName.html  |47 +-
 .../hadoop/hbase/class-use/TableName.html   |   138 +-
 .../org/apache/hadoop/hbase/client/Admin.html   |   682 +-
 .../apache/hadoop/hbase/client/AsyncAdmin.html  |   340 +-
 .../hbase/mapreduce/TableMapReduceUtil.html |58 +-
 .../org/apache/hadoop/hbase/package-use.html|44 +-
 .../org/apache/hadoop/hbase/client/Admin.html   |  5121 +--
 .../apache/hadoop/hbase/client/AsyncAdmin.html  |  2372 +-
 .../hbase/mapreduce/TableMapReduceUtil.html |  1412 +-
 .../apache/hadoop/hbase/util/RegionMover.html   | 4 +-
 book.html   | 4 +-
 bulk-loads.html | 4 +-
 checkstyle-aggregate.html   | 30208 -
 checkstyle.rss  |54 +-
 coc.html| 4 +-
 cygwin.html | 4 +-
 dependencies.html   | 4 +-
 dependency-convergence.html | 4 +-
 dependency-info.html| 4 +-
 dependency-management.html  | 4 +-
 devapidocs/allclasses-frame.html| 2 +-
 devapidocs/allclasses-noframe.html  | 2 +-
 devapidocs/constant-values.html | 6 +-
 devapidocs/deprecated-list.html |86 +-
 devapidocs/index-all.html   |   198 +-
 .../RegionMetricsBuilder.RegionMetricsImpl.html |82 +-
 .../hadoop/hbase/RegionMetricsBuilder.html  |   105 +-
 .../hadoop/hbase/backup/package-tree.html   | 2 +-
 .../hbase/class-use/ClusterMetrics.Option.html  |28 +-
 .../hadoop/hbase/class-use/ClusterMetrics.html  |   290 +
 .../hadoop/hbase/class-use/ClusterStatus.html   |   275 +-
 .../hadoop/hbase/class-use/RegionLoad.html  |   113 -
 .../hadoop/hbase/class-use/RegionMetrics.html   |98 +
 .../hadoop/hbase/class-use/ServerName.html  |79 +-
 .../hadoop/hbase/class-use/TableName.html   |   380 +-
 .../org/apache/hadoop/hbase/client/Admin.html   |   682 +-
 .../apache/hadoop/hbase/client/AsyncAdmin.html  |   340 +-
 .../hadoop/hbase/client/AsyncHBaseAdmin.html|70 +-
 .../hadoop/hbase/client/ClientScanner.html  |   104 +-
 ...ClusterStatusListener.DeadServerHandler.html | 4 +-
 .../client/ClusterStatusListener.Listener.html  | 6 +-
 MulticastListener.ClusterStatusHandler.html |10 +-
 ...ClusterStatusListener.MulticastListener.html |12 +-
 .../hbase/client/ClusterStatusListener.html |26 +-
 .../client/HBaseAdmin.AbortProcedureFuture.html | 8 +-
 .../HBaseAdmin.AddColumnFamilyFuture.html   | 6 +-
 .../client/HBaseAdmin.CreateTableFuture.html|14 +-
 .../HBaseAdmin.DeleteColumnFamilyFuture.html| 6 +-
 .../client/HBaseAdmin.DeleteTableFuture.html|10 +-
 .../client/HBaseAdmin.DisableTableFuture.html   | 8 +-
 .../client/HBaseAdmin.EnableTableFuture.html| 8 +-
 .../HBaseAdmin.MergeTableRegionsFuture.html | 8 +-
 .../HBaseAdmin.ModifyColumnFamilyFuture.html| 6 +-
 .../client/HBaseAdmin.ModifyTableFuture.html|10 +-
 .../client/HBaseAdmin.NamespaceFuture.html  |12 +-
 ...in.ProcedureFuture.WaitForStateCallable.html | 8 +-
 .../client/HBaseAdmin.ProcedureFuture.html  |48 +-
 .../HBaseAdmin.RestoreSnapshotFuture.html   | 8 +-
 .../HBaseAdmin.SplitTableRegionFuture.html  | 8 +-
 ...n.TableFuture.TableWaitForStateCallable.html | 8 +-
 .../hbase/client/HBaseAdmin.TableFuture.html|30 +-
 .../client/HBaseAdmin.ThrowableAbortable.html   | 8 +-
 .../client/HBaseAdmin.TruncateTableFuture.html  |10 +-
 .../apache/hadoop/hbase/client/HBaseAdmin.html  |   866 +-
 ...dmin.AddColumnFamilyProcedureBiConsumer.html | 6 +-
 .../client/RawAsyncHBas

[26/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/client/TableSnapshotScanner.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/client/TableSnapshotScanner.html 
b/devapidocs/org/apache/hadoop/hbase/client/TableSnapshotScanner.html
index dd4afb1..c8dd758 100644
--- a/devapidocs/org/apache/hadoop/hbase/client/TableSnapshotScanner.html
+++ b/devapidocs/org/apache/hadoop/hbase/client/TableSnapshotScanner.html
@@ -18,7 +18,7 @@
 catch(err) {
 }
 //-->
-var methods = {"i0":10,"i1":10,"i2":10,"i3":10};
+var methods = {"i0":10,"i1":10,"i2":10,"i3":10,"i4":10,"i5":10,"i6":10};
 var tabs = {65535:["t0","All Methods"],2:["t2","Instance 
Methods"],8:["t4","Concrete Methods"]};
 var altColor = "altColor";
 var rowColor = "rowColor";
@@ -119,7 +119,7 @@ var activeTableTab = "activeTableTab";
 
 
 @InterfaceAudience.Private
-public class TableSnapshotScanner
+public class TableSnapshotScanner
 extends AbstractClientScanner
 A Scanner which performs a scan over snapshot files. Using 
this class requires copying the
  snapshot to a temporary empty directory, which will copy the snapshot 
reference files into that
@@ -205,6 +205,10 @@ extends scan 
 
 
+private boolean
+snapshotAlreadyRestored 
+
+
 private http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String
 snapshotName 
 
@@ -234,11 +238,19 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String snapshotName,
-Scan scan)
+Scan scan) 
+
+
+TableSnapshotScanner(org.apache.hadoop.conf.Configuration conf,
+org.apache.hadoop.fs.Path rootDir,
+org.apache.hadoop.fs.Path restoreDir,
+http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String snapshotName,
+Scan scan,
+boolean snapshotAlreadyRestored)
 Creates a TableSnapshotScanner.
 
 
-
+
 TableSnapshotScanner(org.apache.hadoop.conf.Configuration conf,
 org.apache.hadoop.fs.Path restoreDir,
 http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String snapshotName,
@@ -262,22 +274,34 @@ extends Method and Description
 
 
+private void
+cleanup() 
+
+
 void
 close()
 Closes the scanner and releases any resources it has 
allocated
 
 
-
-private void
-init() 
-
 
+private boolean
+isValidRegion(RegionInfo hri) 
+
+
 Result
 next()
 Grab the next row's worth of values.
 
 
-
+
+private void
+openWithoutRestoringSnapshot() 
+
+
+private void
+openWithRestoringSnapshot() 
+
+
 boolean
 renewLease()
 Allow the client to renew the scanner's lease on the 
server.
@@ -332,7 +356,7 @@ extends 
 
 LOG
-private static final org.slf4j.Logger LOG
+private static final org.slf4j.Logger LOG
 
 
 
@@ -341,7 +365,7 @@ extends 
 
 conf
-private org.apache.hadoop.conf.Configuration conf
+private org.apache.hadoop.conf.Configuration conf
 
 
 
@@ -350,7 +374,7 @@ extends 
 
 snapshotName
-private http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String snapshotName
+private http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String snapshotName
 
 
 
@@ -359,7 +383,7 @@ extends 
 
 fs
-private org.apache.hadoop.fs.FileSystem fs
+private org.apache.hadoop.fs.FileSystem fs
 
 
 
@@ -368,7 +392,7 @@ extends 
 
 rootDir
-private org.apache.hadoop.fs.Path rootDir
+private org.apache.hadoop.fs.Path rootDir
 
 
 
@@ -377,7 +401,7 @@ extends 
 
 restoreDir
-private org.apache.hadoop.fs.Path restoreDir
+private org.apache.hadoop.fs.Path restoreDir
 
 
 
@@ -386,7 +410,7 @@ extends 
 
 scan
-private Scan scan
+private Scan scan
 
 
 
@@ -395,7 +419,7 @@ extends 
 
 regions
-private http://docs.oracle.com/javase/8/docs/api/java/util/ArrayList.html?is-external=true";
 title="class or interface in java.util">ArrayList regions
+private http://docs.oracle.com/javase/8/docs/api/java/util/ArrayList.html?is-external=true";
 title="class or interface in java.util">ArrayList regions
 
 
 
@@ -404,7 +428,16 @@ extends 
 
 htd
-private TableDescriptor htd
+private TableDescriptor htd
+
+
+
+
+
+
+
+snapshotAlreadyRestored
+private final boolean snapshotAlreadyRestored
 
 
 
@@ -413,7 +446,7 @@ extends 
 
 currentRegionScanner
-private ClientSideRegionScanner currentRegionScanner
+private ClientSideRegionScanner currentRegionScanner
 
 
 
@@ -422,7 +455,7 @@ extends 
 
 currentRegion
-private int currentRegion
+private int currentRegion
 
 
 
@@ -439,7 +472,7 @@ extends 
 
 TableSnapshotScanner
-publi

[32/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.html 
b/devapidocs/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.html
index d5264f6..89f9554 100644
--- a/devapidocs/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.html
+++ b/devapidocs/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.html
@@ -466,12 +466,12 @@ implements 
-http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFuture
-getClusterStatus() 
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFuture
+getClusterMetrics() 
 
 
-http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFuture
-getClusterStatus(http://docs.oracle.com/javase/8/docs/api/java/util/EnumSet.html?is-external=true";
 title="class or interface in java.util">EnumSet options) 
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFuture
+getClusterMetrics(http://docs.oracle.com/javase/8/docs/api/java/util/EnumSet.html?is-external=true";
 title="class or interface in java.util">EnumSet options) 
 
 
 http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFuture
@@ -529,16 +529,16 @@ implements 
-http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
-getRegionLoads(ServerName serverName)
-Get a list of RegionLoad of all regions hosted on a 
region seerver.
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
+getRegionMetrics(ServerName serverName)
+Get a list of RegionMetrics of 
all regions hosted on a region seerver.
 
 
 
-http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
-getRegionLoads(ServerName serverName,
-  TableName tableName)
-Get a list of RegionLoad of all regions hosted on a 
region seerver for a table.
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
+getRegionMetrics(ServerName serverName,
+TableName tableName)
+Get a list of RegionMetrics of 
all regions hosted on a region seerver for a table.
 
 
 
@@ -1005,7 +1005,7 @@ implements AsyncAdmin
-addReplicationPeer,
 balance,
 compact,
 compact,
 getBackupMasters,
 getCompactionState,
 getMaster,
 getMasterCoprocessors, getMasterInfoPort,
 getRegionServers,
 listTableDescriptors,
 listTableNames,
 majorCompact,
 majorCompact,
 snapshot,
 snapshot
+addReplicationPeer,
 balance,
 compact,
 compact,
 getBackupMasters,
 getCompactionState,
 getMaster,
 getMasterCoprocessorNames, getMasterInfoPort,
 getRegionServers,
 listTableDescriptors,
 listTableNames,
 majorCompact,
 majorCompact,
 snapshot,
 snapshot
 
 
 
@@ -2877,31 +2877,31 @@ implements 
+
 
 
 
 
-getClusterStatus
-public http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFuture getClusterStatus()
+getClusterMetrics
+public http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFuture getClusterMetrics()
 
 Specified by:
-getClusterStatus in
 interface AsyncAdmin
+getClusterMetrics in
 interface AsyncAdmin
 Returns:
 cluster status wrapped by http://do

[30/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/client/HBaseAdmin.ProcedureFuture.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/client/HBaseAdmin.ProcedureFuture.html 
b/devapidocs/org/apache/hadoop/hbase/client/HBaseAdmin.ProcedureFuture.html
index 29c4f48..81617c2 100644
--- a/devapidocs/org/apache/hadoop/hbase/client/HBaseAdmin.ProcedureFuture.html
+++ b/devapidocs/org/apache/hadoop/hbase/client/HBaseAdmin.ProcedureFuture.html
@@ -123,7 +123,7 @@ var activeTableTab = "activeTableTab";
 
 @InterfaceAudience.Private
  @InterfaceStability.Evolving
-protected static class HBaseAdmin.ProcedureFuture
+protected static class HBaseAdmin.ProcedureFuture
 extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object
 implements http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/Future.html?is-external=true";
 title="class or interface in java.util.concurrent">Future
 Future that waits on a procedure result.
@@ -328,7 +328,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/util/concurren
 
 
 exception
-private http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/ExecutionException.html?is-external=true";
 title="class or interface in java.util.concurrent">ExecutionException exception
+private http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/ExecutionException.html?is-external=true";
 title="class or interface in java.util.concurrent">ExecutionException exception
 
 
 
@@ -337,7 +337,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/util/concurren
 
 
 procResultFound
-private boolean procResultFound
+private boolean procResultFound
 
 
 
@@ -346,7 +346,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/util/concurren
 
 
 done
-private boolean done
+private boolean done
 
 
 
@@ -355,7 +355,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/util/concurren
 
 
 cancelled
-private boolean cancelled
+private boolean cancelled
 
 
 
@@ -364,7 +364,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/util/concurren
 
 
 result
-private V result
+private V result
 
 
 
@@ -373,7 +373,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/util/concurren
 
 
 admin
-private final HBaseAdmin admin
+private final HBaseAdmin admin
 
 
 
@@ -382,7 +382,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/util/concurren
 
 
 procId
-private final http://docs.oracle.com/javase/8/docs/api/java/lang/Long.html?is-external=true";
 title="class or interface in java.lang">Long procId
+private final http://docs.oracle.com/javase/8/docs/api/java/lang/Long.html?is-external=true";
 title="class or interface in java.lang">Long procId
 
 
 
@@ -399,7 +399,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/util/concurren
 
 
 ProcedureFuture
-public ProcedureFuture(HBaseAdmin admin,
+public ProcedureFuture(HBaseAdmin admin,
http://docs.oracle.com/javase/8/docs/api/java/lang/Long.html?is-external=true";
 title="class or interface in java.lang">Long procId)
 
 
@@ -417,7 +417,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/util/concurren
 
 
 cancel
-public boolean cancel(boolean mayInterruptIfRunning)
+public boolean cancel(boolean mayInterruptIfRunning)
 
 Specified by:
 http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/Future.html?is-external=true#cancel-boolean-";
 title="class or interface in java.util.concurrent">cancel in 
interface http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/Future.html?is-external=true";
 title="class or interface in java.util.concurrent">Future
@@ -430,7 +430,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/util/concurren
 
 
 isCancelled
-public boolean isCancelled()
+public boolean isCancelled()
 
 Specified by:
 http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/Future.html?is-external=true#isCancelled--";
 title="class or interface in 
java.util.concurrent">isCancelled in interface http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/Future.html?is-external=true";
 title="class or interface in java.util.concurrent">Future
@@ -443,7 +443,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/util/concurren
 
 
 abortProcedureResult
-protected org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse abortProcedureResult(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest request)
+protected org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureResponse abortProcedureResult(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.AbortProcedureRequest request)

   throws http://docs.o

[46/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/apidocs/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.html
--
diff --git a/apidocs/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.html 
b/apidocs/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.html
index 6966618..757a617 100644
--- a/apidocs/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.html
+++ b/apidocs/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.html
@@ -796,7 +796,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 initTableSnapshotMapperJob
-public static void initTableSnapshotMapperJob(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String snapshotName,
+public static void initTableSnapshotMapperJob(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String snapshotName,
   Scan scan,
   http://docs.oracle.com/javase/8/docs/api/java/lang/Class.html?is-external=true";
 title="class or interface in java.lang">Class mapper,
   http://docs.oracle.com/javase/8/docs/api/java/lang/Class.html?is-external=true";
 title="class or interface in java.lang">Class outputKeyClass,
@@ -805,8 +805,8 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
   boolean addDependencyJars,
   
org.apache.hadoop.fs.Path tmpRestoreDir)
throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
-Sets up the job for reading from a table snapshot. It 
bypasses hbase servers
- and read directly from snapshot files.
+Sets up the job for reading from a table snapshot. It 
bypasses hbase servers and read directly
+ from snapshot files.
 
 Parameters:
 snapshotName - The name of the snapshot (of a table) to read 
from.
@@ -814,13 +814,13 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 mapper - The mapper class to use.
 outputKeyClass - The class of the output key.
 outputValueClass - The class of the output value.
-job - The current job to adjust.  Make sure the passed job is
- carrying all necessary HBase configuration.
-addDependencyJars - upload HBase jars and jars for any of the 
configured
-   job classes via the distributed cache (tmpjars).
+job - The current job to adjust. Make sure the passed job is 
carrying all necessary HBase
+  configuration.
+addDependencyJars - upload HBase jars and jars for any of the 
configured job classes via
+  the distributed cache (tmpjars).
 tmpRestoreDir - a temporary directory to copy the snapshot 
files into. Current user should
- have write permissions to this directory, and this should not be a 
subdirectory of rootdir.
- After the job is finished, restore directory can be deleted.
+  have write permissions to this directory, and this should not be a 
subdirectory of
+  rootdir. After the job is finished, restore directory can be 
deleted.
 Throws:
 http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException - When setting up 
the details fails.
 See Also:
@@ -834,7 +834,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 initTableSnapshotMapperJob
-public static void initTableSnapshotMapperJob(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String snapshotName,
+public static void initTableSnapshotMapperJob(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String snapshotName,
   Scan scan,
   http://docs.oracle.com/javase/8/docs/api/java/lang/Class.html?is-external=true";
 title="class or interface in java.lang">Class mapper,
   http://docs.oracle.com/javase/8/docs/api/java/lang/Class.html?is-external=true";
 title="class or interface in java.lang">Class outputKeyClass,
@@ -876,7 +876,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 initTableMapperJob
-public static void initTableMapperJob(http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List scans,
+public static void initTableMapperJob(http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interfac

[50/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/apache_hbase_reference_guide.pdf
--
diff --git a/apache_hbase_reference_guide.pdf b/apache_hbase_reference_guide.pdf
index 9045593..62dbb0c 100644
--- a/apache_hbase_reference_guide.pdf
+++ b/apache_hbase_reference_guide.pdf
@@ -5,8 +5,8 @@
 /Author (Apache HBase Team)
 /Creator (Asciidoctor PDF 1.5.0.alpha.15, based on Prawn 2.2.2)
 /Producer (Apache HBase Team)
-/ModDate (D:20180103144723+00'00')
-/CreationDate (D:20180103144723+00'00')
+/ModDate (D:20180104144724+00'00')
+/CreationDate (D:20180104144724+00'00')
 >>
 endobj
 2 0 obj
@@ -742151,7 +742151,7 @@ endobj
 >>
 endobj
 3861 0 obj
-<< /Length 37475
+<< /Length 37466
 >>
 stream
 q
@@ -743748,7 +743748,7 @@ S
 BT
 383.773 519.756 Td
 /F1.0 10.5 Tf
-<7375706572757365727c676c6f62616c284129> Tj
+[<616e> 20.0195 <792075736572>] TJ
 ET
 
 0.0 0.0 0.0 scn
@@ -790904,960 +790904,960 @@ xref
 0011075920 0 n 
 003322 0 n 
 003670 0 n 
-0011151201 0 n 
-0011151549 0 n 
-0011184116 0 n 
-0011184464 0 n 
-0011198024 0 n 
-0011198372 0 n 
-0011209578 0 n 
-0011210016 0 n 
-0011210064 0 n 
-0011210213 0 n 
-0011210410 0 n 
-0011210545 0 n 
-0011210682 0 n 
-0011210821 0 n 
-0011210963 0 n 
-0011216347 0 n 
-0011216749 0 n 
-0011257406 0 n 
-0011285169 0 n 
-0011293033 0 n 
-0011293408 0 n 
-0011339449 0 n 
-0011339498 0 n 
-0011357711 0 n 
-0011358117 0 n 
-0011358300 0 n 
-0011358347 0 n 
-0011391914 0 n 
-0011392288 0 n 
-0011423897 0 n 
-0011424308 0 n 
-0011424501 0 n 
-0011424692 0 n 
-0011424903 0 n 
-0011424951 0 n 
-0011424999 0 n 
-0011439776 0 n 
-0011440192 0 n 
-0011440387 0 n 
-0011440523 0 n 
-0011440572 0 n 
-0011440780 0 n 
-0011440922 0 n 
-0011440971 0 n 
-0011441107 0 n 
-0011441156 0 n 
-0011454272 0 n 
-0011454687 0 n 
-0011454819 0 n 
-0011454961 0 n 
-0011455009 0 n 
-0011455058 0 n 
-0011455648 0 n 
-0011455697 0 n 
-0011468906 0 n 
-0011469280 0 n 
-0011469329 0 n 
-0011478001 0 n 
-0011478362 0 n 
-0011486199 0 n 
-0011486573 0 n 
-0011486621 0 n 
-0011487948 0 n 
-0011488324 0 n 
-0011488372 0 n 
-0011488420 0 n 
-0011488589 0 n 
-0011488637 0 n 
-0011488819 0 n 
-0011491777 0 n 
-0011492171 0 n 
-0011492219 0 n 
-0011492404 0 n 
-0011492597 0 n 
-0011492774 0 n 
-0011492950 0 n 
-0011500151 0 n 
-0011500526 0 n 
-0011500574 0 n 
-0011500983 0 n 
-0011501031 0 n 
-0011501079 0 n 
-0011526966 0 n 
-0011531529 0 n 
-0011531578 0 n 
-0011531627 0 n 
-0011531676 0 n 
-0011539596 0 n 
-0011539957 0 n 
-0011540005 0 n 
-0011586773 0 n 
-0011593584 0 n 
-0011593946 0 n 
-0011593994 0 n 
-0011603945 0 n 
-0011604306 0 n 
-0011604354 0 n 
-0011604402 0 n 
-0011616672 0 n 
-0011617033 0 n 
-0011617081 0 n 
-0011630835 0 n 
-0011631196 0 n 
-0011631244 0 n 
-0011631821 0 n 
-0011631869 0 n 
-0011631917 0 n 
-0011649848 0 n 
-0011650196 0 n 
-0011650244 0 n 
-0011662711 0 n 
-0011663163 0 n 
-0011663211 0 n 
-0011663259 0 n 
-0011663399 0 n 
-0011663526 0 n 
-0011663676 0 n 
-0011663813 0 n 
-0011663861 0 n 
-0011663993 0 n 
-0011664040 0 n 
-0011664167 0 n 
-0011664292 0 n 
-0011664340 0 n 
-0011664468 0 n 
-0011664682 0 n 
-0011674959 0 n 
-0011675339 0 n 
-0011675387 0 n 
-0011675523 0 n 
-0011682828 0 n 
-0011683325 0 n 
-0011683373 0 n 
-0011683421 0 n 
-0011683592 0 n 
-0011683764 0 n 
-0011684020 0 n 
-0011684069 0 n 
-0011684318 0 n 
-0011684546 0 n 
-0011684790 0 n 
-0011684839 0 n 
-0011685032 0 n 
-0011685242 0 n 
-0011685455 0 n 
-0011685504 0 n 
-0011685696 0 n 
-0011685929 0 n 
-0011686114 0 n 
-0011686163 0 n 
-0011686593 0 n 
-0011686786 0 n 
-0011686835 0 n 
-0011687029 0 n 
-0011689013 0 n 
-0011689380 0 n 
-0011689428 0 n 
-0011689622 0 n 
-0011692998 0 n 
-0011693374 0 n 
-0011693422 0 n 
-0011693470 0 n 
-0011693654 0 n 
-0011693702 0 n 
-0011693896 0 n 
-0011697510 0 n 
-0011697949 0 n 
-0011697997 0 n 
-0011702257 0 n 
-0011702787 0 n 
-0011702981 0 n 
-0011703165 0 n 
-0011703358 0 n 
-0011703555 0 n 
-0011703750 0 n 
-0011703936 0 n 
-0011728163 0 n 
-0011728592 0 n 
-0011728640 0 n 
-0011728831 0 n 
-0011729012 0 n 
-0011729060 0 n 
-0011729239 0 n 
-0011729489 0 n 
-0011729672 0 n 
-0011736063 0 n 
-0011736411 0 n 
-0011766276 0 n 
-

[12/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.PrimaryRegionCountSkewCostFunction.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.PrimaryRegionCountSkewCostFunction.html
 
b/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.PrimaryRegionCountSkewCostFunction.html
index 763eec0..c4f0db1 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.PrimaryRegionCountSkewCostFunction.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.PrimaryRegionCountSkewCostFunction.html
@@ -118,7 +118,7 @@ var activeTableTab = "activeTableTab";
 
 
 
-static class StochasticLoadBalancer.PrimaryRegionCountSkewCostFunction
+static class StochasticLoadBalancer.PrimaryRegionCountSkewCostFunction
 extends StochasticLoadBalancer.CostFunction
 Compute the cost of a potential cluster state from skew in 
number of
  primary regions on a cluster.
@@ -230,7 +230,7 @@ extends 
 
 PRIMARY_REGION_COUNT_SKEW_COST_KEY
-private static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String PRIMARY_REGION_COUNT_SKEW_COST_KEY
+private static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String PRIMARY_REGION_COUNT_SKEW_COST_KEY
 
 See Also:
 Constant
 Field Values
@@ -243,7 +243,7 @@ extends 
 
 DEFAULT_PRIMARY_REGION_COUNT_SKEW_COST
-private static final float DEFAULT_PRIMARY_REGION_COUNT_SKEW_COST
+private static final float DEFAULT_PRIMARY_REGION_COUNT_SKEW_COST
 
 See Also:
 Constant
 Field Values
@@ -256,7 +256,7 @@ extends 
 
 stats
-private double[] stats
+private double[] stats
 
 
 
@@ -273,7 +273,7 @@ extends 
 
 PrimaryRegionCountSkewCostFunction
-PrimaryRegionCountSkewCostFunction(org.apache.hadoop.conf.Configuration conf)
+PrimaryRegionCountSkewCostFunction(org.apache.hadoop.conf.Configuration conf)
 
 
 
@@ -290,7 +290,7 @@ extends 
 
 cost
-double cost()
+double cost()
 
 Specified by:
 cost in
 class StochasticLoadBalancer.CostFunction

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.RackLocalityCostFunction.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.RackLocalityCostFunction.html
 
b/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.RackLocalityCostFunction.html
index 268e2f3..5550840 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.RackLocalityCostFunction.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.RackLocalityCostFunction.html
@@ -123,7 +123,7 @@ var activeTableTab = "activeTableTab";
 
 
 
-static class StochasticLoadBalancer.RackLocalityCostFunction
+static class StochasticLoadBalancer.RackLocalityCostFunction
 extends StochasticLoadBalancer.LocalityBasedCostFunction
 
 
@@ -239,7 +239,7 @@ extends 
 
 RACK_LOCALITY_COST_KEY
-private static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String RACK_LOCALITY_COST_KEY
+private static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String RACK_LOCALITY_COST_KEY
 
 See Also:
 Constant
 Field Values
@@ -252,7 +252,7 @@ extends 
 
 DEFAULT_RACK_LOCALITY_COST
-private static final float DEFAULT_RACK_LOCALITY_COST
+private static final float DEFAULT_RACK_LOCALITY_COST
 
 See Also:
 Constant
 Field Values
@@ -273,7 +273,7 @@ extends 
 
 RackLocalityCostFunction
-public RackLocalityCostFunction(org.apache.hadoop.conf.Configuration conf,
+public RackLocalityCostFunction(org.apache.hadoop.conf.Configuration conf,
 MasterServices services)
 
 
@@ -291,7 +291,7 @@ extends 
 
 regionIndexToEntityIndex
-int regionIndexToEntityIndex(int region)
+int regionIndexToEntityIndex(int region)
 Description copied from 
class: StochasticLoadBalancer.LocalityBasedCostFunction
 Maps region to the current entity (server or rack) on which 
it is stored
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.RandomCandidateGenerator.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.RandomCandidateGenerator.html
 
b/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.RandomCandidateGenerator.html
index 233412a..992fe88 100644
--- 
a/devap

[42/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/apidocs/src-html/org/apache/hadoop/hbase/util/RegionMover.html
--
diff --git a/apidocs/src-html/org/apache/hadoop/hbase/util/RegionMover.html 
b/apidocs/src-html/org/apache/hadoop/hbase/util/RegionMover.html
index 1143409..e50230e 100644
--- a/apidocs/src-html/org/apache/hadoop/hbase/util/RegionMover.html
+++ b/apidocs/src-html/org/apache/hadoop/hbase/util/RegionMover.html
@@ -755,7 +755,7 @@
 747   * @throws IOException
 748   */
 749  private void 
stripMaster(ArrayList regionServers, Admin admin) throws 
IOException {
-750ServerName master = 
admin.getClusterStatus(EnumSet.of(Option.MASTER)).getMaster();
+750ServerName master = 
admin.getClusterMetrics(EnumSet.of(Option.MASTER)).getMasterName();
 751String masterHostname = 
master.getHostname();
 752int masterPort = master.getPort();
 753try {
@@ -833,7 +833,7 @@
 825   */
 826  private ArrayList 
getServers(Admin admin) throws IOException {
 827ArrayList 
serverInfo = new ArrayList<>(
-828
admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS)).getServers());
+828
admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet());
 829ArrayList regionServers 
= new ArrayList<>(serverInfo.size());
 830for (ServerName server : serverInfo) 
{
 831  
regionServers.add(server.getServerName().toLowerCase());

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/book.html
--
diff --git a/book.html b/book.html
index a026ac4..84af278 100644
--- a/book.html
+++ b/book.html
@@ -35095,7 +35095,7 @@ In case the table goes out of date, the unit tests 
which check for accuracy of p
 
 
 getClusterStatus
-superuser|global(A)
+any 
user
 
 
 Region
@@ -37140,7 +37140,7 @@ The server will return cellblocks compressed using this 
same compressor as long
 
 
 Version 3.0.0-SNAPSHOT
-Last updated 2018-01-03 14:29:41 UTC
+Last updated 2018-01-04 14:29:44 UTC
 
 
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/bulk-loads.html
--
diff --git a/bulk-loads.html b/bulk-loads.html
index e936ef1..10e19e8 100644
--- a/bulk-loads.html
+++ b/bulk-loads.html
@@ -7,7 +7,7 @@
   
 
 
-
+
 
 Apache HBase –  
   Bulk Loads in Apache HBase (TM)
@@ -311,7 +311,7 @@ under the License. -->
 https://www.apache.org/";>The Apache Software 
Foundation.
 All rights reserved.  
 
-  Last Published: 
2018-01-03
+  Last Published: 
2018-01-04
 
 
 



[48/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/apidocs/org/apache/hadoop/hbase/client/Admin.html
--
diff --git a/apidocs/org/apache/hadoop/hbase/client/Admin.html 
b/apidocs/org/apache/hadoop/hbase/client/Admin.html
index b1b8b15..92c3cec 100644
--- a/apidocs/org/apache/hadoop/hbase/client/Admin.html
+++ b/apidocs/org/apache/hadoop/hbase/client/Admin.html
@@ -18,7 +18,7 @@
 catch(err) {
 }
 //-->
-var methods = 
{"i0":6,"i1":6,"i2":6,"i3":50,"i4":6,"i5":6,"i6":18,"i7":6,"i8":6,"i9":6,"i10":6,"i11":6,"i12":50,"i13":50,"i14":6,"i15":6,"i16":6,"i17":6,"i18":6,"i19":6,"i20":6,"i21":6,"i22":6,"i23":6,"i24":6,"i25":38,"i26":38,"i27":38,"i28":38,"i29":6,"i30":6,"i31":6,"i32":6,"i33":6,"i34":6,"i35":6,"i36":50,"i37":6,"i38":6,"i39":6,"i40":6,"i41":6,"i42":6,"i43":6,"i44":6,"i45":6,"i46":38,"i47":6,"i48":6,"i49":6,"i50":6,"i51":6,"i52":6,"i53":6,"i54":38,"i55":6,"i56":6,"i57":38,"i58":38,"i59":6,"i60":38,"i61":6,"i62":6,"i63":6,"i64":6,"i65":38,"i66":38,"i67":50,"i68":6,"i69":6,"i70":6,"i71":6,"i72":38,"i73":38,"i74":6,"i75":50,"i76":6,"i77":6,"i78":6,"i79":38,"i80":38,"i81":18,"i82":6,"i83":6,"i84":6,"i85":6,"i86":6,"i87":6,"i88":6,"i89":6,"i90":6,"i91":6,"i92":6,"i93":18,"i94":6,"i95":18,"i96":6,"i97":38,"i98":6,"i99":6,"i100":6,"i101":38,"i102":50,"i103":50,"i104":18,"i105":6,"i106":6,"i107":6,"i108":18,"i109":6,"i110":6,"i111":38,"i112":38,"i113":38,"i114":38,"i115":6,"i116":6,"i11
 
7":6,"i118":6,"i119":6,"i120":6,"i121":6,"i122":6,"i123":6,"i124":6,"i125":50,"i126":6,"i127":38,"i128":6,"i129":6,"i130":18,"i131":6,"i132":6,"i133":6,"i134":6,"i135":6,"i136":6,"i137":6,"i138":38,"i139":6,"i140":6,"i141":6,"i142":6,"i143":6,"i144":38,"i145":6,"i146":6,"i147":6,"i148":38,"i149":38,"i150":6,"i151":38,"i152":38,"i153":38,"i154":38,"i155":38,"i156":6,"i157":38,"i158":6,"i159":6,"i160":6,"i161":6,"i162":6,"i163":6,"i164":6,"i165":38,"i166":6,"i167":6,"i168":6,"i169":50,"i170":6,"i171":6,"i172":6,"i173":6,"i174":6,"i175":38,"i176":6,"i177":38,"i178":6,"i179":6,"i180":6,"i181":6,"i182":6,"i183":6,"i184":6,"i185":6,"i186":6,"i187":6,"i188":6,"i189":6,"i190":6,"i191":6,"i192":6,"i193":50,"i194":6,"i195":50,"i196":50,"i197":50,"i198":6,"i199":50,"i200":6,"i201":6,"i202":6,"i203":6,"i204":6,"i205":6,"i206":6,"i207":6,"i208":38,"i209":38,"i210":6,"i211":6,"i212":6,"i213":6,"i214":6,"i215":50,"i216":6,"i217":6,"i218":6,"i219":6,"i220":6,"i221":6};
+var methods = 
{"i0":6,"i1":6,"i2":6,"i3":50,"i4":6,"i5":6,"i6":18,"i7":6,"i8":6,"i9":6,"i10":6,"i11":6,"i12":50,"i13":50,"i14":6,"i15":6,"i16":6,"i17":6,"i18":6,"i19":6,"i20":6,"i21":6,"i22":6,"i23":6,"i24":6,"i25":38,"i26":38,"i27":38,"i28":38,"i29":6,"i30":6,"i31":6,"i32":6,"i33":6,"i34":6,"i35":6,"i36":50,"i37":6,"i38":6,"i39":6,"i40":6,"i41":6,"i42":6,"i43":6,"i44":6,"i45":6,"i46":38,"i47":6,"i48":6,"i49":6,"i50":6,"i51":6,"i52":6,"i53":6,"i54":38,"i55":6,"i56":6,"i57":38,"i58":38,"i59":6,"i60":38,"i61":6,"i62":6,"i63":6,"i64":6,"i65":38,"i66":38,"i67":50,"i68":6,"i69":6,"i70":6,"i71":6,"i72":38,"i73":38,"i74":6,"i75":50,"i76":6,"i77":6,"i78":6,"i79":38,"i80":38,"i81":18,"i82":18,"i83":6,"i84":50,"i85":6,"i86":6,"i87":6,"i88":6,"i89":6,"i90":6,"i91":6,"i92":6,"i93":6,"i94":18,"i95":18,"i96":50,"i97":18,"i98":6,"i99":38,"i100":6,"i101":6,"i102":6,"i103":38,"i104":18,"i105":6,"i106":6,"i107":6,"i108":18,"i109":6,"i110":6,"i111":38,"i112":38,"i113":38,"i114":38,"i115":6,"i116":6,"i
 
117":6,"i118":6,"i119":6,"i120":6,"i121":6,"i122":6,"i123":6,"i124":6,"i125":50,"i126":6,"i127":38,"i128":6,"i129":6,"i130":18,"i131":6,"i132":6,"i133":6,"i134":6,"i135":6,"i136":6,"i137":6,"i138":38,"i139":6,"i140":6,"i141":6,"i142":6,"i143":6,"i144":38,"i145":6,"i146":6,"i147":6,"i148":38,"i149":38,"i150":6,"i151":38,"i152":38,"i153":38,"i154":38,"i155":38,"i156":6,"i157":38,"i158":6,"i159":6,"i160":6,"i161":6,"i162":6,"i163":6,"i164":6,"i165":38,"i166":6,"i167":6,"i168":6,"i169":50,"i170":6,"i171":6,"i172":6,"i173":6,"i174":6,"i175":38,"i176":6,"i177":38,"i178":6,"i179":6,"i180":6,"i181":6,"i182":6,"i183":6,"i184":6,"i185":6,"i186":6,"i187":6,"i188":6,"i189":6,"i190":6,"i191":6,"i192":6,"i193":50,"i194":6,"i195":50,"i196":50,"i197":50,"i198":6,"i199":50,"i200":6,"i201":6,"i202":6,"i203":6,"i204":6,"i205":6,"i206":6,"i207":6,"i208":38,"i209":38,"i210":6,"i211":6,"i212":6,"i213":6,"i214":6,"i215":50,"i216":6,"i217":6,"i218":6,"i219":6,"i220":6,"i221":6};
 var tabs = {65535:["t0","All Methods"],2:["t2","Instance 
Methods"],4:["t3","Abstract Methods"],16:["t5","Default 
Methods"],32:["t6","Deprecated Methods"]};
 var altColor = "altColor";
 var rowColor = "rowColor";
@@ -106,7 +106,7 @@ var activeTableTab = "activeTableTab";
 
 
 @InterfaceAudience.Public
-public interface Admin
+public interface Admin
 extends org.apache.hadoop.hbase.Abortable, http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.html?is-external=true";
 title="class or interface in java.io">Closeable
 The administrative API for HBase. 

[04/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/tool/Canary.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/tool/Canary.html 
b/devapidocs/org/apache/hadoop/hbase/tool/Canary.html
index eaf6bcb..1873694 100644
--- a/devapidocs/org/apache/hadoop/hbase/tool/Canary.html
+++ b/devapidocs/org/apache/hadoop/hbase/tool/Canary.html
@@ -848,7 +848,7 @@ implements org.apache.hadoop.util.Tool
 
 
 sniff
-private static http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">ListFutureVoid>> sniff(Admin admin,
+private static http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">ListFutureVoid>> sniff(Admin admin,
 Canary.Sink sink,
 http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String tableName,
 http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/ExecutorService.html?is-external=true";
 title="class or interface in 
java.util.concurrent">ExecutorService executor,
@@ -869,7 +869,7 @@ implements org.apache.hadoop.util.Tool
 
 
 sniff
-private static http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">ListFutureVoid>> sniff(Admin admin,
+private static http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">ListFutureVoid>> sniff(Admin admin,
 Canary.Sink sink,
 HTableDescriptor tableDesc,
 http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/ExecutorService.html?is-external=true";
 title="class or interface in 
java.util.concurrent">ExecutorService executor,
@@ -889,7 +889,7 @@ implements org.apache.hadoop.util.Tool
 
 
 main
-public static void main(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String[] args)
+public static void main(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String[] args)
  throws http://docs.oracle.com/javase/8/docs/api/java/lang/Exception.html?is-external=true";
 title="class or interface in java.lang">Exception
 
 Throws:

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.ErrorReporter.ERROR_CODE.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.ErrorReporter.ERROR_CODE.html
 
b/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.ErrorReporter.ERROR_CODE.html
index fb13dad..c86800e 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.ErrorReporter.ERROR_CODE.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.ErrorReporter.ERROR_CODE.html
@@ -122,7 +122,7 @@ var activeTableTab = "activeTableTab";
 
 
 
-public static enum HBaseFsck.ErrorReporter.ERROR_CODE
+public static enum HBaseFsck.ErrorReporter.ERROR_CODE
 extends http://docs.oracle.com/javase/8/docs/api/java/lang/Enum.html?is-external=true";
 title="class or interface in java.lang">Enum
 
 
@@ -309,7 +309,7 @@ the order they are declared.
 
 
 UNKNOWN
-public static final HBaseFsck.ErrorReporter.ERROR_CODE UNKNOWN
+public static final HBaseFsck.ErrorReporter.ERROR_CODE UNKNOWN
 
 
 
@@ -318,7 +318,7 @@ the order they are declared.
 
 
 NO_META_REGION
-public static final HBaseFsck.ErrorReporter.ERRO

[41/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/checkstyle-aggregate.html
--
diff --git a/checkstyle-aggregate.html b/checkstyle-aggregate.html
index 236593f..c7738b4 100644
--- a/checkstyle-aggregate.html
+++ b/checkstyle-aggregate.html
@@ -7,7 +7,7 @@
   
 
 
-
+
 
 Apache HBase – Checkstyle Results
 
@@ -289,7 +289,7 @@
 3471
 0
 0
-19048
+18828
 
 Files
 
@@ -397,7 +397,7 @@
 org/apache/hadoop/hbase/DistributedHBaseCluster.java
 0
 0
-5
+3
 
 org/apache/hadoop/hbase/DoNotRetryIOException.java
 0
@@ -422,7 +422,7 @@
 org/apache/hadoop/hbase/HBaseCluster.java
 0
 0
-4
+2
 
 org/apache/hadoop/hbase/HBaseClusterManager.java
 0
@@ -534,1410 +534,1400 @@
 0
 3
 
-org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java
-0
-0
-2
-
 org/apache/hadoop/hbase/IntegrationTestManyRegions.java
 0
 0
 2
-
+
 org/apache/hadoop/hbase/IntegrationTestMetaReplicas.java
 0
 0
 2
-
+
 org/apache/hadoop/hbase/IntegrationTestRegionReplicaPerf.java
 0
 0
 11
-
+
 org/apache/hadoop/hbase/IntegrationTestRegionReplicaReplication.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/IntegrationTestingUtility.java
 0
 0
 5
-
+
 org/apache/hadoop/hbase/JMXListener.java
 0
 0
 5
-
+
 org/apache/hadoop/hbase/KeyValue.java
 0
 0
 117
-
+
 org/apache/hadoop/hbase/KeyValueTestUtil.java
 0
 0
 8
-
+
 org/apache/hadoop/hbase/KeyValueUtil.java
 0
 0
 29
-
+
 org/apache/hadoop/hbase/LocalHBaseCluster.java
 0
 0
 24
-
+
 org/apache/hadoop/hbase/MetaMockingUtil.java
 0
 0
 4
-
+
 org/apache/hadoop/hbase/MetaMutationAnnotation.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/MetaTableAccessor.java
 0
 0
 114
-
+
 org/apache/hadoop/hbase/MiniHBaseCluster.java
 0
 0
-28
-
+26
+
 org/apache/hadoop/hbase/MockRegionServerServices.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/MultithreadedTestUtil.java
 0
 0
 3
-
+
 org/apache/hadoop/hbase/NamespaceDescriptor.java
 0
 0
 3
-
+
 org/apache/hadoop/hbase/NotAllMetaRegionsOnlineException.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/NotServingRegionException.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/PerformanceEvaluation.java
 0
 0
 29
-
+
 org/apache/hadoop/hbase/PerformanceEvaluationCommons.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/PrivateCellUtil.java
 0
 0
 67
-
+
 org/apache/hadoop/hbase/QosTestHelper.java
 0
 0
 3
-
+
 org/apache/hadoop/hbase/RESTApiClusterManager.java
 0
 0
 2
-
+
 org/apache/hadoop/hbase/RegionLoad.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/RegionLocations.java
 0
 0
 11
-
+
 org/apache/hadoop/hbase/RegionStateListener.java
 0
 0
 2
-
+
 org/apache/hadoop/hbase/ResourceChecker.java
 0
 0
 4
-
+
 org/apache/hadoop/hbase/ScanPerformanceEvaluation.java
 0
 0
 12
-
+
 org/apache/hadoop/hbase/ScheduledChore.java
 0
 0
 5
-
+
 org/apache/hadoop/hbase/Server.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/ServerLoad.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/ServerName.java
 0
 0
 24
-
+
 org/apache/hadoop/hbase/SplitLogCounters.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/SplitLogTask.java
 0
 0
 4
-
+
 org/apache/hadoop/hbase/StripeCompactionsPerformanceEvaluation.java
 0
 0
-5
-
+3
+
 org/apache/hadoop/hbase/TableDescriptors.java
 0
 0
 9
-
+
 org/apache/hadoop/hbase/TableInfoMissingException.java
 0
 0
 6
-
+
 org/apache/hadoop/hbase/TableName.java
 0
 0
 17
-
+
 org/apache/hadoop/hbase/TableNotDisabledException.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/TableNotEnabledException.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/TableNotFoundException.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/TagType.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/TestClassFinder.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/TestClientClusterStatus.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/TestFullLogReconstruction.java
 0
 0
 3
-
+
 org/apache/hadoop/hbase/TestGlobalMemStoreSize.java
 0
 0
 5
-
+
 org/apache/hadoop/hbase/TestHBaseConfiguration.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/TestHBaseOnOtherDfsCluster.java
 0
 0
 2
-
+
 org/apache/hadoop/hbase/TestHBaseTestingUtility.java
 0
 0
 6
-
+
 org/apache/hadoop/hbase/TestHColumnDescriptor.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/TestHColumnDescriptorDefaultVersions.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/TestHDFSBlocksDistribution.java
 0
 0
 2
-
+
 org/apache/hadoop/hbase/TestHTableDescriptor.java
 0
 0
 11
-
+
 org/apache/hadoop/hbase/TestIOFencing.java
 0
 0
 3
-
+
 org/apache/hadoop/hbase/TestInfoServers.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/TestInterfaceAudienceAnnotations.java
 0
 0
 2
-
+
 org/apache/hadoop/hbase/TestJMXListener.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/TestKeyValue.java
 0
 0
 3
-
+
 org/apache/hadoop/hbase/TestLocalHBaseCluster.java
 0
 0
 1
-
+
 org/apache/hadoop/hbase/TestMetaTableAccessor.java
 0
 0
 5
-
+
 org/apache/hadoop/hbase/TestMetaTableAccessorNoCluster.java
 0
 0
 8
-
+
 org/apache/hadoop/hbase/TestMetaTableLocator.java
 0
 0
 35
-
+
 org/apache/hadoo

[28/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.AddColumnFamilyProcedureBiConsumer.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.AddColumnFamilyProcedureBiConsumer.html
 
b/devapidocs/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.AddColumnFamilyProcedureBiConsumer.html
index f2b5705..f9f3d54 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.AddColumnFamilyProcedureBiConsumer.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.AddColumnFamilyProcedureBiConsumer.html
@@ -127,7 +127,7 @@ var activeTableTab = "activeTableTab";
 
 
 
-private class RawAsyncHBaseAdmin.AddColumnFamilyProcedureBiConsumer
+private class RawAsyncHBaseAdmin.AddColumnFamilyProcedureBiConsumer
 extends RawAsyncHBaseAdmin.TableProcedureBiConsumer
 
 
@@ -232,7 +232,7 @@ extends 
 
 AddColumnFamilyProcedureBiConsumer
-AddColumnFamilyProcedureBiConsumer(TableName tableName)
+AddColumnFamilyProcedureBiConsumer(TableName tableName)
 
 
 
@@ -249,7 +249,7 @@ extends 
 
 getOperationType
-http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String getOperationType()
+http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String getOperationType()
 
 Specified by:
 getOperationType in
 class RawAsyncHBaseAdmin.TableProcedureBiConsumer

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.AdminRpcCall.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.AdminRpcCall.html
 
b/devapidocs/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.AdminRpcCall.html
index 7d6b664..5479838 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.AdminRpcCall.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.AdminRpcCall.html
@@ -110,7 +110,7 @@ var activeTableTab = "activeTableTab";
 
 
 http://docs.oracle.com/javase/8/docs/api/java/lang/FunctionalInterface.html?is-external=true";
 title="class or interface in java.lang">@FunctionalInterface
-private static interface RawAsyncHBaseAdmin.AdminRpcCall
+private static interface RawAsyncHBaseAdmin.AdminRpcCall
 
 
 
@@ -159,7 +159,7 @@ private static interface 
 
 call
-void call(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.Interface stub,
+void call(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.Interface stub,
   HBaseRpcController controller,
   REQ req,
   org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback done)

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.Converter.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.Converter.html 
b/devapidocs/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.Converter.html
index 572d7cc..cd4daf6 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.Converter.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.Converter.html
@@ -110,7 +110,7 @@ var activeTableTab = "activeTableTab";
 
 
 http://docs.oracle.com/javase/8/docs/api/java/lang/FunctionalInterface.html?is-external=true";
 title="class or interface in java.lang">@FunctionalInterface
-private static interface RawAsyncHBaseAdmin.Converter
+private static interface RawAsyncHBaseAdmin.Converter
 
 
 
@@ -156,7 +156,7 @@ private static interface 
 
 convert
-D convert(S src)
+D convert(S src)
throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
 
 Throws:

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.CreateNamespaceProcedureBiConsumer.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.CreateNamespaceProcedureBiConsumer.html
 
b/devapidocs/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.CreateNamespaceProcedureBiConsumer.html
index 5942eea..01f186e 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.CreateNamespaceProcedureBiConsumer.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.CreateNamespaceProcedureBiConsumer.html
@@ -127,7 +127,7 @@ var activeTableTab = "activeTableTab";
 
 
 
-private class RawAsyncHBaseAdmin.CreateNamespaceProcedureBiConsumer
+private class RawAsyncHBaseAdmin.CreateN

[45/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/apidocs/src-html/org/apache/hadoop/hbase/client/Admin.html
--
diff --git a/apidocs/src-html/org/apache/hadoop/hbase/client/Admin.html 
b/apidocs/src-html/org/apache/hadoop/hbase/client/Admin.html
index 1c1fdeb..44c2515 100644
--- a/apidocs/src-html/org/apache/hadoop/hbase/client/Admin.html
+++ b/apidocs/src-html/org/apache/hadoop/hbase/client/Admin.html
@@ -33,19 +33,19 @@
 025import java.util.List;
 026import java.util.Map;
 027import java.util.Set;
-028import java.util.TreeMap;
-029import java.util.concurrent.Future;
-030import java.util.regex.Pattern;
-031import 
org.apache.hadoop.conf.Configuration;
-032import 
org.apache.hadoop.hbase.Abortable;
-033import 
org.apache.hadoop.hbase.CacheEvictionStats;
+028import java.util.concurrent.Future;
+029import java.util.regex.Pattern;
+030import 
org.apache.hadoop.conf.Configuration;
+031import 
org.apache.hadoop.hbase.Abortable;
+032import 
org.apache.hadoop.hbase.CacheEvictionStats;
+033import 
org.apache.hadoop.hbase.ClusterMetrics;
 034import 
org.apache.hadoop.hbase.ClusterMetrics.Option;
 035import 
org.apache.hadoop.hbase.ClusterStatus;
 036import 
org.apache.hadoop.hbase.HRegionInfo;
 037import 
org.apache.hadoop.hbase.HTableDescriptor;
 038import 
org.apache.hadoop.hbase.NamespaceDescriptor;
 039import 
org.apache.hadoop.hbase.NamespaceNotFoundException;
-040import 
org.apache.hadoop.hbase.RegionLoad;
+040import 
org.apache.hadoop.hbase.RegionMetrics;
 041import 
org.apache.hadoop.hbase.ServerName;
 042import 
org.apache.hadoop.hbase.TableExistsException;
 043import 
org.apache.hadoop.hbase.TableName;
@@ -64,2578 +64,2581 @@
 056import 
org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
 057import 
org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
 058import 
org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
-059import 
org.apache.hadoop.hbase.util.Bytes;
-060import 
org.apache.hadoop.hbase.util.Pair;
-061import 
org.apache.yetus.audience.InterfaceAudience;
-062
-063/**
-064 * The administrative API for HBase. 
Obtain an instance from {@link Connection#getAdmin()} and
-065 * call {@link #close()} when done.
-066 * 

Admin can be used to create, drop, list, enable and disable and otherwise modify tables, -067 * as well as perform other administrative operations. -068 * -069 * @see ConnectionFactory -070 * @see Connection -071 * @see Table -072 * @since 0.99.0 -073 */ -074@InterfaceAudience.Public -075public interface Admin extends Abortable, Closeable { -076 int getOperationTimeout(); -077 -078 @Override -079 void abort(String why, Throwable e); -080 -081 @Override -082 boolean isAborted(); -083 -084 /** -085 * @return Connection used by this object. -086 */ -087 Connection getConnection(); -088 -089 /** -090 * @param tableName Table to check. -091 * @return true if table exists already. -092 * @throws IOException -093 */ -094 boolean tableExists(TableName tableName) throws IOException; -095 -096 /** -097 * List all the userspace tables. -098 * -099 * @return an array of read-only HTableDescriptors -100 * @throws IOException if a remote or network exception occurs -101 * @deprecated since 2.0 version and will be removed in 3.0 version. -102 * Use {@link #listTableDescriptors()}. -103 * @see #listTableDescriptors() -104 */ -105 @Deprecated -106 HTableDescriptor[] listTables() throws IOException; -107 -108 /** -109 * List all the userspace tables. -110 * -111 * @return a list of TableDescriptors -112 * @throws IOException if a remote or network exception occurs -113 */ -114 List listTableDescriptors() throws IOException; -115 -116 /** -117 * List all the userspace tables that match the given pattern. -118 * -119 * @param pattern The compiled regular expression to match against -120 * @return an array of read-only HTableDescriptors -121 * @throws IOException if a remote or network exception occurs -122 * @see #listTables() -123 * @deprecated since 2.0 version and will be removed in 3.0 version. -124 * Use {@link #listTableDescriptors(java.util.regex.Pattern)}. -125 * @see #listTableDescriptors(Pattern) -126 */ -127 @Deprecated -128 HTableDescriptor[] listTables(Pattern pattern) throws IOException; -129 -130 /** -131 * List all the userspace tables that match the given pattern. -132 * -133 * @param pattern The compiled regular expression to match against -134 * @return a list of TableDescriptors -135 * @throws IOException if a remote or network exception occurs -136 * @see #listTables() -137 */ -138 List listTableDescriptors(Pattern pattern) throws IOException; -139 -140 /** -141 * List all the userspace tables matching the given regular expression. -142 * -143 * @param regex The regular expression to match a


[29/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/client/HBaseAdmin.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/client/HBaseAdmin.html 
b/devapidocs/org/apache/hadoop/hbase/client/HBaseAdmin.html
index 9dbe9f6..41c2a3b 100644
--- a/devapidocs/org/apache/hadoop/hbase/client/HBaseAdmin.html
+++ b/devapidocs/org/apache/hadoop/hbase/client/HBaseAdmin.html
@@ -18,7 +18,7 @@
 catch(err) {
 }
 //-->
-var methods = 
{"i0":10,"i1":10,"i2":10,"i3":10,"i4":10,"i5":10,"i6":10,"i7":10,"i8":10,"i9":9,"i10":10,"i11":10,"i12":10,"i13":10,"i14":10,"i15":10,"i16":10,"i17":10,"i18":10,"i19":10,"i20":10,"i21":10,"i22":10,"i23":10,"i24":10,"i25":10,"i26":42,"i27":42,"i28":42,"i29":42,"i30":10,"i31":10,"i32":10,"i33":10,"i34":10,"i35":10,"i36":10,"i37":10,"i38":10,"i39":10,"i40":10,"i41":10,"i42":10,"i43":10,"i44":10,"i45":10,"i46":10,"i47":10,"i48":10,"i49":42,"i50":10,"i51":10,"i52":10,"i53":10,"i54":10,"i55":10,"i56":10,"i57":10,"i58":10,"i59":10,"i60":10,"i61":10,"i62":10,"i63":10,"i64":10,"i65":10,"i66":10,"i67":10,"i68":10,"i69":10,"i70":10,"i71":10,"i72":10,"i73":10,"i74":10,"i75":10,"i76":10,"i77":10,"i78":10,"i79":9,"i80":10,"i81":10,"i82":9,"i83":10,"i84":10,"i85":10,"i86":10,"i87":10,"i88":10,"i89":10,"i90":10,"i91":10,"i92":10,"i93":41,"i94":10,"i95":10,"i96":10,"i97":10,"i98":10,"i99":42,"i100":10,"i101":10,"i102":10,"i103":10,"i104":10,"i105":10,"i106":10,"i107":10,"i108":10,"i109
 
":10,"i110":10,"i111":10,"i112":10,"i113":10,"i114":9,"i115":10,"i116":10,"i117":10,"i118":42,"i119":10,"i120":10,"i121":10,"i122":10,"i123":10,"i124":10,"i125":10,"i126":10,"i127":10,"i128":10,"i129":10,"i130":10,"i131":10,"i132":10,"i133":10,"i134":10,"i135":10,"i136":10,"i137":10,"i138":10,"i139":10,"i140":10,"i141":10,"i142":10,"i143":10,"i144":10,"i145":10,"i146":10,"i147":10,"i148":10,"i149":10,"i150":10,"i151":10,"i152":10,"i153":10,"i154":10,"i155":10,"i156":10,"i157":10,"i158":10,"i159":10,"i160":10,"i161":10,"i162":10,"i163":10,"i164":10,"i165":10,"i166":10,"i167":10,"i168":10,"i169":10,"i170":42,"i171":10,"i172":10,"i173":10,"i174":10,"i175":10,"i176":10,"i177":10,"i178":10,"i179":10,"i180":10,"i181":10,"i182":10,"i183":10,"i184":10,"i185":10,"i186":10,"i187":10,"i188":10,"i189":10,"i190":10,"i191":10,"i192":10,"i193":10,"i194":10,"i195":10,"i196":42,"i197":10,"i198":10,"i199":10,"i200":10,"i201":10,"i202":10,"i203":10,"i204":10,"i205":10,"i206":10,"i207":10,"i208":10,"i2
 
09":10,"i210":10,"i211":10,"i212":10,"i213":10,"i214":10,"i215":10,"i216":10,"i217":10,"i218":10,"i219":10,"i220":10,"i221":10,"i222":10,"i223":10,"i224":10,"i225":10,"i226":10,"i227":10};
+var methods = 
{"i0":10,"i1":10,"i2":10,"i3":10,"i4":10,"i5":10,"i6":10,"i7":10,"i8":10,"i9":9,"i10":10,"i11":10,"i12":10,"i13":10,"i14":10,"i15":10,"i16":10,"i17":10,"i18":10,"i19":10,"i20":10,"i21":10,"i22":10,"i23":10,"i24":10,"i25":10,"i26":42,"i27":42,"i28":42,"i29":42,"i30":10,"i31":10,"i32":10,"i33":10,"i34":10,"i35":10,"i36":10,"i37":10,"i38":10,"i39":10,"i40":10,"i41":10,"i42":10,"i43":10,"i44":10,"i45":10,"i46":10,"i47":10,"i48":10,"i49":42,"i50":10,"i51":10,"i52":10,"i53":10,"i54":10,"i55":10,"i56":10,"i57":10,"i58":10,"i59":10,"i60":10,"i61":10,"i62":10,"i63":10,"i64":10,"i65":10,"i66":10,"i67":10,"i68":10,"i69":10,"i70":10,"i71":10,"i72":10,"i73":10,"i74":10,"i75":10,"i76":10,"i77":10,"i78":10,"i79":9,"i80":10,"i81":10,"i82":9,"i83":10,"i84":10,"i85":10,"i86":10,"i87":10,"i88":10,"i89":10,"i90":10,"i91":10,"i92":41,"i93":10,"i94":10,"i95":10,"i96":10,"i97":42,"i98":10,"i99":10,"i100":10,"i101":10,"i102":10,"i103":10,"i104":10,"i105":10,"i106":10,"i107":10,"i108":10,"i109
 
":10,"i110":10,"i111":10,"i112":9,"i113":10,"i114":10,"i115":10,"i116":42,"i117":10,"i118":10,"i119":10,"i120":10,"i121":10,"i122":10,"i123":10,"i124":10,"i125":10,"i126":10,"i127":10,"i128":10,"i129":10,"i130":10,"i131":10,"i132":10,"i133":10,"i134":10,"i135":10,"i136":10,"i137":10,"i138":10,"i139":10,"i140":10,"i141":10,"i142":10,"i143":10,"i144":10,"i145":10,"i146":10,"i147":10,"i148":10,"i149":10,"i150":10,"i151":10,"i152":10,"i153":10,"i154":10,"i155":10,"i156":10,"i157":10,"i158":10,"i159":10,"i160":10,"i161":10,"i162":10,"i163":10,"i164":10,"i165":10,"i166":10,"i167":10,"i168":42,"i169":10,"i170":10,"i171":10,"i172":10,"i173":10,"i174":10,"i175":10,"i176":10,"i177":10,"i178":10,"i179":10,"i180":10,"i181":10,"i182":10,"i183":10,"i184":10,"i185":10,"i186":10,"i187":10,"i188":10,"i189":10,"i190":10,"i191":10,"i192":10,"i193":10,"i194":42,"i195":10,"i196":10,"i197":10,"i198":10,"i199":10,"i200":10,"i201":10,"i202":10,"i203":10,"i204":10,"i205":10,"i206":10,"i207":10,"i208":10,"i2
 
09":10,"i210":10,"i211":10,"i212":10,"i213":10,"i214":10,"i215":10,"i216":10,"i217":10,"i218":10,"i219":10,"i220":10,"i221":10,"i222":10,"i223":10,"i224":10,"i225":10};
 var tabs = {65535:["t0","All Methods"],1:["t1","Static 
Methods"],2:["t2","Instance Methods"],8:["t4","

[01/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
Repository: hbase-site
Updated Branches:
  refs/heads/asf-site 1f4f0eec6 -> c7c40c622


http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/wal/WALSplitter.EntryBuffers.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/wal/WALSplitter.EntryBuffers.html 
b/devapidocs/org/apache/hadoop/hbase/wal/WALSplitter.EntryBuffers.html
index b7c7db0..d571cc6 100644
--- a/devapidocs/org/apache/hadoop/hbase/wal/WALSplitter.EntryBuffers.html
+++ b/devapidocs/org/apache/hadoop/hbase/wal/WALSplitter.EntryBuffers.html
@@ -113,7 +113,7 @@ var activeTableTab = "activeTableTab";
 
 
 
-public static class WALSplitter.EntryBuffers
+public static class WALSplitter.EntryBuffers
 extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object
 Class which accumulates edits and separates them into a 
buffer per region
  while simultaneously accounting RAM usage. Blocks if the RAM usage crosses
@@ -250,7 +250,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 controller
-WALSplitter.PipelineController controller
+WALSplitter.PipelineController controller
 
 
 
@@ -259,7 +259,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 buffers
-http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">Map buffers
+http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">Map buffers
 
 
 
@@ -268,7 +268,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 currentlyWriting
-http://docs.oracle.com/javase/8/docs/api/java/util/Set.html?is-external=true";
 title="class or interface in java.util">Set currentlyWriting
+http://docs.oracle.com/javase/8/docs/api/java/util/Set.html?is-external=true";
 title="class or interface in java.util">Set currentlyWriting
 
 
 
@@ -277,7 +277,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 totalBuffered
-long totalBuffered
+long totalBuffered
 
 
 
@@ -286,7 +286,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 maxHeapUsage
-long maxHeapUsage
+long maxHeapUsage
 
 
 
@@ -295,7 +295,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 splitWriterCreationBounded
-boolean splitWriterCreationBounded
+boolean splitWriterCreationBounded
 
 
 
@@ -312,7 +312,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 EntryBuffers
-public EntryBuffers(WALSplitter.PipelineController controller,
+public EntryBuffers(WALSplitter.PipelineController controller,
 long maxHeapUsage)
 
 
@@ -322,7 +322,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 EntryBuffers
-public EntryBuffers(WALSplitter.PipelineController controller,
+public EntryBuffers(WALSplitter.PipelineController controller,
 long maxHeapUsage,
 boolean splitWriterCreationBounded)
 
@@ -341,7 +341,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 appendEntry
-public void appendEntry(WAL.Entry entry)
+public void appendEntry(WAL.Entry entry)
  throws http://docs.oracle.com/javase/8/docs/api/java/lang/InterruptedException.html?is-external=true";
 title="class or interface in java.lang">InterruptedException,
 http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
 Append a log entry into the corresponding region buffer.
@@ -359,7 +359,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 getChunkToWrite
-WALSplitter.RegionEntryBuffer getChunkToWrite()
+WALSplitter.RegionEntryBuffer getChunkToWrite()
 
 Returns:
 RegionEntryBuffer a buffer of edits to be written.
@@ -372,7 +372,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 doneWriting
-void doneWriting(WALSplitter.RegionEntryBuffer buffer)
+void doneWriting(WALSplitter.RegionEntryBuffer buffer)
 
 
 
@@ -381,7 +381,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 isRegionCurrentlyWriting
-boolean isRegionCurrentlyWriting(byte[] region)
+boolean isRegionCurrentlyWriting(byte[] region)
 
 
 
@@ -390,7 +390,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 waitUntilDrained
-public void waitUntilDrained()
+public void waitUntilDrained()
 
 
 



[08/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.html 
b/devapidocs/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.html
index aecc1f6..4b22219 100644
--- a/devapidocs/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.html
+++ b/devapidocs/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.html
@@ -114,7 +114,7 @@ var activeTableTab = "activeTableTab";
 
 
 @InterfaceAudience.Private
-public class RSGroupBasedLoadBalancer
+public class RSGroupBasedLoadBalancer
 extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object
 implements RSGroupableBalancer
 GroupBasedLoadBalancer, used when Region Server Grouping is 
configured (HBase-6721)
@@ -147,7 +147,7 @@ implements Field and Description
 
 
-private ClusterStatus
+private ClusterMetrics
 clusterStatus 
 
 
@@ -330,7 +330,7 @@ implements 
 void
-setClusterStatus(ClusterStatus st)
+setClusterMetrics(ClusterMetrics sm)
 Set the current cluster status.
 
 
@@ -389,7 +389,7 @@ implements 
 
 LOG
-private static final org.slf4j.Logger LOG
+private static final org.slf4j.Logger LOG
 
 
 
@@ -398,7 +398,7 @@ implements 
 
 config
-private org.apache.hadoop.conf.Configuration config
+private org.apache.hadoop.conf.Configuration config
 
 
 
@@ -407,7 +407,7 @@ implements 
 
 clusterStatus
-private ClusterStatus clusterStatus
+private ClusterMetrics clusterStatus
 
 
 
@@ -416,7 +416,7 @@ implements 
 
 masterServices
-private MasterServices masterServices
+private MasterServices masterServices
 
 
 
@@ -425,7 +425,7 @@ implements 
 
 rsGroupInfoManager
-private volatile RSGroupInfoManager rsGroupInfoManager
+private volatile RSGroupInfoManager rsGroupInfoManager
 
 
 
@@ -434,7 +434,7 @@ implements 
 
 internalBalancer
-private LoadBalancer internalBalancer
+private LoadBalancer internalBalancer
 
 
 
@@ -452,7 +452,7 @@ implements 
 RSGroupBasedLoadBalancer
 @InterfaceAudience.Private
-public RSGroupBasedLoadBalancer()
+public RSGroupBasedLoadBalancer()
 Used by reflection in LoadBalancerFactory.
 
 
@@ -470,7 +470,7 @@ public 
 
 getConf
-public org.apache.hadoop.conf.Configuration getConf()
+public org.apache.hadoop.conf.Configuration getConf()
 
 Specified by:
 getConf in 
interface org.apache.hadoop.conf.Configurable
@@ -483,25 +483,25 @@ public 
 
 setConf
-public void setConf(org.apache.hadoop.conf.Configuration conf)
+public void setConf(org.apache.hadoop.conf.Configuration conf)
 
 Specified by:
 setConf in 
interface org.apache.hadoop.conf.Configurable
 
 
 
-
+
 
 
 
 
-setClusterStatus
-public void setClusterStatus(ClusterStatus st)
-Description copied from 
interface: LoadBalancer
+setClusterMetrics
+public void setClusterMetrics(ClusterMetrics sm)
+Description copied from 
interface: LoadBalancer
 Set the current cluster status.  This allows a LoadBalancer 
to map host name to a server
 
 Specified by:
-setClusterStatus in
 interface LoadBalancer
+setClusterMetrics in
 interface LoadBalancer
 
 
 
@@ -511,7 +511,7 @@ public 
 
 setMasterServices
-public void setMasterServices(MasterServices masterServices)
+public void setMasterServices(MasterServices masterServices)
 Description copied from 
interface: LoadBalancer
 Set the master service.
 
@@ -526,7 +526,7 @@ public 
 
 balanceCluster
-public http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List balanceCluster(TableName tableName,
+public http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List balanceCluster(TableName tableName,
http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">MapList> clusterState)
 throws HBaseIOException
 Description copied from 
interface: LoadBalancer
@@ -547,7 +547,7 @@ public 
 
 balanceCluster
-public http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List balanceCluster(http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">MapList> clusterState)
+public http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List balanceCluster(h

[14/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.RandomizedMatrix.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.RandomizedMatrix.html
 
b/devapidocs/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.RandomizedMatrix.html
index c4b0754..1e3aa89 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.RandomizedMatrix.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.RandomizedMatrix.html
@@ -113,7 +113,7 @@ var activeTableTab = "activeTableTab";
 
 
 
-protected static class RegionPlacementMaintainer.RandomizedMatrix
+protected static class RegionPlacementMaintainer.RandomizedMatrix
 extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object
 Some algorithms for solving the assignment problem may 
traverse workers or
  jobs in linear order which may result in skewing the assignments of the
@@ -255,7 +255,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 rows
-private final int rows
+private final int rows
 
 
 
@@ -264,7 +264,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 cols
-private final int cols
+private final int cols
 
 
 
@@ -273,7 +273,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 rowTransform
-private final int[] rowTransform
+private final int[] rowTransform
 
 
 
@@ -282,7 +282,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 rowInverse
-private final int[] rowInverse
+private final int[] rowInverse
 
 
 
@@ -291,7 +291,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 colTransform
-private final int[] colTransform
+private final int[] colTransform
 
 
 
@@ -300,7 +300,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 colInverse
-private final int[] colInverse
+private final int[] colInverse
 
 
 
@@ -317,7 +317,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 RandomizedMatrix
-public RandomizedMatrix(int rows,
+public RandomizedMatrix(int rows,
 int cols)
 Create a randomization scheme for a matrix of a given 
size.
 
@@ -341,7 +341,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 transform
-public float[][] transform(float[][] matrix)
+public float[][] transform(float[][] matrix)
 Copy a given matrix into a new matrix, transforming each 
row index and
  each column index according to the randomization scheme that was created
  at construction time.
@@ -359,7 +359,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 invert
-public float[][] invert(float[][] matrix)
+public float[][] invert(float[][] matrix)
 Copy a given matrix into a new matrix, transforming each 
row index and
  each column index according to the inverse of the randomization scheme
  that was created at construction time.
@@ -377,7 +377,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 invertIndices
-public int[] invertIndices(int[] indices)
+public int[] invertIndices(int[] indices)
 Given an array where each element indices[i] 
represents the
  randomized column index corresponding to randomized row index i,
  create a new array with the corresponding inverted indices.

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.html 
b/devapidocs/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.html
index bb0399a..5d7b682 100644
--- a/devapidocs/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.html
+++ b/devapidocs/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.html
@@ -599,7 +599,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 getNewAssignmentPlan
-public FavoredNodesPlan getNewAssignmentPlan()
+public FavoredNodesPlan getNewAssignmentPlan()
   throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
 
 Throws:
@@ -613,7 +613,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 printAssignmentPlan
-public static void printAssignmentPlan(FavoredNodesPlan plan)
+public static void printAssignmentPlan(FavoredNodesPlan plan)
 Print the assignment plan to the system output stream
 
 Parameters:
@@ -627,7 +627,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 updateAssignmentPlanToMeta
-pub

[38/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/RegionMetricsBuilder.RegionMetricsImpl.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/RegionMetricsBuilder.RegionMetricsImpl.html
 
b/devapidocs/org/apache/hadoop/hbase/RegionMetricsBuilder.RegionMetricsImpl.html
index fc9638f..b759bfd 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/RegionMetricsBuilder.RegionMetricsImpl.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/RegionMetricsBuilder.RegionMetricsImpl.html
@@ -117,7 +117,7 @@ var activeTableTab = "activeTableTab";
 
 
 
-private static class RegionMetricsBuilder.RegionMetricsImpl
+private static class RegionMetricsBuilder.RegionMetricsImpl
 extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object
 implements RegionMetrics
 
@@ -385,7 +385,7 @@ implements 
 
 name
-private final byte[] name
+private final byte[] name
 
 
 
@@ -394,7 +394,7 @@ implements 
 
 storeCount
-private final int storeCount
+private final int storeCount
 
 
 
@@ -403,7 +403,7 @@ implements 
 
 storeFileCount
-private final int storeFileCount
+private final int storeFileCount
 
 
 
@@ -412,7 +412,7 @@ implements 
 
 compactingCellCount
-private final long compactingCellCount
+private final long compactingCellCount
 
 
 
@@ -421,7 +421,7 @@ implements 
 
 compactedCellCount
-private final long compactedCellCount
+private final long compactedCellCount
 
 
 
@@ -430,7 +430,7 @@ implements 
 
 storeFileSize
-private final Size storeFileSize
+private final Size storeFileSize
 
 
 
@@ -439,7 +439,7 @@ implements 
 
 memStoreSize
-private final Size memStoreSize
+private final Size memStoreSize
 
 
 
@@ -448,7 +448,7 @@ implements 
 
 indexSize
-private final Size indexSize
+private final Size indexSize
 
 
 
@@ -457,7 +457,7 @@ implements 
 
 rootLevelIndexSize
-private final Size rootLevelIndexSize
+private final Size rootLevelIndexSize
 
 
 
@@ -466,7 +466,7 @@ implements 
 
 uncompressedDataIndexSize
-private final Size uncompressedDataIndexSize
+private final Size uncompressedDataIndexSize
 
 
 
@@ -475,7 +475,7 @@ implements 
 
 bloomFilterSize
-private final Size bloomFilterSize
+private final Size bloomFilterSize
 
 
 
@@ -484,7 +484,7 @@ implements 
 
 uncompressedStoreFileSize
-private final Size uncompressedStoreFileSize
+private final Size uncompressedStoreFileSize
 
 
 
@@ -493,7 +493,7 @@ implements 
 
 writeRequestCount
-private final long writeRequestCount
+private final long writeRequestCount
 
 
 
@@ -502,7 +502,7 @@ implements 
 
 readRequestCount
-private final long readRequestCount
+private final long readRequestCount
 
 
 
@@ -511,7 +511,7 @@ implements 
 
 filteredReadRequestCount
-private final long filteredReadRequestCount
+private final long filteredReadRequestCount
 
 
 
@@ -520,7 +520,7 @@ implements 
 
 completedSequenceId
-private final long completedSequenceId
+private final long completedSequenceId
 
 
 
@@ -529,7 +529,7 @@ implements 
 
 storeSequenceIds
-private final http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">MapLong> storeSequenceIds
+private final http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">MapLong> storeSequenceIds
 
 
 
@@ -538,7 +538,7 @@ implements 
 
 dataLocality
-private final float dataLocality
+private final float dataLocality
 
 
 
@@ -547,7 +547,7 @@ implements 
 
 lastMajorCompactionTimestamp
-private final long lastMajorCompactionTimestamp
+private final long lastMajorCompactionTimestamp
 
 
 
@@ -564,7 +564,7 @@ implements 
 
 RegionMetricsImpl
-RegionMetricsImpl(byte[] name,
+RegionMetricsImpl(byte[] name,
   int storeCount,
   int storeFileCount,
   long compactingCellCount,
@@ -599,7 +599,7 @@ implements 
 
 getRegionName
-public byte[] getRegionName()
+public byte[] getRegionName()
 
 Specified by:
 getRegionName in
 interface RegionMetrics
@@ -614,7 +614,7 @@ implements 
 
 getStoreCount
-public int getStoreCount()
+public int getStoreCount()
 
 Specified by:
 getStoreCount in
 interface RegionMetrics
@@ -629,7 +629,7 @@ implements 
 
 getStoreFileCount
-public int getStoreFileCount()
+public int getStoreFileCount()
 
 Specified by:
 getStoreFileCount in
 interface RegionMetrics
@@ -644,7 +644,7 @@ implements 
 
 getStoreFileSize
-public Size getStoreFileSize()
+public Size getStoreFileSize()
 
 Specified by:
 getStoreFileSize in
 interface RegionMetrics
@@ -659,7 +659,7 @@ implements 
 
 getMemStoreSize
-public Size getMemStore

[25/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/coprocessor/MasterObserver.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/coprocessor/MasterObserver.html 
b/devapidocs/org/apache/hadoop/hbase/coprocessor/MasterObserver.html
index 9a4f620..92114a4 100644
--- a/devapidocs/org/apache/hadoop/hbase/coprocessor/MasterObserver.html
+++ b/devapidocs/org/apache/hadoop/hbase/coprocessor/MasterObserver.html
@@ -107,7 +107,7 @@ var activeTableTab = "activeTableTab";
 
 @InterfaceAudience.LimitedPrivate(value="Coprocesssor")
  @InterfaceStability.Evolving
-public interface MasterObserver
+public interface MasterObserver
 Defines coprocessor hooks for interacting with operations 
on the
  HMaster 
process.
  
@@ -353,8 +353,8 @@ public interface 
 default void
-postGetClusterStatus(ObserverContext ctx,
-ClusterStatus status)
+postGetClusterMetrics(ObserverContext ctx,
+ ClusterMetrics status)
 Called after get cluster status.
 
 
@@ -833,7 +833,7 @@ public interface 
 default void
-preGetClusterStatus(ObserverContext ctx)
+preGetClusterMetrics(ObserverContext ctx)
 Called before get cluster status.
 
 
@@ -1217,7 +1217,7 @@ public interface 
 
 preCreateTable
-default void preCreateTable(ObserverContext ctx,
+default void preCreateTable(ObserverContext ctx,
 TableDescriptor desc,
 RegionInfo[] regions)
  throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
@@ -1240,7 +1240,7 @@ public interface 
 
 postCreateTable
-default void postCreateTable(ObserverContext ctx,
+default void postCreateTable(ObserverContext ctx,
  TableDescriptor desc,
  RegionInfo[] regions)
   throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
@@ -1262,7 +1262,7 @@ public interface 
 
 preCreateTableAction
-default void preCreateTableAction(ObserverContext ctx,
+default void preCreateTableAction(ObserverContext ctx,
   TableDescriptor desc,
   RegionInfo[] regions)
throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
@@ -1285,7 +1285,7 @@ public interface 
 
 postCompletedCreateTableAction
-default void postCompletedCreateTableAction(ObserverContext ctx,
+default void postCompletedCreateTableAction(ObserverContext ctx,
 TableDescriptor desc,
 RegionInfo[] regions)
  throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
@@ -1308,7 +1308,7 @@ public interface 
 
 preDeleteTable
-default void preDeleteTable(ObserverContext ctx,
+default void preDeleteTable(ObserverContext ctx,
 TableName tableName)
  throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
 Called before HMaster deletes a
@@ -1328,7 +1328,7 @@ public interface 
 
 postDeleteTable
-default void postDeleteTable(ObserverContext ctx,
+default void postDeleteTable(ObserverContext ctx,
  TableName tableName)
   throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
 Called after the deleteTable operation has been requested.  
Called as part
@@ -1348,7 +1348,7 @@ public interface 
 
 preDeleteTableAction
-default void preDeleteTableAction(ObserverContext ctx,
+default void preDeleteTableAction(ObserverContext ctx,
   TableName tableName)
throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
 Called before HMaster deletes a
@@ -1369,7 +1369,7 @@ public interface 
 
 postCom

[47/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/apidocs/org/apache/hadoop/hbase/client/AsyncAdmin.html
--
diff --git a/apidocs/org/apache/hadoop/hbase/client/AsyncAdmin.html 
b/apidocs/org/apache/hadoop/hbase/client/AsyncAdmin.html
index 870079e..775ff45 100644
--- a/apidocs/org/apache/hadoop/hbase/client/AsyncAdmin.html
+++ b/apidocs/org/apache/hadoop/hbase/client/AsyncAdmin.html
@@ -102,7 +102,7 @@ var activeTableTab = "activeTableTab";
 
 
 @InterfaceAudience.Public
-public interface AsyncAdmin
+public interface AsyncAdmin
 The asynchronous administrative API for HBase.
 
 Since:
@@ -434,12 +434,12 @@ public interface getBackupMasters() 
 
 
-http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFuture
-getClusterStatus() 
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFuture
+getClusterMetrics() 
 
 
-http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFuture
-getClusterStatus(http://docs.oracle.com/javase/8/docs/api/java/util/EnumSet.html?is-external=true";
 title="class or interface in 
java.util">EnumSet options) 
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFuture
+getClusterMetrics(http://docs.oracle.com/javase/8/docs/api/java/util/EnumSet.html?is-external=true";
 title="class or interface in 
java.util">EnumSet options) 
 
 
 default http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFuture
@@ -490,7 +490,7 @@ public interface 
 default http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureListString>>
-getMasterCoprocessors() 
+getMasterCoprocessorNames() 
 
 
 default http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureInteger>
@@ -517,16 +517,16 @@ public interface 
-http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
-getRegionLoads(ServerName serverName)
-Get a list of RegionLoad of all regions hosted on a 
region seerver.
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
+getRegionMetrics(ServerName serverName)
+Get a list of RegionMetrics of 
all regions hosted on a region seerver.
 
 
 
-http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
-getRegionLoads(ServerName serverName,
-  TableName tableName)
-Get a list of RegionLoad of all regions hosted on a 
region seerver for a table.
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
+getRegionMetrics(ServerName serverName,
+TableName tableName)
+Get a list of RegionMetrics of 
all regions hosted on a region seerver for a table.
 
 
 
@@ -1041,7 +1041,7 @@ public interface 
 
 tableExists
-http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/C

[06/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/security/access/CoprocessorWhitelistMasterObserver.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/security/access/CoprocessorWhitelistMasterObserver.html
 
b/devapidocs/org/apache/hadoop/hbase/security/access/CoprocessorWhitelistMasterObserver.html
index a9b316c..b9ee03e 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/security/access/CoprocessorWhitelistMasterObserver.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/security/access/CoprocessorWhitelistMasterObserver.html
@@ -255,7 +255,7 @@ implements MasterObserver
-postAbortProcedure,
 postAddReplicationPeer,
 postAddRSGroup,
 postAssign,
 postBalance, postBalanceRSGroup,
 postBalanceSwitch,
 postClearDeadServers,
 postCloneSnapshot,
 postCompletedCreateTableAction,
 postCompletedDeleteTableAction,
 postCompletedDisableTableAction,
 postCompletedEnableTableAction,
 postCompletedMergeRegionsAction,
 postCompletedModifyTableAction,
 postCompletedSplitRegionAction,
 postCompletedTruncateTableAction, postCreateNamespace,
 postCreateTable,
 postDecommissionRegionServers,
 postDeleteNamespace,
 postDeleteSnapshot,
 postDeleteTable,
 postDisableReplicationPeer,
 postDisableTable,
 postEnableReplicationPeer, postEnableTable,
 postGetClusterStatus,
 postGetLocks,
 postGetNamespaceDescriptor,
 postGetProcedures, postGetReplicationPeerConfig,
 postGetTableDescriptors,
 postGetTableNames,
 postListDecommissionedRegionServers,
 postListNamespaceDescriptors,
 postListReplicationPeers,
 postListSnapshot,
 postLockHeartbeat,
 postMergeRegions,
 postMergeRegionsCommitAction,
 postModifyNamespace,
 postModifyTable,
 postMove, postMoveServers,
 postMoveServersAndTables,
 postMoveTables,
 postRecommissionRegionServer,
 postRegionOffline,
 postRemoveReplicationPeer,
 postRemoveRSGroup,
 postRemoveServers,
 postRequestLock, postRestoreSnapshot,
 postRollBackMergeRegionsAction,
 postRollBackSplitRegionAction,
 pos
 tSetNamespaceQuota, postSetSplitOrMergeEnabled,
 postSetTableQuota,
 postSetUserQuota,
 postSetUserQuota, postSetUserQuota,
 postSnapshot,
 postStartMaster,
 postTableFlush,
 postTruncateTable,
 postUnassign,
 postUpdateReplicationPeerConfig,
 preAbortProcedure,
 preAddReplicationPeer,
 preAddRSGroup,
 preAssign,
 preBalance,
 preBalanceRSGroup,
 preBalanceSwitch, preClearDeadServers,
 preCloneSnapshot,
 preCreateNamespace,
 preCreateTableAction, href="../../../../../../org/apache/hadoop/hbase/coprocessor/MasterObserver.html#preDecommissionRegionServers-org.apache.hadoop.hbase.coprocessor.ObserverContext-java.util.List-boolean-">preDecommissionRegionServers,
 > href="../../../../../../org/apache/hadoop/hbase/coprocessor/MasterObserver.html#preDeleteNamespace-org.apache.hadoop.hbase.coprocessor.ObserverContext-java.lang.String-">preDeleteNamespace,
 > href="../../../../../../org/apache/hadoop/hbase/coprocessor/MasterObserver.html#preDeleteSnapshot-org.apache.hadoop.hbase.coprocessor.ObserverContext-org.apache.hadoop.hbase.client.SnapshotDescription-">preDeleteSnapshot,
 > href="../../../../../../org/apache/hadoop/hbase/coprocessor/MasterObserver.html#preDeleteTable-org.apache.hadoop.hbase.coprocessor.ObserverContext-org.apache.hadoop.hbase.TableName-">preDeleteTable,
 > href="../../../../../../org/apache/hadoop/hbase/coprocessor/MasterObserver.html#preDeleteTableAction-org.apache.
 
hadoop.hbase.coprocessor.ObserverContext-org.apache.hadoop.hbase.TableName-">preDeleteTableAction,
 preDisableReplicationPeer,
 preDisableTable,
 preDisableTableAction,
 preEnableReplicationPeer,
 preEnableTable,
 preEnableTableAction,
 preGetClusterStatus,
 preGetLocks,
 preGetNamespaceDescriptor,
 preGetProcedures,
 preGetReplicationPeerConfig,
 preGetTableDescriptors,
 preGetTableNames,
 preListDecommissionedRegionServers,
 preListNamespaceDescriptors,
 preListReplicationPeers,
 preListSnapshot,
 preLockHeartbeat,
 preMasterInitialization, preMergeRegions,
 preMergeRegionsAction,
 preMergeRegionsCommitAction,
 preModifyNamespace,
 preModifyTableAction,
 preMove,
 preMoveServers,
 preMoveServersAndTables,
 preMoveTables,
 preRecommissionRegionServer,
 preRegionOffline,
 preRemoveReplicationPeer,
 preRemoveRSGroup, preRemoveServers,
 preRequestLock,
 preRestoreSnapshot,
 preSetNamespaceQuota, preSetSplitOrMergeEnabled,
 preSetTableQuota,
 preSetUserQuota,
 preSetUserQuota, preSetUserQuota,
 preShutdown,
 preSnapshot,
 preSplitRegion,
 preSplitRegionAction,
 preSplitRegionAfterMETAAction,
 preSplitRegionBeforeMETAAction,
 preStopMaster,
 preTableFlu
 sh, preTruncateTable,
 preTruncateTableAction,
 preUnassign,
 preUpdateReplicationPeerConfig
+postAbortProcedure,
 postAddReplicationPeer,
 postAddRSGroup,
 postAssign,
 postBalance, postBalanceRSGroup,
 postBalanceSwitch,
 postClearDeadServers,
 postCloneSnapshot,
 postCompletedCreateTableAction,
 postCompletedDeleteTableAction,
 postCompletedDisableTableAction,
 postC

[02/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.WorkItemRegion.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.WorkItemRegion.html 
b/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.WorkItemRegion.html
index 6cdb45b..d7f94ed 100644
--- a/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.WorkItemRegion.html
+++ b/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.WorkItemRegion.html
@@ -117,7 +117,7 @@ var activeTableTab = "activeTableTab";
 
 
 
-static class HBaseFsck.WorkItemRegion
+static class HBaseFsck.WorkItemRegion
 extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object
 implements http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/Callable.html?is-external=true";
 title="class or interface in java.util.concurrent">CallableVoid>
 Contact a region server and get all information from 
it
@@ -226,7 +226,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/util/concurren
 
 
 hbck
-private final HBaseFsck hbck
+private final HBaseFsck hbck
 
 
 
@@ -235,7 +235,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/util/concurren
 
 
 rsinfo
-private final ServerName rsinfo
+private final ServerName rsinfo
 
 
 
@@ -244,7 +244,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/util/concurren
 
 
 errors
-private final HBaseFsck.ErrorReporter 
errors
+private final HBaseFsck.ErrorReporter 
errors
 
 
 
@@ -253,7 +253,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/util/concurren
 
 
 connection
-private final ClusterConnection connection
+private final ClusterConnection connection
 
 
 
@@ -270,7 +270,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/util/concurren
 
 
 WorkItemRegion
-WorkItemRegion(HBaseFsck hbck,
+WorkItemRegion(HBaseFsck hbck,
ServerName info,
HBaseFsck.ErrorReporter errors,
ClusterConnection connection)
@@ -290,7 +290,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/util/concurren
 
 
 call
-public http://docs.oracle.com/javase/8/docs/api/java/lang/Void.html?is-external=true";
 title="class or interface in java.lang">Void call()
+public http://docs.oracle.com/javase/8/docs/api/java/lang/Void.html?is-external=true";
 title="class or interface in java.lang">Void call()
   throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
 
 Specified by:
@@ -306,7 +306,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/util/concurren
 
 
 filterRegions
-private http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List filterRegions(http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List regions)
+private http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List filterRegions(http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List regions)
 
 
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.html 
b/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.html
index 554de30..c535257 100644
--- a/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.html
+++ b/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.html
@@ -545,7 +545,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 startMillis 
 
 
-private ClusterStatus
+private ClusterMetrics
 status 
 
 
@@ -1577,7 +1577,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 
 
 status
-private ClusterStatus status
+private ClusterMetrics status
 
 
 
@@ -2068,7 +2068,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 
 
 cmp
-static final http://docs.oracle.com/javase/8/docs/api/java/util/Comparator.html?is-external=true";
 title="class or interface in java.util">Comparator cmp
+static final http://docs.oracle.com/javase/8/docs/api/java/util/Comparator.html?is-external=true";
 title="class or interface in java.util">Comparator cmp
 
 
 
@@ -3157,7 +3157,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 
 
 checkIntegrity
-http://docs.oracle.com/javase/8/docs/api/java/util/SortedMap.html?is-external=true";
 title

[18/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/master/LoadBalancer.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/master/LoadBalancer.html 
b/devapidocs/org/apache/hadoop/hbase/master/LoadBalancer.html
index e44efee..ab927d5 100644
--- a/devapidocs/org/apache/hadoop/hbase/master/LoadBalancer.html
+++ b/devapidocs/org/apache/hadoop/hbase/master/LoadBalancer.html
@@ -114,7 +114,7 @@ var activeTableTab = "activeTableTab";
 
 
 @InterfaceAudience.Private
-public interface LoadBalancer
+public interface LoadBalancer
 extends org.apache.hadoop.conf.Configurable, Stoppable, ConfigurationObserver
 Makes decisions about the placement and movement of Regions 
across
  RegionServers.
@@ -253,7 +253,7 @@ extends org.apache.hadoop.conf.Configurable, 
 void
-setClusterStatus(ClusterStatus st)
+setClusterMetrics(ClusterMetrics st)
 Set the current cluster status.
 
 
@@ -298,7 +298,7 @@ extends org.apache.hadoop.conf.Configurable, 
 
 TABLES_ON_MASTER
-static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String TABLES_ON_MASTER
+static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String TABLES_ON_MASTER
 Master can carry regions as of hbase-2.0.0.
  By default, it carries no tables.
  TODO: Add any | system as flags to indicate what it can do.
@@ -314,7 +314,7 @@ extends org.apache.hadoop.conf.Configurable, 
 
 SYSTEM_TABLES_ON_MASTER
-static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String SYSTEM_TABLES_ON_MASTER
+static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String SYSTEM_TABLES_ON_MASTER
 Master carries system tables.
 
 See Also:
@@ -328,7 +328,7 @@ extends org.apache.hadoop.conf.Configurable, 
 
 BOGUS_SERVER_NAME
-static final ServerName BOGUS_SERVER_NAME
+static final ServerName BOGUS_SERVER_NAME
 
 
 
@@ -339,13 +339,13 @@ extends org.apache.hadoop.conf.Configurable, 
+
 
 
 
 
-setClusterStatus
-void setClusterStatus(ClusterStatus st)
+setClusterMetrics
+void setClusterMetrics(ClusterMetrics st)
 Set the current cluster status.  This allows a LoadBalancer 
to map host name to a server
 
 Parameters:
@@ -359,7 +359,7 @@ extends org.apache.hadoop.conf.Configurable, 
 
 setClusterLoad
-void setClusterLoad(http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">MapMapList>> ClusterLoad)
+void setClusterLoad(http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">MapMapList>> ClusterLoad)
 Pass RegionStates and allow balancer to set the current 
cluster load.
 
 Parameters:
@@ -373,7 +373,7 @@ extends org.apache.hadoop.conf.Configurable, 
 
 setMasterServices
-void setMasterServices(MasterServices masterServices)
+void setMasterServices(MasterServices masterServices)
 Set the master service.
 
 Parameters:
@@ -387,7 +387,7 @@ extends org.apache.hadoop.conf.Configurable, 
 
 balanceCluster
-http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List balanceCluster(TableName tableName,
+http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List balanceCluster(TableName tableName,
 http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">MapList> clusterState)
  throws HBaseIOException
 Perform the major balance operation
@@ -408,7 +408,7 @@ extends org.apache.hadoop.conf.Configurable, 
 
 balanceCluster
-http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List balanceCluster(http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">Map

[15/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/master/MasterRpcServices.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/master/MasterRpcServices.html 
b/devapidocs/org/apache/hadoop/hbase/master/MasterRpcServices.html
index c4c5327..ce948d2 100644
--- a/devapidocs/org/apache/hadoop/hbase/master/MasterRpcServices.html
+++ b/devapidocs/org/apache/hadoop/hbase/master/MasterRpcServices.html
@@ -119,7 +119,7 @@ var activeTableTab = "activeTableTab";
 
 
 @InterfaceAudience.Private
-public class MasterRpcServices
+public class MasterRpcServices
 extends RSRpcServices
 implements 
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService.BlockingInterface,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStatusService.BlockingInterface,
 
org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockService.BlockingInterface
 Implements the master RPC services.
@@ -779,7 +779,7 @@ implements 
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.Master
 
 
 LOG
-private static final org.slf4j.Logger LOG
+private static final org.slf4j.Logger LOG
 
 
 
@@ -788,7 +788,7 @@ implements 
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.Master
 
 
 master
-private final HMaster master
+private final HMaster master
 
 
 
@@ -805,7 +805,7 @@ implements 
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.Master
 
 
 MasterRpcServices
-public MasterRpcServices(HMaster m)
+public MasterRpcServices(HMaster m)
   throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
 
 Throws:
@@ -827,7 +827,7 @@ implements 
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.Master
 
 
 createConfigurationSubset
-private org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse.Builder createConfigurationSubset()
+private org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse.Builder createConfigurationSubset()
 
 Returns:
 Subset of configuration to pass initializing regionservers: e.g.
@@ -841,7 +841,7 @@ implements 
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.Master
 
 
 addConfig
-private org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse.Builder addConfig(org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse.Builder resp,
+private org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse.Builder addConfig(org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse.Builder resp,

  http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String key)
 
 
@@ -851,7 +851,7 @@ implements 
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.Master
 
 
 createRpcServer
-protected RpcServerInterface createRpcServer(Server server,
+protected RpcServerInterface createRpcServer(Server server,
  
org.apache.hadoop.conf.Configuration conf,
  RpcSchedulerFactory rpcSchedulerFactory,
  http://docs.oracle.com/javase/8/docs/api/java/net/InetSocketAddress.html?is-external=true";
 title="class or interface in java.net">InetSocketAddress bindAddress,
@@ -871,7 +871,7 @@ implements 
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.Master
 
 
 createPriority
-protected PriorityFunction createPriority()
+protected PriorityFunction createPriority()
 
 Overrides:
 createPriority in
 class RSRpcServices
@@ -884,7 +884,7 @@ implements 
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.Master
 
 
 switchBalancer
-boolean switchBalancer(boolean b,
+boolean switchBalancer(boolean b,
MasterRpcServices.BalanceSwitchMode mode)
 throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
 Assigns balancer switch according to BalanceSwitchMode
@@ -905,7 +905,7 @@ implements 
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.Master
 
 
 synchronousBalanceSwitch
-boolean synchronousBalanceSwitch(boolean b)
+boolean synchronousBalanceSwitch(boolean b)
   throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
 
 Throws:
@@ -919,7 +919,7 @

[39/51] [partial] hbase-site git commit: Published site at .

2018-01-04 Thread git-site-role
http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/index-all.html
--
diff --git a/devapidocs/index-all.html b/devapidocs/index-all.html
index fd5453a..362e93b 100644
--- a/devapidocs/index-all.html
+++ b/devapidocs/index-all.html
@@ -5198,9 +5198,9 @@
 BalancerRegionLoad - Class in org.apache.hadoop.hbase.master.balancer
 
 Wrapper class for the few fields required by the StochasticLoadBalancer
- from the full RegionLoad.
+ from the full RegionMetrics.
 
-BalancerRegionLoad(RegionLoad)
 - Constructor for class org.apache.hadoop.hbase.master.balancer.BalancerRegionLoad
+BalancerRegionLoad(RegionMetrics)
 - Constructor for class org.apache.hadoop.hbase.master.balancer.BalancerRegionLoad
  
 balanceRSGroup(String)
 - Method in interface org.apache.hadoop.hbase.rsgroup.RSGroupAdmin
 
@@ -11312,6 +11312,8 @@
 
 Cancel threads if they haven't finished.
 
+cleanup()
 - Method in class org.apache.hadoop.hbase.client.TableSnapshotScanner
+ 
 cleanup(IOException)
 - Method in class org.apache.hadoop.hbase.ipc.BlockingRpcConnection.CallSender
 
 Cleans the call not yet sent when we finish.
@@ -13091,6 +13093,8 @@
  
 ClusterMetricsBuilder.ClusterMetricsImpl - 
Class in org.apache.hadoop.hbase
  
+ClusterMetricsEncoder(InetSocketAddress)
 - Constructor for class org.apache.hadoop.hbase.master.ClusterStatusPublisher.MulticastPublisher.ClusterMetricsEncoder
+ 
 ClusterMetricsImpl(String,
 List, Map, ServerName, 
List, List, String, List, 
Boolean, int) - Constructor for class org.apache.hadoop.hbase.ClusterMetricsBuilder.ClusterMetricsImpl
  
 clusterRequestsCounter
 - Variable in class org.apache.hadoop.hbase.master.MetricsMasterSourceImpl
@@ -13158,8 +13162,6 @@
  
 clusterStatusChore
 - Variable in class org.apache.hadoop.hbase.master.HMaster
  
-ClusterStatusEncoder(InetSocketAddress)
 - Constructor for class org.apache.hadoop.hbase.master.ClusterStatusPublisher.MulticastPublisher.ClusterStatusEncoder
- 
 ClusterStatusHandler()
 - Constructor for class org.apache.hadoop.hbase.client.ClusterStatusListener.MulticastListener.ClusterStatusHandler
  
 ClusterStatusListener - Class in org.apache.hadoop.hbase.client
@@ -13196,7 +13198,7 @@
  
 ClusterStatusPublisher.MulticastPublisher - 
Class in org.apache.hadoop.hbase.master
  
-ClusterStatusPublisher.MulticastPublisher.ClusterStatusEncoder
 - Class in org.apache.hadoop.hbase.master
+ClusterStatusPublisher.MulticastPublisher.ClusterMetricsEncoder
 - Class in org.apache.hadoop.hbase.master
  
 ClusterStatusPublisher.MulticastPublisher.HBaseDatagramChannelFactory - Class in org.apache.hadoop.hbase.master
  
@@ -26430,7 +26432,7 @@
  
 encode(Cell,
 HFileBlockEncodingContext, DataOutputStream) - Method in class 
org.apache.hadoop.hbase.io.hfile.NoOpDataBlockEncoder
  
-encode(ChannelHandlerContext,
 ClusterStatus, List) - Method in class 
org.apache.hadoop.hbase.master.ClusterStatusPublisher.MulticastPublisher.ClusterStatusEncoder
+encode(ChannelHandlerContext,
 ClusterMetrics, List) - Method in class 
org.apache.hadoop.hbase.master.ClusterStatusPublisher.MulticastPublisher.ClusterMetricsEncoder
  
 encode(PositionedByteRange,
 T) - Method in interface org.apache.hadoop.hbase.types.DataType
 
@@ -35204,41 +35206,50 @@
  
 getClusterKey()
 - Method in class org.apache.hadoop.hbase.replication.ReplicationPeerConfig
  
-getClusterSchema()
 - Method in class org.apache.hadoop.hbase.master.HMaster
- 
-getClusterSchema()
 - Method in interface org.apache.hadoop.hbase.master.MasterServices
- 
-getClusterState()
 - Method in class org.apache.hadoop.hbase.master.balancer.ClusterLoadState
- 
-getClusterStatus()
 - Method in interface org.apache.hadoop.hbase.client.Admin
+getClusterMetrics()
 - Method in interface org.apache.hadoop.hbase.client.Admin
 
-Get whole cluster status, containing status about:
+Get whole cluster metrics, containing status about:
 
-getClusterStatus(EnumSet)
 - Method in interface org.apache.hadoop.hbase.client.Admin
+getClusterMetrics(EnumSet)
 - Method in interface org.apache.hadoop.hbase.client.Admin
 
 Get cluster status with a set of ClusterMetrics.Option to get desired 
status.
 
-getClusterStatus()
 - Method in interface org.apache.hadoop.hbase.client.AsyncAdmin
+getClusterMetrics()
 - Method in interface org.apache.hadoop.hbase.client.AsyncAdmin
  
-getClusterStatus(EnumSet)
 - Method in interface org.apache.hadoop.hbase.client.AsyncAdmin
+getClusterMetrics(EnumSet)
 - Method in interface org.apache.hadoop.hbase.client.AsyncAdmin
  
-getClusterStatus()
 - Method in class org.apache.hadoop.hbase.client.AsyncHBaseAdmin
+getClusterMetrics()
 - Method in class org.apache.hadoop.hbase.client.AsyncHBaseAdmin
  
-getClusterStat

[49/51] [partial] hbase-site git commit: Published site at .

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/apidocs/org/apache/hadoop/hbase/class-use/RegionLoad.html
--
diff --git a/apidocs/org/apache/hadoop/hbase/class-use/RegionLoad.html 
b/apidocs/org/apache/hadoop/hbase/class-use/RegionLoad.html
index 434fab7..e797088 100644
--- a/apidocs/org/apache/hadoop/hbase/class-use/RegionLoad.html
+++ b/apidocs/org/apache/hadoop/hbase/class-use/RegionLoad.html
@@ -86,12 +86,6 @@
 org.apache.hadoop.hbase
  
 
-
-org.apache.hadoop.hbase.client
-
-Provides HBase Client
-
-
 
 
 
@@ -120,65 +114,6 @@
 
 
 
-
-
-
-Uses of RegionLoad in org.apache.hadoop.hbase.client
-
-Methods in org.apache.hadoop.hbase.client
 that return types with arguments of type RegionLoad 
-
-Modifier and Type
-Method and Description
-
-
-
-default http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">Map
-Admin.getRegionLoad(ServerName serverName)
-Deprecated. 
-since 2.0 version and will 
be removed in 3.0 version.
- use Admin.getRegionLoads(ServerName)
-
-
-
-
-default http://docs.oracle.com/javase/8/docs/api/java/util/Map.html?is-external=true";
 title="class or interface in java.util">Map
-Admin.getRegionLoad(ServerName serverName,
- TableName tableName)
-Deprecated. 
-since 2.0 version and will 
be removed in 3.0 version.
- use Admin.getRegionLoads(ServerName,
 TableName)
-
-
-
-
-http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
-AsyncAdmin.getRegionLoads(ServerName serverName)
-Get a list of RegionLoad of all regions hosted on a 
region seerver.
-
-
-
-default http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List
-Admin.getRegionLoads(ServerName serverName)
-Get RegionLoad of all regions hosted on a 
regionserver.
-
-
-
-http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
-AsyncAdmin.getRegionLoads(ServerName serverName,
-  TableName tableName)
-Get a list of RegionLoad of all regions hosted on a 
region seerver for a table.
-
-
-
-http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List
-Admin.getRegionLoads(ServerName serverName,
-  TableName tableName)
-Get RegionLoad of all regions hosted on a 
regionserver for a table.
-
-
-
-
-
 
 
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/apidocs/org/apache/hadoop/hbase/class-use/RegionMetrics.html
--
diff --git a/apidocs/org/apache/hadoop/hbase/class-use/RegionMetrics.html 
b/apidocs/org/apache/hadoop/hbase/class-use/RegionMetrics.html
index 256e1d5..a804e4e 100644
--- a/apidocs/org/apache/hadoop/hbase/class-use/RegionMetrics.html
+++ b/apidocs/org/apache/hadoop/hbase/class-use/RegionMetrics.html
@@ -86,6 +86,12 @@
 org.apache.hadoop.hbase
  
 
+
+org.apache.hadoop.hbase.client
+
+Provides HBase Client
+
+
 
 
 
@@ -133,6 +139,46 @@
 
 
 
+
+
+
+Uses of RegionMetrics in org.apache.hadoop.hbase.client
+
+Methods in org.apache.hadoop.hbase.client
 that return types with arguments of type RegionMetrics 
+
+Modifier and Type
+Method and Description
+
+
+
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
+AsyncAdmin.getRegionMetrics(ServerName serverName)
+Get a list of RegionMetrics of 
all regions hosted on a region seerver.
+
+
+
+default http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List
+Admin.getRegionMetrics(ServerName serverName)
+Get RegionMetrics of 
all regions hosted on a regionserver.
+
+
+
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
+AsyncAdmin.getRegionMetrics(ServerName serverName,
+TableName tableN

[33/51] [partial] hbase-site git commit: Published site at .

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/client/AsyncAdmin.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/client/AsyncAdmin.html 
b/devapidocs/org/apache/hadoop/hbase/client/AsyncAdmin.html
index 58f7b8c..616ee36 100644
--- a/devapidocs/org/apache/hadoop/hbase/client/AsyncAdmin.html
+++ b/devapidocs/org/apache/hadoop/hbase/client/AsyncAdmin.html
@@ -106,7 +106,7 @@ var activeTableTab = "activeTableTab";
 
 
 @InterfaceAudience.Public
-public interface AsyncAdmin
+public interface AsyncAdmin
 The asynchronous administrative API for HBase.
 
 Since:
@@ -438,12 +438,12 @@ public interface getBackupMasters() 
 
 
-http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFuture
-getClusterStatus() 
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFuture
+getClusterMetrics() 
 
 
-http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFuture
-getClusterStatus(http://docs.oracle.com/javase/8/docs/api/java/util/EnumSet.html?is-external=true";
 title="class or interface in java.util">EnumSet options) 
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFuture
+getClusterMetrics(http://docs.oracle.com/javase/8/docs/api/java/util/EnumSet.html?is-external=true";
 title="class or interface in java.util">EnumSet options) 
 
 
 default http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFuture
@@ -494,7 +494,7 @@ public interface 
 default http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureListString>>
-getMasterCoprocessors() 
+getMasterCoprocessorNames() 
 
 
 default http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureInteger>
@@ -521,16 +521,16 @@ public interface 
-http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
-getRegionLoads(ServerName serverName)
-Get a list of RegionLoad of all regions hosted on a 
region seerver.
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
+getRegionMetrics(ServerName serverName)
+Get a list of RegionMetrics of 
all regions hosted on a region seerver.
 
 
 
-http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
-getRegionLoads(ServerName serverName,
-  TableName tableName)
-Get a list of RegionLoad of all regions hosted on a 
region seerver for a table.
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFutureList>
+getRegionMetrics(ServerName serverName,
+TableName tableName)
+Get a list of RegionMetrics of 
all regions hosted on a region seerver for a table.
 
 
 
@@ -1045,7 +1045,7 @@ public interface 
 
 tableExists
-http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=t

[43/51] [partial] hbase-site git commit: Published site at .

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/apidocs/src-html/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.html
--
diff --git 
a/apidocs/src-html/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.html 
b/apidocs/src-html/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.html
index 5fc1559..c57c5df 100644
--- a/apidocs/src-html/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.html
+++ b/apidocs/src-html/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.html
@@ -353,714 +353,712 @@
 345  }
 346
 347  /**
-348   * Sets up the job for reading from a 
table snapshot. It bypasses hbase servers
-349   * and read directly from snapshot 
files.
-350   *
-351   * @param snapshotName The name of the 
snapshot (of a table) to read from.
-352   * @param scan  The scan instance with 
the columns, time range etc.
-353   * @param mapper  The mapper class to 
use.
-354   * @param outputKeyClass  The class of 
the output key.
-355   * @param outputValueClass  The class 
of the output value.
-356   * @param job  The current job to 
adjust.  Make sure the passed job is
-357   * carrying all necessary HBase 
configuration.
-358   * @param addDependencyJars upload 
HBase jars and jars for any of the configured
-359   *   job classes via the 
distributed cache (tmpjars).
-360   *
-361   * @param tmpRestoreDir a temporary 
directory to copy the snapshot files into. Current user should
-362   * have write permissions to this 
directory, and this should not be a subdirectory of rootdir.
-363   * After the job is finished, restore 
directory can be deleted.
-364   * @throws IOException When setting up 
the details fails.
-365   * @see TableSnapshotInputFormat
-366   */
-367  public static void 
initTableSnapshotMapperJob(String snapshotName, Scan scan,
-368  Class 
mapper,
-369  Class outputKeyClass,
-370  Class outputValueClass, 
Job job,
-371  boolean addDependencyJars, Path 
tmpRestoreDir)
-372  throws IOException {
-373
TableSnapshotInputFormat.setInput(job, snapshotName, tmpRestoreDir);
-374initTableMapperJob(snapshotName, 
scan, mapper, outputKeyClass,
-375outputValueClass, job, 
addDependencyJars, false, TableSnapshotInputFormat.class);
-376
resetCacheConfig(job.getConfiguration());
-377  }
-378
-379  /**
-380   * Sets up the job for reading from a 
table snapshot. It bypasses hbase servers
-381   * and read directly from snapshot 
files.
-382   *
-383   * @param snapshotName The name of the 
snapshot (of a table) to read from.
-384   * @param scan  The scan instance with 
the columns, time range etc.
-385   * @param mapper  The mapper class to 
use.
-386   * @param outputKeyClass  The class of 
the output key.
-387   * @param outputValueClass  The class 
of the output value.
-388   * @param job  The current job to 
adjust.  Make sure the passed job is
-389   * carrying all necessary HBase 
configuration.
-390   * @param addDependencyJars upload 
HBase jars and jars for any of the configured
-391   *   job classes via the 
distributed cache (tmpjars).
-392   *
-393   * @param tmpRestoreDir a temporary 
directory to copy the snapshot files into. Current user should
-394   * have write permissions to this 
directory, and this should not be a subdirectory of rootdir.
-395   * After the job is finished, restore 
directory can be deleted.
-396   * @param splitAlgo algorithm to 
split
-397   * @param numSplitsPerRegion how many 
input splits to generate per one region
-398   * @throws IOException When setting up 
the details fails.
-399   * @see TableSnapshotInputFormat
-400   */
-401  public static void 
initTableSnapshotMapperJob(String snapshotName, Scan scan,
-402  
  Class mapper,
-403  
  Class outputKeyClass,
-404  
  Class outputValueClass, Job job,
-405  
  boolean addDependencyJars, Path tmpRestoreDir,
-406  
  RegionSplitter.SplitAlgorithm splitAlgo,
-407  
  int numSplitsPerRegion)
-408  throws IOException {
-409
TableSnapshotInputFormat.setInput(job, snapshotName, tmpRestoreDir, 
splitAlgo,
-410numSplitsPerRegion);
-411initTableMapperJob(snapshotName, 
scan, mapper, outputKeyClass,
-412outputValueClass, job, 
addDependencyJars, false, TableSnapshotInputFormat.class);
-413
resetCacheConfig(job.getConfiguration());
-414  }
-415
-416  /**
-417   * Use this before submitting a Multi 
TableMap job. It will appropriately set
-418   * up the job.
-419   *
-420   * @param scans The list of {@link 
Scan} objects to read from.
-421   * @param mapper The mapper class to 
use.
-422   * @param outputKeyClass The cla

[40/51] [partial] hbase-site git commit: Published site at .

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/checkstyle.rss
--
diff --git a/checkstyle.rss b/checkstyle.rss
index ba429a2..272dead 100644
--- a/checkstyle.rss
+++ b/checkstyle.rss
@@ -26,7 +26,7 @@ under the License.
 ©2007 - 2018 The Apache Software Foundation
 
   File: 3471,
- Errors: 19048,
+ Errors: 18828,
  Warnings: 0,
  Infos: 0
   
@@ -909,7 +909,7 @@ under the License.
   0
 
 
-  5
+  0
 
   
   
@@ -3723,7 +3723,7 @@ under the License.
   0
 
 
-  19
+  18
 
   
   
@@ -4577,7 +4577,7 @@ under the License.
   0
 
 
-  7
+  6
 
   
   
@@ -7783,7 +7783,7 @@ under the License.
   0
 
 
-  4
+  3
 
   
   
@@ -8455,7 +8455,7 @@ under the License.
   0
 
 
-  4
+  2
 
   
   
@@ -8791,7 +8791,7 @@ under the License.
   0
 
 
-  28
+  26
 
   
   
@@ -11689,7 +11689,7 @@ under the License.
   0
 
 
-  6
+  4
 
   
   
@@ -12753,7 +12753,7 @@ under the License.
   0
 
 
-  5
+  7
 
   
   
@@ -15819,7 +15819,7 @@ under the License.
   0
 
 
-  10
+  9
 
   
   
@@ -17387,7 +17387,7 @@ under the License.
   0
 
 
-  2
+  0
 
   
   
@@ -18115,7 +18115,7 @@ under the License.
   0
 
 
-  5
+  3
 
   
   
@@ -20075,7 +20075,7 @@ under the License.
   0
 
 
-  14
+  12
 
   
   
@@ -23197,7 +23197,7 @@ under the License.
   0
 
 
-  4
+  1
 
   
   
@@ -23575,7 +23575,7 @@ under the License.
   0
 
 
-  17
+  16
 
   
   
@@ -25717,7 +25717,7 @@ under the License.
   0
 
 
-  156
+  155
 
   
   
@@ -26515,7 +26515,7 @@ under the License.
   0
 
 
-  2
+  1
 
   
   
@@ -28419,7 +28419,7 @@ under the License.
   0
 
 
-  5
+  3
 
   
   
@@ -30323,7 +30323,7 @@ under the License.
   0
 
 
-  101
+  89
 
   
   
@@ -35797,7 +35797,7 @@ under the License.
   0
 
 
-  6
+  4
 
   
   
@@ -37239,7 +37239,7 @@ under the License.
   0
 
 
-  195
+  26
 
   
   
@@ -37379,7 +37379,7 @@ under the License.
   0
 
 
-  58
+  55
 
   
   
@@ -38373,7 +38373,7 @@ under the License.
   0
 
 
-  33
+  32
 
   
   
@@ -3848

[34/51] [partial] hbase-site git commit: Published site at .

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/client/Admin.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/client/Admin.html 
b/devapidocs/org/apache/hadoop/hbase/client/Admin.html
index 4c9b7e4..1db8dc5 100644
--- a/devapidocs/org/apache/hadoop/hbase/client/Admin.html
+++ b/devapidocs/org/apache/hadoop/hbase/client/Admin.html
@@ -18,7 +18,7 @@
 catch(err) {
 }
 //-->
-var methods = 
{"i0":6,"i1":6,"i2":6,"i3":50,"i4":6,"i5":6,"i6":18,"i7":6,"i8":6,"i9":6,"i10":6,"i11":6,"i12":50,"i13":50,"i14":6,"i15":6,"i16":6,"i17":6,"i18":6,"i19":6,"i20":6,"i21":6,"i22":6,"i23":6,"i24":6,"i25":38,"i26":38,"i27":38,"i28":38,"i29":6,"i30":6,"i31":6,"i32":6,"i33":6,"i34":6,"i35":6,"i36":50,"i37":6,"i38":6,"i39":6,"i40":6,"i41":6,"i42":6,"i43":6,"i44":6,"i45":6,"i46":38,"i47":6,"i48":6,"i49":6,"i50":6,"i51":6,"i52":6,"i53":6,"i54":38,"i55":6,"i56":6,"i57":38,"i58":38,"i59":6,"i60":38,"i61":6,"i62":6,"i63":6,"i64":6,"i65":38,"i66":38,"i67":50,"i68":6,"i69":6,"i70":6,"i71":6,"i72":38,"i73":38,"i74":6,"i75":50,"i76":6,"i77":6,"i78":6,"i79":38,"i80":38,"i81":18,"i82":6,"i83":6,"i84":6,"i85":6,"i86":6,"i87":6,"i88":6,"i89":6,"i90":6,"i91":6,"i92":6,"i93":18,"i94":6,"i95":18,"i96":6,"i97":38,"i98":6,"i99":6,"i100":6,"i101":38,"i102":50,"i103":50,"i104":18,"i105":6,"i106":6,"i107":6,"i108":18,"i109":6,"i110":6,"i111":38,"i112":38,"i113":38,"i114":38,"i115":6,"i116":6,"i11
 
7":6,"i118":6,"i119":6,"i120":6,"i121":6,"i122":6,"i123":6,"i124":6,"i125":50,"i126":6,"i127":38,"i128":6,"i129":6,"i130":18,"i131":6,"i132":6,"i133":6,"i134":6,"i135":6,"i136":6,"i137":6,"i138":38,"i139":6,"i140":6,"i141":6,"i142":6,"i143":6,"i144":38,"i145":6,"i146":6,"i147":6,"i148":38,"i149":38,"i150":6,"i151":38,"i152":38,"i153":38,"i154":38,"i155":38,"i156":6,"i157":38,"i158":6,"i159":6,"i160":6,"i161":6,"i162":6,"i163":6,"i164":6,"i165":38,"i166":6,"i167":6,"i168":6,"i169":50,"i170":6,"i171":6,"i172":6,"i173":6,"i174":6,"i175":38,"i176":6,"i177":38,"i178":6,"i179":6,"i180":6,"i181":6,"i182":6,"i183":6,"i184":6,"i185":6,"i186":6,"i187":6,"i188":6,"i189":6,"i190":6,"i191":6,"i192":6,"i193":50,"i194":6,"i195":50,"i196":50,"i197":50,"i198":6,"i199":50,"i200":6,"i201":6,"i202":6,"i203":6,"i204":6,"i205":6,"i206":6,"i207":6,"i208":38,"i209":38,"i210":6,"i211":6,"i212":6,"i213":6,"i214":6,"i215":50,"i216":6,"i217":6,"i218":6,"i219":6,"i220":6,"i221":6};
+var methods = 
{"i0":6,"i1":6,"i2":6,"i3":50,"i4":6,"i5":6,"i6":18,"i7":6,"i8":6,"i9":6,"i10":6,"i11":6,"i12":50,"i13":50,"i14":6,"i15":6,"i16":6,"i17":6,"i18":6,"i19":6,"i20":6,"i21":6,"i22":6,"i23":6,"i24":6,"i25":38,"i26":38,"i27":38,"i28":38,"i29":6,"i30":6,"i31":6,"i32":6,"i33":6,"i34":6,"i35":6,"i36":50,"i37":6,"i38":6,"i39":6,"i40":6,"i41":6,"i42":6,"i43":6,"i44":6,"i45":6,"i46":38,"i47":6,"i48":6,"i49":6,"i50":6,"i51":6,"i52":6,"i53":6,"i54":38,"i55":6,"i56":6,"i57":38,"i58":38,"i59":6,"i60":38,"i61":6,"i62":6,"i63":6,"i64":6,"i65":38,"i66":38,"i67":50,"i68":6,"i69":6,"i70":6,"i71":6,"i72":38,"i73":38,"i74":6,"i75":50,"i76":6,"i77":6,"i78":6,"i79":38,"i80":38,"i81":18,"i82":18,"i83":6,"i84":50,"i85":6,"i86":6,"i87":6,"i88":6,"i89":6,"i90":6,"i91":6,"i92":6,"i93":6,"i94":18,"i95":18,"i96":50,"i97":18,"i98":6,"i99":38,"i100":6,"i101":6,"i102":6,"i103":38,"i104":18,"i105":6,"i106":6,"i107":6,"i108":18,"i109":6,"i110":6,"i111":38,"i112":38,"i113":38,"i114":38,"i115":6,"i116":6,"i
 
117":6,"i118":6,"i119":6,"i120":6,"i121":6,"i122":6,"i123":6,"i124":6,"i125":50,"i126":6,"i127":38,"i128":6,"i129":6,"i130":18,"i131":6,"i132":6,"i133":6,"i134":6,"i135":6,"i136":6,"i137":6,"i138":38,"i139":6,"i140":6,"i141":6,"i142":6,"i143":6,"i144":38,"i145":6,"i146":6,"i147":6,"i148":38,"i149":38,"i150":6,"i151":38,"i152":38,"i153":38,"i154":38,"i155":38,"i156":6,"i157":38,"i158":6,"i159":6,"i160":6,"i161":6,"i162":6,"i163":6,"i164":6,"i165":38,"i166":6,"i167":6,"i168":6,"i169":50,"i170":6,"i171":6,"i172":6,"i173":6,"i174":6,"i175":38,"i176":6,"i177":38,"i178":6,"i179":6,"i180":6,"i181":6,"i182":6,"i183":6,"i184":6,"i185":6,"i186":6,"i187":6,"i188":6,"i189":6,"i190":6,"i191":6,"i192":6,"i193":50,"i194":6,"i195":50,"i196":50,"i197":50,"i198":6,"i199":50,"i200":6,"i201":6,"i202":6,"i203":6,"i204":6,"i205":6,"i206":6,"i207":6,"i208":38,"i209":38,"i210":6,"i211":6,"i212":6,"i213":6,"i214":6,"i215":50,"i216":6,"i217":6,"i218":6,"i219":6,"i220":6,"i221":6};
 var tabs = {65535:["t0","All Methods"],2:["t2","Instance 
Methods"],4:["t3","Abstract Methods"],16:["t5","Default 
Methods"],32:["t6","Deprecated Methods"]};
 var altColor = "altColor";
 var rowColor = "rowColor";
@@ -110,7 +110,7 @@ var activeTableTab = "activeTableTab";
 
 
 @InterfaceAudience.Public
-public interface Admin
+public interface Admin
 extends Abortable, http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.html?is-external=true";
 title="class or interface in java.io">Closeable
 The administrative API for HBase. Obtain an

[23/51] [partial] hbase-site git commit: Published site at .

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/coprocessor/class-use/ObserverContext.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/coprocessor/class-use/ObserverContext.html 
b/devapidocs/org/apache/hadoop/hbase/coprocessor/class-use/ObserverContext.html
index 429ebaa..b4404ce 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/coprocessor/class-use/ObserverContext.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/coprocessor/class-use/ObserverContext.html
@@ -612,8 +612,8 @@
 
 
 default void
-MasterObserver.postGetClusterStatus(ObserverContext ctx,
-ClusterStatus status)
+MasterObserver.postGetClusterMetrics(ObserverContext ctx,
+ ClusterMetrics status)
 Called after get cluster status.
 
 
@@ -1452,7 +1452,7 @@
 
 
 default void
-MasterObserver.preGetClusterStatus(ObserverContext ctx)
+MasterObserver.preGetClusterMetrics(ObserverContext ctx)
 Called before get cluster status.
 
 
@@ -2678,166 +2678,162 @@
 
 
 void
-AccessController.preGetClusterStatus(ObserverContext ctx) 
-
-
-void
 AccessController.preGetLocks(ObserverContext ctx) 
 
-
+
 void
 AccessController.preGetNamespaceDescriptor(ObserverContext ctx,
  http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in 
java.lang">String namespace) 
 
-
+
 void
 AccessController.preGetOp(ObserverContext c,
 Get get,
 http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List result) 
 
-
+
 void
 AccessController.preGetProcedures(ObserverContext ctx) 
 
-
+
 void
 AccessController.preGetReplicationPeerConfig(ObserverContext ctx,
http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in 
java.lang">String peerId) 
 
-
+
 void
 AccessController.preGetTableDescriptors(ObserverContext ctx,
   http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List tableNamesList,
   http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List descriptors,
   http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in 
java.lang">String regex) 
 
-
+
 Result
 AccessController.preIncrement(ObserverContext c,
 Increment increment) 
 
-
+
 Result
 AccessController.preIncrementAfterRowLock(ObserverContext c,
 Increment increment) 
 
-
+
 void
 AccessController.preListDecommissionedRegionServers(ObserverContext ctx) 
 
-
+
 void
 AccessController.preListReplicationPeers(ObserverContext ctx,
http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in 
java.lang">String regex) 
 
-
+
 void
 AccessController.preListSnapshot(ObserverContext ctx,
SnapshotDescription snapshot) 
 
-
+
 void
 AccessController.preLockHeartbeat(ObserverContext ctx,
 TableName tableName,
 http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in 
java.lang">String description) 
 
-
+
 void
 AccessController.preMergeRegions(ObserverContext ctx,
RegionInfo[] regionsToMerge) 
 
-
+
 void
 AccessController.preModifyNamespace(ObserverContext ctx,
   NamespaceDescriptor ns) 
 
-
+
 void
 AccessController.preModifyTable(ObserverContext c,
   TableName tableName,
   TableDescriptor htd) 
 
-
+
 void
 CoprocessorWhitelistMasterObserver.preModifyTable(ObserverContext ctx,
   TableName tableName,
   TableDescriptor htd) 
 
-
+
 void
 AccessController.preMove(ObserverContext c,
RegionInfo region,
ServerName srcServer,
ServerName destServer) 
 
-
+
 void
 AccessController.preMoveServers(ObserverContext ctx,
   http://docs.oracle.com/javase/8/do

[19/51] [partial] hbase-site git commit: Published site at .

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/master/ClusterStatusPublisher.MulticastPublisher.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/master/ClusterStatusPublisher.MulticastPublisher.html
 
b/devapidocs/org/apache/hadoop/hbase/master/ClusterStatusPublisher.MulticastPublisher.html
index f2817b6..2610912 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/master/ClusterStatusPublisher.MulticastPublisher.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/master/ClusterStatusPublisher.MulticastPublisher.html
@@ -50,7 +50,7 @@ var activeTableTab = "activeTableTab";
 
 
 Prev Class
-Next Class
+Next Class
 
 
 Frames
@@ -118,7 +118,7 @@ var activeTableTab = "activeTableTab";
 
 
 @InterfaceAudience.LimitedPrivate(value="Configuration")
-public static class ClusterStatusPublisher.MulticastPublisher
+public static class ClusterStatusPublisher.MulticastPublisher
 extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object
 implements ClusterStatusPublisher.Publisher
 
@@ -141,7 +141,7 @@ implements 
 private static class 
-ClusterStatusPublisher.MulticastPublisher.ClusterStatusEncoder 
+ClusterStatusPublisher.MulticastPublisher.ClusterMetricsEncoder 
 
 
 private static class 
@@ -212,7 +212,7 @@ implements 
 void
-publish(ClusterStatus cs) 
+publish(ClusterMetrics cs) 
 
 
 
@@ -242,7 +242,7 @@ implements 
 
 channel
-private org.apache.hbase.thirdparty.io.netty.channel.socket.DatagramChannel
 channel
+private org.apache.hbase.thirdparty.io.netty.channel.socket.DatagramChannel
 channel
 
 
 
@@ -251,7 +251,7 @@ implements 
 
 group
-private 
final org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup group
+private 
final org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup group
 
 
 
@@ -268,7 +268,7 @@ implements 
 
 MulticastPublisher
-public MulticastPublisher()
+public MulticastPublisher()
 
 
 
@@ -285,7 +285,7 @@ implements 
 
 connect
-public void connect(org.apache.hadoop.conf.Configuration conf)
+public void connect(org.apache.hadoop.conf.Configuration conf)
  throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
 
 Specified by:
@@ -295,16 +295,16 @@ implements 
+
 
 
 
 
 publish
-public void publish(ClusterStatus cs)
+public void publish(ClusterMetrics cs)
 
 Specified by:
-publish in
 interface ClusterStatusPublisher.Publisher
+publish in
 interface ClusterStatusPublisher.Publisher
 
 
 
@@ -314,7 +314,7 @@ implements 
 
 close
-public void close()
+public void close()
 
 Specified by:
 http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.html?is-external=true#close--";
 title="class or interface in java.io">close in 
interface http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.html?is-external=true";
 title="class or interface in java.io">Closeable
@@ -354,7 +354,7 @@ implements 
 
 Prev Class
-Next Class
+Next Class
 
 
 Frames

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/master/ClusterStatusPublisher.Publisher.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/master/ClusterStatusPublisher.Publisher.html
 
b/devapidocs/org/apache/hadoop/hbase/master/ClusterStatusPublisher.Publisher.html
index e9838f8..d4a2c86 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/master/ClusterStatusPublisher.Publisher.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/master/ClusterStatusPublisher.Publisher.html
@@ -113,7 +113,7 @@ var activeTableTab = "activeTableTab";
 
 
 
-public static interface ClusterStatusPublisher.Publisher
+public static interface ClusterStatusPublisher.Publisher
 extends http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.html?is-external=true";
 title="class or interface in java.io">Closeable
 
 
@@ -143,7 +143,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.html
 
 
 void
-publish(ClusterStatus cs) 
+publish(ClusterMetrics cs) 
 
 
 
@@ -166,7 +166,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.html
 
 
 connect
-void connect(org.apache.hadoop.conf.Configuration conf)
+void connect(org.apache.hadoop.conf.Configuration conf)
   throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
 
 Throws:
@@ -174,13 +174,13 @@ extends http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.html
 
 
 
-
+
 
 
 
 
 publish
-void publish(ClusterStatus cs)
+void publish(ClusterMetrics cs)
 
 
 
@@ -189,7 +189,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.html
 
 
 close
-void close()
+void close()
 
 Specified by:
 http://docs.oracle.com/javase/8/docs/api/java/lang/AutoCloseable

[07/51] [partial] hbase-site git commit: Published site at .

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/security/access/AccessController.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/security/access/AccessController.html 
b/devapidocs/org/apache/hadoop/hbase/security/access/AccessController.html
index 9e7d3ae..c7b483d 100644
--- a/devapidocs/org/apache/hadoop/hbase/security/access/AccessController.html
+++ b/devapidocs/org/apache/hadoop/hbase/security/access/AccessController.html
@@ -18,7 +18,7 @@
 catch(err) {
 }
 //-->
-var methods = 
{"i0":9,"i1":10,"i2":10,"i3":10,"i4":10,"i5":10,"i6":9,"i7":10,"i8":10,"i9":10,"i10":10,"i11":10,"i12":10,"i13":10,"i14":10,"i15":10,"i16":10,"i17":10,"i18":10,"i19":10,"i20":10,"i21":10,"i22":10,"i23":10,"i24":9,"i25":9,"i26":10,"i27":10,"i28":10,"i29":10,"i30":10,"i31":10,"i32":10,"i33":10,"i34":10,"i35":10,"i36":10,"i37":10,"i38":10,"i39":10,"i40":10,"i41":10,"i42":10,"i43":10,"i44":10,"i45":10,"i46":10,"i47":10,"i48":10,"i49":10,"i50":10,"i51":10,"i52":10,"i53":10,"i54":10,"i55":10,"i56":10,"i57":10,"i58":10,"i59":10,"i60":10,"i61":10,"i62":10,"i63":10,"i64":10,"i65":10,"i66":10,"i67":10,"i68":10,"i69":10,"i70":10,"i71":10,"i72":10,"i73":10,"i74":10,"i75":10,"i76":10,"i77":10,"i78":10,"i79":10,"i80":10,"i81":10,"i82":10,"i83":10,"i84":10,"i85":10,"i86":10,"i87":10,"i88":10,"i89":10,"i90":10,"i91":10,"i92":10,"i93":10,"i94":10,"i95":10,"i96":10,"i97":10,"i98":10,"i99":10,"i100":10,"i101":10,"i102":10,"i103":10,"i104":10,"i105":10,"i106":10,"i107":10,"i108":10,"i109"
 
:10,"i110":10,"i111":10,"i112":10,"i113":10,"i114":10,"i115":10,"i116":10,"i117":10,"i118":10,"i119":10,"i120":10,"i121":10,"i122":10,"i123":10,"i124":10,"i125":10,"i126":10,"i127":10,"i128":10,"i129":10,"i130":10,"i131":10,"i132":10,"i133":10,"i134":10,"i135":10,"i136":10,"i137":10,"i138":10,"i139":10,"i140":10,"i141":10,"i142":10,"i143":10,"i144":10,"i145":10,"i146":10,"i147":10,"i148":10};
+var methods = 
{"i0":9,"i1":10,"i2":10,"i3":10,"i4":10,"i5":10,"i6":9,"i7":10,"i8":10,"i9":10,"i10":10,"i11":10,"i12":10,"i13":10,"i14":10,"i15":10,"i16":10,"i17":10,"i18":10,"i19":10,"i20":10,"i21":10,"i22":10,"i23":10,"i24":9,"i25":9,"i26":10,"i27":10,"i28":10,"i29":10,"i30":10,"i31":10,"i32":10,"i33":10,"i34":10,"i35":10,"i36":10,"i37":10,"i38":10,"i39":10,"i40":10,"i41":10,"i42":10,"i43":10,"i44":10,"i45":10,"i46":10,"i47":10,"i48":10,"i49":10,"i50":10,"i51":10,"i52":10,"i53":10,"i54":10,"i55":10,"i56":10,"i57":10,"i58":10,"i59":10,"i60":10,"i61":10,"i62":10,"i63":10,"i64":10,"i65":10,"i66":10,"i67":10,"i68":10,"i69":10,"i70":10,"i71":10,"i72":10,"i73":10,"i74":10,"i75":10,"i76":10,"i77":10,"i78":10,"i79":10,"i80":10,"i81":10,"i82":10,"i83":10,"i84":10,"i85":10,"i86":10,"i87":10,"i88":10,"i89":10,"i90":10,"i91":10,"i92":10,"i93":10,"i94":10,"i95":10,"i96":10,"i97":10,"i98":10,"i99":10,"i100":10,"i101":10,"i102":10,"i103":10,"i104":10,"i105":10,"i106":10,"i107":10,"i108":10,"i109"
 
:10,"i110":10,"i111":10,"i112":10,"i113":10,"i114":10,"i115":10,"i116":10,"i117":10,"i118":10,"i119":10,"i120":10,"i121":10,"i122":10,"i123":10,"i124":10,"i125":10,"i126":10,"i127":10,"i128":10,"i129":10,"i130":10,"i131":10,"i132":10,"i133":10,"i134":10,"i135":10,"i136":10,"i137":10,"i138":10,"i139":10,"i140":10,"i141":10,"i142":10,"i143":10,"i144":10,"i145":10,"i146":10,"i147":10};
 var tabs = {65535:["t0","All Methods"],1:["t1","Static 
Methods"],2:["t2","Instance Methods"],8:["t4","Concrete Methods"]};
 var altColor = "altColor";
 var rowColor = "rowColor";
@@ -114,7 +114,7 @@ var activeTableTab = "activeTableTab";
 
 
 @InterfaceAudience.LimitedPrivate(value="Configuration")
-public class AccessController
+public class AccessController
 extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object
 implements MasterCoprocessor, RegionCoprocessor, RegionServerCoprocessor, 
org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService.Interface,
 MasterObserver, 
RegionObserver, 
RegionServerObserver, EndpointObserver, 
BulkLoadObserver
 Provides basic authorization checks for data access and 
administrative
@@ -959,24 +959,18 @@ implements 
 void
-preGetClusterStatus(ObserverContext ctx)
-Called before get cluster status.
-
-
-
-void
 preGetLocks(ObserverContext ctx)
 Called before a getLocks request has been processed.
 
 
-
+
 void
 preGetNamespaceDescriptor(ObserverContext ctx,
  http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String namespace)
 Called before a getNamespaceDescriptor request has been 
processed.
 
 
-
+
 void
 preGetOp(ObserverContext c,
 Get get,
@@ -984,20 +978,20 @@ implements Called before the client performs a

[13/51] [partial] hbase-site git commit: Published site at .

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.CandidateGenerator.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.CandidateGenerator.html
 
b/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.CandidateGenerator.html
index 3e22e30..ac39a7c 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.CandidateGenerator.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.CandidateGenerator.html
@@ -117,7 +117,7 @@ var activeTableTab = "activeTableTab";
 
 
 
-abstract static class StochasticLoadBalancer.CandidateGenerator
+abstract static class StochasticLoadBalancer.CandidateGenerator
 extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object
 Generates a candidate action to be applied to the cluster 
for cost function search
 
@@ -232,7 +232,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 CandidateGenerator
-CandidateGenerator()
+CandidateGenerator()
 
 
 
@@ -249,7 +249,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 generate
-abstract BaseLoadBalancer.Cluster.Action generate(BaseLoadBalancer.Cluster cluster)
+abstract BaseLoadBalancer.Cluster.Action generate(BaseLoadBalancer.Cluster cluster)
 
 
 
@@ -258,7 +258,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 pickRandomRegion
-protected int pickRandomRegion(BaseLoadBalancer.Cluster cluster,
+protected int pickRandomRegion(BaseLoadBalancer.Cluster cluster,
int server,
double chanceOfNoSwap)
 From a list of regions pick a random one. Null can be 
returned which
@@ -282,7 +282,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 pickRandomServer
-protected int pickRandomServer(BaseLoadBalancer.Cluster cluster)
+protected int pickRandomServer(BaseLoadBalancer.Cluster cluster)
 
 
 
@@ -291,7 +291,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 pickRandomRack
-protected int pickRandomRack(BaseLoadBalancer.Cluster cluster)
+protected int pickRandomRack(BaseLoadBalancer.Cluster cluster)
 
 
 
@@ -300,7 +300,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 pickOtherRandomServer
-protected int pickOtherRandomServer(BaseLoadBalancer.Cluster cluster,
+protected int pickOtherRandomServer(BaseLoadBalancer.Cluster cluster,
 int serverIndex)
 
 
@@ -310,7 +310,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 pickOtherRandomRack
-protected int pickOtherRandomRack(BaseLoadBalancer.Cluster cluster,
+protected int pickOtherRandomRack(BaseLoadBalancer.Cluster cluster,
   int rackIndex)
 
 
@@ -320,7 +320,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 pickRandomRegions
-protected BaseLoadBalancer.Cluster.Action pickRandomRegions(BaseLoadBalancer.Cluster cluster,
+protected BaseLoadBalancer.Cluster.Action pickRandomRegions(BaseLoadBalancer.Cluster cluster,
 
int thisServer,
 
int otherServer)
 
@@ -331,7 +331,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 getAction
-protected BaseLoadBalancer.Cluster.Action getAction(int fromServer,
+protected BaseLoadBalancer.Cluster.Action getAction(int fromServer,
 int fromRegion,
 int toServer,
 int toRegion)
@@ -343,7 +343,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 getRandomIterationOrder
-protected http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">ListInteger> getRandomIterationOrder(int length)
+protected http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">ListInteger> getRandomIterationOrder(int length)
 Returns a random iteration order of indexes of an array 
with size length
 
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.CostFromRegionLoadAsRateFunction.html

[10/51] [partial] hbase-site git commit: Published site at .

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/package-use.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/package-use.html 
b/devapidocs/org/apache/hadoop/hbase/package-use.html
index 5a3a58b..d9cf5eb 100644
--- a/devapidocs/org/apache/hadoop/hbase/package-use.html
+++ b/devapidocs/org/apache/hadoop/hbase/package-use.html
@@ -1063,11 +1063,16 @@ service.
 
 
 
+ClusterMetrics
+Metrics information on the HBase cluster.
+
+
+
 ClusterMetrics.Option
 Kinds of ClusterMetrics
 
 
-
+
 ClusterStatus
 Deprecated. 
 As of release 2.0.0, this 
will be removed in HBase 3.0.0
@@ -1075,32 +1080,32 @@ service.
 
 
 
-
+
 CompareOperator
 Generic set of comparison operators.
 
 
-
+
 DoNotRetryIOException
 Subclass if exception is not meant to be retried: e.g.
 
 
-
+
 ExtendedCell
 Extension to Cell with server side required 
functions.
 
 
-
+
 HBaseIOException
 All hbase specific IOExceptions should be subclasses of 
HBaseIOException
 
 
-
+
 HColumnDescriptor
 Deprecated. 
 
 
-
+
 HRegionInfo
 Deprecated. 
 As of release 2.0.0, this 
will be removed in HBase 3.0.0.
@@ -1108,13 +1113,13 @@ service.
 
 
 
-
+
 HRegionLocation
 Data structure to hold RegionInfo and the address for the 
hosting
  HRegionServer.
 
 
-
+
 HTableDescriptor
 Deprecated. 
 As of release 2.0.0, this 
will be removed in HBase 3.0.0.
@@ -1122,54 +1127,46 @@ service.
 
 
 
-
+
 KeepDeletedCells
 Ways to keep cells marked for delete around.
 
 
-
+
 KeyValue
 An HBase Key/Value.
 
 
-
+
 MasterNotRunningException
 Thrown if the master is not running
 
 
-
+
 MemoryCompactionPolicy
 Enum describing all possible memory compaction 
policies
 
 
-
+
 NamespaceDescriptor
 Namespace POJO class.
 
 
-
+
 NamespaceNotFoundException
 Thrown when a namespace can not be located
 
 
-
+
 RawCell
 An extended version of cell that gives more power to 
CPs
 
 
-
+
 RegionException
 Thrown when something happens related to region 
handling.
 
 
-
-RegionLoad
-Deprecated. 
-As of release 2.0.0, this 
will be removed in HBase 3.0.0
- Use RegionMetrics instead.
-
-
-
 
 RegionLocations
 Container for holding a list of HRegionLocation's that correspond to 
the
@@ -1177,31 +1174,36 @@ service.
 
 
 
+RegionMetrics
+Encapsulates per-region load metrics.
+
+
+
 ServerName
 Name of a particular incarnation of an HBase Server.
 
 
-
+
 TableExistsException
 Thrown when a table exists but should not
 
 
-
+
 TableName
 Immutable POJO class for representing a table name.
 
 
-
+
 TableNotFoundException
 Thrown when a table can not be located
 
 
-
+
 Tag
 Tags are part of cells and helps to add metadata about 
them.
 
 
-
+
 ZooKeeperConnectionException
 Thrown if the client can't connect to zookeeper
 
@@ -1441,11 +1443,8 @@ service.
 
 
 
-ClusterStatus
-Deprecated. 
-As of release 2.0.0, this 
will be removed in HBase 3.0.0
- Use ClusterMetrics 
instead.
-
+ClusterMetrics
+Metrics information on the HBase cluster.
 
 
 
@@ -1956,16 +1955,13 @@ service.
 
 
 
-ClusterMetrics.Option
-Kinds of ClusterMetrics
+ClusterMetrics
+Metrics information on the HBase cluster.
 
 
 
-ClusterStatus
-Deprecated. 
-As of release 2.0.0, this 
will be removed in HBase 3.0.0
- Use ClusterMetrics 
instead.
-
+ClusterMetrics.Option
+Kinds of ClusterMetrics
 
 
 
@@ -2137,11 +2133,8 @@ service.
 
 
 
-ClusterStatus
-Deprecated. 
-As of release 2.0.0, this 
will be removed in HBase 3.0.0
- Use ClusterMetrics 
instead.
-
+ClusterMetrics
+Metrics information on the HBase cluster.
 
 
 
@@ -2155,11 +2148,8 @@ service.
 
 
 
-RegionLoad
-Deprecated. 
-As of release 2.0.0, this 
will be removed in HBase 3.0.0
- Use RegionMetrics instead.
-
+RegionMetrics
+Encapsulates per-region load metrics.
 
 
 
@@ -3205,11 +3195,8 @@ service.
 
 
 
-ClusterStatus
-Deprecated. 
-As of release 2.0.0, this 
will be removed in HBase 3.0.0
- Use ClusterMetrics 
instead.
-
+ClusterMetrics
+Metrics information on the HBase cluster.
 
 
 
@@ -3628,11 +3615,8 @@ service.
 
 
 
-ClusterStatus
-Deprecated. 
-As of release 2.0.0, this 
will be removed in HBase 3.0.0
- Use ClusterMetrics 
instead.
-
+ClusterMetrics
+Metrics information on the HBase cluster.
 
 
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/procedure2/package-tree.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/procedure2/package-tree.html 
b/devapidocs/org/apache/hadoop/hbase/procedure2/package-tree.html
index 90bcee3..cde2f87 100644
--- a/devapidocs/org/apache/hadoop/hbase/procedure2/package-tree.html
+++ b/devapidocs/org/apache/hadoop/hbase/procedure2/package-tree.html
@@ -204,9 +204,9 @@
 java.lang.http://docs.oracle.com/javase/8/docs/api/java/lang/Enum.html?is-external=true";
 title="class or interface in java.lan

[17/51] [partial] hbase-site git commit: Published site at .

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/master/MasterCoprocessorHost.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/master/MasterCoprocessorHost.html 
b/devapidocs/org/apache/hadoop/hbase/master/MasterCoprocessorHost.html
index 0e397dc..e3f0e57 100644
--- a/devapidocs/org/apache/hadoop/hbase/master/MasterCoprocessorHost.html
+++ b/devapidocs/org/apache/hadoop/hbase/master/MasterCoprocessorHost.html
@@ -115,7 +115,7 @@ var activeTableTab = "activeTableTab";
 
 
 @InterfaceAudience.Private
-public class MasterCoprocessorHost
+public class MasterCoprocessorHost
 extends CoprocessorHost
 Provides the coprocessor framework and environment for 
master oriented
  operations.  HMaster interacts with the 
loaded coprocessors
@@ -377,7 +377,7 @@ extends 
 void
-postGetClusterStatus(ClusterStatus status) 
+postGetClusterMetrics(ClusterMetrics status) 
 
 
 void
@@ -680,7 +680,7 @@ extends 
 void
-preGetClusterStatus() 
+preGetClusterMetrics() 
 
 
 void
@@ -956,7 +956,7 @@ extends 
 
 LOG
-private static final org.slf4j.Logger LOG
+private static final org.slf4j.Logger LOG
 
 
 
@@ -965,7 +965,7 @@ extends 
 
 masterServices
-private MasterServices masterServices
+private MasterServices masterServices
 
 
 
@@ -974,7 +974,7 @@ extends 
 
 masterObserverGetter
-private CoprocessorHost.ObserverGetter 
masterObserverGetter
+private CoprocessorHost.ObserverGetter 
masterObserverGetter
 
 
 
@@ -991,7 +991,7 @@ extends 
 
 MasterCoprocessorHost
-public MasterCoprocessorHost(MasterServices services,
+public MasterCoprocessorHost(MasterServices services,
  
org.apache.hadoop.conf.Configuration conf)
 
 
@@ -1009,7 +1009,7 @@ extends 
 
 createEnvironment
-public MasterCoprocessorHost.MasterEnvironment createEnvironment(MasterCoprocessor instance,
+public MasterCoprocessorHost.MasterEnvironment createEnvironment(MasterCoprocessor instance,
  
int priority,
  int seq,
  
org.apache.hadoop.conf.Configuration conf)
@@ -1027,7 +1027,7 @@ extends 
 
 checkAndGetInstance
-public MasterCoprocessor checkAndGetInstance(http://docs.oracle.com/javase/8/docs/api/java/lang/Class.html?is-external=true";
 title="class or interface in java.lang">Class implClass)
+public MasterCoprocessor checkAndGetInstance(http://docs.oracle.com/javase/8/docs/api/java/lang/Class.html?is-external=true";
 title="class or interface in java.lang">Class implClass)
   throws http://docs.oracle.com/javase/8/docs/api/java/lang/InstantiationException.html?is-external=true";
 title="class or interface in java.lang">InstantiationException,
  http://docs.oracle.com/javase/8/docs/api/java/lang/IllegalAccessException.html?is-external=true";
 title="class or interface in java.lang">IllegalAccessException
 Description copied from 
class: CoprocessorHost
@@ -1051,7 +1051,7 @@ extends 
 
 preCreateNamespace
-public void preCreateNamespace(NamespaceDescriptor ns)
+public void preCreateNamespace(NamespaceDescriptor ns)
 throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
 
 Throws:
@@ -1065,7 +1065,7 @@ extends 
 
 postCreateNamespace
-public void postCreateNamespace(NamespaceDescriptor ns)
+public void postCreateNamespace(NamespaceDescriptor ns)
  throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
 
 Throws:
@@ -1079,7 +1079,7 @@ extends 
 
 preDeleteNamespace
-public void preDeleteNamespace(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String namespaceName)
+public void preDeleteNamespace(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String namespaceName)
 throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
 
 Throws:
@@ -1093,7 +1093,7 @@ extends 
 
 postDeleteNamespace
-public void postDeleteNamespace(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String namespaceName)
+public void postDeleteNamespace(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String namespaceName)
   

[24/51] [partial] hbase-site git commit: Published site at .

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/coprocessor/class-use/MasterCoprocessorEnvironment.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/coprocessor/class-use/MasterCoprocessorEnvironment.html
 
b/devapidocs/org/apache/hadoop/hbase/coprocessor/class-use/MasterCoprocessorEnvironment.html
index 2041055..b03d0d0 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/coprocessor/class-use/MasterCoprocessorEnvironment.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/coprocessor/class-use/MasterCoprocessorEnvironment.html
@@ -331,8 +331,8 @@
 
 
 default void
-MasterObserver.postGetClusterStatus(ObserverContext ctx,
-ClusterStatus status)
+MasterObserver.postGetClusterMetrics(ObserverContext ctx,
+ ClusterMetrics status)
 Called after get cluster status.
 
 
@@ -811,7 +811,7 @@
 
 
 default void
-MasterObserver.preGetClusterStatus(ObserverContext ctx)
+MasterObserver.preGetClusterMetrics(ObserverContext ctx)
 Called before get cluster status.
 
 
@@ -1483,128 +1483,124 @@
 
 
 void
-AccessController.preGetClusterStatus(ObserverContext ctx) 
-
-
-void
 AccessController.preGetLocks(ObserverContext ctx) 
 
-
+
 void
 AccessController.preGetNamespaceDescriptor(ObserverContext ctx,
  http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in 
java.lang">String namespace) 
 
-
+
 void
 AccessController.preGetProcedures(ObserverContext ctx) 
 
-
+
 void
 AccessController.preGetReplicationPeerConfig(ObserverContext ctx,
http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in 
java.lang">String peerId) 
 
-
+
 void
 AccessController.preGetTableDescriptors(ObserverContext ctx,
   http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List tableNamesList,
   http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List descriptors,
   http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in 
java.lang">String regex) 
 
-
+
 void
 AccessController.preListDecommissionedRegionServers(ObserverContext ctx) 
 
-
+
 void
 AccessController.preListReplicationPeers(ObserverContext ctx,
http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in 
java.lang">String regex) 
 
-
+
 void
 AccessController.preListSnapshot(ObserverContext ctx,
SnapshotDescription snapshot) 
 
-
+
 void
 AccessController.preLockHeartbeat(ObserverContext ctx,
 TableName tableName,
 http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in 
java.lang">String description) 
 
-
+
 void
 AccessController.preMergeRegions(ObserverContext ctx,
RegionInfo[] regionsToMerge) 
 
-
+
 void
 AccessController.preModifyNamespace(ObserverContext ctx,
   NamespaceDescriptor ns) 
 
-
+
 void
 AccessController.preModifyTable(ObserverContext c,
   TableName tableName,
   TableDescriptor htd) 
 
-
+
 void
 CoprocessorWhitelistMasterObserver.preModifyTable(ObserverContext ctx,
   TableName tableName,
   TableDescriptor htd) 
 
-
+
 void
 AccessController.preMove(ObserverContext c,
RegionInfo region,
ServerName srcServer,
ServerName destServer) 
 
-
+
 void
 AccessController.preMoveServers(ObserverContext ctx,
   http://docs.oracle.com/javase/8/docs/api/java/util/Set.html?is-external=true";
 title="class or interface in java.util">Set
 servers, http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true"; title="class or interface in java.lang">String targetGroup)  - + void AccessController.preMoveServersAndTables(ObserverContext ctx, http://docs.oracle.com/javase/8/docs/api/java/util/Set.html?is-external=true"

[44/51] [partial] hbase-site git commit: Published site at .

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/apidocs/src-html/org/apache/hadoop/hbase/client/AsyncAdmin.html
--
diff --git a/apidocs/src-html/org/apache/hadoop/hbase/client/AsyncAdmin.html 
b/apidocs/src-html/org/apache/hadoop/hbase/client/AsyncAdmin.html
index 701636d..8663b8c 100644
--- a/apidocs/src-html/org/apache/hadoop/hbase/client/AsyncAdmin.html
+++ b/apidocs/src-html/org/apache/hadoop/hbase/client/AsyncAdmin.html
@@ -26,1198 +26,1200 @@
 018package org.apache.hadoop.hbase.client;
 019
 020import com.google.protobuf.RpcChannel;
-021import java.util.Arrays;
-022import java.util.Collection;
-023import java.util.EnumSet;
-024import java.util.List;
-025import java.util.Map;
-026import java.util.Optional;
-027import java.util.Set;
-028import 
java.util.concurrent.CompletableFuture;
-029import java.util.function.Function;
-030import java.util.regex.Pattern;
+021import java.util.Collection;
+022import java.util.EnumSet;
+023import java.util.List;
+024import java.util.Map;
+025import java.util.Optional;
+026import java.util.Set;
+027import 
java.util.concurrent.CompletableFuture;
+028import java.util.function.Function;
+029import java.util.regex.Pattern;
+030import 
org.apache.hadoop.hbase.ClusterMetrics;
 031import 
org.apache.hadoop.hbase.ClusterMetrics.Option;
-032import 
org.apache.hadoop.hbase.ClusterStatus;
-033import 
org.apache.hadoop.hbase.NamespaceDescriptor;
-034import 
org.apache.hadoop.hbase.RegionLoad;
-035import 
org.apache.hadoop.hbase.ServerName;
-036import 
org.apache.hadoop.hbase.TableName;
-037import 
org.apache.hadoop.hbase.client.replication.TableCFs;
-038import 
org.apache.hadoop.hbase.client.security.SecurityCapability;
-039import 
org.apache.hadoop.hbase.quotas.QuotaFilter;
-040import 
org.apache.hadoop.hbase.quotas.QuotaSettings;
-041import 
org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
-042import 
org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
-043import 
org.apache.yetus.audience.InterfaceAudience;
-044
-045/**
-046 * The asynchronous administrative API 
for HBase.
-047 * @since 2.0.0
-048 */
-049@InterfaceAudience.Public
-050public interface AsyncAdmin {
-051
-052  /**
-053   * @param tableName Table to check.
-054   * @return True if table exists 
already. The return value will be wrapped by a
-055   * {@link CompletableFuture}.
-056   */
-057  CompletableFuture 
tableExists(TableName tableName);
-058
-059  /**
-060   * List all the userspace tables.
-061   * @return - returns a list of 
TableDescriptors wrapped by a {@link CompletableFuture}.
-062   */
-063  default 
CompletableFuture> listTableDescriptors() {
-064return listTableDescriptors(false);
-065  }
-066
-067  /**
-068   * List all the tables.
-069   * @param includeSysTables False to 
match only against userspace tables
-070   * @return - returns a list of 
TableDescriptors wrapped by a {@link CompletableFuture}.
-071   */
-072  
CompletableFuture> 
listTableDescriptors(boolean includeSysTables);
-073
-074  /**
-075   * List all the tables matching the 
given pattern.
-076   * @param pattern The compiled regular 
expression to match against
-077   * @param includeSysTables False to 
match only against userspace tables
-078   * @return - returns a list of 
TableDescriptors wrapped by a {@link CompletableFuture}.
-079   */
-080  
CompletableFuture> 
listTableDescriptors(Pattern pattern,
-081  boolean includeSysTables);
-082
-083  /**
-084   * Get list of table descriptors by 
namespace.
-085   * @param name namespace name
-086   * @return returns a list of 
TableDescriptors wrapped by a {@link CompletableFuture}.
-087   */
-088  
CompletableFuture> 
listTableDescriptorsByNamespace(String name);
-089
-090  /**
-091   * List all of the names of userspace 
tables.
-092   * @return a list of table names 
wrapped by a {@link CompletableFuture}.
-093   * @see #listTableNames(Pattern, 
boolean)
-094   */
-095  default 
CompletableFuture> listTableNames() {
-096return listTableNames(false);
-097  }
-098
-099  /**
-100   * List all of the names of tables.
-101   * @param includeSysTables False to 
match only against userspace tables
-102   * @return a list of table names 
wrapped by a {@link CompletableFuture}.
-103   */
-104  
CompletableFuture> listTableNames(boolean 
includeSysTables);
-105
-106  /**
-107   * List all of the names of userspace 
tables.
-108   * @param pattern The regular 
expression to match against
-109   * @param includeSysTables False to 
match only against userspace tables
-110   * @return a list of table names 
wrapped by a {@link CompletableFuture}.
-111   */
-112  
CompletableFuture> listTableNames(Pattern pattern, 
boolean includeSysTables);
-113
-114  /**
-115   * Get list of table names by 
namespace.
-116   * @param name

[37/51] [partial] hbase-site git commit: Published site at .

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/class-use/ClusterMetrics.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/class-use/ClusterMetrics.html 
b/devapidocs/org/apache/hadoop/hbase/class-use/ClusterMetrics.html
index e5d9324..0c3406e 100644
--- a/devapidocs/org/apache/hadoop/hbase/class-use/ClusterMetrics.html
+++ b/devapidocs/org/apache/hadoop/hbase/class-use/ClusterMetrics.html
@@ -86,6 +86,34 @@
 org.apache.hadoop.hbase
  
 
+
+org.apache.hadoop.hbase.client
+
+Provides HBase Client
+
+
+
+org.apache.hadoop.hbase.coprocessor
+
+Table of Contents
+
+
+
+org.apache.hadoop.hbase.master
+ 
+
+
+org.apache.hadoop.hbase.master.balancer
+ 
+
+
+org.apache.hadoop.hbase.rsgroup
+ 
+
+
+org.apache.hadoop.hbase.util
+ 
+
 
 
 
@@ -176,6 +204,268 @@
 
 
 
+
+
+
+Uses of ClusterMetrics in org.apache.hadoop.hbase.client
+
+Methods in org.apache.hadoop.hbase.client
 that return ClusterMetrics 
+
+Modifier and Type
+Method and Description
+
+
+
+default ClusterMetrics
+Admin.getClusterMetrics()
+Get whole cluster metrics, containing status about:
+
+
+
+ClusterMetrics
+Admin.getClusterMetrics(http://docs.oracle.com/javase/8/docs/api/java/util/EnumSet.html?is-external=true";
 title="class or interface in java.util">EnumSet options)
+Get cluster status with a set of ClusterMetrics.Option 
to get desired status.
+
+
+
+ClusterMetrics
+HBaseAdmin.getClusterMetrics(http://docs.oracle.com/javase/8/docs/api/java/util/EnumSet.html?is-external=true";
 title="class or interface in java.util">EnumSet options) 
+
+
+
+
+Methods in org.apache.hadoop.hbase.client
 that return types with arguments of type ClusterMetrics 
+
+Modifier and Type
+Method and Description
+
+
+
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFuture
+AsyncHBaseAdmin.getClusterMetrics() 
+
+
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFuture
+AsyncAdmin.getClusterMetrics() 
+
+
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFuture
+RawAsyncHBaseAdmin.getClusterMetrics() 
+
+
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFuture
+AsyncHBaseAdmin.getClusterMetrics(http://docs.oracle.com/javase/8/docs/api/java/util/EnumSet.html?is-external=true";
 title="class or interface in java.util">EnumSet options) 
+
+
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFuture
+AsyncAdmin.getClusterMetrics(http://docs.oracle.com/javase/8/docs/api/java/util/EnumSet.html?is-external=true";
 title="class or interface in java.util">EnumSet options) 
+
+
+http://docs.oracle.com/javase/8/docs/api/java/util/concurrent/CompletableFuture.html?is-external=true";
 title="class or interface in java.util.concurrent">CompletableFuture
+RawAsyncHBaseAdmin.getClusterMetrics(http://docs.oracle.com/javase/8/docs/api/java/util/EnumSet.html?is-external=true";
 title="class or interface in java.util">EnumSet options) 
+
+
+
+
+Methods in org.apache.hadoop.hbase.client
 with parameters of type ClusterMetrics 
+
+Modifier and Type
+Method and Description
+
+
+
+void
+ClusterStatusListener.receive(ClusterMetrics ncs)
+Acts upon the reception of a new cluster status.
+
+
+
+
+
+
+
+
+Uses of ClusterMetrics in org.apache.hadoop.hbase.coprocessor
+
+Methods in org.apache.hadoop.hbase.coprocessor
 with parameters of type ClusterMetrics 
+
+Modifier and Type
+Method and Description
+
+
+
+default void
+MasterObserver.postGetClusterMetrics(ObserverContext ctx,
+ ClusterMetrics status)
+Called after get cluster status.
+
+
+
+
+
+
+
+
+Uses of ClusterMetrics in org.apache.hadoop.hbase.master
+
+Methods in org.apache.hadoop.hbase.master
 that return ClusterMetrics 
+
+Modifier and Type
+Method and Description
+
+
+
+ClusterMetrics
+HMaster.getClusterMetrics() 
+
+
+ClusterMetrics
+HMaster.getClusterMetrics(http://docs.oracle.com/javase/8/docs/api/java/util/EnumSet.html?is-external=true";
 title="class or interface in java.util">EnumSet options) 
+
+
+ClusterMetrics
+HMaster.getClusterMetricsWithoutCoprocessor() 
+
+
+ClusterMetrics
+HMaster.getClusterMetricsWithoutCoprocessor(http://docs.oracle.com/java

[09/51] [partial] hbase-site git commit: Published site at .

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/regionserver/querymatcher/package-tree.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/regionserver/querymatcher/package-tree.html
 
b/devapidocs/org/apache/hadoop/hbase/regionserver/querymatcher/package-tree.html
index e97b095..af72bf3 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/regionserver/querymatcher/package-tree.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/regionserver/querymatcher/package-tree.html
@@ -130,9 +130,9 @@
 
 java.lang.http://docs.oracle.com/javase/8/docs/api/java/lang/Enum.html?is-external=true";
 title="class or interface in java.lang">Enum (implements java.lang.http://docs.oracle.com/javase/8/docs/api/java/lang/Comparable.html?is-external=true";
 title="class or interface in java.lang">Comparable, java.io.http://docs.oracle.com/javase/8/docs/api/java/io/Serializable.html?is-external=true";
 title="class or interface in java.io">Serializable)
 
-org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher.MatchCode
 org.apache.hadoop.hbase.regionserver.querymatcher.StripeCompactionScanQueryMatcher.DropDeletesInOutput
 org.apache.hadoop.hbase.regionserver.querymatcher.DeleteTracker.DeleteResult
+org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher.MatchCode
 
 
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/regionserver/wal/package-tree.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/regionserver/wal/package-tree.html 
b/devapidocs/org/apache/hadoop/hbase/regionserver/wal/package-tree.html
index 844f9ca..12b7c54 100644
--- a/devapidocs/org/apache/hadoop/hbase/regionserver/wal/package-tree.html
+++ b/devapidocs/org/apache/hadoop/hbase/regionserver/wal/package-tree.html
@@ -238,8 +238,8 @@
 
 java.lang.http://docs.oracle.com/javase/8/docs/api/java/lang/Enum.html?is-external=true";
 title="class or interface in java.lang">Enum (implements java.lang.http://docs.oracle.com/javase/8/docs/api/java/lang/Comparable.html?is-external=true";
 title="class or interface in java.lang">Comparable, java.io.http://docs.oracle.com/javase/8/docs/api/java/io/Serializable.html?is-external=true";
 title="class or interface in java.io">Serializable)
 
-org.apache.hadoop.hbase.regionserver.wal.RingBufferTruck.Type
 org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.WALHdrResult
+org.apache.hadoop.hbase.regionserver.wal.RingBufferTruck.Type
 
 
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.WarnOnlyAbortable.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.WarnOnlyAbortable.html
 
b/devapidocs/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.WarnOnlyAbortable.html
index d72d004..7ce2320 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.WarnOnlyAbortable.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.WarnOnlyAbortable.html
@@ -117,7 +117,7 @@ var activeTableTab = "activeTableTab";
 
 
 
-private static class ReplicationLogCleaner.WarnOnlyAbortable
+private static class ReplicationLogCleaner.WarnOnlyAbortable
 extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object
 implements Abortable
 
@@ -198,7 +198,7 @@ implements 
 
 WarnOnlyAbortable
-private WarnOnlyAbortable()
+private WarnOnlyAbortable()
 
 
 
@@ -215,7 +215,7 @@ implements 
 
 abort
-public void abort(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String why,
+public void abort(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String why,
   http://docs.oracle.com/javase/8/docs/api/java/lang/Throwable.html?is-external=true";
 title="class or interface in java.lang">Throwable e)
 Description copied from 
interface: Abortable
 Abort the server or client.
@@ -234,7 +234,7 @@ implements 
 
 isAborted
-public boolean isAborted()
+public boolean isAborted()
 Description copied from 
interface: Abortable
 Check if the server or client was aborted.
 

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.html
 
b/devapidocs/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.html
index 31e194a..7b736a8 1006

[31/51] [partial] hbase-site git commit: Published site at .

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/client/ClusterStatusListener.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/client/ClusterStatusListener.html 
b/devapidocs/org/apache/hadoop/hbase/client/ClusterStatusListener.html
index b31c22c..7ae608c 100644
--- a/devapidocs/org/apache/hadoop/hbase/client/ClusterStatusListener.html
+++ b/devapidocs/org/apache/hadoop/hbase/client/ClusterStatusListener.html
@@ -114,7 +114,7 @@ var activeTableTab = "activeTableTab";
 
 
 @InterfaceAudience.Private
-class ClusterStatusListener
+class ClusterStatusListener
 extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object
 implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.html?is-external=true";
 title="class or interface in java.io">Closeable
 A class that receives the cluster status, and provide it as 
a set of service to the client.
@@ -243,7 +243,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 
 
 void
-receive(ClusterStatus ncs)
+receive(ClusterMetrics ncs)
 Acts upon the reception of a new cluster status.
 
 
@@ -275,7 +275,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 
 
 LOG
-private static final org.slf4j.Logger LOG
+private static final org.slf4j.Logger LOG
 
 
 
@@ -284,7 +284,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 
 
 deadServers
-private final http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List deadServers
+private final http://docs.oracle.com/javase/8/docs/api/java/util/List.html?is-external=true";
 title="class or interface in java.util">List deadServers
 
 
 
@@ -293,7 +293,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 
 
 deadServerHandler
-protected final ClusterStatusListener.DeadServerHandler deadServerHandler
+protected final ClusterStatusListener.DeadServerHandler deadServerHandler
 
 
 
@@ -302,7 +302,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 
 
 listener
-private final ClusterStatusListener.Listener listener
+private final ClusterStatusListener.Listener listener
 
 
 
@@ -311,7 +311,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 
 
 STATUS_LISTENER_CLASS
-public static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String STATUS_LISTENER_CLASS
+public static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String STATUS_LISTENER_CLASS
 The implementation class to use to read the status.
 
 See Also:
@@ -325,7 +325,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 
 
 DEFAULT_STATUS_LISTENER_CLASS
-public static final http://docs.oracle.com/javase/8/docs/api/java/lang/Class.html?is-external=true";
 title="class or interface in java.lang">Class DEFAULT_STATUS_LISTENER_CLASS
+public static final http://docs.oracle.com/javase/8/docs/api/java/lang/Class.html?is-external=true";
 title="class or interface in java.lang">Class DEFAULT_STATUS_LISTENER_CLASS
 
 
 
@@ -342,7 +342,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 
 
 ClusterStatusListener
-public ClusterStatusListener(ClusterStatusListener.DeadServerHandler dsh,
+public ClusterStatusListener(ClusterStatusListener.DeadServerHandler dsh,
  org.apache.hadoop.conf.Configuration conf,
  http://docs.oracle.com/javase/8/docs/api/java/lang/Class.html?is-external=true";
 title="class or interface in java.lang">Class listenerClass)
   throws http://docs.oracle.com/javase/8/docs/api/java/io/IOException.html?is-external=true";
 title="class or interface in java.io">IOException
@@ -360,13 +360,13 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 
 
 Method Detail
-
+
 
 
 
 
 receive
-public void receive(ClusterStatus ncs)
+public void receive(ClusterMetrics ncs)
 Acts upon the reception of a new cluster status.
 
 Parameters:
@@ -380,7 +380,7 @@ implements http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.h
 
 
 close
-public void close()
+public void close()
 
 Specified by:
 http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.html?is-external=true#close--";
 title="class or interface in java.io">close in 
interface http://docs.oracle.com/javase/8/docs/api/java/io/Closeable.html?is-external=true";
 title="class or interface in java.io">Closeable
@@ -395,7 +395,7 @@ implements http:

[03/51] [partial] hbase-site git commit: Published site at .

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.OnlineEntry.html
--
diff --git a/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.OnlineEntry.html 
b/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.OnlineEntry.html
index 3cef254..0f033c6 100644
--- a/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.OnlineEntry.html
+++ b/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.OnlineEntry.html
@@ -113,7 +113,7 @@ var activeTableTab = "activeTableTab";
 
 
 
-static class HBaseFsck.OnlineEntry
+static class HBaseFsck.OnlineEntry
 extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object
 Stores the regioninfo retrieved from Online region 
servers.
 
@@ -206,7 +206,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 hri
-RegionInfo hri
+RegionInfo hri
 
 
 
@@ -215,7 +215,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 hsa
-ServerName hsa
+ServerName hsa
 
 
 
@@ -232,7 +232,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 OnlineEntry
-OnlineEntry()
+OnlineEntry()
 
 
 
@@ -249,7 +249,7 @@ extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?
 
 
 toString
-public http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String toString()
+public http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String toString()
 
 Overrides:
 http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true#toString--";
 title="class or interface in java.lang">toString in 
class http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.PrintingErrorReporter.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.PrintingErrorReporter.html 
b/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.PrintingErrorReporter.html
index c1666dc..f98492d 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.PrintingErrorReporter.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/util/HBaseFsck.PrintingErrorReporter.html
@@ -117,7 +117,7 @@ var activeTableTab = "activeTableTab";
 
 
 
-static class HBaseFsck.PrintingErrorReporter
+static class HBaseFsck.PrintingErrorReporter
 extends http://docs.oracle.com/javase/8/docs/api/java/lang/Object.html?is-external=true";
 title="class or interface in java.lang">Object
 implements HBaseFsck.ErrorReporter
 
@@ -301,7 +301,7 @@ implements 
 
 errorCount
-public int errorCount
+public int errorCount
 
 
 
@@ -310,7 +310,7 @@ implements 
 
 showProgress
-private int showProgress
+private int showProgress
 
 
 
@@ -319,7 +319,7 @@ implements 
 
 progressThreshold
-private static final int progressThreshold
+private static final int progressThreshold
 
 See Also:
 Constant
 Field Values
@@ -332,7 +332,7 @@ implements 
 
 errorTables
-http://docs.oracle.com/javase/8/docs/api/java/util/Set.html?is-external=true";
 title="class or interface in java.util">Set errorTables
+http://docs.oracle.com/javase/8/docs/api/java/util/Set.html?is-external=true";
 title="class or interface in java.util">Set errorTables
 
 
 
@@ -341,7 +341,7 @@ implements 
 
 errorList
-private http://docs.oracle.com/javase/8/docs/api/java/util/ArrayList.html?is-external=true";
 title="class or interface in java.util">ArrayList errorList
+private http://docs.oracle.com/javase/8/docs/api/java/util/ArrayList.html?is-external=true";
 title="class or interface in java.util">ArrayList errorList
 
 
 
@@ -358,7 +358,7 @@ implements 
 
 PrintingErrorReporter
-PrintingErrorReporter()
+PrintingErrorReporter()
 
 
 
@@ -375,7 +375,7 @@ implements 
 
 clear
-public void clear()
+public void clear()
 
 Specified by:
 clear in
 interface HBaseFsck.ErrorReporter
@@ -388,7 +388,7 @@ implements 
 
 reportError
-public void reportError(HBaseFsck.ErrorReporter.ERROR_CODE errorCode,
+public void reportError(HBaseFsck.ErrorReporter.ERROR_CODE errorCode,
 http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String message)
 
 Specified by:
@@ -402,7 +402,7 @@ implements 
 
 reportError
-public void reportError(HBaseFsck.ErrorReporter.ERROR_CODE errorCode,
+public void reportError(HBaseFsck.ErrorReporter.ERROR_CODE errorCode,
 http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?i

[20/51] [partial] hbase-site git commit: Published site at .

http://git-wip-us.apache.org/repos/asf/hbase-site/blob/c7c40c62/devapidocs/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.html
--
diff --git 
a/devapidocs/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.html
 
b/devapidocs/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.html
index 526491c..afa053f 100644
--- 
a/devapidocs/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.html
+++ 
b/devapidocs/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.html
@@ -18,7 +18,7 @@
 catch(err) {
 }
 //-->
-var methods = {"i0":10,"i1":10,"i2":9,"i3":9,"i4":9,"i5":10,"i6":9,"i7":9};
+var methods = 
{"i0":10,"i1":10,"i2":9,"i3":9,"i4":9,"i5":10,"i6":10,"i7":9,"i8":9};
 var tabs = {65535:["t0","All Methods"],1:["t1","Static 
Methods"],2:["t2","Instance Methods"],8:["t4","Concrete Methods"]};
 var altColor = "altColor";
 var rowColor = "rowColor";
@@ -118,7 +118,7 @@ var activeTableTab = "activeTableTab";
 
 
 
-public class VerifyReplication
+public class VerifyReplication
 extends org.apache.hadoop.conf.Configured
 implements org.apache.hadoop.util.Tool
 This map-only job compares the data from a local table with 
a remote one.
@@ -315,15 +315,20 @@ implements org.apache.hadoop.util.Tool
 printUsage(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in 
java.lang">String errorMsg) 
 
 
+private void
+restoreSnapshotForPeerCluster(org.apache.hadoop.conf.Configuration conf,
+ http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in 
java.lang">String peerQuorumAddress) 
+
+
 int
 run(http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in 
java.lang">String[] args) 
 
-
+
 private static void
 setRowPrefixFilter(Scan scan,
   http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in 
java.lang">String rowPrefixes) 
 
-
+
 private static void
 setStartAndStopRows(Scan scan,
byte[] startPrefixRow,
@@ -371,7 +376,7 @@ implements org.apache.hadoop.util.Tool
 
 
 LOG
-private static final org.slf4j.Logger LOG
+private static final org.slf4j.Logger LOG
 
 
 
@@ -380,7 +385,7 @@ implements org.apache.hadoop.util.Tool
 
 
 NAME
-public static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String NAME
+public static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String NAME
 
 See Also:
 Constant
 Field Values
@@ -393,7 +398,7 @@ implements org.apache.hadoop.util.Tool
 
 
 PEER_CONFIG_PREFIX
-private static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String PEER_CONFIG_PREFIX
+private static final http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String PEER_CONFIG_PREFIX
 
 See Also:
 Constant
 Field Values
@@ -406,7 +411,7 @@ implements org.apache.hadoop.util.Tool
 
 
 startTime
-long startTime
+long startTime
 
 
 
@@ -415,7 +420,7 @@ implements org.apache.hadoop.util.Tool
 
 
 endTime
-long endTime
+long endTime
 
 
 
@@ -424,7 +429,7 @@ implements org.apache.hadoop.util.Tool
 
 
 batch
-int batch
+int batch
 
 
 
@@ -433,7 +438,7 @@ implements org.apache.hadoop.util.Tool
 
 
 versions
-int versions
+int versions
 
 
 
@@ -442,7 +447,7 @@ implements org.apache.hadoop.util.Tool
 
 
 tableName
-http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String tableName
+http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String tableName
 
 
 
@@ -451,7 +456,7 @@ implements org.apache.hadoop.util.Tool
 
 
 families
-http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String families
+http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String families
 
 
 
@@ -460,7 +465,7 @@ implements org.apache.hadoop.util.Tool
 
 
 delimiter
-http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String delimiter
+http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String delimiter
 
 
 
@@ -469,7 +474,7 @@ implements org.apache.hadoop.util.Tool
 
 
 peerId
-http://docs.oracle.com/javase/8/docs/api/java/lang/String.html?is-external=true";
 title="class or interface in java.lang">String peerId

hbase-site git commit: INFRA-10751 Empty commit

Repository: hbase-site
Updated Branches:
  refs/heads/asf-site c7c40c622 -> 535897ee2


INFRA-10751 Empty commit


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

Branch: refs/heads/asf-site
Commit: 535897ee2641dda9ac102da5d8a6bbe00bc5501c
Parents: c7c40c6
Author: jenkins 
Authored: Thu Jan 4 15:20:03 2018 +
Committer: jenkins 
Committed: Thu Jan 4 15:20:03 2018 +

--

--




hbase git commit: HBASE-19688 TimeToLiveProcedureWALCleaner should extends BaseLogCleanerDelegate

Repository: hbase
Updated Branches:
  refs/heads/master 519543594 -> bff937a76


HBASE-19688 TimeToLiveProcedureWALCleaner should extends BaseLogCleanerDelegate

Signed-off-by: tedyu 


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

Branch: refs/heads/master
Commit: bff937a767ecb851a4ba312ece52b50b84df4976
Parents: 5195435
Author: Reid Chan 
Authored: Tue Jan 2 22:16:34 2018 +0800
Committer: tedyu 
Committed: Thu Jan 4 10:26:23 2018 -0800

--
 .../hadoop/hbase/master/cleaner/TimeToLiveProcedureWALCleaner.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/bff937a7/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/TimeToLiveProcedureWALCleaner.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/TimeToLiveProcedureWALCleaner.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/TimeToLiveProcedureWALCleaner.java
index 467accd..5535a4b 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/TimeToLiveProcedureWALCleaner.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/TimeToLiveProcedureWALCleaner.java
@@ -31,7 +31,7 @@ import org.slf4j.LoggerFactory;
  * deleted. By default they are allowed to live for {@value #DEFAULT_TTL}
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
-public class TimeToLiveProcedureWALCleaner extends BaseFileCleanerDelegate {
+public class TimeToLiveProcedureWALCleaner extends BaseLogCleanerDelegate {
   private static final Logger LOG =
   LoggerFactory.getLogger(TimeToLiveProcedureWALCleaner.class.getName());
   public static final String TTL_CONF_KEY = 
"hbase.master.procedurewalcleaner.ttl";



hbase git commit: HBASE-19688 TimeToLiveProcedureWALCleaner should extends BaseLogCleanerDelegate

Repository: hbase
Updated Branches:
  refs/heads/branch-2 0b62528db -> e35fec284


HBASE-19688 TimeToLiveProcedureWALCleaner should extends BaseLogCleanerDelegate

Signed-off-by: tedyu 


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

Branch: refs/heads/branch-2
Commit: e35fec284d756892f09cbdd6eedf1bc66d8006fb
Parents: 0b62528
Author: Reid Chan 
Authored: Tue Jan 2 22:16:34 2018 +0800
Committer: tedyu 
Committed: Thu Jan 4 10:26:51 2018 -0800

--
 .../hadoop/hbase/master/cleaner/TimeToLiveProcedureWALCleaner.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/e35fec28/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/TimeToLiveProcedureWALCleaner.java
--
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/TimeToLiveProcedureWALCleaner.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/TimeToLiveProcedureWALCleaner.java
index 467accd..5535a4b 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/TimeToLiveProcedureWALCleaner.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/TimeToLiveProcedureWALCleaner.java
@@ -31,7 +31,7 @@ import org.slf4j.LoggerFactory;
  * deleted. By default they are allowed to live for {@value #DEFAULT_TTL}
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
-public class TimeToLiveProcedureWALCleaner extends BaseFileCleanerDelegate {
+public class TimeToLiveProcedureWALCleaner extends BaseLogCleanerDelegate {
   private static final Logger LOG =
   LoggerFactory.getLogger(TimeToLiveProcedureWALCleaner.class.getName());
   public static final String TTL_CONF_KEY = 
"hbase.master.procedurewalcleaner.ttl";



[03/32] hbase git commit: HBASE-19686 Use KeyLocker instead of ReentrantLock in PeerProcedureHandlerImpl

HBASE-19686 Use KeyLocker instead of ReentrantLock in PeerProcedureHandlerImpl


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

Branch: refs/heads/HBASE-19397
Commit: 2301a456dffce6a4492fd84e3e69cac6038f7c24
Parents: 8cf670e
Author: zhangduo 
Authored: Tue Jan 2 16:13:55 2018 +0800
Committer: zhangduo 
Committed: Fri Jan 5 13:54:37 2018 +0800

--
 .../regionserver/PeerProcedureHandlerImpl.java  | 41 ++--
 1 file changed, 29 insertions(+), 12 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/2301a456/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
index 1efe180..c09c6a0 100644
--- 
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
@@ -19,10 +19,10 @@
 package org.apache.hadoop.hbase.replication.regionserver;
 
 import java.io.IOException;
-import java.util.concurrent.locks.ReentrantLock;
-
+import java.util.concurrent.locks.Lock;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
+import org.apache.hadoop.hbase.util.KeyLocker;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -32,7 +32,7 @@ public class PeerProcedureHandlerImpl implements 
PeerProcedureHandler {
   private static final Logger LOG = 
LoggerFactory.getLogger(PeerProcedureHandlerImpl.class);
 
   private final ReplicationSourceManager replicationSourceManager;
-  private final ReentrantLock peersLock = new ReentrantLock();
+  private final KeyLocker peersLock = new KeyLocker<>();
 
   public PeerProcedureHandlerImpl(ReplicationSourceManager 
replicationSourceManager) {
 this.replicationSourceManager = replicationSourceManager;
@@ -40,40 +40,57 @@ public class PeerProcedureHandlerImpl implements 
PeerProcedureHandler {
 
   @Override
   public void addPeer(String peerId) throws ReplicationException, IOException {
-peersLock.lock();
+Lock peerLock = peersLock.acquireLock(peerId);
 try {
   replicationSourceManager.addPeer(peerId);
 } finally {
-  peersLock.unlock();
+  peerLock.unlock();
 }
   }
 
   @Override
   public void removePeer(String peerId) throws ReplicationException, 
IOException {
-peersLock.lock();
+Lock peerLock = peersLock.acquireLock(peerId);
 try {
   if (replicationSourceManager.getReplicationPeers().getPeer(peerId) != 
null) {
 replicationSourceManager.removePeer(peerId);
   }
 } finally {
-  peersLock.unlock();
+  peerLock.unlock();
 }
   }
 
   @Override
   public void disablePeer(String peerId) throws ReplicationException, 
IOException {
-PeerState newState = 
replicationSourceManager.getReplicationPeers().refreshPeerState(peerId);
-LOG.info("disable replication peer, id: " + peerId + ", new state: " + 
newState);
+PeerState newState;
+Lock peerLock = peersLock.acquireLock(peerId);
+try {
+  newState = 
replicationSourceManager.getReplicationPeers().refreshPeerState(peerId);
+} finally {
+  peerLock.unlock();
+}
+LOG.info("disable replication peer, id: {}, new state: {}", peerId, 
newState);
   }
 
   @Override
   public void enablePeer(String peerId) throws ReplicationException, 
IOException {
-PeerState newState = 
replicationSourceManager.getReplicationPeers().refreshPeerState(peerId);
-LOG.info("enable replication peer, id: " + peerId + ", new state: " + 
newState);
+PeerState newState;
+Lock peerLock = peersLock.acquireLock(peerId);
+try {
+  newState = 
replicationSourceManager.getReplicationPeers().refreshPeerState(peerId);
+} finally {
+  peerLock.unlock();
+}
+LOG.info("enable replication peer, id: {}, new state: {}", peerId, 
newState);
   }
 
   @Override
   public void updatePeerConfig(String peerId) throws ReplicationException, 
IOException {
-replicationSourceManager.getReplicationPeers().refreshPeerConfig(peerId);
+Lock peerLock = peersLock.acquireLock(peerId);
+try {
+  replicationSourceManager.getReplicationPeers().refreshPeerConfig(peerId);
+} finally {
+  peerLock.unlock();
+}

[22/32] hbase git commit: HBASE-19544 Add UTs for testing concurrent modifications on replication peer

HBASE-19544 Add UTs for testing concurrent modifications on replication peer

Signed-off-by: zhangduo 


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

Branch: refs/heads/HBASE-19397
Commit: 8996872892e6e7056726361bdb741b5586dccb8a
Parents: 2301a45
Author: Guanghao Zhang 
Authored: Tue Jan 2 17:07:41 2018 +0800
Committer: zhangduo 
Committed: Fri Jan 5 13:54:37 2018 +0800

--
 .../replication/TestReplicationAdmin.java   | 69 
 1 file changed, 69 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/89968728/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 9b71595..89cf393 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
@@ -31,6 +31,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.regex.Pattern;
 
 import org.apache.hadoop.conf.Configuration;
@@ -55,6 +56,8 @@ 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;
 
 /**
  * Unit testing of ReplicationAdmin
@@ -62,6 +65,8 @@ import org.junit.rules.TestName;
 @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();
 
@@ -111,6 +116,70 @@ public class TestReplicationAdmin {
   }
 
   @Test
+  public void testConcurrentPeerOperations() throws Exception {
+int threadNum = 5;
+AtomicLong successCount = new AtomicLong(0);
+
+// Test concurrent add peer operation
+Thread[] addPeers = new Thread[threadNum];
+for (int i = 0; i < threadNum; i++) {
+  addPeers[i] = new Thread(() -> {
+try {
+  hbaseAdmin.addReplicationPeer(ID_ONE,
+ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build());
+  successCount.incrementAndGet();
+} catch (Exception e) {
+  LOG.debug("Got exception when add replication peer", e);
+}
+  });
+  addPeers[i].start();
+}
+for (Thread addPeer : addPeers) {
+  addPeer.join();
+}
+assertEquals(1, successCount.get());
+
+// Test concurrent remove peer operation
+successCount.set(0);
+Thread[] removePeers = new Thread[threadNum];
+for (int i = 0; i < threadNum; i++) {
+  removePeers[i] = new Thread(() -> {
+try {
+  hbaseAdmin.removeReplicationPeer(ID_ONE);
+  successCount.incrementAndGet();
+} catch (Exception e) {
+  LOG.debug("Got exception when remove replication peer", e);
+}
+  });
+  removePeers[i].start();
+}
+for (Thread removePeer : removePeers) {
+  removePeer.join();
+}
+assertEquals(1, successCount.get());
+
+// Test concurrent add peer operation again
+successCount.set(0);
+addPeers = new Thread[threadNum];
+for (int i = 0; i < threadNum; i++) {
+  addPeers[i] = new Thread(() -> {
+try {
+  hbaseAdmin.addReplicationPeer(ID_ONE,
+ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build());
+  successCount.incrementAndGet();
+} catch (Exception e) {
+  LOG.debug("Got exception when add replication peer", e);
+}
+  });
+  addPeers[i].start();
+}
+for (Thread addPeer : addPeers) {
+  addPeer.join();
+}
+assertEquals(1, successCount.get());
+  }
+
+  @Test
   public void testAddInvalidPeer() {
 ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder();
 builder.setClusterKey(KEY_ONE);



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

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/5615dbb5
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/5615dbb5
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/5615dbb5

Branch: refs/heads/HBASE-19397
Commit: 5615dbb529bb46b889a1904d20359af44535e4f9
Parents: 4bf6439
Author: zhangduo 
Authored: Fri Dec 22 14:37:28 2017 +0800
Committer: zhangduo 
Committed: Fri Jan 5 13:54:37 2018 +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   |  62 ++-
 .../hbase/master/MockNoopMasterServices.java|  12 +-
 .../hbase/master/TestMasterNoCluster.java   |   4 +-
 .../TestReplicationDisableInactivePeer.java |   6 +-
 26 files changed, 1750 insertions(+), 312 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/5615dbb5/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  List nullToEmpty(List 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/5615dbb5/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 @@
   org.apache.hbase
   hbase-zookeeper
 
+
+  org.apache.hbase
+  hbase-common
+  test-jar
+  test
+
+
+  org.apache.hbase
+  hbase-zookeeper
+  test-jar
+  test
+
 
 
   org.apache.commons

http://git-wip-us.apache.org/repos/asf/hbase/blob/5615dbb5/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 000..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:/

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

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/4bf64391
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/4bf64391
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/4bf64391

Branch: refs/heads/HBASE-19397
Commit: 4bf64391c1f8c46dcbb4a285b722d975f17969c5
Parents: 8d8d29c
Author: huzheng 
Authored: Wed Dec 20 10:47:18 2017 +0800
Committer: zhangduo 
Committed: Fri Jan 5 13:54:37 2018 +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 ++-
 .../TestReplicationAdminUsingProcedure.java | 226 +++
 .../replication/DummyModifyPeerProcedure.java   |  48 
 .../TestDummyModifyPeerProcedure.java   |  80 ---
 .../TestReplicationTrackerZKImpl.java   |  51 -
 .../TestReplicationSourceManager.java   |  32 ++-
 21 files changed, 532 insertions(+), 471 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/4bf64391/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 d549607..2f2dc86 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 bytes.length 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/4bf64391/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 0706129..5b5d500 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;
@@ -279,7 +281,7 @@ public final class ProtobufUtil {
* byte array that is bytes.length plus {@link 
ProtobufMagic#PB_MAG

[05/32] hbase git commit: HBASE-19642 Fix locking for peer modification procedure

HBASE-19642 Fix locking for peer modification procedure


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

Branch: refs/heads/HBASE-19397
Commit: 2627dc5935c764f1e8dbd8ab8e425159bb263cf5
Parents: e27bf03
Author: zhangduo 
Authored: Wed Dec 27 18:27:13 2017 +0800
Committer: zhangduo 
Committed: Fri Jan 5 13:54:37 2018 +0800

--
 .../procedure/MasterProcedureScheduler.java | 14 +
 .../master/replication/ModifyPeerProcedure.java | 21 +---
 2 files changed, 32 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/2627dc59/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 a25217c..4ecb3b1 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
@@ -610,6 +610,20 @@ public class MasterProcedureScheduler extends 
AbstractProcedureScheduler {
 public boolean requireExclusiveLock(Procedure proc) {
   return requirePeerExclusiveLock((PeerProcedureInterface) proc);
 }
+
+@Override
+public boolean isAvailable() {
+  if (isEmpty()) {
+return false;
+  }
+  if (getLockStatus().hasExclusiveLock()) {
+// if we have an exclusive lock already taken
+// only child of the lock owner can be executed
+Procedure nextProc = peek();
+return nextProc != null && getLockStatus().hasLockAccess(nextProc);
+  }
+  return true;
+}
   }
 
   // 


http://git-wip-us.apache.org/repos/asf/hbase/blob/2627dc59/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 279fbc7..a682606 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
@@ -46,6 +46,8 @@ public abstract class ModifyPeerProcedure
 
   protected String peerId;
 
+  private volatile boolean locked;
+
   // used to keep compatible with old client where we can only returns after 
updateStorage.
   protected ProcedurePrepareLatch latch;
 
@@ -145,17 +147,30 @@ public abstract class ModifyPeerProcedure
 
   @Override
   protected LockState acquireLock(MasterProcedureEnv env) {
-return env.getProcedureScheduler().waitPeerExclusiveLock(this, peerId)
-  ? LockState.LOCK_EVENT_WAIT
-  : LockState.LOCK_ACQUIRED;
+if (env.getProcedureScheduler().waitPeerExclusiveLock(this, peerId)) {
+  return  LockState.LOCK_EVENT_WAIT;
+}
+locked = true;
+return LockState.LOCK_ACQUIRED;
   }
 
   @Override
   protected void releaseLock(MasterProcedureEnv env) {
+locked = false;
 env.getProcedureScheduler().wakePeerExclusiveLock(this, peerId);
   }
 
   @Override
+  protected boolean holdLock(MasterProcedureEnv env) {
+return true;
+  }
+
+  @Override
+  protected boolean hasLock(MasterProcedureEnv env) {
+return locked;
+  }
+
+  @Override
   protected void rollbackState(MasterProcedureEnv env, PeerModificationState 
state)
   throws IOException, InterruptedException {
 if (state == PeerModificationState.PRE_PEER_MODIFICATION) {



[16/32] hbase git commit: HBASE-19599 Remove ReplicationQueuesClient, use ReplicationQueueStorage directly

HBASE-19599 Remove ReplicationQueuesClient, use ReplicationQueueStorage directly


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

Branch: refs/heads/HBASE-19397
Commit: 710d5ad08c4227d5f0f2ae645fb00b31a3170c06
Parents: 5615dbb
Author: zhangduo 
Authored: Mon Dec 25 18:49:56 2017 +0800
Committer: zhangduo 
Committed: Fri Jan 5 13:54:37 2018 +0800

--
 .../hbase/replication/ReplicationFactory.java   |  19 +-
 .../replication/ReplicationPeersZKImpl.java |  24 +-
 .../replication/ReplicationQueueStorage.java|  26 +-
 .../replication/ReplicationQueuesClient.java|  93 -
 .../ReplicationQueuesClientArguments.java   |  40 --
 .../ReplicationQueuesClientZKImpl.java  | 176 -
 .../replication/ZKReplicationQueueStorage.java  |  90 -
 .../replication/TestReplicationStateBasic.java  | 378 +++
 .../replication/TestReplicationStateZKImpl.java | 148 
 .../TestZKReplicationQueueStorage.java  |  74 
 .../cleaner/ReplicationZKNodeCleaner.java   |  71 ++--
 .../cleaner/ReplicationZKNodeCleanerChore.java  |   5 +-
 .../replication/ReplicationPeerManager.java |  31 +-
 .../master/ReplicationHFileCleaner.java | 109 ++
 .../master/ReplicationLogCleaner.java   |  35 +-
 .../regionserver/DumpReplicationQueues.java |  78 ++--
 .../hbase/util/hbck/ReplicationChecker.java |  14 +-
 .../client/TestAsyncReplicationAdminApi.java|  31 +-
 .../replication/TestReplicationAdmin.java   |   2 +
 .../hbase/master/cleaner/TestLogsCleaner.java   |  30 +-
 .../cleaner/TestReplicationHFileCleaner.java|  59 +--
 .../cleaner/TestReplicationZKNodeCleaner.java   |  12 +-
 .../replication/TestReplicationStateBasic.java  | 378 ---
 .../replication/TestReplicationStateZKImpl.java | 227 ---
 .../TestReplicationSourceManagerZkImpl.java |  84 ++---
 25 files changed, 907 insertions(+), 1327 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/710d5ad0/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
--
diff --git 
a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
 
b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
index 9f4ad18..6c1c213 100644
--- 
a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
+++ 
b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.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
@@ -37,20 +36,14 @@ public class ReplicationFactory {
   args);
   }
 
-  public static ReplicationQueuesClient
-  getReplicationQueuesClient(ReplicationQueuesClientArguments args) throws 
Exception {
-return (ReplicationQueuesClient) ConstructorUtils
-.invokeConstructor(ReplicationQueuesClientZKImpl.class, args);
-  }
-
-  public static ReplicationPeers getReplicationPeers(final ZKWatcher zk, 
Configuration conf,
- Abortable abortable) {
+  public static ReplicationPeers getReplicationPeers(ZKWatcher zk, 
Configuration conf,
+  Abortable abortable) {
 return getReplicationPeers(zk, conf, null, abortable);
   }
 
-  public static ReplicationPeers getReplicationPeers(final ZKWatcher zk, 
Configuration conf,
- final 
ReplicationQueuesClient queuesClient, Abortable abortable) {
-return new ReplicationPeersZKImpl(zk, conf, queuesClient, abortable);
+  public static ReplicationPeers getReplicationPeers(ZKWatcher zk, 
Configuration conf,
+  ReplicationQueueStorage queueStorage, Abortable abortable) {
+return new ReplicationPeersZKImpl(zk, conf, queueStorage, abortable);
   }
 
   public static ReplicationTracker getReplicationTracker(ZKWatcher zookeeper,

http://git-wip-us.apache.org/repos/asf/hbase/blob/710d5ad0/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 8e2c5f4..f2e5647 100644
--- 
a/hbase-replication/src/main/java/org/apache/hadoop/hbase/repli

[28/32] hbase git commit: HBASE-19661 Replace ReplicationStateZKBase with ZKReplicationStorageBase

HBASE-19661 Replace ReplicationStateZKBase with ZKReplicationStorageBase


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

Branch: refs/heads/HBASE-19397
Commit: 5d1e3c858f389306d660d9f0fc86e433e7a6f26c
Parents: 1cfcc8b
Author: huzheng 
Authored: Fri Dec 29 15:55:28 2017 +0800
Committer: zhangduo 
Committed: Fri Jan 5 13:54:37 2018 +0800

--
 .../hbase/replication/ReplicationFactory.java   |   5 +-
 .../replication/ReplicationStateZKBase.java | 153 ---
 .../replication/ReplicationTrackerZKImpl.java   |  18 ++-
 .../replication/ZKReplicationPeerStorage.java   |  24 ++-
 .../replication/ZKReplicationStorageBase.java   |  13 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |   4 +-
 .../master/ReplicationPeerConfigUpgrader.java   | 128 
 .../regionserver/DumpReplicationQueues.java |  18 +--
 .../replication/regionserver/Replication.java   |   3 +-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java |   3 +-
 .../TestReplicationTrackerZKImpl.java   |   3 +-
 .../replication/master/TestTableCFsUpdater.java |  41 ++---
 .../TestReplicationSourceManager.java   |   6 +-
 13 files changed, 135 insertions(+), 284 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/5d1e3c85/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
--
diff --git 
a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
 
b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
index 6c66aff..2a970ba 100644
--- 
a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
+++ 
b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
@@ -33,9 +33,8 @@ public class ReplicationFactory {
 return new ReplicationPeers(zk, conf);
   }
 
-  public static ReplicationTracker getReplicationTracker(ZKWatcher zookeeper,
-  final ReplicationPeers replicationPeers, Configuration conf, Abortable 
abortable,
+  public static ReplicationTracker getReplicationTracker(ZKWatcher zookeeper, 
Abortable abortable,
   Stoppable stopper) {
-return new ReplicationTrackerZKImpl(zookeeper, replicationPeers, conf, 
abortable, stopper);
+return new ReplicationTrackerZKImpl(zookeeper, abortable, stopper);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5d1e3c85/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
deleted file mode 100644
index f49537c..000
--- 
a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
+++ /dev/null
@@ -1,153 +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.replication;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import 
org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.hbase.thirdparty.com.google.protobuf.CodedOutputStream;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
-import org.apache.hadoop.hbase.zookeeper.ZKConfig;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.yetus.audience.Inte

[21/32] hbase git commit: HBASE-19617 Remove ReplicationQueues, use ReplicationQueueStorage directly

HBASE-19617 Remove ReplicationQueues, use ReplicationQueueStorage directly


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

Branch: refs/heads/HBASE-19397
Commit: 0f4da157a764e3d36d9597bf74456e96c41a8b25
Parents: 2627dc5
Author: zhangduo 
Authored: Wed Dec 27 22:03:51 2017 +0800
Committer: zhangduo 
Committed: Fri Jan 5 13:54:37 2018 +0800

--
 .../hbase/replication/ReplicationFactory.java   |   9 +-
 .../hbase/replication/ReplicationQueues.java| 160 ---
 .../replication/ReplicationQueuesArguments.java |  70 ---
 .../replication/ReplicationQueuesZKImpl.java| 407 -
 .../hbase/replication/ReplicationTableBase.java | 442 ---
 .../replication/ReplicationTrackerZKImpl.java   |  21 +-
 .../replication/ZKReplicationQueueStorage.java  |  22 +
 .../replication/TestReplicationStateBasic.java  | 131 +++---
 .../replication/TestReplicationStateZKImpl.java |  41 +-
 .../regionserver/DumpReplicationQueues.java |  15 +-
 .../RecoveredReplicationSource.java |  17 +-
 .../RecoveredReplicationSourceShipper.java  |  22 +-
 .../replication/regionserver/Replication.java   |  41 +-
 .../regionserver/ReplicationSource.java |  23 +-
 .../ReplicationSourceInterface.java |  11 +-
 .../regionserver/ReplicationSourceManager.java  | 261 ++-
 .../regionserver/ReplicationSyncUp.java |  29 +-
 .../hbase/master/cleaner/TestLogsCleaner.java   |  12 +-
 .../cleaner/TestReplicationHFileCleaner.java|  26 +-
 .../cleaner/TestReplicationZKNodeCleaner.java   |  22 +-
 .../replication/ReplicationSourceDummy.java |   6 +-
 .../replication/TestReplicationSyncUpTool.java  |   6 +-
 .../TestReplicationSourceManager.java   | 104 ++---
 .../TestReplicationSourceManagerZkImpl.java |  58 +--
 24 files changed, 385 insertions(+), 1571 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/0f4da157/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
--
diff --git 
a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
 
b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
index 6c1c213..5e70e57 100644
--- 
a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
+++ 
b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
@@ -17,12 +17,11 @@
  */
 package org.apache.hadoop.hbase.replication;
 
-import org.apache.commons.lang3.reflect.ConstructorUtils;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * A factory class for instantiating replication objects that deal with 
replication state.
@@ -30,12 +29,6 @@ import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 @InterfaceAudience.Private
 public class ReplicationFactory {
 
-  public static ReplicationQueues 
getReplicationQueues(ReplicationQueuesArguments args)
-  throws Exception {
-return (ReplicationQueues) 
ConstructorUtils.invokeConstructor(ReplicationQueuesZKImpl.class,
-  args);
-  }
-
   public static ReplicationPeers getReplicationPeers(ZKWatcher zk, 
Configuration conf,
   Abortable abortable) {
 return getReplicationPeers(zk, conf, null, abortable);

http://git-wip-us.apache.org/repos/asf/hbase/blob/0f4da157/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
--
diff --git 
a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
 
b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
deleted file mode 100644
index 7f440b1..000
--- 
a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
+++ /dev/null
@@ -1,160 +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/l

  1   2   >