You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by sy...@apache.org on 2017/01/07 08:04:43 UTC

[01/16] hbase git commit: HBASE-17403 ClientAsyncPrefetchScanner doesn’t load any data if the MaxResultSize is too small (ChiaPing Tsai)

Repository: hbase
Updated Branches:
  refs/heads/hbase-12439 05ab41d1b -> 6fecf55a7


HBASE-17403 ClientAsyncPrefetchScanner doesn\u2019t load any data if the MaxResultSize is too small (ChiaPing Tsai)


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

Branch: refs/heads/hbase-12439
Commit: 63bd8be576b02981afda4cc491f6de89253e9ca9
Parents: 05ab41d
Author: tedyu <yu...@gmail.com>
Authored: Wed Jan 4 08:21:25 2017 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Wed Jan 4 08:21:25 2017 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/client/ClientAsyncPrefetchScanner.java |  6 +++---
 .../hbase/TestServerSideScanMetricsFromClientSide.java  | 12 +++++++++++-
 2 files changed, 14 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/63bd8be5/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java
index 6b70a88..ee323a9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java
@@ -186,7 +186,7 @@ public class ClientAsyncPrefetchScanner extends ClientScanner {
         capacity = DEFAULT_QUEUE_CAPACITY;
       }
     }
-    return capacity;
+    return Math.max(capacity, 1);
   }
 
   private boolean prefetchCondition() {
@@ -197,11 +197,11 @@ public class ClientAsyncPrefetchScanner extends ClientScanner {
   }
 
   private int getCountThreshold() {
-    return cacheCapacity / 2 ;
+    return Math.max(cacheCapacity / 2, 1);
   }
 
   private long getSizeThreshold() {
-    return maxScannerResultSize / 2 ;
+    return Math.max(maxScannerResultSize / 2, 1);
   }
 
   private long getCacheSizeInBytes() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/63bd8be5/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerSideScanMetricsFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerSideScanMetricsFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerSideScanMetricsFromClientSide.java
index 6f91515..b516cbb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerSideScanMetricsFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerSideScanMetricsFromClientSide.java
@@ -152,11 +152,21 @@ public class TestServerSideScanMetricsFromClientSide {
   }
 
   @Test
-  public void testRowsSeenMetric() throws Exception {
+  public void testRowsSeenMetricWithSync() throws Exception {
+    testRowsSeenMetric(false);
+  }
+
+  @Test
+  public void testRowsSeenMetricWithAsync() throws Exception {
+    testRowsSeenMetric(true);
+  }
+
+  private void testRowsSeenMetric(boolean async) throws Exception {
     // Base scan configuration
     Scan baseScan;
     baseScan = new Scan();
     baseScan.setScanMetricsEnabled(true);
+    baseScan.setAsyncPrefetch(async);
     testRowsSeenMetric(baseScan);
 
     // Test case that only a single result will be returned per RPC to the serer


[11/16] hbase git commit: HBASE-17290 Potential loss of data for replication of bulk loaded hfiles

Posted by sy...@apache.org.
HBASE-17290 Potential loss of data for replication of bulk loaded hfiles


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

Branch: refs/heads/hbase-12439
Commit: 5f631b9653a4bf86a2bebed58abed747c04b704f
Parents: 629b04f
Author: Ashish Singhi <as...@apache.org>
Authored: Fri Jan 6 16:15:49 2017 +0530
Committer: Ashish Singhi <as...@apache.org>
Committed: Fri Jan 6 16:18:20 2017 +0530

----------------------------------------------------------------------
 .../hbase/replication/ReplicationQueues.java    |  6 +-
 .../replication/ReplicationQueuesZKImpl.java    | 11 ++--
 .../TableBasedReplicationQueuesImpl.java        |  4 +-
 .../hbase/regionserver/HRegionServer.java       |  4 ++
 .../regionserver/HFileReplicator.java           |  2 +-
 .../replication/regionserver/Replication.java   | 55 +++++++----------
 .../regionserver/ReplicationObserver.java       | 62 ++++++++++++++++++++
 .../regionserver/ReplicationSource.java         | 11 ++--
 .../ReplicationSourceInterface.java             |  6 +-
 .../regionserver/ReplicationSourceManager.java  |  4 +-
 .../cleaner/TestReplicationHFileCleaner.java    |  9 +--
 .../replication/ReplicationSourceDummy.java     |  3 +-
 .../replication/TestReplicationStateBasic.java  | 33 ++++++-----
 13 files changed, 140 insertions(+), 70 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5f631b96/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
index 0ae27d0..be5a590 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.replication;
 import java.util.List;
 import java.util.SortedSet;
 
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.util.Pair;
 
@@ -144,10 +145,11 @@ public interface ReplicationQueues {
   /**
    * Add new hfile references to the queue.
    * @param peerId peer cluster id to which the hfiles need to be replicated
-   * @param files list of hfile references to be added
+   * @param pairs list of pairs of { HFile location in staging dir, HFile path in region dir which
+   *          will be added in the queue }
    * @throws ReplicationException if fails to add a hfile reference
    */
-  void addHFileRefs(String peerId, List<String> files) throws ReplicationException;
+  void addHFileRefs(String peerId, List<Pair<Path, Path>> pairs) throws ReplicationException;
 
   /**
    * Remove hfile references from the queue.

http://git-wip-us.apache.org/repos/asf/hbase/blob/5f631b96/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
index 7c548d9..1de1315 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
@@ -27,6 +27,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
@@ -319,16 +320,18 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
   }
 
   @Override
-  public void addHFileRefs(String peerId, List<String> files) throws ReplicationException {
+  public void addHFileRefs(String peerId, List<Pair<Path, Path>> pairs)
+      throws ReplicationException {
     String peerZnode = ZKUtil.joinZNode(this.hfileRefsZNode, peerId);
     boolean debugEnabled = LOG.isDebugEnabled();
     if (debugEnabled) {
-      LOG.debug("Adding hfile references " + files + " in queue " + peerZnode);
+      LOG.debug("Adding hfile references " + pairs + " in queue " + peerZnode);
     }
     List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>();
-    int size = files.size();
+    int size = pairs.size();
     for (int i = 0; i < size; i++) {
-      listOfOps.add(ZKUtilOp.createAndFailSilent(ZKUtil.joinZNode(peerZnode, files.get(i)),
+      listOfOps.add(ZKUtilOp.createAndFailSilent(
+        ZKUtil.joinZNode(peerZnode, pairs.get(i).getSecond().getName()),
         HConstants.EMPTY_BYTE_ARRAY));
     }
     if (debugEnabled) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/5f631b96/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java
index 28b9bdf..1023e0d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/TableBasedReplicationQueuesImpl.java
@@ -23,6 +23,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
@@ -307,7 +308,8 @@ public class TableBasedReplicationQueuesImpl extends ReplicationTableBase
   }
 
   @Override
-  public void addHFileRefs(String peerId, List<String> files) throws ReplicationException {
+  public void addHFileRefs(String peerId, List<Pair<Path, Path>> pairs)
+      throws ReplicationException {
     // TODO
     throw new NotImplementedException();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5f631b96/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 853d699..3c9d54f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -125,6 +125,7 @@ import org.apache.hadoop.hbase.regionserver.throttle.FlushThroughputControllerFa
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
+import org.apache.hadoop.hbase.replication.regionserver.Replication;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationLoad;
 import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.UserProvider;
@@ -524,6 +525,9 @@ public class HRegionServer extends HasThread implements
     checkCodecs(this.conf);
     this.userProvider = UserProvider.instantiate(conf);
     FSUtils.setupShortCircuitRead(this.conf);
+
+    Replication.decorateRegionServerConfiguration(this.conf);
+
     // Disable usage of meta replicas in the regionserver
     this.conf.setBoolean(HConstants.USE_META_REPLICAS, false);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/5f631b96/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HFileReplicator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HFileReplicator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HFileReplicator.java
index 256c24c..35aa1fb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HFileReplicator.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HFileReplicator.java
@@ -380,7 +380,7 @@ public class HFileReplicator {
           } catch (FileNotFoundException e1) {
             // This will mean that the hfile does not exists any where in source cluster FS. So we
             // cannot do anything here just log and continue.
-            LOG.error("Failed to copy hfile from " + sourceHFilePath + " to " + localHFilePath
+            LOG.debug("Failed to copy hfile from " + sourceHFilePath + " to " + localHFilePath
                 + ". Hence ignoring this hfile from replication..",
               e1);
             continue;

http://git-wip-us.apache.org/repos/asf/hbase/blob/5f631b96/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
index 5f87690..d3f9ba2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
@@ -23,7 +23,6 @@ import static org.apache.hadoop.hbase.HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.NavigableMap;
 import java.util.UUID;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
@@ -32,6 +31,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -44,8 +44,7 @@ import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.regionserver.ReplicationSinkService;
 import org.apache.hadoop.hbase.regionserver.ReplicationSourceService;
 import org.apache.hadoop.hbase.replication.ReplicationException;
@@ -236,34 +235,6 @@ public class Replication extends WALActionsListener.Base implements
     scopeWALEdits(logKey, logEdit, this.conf, this.getReplicationManager());
   }
 
-  @Override
-  public void postAppend(long entryLen, long elapsedTimeMillis, final WALKey logKey,
-      final WALEdit edit) throws IOException {
-    NavigableMap<byte[], Integer> scopes = logKey.getReplicationScopes();
-    if (this.replicationForBulkLoadData && scopes != null && !scopes.isEmpty()) {
-      TableName tableName = logKey.getTablename();
-      for (Cell c : edit.getCells()) {
-        // Only check for bulk load events
-        if (CellUtil.matchingQualifier(c, WALEdit.BULK_LOAD)) {
-          BulkLoadDescriptor bld = null;
-          try {
-            bld = WALEdit.getBulkLoadDescriptor(c);
-          } catch (IOException e) {
-            LOG.error("Failed to get bulk load events information from the wal file.", e);
-            throw e;
-          }
-
-          for (StoreDescriptor s : bld.getStoresList()) {
-            byte[] fam = s.getFamilyName().toByteArray();
-            if (scopes.containsKey(fam)) {
-              addHFileRefsToQueue(this.getReplicationManager(), tableName, fam, s);
-            }
-          }
-        }
-      }
-    }
-  }
-
   /**
    * Utility method used to set the correct scopes on each log key. Doesn't set a scope on keys from
    * compaction WAL edits and if the scope is local.
@@ -298,10 +269,10 @@ public class Replication extends WALActionsListener.Base implements
     }
   }
 
-  private static void addHFileRefsToQueue(ReplicationSourceManager replicationManager,
-      TableName tableName, byte[] family, StoreDescriptor s) throws IOException {
+  void addHFileRefsToQueue(TableName tableName, byte[] family, List<Pair<Path, Path>> pairs)
+      throws IOException {
     try {
-      replicationManager.addHFileRefs(tableName, family, s.getStoreFileList());
+      this.replicationManager.addHFileRefs(tableName, family, pairs);
     } catch (ReplicationException e) {
       LOG.error("Failed to add hfile references in the replication queue.", e);
       throw new IOException(e);
@@ -337,6 +308,22 @@ public class Replication extends WALActionsListener.Base implements
     }
   }
 
+  /**
+   * This method modifies the region server's configuration in order to inject replication-related
+   * features
+   * @param conf region server configurations
+   */
+  public static void decorateRegionServerConfiguration(Configuration conf) {
+    if (isReplicationForBulkLoadDataEnabled(conf)) {
+      String plugins = conf.get(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY, "");
+      String rsCoprocessorClass = ReplicationObserver.class.getCanonicalName();
+      if (!plugins.contains(rsCoprocessorClass)) {
+        conf.set(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY,
+          plugins + "," + rsCoprocessorClass);
+      }
+    }
+  }
+
   /*
    * Statistics thread. Periodically prints the cache statistics to the log.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/5f631b96/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationObserver.java
new file mode 100644
index 0000000..03046b4
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationObserver.java
@@ -0,0 +1,62 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.replication.regionserver;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.util.Pair;
+
+/**
+ * An Observer to facilitate replication operations
+ */
+
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
+public class ReplicationObserver extends BaseRegionObserver {
+  private static final Log LOG = LogFactory.getLog(ReplicationObserver.class);
+
+  @Override
+  public void preCommitStoreFile(final ObserverContext<RegionCoprocessorEnvironment> ctx,
+      final byte[] family, final List<Pair<Path, Path>> pairs) throws IOException {
+    RegionCoprocessorEnvironment env = ctx.getEnvironment();
+    Configuration c = env.getConfiguration();
+    if (pairs == null || pairs.isEmpty()
+        || !c.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
+          HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT)) {
+      LOG.debug("Skipping recording bulk load entries in preCommitStoreFile for bulkloaded "
+          + "data replication.");
+      return;
+    }
+    HRegionServer rs = (HRegionServer) env.getRegionServerServices();
+    Replication rep = (Replication) rs.getReplicationSourceService();
+    rep.addHFileRefsToQueue(env.getRegionInfo().getTable(), family, pairs);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/5f631b96/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
index 3eeb4b8..7a229eb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -73,6 +73,7 @@ import org.apache.hadoop.hbase.util.CancelableProgressable;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.LeaseNotRecoveredException;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.hadoop.hbase.wal.WAL;
@@ -253,7 +254,7 @@ public class ReplicationSource extends Thread
   }
 
   @Override
-  public void addHFileRefs(TableName tableName, byte[] family, List<String> files)
+  public void addHFileRefs(TableName tableName, byte[] family, List<Pair<Path, Path>> pairs)
       throws ReplicationException {
     String peerId = peerClusterZnode;
     if (peerId.contains("-")) {
@@ -266,8 +267,8 @@ public class ReplicationSource extends Thread
       List<String> tableCfs = tableCFMap.get(tableName);
       if (tableCFMap.containsKey(tableName)
           && (tableCfs == null || tableCfs.contains(Bytes.toString(family)))) {
-        this.replicationQueues.addHFileRefs(peerId, files);
-        metrics.incrSizeOfHFileRefsQueue(files.size());
+        this.replicationQueues.addHFileRefs(peerId, pairs);
+        metrics.incrSizeOfHFileRefsQueue(pairs.size());
       } else {
         LOG.debug("HFiles will not be replicated belonging to the table " + tableName + " family "
             + Bytes.toString(family) + " to peer id " + peerId);
@@ -275,8 +276,8 @@ public class ReplicationSource extends Thread
     } else {
       // user has explicitly not defined any table cfs for replication, means replicate all the
       // data
-      this.replicationQueues.addHFileRefs(peerId, files);
-      metrics.incrSizeOfHFileRefsQueue(files.size());
+      this.replicationQueues.addHFileRefs(peerId, pairs);
+      metrics.incrSizeOfHFileRefsQueue(pairs.size());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/5f631b96/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 7f4a9f7..8d5451c 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
@@ -32,6 +32,7 @@ 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.util.Pair;
 
 /**
  * Interface that defines a replication source
@@ -112,10 +113,11 @@ public interface ReplicationSourceInterface {
    * Add hfile names to the queue to be replicated.
    * @param tableName Name of the table these files belongs to
    * @param family Name of the family these files belong to
-   * @param files files whose names needs to be added to the queue to be replicated
+   * @param pairs list of pairs of { HFile location in staging dir, HFile path in region dir which
+   *          will be added in the queue for replication}
    * @throws ReplicationException If failed to add hfile references
    */
-  void addHFileRefs(TableName tableName, byte[] family, List<String> files)
+  void addHFileRefs(TableName tableName, byte[] family, List<Pair<Path, Path>> pairs)
       throws ReplicationException;
 
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5f631b96/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 ef4093e..5b574da 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
@@ -846,10 +846,10 @@ public class ReplicationSourceManager implements ReplicationListener {
     return stats.toString();
   }
 
-  public void addHFileRefs(TableName tableName, byte[] family, List<String> files)
+  public void addHFileRefs(TableName tableName, byte[] family, List<Pair<Path, Path>> pairs)
       throws ReplicationException {
     for (ReplicationSourceInterface source : this.sources) {
-      source.addHFileRefs(tableName, family, files);
+      source.addHFileRefs(tableName, family, pairs);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/5f631b96/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
index fc3e516..817cfb4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner;
 import org.apache.hadoop.hbase.replication.regionserver.Replication;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@@ -138,8 +139,8 @@ public class TestReplicationHFileCleaner {
         + "for it in the queue.",
       cleaner.isFileDeletable(fs.getFileStatus(file)));
 
-    List<String> files = new ArrayList<String>(1);
-    files.add(file.getName());
+    List<Pair<Path, Path>> files = new ArrayList<>(1);
+    files.add(new Pair<Path, Path>(null, file));
     // 4. Add the file to hfile-refs queue
     rq.addHFileRefs(peerId, files);
     // 5. Assert file should not be deletable
@@ -166,8 +167,8 @@ public class TestReplicationHFileCleaner {
     f.setPath(notDeletablefile);
     files.add(f);
 
-    List<String> hfiles = new ArrayList<>(1);
-    hfiles.add(notDeletablefile.getName());
+    List<Pair<Path, Path>> hfiles = new ArrayList<>(1);
+    hfiles.add(new Pair<Path, Path>(null, notDeletablefile));
     // 2. Add one file to hfile-refs queue
     rq.addHFileRefs(peerId, hfiles);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/5f631b96/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
index abe484e..57e54d7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.replication.regionserver.MetricsSource;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceInterface;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager;
+import org.apache.hadoop.hbase.util.Pair;
 
 /**
  * Source that does nothing at all, helpful to test ReplicationSourceManager
@@ -93,7 +94,7 @@ public class ReplicationSourceDummy implements ReplicationSourceInterface {
   }
 
   @Override
-  public void addHFileRefs(TableName tableName, byte[] family, List<String> files)
+  public void addHFileRefs(TableName tableName, byte[] family, List<Pair<Path, Path>> files)
       throws ReplicationException {
     return;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5f631b96/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
index fcab105..f8be9a7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
@@ -25,7 +25,9 @@ import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 import org.apache.zookeeper.KeeperException;
 import org.junit.Before;
@@ -202,10 +204,10 @@ public abstract class TestReplicationStateBasic {
     rq1.init(server1);
     rqc.init();
 
-    List<String> files1 = new ArrayList<String>(3);
-    files1.add("file_1");
-    files1.add("file_2");
-    files1.add("file_3");
+    List<Pair<Path, Path>> files1 = new ArrayList<>(3);
+    files1.add(new Pair<Path, Path>(null, new Path("file_1")));
+    files1.add(new Pair<Path, Path>(null, new Path("file_2")));
+    files1.add(new Pair<Path, Path>(null, new Path("file_3")));
     assertNull(rqc.getReplicableHFiles(ID_ONE));
     assertEquals(0, rqc.getAllPeersFromHFileRefsQueue().size());
     rp.registerPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
@@ -213,13 +215,16 @@ public abstract class TestReplicationStateBasic {
     rq1.addHFileRefs(ID_ONE, files1);
     assertEquals(1, rqc.getAllPeersFromHFileRefsQueue().size());
     assertEquals(3, rqc.getReplicableHFiles(ID_ONE).size());
-    List<String> files2 = new ArrayList<>(files1);
-    String removedString = files2.remove(0);
-    rq1.removeHFileRefs(ID_ONE, files2);
+    List<String> hfiles2 = new ArrayList<>();
+    for (Pair<Path, Path> p : files1) {
+      hfiles2.add(p.getSecond().getName());
+    }
+    String removedString = hfiles2.remove(0);
+    rq1.removeHFileRefs(ID_ONE, hfiles2);
     assertEquals(1, rqc.getReplicableHFiles(ID_ONE).size());
-    files2 = new ArrayList<>(1);
-    files2.add(removedString);
-    rq1.removeHFileRefs(ID_ONE, files2);
+    hfiles2 = new ArrayList<>(1);
+    hfiles2.add(removedString);
+    rq1.removeHFileRefs(ID_ONE, hfiles2);
     assertEquals(0, rqc.getReplicableHFiles(ID_ONE).size());
     rp.unregisterPeer(ID_ONE);
   }
@@ -235,10 +240,10 @@ public abstract class TestReplicationStateBasic {
     rp.registerPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO));
     rq1.addPeerToHFileRefs(ID_TWO);
 
-    List<String> files1 = new ArrayList<String>(3);
-    files1.add("file_1");
-    files1.add("file_2");
-    files1.add("file_3");
+    List<Pair<Path, Path>> files1 = new ArrayList<>(3);
+    files1.add(new Pair<Path, Path>(null, new Path("file_1")));
+    files1.add(new Pair<Path, Path>(null, new Path("file_2")));
+    files1.add(new Pair<Path, Path>(null, new Path("file_3")));
     rq1.addHFileRefs(ID_ONE, files1);
     rq1.addHFileRefs(ID_TWO, files1);
     assertEquals(2, rqc.getAllPeersFromHFileRefsQueue().size());


[03/16] hbase git commit: HBASE-17390 Added master and backup masters to online update of configuration

Posted by sy...@apache.org.
HBASE-17390 Added master and backup masters to online update of configuration

Signed-off-by: Jerry He <je...@apache.org>


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

Branch: refs/heads/hbase-12439
Commit: dba103e1b6e27330a960513c65af49254d56078b
Parents: 20a7ae2
Author: Jan Hentschel <ja...@ultratendency.com>
Authored: Thu Dec 29 18:55:22 2016 +0100
Committer: Jerry He <je...@apache.org>
Committed: Wed Jan 4 22:08:05 2017 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |  6 +++
 .../hbase/client/TestUpdateConfiguration.java   | 40 +++++++++++++++++++-
 2 files changed, 45 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/dba103e1/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index 89d1b49..3c84929 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -3033,6 +3033,12 @@ public class HBaseAdmin implements Admin {
     for (ServerName server : this.getClusterStatus().getServers()) {
       updateConfiguration(server);
     }
+
+    updateConfiguration(this.getClusterStatus().getMaster());
+
+    for (ServerName server : this.getClusterStatus().getBackupMasters()) {
+      updateConfiguration(server);
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/dba103e1/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestUpdateConfiguration.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestUpdateConfiguration.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestUpdateConfiguration.java
index 73e493b..731e02f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestUpdateConfiguration.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestUpdateConfiguration.java
@@ -43,7 +43,7 @@ public class TestUpdateConfiguration {
   
   @BeforeClass
   public static void setup() throws Exception {
-    TEST_UTIL.startMiniCluster();
+    TEST_UTIL.startMiniCluster(2, 1);
   }
 
   @Test
@@ -74,4 +74,42 @@ public class TestUpdateConfiguration {
     // restore hbase-site.xml
     Files.copy(cnf3Path, cnfPath, StandardCopyOption.REPLACE_EXISTING);
   }
+
+  @Test
+  public void testAllOnlineConfigChange() throws IOException {
+    LOG.debug("Starting the test");
+    Admin admin = TEST_UTIL.getAdmin();
+    admin.updateConfiguration();
+  }
+
+  @Test
+  public void testAllCustomOnlineConfigChange() throws IOException {
+    LOG.debug("Starting the test");
+    Path cnfPath = FileSystems.getDefault().getPath("target/test-classes/hbase-site.xml");
+    Path cnf2Path = FileSystems.getDefault().getPath("target/test-classes/hbase-site2.xml");
+    Path cnf3Path = FileSystems.getDefault().getPath("target/test-classes/hbase-site3.xml");
+    // make a backup of hbase-site.xml
+    Files.copy(cnfPath, cnf3Path, StandardCopyOption.REPLACE_EXISTING);
+    // update hbase-site.xml by overwriting it
+    Files.copy(cnf2Path, cnfPath, StandardCopyOption.REPLACE_EXISTING);
+
+    Admin admin = TEST_UTIL.getAdmin();
+    admin.updateConfiguration();
+
+    // Check the configuration of the Masters
+    Configuration masterConfiguration = TEST_UTIL.getMiniHBaseCluster().getMaster(0).getConfiguration();
+    int custom = masterConfiguration.getInt("hbase.custom.config", 0);
+    assertEquals(custom, 1000);
+    Configuration backupMasterConfiguration = TEST_UTIL.getMiniHBaseCluster().getMaster(1).getConfiguration();
+    custom = backupMasterConfiguration.getInt("hbase.custom.config", 0);
+    assertEquals(custom, 1000);
+
+    // Check the configuration of the RegionServer
+    Configuration regionServerConfiguration = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0).getConfiguration();
+    custom = regionServerConfiguration.getInt("hbase.custom.config", 0);
+    assertEquals(custom, 1000);
+
+    // restore hbase-site.xml
+    Files.copy(cnf3Path, cnfPath, StandardCopyOption.REPLACE_EXISTING);
+  }
 }


[14/16] hbase git commit: HBASE-17431 Incorrect precheck condition in RoundRobinPool#get() - revert due to test failure

Posted by sy...@apache.org.
HBASE-17431 Incorrect precheck condition in RoundRobinPool#get() - revert due to test failure


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

Branch: refs/heads/hbase-12439
Commit: 4c98f97c318995593bbe8b67ba856003cb5ed463
Parents: 6d48eb0
Author: tedyu <yu...@gmail.com>
Authored: Fri Jan 6 12:12:49 2017 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Fri Jan 6 12:12:49 2017 -0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/4c98f97c/hbase-client/src/main/java/org/apache/hadoop/hbase/util/PoolMap.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/util/PoolMap.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/util/PoolMap.java
index f89215b..b683fcc 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/util/PoolMap.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/util/PoolMap.java
@@ -360,7 +360,7 @@ public class PoolMap<K, V> implements Map<K, V> {
 
     @Override
     public R get() {
-      if (super.size() <= 0) {
+      if (super.size() < maxSize) {
         return null;
       }
       nextResource %= super.size();


[07/16] hbase git commit: HBASE-17388 Move ReplicationPeer and other replication related PB messages to the replication.proto

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/e02ae772/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ZooKeeperProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ZooKeeperProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ZooKeeperProtos.java
index 90ec659..de5827e 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ZooKeeperProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ZooKeeperProtos.java
@@ -4120,3978 +4120,6 @@ public final class ZooKeeperProtos {
 
   }
 
-  public interface TableCFOrBuilder extends
-      // @@protoc_insertion_point(interface_extends:hbase.pb.TableCF)
-      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
-
-    /**
-     * <code>optional .hbase.pb.TableName table_name = 1;</code>
-     */
-    boolean hasTableName();
-    /**
-     * <code>optional .hbase.pb.TableName table_name = 1;</code>
-     */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName();
-    /**
-     * <code>optional .hbase.pb.TableName table_name = 1;</code>
-     */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder();
-
-    /**
-     * <code>repeated bytes families = 2;</code>
-     */
-    java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString> getFamiliesList();
-    /**
-     * <code>repeated bytes families = 2;</code>
-     */
-    int getFamiliesCount();
-    /**
-     * <code>repeated bytes families = 2;</code>
-     */
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilies(int index);
-  }
-  /**
-   * Protobuf type {@code hbase.pb.TableCF}
-   */
-  public  static final class TableCF extends
-      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
-      // @@protoc_insertion_point(message_implements:hbase.pb.TableCF)
-      TableCFOrBuilder {
-    // Use TableCF.newBuilder() to construct.
-    private TableCF(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
-      super(builder);
-    }
-    private TableCF() {
-      families_ = java.util.Collections.emptyList();
-    }
-
-    @java.lang.Override
-    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
-    getUnknownFields() {
-      return this.unknownFields;
-    }
-    private TableCF(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
-      this();
-      int mutable_bitField0_ = 0;
-      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
-      try {
-        boolean done = false;
-        while (!done) {
-          int tag = input.readTag();
-          switch (tag) {
-            case 0:
-              done = true;
-              break;
-            default: {
-              if (!parseUnknownField(input, unknownFields,
-                                     extensionRegistry, tag)) {
-                done = true;
-              }
-              break;
-            }
-            case 10: {
-              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null;
-              if (((bitField0_ & 0x00000001) == 0x00000001)) {
-                subBuilder = tableName_.toBuilder();
-              }
-              tableName_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry);
-              if (subBuilder != null) {
-                subBuilder.mergeFrom(tableName_);
-                tableName_ = subBuilder.buildPartial();
-              }
-              bitField0_ |= 0x00000001;
-              break;
-            }
-            case 18: {
-              if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
-                families_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString>();
-                mutable_bitField0_ |= 0x00000002;
-              }
-              families_.add(input.readBytes());
-              break;
-            }
-          }
-        }
-      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
-        throw e.setUnfinishedMessage(this);
-      } catch (java.io.IOException e) {
-        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
-            e).setUnfinishedMessage(this);
-      } finally {
-        if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
-          families_ = java.util.Collections.unmodifiableList(families_);
-        }
-        this.unknownFields = unknownFields.build();
-        makeExtensionsImmutable();
-      }
-    }
-    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-        getDescriptor() {
-      return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCF_descriptor;
-    }
-
-    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-        internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCF_fieldAccessorTable
-          .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF.class, org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF.Builder.class);
-    }
-
-    private int bitField0_;
-    public static final int TABLE_NAME_FIELD_NUMBER = 1;
-    private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_;
-    /**
-     * <code>optional .hbase.pb.TableName table_name = 1;</code>
-     */
-    public boolean hasTableName() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
-    }
-    /**
-     * <code>optional .hbase.pb.TableName table_name = 1;</code>
-     */
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName() {
-      return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
-    }
-    /**
-     * <code>optional .hbase.pb.TableName table_name = 1;</code>
-     */
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
-      return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
-    }
-
-    public static final int FAMILIES_FIELD_NUMBER = 2;
-    private java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString> families_;
-    /**
-     * <code>repeated bytes families = 2;</code>
-     */
-    public java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString>
-        getFamiliesList() {
-      return families_;
-    }
-    /**
-     * <code>repeated bytes families = 2;</code>
-     */
-    public int getFamiliesCount() {
-      return families_.size();
-    }
-    /**
-     * <code>repeated bytes families = 2;</code>
-     */
-    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilies(int index) {
-      return families_.get(index);
-    }
-
-    private byte memoizedIsInitialized = -1;
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized == 1) return true;
-      if (isInitialized == 0) return false;
-
-      if (hasTableName()) {
-        if (!getTableName().isInitialized()) {
-          memoizedIsInitialized = 0;
-          return false;
-        }
-      }
-      memoizedIsInitialized = 1;
-      return true;
-    }
-
-    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
-                        throws java.io.IOException {
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeMessage(1, getTableName());
-      }
-      for (int i = 0; i < families_.size(); i++) {
-        output.writeBytes(2, families_.get(i));
-      }
-      unknownFields.writeTo(output);
-    }
-
-    public int getSerializedSize() {
-      int size = memoizedSize;
-      if (size != -1) return size;
-
-      size = 0;
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeMessageSize(1, getTableName());
-      }
-      {
-        int dataSize = 0;
-        for (int i = 0; i < families_.size(); i++) {
-          dataSize += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-            .computeBytesSizeNoTag(families_.get(i));
-        }
-        size += dataSize;
-        size += 1 * getFamiliesList().size();
-      }
-      size += unknownFields.getSerializedSize();
-      memoizedSize = size;
-      return size;
-    }
-
-    private static final long serialVersionUID = 0L;
-    @java.lang.Override
-    public boolean equals(final java.lang.Object obj) {
-      if (obj == this) {
-       return true;
-      }
-      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF)) {
-        return super.equals(obj);
-      }
-      org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF other = (org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF) obj;
-
-      boolean result = true;
-      result = result && (hasTableName() == other.hasTableName());
-      if (hasTableName()) {
-        result = result && getTableName()
-            .equals(other.getTableName());
-      }
-      result = result && getFamiliesList()
-          .equals(other.getFamiliesList());
-      result = result && unknownFields.equals(other.unknownFields);
-      return result;
-    }
-
-    @java.lang.Override
-    public int hashCode() {
-      if (memoizedHashCode != 0) {
-        return memoizedHashCode;
-      }
-      int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasTableName()) {
-        hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
-        hash = (53 * hash) + getTableName().hashCode();
-      }
-      if (getFamiliesCount() > 0) {
-        hash = (37 * hash) + FAMILIES_FIELD_NUMBER;
-        hash = (53 * hash) + getFamiliesList().hashCode();
-      }
-      hash = (29 * hash) + unknownFields.hashCode();
-      memoizedHashCode = hash;
-      return hash;
-    }
-
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
-        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(byte[] data)
-        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(
-        byte[] data,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-          .parseWithIOException(PARSER, input);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(
-        java.io.InputStream input,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-          .parseWithIOException(PARSER, input, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF parseDelimitedFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-          .parseDelimitedWithIOException(PARSER, input);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF parseDelimitedFrom(
-        java.io.InputStream input,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
-        throws java.io.IOException {
-      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-          .parseWithIOException(PARSER, input);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-          .parseWithIOException(PARSER, input, extensionRegistry);
-    }
-
-    public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder() {
-      return DEFAULT_INSTANCE.toBuilder();
-    }
-    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF prototype) {
-      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
-    }
-    public Builder toBuilder() {
-      return this == DEFAULT_INSTANCE
-          ? new Builder() : new Builder().mergeFrom(this);
-    }
-
-    @java.lang.Override
-    protected Builder newBuilderForType(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
-      Builder builder = new Builder(parent);
-      return builder;
-    }
-    /**
-     * Protobuf type {@code hbase.pb.TableCF}
-     */
-    public static final class Builder extends
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
-        // @@protoc_insertion_point(builder_implements:hbase.pb.TableCF)
-        org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder {
-      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-          getDescriptor() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCF_descriptor;
-      }
-
-      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-          internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCF_fieldAccessorTable
-            .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF.class, org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF.Builder.class);
-      }
-
-      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF.newBuilder()
-      private Builder() {
-        maybeForceBuilderInitialization();
-      }
-
-      private Builder(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
-        super(parent);
-        maybeForceBuilderInitialization();
-      }
-      private void maybeForceBuilderInitialization() {
-        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-                .alwaysUseFieldBuilders) {
-          getTableNameFieldBuilder();
-        }
-      }
-      public Builder clear() {
-        super.clear();
-        if (tableNameBuilder_ == null) {
-          tableName_ = null;
-        } else {
-          tableNameBuilder_.clear();
-        }
-        bitField0_ = (bitField0_ & ~0x00000001);
-        families_ = java.util.Collections.emptyList();
-        bitField0_ = (bitField0_ & ~0x00000002);
-        return this;
-      }
-
-      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-          getDescriptorForType() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCF_descriptor;
-      }
-
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF.getDefaultInstance();
-      }
-
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF build() {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF result = buildPartial();
-        if (!result.isInitialized()) {
-          throw newUninitializedMessageException(result);
-        }
-        return result;
-      }
-
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF buildPartial() {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF result = new org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF(this);
-        int from_bitField0_ = bitField0_;
-        int to_bitField0_ = 0;
-        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-          to_bitField0_ |= 0x00000001;
-        }
-        if (tableNameBuilder_ == null) {
-          result.tableName_ = tableName_;
-        } else {
-          result.tableName_ = tableNameBuilder_.build();
-        }
-        if (((bitField0_ & 0x00000002) == 0x00000002)) {
-          families_ = java.util.Collections.unmodifiableList(families_);
-          bitField0_ = (bitField0_ & ~0x00000002);
-        }
-        result.families_ = families_;
-        result.bitField0_ = to_bitField0_;
-        onBuilt();
-        return result;
-      }
-
-      public Builder clone() {
-        return (Builder) super.clone();
-      }
-      public Builder setField(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
-          Object value) {
-        return (Builder) super.setField(field, value);
-      }
-      public Builder clearField(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
-        return (Builder) super.clearField(field);
-      }
-      public Builder clearOneof(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
-        return (Builder) super.clearOneof(oneof);
-      }
-      public Builder setRepeatedField(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
-          int index, Object value) {
-        return (Builder) super.setRepeatedField(field, index, value);
-      }
-      public Builder addRepeatedField(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
-          Object value) {
-        return (Builder) super.addRepeatedField(field, value);
-      }
-      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
-        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF) {
-          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF)other);
-        } else {
-          super.mergeFrom(other);
-          return this;
-        }
-      }
-
-      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF other) {
-        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF.getDefaultInstance()) return this;
-        if (other.hasTableName()) {
-          mergeTableName(other.getTableName());
-        }
-        if (!other.families_.isEmpty()) {
-          if (families_.isEmpty()) {
-            families_ = other.families_;
-            bitField0_ = (bitField0_ & ~0x00000002);
-          } else {
-            ensureFamiliesIsMutable();
-            families_.addAll(other.families_);
-          }
-          onChanged();
-        }
-        this.mergeUnknownFields(other.unknownFields);
-        onChanged();
-        return this;
-      }
-
-      public final boolean isInitialized() {
-        if (hasTableName()) {
-          if (!getTableName().isInitialized()) {
-            return false;
-          }
-        }
-        return true;
-      }
-
-      public Builder mergeFrom(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws java.io.IOException {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF parsedMessage = null;
-        try {
-          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF) e.getUnfinishedMessage();
-          throw e.unwrapIOException();
-        } finally {
-          if (parsedMessage != null) {
-            mergeFrom(parsedMessage);
-          }
-        }
-        return this;
-      }
-      private int bitField0_;
-
-      private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
-      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
-      /**
-       * <code>optional .hbase.pb.TableName table_name = 1;</code>
-       */
-      public boolean hasTableName() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
-      }
-      /**
-       * <code>optional .hbase.pb.TableName table_name = 1;</code>
-       */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName() {
-        if (tableNameBuilder_ == null) {
-          return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
-        } else {
-          return tableNameBuilder_.getMessage();
-        }
-      }
-      /**
-       * <code>optional .hbase.pb.TableName table_name = 1;</code>
-       */
-      public Builder setTableName(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName value) {
-        if (tableNameBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          tableName_ = value;
-          onChanged();
-        } else {
-          tableNameBuilder_.setMessage(value);
-        }
-        bitField0_ |= 0x00000001;
-        return this;
-      }
-      /**
-       * <code>optional .hbase.pb.TableName table_name = 1;</code>
-       */
-      public Builder setTableName(
-          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
-        if (tableNameBuilder_ == null) {
-          tableName_ = builderForValue.build();
-          onChanged();
-        } else {
-          tableNameBuilder_.setMessage(builderForValue.build());
-        }
-        bitField0_ |= 0x00000001;
-        return this;
-      }
-      /**
-       * <code>optional .hbase.pb.TableName table_name = 1;</code>
-       */
-      public Builder mergeTableName(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName value) {
-        if (tableNameBuilder_ == null) {
-          if (((bitField0_ & 0x00000001) == 0x00000001) &&
-              tableName_ != null &&
-              tableName_ != org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()) {
-            tableName_ =
-              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.newBuilder(tableName_).mergeFrom(value).buildPartial();
-          } else {
-            tableName_ = value;
-          }
-          onChanged();
-        } else {
-          tableNameBuilder_.mergeFrom(value);
-        }
-        bitField0_ |= 0x00000001;
-        return this;
-      }
-      /**
-       * <code>optional .hbase.pb.TableName table_name = 1;</code>
-       */
-      public Builder clearTableName() {
-        if (tableNameBuilder_ == null) {
-          tableName_ = null;
-          onChanged();
-        } else {
-          tableNameBuilder_.clear();
-        }
-        bitField0_ = (bitField0_ & ~0x00000001);
-        return this;
-      }
-      /**
-       * <code>optional .hbase.pb.TableName table_name = 1;</code>
-       */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder getTableNameBuilder() {
-        bitField0_ |= 0x00000001;
-        onChanged();
-        return getTableNameFieldBuilder().getBuilder();
-      }
-      /**
-       * <code>optional .hbase.pb.TableName table_name = 1;</code>
-       */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
-        if (tableNameBuilder_ != null) {
-          return tableNameBuilder_.getMessageOrBuilder();
-        } else {
-          return tableName_ == null ?
-              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
-        }
-      }
-      /**
-       * <code>optional .hbase.pb.TableName table_name = 1;</code>
-       */
-      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
-          getTableNameFieldBuilder() {
-        if (tableNameBuilder_ == null) {
-          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
-              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
-                  getTableName(),
-                  getParentForChildren(),
-                  isClean());
-          tableName_ = null;
-        }
-        return tableNameBuilder_;
-      }
-
-      private java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString> families_ = java.util.Collections.emptyList();
-      private void ensureFamiliesIsMutable() {
-        if (!((bitField0_ & 0x00000002) == 0x00000002)) {
-          families_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString>(families_);
-          bitField0_ |= 0x00000002;
-         }
-      }
-      /**
-       * <code>repeated bytes families = 2;</code>
-       */
-      public java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString>
-          getFamiliesList() {
-        return java.util.Collections.unmodifiableList(families_);
-      }
-      /**
-       * <code>repeated bytes families = 2;</code>
-       */
-      public int getFamiliesCount() {
-        return families_.size();
-      }
-      /**
-       * <code>repeated bytes families = 2;</code>
-       */
-      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilies(int index) {
-        return families_.get(index);
-      }
-      /**
-       * <code>repeated bytes families = 2;</code>
-       */
-      public Builder setFamilies(
-          int index, org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  ensureFamiliesIsMutable();
-        families_.set(index, value);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated bytes families = 2;</code>
-       */
-      public Builder addFamilies(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  ensureFamiliesIsMutable();
-        families_.add(value);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated bytes families = 2;</code>
-       */
-      public Builder addAllFamilies(
-          java.lang.Iterable<? extends org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString> values) {
-        ensureFamiliesIsMutable();
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
-            values, families_);
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>repeated bytes families = 2;</code>
-       */
-      public Builder clearFamilies() {
-        families_ = java.util.Collections.emptyList();
-        bitField0_ = (bitField0_ & ~0x00000002);
-        onChanged();
-        return this;
-      }
-      public final Builder setUnknownFields(
-          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
-        return super.setUnknownFields(unknownFields);
-      }
-
-      public final Builder mergeUnknownFields(
-          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
-        return super.mergeUnknownFields(unknownFields);
-      }
-
-
-      // @@protoc_insertion_point(builder_scope:hbase.pb.TableCF)
-    }
-
-    // @@protoc_insertion_point(class_scope:hbase.pb.TableCF)
-    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF DEFAULT_INSTANCE;
-    static {
-      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF();
-    }
-
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF getDefaultInstance() {
-      return DEFAULT_INSTANCE;
-    }
-
-    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<TableCF>
-        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<TableCF>() {
-      public TableCF parsePartialFrom(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
-          return new TableCF(input, extensionRegistry);
-      }
-    };
-
-    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<TableCF> parser() {
-      return PARSER;
-    }
-
-    @java.lang.Override
-    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<TableCF> getParserForType() {
-      return PARSER;
-    }
-
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF getDefaultInstanceForType() {
-      return DEFAULT_INSTANCE;
-    }
-
-  }
-
-  public interface ReplicationPeerOrBuilder extends
-      // @@protoc_insertion_point(interface_extends:hbase.pb.ReplicationPeer)
-      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
-
-    /**
-     * <pre>
-     * clusterkey is the concatenation of the slave cluster's
-     * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
-     * </pre>
-     *
-     * <code>required string clusterkey = 1;</code>
-     */
-    boolean hasClusterkey();
-    /**
-     * <pre>
-     * clusterkey is the concatenation of the slave cluster's
-     * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
-     * </pre>
-     *
-     * <code>required string clusterkey = 1;</code>
-     */
-    java.lang.String getClusterkey();
-    /**
-     * <pre>
-     * clusterkey is the concatenation of the slave cluster's
-     * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
-     * </pre>
-     *
-     * <code>required string clusterkey = 1;</code>
-     */
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
-        getClusterkeyBytes();
-
-    /**
-     * <code>optional string replicationEndpointImpl = 2;</code>
-     */
-    boolean hasReplicationEndpointImpl();
-    /**
-     * <code>optional string replicationEndpointImpl = 2;</code>
-     */
-    java.lang.String getReplicationEndpointImpl();
-    /**
-     * <code>optional string replicationEndpointImpl = 2;</code>
-     */
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
-        getReplicationEndpointImplBytes();
-
-    /**
-     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-     */
-    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair> 
-        getDataList();
-    /**
-     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-     */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair getData(int index);
-    /**
-     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-     */
-    int getDataCount();
-    /**
-     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-     */
-    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder> 
-        getDataOrBuilderList();
-    /**
-     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-     */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder getDataOrBuilder(
-        int index);
-
-    /**
-     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
-     */
-    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair> 
-        getConfigurationList();
-    /**
-     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
-     */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair getConfiguration(int index);
-    /**
-     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
-     */
-    int getConfigurationCount();
-    /**
-     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
-     */
-    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder> 
-        getConfigurationOrBuilderList();
-    /**
-     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
-     */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder getConfigurationOrBuilder(
-        int index);
-
-    /**
-     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
-     */
-    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF> 
-        getTableCfsList();
-    /**
-     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
-     */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF getTableCfs(int index);
-    /**
-     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
-     */
-    int getTableCfsCount();
-    /**
-     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
-     */
-    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder> 
-        getTableCfsOrBuilderList();
-    /**
-     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
-     */
-    org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder getTableCfsOrBuilder(
-        int index);
-
-    /**
-     * <code>repeated bytes namespaces = 6;</code>
-     */
-    java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString> getNamespacesList();
-    /**
-     * <code>repeated bytes namespaces = 6;</code>
-     */
-    int getNamespacesCount();
-    /**
-     * <code>repeated bytes namespaces = 6;</code>
-     */
-    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getNamespaces(int index);
-
-    /**
-     * <code>optional int64 bandwidth = 7;</code>
-     */
-    boolean hasBandwidth();
-    /**
-     * <code>optional int64 bandwidth = 7;</code>
-     */
-    long getBandwidth();
-  }
-  /**
-   * <pre>
-   **
-   * Used by replication. Holds a replication peer key.
-   * </pre>
-   *
-   * Protobuf type {@code hbase.pb.ReplicationPeer}
-   */
-  public  static final class ReplicationPeer extends
-      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
-      // @@protoc_insertion_point(message_implements:hbase.pb.ReplicationPeer)
-      ReplicationPeerOrBuilder {
-    // Use ReplicationPeer.newBuilder() to construct.
-    private ReplicationPeer(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
-      super(builder);
-    }
-    private ReplicationPeer() {
-      clusterkey_ = "";
-      replicationEndpointImpl_ = "";
-      data_ = java.util.Collections.emptyList();
-      configuration_ = java.util.Collections.emptyList();
-      tableCfs_ = java.util.Collections.emptyList();
-      namespaces_ = java.util.Collections.emptyList();
-      bandwidth_ = 0L;
-    }
-
-    @java.lang.Override
-    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
-    getUnknownFields() {
-      return this.unknownFields;
-    }
-    private ReplicationPeer(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
-      this();
-      int mutable_bitField0_ = 0;
-      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
-      try {
-        boolean done = false;
-        while (!done) {
-          int tag = input.readTag();
-          switch (tag) {
-            case 0:
-              done = true;
-              break;
-            default: {
-              if (!parseUnknownField(input, unknownFields,
-                                     extensionRegistry, tag)) {
-                done = true;
-              }
-              break;
-            }
-            case 10: {
-              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
-              bitField0_ |= 0x00000001;
-              clusterkey_ = bs;
-              break;
-            }
-            case 18: {
-              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
-              bitField0_ |= 0x00000002;
-              replicationEndpointImpl_ = bs;
-              break;
-            }
-            case 26: {
-              if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
-                data_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair>();
-                mutable_bitField0_ |= 0x00000004;
-              }
-              data_.add(
-                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair.PARSER, extensionRegistry));
-              break;
-            }
-            case 34: {
-              if (!((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
-                configuration_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair>();
-                mutable_bitField0_ |= 0x00000008;
-              }
-              configuration_.add(
-                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair.PARSER, extensionRegistry));
-              break;
-            }
-            case 42: {
-              if (!((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
-                tableCfs_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF>();
-                mutable_bitField0_ |= 0x00000010;
-              }
-              tableCfs_.add(
-                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF.PARSER, extensionRegistry));
-              break;
-            }
-            case 50: {
-              if (!((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
-                namespaces_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString>();
-                mutable_bitField0_ |= 0x00000020;
-              }
-              namespaces_.add(input.readBytes());
-              break;
-            }
-            case 56: {
-              bitField0_ |= 0x00000004;
-              bandwidth_ = input.readInt64();
-              break;
-            }
-          }
-        }
-      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
-        throw e.setUnfinishedMessage(this);
-      } catch (java.io.IOException e) {
-        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
-            e).setUnfinishedMessage(this);
-      } finally {
-        if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
-          data_ = java.util.Collections.unmodifiableList(data_);
-        }
-        if (((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
-          configuration_ = java.util.Collections.unmodifiableList(configuration_);
-        }
-        if (((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
-          tableCfs_ = java.util.Collections.unmodifiableList(tableCfs_);
-        }
-        if (((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
-          namespaces_ = java.util.Collections.unmodifiableList(namespaces_);
-        }
-        this.unknownFields = unknownFields.build();
-        makeExtensionsImmutable();
-      }
-    }
-    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-        getDescriptor() {
-      return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_ReplicationPeer_descriptor;
-    }
-
-    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-        internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_ReplicationPeer_fieldAccessorTable
-          .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer.class, org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer.Builder.class);
-    }
-
-    private int bitField0_;
-    public static final int CLUSTERKEY_FIELD_NUMBER = 1;
-    private volatile java.lang.Object clusterkey_;
-    /**
-     * <pre>
-     * clusterkey is the concatenation of the slave cluster's
-     * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
-     * </pre>
-     *
-     * <code>required string clusterkey = 1;</code>
-     */
-    public boolean hasClusterkey() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
-    }
-    /**
-     * <pre>
-     * clusterkey is the concatenation of the slave cluster's
-     * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
-     * </pre>
-     *
-     * <code>required string clusterkey = 1;</code>
-     */
-    public java.lang.String getClusterkey() {
-      java.lang.Object ref = clusterkey_;
-      if (ref instanceof java.lang.String) {
-        return (java.lang.String) ref;
-      } else {
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
-            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
-        java.lang.String s = bs.toStringUtf8();
-        if (bs.isValidUtf8()) {
-          clusterkey_ = s;
-        }
-        return s;
-      }
-    }
-    /**
-     * <pre>
-     * clusterkey is the concatenation of the slave cluster's
-     * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
-     * </pre>
-     *
-     * <code>required string clusterkey = 1;</code>
-     */
-    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
-        getClusterkeyBytes() {
-      java.lang.Object ref = clusterkey_;
-      if (ref instanceof java.lang.String) {
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
-            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
-                (java.lang.String) ref);
-        clusterkey_ = b;
-        return b;
-      } else {
-        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
-      }
-    }
-
-    public static final int REPLICATIONENDPOINTIMPL_FIELD_NUMBER = 2;
-    private volatile java.lang.Object replicationEndpointImpl_;
-    /**
-     * <code>optional string replicationEndpointImpl = 2;</code>
-     */
-    public boolean hasReplicationEndpointImpl() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
-    }
-    /**
-     * <code>optional string replicationEndpointImpl = 2;</code>
-     */
-    public java.lang.String getReplicationEndpointImpl() {
-      java.lang.Object ref = replicationEndpointImpl_;
-      if (ref instanceof java.lang.String) {
-        return (java.lang.String) ref;
-      } else {
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
-            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
-        java.lang.String s = bs.toStringUtf8();
-        if (bs.isValidUtf8()) {
-          replicationEndpointImpl_ = s;
-        }
-        return s;
-      }
-    }
-    /**
-     * <code>optional string replicationEndpointImpl = 2;</code>
-     */
-    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
-        getReplicationEndpointImplBytes() {
-      java.lang.Object ref = replicationEndpointImpl_;
-      if (ref instanceof java.lang.String) {
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
-            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
-                (java.lang.String) ref);
-        replicationEndpointImpl_ = b;
-        return b;
-      } else {
-        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
-      }
-    }
-
-    public static final int DATA_FIELD_NUMBER = 3;
-    private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair> data_;
-    /**
-     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-     */
-    public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair> getDataList() {
-      return data_;
-    }
-    /**
-     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-     */
-    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder> 
-        getDataOrBuilderList() {
-      return data_;
-    }
-    /**
-     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-     */
-    public int getDataCount() {
-      return data_.size();
-    }
-    /**
-     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-     */
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair getData(int index) {
-      return data_.get(index);
-    }
-    /**
-     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-     */
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder getDataOrBuilder(
-        int index) {
-      return data_.get(index);
-    }
-
-    public static final int CONFIGURATION_FIELD_NUMBER = 4;
-    private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair> configuration_;
-    /**
-     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
-     */
-    public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair> getConfigurationList() {
-      return configuration_;
-    }
-    /**
-     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
-     */
-    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder> 
-        getConfigurationOrBuilderList() {
-      return configuration_;
-    }
-    /**
-     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
-     */
-    public int getConfigurationCount() {
-      return configuration_.size();
-    }
-    /**
-     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
-     */
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair getConfiguration(int index) {
-      return configuration_.get(index);
-    }
-    /**
-     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
-     */
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder getConfigurationOrBuilder(
-        int index) {
-      return configuration_.get(index);
-    }
-
-    public static final int TABLE_CFS_FIELD_NUMBER = 5;
-    private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF> tableCfs_;
-    /**
-     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
-     */
-    public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF> getTableCfsList() {
-      return tableCfs_;
-    }
-    /**
-     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
-     */
-    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder> 
-        getTableCfsOrBuilderList() {
-      return tableCfs_;
-    }
-    /**
-     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
-     */
-    public int getTableCfsCount() {
-      return tableCfs_.size();
-    }
-    /**
-     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
-     */
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCF getTableCfs(int index) {
-      return tableCfs_.get(index);
-    }
-    /**
-     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
-     */
-    public org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder getTableCfsOrBuilder(
-        int index) {
-      return tableCfs_.get(index);
-    }
-
-    public static final int NAMESPACES_FIELD_NUMBER = 6;
-    private java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString> namespaces_;
-    /**
-     * <code>repeated bytes namespaces = 6;</code>
-     */
-    public java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString>
-        getNamespacesList() {
-      return namespaces_;
-    }
-    /**
-     * <code>repeated bytes namespaces = 6;</code>
-     */
-    public int getNamespacesCount() {
-      return namespaces_.size();
-    }
-    /**
-     * <code>repeated bytes namespaces = 6;</code>
-     */
-    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getNamespaces(int index) {
-      return namespaces_.get(index);
-    }
-
-    public static final int BANDWIDTH_FIELD_NUMBER = 7;
-    private long bandwidth_;
-    /**
-     * <code>optional int64 bandwidth = 7;</code>
-     */
-    public boolean hasBandwidth() {
-      return ((bitField0_ & 0x00000004) == 0x00000004);
-    }
-    /**
-     * <code>optional int64 bandwidth = 7;</code>
-     */
-    public long getBandwidth() {
-      return bandwidth_;
-    }
-
-    private byte memoizedIsInitialized = -1;
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized == 1) return true;
-      if (isInitialized == 0) return false;
-
-      if (!hasClusterkey()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      for (int i = 0; i < getDataCount(); i++) {
-        if (!getData(i).isInitialized()) {
-          memoizedIsInitialized = 0;
-          return false;
-        }
-      }
-      for (int i = 0; i < getConfigurationCount(); i++) {
-        if (!getConfiguration(i).isInitialized()) {
-          memoizedIsInitialized = 0;
-          return false;
-        }
-      }
-      for (int i = 0; i < getTableCfsCount(); i++) {
-        if (!getTableCfs(i).isInitialized()) {
-          memoizedIsInitialized = 0;
-          return false;
-        }
-      }
-      memoizedIsInitialized = 1;
-      return true;
-    }
-
-    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
-                        throws java.io.IOException {
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, clusterkey_);
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 2, replicationEndpointImpl_);
-      }
-      for (int i = 0; i < data_.size(); i++) {
-        output.writeMessage(3, data_.get(i));
-      }
-      for (int i = 0; i < configuration_.size(); i++) {
-        output.writeMessage(4, configuration_.get(i));
-      }
-      for (int i = 0; i < tableCfs_.size(); i++) {
-        output.writeMessage(5, tableCfs_.get(i));
-      }
-      for (int i = 0; i < namespaces_.size(); i++) {
-        output.writeBytes(6, namespaces_.get(i));
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeInt64(7, bandwidth_);
-      }
-      unknownFields.writeTo(output);
-    }
-
-    public int getSerializedSize() {
-      int size = memoizedSize;
-      if (size != -1) return size;
-
-      size = 0;
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, clusterkey_);
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(2, replicationEndpointImpl_);
-      }
-      for (int i = 0; i < data_.size(); i++) {
-        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeMessageSize(3, data_.get(i));
-      }
-      for (int i = 0; i < configuration_.size(); i++) {
-        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeMessageSize(4, configuration_.get(i));
-      }
-      for (int i = 0; i < tableCfs_.size(); i++) {
-        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeMessageSize(5, tableCfs_.get(i));
-      }
-      {
-        int dataSize = 0;
-        for (int i = 0; i < namespaces_.size(); i++) {
-          dataSize += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-            .computeBytesSizeNoTag(namespaces_.get(i));
-        }
-        size += dataSize;
-        size += 1 * getNamespacesList().size();
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
-          .computeInt64Size(7, bandwidth_);
-      }
-      size += unknownFields.getSerializedSize();
-      memoizedSize = size;
-      return size;
-    }
-
-    private static final long serialVersionUID = 0L;
-    @java.lang.Override
-    public boolean equals(final java.lang.Object obj) {
-      if (obj == this) {
-       return true;
-      }
-      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer)) {
-        return super.equals(obj);
-      }
-      org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer other = (org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer) obj;
-
-      boolean result = true;
-      result = result && (hasClusterkey() == other.hasClusterkey());
-      if (hasClusterkey()) {
-        result = result && getClusterkey()
-            .equals(other.getClusterkey());
-      }
-      result = result && (hasReplicationEndpointImpl() == other.hasReplicationEndpointImpl());
-      if (hasReplicationEndpointImpl()) {
-        result = result && getReplicationEndpointImpl()
-            .equals(other.getReplicationEndpointImpl());
-      }
-      result = result && getDataList()
-          .equals(other.getDataList());
-      result = result && getConfigurationList()
-          .equals(other.getConfigurationList());
-      result = result && getTableCfsList()
-          .equals(other.getTableCfsList());
-      result = result && getNamespacesList()
-          .equals(other.getNamespacesList());
-      result = result && (hasBandwidth() == other.hasBandwidth());
-      if (hasBandwidth()) {
-        result = result && (getBandwidth()
-            == other.getBandwidth());
-      }
-      result = result && unknownFields.equals(other.unknownFields);
-      return result;
-    }
-
-    @java.lang.Override
-    public int hashCode() {
-      if (memoizedHashCode != 0) {
-        return memoizedHashCode;
-      }
-      int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasClusterkey()) {
-        hash = (37 * hash) + CLUSTERKEY_FIELD_NUMBER;
-        hash = (53 * hash) + getClusterkey().hashCode();
-      }
-      if (hasReplicationEndpointImpl()) {
-        hash = (37 * hash) + REPLICATIONENDPOINTIMPL_FIELD_NUMBER;
-        hash = (53 * hash) + getReplicationEndpointImpl().hashCode();
-      }
-      if (getDataCount() > 0) {
-        hash = (37 * hash) + DATA_FIELD_NUMBER;
-        hash = (53 * hash) + getDataList().hashCode();
-      }
-      if (getConfigurationCount() > 0) {
-        hash = (37 * hash) + CONFIGURATION_FIELD_NUMBER;
-        hash = (53 * hash) + getConfigurationList().hashCode();
-      }
-      if (getTableCfsCount() > 0) {
-        hash = (37 * hash) + TABLE_CFS_FIELD_NUMBER;
-        hash = (53 * hash) + getTableCfsList().hashCode();
-      }
-      if (getNamespacesCount() > 0) {
-        hash = (37 * hash) + NAMESPACES_FIELD_NUMBER;
-        hash = (53 * hash) + getNamespacesList().hashCode();
-      }
-      if (hasBandwidth()) {
-        hash = (37 * hash) + BANDWIDTH_FIELD_NUMBER;
-        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
-            getBandwidth());
-      }
-      hash = (29 * hash) + unknownFields.hashCode();
-      memoizedHashCode = hash;
-      return hash;
-    }
-
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer parseFrom(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
-        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer parseFrom(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer parseFrom(byte[] data)
-        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer parseFrom(
-        byte[] data,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer parseFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-          .parseWithIOException(PARSER, input);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer parseFrom(
-        java.io.InputStream input,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-          .parseWithIOException(PARSER, input, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer parseDelimitedFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-          .parseDelimitedWithIOException(PARSER, input);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer parseDelimitedFrom(
-        java.io.InputStream input,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer parseFrom(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
-        throws java.io.IOException {
-      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-          .parseWithIOException(PARSER, input);
-    }
-    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer parseFrom(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-          .parseWithIOException(PARSER, input, extensionRegistry);
-    }
-
-    public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder() {
-      return DEFAULT_INSTANCE.toBuilder();
-    }
-    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer prototype) {
-      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
-    }
-    public Builder toBuilder() {
-      return this == DEFAULT_INSTANCE
-          ? new Builder() : new Builder().mergeFrom(this);
-    }
-
-    @java.lang.Override
-    protected Builder newBuilderForType(
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
-      Builder builder = new Builder(parent);
-      return builder;
-    }
-    /**
-     * <pre>
-     **
-     * Used by replication. Holds a replication peer key.
-     * </pre>
-     *
-     * Protobuf type {@code hbase.pb.ReplicationPeer}
-     */
-    public static final class Builder extends
-        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
-        // @@protoc_insertion_point(builder_implements:hbase.pb.ReplicationPeer)
-        org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeerOrBuilder {
-      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-          getDescriptor() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_ReplicationPeer_descriptor;
-      }
-
-      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
-          internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_ReplicationPeer_fieldAccessorTable
-            .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer.class, org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer.Builder.class);
-      }
-
-      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer.newBuilder()
-      private Builder() {
-        maybeForceBuilderInitialization();
-      }
-
-      private Builder(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
-        super(parent);
-        maybeForceBuilderInitialization();
-      }
-      private void maybeForceBuilderInitialization() {
-        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
-                .alwaysUseFieldBuilders) {
-          getDataFieldBuilder();
-          getConfigurationFieldBuilder();
-          getTableCfsFieldBuilder();
-        }
-      }
-      public Builder clear() {
-        super.clear();
-        clusterkey_ = "";
-        bitField0_ = (bitField0_ & ~0x00000001);
-        replicationEndpointImpl_ = "";
-        bitField0_ = (bitField0_ & ~0x00000002);
-        if (dataBuilder_ == null) {
-          data_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000004);
-        } else {
-          dataBuilder_.clear();
-        }
-        if (configurationBuilder_ == null) {
-          configuration_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000008);
-        } else {
-          configurationBuilder_.clear();
-        }
-        if (tableCfsBuilder_ == null) {
-          tableCfs_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000010);
-        } else {
-          tableCfsBuilder_.clear();
-        }
-        namespaces_ = java.util.Collections.emptyList();
-        bitField0_ = (bitField0_ & ~0x00000020);
-        bandwidth_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000040);
-        return this;
-      }
-
-      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
-          getDescriptorForType() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_ReplicationPeer_descriptor;
-      }
-
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer.getDefaultInstance();
-      }
-
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer build() {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer result = buildPartial();
-        if (!result.isInitialized()) {
-          throw newUninitializedMessageException(result);
-        }
-        return result;
-      }
-
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer buildPartial() {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer result = new org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer(this);
-        int from_bitField0_ = bitField0_;
-        int to_bitField0_ = 0;
-        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-          to_bitField0_ |= 0x00000001;
-        }
-        result.clusterkey_ = clusterkey_;
-        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
-          to_bitField0_ |= 0x00000002;
-        }
-        result.replicationEndpointImpl_ = replicationEndpointImpl_;
-        if (dataBuilder_ == null) {
-          if (((bitField0_ & 0x00000004) == 0x00000004)) {
-            data_ = java.util.Collections.unmodifiableList(data_);
-            bitField0_ = (bitField0_ & ~0x00000004);
-          }
-          result.data_ = data_;
-        } else {
-          result.data_ = dataBuilder_.build();
-        }
-        if (configurationBuilder_ == null) {
-          if (((bitField0_ & 0x00000008) == 0x00000008)) {
-            configuration_ = java.util.Collections.unmodifiableList(configuration_);
-            bitField0_ = (bitField0_ & ~0x00000008);
-          }
-          result.configuration_ = configuration_;
-        } else {
-          result.configuration_ = configurationBuilder_.build();
-        }
-        if (tableCfsBuilder_ == null) {
-          if (((bitField0_ & 0x00000010) == 0x00000010)) {
-            tableCfs_ = java.util.Collections.unmodifiableList(tableCfs_);
-            bitField0_ = (bitField0_ & ~0x00000010);
-          }
-          result.tableCfs_ = tableCfs_;
-        } else {
-          result.tableCfs_ = tableCfsBuilder_.build();
-        }
-        if (((bitField0_ & 0x00000020) == 0x00000020)) {
-          namespaces_ = java.util.Collections.unmodifiableList(namespaces_);
-          bitField0_ = (bitField0_ & ~0x00000020);
-        }
-        result.namespaces_ = namespaces_;
-        if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
-          to_bitField0_ |= 0x00000004;
-        }
-        result.bandwidth_ = bandwidth_;
-        result.bitField0_ = to_bitField0_;
-        onBuilt();
-        return result;
-      }
-
-      public Builder clone() {
-        return (Builder) super.clone();
-      }
-      public Builder setField(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
-          Object value) {
-        return (Builder) super.setField(field, value);
-      }
-      public Builder clearField(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
-        return (Builder) super.clearField(field);
-      }
-      public Builder clearOneof(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
-        return (Builder) super.clearOneof(oneof);
-      }
-      public Builder setRepeatedField(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
-          int index, Object value) {
-        return (Builder) super.setRepeatedField(field, index, value);
-      }
-      public Builder addRepeatedField(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
-          Object value) {
-        return (Builder) super.addRepeatedField(field, value);
-      }
-      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
-        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer) {
-          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer)other);
-        } else {
-          super.mergeFrom(other);
-          return this;
-        }
-      }
-
-      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer other) {
-        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer.getDefaultInstance()) return this;
-        if (other.hasClusterkey()) {
-          bitField0_ |= 0x00000001;
-          clusterkey_ = other.clusterkey_;
-          onChanged();
-        }
-        if (other.hasReplicationEndpointImpl()) {
-          bitField0_ |= 0x00000002;
-          replicationEndpointImpl_ = other.replicationEndpointImpl_;
-          onChanged();
-        }
-        if (dataBuilder_ == null) {
-          if (!other.data_.isEmpty()) {
-            if (data_.isEmpty()) {
-              data_ = other.data_;
-              bitField0_ = (bitField0_ & ~0x00000004);
-            } else {
-              ensureDataIsMutable();
-              data_.addAll(other.data_);
-            }
-            onChanged();
-          }
-        } else {
-          if (!other.data_.isEmpty()) {
-            if (dataBuilder_.isEmpty()) {
-              dataBuilder_.dispose();
-              dataBuilder_ = null;
-              data_ = other.data_;
-              bitField0_ = (bitField0_ & ~0x00000004);
-              dataBuilder_ = 
-                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
-                   getDataFieldBuilder() : null;
-            } else {
-              dataBuilder_.addAllMessages(other.data_);
-            }
-          }
-        }
-        if (configurationBuilder_ == null) {
-          if (!other.configuration_.isEmpty()) {
-            if (configuration_.isEmpty()) {
-              configuration_ = other.configuration_;
-              bitField0_ = (bitField0_ & ~0x00000008);
-            } else {
-              ensureConfigurationIsMutable();
-              configuration_.addAll(other.configuration_);
-            }
-            onChanged();
-          }
-        } else {
-          if (!other.configuration_.isEmpty()) {
-            if (configurationBuilder_.isEmpty()) {
-              configurationBuilder_.dispose();
-              configurationBuilder_ = null;
-              configuration_ = other.configuration_;
-              bitField0_ = (bitField0_ & ~0x00000008);
-              configurationBuilder_ = 
-                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
-                   getConfigurationFieldBuilder() : null;
-            } else {
-              configurationBuilder_.addAllMessages(other.configuration_);
-            }
-          }
-        }
-        if (tableCfsBuilder_ == null) {
-          if (!other.tableCfs_.isEmpty()) {
-            if (tableCfs_.isEmpty()) {
-              tableCfs_ = other.tableCfs_;
-              bitField0_ = (bitField0_ & ~0x00000010);
-            } else {
-              ensureTableCfsIsMutable();
-              tableCfs_.addAll(other.tableCfs_);
-            }
-            onChanged();
-          }
-        } else {
-          if (!other.tableCfs_.isEmpty()) {
-            if (tableCfsBuilder_.isEmpty()) {
-              tableCfsBuilder_.dispose();
-              tableCfsBuilder_ = null;
-              tableCfs_ = other.tableCfs_;
-              bitField0_ = (bitField0_ & ~0x00000010);
-              tableCfsBuilder_ = 
-                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
-                   getTableCfsFieldBuilder() : null;
-            } else {
-              tableCfsBuilder_.addAllMessages(other.tableCfs_);
-            }
-          }
-        }
-        if (!other.namespaces_.isEmpty()) {
-          if (namespaces_.isEmpty()) {
-            namespaces_ = other.namespaces_;
-            bitField0_ = (bitField0_ & ~0x00000020);
-          } else {
-            ensureNamespacesIsMutable();
-            namespaces_.addAll(other.namespaces_);
-          }
-          onChanged();
-        }
-        if (other.hasBandwidth()) {
-          setBandwidth(other.getBandwidth());
-        }
-        this.mergeUnknownFields(other.unknownFields);
-        onChanged();
-        return this;
-      }
-
-      public final boolean isInitialized() {
-        if (!hasClusterkey()) {
-          return false;
-        }
-        for (int i = 0; i < getDataCount(); i++) {
-          if (!getData(i).isInitialized()) {
-            return false;
-          }
-        }
-        for (int i = 0; i < getConfigurationCount(); i++) {
-          if (!getConfiguration(i).isInitialized()) {
-            return false;
-          }
-        }
-        for (int i = 0; i < getTableCfsCount(); i++) {
-          if (!getTableCfs(i).isInitialized()) {
-            return false;
-          }
-        }
-        return true;
-      }
-
-      public Builder mergeFrom(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws java.io.IOException {
-        org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer parsedMessage = null;
-        try {
-          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.ReplicationPeer) e.getUnfinishedMessage();
-          throw e.unwrapIOException();
-        } finally {
-          if (parsedMessage != null) {
-            mergeFrom(parsedMessage);
-          }
-        }
-        return this;
-      }
-      private int bitField0_;
-
-      private java.lang.Object clusterkey_ = "";
-      /**
-       * <pre>
-       * clusterkey is the concatenation of the slave cluster's
-       * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
-       * </pre>
-       *
-       * <code>required string clusterkey = 1;</code>
-       */
-      public boolean hasClusterkey() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
-      }
-      /**
-       * <pre>
-       * clusterkey is the concatenation of the slave cluster's
-       * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
-       * </pre>
-       *
-       * <code>required string clusterkey = 1;</code>
-       */
-      public java.lang.String getClusterkey() {
-        java.lang.Object ref = clusterkey_;
-        if (!(ref instanceof java.lang.String)) {
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
-              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
-          java.lang.String s = bs.toStringUtf8();
-          if (bs.isValidUtf8()) {
-            clusterkey_ = s;
-          }
-          return s;
-        } else {
-          return (java.lang.String) ref;
-        }
-      }
-      /**
-       * <pre>
-       * clusterkey is the concatenation of the slave cluster's
-       * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
-       * </pre>
-       *
-       * <code>required string clusterkey = 1;</code>
-       */
-      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
-          getClusterkeyBytes() {
-        java.lang.Object ref = clusterkey_;
-        if (ref instanceof String) {
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
-              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
-                  (java.lang.String) ref);
-          clusterkey_ = b;
-          return b;
-        } else {
-          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
-        }
-      }
-      /**
-       * <pre>
-       * clusterkey is the concatenation of the slave cluster's
-       * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
-       * </pre>
-       *
-       * <code>required string clusterkey = 1;</code>
-       */
-      public Builder setClusterkey(
-          java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000001;
-        clusterkey_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <pre>
-       * clusterkey is the concatenation of the slave cluster's
-       * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
-       * </pre>
-       *
-       * <code>required string clusterkey = 1;</code>
-       */
-      public Builder clearClusterkey() {
-        bitField0_ = (bitField0_ & ~0x00000001);
-        clusterkey_ = getDefaultInstance().getClusterkey();
-        onChanged();
-        return this;
-      }
-      /**
-       * <pre>
-       * clusterkey is the concatenation of the slave cluster's
-       * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
-       * </pre>
-       *
-       * <code>required string clusterkey = 1;</code>
-       */
-      public Builder setClusterkeyBytes(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000001;
-        clusterkey_ = value;
-        onChanged();
-        return this;
-      }
-
-      private java.lang.Object replicationEndpointImpl_ = "";
-      /**
-       * <code>optional string replicationEndpointImpl = 2;</code>
-       */
-      public boolean hasReplicationEndpointImpl() {
-        return ((bitField0_ & 0x00000002) == 0x00000002);
-      }
-      /**
-       * <code>optional string replicationEndpointImpl = 2;</code>
-       */
-      public java.lang.String getReplicationEndpointImpl() {
-        java.lang.Object ref = replicationEndpointImpl_;
-        if (!(ref instanceof java.lang.String)) {
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
-              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
-          java.lang.String s = bs.toStringUtf8();
-          if (bs.isValidUtf8()) {
-            replicationEndpointImpl_ = s;
-          }
-          return s;
-        } else {
-          return (java.lang.String) ref;
-        }
-      }
-      /**
-       * <code>optional string replicationEndpointImpl = 2;</code>
-       */
-      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
-          getReplicationEndpointImplBytes() {
-        java.lang.Object ref = replicationEndpointImpl_;
-        if (ref instanceof String) {
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
-              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
-                  (java.lang.String) ref);
-          replicationEndpointImpl_ = b;
-          return b;
-        } else {
-          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
-        }
-      }
-      /**
-       * <code>optional string replicationEndpointImpl = 2;</code>
-       */
-      public Builder setReplicationEndpointImpl(
-          java.lang.String value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000002;
-        replicationEndpointImpl_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional string replicationEndpointImpl = 2;</code>
-       */
-      public Builder clearReplicationEndpointImpl() {
-        bitField0_ = (bitField0_ & ~0x00000002);
-        replicationEndpointImpl_ = getDefaultInstance().getReplicationEndpointImpl();
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional string replicationEndpointImpl = 2;</code>
-       */
-      public Builder setReplicationEndpointImplBytes(
-          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000002;
-        replicationEndpointImpl_ = value;
-        onChanged();
-        return this;
-      }
-
-      private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair> data_ =
-        java.util.Collections.emptyList();
-      private void ensureDataIsMutable() {
-        if (!((bitField0_ & 0x00000004) == 0x00000004)) {
-          data_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair>(data_);
-          bitField0_ |= 0x00000004;
-         }
-      }
-
-      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
-          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder> dataBuilder_;
-
-      /**
-       * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-       */
-      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair> getDataList() {
-        if (dataBuilder_ == null) {
-          return java.util.Collections.unmodifiableList(data_);
-        } else {
-          return dataBuilder_.getMessageList();
-        }
-      }
-      /**
-       * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-       */
-      public int getDataCount() {
-        if (dataBuilder_ == null) {
-          return data_.size();
-        } else {
-          return dataBuilder_.getCount();
-        }
-      }
-      /**
-       * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-       */
-      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair getData(int index) {
-        if (dataBuilder_ == null) {
-          return data_.get(index);
-        } else {
-          return dataBuilder_.getMessage(index);
-        }
-      }
-      /**
-       * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-       */
-      public Builder setData(
-          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair value) {
-        if (dataBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          ensureDataIsMutable();
-          data_.set(index, value);
-          onChanged();
-        } else {
-          dataBuilder_.setMessage(index, value);
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-       */
-      public Builder setData(
-          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair.Builder builderForValue) {
-        if (dataBuilder_ == null) {
-          ensureDataIsMutable();
-          data_.set(index, builderForValue.build());
-          onChanged();
-        } else {
-          dataBuilder_.setMessage(index, builderForValue.build());
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-       */
-      public Builder addData(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair value) {
-        if (dataBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          ensureDataIsMutable();
-          data_.add(value);
-          onChanged();
-        } else {
-          dataBuilder_.addMessage(value);
-        }
-        return this;
-      }
-      /**
-       * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
-       */
-      public Builder ad

<TRUNCATED>

[16/16] hbase git commit: HBASE-17424 Disable external entity parsing in RemoteAdmin

Posted by sy...@apache.org.
HBASE-17424 Disable external entity parsing in RemoteAdmin


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

Branch: refs/heads/hbase-12439
Commit: 6fecf55a7eb05d4f85a17ad9f56d9728f1c5d3e7
Parents: b2a9be0
Author: Josh Elser <el...@apache.org>
Authored: Tue Jan 3 19:02:08 2017 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Fri Jan 6 16:38:50 2017 -0500

----------------------------------------------------------------------
 .../hadoop/hbase/rest/client/RemoteAdmin.java   | 26 ++++++-
 .../hbase/rest/client/TestXmlParsing.java       | 76 ++++++++++++++++++++
 2 files changed, 100 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6fecf55a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteAdmin.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteAdmin.java
index e8845eb..de1e23f 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteAdmin.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteAdmin.java
@@ -26,6 +26,9 @@ import java.io.InterruptedIOException;
 import javax.xml.bind.JAXBContext;
 import javax.xml.bind.JAXBException;
 import javax.xml.bind.Unmarshaller;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.XMLStreamReader;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
@@ -209,12 +212,12 @@ public class RemoteAdmin {
         try {
 
           return (StorageClusterVersionModel) getUnmarsheller().unmarshal(
-              new ByteArrayInputStream(response.getBody()));
+              getInputStream(response));
         } catch (JAXBException jaxbe) {
 
           throw new IOException(
               "Issue parsing StorageClusterVersionModel object in XML form: "
-                  + jaxbe.getLocalizedMessage());
+                  + jaxbe.getLocalizedMessage(), jaxbe);
         }
       case 404:
         throw new IOException("Cluster version not found");
@@ -398,4 +401,23 @@ public class RemoteAdmin {
     throw new IOException("get request to " + path.toString()
         + " request timed out");
   }
+
+  /**
+   * Convert the REST server's response to an XML reader.
+   *
+   * @param response The REST server's response.
+   * @return A reader over the parsed XML document.
+   * @throws IOException If the document fails to parse
+   */
+  private XMLStreamReader getInputStream(Response response) throws IOException {
+    try {
+      // Prevent the parser from reading XMl with external entities defined
+      XMLInputFactory xif = XMLInputFactory.newFactory();
+      xif.setProperty(XMLInputFactory.IS_SUPPORTING_EXTERNAL_ENTITIES, false);
+      xif.setProperty(XMLInputFactory.SUPPORT_DTD, false);
+      return xif.createXMLStreamReader(new ByteArrayInputStream(response.getBody()));
+    } catch (XMLStreamException e) {
+      throw new IOException("Failed to parse XML", e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6fecf55a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestXmlParsing.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestXmlParsing.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestXmlParsing.java
new file mode 100644
index 0000000..56dc05e
--- /dev/null
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestXmlParsing.java
@@ -0,0 +1,76 @@
+/*
+ * 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.rest.client;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.rest.Constants;
+import org.apache.hadoop.hbase.rest.model.StorageClusterVersionModel;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.util.StringUtils;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test class for {@link RemoteAdmin} to verify XML is parsed in a certain manner.
+ */
+@Category(SmallTests.class)
+public class TestXmlParsing {
+
+  @Test
+  public void testParsingClusterVersion() throws Exception {
+    final String xml = "<?xml version=\"1.0\" encoding=\"UTF-8\" standalone=\"yes\"?>"
+        + "<ClusterVersion>2.0.0</ClusterVersion>";
+    Client client = mock(Client.class);
+    RemoteAdmin admin = new RemoteAdmin(client, HBaseConfiguration.create(), null);
+    Response resp = new Response(200, null, xml.getBytes());
+
+    when(client.get("/version/cluster", Constants.MIMETYPE_XML)).thenReturn(resp);
+
+    StorageClusterVersionModel cv = admin.getClusterVersion();
+    assertEquals("2.0.0", cv.getVersion());
+  }
+
+  @Test
+  public void testFailOnExternalEntities() throws Exception {
+    final String externalEntitiesXml =
+        "<?xml version=\"1.0\" encoding=\"UTF-8\"?>"
+        + " <!DOCTYPE foo [ <!ENTITY xxe SYSTEM \"/tmp/foo\"> ] >"
+        + " <ClusterVersion>&xee;</ClusterVersion>";
+    Client client = mock(Client.class);
+    RemoteAdmin admin = new RemoteAdmin(client, HBaseConfiguration.create(), null);
+    Response resp = new Response(200, null, externalEntitiesXml.getBytes());
+
+    when(client.get("/version/cluster", Constants.MIMETYPE_XML)).thenReturn(resp);
+
+    try {
+      admin.getClusterVersion();
+      fail("Expected getClusterVersion() to throw an exception");
+    } catch (IOException e) {
+      final String exceptionText = StringUtils.stringifyException(e);
+      final String expectedText = "The entity \"xee\" was referenced, but not declared.";
+      assertTrue("Exception does not contain expected text", exceptionText.contains(expectedText));
+    }
+  }
+}


[02/16] hbase git commit: HBASE-17409 Limit jsonp callback name to prevent xss

Posted by sy...@apache.org.
HBASE-17409 Limit jsonp callback name to prevent xss


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

Branch: refs/heads/hbase-12439
Commit: 20a7ae28657c41027486f7d2c8e1b260b692f418
Parents: 63bd8be
Author: Josh Elser <el...@apache.org>
Authored: Tue Jan 3 17:46:45 2017 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Wed Jan 4 17:05:07 2017 -0500

----------------------------------------------------------------------
 .../hadoop/hbase/http/jmx/JMXJsonServlet.java   | 29 ++++++++++++++++----
 .../hbase/http/jmx/TestJMXJsonServlet.java      | 25 +++++++++++++++++
 2 files changed, 48 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/20a7ae28/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
index 45c2c15..3abad3a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
@@ -149,9 +149,7 @@ public class JMXJsonServlet extends HttpServlet {
    *          The servlet response we are creating
    */
   @Override
-  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="XSS_REQUEST_PARAMETER_TO_SERVLET_WRITER",
-    justification="TODO: See HBASE-15122")
-  public void doGet(HttpServletRequest request, HttpServletResponse response) {
+  public void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException {
     try {
       if (!HttpServer.isInstrumentationAccessAllowed(getServletContext(), request, response)) {
         return;
@@ -160,11 +158,11 @@ public class JMXJsonServlet extends HttpServlet {
       PrintWriter writer = null;
       JSONBean.Writer beanWriter = null;
       try {
+        jsonpcb = checkCallbackName(request.getParameter(CALLBACK_PARAM));
         writer = response.getWriter();
         beanWriter = this.jsonBeanWriter.open(writer);
  
         // "callback" parameter implies JSONP outpout
-        jsonpcb = request.getParameter(CALLBACK_PARAM);
         if (jsonpcb != null) {
           response.setContentType("application/javascript; charset=utf8");
           writer.write(jsonpcb + "(");
@@ -214,10 +212,29 @@ public class JMXJsonServlet extends HttpServlet {
       }
     } catch (IOException e) {
       LOG.error("Caught an exception while processing JMX request", e);
-      response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+      response.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
     } catch (MalformedObjectNameException e) {
       LOG.error("Caught an exception while processing JMX request", e);
-      response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
+      response.sendError(HttpServletResponse.SC_BAD_REQUEST);
     }
   }
+
+  /**
+   * Verifies that the callback property, if provided, is purely alphanumeric.
+   * This prevents a malicious callback name (that is javascript code) from being
+   * returned by the UI to an unsuspecting user.
+   *
+   * @param callbackName The callback name, can be null.
+   * @return The callback name
+   * @throws IOException If the name is disallowed.
+   */
+  private String checkCallbackName(String callbackName) throws IOException {
+    if (null == callbackName) {
+      return null;
+    }
+    if (callbackName.matches("[A-Za-z0-9_]+")) {
+      return callbackName;
+    }
+    throw new IOException("'callback' must be alphanumeric");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/20a7ae28/hbase-server/src/test/java/org/apache/hadoop/hbase/http/jmx/TestJMXJsonServlet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/jmx/TestJMXJsonServlet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/jmx/TestJMXJsonServlet.java
index 031ddce..d7e68d5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/jmx/TestJMXJsonServlet.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/jmx/TestJMXJsonServlet.java
@@ -17,10 +17,14 @@
 
 package org.apache.hadoop.hbase.http.jmx;
 
+import java.net.HttpURLConnection;
 import java.net.URL;
+import java.net.URLEncoder;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import javax.servlet.http.HttpServletResponse;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
@@ -39,6 +43,9 @@ public class TestJMXJsonServlet extends HttpServerFunctionalTest {
   private static URL baseUrl;
 
   @BeforeClass public static void setup() throws Exception {
+    // Eclipse doesn't pick this up correctly from the plugin
+    // configuration in the pom.
+    System.setProperty(HttpServerFunctionalTest.TEST_BUILD_WEBAPPS, "target/test-classes/webapps");
     server = createTestServer();
     server.start();
     baseUrl = getServerURL(server);
@@ -106,4 +113,22 @@ public class TestJMXJsonServlet extends HttpServerFunctionalTest {
     assertReFind("\\}\\);$", result);
 
   }
+
+  @Test
+  public void testDisallowedJSONPCallback() throws Exception {
+    String callback = "function(){alert('bigproblems!')};foo";
+    URL url = new URL(
+        baseUrl, "/jmx?qry=java.lang:type=Memory&callback="+URLEncoder.encode(callback, "UTF-8"));
+    HttpURLConnection cnxn = (HttpURLConnection) url.openConnection();
+    assertEquals(HttpServletResponse.SC_INTERNAL_SERVER_ERROR, cnxn.getResponseCode());
+  }
+
+  @Test
+  public void testUnderscoresInJSONPCallback() throws Exception {
+    String callback = "my_function";
+    URL url = new URL(
+        baseUrl, "/jmx?qry=java.lang:type=Memory&callback="+URLEncoder.encode(callback, "UTF-8"));
+    HttpURLConnection cnxn = (HttpURLConnection) url.openConnection();
+    assertEquals(HttpServletResponse.SC_OK, cnxn.getResponseCode());
+  }
 }


[08/16] hbase git commit: HBASE-17388 Move ReplicationPeer and other replication related PB messages to the replication.proto

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/e02ae772/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ReplicationProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ReplicationProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ReplicationProtos.java
index 51d3fc9..e4fdfe8 100644
--- a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ReplicationProtos.java
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/generated/ReplicationProtos.java
@@ -14,6 +14,3978 @@ public final class ReplicationProtos {
     registerAllExtensions(
         (org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite) registry);
   }
+  public interface TableCFOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.TableCF)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    boolean hasTableName();
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName();
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder();
+
+    /**
+     * <code>repeated bytes families = 2;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString> getFamiliesList();
+    /**
+     * <code>repeated bytes families = 2;</code>
+     */
+    int getFamiliesCount();
+    /**
+     * <code>repeated bytes families = 2;</code>
+     */
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilies(int index);
+  }
+  /**
+   * Protobuf type {@code hbase.pb.TableCF}
+   */
+  public  static final class TableCF extends
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:hbase.pb.TableCF)
+      TableCFOrBuilder {
+    // Use TableCF.newBuilder() to construct.
+    private TableCF(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private TableCF() {
+      families_ = java.util.Collections.emptyList();
+    }
+
+    @java.lang.Override
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private TableCF(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      int mutable_bitField0_ = 0;
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = tableName_.toBuilder();
+              }
+              tableName_ = input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(tableName_);
+                tableName_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
+              break;
+            }
+            case 18: {
+              if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+                families_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString>();
+                mutable_bitField0_ |= 0x00000002;
+              }
+              families_.add(input.readBytes());
+              break;
+            }
+          }
+        }
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+          families_ = java.util.Collections.unmodifiableList(families_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.internal_static_hbase_pb_TableCF_descriptor;
+    }
+
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.internal_static_hbase_pb_TableCF_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF.class, org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF.Builder.class);
+    }
+
+    private int bitField0_;
+    public static final int TABLE_NAME_FIELD_NUMBER = 1;
+    private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_;
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    public boolean hasTableName() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName() {
+      return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
+    }
+    /**
+     * <code>optional .hbase.pb.TableName table_name = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+      return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
+    }
+
+    public static final int FAMILIES_FIELD_NUMBER = 2;
+    private java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString> families_;
+    /**
+     * <code>repeated bytes families = 2;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString>
+        getFamiliesList() {
+      return families_;
+    }
+    /**
+     * <code>repeated bytes families = 2;</code>
+     */
+    public int getFamiliesCount() {
+      return families_.size();
+    }
+    /**
+     * <code>repeated bytes families = 2;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilies(int index) {
+      return families_.get(index);
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      if (hasTableName()) {
+        if (!getTableName().isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, getTableName());
+      }
+      for (int i = 0; i < families_.size(); i++) {
+        output.writeBytes(2, families_.get(i));
+      }
+      unknownFields.writeTo(output);
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, getTableName());
+      }
+      {
+        int dataSize = 0;
+        for (int i = 0; i < families_.size(); i++) {
+          dataSize += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+            .computeBytesSizeNoTag(families_.get(i));
+        }
+        size += dataSize;
+        size += 1 * getFamiliesList().size();
+      }
+      size += unknownFields.getSerializedSize();
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF other = (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF) obj;
+
+      boolean result = true;
+      result = result && (hasTableName() == other.hasTableName());
+      if (hasTableName()) {
+        result = result && getTableName()
+            .equals(other.getTableName());
+      }
+      result = result && getFamiliesList()
+          .equals(other.getFamiliesList());
+      result = result && unknownFields.equals(other.unknownFields);
+      return result;
+    }
+
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasTableName()) {
+        hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getTableName().hashCode();
+      }
+      if (getFamiliesCount() > 0) {
+        hash = (37 * hash) + FAMILIES_FIELD_NUMBER;
+        hash = (53 * hash) + getFamiliesList().hashCode();
+      }
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF parseFrom(byte[] data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF parseFrom(
+        byte[] data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF parseFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF parseDelimitedFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code hbase.pb.TableCF}
+     */
+    public static final class Builder extends
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:hbase.pb.TableCF)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCFOrBuilder {
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.internal_static_hbase_pb_TableCF_descriptor;
+      }
+
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.internal_static_hbase_pb_TableCF_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF.class, org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+                .alwaysUseFieldBuilders) {
+          getTableNameFieldBuilder();
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        if (tableNameBuilder_ == null) {
+          tableName_ = null;
+        } else {
+          tableNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        families_ = java.util.Collections.emptyList();
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.internal_static_hbase_pb_TableCF_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF result = new org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (tableNameBuilder_ == null) {
+          result.tableName_ = tableName_;
+        } else {
+          result.tableName_ = tableNameBuilder_.build();
+        }
+        if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          families_ = java.util.Collections.unmodifiableList(families_);
+          bitField0_ = (bitField0_ & ~0x00000002);
+        }
+        result.families_ = families_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder clone() {
+        return (Builder) super.clone();
+      }
+      public Builder setField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.setField(field, value);
+      }
+      public Builder clearField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
+        return (Builder) super.clearField(field);
+      }
+      public Builder clearOneof(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+        return (Builder) super.clearOneof(oneof);
+      }
+      public Builder setRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          int index, Object value) {
+        return (Builder) super.setRepeatedField(field, index, value);
+      }
+      public Builder addRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.addRepeatedField(field, value);
+      }
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF other) {
+        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF.getDefaultInstance()) return this;
+        if (other.hasTableName()) {
+          mergeTableName(other.getTableName());
+        }
+        if (!other.families_.isEmpty()) {
+          if (families_.isEmpty()) {
+            families_ = other.families_;
+            bitField0_ = (bitField0_ & ~0x00000002);
+          } else {
+            ensureFamiliesIsMutable();
+            families_.addAll(other.families_);
+          }
+          onChanged();
+        }
+        this.mergeUnknownFields(other.unknownFields);
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (hasTableName()) {
+          if (!getTableName().isInitialized()) {
+            return false;
+          }
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      private org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName tableName_ = null;
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public boolean hasTableName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName getTableName() {
+        if (tableNameBuilder_ == null) {
+          return tableName_ == null ? org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
+        } else {
+          return tableNameBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public Builder setTableName(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          tableName_ = value;
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public Builder setTableName(
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
+        if (tableNameBuilder_ == null) {
+          tableName_ = builderForValue.build();
+          onChanged();
+        } else {
+          tableNameBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public Builder mergeTableName(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName value) {
+        if (tableNameBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              tableName_ != null &&
+              tableName_ != org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()) {
+            tableName_ =
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.newBuilder(tableName_).mergeFrom(value).buildPartial();
+          } else {
+            tableName_ = value;
+          }
+          onChanged();
+        } else {
+          tableNameBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public Builder clearTableName() {
+        if (tableNameBuilder_ == null) {
+          tableName_ = null;
+          onChanged();
+        } else {
+          tableNameBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder getTableNameBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getTableNameFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+        if (tableNameBuilder_ != null) {
+          return tableNameBuilder_.getMessageOrBuilder();
+        } else {
+          return tableName_ == null ?
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.getDefaultInstance() : tableName_;
+        }
+      }
+      /**
+       * <code>optional .hbase.pb.TableName table_name = 1;</code>
+       */
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+          getTableNameFieldBuilder() {
+        if (tableNameBuilder_ == null) {
+          tableNameBuilder_ = new org.apache.hadoop.hbase.shaded.com.google.protobuf.SingleFieldBuilderV3<
+              org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
+                  getTableName(),
+                  getParentForChildren(),
+                  isClean());
+          tableName_ = null;
+        }
+        return tableNameBuilder_;
+      }
+
+      private java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString> families_ = java.util.Collections.emptyList();
+      private void ensureFamiliesIsMutable() {
+        if (!((bitField0_ & 0x00000002) == 0x00000002)) {
+          families_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString>(families_);
+          bitField0_ |= 0x00000002;
+         }
+      }
+      /**
+       * <code>repeated bytes families = 2;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString>
+          getFamiliesList() {
+        return java.util.Collections.unmodifiableList(families_);
+      }
+      /**
+       * <code>repeated bytes families = 2;</code>
+       */
+      public int getFamiliesCount() {
+        return families_.size();
+      }
+      /**
+       * <code>repeated bytes families = 2;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getFamilies(int index) {
+        return families_.get(index);
+      }
+      /**
+       * <code>repeated bytes families = 2;</code>
+       */
+      public Builder setFamilies(
+          int index, org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureFamiliesIsMutable();
+        families_.set(index, value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated bytes families = 2;</code>
+       */
+      public Builder addFamilies(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureFamiliesIsMutable();
+        families_.add(value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated bytes families = 2;</code>
+       */
+      public Builder addAllFamilies(
+          java.lang.Iterable<? extends org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString> values) {
+        ensureFamiliesIsMutable();
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessageLite.Builder.addAll(
+            values, families_);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated bytes families = 2;</code>
+       */
+      public Builder clearFamilies() {
+        families_ = java.util.Collections.emptyList();
+        bitField0_ = (bitField0_ & ~0x00000002);
+        onChanged();
+        return this;
+      }
+      public final Builder setUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.setUnknownFields(unknownFields);
+      }
+
+      public final Builder mergeUnknownFields(
+          final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet unknownFields) {
+        return super.mergeUnknownFields(unknownFields);
+      }
+
+
+      // @@protoc_insertion_point(builder_scope:hbase.pb.TableCF)
+    }
+
+    // @@protoc_insertion_point(class_scope:hbase.pb.TableCF)
+    private static final org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF DEFAULT_INSTANCE;
+    static {
+      DEFAULT_INSTANCE = new org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF();
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF getDefaultInstance() {
+      return DEFAULT_INSTANCE;
+    }
+
+    @java.lang.Deprecated public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<TableCF>
+        PARSER = new org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractParser<TableCF>() {
+      public TableCF parsePartialFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+          return new TableCF(input, extensionRegistry);
+      }
+    };
+
+    public static org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<TableCF> parser() {
+      return PARSER;
+    }
+
+    @java.lang.Override
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.Parser<TableCF> getParserForType() {
+      return PARSER;
+    }
+
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF getDefaultInstanceForType() {
+      return DEFAULT_INSTANCE;
+    }
+
+  }
+
+  public interface ReplicationPeerOrBuilder extends
+      // @@protoc_insertion_point(interface_extends:hbase.pb.ReplicationPeer)
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.MessageOrBuilder {
+
+    /**
+     * <pre>
+     * clusterkey is the concatenation of the slave cluster's
+     * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+     * </pre>
+     *
+     * <code>required string clusterkey = 1;</code>
+     */
+    boolean hasClusterkey();
+    /**
+     * <pre>
+     * clusterkey is the concatenation of the slave cluster's
+     * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+     * </pre>
+     *
+     * <code>required string clusterkey = 1;</code>
+     */
+    java.lang.String getClusterkey();
+    /**
+     * <pre>
+     * clusterkey is the concatenation of the slave cluster's
+     * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+     * </pre>
+     *
+     * <code>required string clusterkey = 1;</code>
+     */
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+        getClusterkeyBytes();
+
+    /**
+     * <code>optional string replicationEndpointImpl = 2;</code>
+     */
+    boolean hasReplicationEndpointImpl();
+    /**
+     * <code>optional string replicationEndpointImpl = 2;</code>
+     */
+    java.lang.String getReplicationEndpointImpl();
+    /**
+     * <code>optional string replicationEndpointImpl = 2;</code>
+     */
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+        getReplicationEndpointImplBytes();
+
+    /**
+     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair> 
+        getDataList();
+    /**
+     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair getData(int index);
+    /**
+     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+     */
+    int getDataCount();
+    /**
+     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder> 
+        getDataOrBuilderList();
+    /**
+     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder getDataOrBuilder(
+        int index);
+
+    /**
+     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair> 
+        getConfigurationList();
+    /**
+     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair getConfiguration(int index);
+    /**
+     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
+     */
+    int getConfigurationCount();
+    /**
+     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder> 
+        getConfigurationOrBuilderList();
+    /**
+     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder getConfigurationOrBuilder(
+        int index);
+
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF> 
+        getTableCfsList();
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF getTableCfs(int index);
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    int getTableCfsCount();
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCFOrBuilder> 
+        getTableCfsOrBuilderList();
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCFOrBuilder getTableCfsOrBuilder(
+        int index);
+
+    /**
+     * <code>repeated bytes namespaces = 6;</code>
+     */
+    java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString> getNamespacesList();
+    /**
+     * <code>repeated bytes namespaces = 6;</code>
+     */
+    int getNamespacesCount();
+    /**
+     * <code>repeated bytes namespaces = 6;</code>
+     */
+    org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getNamespaces(int index);
+
+    /**
+     * <code>optional int64 bandwidth = 7;</code>
+     */
+    boolean hasBandwidth();
+    /**
+     * <code>optional int64 bandwidth = 7;</code>
+     */
+    long getBandwidth();
+  }
+  /**
+   * <pre>
+   **
+   * Used by replication. Holds a replication peer key.
+   * </pre>
+   *
+   * Protobuf type {@code hbase.pb.ReplicationPeer}
+   */
+  public  static final class ReplicationPeer extends
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3 implements
+      // @@protoc_insertion_point(message_implements:hbase.pb.ReplicationPeer)
+      ReplicationPeerOrBuilder {
+    // Use ReplicationPeer.newBuilder() to construct.
+    private ReplicationPeer(org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<?> builder) {
+      super(builder);
+    }
+    private ReplicationPeer() {
+      clusterkey_ = "";
+      replicationEndpointImpl_ = "";
+      data_ = java.util.Collections.emptyList();
+      configuration_ = java.util.Collections.emptyList();
+      tableCfs_ = java.util.Collections.emptyList();
+      namespaces_ = java.util.Collections.emptyList();
+      bandwidth_ = 0L;
+    }
+
+    @java.lang.Override
+    public final org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet
+    getUnknownFields() {
+      return this.unknownFields;
+    }
+    private ReplicationPeer(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      this();
+      int mutable_bitField0_ = 0;
+      org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
+              bitField0_ |= 0x00000001;
+              clusterkey_ = bs;
+              break;
+            }
+            case 18: {
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = input.readBytes();
+              bitField0_ |= 0x00000002;
+              replicationEndpointImpl_ = bs;
+              break;
+            }
+            case 26: {
+              if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+                data_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair>();
+                mutable_bitField0_ |= 0x00000004;
+              }
+              data_.add(
+                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair.PARSER, extensionRegistry));
+              break;
+            }
+            case 34: {
+              if (!((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
+                configuration_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair>();
+                mutable_bitField0_ |= 0x00000008;
+              }
+              configuration_.add(
+                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair.PARSER, extensionRegistry));
+              break;
+            }
+            case 42: {
+              if (!((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
+                tableCfs_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF>();
+                mutable_bitField0_ |= 0x00000010;
+              }
+              tableCfs_.add(
+                  input.readMessage(org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF.PARSER, extensionRegistry));
+              break;
+            }
+            case 50: {
+              if (!((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
+                namespaces_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString>();
+                mutable_bitField0_ |= 0x00000020;
+              }
+              namespaces_.add(input.readBytes());
+              break;
+            }
+            case 56: {
+              bitField0_ |= 0x00000004;
+              bandwidth_ = input.readInt64();
+              break;
+            }
+          }
+        }
+      } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException(
+            e).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+          data_ = java.util.Collections.unmodifiableList(data_);
+        }
+        if (((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
+          configuration_ = java.util.Collections.unmodifiableList(configuration_);
+        }
+        if (((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
+          tableCfs_ = java.util.Collections.unmodifiableList(tableCfs_);
+        }
+        if (((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
+          namespaces_ = java.util.Collections.unmodifiableList(namespaces_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.internal_static_hbase_pb_ReplicationPeer_descriptor;
+    }
+
+    protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.internal_static_hbase_pb_ReplicationPeer_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.class, org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.Builder.class);
+    }
+
+    private int bitField0_;
+    public static final int CLUSTERKEY_FIELD_NUMBER = 1;
+    private volatile java.lang.Object clusterkey_;
+    /**
+     * <pre>
+     * clusterkey is the concatenation of the slave cluster's
+     * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+     * </pre>
+     *
+     * <code>required string clusterkey = 1;</code>
+     */
+    public boolean hasClusterkey() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <pre>
+     * clusterkey is the concatenation of the slave cluster's
+     * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+     * </pre>
+     *
+     * <code>required string clusterkey = 1;</code>
+     */
+    public java.lang.String getClusterkey() {
+      java.lang.Object ref = clusterkey_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          clusterkey_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <pre>
+     * clusterkey is the concatenation of the slave cluster's
+     * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+     * </pre>
+     *
+     * <code>required string clusterkey = 1;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+        getClusterkeyBytes() {
+      java.lang.Object ref = clusterkey_;
+      if (ref instanceof java.lang.String) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        clusterkey_ = b;
+        return b;
+      } else {
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    public static final int REPLICATIONENDPOINTIMPL_FIELD_NUMBER = 2;
+    private volatile java.lang.Object replicationEndpointImpl_;
+    /**
+     * <code>optional string replicationEndpointImpl = 2;</code>
+     */
+    public boolean hasReplicationEndpointImpl() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional string replicationEndpointImpl = 2;</code>
+     */
+    public java.lang.String getReplicationEndpointImpl() {
+      java.lang.Object ref = replicationEndpointImpl_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs = 
+            (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          replicationEndpointImpl_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string replicationEndpointImpl = 2;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+        getReplicationEndpointImplBytes() {
+      java.lang.Object ref = replicationEndpointImpl_;
+      if (ref instanceof java.lang.String) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+            org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        replicationEndpointImpl_ = b;
+        return b;
+      } else {
+        return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    public static final int DATA_FIELD_NUMBER = 3;
+    private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair> data_;
+    /**
+     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair> getDataList() {
+      return data_;
+    }
+    /**
+     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder> 
+        getDataOrBuilderList() {
+      return data_;
+    }
+    /**
+     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+     */
+    public int getDataCount() {
+      return data_.size();
+    }
+    /**
+     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair getData(int index) {
+      return data_.get(index);
+    }
+    /**
+     * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder getDataOrBuilder(
+        int index) {
+      return data_.get(index);
+    }
+
+    public static final int CONFIGURATION_FIELD_NUMBER = 4;
+    private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair> configuration_;
+    /**
+     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair> getConfigurationList() {
+      return configuration_;
+    }
+    /**
+     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder> 
+        getConfigurationOrBuilderList() {
+      return configuration_;
+    }
+    /**
+     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
+     */
+    public int getConfigurationCount() {
+      return configuration_.size();
+    }
+    /**
+     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair getConfiguration(int index) {
+      return configuration_.get(index);
+    }
+    /**
+     * <code>repeated .hbase.pb.NameStringPair configuration = 4;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPairOrBuilder getConfigurationOrBuilder(
+        int index) {
+      return configuration_.get(index);
+    }
+
+    public static final int TABLE_CFS_FIELD_NUMBER = 5;
+    private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF> tableCfs_;
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF> getTableCfsList() {
+      return tableCfs_;
+    }
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    public java.util.List<? extends org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCFOrBuilder> 
+        getTableCfsOrBuilderList() {
+      return tableCfs_;
+    }
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    public int getTableCfsCount() {
+      return tableCfs_.size();
+    }
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCF getTableCfs(int index) {
+      return tableCfs_.get(index);
+    }
+    /**
+     * <code>repeated .hbase.pb.TableCF table_cfs = 5;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TableCFOrBuilder getTableCfsOrBuilder(
+        int index) {
+      return tableCfs_.get(index);
+    }
+
+    public static final int NAMESPACES_FIELD_NUMBER = 6;
+    private java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString> namespaces_;
+    /**
+     * <code>repeated bytes namespaces = 6;</code>
+     */
+    public java.util.List<org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString>
+        getNamespacesList() {
+      return namespaces_;
+    }
+    /**
+     * <code>repeated bytes namespaces = 6;</code>
+     */
+    public int getNamespacesCount() {
+      return namespaces_.size();
+    }
+    /**
+     * <code>repeated bytes namespaces = 6;</code>
+     */
+    public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString getNamespaces(int index) {
+      return namespaces_.get(index);
+    }
+
+    public static final int BANDWIDTH_FIELD_NUMBER = 7;
+    private long bandwidth_;
+    /**
+     * <code>optional int64 bandwidth = 7;</code>
+     */
+    public boolean hasBandwidth() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>optional int64 bandwidth = 7;</code>
+     */
+    public long getBandwidth() {
+      return bandwidth_;
+    }
+
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized == 1) return true;
+      if (isInitialized == 0) return false;
+
+      if (!hasClusterkey()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      for (int i = 0; i < getDataCount(); i++) {
+        if (!getData(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      for (int i = 0; i < getConfigurationCount(); i++) {
+        if (!getConfiguration(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      for (int i = 0; i < getTableCfsCount(); i++) {
+        if (!getTableCfs(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 1, clusterkey_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.writeString(output, 2, replicationEndpointImpl_);
+      }
+      for (int i = 0; i < data_.size(); i++) {
+        output.writeMessage(3, data_.get(i));
+      }
+      for (int i = 0; i < configuration_.size(); i++) {
+        output.writeMessage(4, configuration_.get(i));
+      }
+      for (int i = 0; i < tableCfs_.size(); i++) {
+        output.writeMessage(5, tableCfs_.get(i));
+      }
+      for (int i = 0; i < namespaces_.size(); i++) {
+        output.writeBytes(6, namespaces_.get(i));
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeInt64(7, bandwidth_);
+      }
+      unknownFields.writeTo(output);
+    }
+
+    public int getSerializedSize() {
+      int size = memoizedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(1, clusterkey_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.computeStringSize(2, replicationEndpointImpl_);
+      }
+      for (int i = 0; i < data_.size(); i++) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(3, data_.get(i));
+      }
+      for (int i = 0; i < configuration_.size(); i++) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(4, configuration_.get(i));
+      }
+      for (int i = 0; i < tableCfs_.size(); i++) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeMessageSize(5, tableCfs_.get(i));
+      }
+      {
+        int dataSize = 0;
+        for (int i = 0; i < namespaces_.size(); i++) {
+          dataSize += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+            .computeBytesSizeNoTag(namespaces_.get(i));
+        }
+        size += dataSize;
+        size += 1 * getNamespacesList().size();
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream
+          .computeInt64Size(7, bandwidth_);
+      }
+      size += unknownFields.getSerializedSize();
+      memoizedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer other = (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer) obj;
+
+      boolean result = true;
+      result = result && (hasClusterkey() == other.hasClusterkey());
+      if (hasClusterkey()) {
+        result = result && getClusterkey()
+            .equals(other.getClusterkey());
+      }
+      result = result && (hasReplicationEndpointImpl() == other.hasReplicationEndpointImpl());
+      if (hasReplicationEndpointImpl()) {
+        result = result && getReplicationEndpointImpl()
+            .equals(other.getReplicationEndpointImpl());
+      }
+      result = result && getDataList()
+          .equals(other.getDataList());
+      result = result && getConfigurationList()
+          .equals(other.getConfigurationList());
+      result = result && getTableCfsList()
+          .equals(other.getTableCfsList());
+      result = result && getNamespacesList()
+          .equals(other.getNamespacesList());
+      result = result && (hasBandwidth() == other.hasBandwidth());
+      if (hasBandwidth()) {
+        result = result && (getBandwidth()
+            == other.getBandwidth());
+      }
+      result = result && unknownFields.equals(other.unknownFields);
+      return result;
+    }
+
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasClusterkey()) {
+        hash = (37 * hash) + CLUSTERKEY_FIELD_NUMBER;
+        hash = (53 * hash) + getClusterkey().hashCode();
+      }
+      if (hasReplicationEndpointImpl()) {
+        hash = (37 * hash) + REPLICATIONENDPOINTIMPL_FIELD_NUMBER;
+        hash = (53 * hash) + getReplicationEndpointImpl().hashCode();
+      }
+      if (getDataCount() > 0) {
+        hash = (37 * hash) + DATA_FIELD_NUMBER;
+        hash = (53 * hash) + getDataList().hashCode();
+      }
+      if (getConfigurationCount() > 0) {
+        hash = (37 * hash) + CONFIGURATION_FIELD_NUMBER;
+        hash = (53 * hash) + getConfigurationList().hashCode();
+      }
+      if (getTableCfsCount() > 0) {
+        hash = (37 * hash) + TABLE_CFS_FIELD_NUMBER;
+        hash = (53 * hash) + getTableCfsList().hashCode();
+      }
+      if (getNamespacesCount() > 0) {
+        hash = (37 * hash) + NAMESPACES_FIELD_NUMBER;
+        hash = (53 * hash) + getNamespacesList().hashCode();
+      }
+      if (hasBandwidth()) {
+        hash = (37 * hash) + BANDWIDTH_FIELD_NUMBER;
+        hash = (53 * hash) + org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.hashLong(
+            getBandwidth());
+      }
+      hash = (29 * hash) + unknownFields.hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer parseFrom(byte[] data)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer parseFrom(
+        byte[] data,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer parseFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer parseDelimitedFrom(
+        java.io.InputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseDelimitedWithIOException(PARSER, input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input);
+    }
+    public static org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer parseFrom(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+          .parseWithIOException(PARSER, input, extensionRegistry);
+    }
+
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder() {
+      return DEFAULT_INSTANCE.toBuilder();
+    }
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer prototype) {
+      return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() {
+      return this == DEFAULT_INSTANCE
+          ? new Builder() : new Builder().mergeFrom(this);
+    }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * <pre>
+     **
+     * Used by replication. Holds a replication peer key.
+     * </pre>
+     *
+     * Protobuf type {@code hbase.pb.ReplicationPeer}
+     */
+    public static final class Builder extends
+        org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.Builder<Builder> implements
+        // @@protoc_insertion_point(builder_implements:hbase.pb.ReplicationPeer)
+        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeerOrBuilder {
+      public static final org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.internal_static_hbase_pb_ReplicationPeer_descriptor;
+      }
+
+      protected org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.internal_static_hbase_pb_ReplicationPeer_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.class, org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3
+                .alwaysUseFieldBuilders) {
+          getDataFieldBuilder();
+          getConfigurationFieldBuilder();
+          getTableCfsFieldBuilder();
+        }
+      }
+      public Builder clear() {
+        super.clear();
+        clusterkey_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        replicationEndpointImpl_ = "";
+        bitField0_ = (bitField0_ & ~0x00000002);
+        if (dataBuilder_ == null) {
+          data_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000004);
+        } else {
+          dataBuilder_.clear();
+        }
+        if (configurationBuilder_ == null) {
+          configuration_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000008);
+        } else {
+          configurationBuilder_.clear();
+        }
+        if (tableCfsBuilder_ == null) {
+          tableCfs_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000010);
+        } else {
+          tableCfsBuilder_.clear();
+        }
+        namespaces_ = java.util.Collections.emptyList();
+        bitField0_ = (bitField0_ & ~0x00000020);
+        bandwidth_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000040);
+        return this;
+      }
+
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.internal_static_hbase_pb_ReplicationPeer_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer build() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer buildPartial() {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer result = new org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.clusterkey_ = clusterkey_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.replicationEndpointImpl_ = replicationEndpointImpl_;
+        if (dataBuilder_ == null) {
+          if (((bitField0_ & 0x00000004) == 0x00000004)) {
+            data_ = java.util.Collections.unmodifiableList(data_);
+            bitField0_ = (bitField0_ & ~0x00000004);
+          }
+          result.data_ = data_;
+        } else {
+          result.data_ = dataBuilder_.build();
+        }
+        if (configurationBuilder_ == null) {
+          if (((bitField0_ & 0x00000008) == 0x00000008)) {
+            configuration_ = java.util.Collections.unmodifiableList(configuration_);
+            bitField0_ = (bitField0_ & ~0x00000008);
+          }
+          result.configuration_ = configuration_;
+        } else {
+          result.configuration_ = configurationBuilder_.build();
+        }
+        if (tableCfsBuilder_ == null) {
+          if (((bitField0_ & 0x00000010) == 0x00000010)) {
+            tableCfs_ = java.util.Collections.unmodifiableList(tableCfs_);
+            bitField0_ = (bitField0_ & ~0x00000010);
+          }
+          result.tableCfs_ = tableCfs_;
+        } else {
+          result.tableCfs_ = tableCfsBuilder_.build();
+        }
+        if (((bitField0_ & 0x00000020) == 0x00000020)) {
+          namespaces_ = java.util.Collections.unmodifiableList(namespaces_);
+          bitField0_ = (bitField0_ & ~0x00000020);
+        }
+        result.namespaces_ = namespaces_;
+        if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.bandwidth_ = bandwidth_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder clone() {
+        return (Builder) super.clone();
+      }
+      public Builder setField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.setField(field, value);
+      }
+      public Builder clearField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field) {
+        return (Builder) super.clearField(field);
+      }
+      public Builder clearOneof(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor oneof) {
+        return (Builder) super.clearOneof(oneof);
+      }
+      public Builder setRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          int index, Object value) {
+        return (Builder) super.setRepeatedField(field, index, value);
+      }
+      public Builder addRepeatedField(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor field,
+          Object value) {
+        return (Builder) super.addRepeatedField(field, value);
+      }
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer other) {
+        if (other == org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer.getDefaultInstance()) return this;
+        if (other.hasClusterkey()) {
+          bitField0_ |= 0x00000001;
+          clusterkey_ = other.clusterkey_;
+          onChanged();
+        }
+        if (other.hasReplicationEndpointImpl()) {
+          bitField0_ |= 0x00000002;
+          replicationEndpointImpl_ = other.replicationEndpointImpl_;
+          onChanged();
+        }
+        if (dataBuilder_ == null) {
+          if (!other.data_.isEmpty()) {
+            if (data_.isEmpty()) {
+              data_ = other.data_;
+              bitField0_ = (bitField0_ & ~0x00000004);
+            } else {
+              ensureDataIsMutable();
+              data_.addAll(other.data_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.data_.isEmpty()) {
+            if (dataBuilder_.isEmpty()) {
+              dataBuilder_.dispose();
+              dataBuilder_ = null;
+              data_ = other.data_;
+              bitField0_ = (bitField0_ & ~0x00000004);
+              dataBuilder_ = 
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                   getDataFieldBuilder() : null;
+            } else {
+              dataBuilder_.addAllMessages(other.data_);
+            }
+          }
+        }
+        if (configurationBuilder_ == null) {
+          if (!other.configuration_.isEmpty()) {
+            if (configuration_.isEmpty()) {
+              configuration_ = other.configuration_;
+              bitField0_ = (bitField0_ & ~0x00000008);
+            } else {
+              ensureConfigurationIsMutable();
+              configuration_.addAll(other.configuration_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.configuration_.isEmpty()) {
+            if (configurationBuilder_.isEmpty()) {
+              configurationBuilder_.dispose();
+              configurationBuilder_ = null;
+              configuration_ = other.configuration_;
+              bitField0_ = (bitField0_ & ~0x00000008);
+              configurationBuilder_ = 
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                   getConfigurationFieldBuilder() : null;
+            } else {
+              configurationBuilder_.addAllMessages(other.configuration_);
+            }
+          }
+        }
+        if (tableCfsBuilder_ == null) {
+          if (!other.tableCfs_.isEmpty()) {
+            if (tableCfs_.isEmpty()) {
+              tableCfs_ = other.tableCfs_;
+              bitField0_ = (bitField0_ & ~0x00000010);
+            } else {
+              ensureTableCfsIsMutable();
+              tableCfs_.addAll(other.tableCfs_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.tableCfs_.isEmpty()) {
+            if (tableCfsBuilder_.isEmpty()) {
+              tableCfsBuilder_.dispose();
+              tableCfsBuilder_ = null;
+              tableCfs_ = other.tableCfs_;
+              bitField0_ = (bitField0_ & ~0x00000010);
+              tableCfsBuilder_ = 
+                org.apache.hadoop.hbase.shaded.com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders ?
+                   getTableCfsFieldBuilder() : null;
+            } else {
+              tableCfsBuilder_.addAllMessages(other.tableCfs_);
+            }
+          }
+        }
+        if (!other.namespaces_.isEmpty()) {
+          if (namespaces_.isEmpty()) {
+            namespaces_ = other.namespaces_;
+            bitField0_ = (bitField0_ & ~0x00000020);
+          } else {
+            ensureNamespacesIsMutable();
+            namespaces_.addAll(other.namespaces_);
+          }
+          onChanged();
+        }
+        if (other.hasBandwidth()) {
+          setBandwidth(other.getBandwidth());
+        }
+        this.mergeUnknownFields(other.unknownFields);
+        onChanged();
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasClusterkey()) {
+          return false;
+        }
+        for (int i = 0; i < getDataCount(); i++) {
+          if (!getData(i).isInitialized()) {
+            return false;
+          }
+        }
+        for (int i = 0; i < getConfigurationCount(); i++) {
+          if (!getConfiguration(i).isInitialized()) {
+            return false;
+          }
+        }
+        for (int i = 0; i < getTableCfsCount(); i++) {
+          if (!getTableCfs(i).isInitialized()) {
+            return false;
+          }
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream input,
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationPeer) e.getUnfinishedMessage();
+          throw e.unwrapIOException();
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      private java.lang.Object clusterkey_ = "";
+      /**
+       * <pre>
+       * clusterkey is the concatenation of the slave cluster's
+       * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+       * </pre>
+       *
+       * <code>required string clusterkey = 1;</code>
+       */
+      public boolean hasClusterkey() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <pre>
+       * clusterkey is the concatenation of the slave cluster's
+       * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+       * </pre>
+       *
+       * <code>required string clusterkey = 1;</code>
+       */
+      public java.lang.String getClusterkey() {
+        java.lang.Object ref = clusterkey_;
+        if (!(ref instanceof java.lang.String)) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          if (bs.isValidUtf8()) {
+            clusterkey_ = s;
+          }
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <pre>
+       * clusterkey is the concatenation of the slave cluster's
+       * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+       * </pre>
+       *
+       * <code>required string clusterkey = 1;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+          getClusterkeyBytes() {
+        java.lang.Object ref = clusterkey_;
+        if (ref instanceof String) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          clusterkey_ = b;
+          return b;
+        } else {
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <pre>
+       * clusterkey is the concatenation of the slave cluster's
+       * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+       * </pre>
+       *
+       * <code>required string clusterkey = 1;</code>
+       */
+      public Builder setClusterkey(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        clusterkey_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <pre>
+       * clusterkey is the concatenation of the slave cluster's
+       * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+       * </pre>
+       *
+       * <code>required string clusterkey = 1;</code>
+       */
+      public Builder clearClusterkey() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        clusterkey_ = getDefaultInstance().getClusterkey();
+        onChanged();
+        return this;
+      }
+      /**
+       * <pre>
+       * clusterkey is the concatenation of the slave cluster's
+       * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+       * </pre>
+       *
+       * <code>required string clusterkey = 1;</code>
+       */
+      public Builder setClusterkeyBytes(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        clusterkey_ = value;
+        onChanged();
+        return this;
+      }
+
+      private java.lang.Object replicationEndpointImpl_ = "";
+      /**
+       * <code>optional string replicationEndpointImpl = 2;</code>
+       */
+      public boolean hasReplicationEndpointImpl() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional string replicationEndpointImpl = 2;</code>
+       */
+      public java.lang.String getReplicationEndpointImpl() {
+        java.lang.Object ref = replicationEndpointImpl_;
+        if (!(ref instanceof java.lang.String)) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString bs =
+              (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
+          java.lang.String s = bs.toStringUtf8();
+          if (bs.isValidUtf8()) {
+            replicationEndpointImpl_ = s;
+          }
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string replicationEndpointImpl = 2;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString
+          getReplicationEndpointImplBytes() {
+        java.lang.Object ref = replicationEndpointImpl_;
+        if (ref instanceof String) {
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString b = 
+              org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          replicationEndpointImpl_ = b;
+          return b;
+        } else {
+          return (org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string replicationEndpointImpl = 2;</code>
+       */
+      public Builder setReplicationEndpointImpl(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        replicationEndpointImpl_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string replicationEndpointImpl = 2;</code>
+       */
+      public Builder clearReplicationEndpointImpl() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        replicationEndpointImpl_ = getDefaultInstance().getReplicationEndpointImpl();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string replicationEndpointImpl = 2;</code>
+       */
+      public Builder setReplicationEndpointImplBytes(
+          org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        replicationEndpointImpl_ = value;
+        onChanged();
+        return this;
+      }
+
+      private java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair> data_ =
+        java.util.Collections.emptyList();
+      private void ensureDataIsMutable() {
+        if (!((bitField0_ & 0x00000004) == 0x00000004)) {
+          data_ = new java.util.ArrayList<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair>(data_);
+          bitField0_ |= 0x00000004;
+         }
+      }
+
+      private org.apache.hadoop.hbase.shaded.com.google.protobuf.RepeatedFieldBuilderV3<
+          org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair.Builder, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder> dataBuilder_;
+
+      /**
+       * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+       */
+      public java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair> getDataList() {
+        if (dataBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(data_);
+        } else {
+          return dataBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+       */
+      public int getDataCount() {
+        if (dataBuilder_ == null) {
+          return data_.size();
+        } else {
+          return dataBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+       */
+      public org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair getData(int index) {
+        if (dataBuilder_ == null) {
+          return data_.get(index);
+        } else {
+          return dataBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+       */
+      public Builder setData(
+          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair value) {
+        if (dataBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureDataIsMutable();
+          data_.set(index, value);
+          onChanged();
+        } else {
+          dataBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+       */
+      public Builder setData(
+          int index, org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair.Builder builderForValue) {
+        if (dataBuilder_ == null) {
+          ensureDataIsMutable();
+          data_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          dataBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .hbase.pb.BytesBytesPair data = 3;</code>
+       */
+      public Builder addData(org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BytesBytesPair value) {
+        if (dataBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+      

<TRUNCATED>

[10/16] hbase git commit: HBASE-15172 Support setting storage policy in bulkload

Posted by sy...@apache.org.
HBASE-15172 Support setting storage policy in bulkload


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

Branch: refs/heads/hbase-12439
Commit: 629b04f44f19b9589c9bcfb84da0cf5e0d4d1f18
Parents: e02ae77
Author: Yu Li <li...@apache.org>
Authored: Fri Jan 6 18:35:38 2017 +0800
Committer: Yu Li <li...@apache.org>
Committed: Fri Jan 6 18:35:38 2017 +0800

----------------------------------------------------------------------
 .../hbase/mapreduce/HFileOutputFormat2.java     | 31 ++++++++-
 .../hbase/mapreduce/TestHFileOutputFormat2.java | 70 ++++++++++++++++++++
 2 files changed, 100 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/629b04f4/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
index 22a73c9..6987bf7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
@@ -68,6 +68,7 @@ import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
@@ -125,6 +126,9 @@ public class HFileOutputFormat2
   private static final String OUTPUT_TABLE_NAME_CONF_KEY =
       "hbase.mapreduce.hfileoutputformat.table.name";
 
+  public static final String STORAGE_POLICY_PROPERTY = "hbase.hstore.storagepolicy";
+  public static final String STORAGE_POLICY_PROPERTY_CF_PREFIX = STORAGE_POLICY_PROPERTY + ".";
+
   @Override
   public RecordWriter<ImmutableBytesWritable, Cell> getRecordWriter(
       final TaskAttemptContext context) throws IOException, InterruptedException {
@@ -230,7 +234,9 @@ public class HFileOutputFormat2
 
       // If this is a new column family, verify that the directory exists
       if (wl == null) {
-        fs.mkdirs(new Path(outputDir, Bytes.toString(family)));
+        Path cfPath = new Path(outputDir, Bytes.toString(family));
+        fs.mkdirs(cfPath);
+        configureStoragePolicy(conf, fs, family, cfPath);
       }
 
       // If any of the HFiles for the column families has reached
@@ -382,6 +388,29 @@ public class HFileOutputFormat2
     }
   }
 
+  /**
+   * Configure block storage policy for CF after the directory is created.
+   */
+  static void configureStoragePolicy(final Configuration conf, final FileSystem fs,
+      byte[] family, Path cfPath) {
+    if (null == conf || null == fs || null == family || null == cfPath) {
+      return;
+    }
+
+    String policy =
+        conf.get(STORAGE_POLICY_PROPERTY_CF_PREFIX + Bytes.toString(family),
+          conf.get(STORAGE_POLICY_PROPERTY));
+    if (null != policy && !policy.trim().isEmpty()) {
+      try {
+        if (fs instanceof DistributedFileSystem) {
+          ((DistributedFileSystem) fs).setStoragePolicy(cfPath, policy.trim());
+        }
+      } catch (Throwable e) {
+        LOG.warn("failed to set block storage policy of [" + cfPath + "] to [" + policy + "]", e);
+      }
+    }
+  }
+
   /*
    * Data structure to hold a Writer and amount of data written on it.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/629b04f4/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
index 486c961..21a39d4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -92,6 +93,10 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.util.Writables;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Mapper;
@@ -1292,5 +1297,70 @@ public class TestHFileOutputFormat2  {
     }
   }
 
+  @Test
+  public void testBlockStoragePolicy() throws Exception {
+    util = new HBaseTestingUtility();
+    Configuration conf = util.getConfiguration();
+    conf.set(HFileOutputFormat2.STORAGE_POLICY_PROPERTY, "ALL_SSD");
+    conf.set(HFileOutputFormat2.STORAGE_POLICY_PROPERTY_CF_PREFIX + Bytes.toString(FAMILIES[0]),
+      "ONE_SSD");
+    Path cf1Dir = new Path(util.getDataTestDir(), Bytes.toString(FAMILIES[0]));
+    Path cf2Dir = new Path(util.getDataTestDir(), Bytes.toString(FAMILIES[1]));
+    util.startMiniDFSCluster(3);
+    FileSystem fs = util.getDFSCluster().getFileSystem();
+    try {
+      fs.mkdirs(cf1Dir);
+      fs.mkdirs(cf2Dir);
+
+      // the original block storage policy would be NULL
+      String spA = getStoragePolicyName(fs, cf1Dir);
+      String spB = getStoragePolicyName(fs, cf2Dir);
+      LOG.debug("Storage policy of cf 0: [" + spA + "].");
+      LOG.debug("Storage policy of cf 1: [" + spB + "].");
+      assertNull(spA);
+      assertNull(spB);
+
+      // alter table cf schema to change storage policies
+      HFileOutputFormat2.configureStoragePolicy(conf, fs, FAMILIES[0], cf1Dir);
+      HFileOutputFormat2.configureStoragePolicy(conf, fs, FAMILIES[1], cf2Dir);
+      spA = getStoragePolicyName(fs, cf1Dir);
+      spB = getStoragePolicyName(fs, cf2Dir);
+      LOG.debug("Storage policy of cf 0: [" + spA + "].");
+      LOG.debug("Storage policy of cf 1: [" + spB + "].");
+      assertNotNull(spA);
+      assertEquals("ONE_SSD", spA);
+      assertNotNull(spB);
+      assertEquals("ALL_SSD", spB);
+    } finally {
+      fs.delete(cf1Dir, true);
+      fs.delete(cf2Dir, true);
+      util.shutdownMiniDFSCluster();
+    }
+  }
+
+  private String getStoragePolicyName(FileSystem fs, Path path) {
+    try {
+      if (fs instanceof DistributedFileSystem) {
+        DistributedFileSystem dfs = (DistributedFileSystem) fs;
+        HdfsFileStatus status = dfs.getClient().getFileInfo(path.toUri().getPath());
+        if (null != status) {
+          byte storagePolicyId = status.getStoragePolicy();
+          if (storagePolicyId != BlockStoragePolicySuite.ID_UNSPECIFIED) {
+            BlockStoragePolicy[] policies = dfs.getStoragePolicies();
+            for (BlockStoragePolicy policy : policies) {
+              if (policy.getId() == storagePolicyId) {
+                return policy.getName();
+              }
+            }
+          }
+        }
+      }
+    } catch (Throwable e) {
+      LOG.warn("failed to get block storage policy of [" + path + "]", e);
+    }
+
+    return null;
+  }
+
 }
 


[15/16] hbase git commit: HBASE-17408 Introduce per request limit by number of mutations (ChiaPing Tsai)

Posted by sy...@apache.org.
HBASE-17408 Introduce per request limit by number of mutations (ChiaPing Tsai)


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

Branch: refs/heads/hbase-12439
Commit: b2a9be02ac10908438dc84e9d483bc8785d7ce19
Parents: 4c98f97
Author: tedyu <yu...@gmail.com>
Authored: Fri Jan 6 13:07:15 2017 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Fri Jan 6 13:07:15 2017 -0800

----------------------------------------------------------------------
 .../hbase/client/SimpleRequestController.java   | 164 +++++++++++++------
 .../client/TestSimpleRequestController.java     |  91 +++++++---
 .../hbase/coprocessor/RegionServerObserver.java |   2 +-
 3 files changed, 182 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b2a9be02/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SimpleRequestController.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SimpleRequestController.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SimpleRequestController.java
index 6343af6..de2cbe1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SimpleRequestController.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SimpleRequestController.java
@@ -49,30 +49,39 @@ import org.apache.hadoop.hbase.util.EnvironmentEdge;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
 /**
- * Holds back the request if the submitted size or number has reached the
- * threshold.
+ * Holds back the requests if they reach any thresholds.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 class SimpleRequestController implements RequestController {
   private static final Log LOG = LogFactory.getLog(SimpleRequestController.class);
   /**
-   * The maximum size of single RegionServer.
+   * The maximum heap size for each request.
    */
   public static final String HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE = "hbase.client.max.perrequest.heapsize";
 
   /**
-   * Default value of #HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE
+   * Default value of {@link #HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE}.
    */
   @VisibleForTesting
   static final long DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE = 4194304;
 
   /**
+   * The maximum number of rows for each request.
+   */
+  public static final String HBASE_CLIENT_MAX_PERREQUEST_ROWS = "hbase.client.max.perrequest.rows";
+  /**
+   * Default value of {@link #HBASE_CLIENT_MAX_PERREQUEST_ROWS}.
+   */
+  @VisibleForTesting
+  static final long DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_ROWS = 2048;
+
+  /**
    * The maximum size of submit.
    */
   public static final String HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE = "hbase.client.max.submit.heapsize";
   /**
-   * Default value of #HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE
+   * Default value of {@link #HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE}.
    */
   @VisibleForTesting
   static final long DEFAULT_HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE = DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE;
@@ -89,9 +98,13 @@ class SimpleRequestController implements RequestController {
   private final int maxTotalConcurrentTasks;
 
   /**
-   * The max heap size of all tasks simultaneously executed on a server.
+   * The maximum heap size for each request.
    */
   private final long maxHeapSizePerRequest;
+  /**
+   * The maximum number of rows for each request.
+   */
+  private final long maxRowsPerRequest;
   private final long maxHeapSizeSubmit;
   /**
    * The number of tasks we run in parallel on a single region. With 1 (the
@@ -116,41 +129,46 @@ class SimpleRequestController implements RequestController {
   private static final int DEFAULT_THRESHOLD_TO_LOG_REGION_DETAILS = 2;
   private final int thresholdToLogRegionDetails;
   SimpleRequestController(final Configuration conf) {
-    this.maxTotalConcurrentTasks = conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS,
+    this.maxTotalConcurrentTasks = checkAndGet(conf,
+            HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS,
             HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS);
-    this.maxConcurrentTasksPerServer = conf.getInt(HConstants.HBASE_CLIENT_MAX_PERSERVER_TASKS,
+    this.maxConcurrentTasksPerServer = checkAndGet(conf,
+            HConstants.HBASE_CLIENT_MAX_PERSERVER_TASKS,
             HConstants.DEFAULT_HBASE_CLIENT_MAX_PERSERVER_TASKS);
-    this.maxConcurrentTasksPerRegion = conf.getInt(HConstants.HBASE_CLIENT_MAX_PERREGION_TASKS,
+    this.maxConcurrentTasksPerRegion = checkAndGet(conf,
+            HConstants.HBASE_CLIENT_MAX_PERREGION_TASKS,
             HConstants.DEFAULT_HBASE_CLIENT_MAX_PERREGION_TASKS);
-    this.maxHeapSizePerRequest = conf.getLong(HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE,
+    this.maxHeapSizePerRequest = checkAndGet(conf,
+            HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE,
             DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE);
-    this.maxHeapSizeSubmit = conf.getLong(HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE, DEFAULT_HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE);
-    this.thresholdToLogUndoneTaskDetails =
-        conf.getInt(THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS,
+    this.maxRowsPerRequest = checkAndGet(conf,
+            HBASE_CLIENT_MAX_PERREQUEST_ROWS,
+            DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_ROWS);
+    this.maxHeapSizeSubmit = checkAndGet(conf,
+            HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE,
+            DEFAULT_HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE);
+    this.thresholdToLogUndoneTaskDetails = conf.getInt(
+          THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS,
           DEFAULT_THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS);
-    this.thresholdToLogRegionDetails =
-        conf.getInt(THRESHOLD_TO_LOG_REGION_DETAILS,
+    this.thresholdToLogRegionDetails = conf.getInt(
+          THRESHOLD_TO_LOG_REGION_DETAILS,
           DEFAULT_THRESHOLD_TO_LOG_REGION_DETAILS);
-    if (this.maxTotalConcurrentTasks <= 0) {
-      throw new IllegalArgumentException("maxTotalConcurrentTasks=" + maxTotalConcurrentTasks);
-    }
-    if (this.maxConcurrentTasksPerServer <= 0) {
-      throw new IllegalArgumentException("maxConcurrentTasksPerServer="
-              + maxConcurrentTasksPerServer);
-    }
-    if (this.maxConcurrentTasksPerRegion <= 0) {
-      throw new IllegalArgumentException("maxConcurrentTasksPerRegion="
-              + maxConcurrentTasksPerRegion);
-    }
-    if (this.maxHeapSizePerRequest <= 0) {
-      throw new IllegalArgumentException("maxHeapSizePerServer="
-              + maxHeapSizePerRequest);
+  }
+
+  private static int checkAndGet(Configuration conf, String key, int defaultValue) {
+    int value = conf.getInt(key, defaultValue);
+    if (value <= 0) {
+      throw new IllegalArgumentException(key + "=" + value);
     }
+    return value;
+  }
 
-    if (this.maxHeapSizeSubmit <= 0) {
-      throw new IllegalArgumentException("maxHeapSizeSubmit="
-              + maxHeapSizeSubmit);
+  private static long checkAndGet(Configuration conf, String key, long defaultValue) {
+    long value = conf.getLong(key, defaultValue);
+    if (value <= 0) {
+      throw new IllegalArgumentException(key + "=" + value);
     }
+    return value;
   }
 
   @VisibleForTesting
@@ -163,10 +181,10 @@ class SimpleRequestController implements RequestController {
         if (isEnd) {
           return ReturnCode.END;
         }
-        long rowSize = (row instanceof Mutation) ? ((Mutation) row).heapSize() : 0;
+        long heapSizeOfRow = (row instanceof Mutation) ? ((Mutation) row).heapSize() : 0;
         ReturnCode code = ReturnCode.INCLUDE;
         for (RowChecker checker : checkers) {
-          switch (checker.canTakeOperation(loc, rowSize)) {
+          switch (checker.canTakeOperation(loc, heapSizeOfRow)) {
             case END:
               isEnd = true;
               code = ReturnCode.END;
@@ -183,7 +201,7 @@ class SimpleRequestController implements RequestController {
           }
         }
         for (RowChecker checker : checkers) {
-          checker.notifyFinal(code, loc, rowSize);
+          checker.notifyFinal(code, loc, heapSizeOfRow);
         }
         return code;
       }
@@ -208,15 +226,16 @@ class SimpleRequestController implements RequestController {
 
   @Override
   public Checker newChecker() {
-    List<RowChecker> checkers = new ArrayList<>(3);
+    List<RowChecker> checkers = new ArrayList<>(4);
     checkers.add(new TaskCountChecker(maxTotalConcurrentTasks,
             maxConcurrentTasksPerServer,
             maxConcurrentTasksPerRegion,
             tasksInProgress,
             taskCounterPerServer,
             taskCounterPerRegion));
-    checkers.add(new RequestSizeChecker(maxHeapSizePerRequest));
+    checkers.add(new RequestHeapSizeChecker(maxHeapSizePerRequest));
     checkers.add(new SubmittedSizeChecker(maxHeapSizeSubmit));
+    checkers.add(new RequestRowsChecker(maxRowsPerRequest));
     return newChecker(checkers);
   }
 
@@ -323,7 +342,7 @@ class SimpleRequestController implements RequestController {
     }
 
     @Override
-    public ReturnCode canTakeOperation(HRegionLocation loc, long rowSize) {
+    public ReturnCode canTakeOperation(HRegionLocation loc, long heapSizeOfRow) {
       if (heapSize >= maxHeapSizeSubmit) {
         return ReturnCode.END;
       }
@@ -331,9 +350,9 @@ class SimpleRequestController implements RequestController {
     }
 
     @Override
-    public void notifyFinal(ReturnCode code, HRegionLocation loc, long rowSize) {
+    public void notifyFinal(ReturnCode code, HRegionLocation loc, long heapSizeOfRow) {
       if (code == ReturnCode.INCLUDE) {
-        heapSize += rowSize;
+        heapSize += heapSizeOfRow;
       }
     }
 
@@ -413,11 +432,11 @@ class SimpleRequestController implements RequestController {
      * tasks for server.
      *
      * @param loc
-     * @param rowSize
+     * @param heapSizeOfRow
      * @return
      */
     @Override
-    public ReturnCode canTakeOperation(HRegionLocation loc, long rowSize) {
+    public ReturnCode canTakeOperation(HRegionLocation loc, long heapSizeOfRow) {
 
       HRegionInfo regionInfo = loc.getRegionInfo();
       if (regionsIncluded.contains(regionInfo)) {
@@ -444,7 +463,7 @@ class SimpleRequestController implements RequestController {
     }
 
     @Override
-    public void notifyFinal(ReturnCode code, HRegionLocation loc, long rowSize) {
+    public void notifyFinal(ReturnCode code, HRegionLocation loc, long heapSizeOfRow) {
       if (code == ReturnCode.INCLUDE) {
         regionsIncluded.add(loc.getRegionInfo());
         serversIncluded.add(loc.getServerName());
@@ -454,15 +473,54 @@ class SimpleRequestController implements RequestController {
   }
 
   /**
-   * limit the request size for each regionserver.
+   * limit the number of rows for each request.
+   */
+  @VisibleForTesting
+  static class RequestRowsChecker implements RowChecker {
+
+    private final long maxRowsPerRequest;
+    private final Map<ServerName, Long> serverRows = new HashMap<>();
+
+    RequestRowsChecker(final long maxRowsPerRequest) {
+      this.maxRowsPerRequest = maxRowsPerRequest;
+    }
+
+    @Override
+    public void reset() {
+      serverRows.clear();
+    }
+
+    @Override
+    public ReturnCode canTakeOperation(HRegionLocation loc, long heapSizeOfRow) {
+      long currentRows = serverRows.containsKey(loc.getServerName())
+              ? serverRows.get(loc.getServerName()) : 0L;
+      // accept at least one row
+      if (currentRows == 0 || currentRows < maxRowsPerRequest) {
+        return ReturnCode.INCLUDE;
+      }
+      return ReturnCode.SKIP;
+    }
+
+    @Override
+    public void notifyFinal(ReturnCode code, HRegionLocation loc, long heapSizeOfRow) {
+      if (code == ReturnCode.INCLUDE) {
+        long currentRows = serverRows.containsKey(loc.getServerName())
+                ? serverRows.get(loc.getServerName()) : 0L;
+        serverRows.put(loc.getServerName(), currentRows + 1);
+      }
+    }
+  }
+
+  /**
+   * limit the heap size for each request.
    */
   @VisibleForTesting
-  static class RequestSizeChecker implements RowChecker {
+  static class RequestHeapSizeChecker implements RowChecker {
 
     private final long maxHeapSizePerRequest;
     private final Map<ServerName, Long> serverRequestSizes = new HashMap<>();
 
-    RequestSizeChecker(final long maxHeapSizePerRequest) {
+    RequestHeapSizeChecker(final long maxHeapSizePerRequest) {
       this.maxHeapSizePerRequest = maxHeapSizePerRequest;
     }
 
@@ -472,23 +530,23 @@ class SimpleRequestController implements RequestController {
     }
 
     @Override
-    public ReturnCode canTakeOperation(HRegionLocation loc, long rowSize) {
+    public ReturnCode canTakeOperation(HRegionLocation loc, long heapSizeOfRow) {
       // Is it ok for limit of request size?
       long currentRequestSize = serverRequestSizes.containsKey(loc.getServerName())
               ? serverRequestSizes.get(loc.getServerName()) : 0L;
       // accept at least one request
-      if (currentRequestSize == 0 || currentRequestSize + rowSize <= maxHeapSizePerRequest) {
+      if (currentRequestSize == 0 || currentRequestSize + heapSizeOfRow <= maxHeapSizePerRequest) {
         return ReturnCode.INCLUDE;
       }
       return ReturnCode.SKIP;
     }
 
     @Override
-    public void notifyFinal(ReturnCode code, HRegionLocation loc, long rowSize) {
+    public void notifyFinal(ReturnCode code, HRegionLocation loc, long heapSizeOfRow) {
       if (code == ReturnCode.INCLUDE) {
         long currentRequestSize = serverRequestSizes.containsKey(loc.getServerName())
                 ? serverRequestSizes.get(loc.getServerName()) : 0L;
-        serverRequestSizes.put(loc.getServerName(), currentRequestSize + rowSize);
+        serverRequestSizes.put(loc.getServerName(), currentRequestSize + heapSizeOfRow);
       }
     }
   }
@@ -499,7 +557,7 @@ class SimpleRequestController implements RequestController {
   @VisibleForTesting
   interface RowChecker {
 
-    ReturnCode canTakeOperation(HRegionLocation loc, long rowSize);
+    ReturnCode canTakeOperation(HRegionLocation loc, long heapSizeOfRow);
 
     /**
      * Add the final ReturnCode to the checker. The ReturnCode may be reversed,
@@ -507,9 +565,9 @@ class SimpleRequestController implements RequestController {
      *
      * @param code The final decision
      * @param loc the destination of data
-     * @param rowSize the data size
+     * @param heapSizeOfRow the data size
      */
-    void notifyFinal(ReturnCode code, HRegionLocation loc, long rowSize);
+    void notifyFinal(ReturnCode code, HRegionLocation loc, long heapSizeOfRow);
 
     /**
      * Reset the inner state.

http://git-wip-us.apache.org/repos/asf/hbase/blob/b2a9be02/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSimpleRequestController.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSimpleRequestController.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSimpleRequestController.java
index b46e572..c0ece3c 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSimpleRequestController.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSimpleRequestController.java
@@ -56,7 +56,6 @@ public class TestSimpleRequestController {
   private static final byte[] DUMMY_BYTES_3 = "DUMMY_BYTES_3".getBytes();
   private static final ServerName SN = ServerName.valueOf("s1:1,1");
   private static final ServerName SN2 = ServerName.valueOf("s2:2,2");
-  private static final ServerName SN3 = ServerName.valueOf("s3:3,3");
   private static final HRegionInfo HRI1
           = new HRegionInfo(DUMMY_TABLE, DUMMY_BYTES_1, DUMMY_BYTES_2, false, 1);
   private static final HRegionInfo HRI2
@@ -68,7 +67,7 @@ public class TestSimpleRequestController {
   private static final HRegionLocation LOC3 = new HRegionLocation(HRI3, SN2);
 
   @Test
-  public void testIllegalRequestSize() {
+  public void testIllegalRequestHeapSize() {
     testIllegalArgument(SimpleRequestController.HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE, -1);
   }
 
@@ -87,9 +86,14 @@ public class TestSimpleRequestController {
     testIllegalArgument(SimpleRequestController.HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE, -1);
   }
 
+  @Test
+  public void testIllegalRequestRows() {
+    testIllegalArgument(SimpleRequestController.HBASE_CLIENT_MAX_PERREQUEST_ROWS, -1);
+  }
+
   private void testIllegalArgument(String key, long value) {
     Configuration conf = HBaseConfiguration.create();
-    conf.setLong(SimpleRequestController.HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE, -1);
+    conf.setLong(key, value);
     try {
       SimpleRequestController controller = new SimpleRequestController(conf);
       fail("The " + key + " must be bigger than zero");
@@ -121,7 +125,7 @@ public class TestSimpleRequestController {
             tasksInProgress, taskCounterPerServer, taskCounterPerRegion);
     final long maxHeapSizePerRequest = 2 * 1024 * 1024;
     // unlimiited
-    SimpleRequestController.RequestSizeChecker sizeChecker = new SimpleRequestController.RequestSizeChecker(maxHeapSizePerRequest);
+    SimpleRequestController.RequestHeapSizeChecker sizeChecker = new SimpleRequestController.RequestHeapSizeChecker(maxHeapSizePerRequest);
     RequestController.Checker checker = SimpleRequestController.newChecker(Arrays.asList(countChecker, sizeChecker));
     ReturnCode loc1Code = checker.canTakeRow(LOC1, createPut(maxHeapSizePerRequest));
     assertEquals(ReturnCode.INCLUDE, loc1Code);
@@ -151,10 +155,10 @@ public class TestSimpleRequestController {
   }
 
   @Test
-  public void testRequestSizeCheckerr() throws IOException {
+  public void testRequestHeapSizeChecker() throws IOException {
     final long maxHeapSizePerRequest = 2 * 1024 * 1024;
-    SimpleRequestController.RequestSizeChecker checker
-            = new SimpleRequestController.RequestSizeChecker(maxHeapSizePerRequest);
+    SimpleRequestController.RequestHeapSizeChecker checker
+            = new SimpleRequestController.RequestHeapSizeChecker(maxHeapSizePerRequest);
 
     // inner state is unchanged.
     for (int i = 0; i != 10; ++i) {
@@ -193,6 +197,51 @@ public class TestSimpleRequestController {
   }
 
   @Test
+  public void testRequestRowsChecker() throws IOException {
+    final long maxRowCount = 100;
+    SimpleRequestController.RequestRowsChecker checker
+      = new SimpleRequestController.RequestRowsChecker(maxRowCount);
+
+    final long heapSizeOfRow = 100; //unused
+    // inner state is unchanged.
+    for (int i = 0; i != 10; ++i) {
+      ReturnCode code = checker.canTakeOperation(LOC1, heapSizeOfRow);
+      assertEquals(ReturnCode.INCLUDE, code);
+      code = checker.canTakeOperation(LOC2, heapSizeOfRow);
+      assertEquals(ReturnCode.INCLUDE, code);
+    }
+
+    // accept the data located on LOC1 region.
+    for (int i = 0; i != maxRowCount; ++i) {
+      ReturnCode acceptCode = checker.canTakeOperation(LOC1, heapSizeOfRow);
+      assertEquals(ReturnCode.INCLUDE, acceptCode);
+      checker.notifyFinal(acceptCode, LOC1, heapSizeOfRow);
+    }
+
+    // the sn server reachs the limit.
+    for (int i = 0; i != 10; ++i) {
+      ReturnCode code = checker.canTakeOperation(LOC1, heapSizeOfRow);
+      assertNotEquals(ReturnCode.INCLUDE, code);
+      code = checker.canTakeOperation(LOC2, heapSizeOfRow);
+      assertNotEquals(ReturnCode.INCLUDE, code);
+    }
+
+    // the request to sn2 server should be accepted.
+    for (int i = 0; i != 10; ++i) {
+      ReturnCode code = checker.canTakeOperation(LOC3, heapSizeOfRow);
+      assertEquals(ReturnCode.INCLUDE, code);
+    }
+
+    checker.reset();
+    for (int i = 0; i != 10; ++i) {
+      ReturnCode code = checker.canTakeOperation(LOC1, heapSizeOfRow);
+      assertEquals(ReturnCode.INCLUDE, code);
+      code = checker.canTakeOperation(LOC2, heapSizeOfRow);
+      assertEquals(ReturnCode.INCLUDE, code);
+    }
+  }
+
+  @Test
   public void testSubmittedSizeChecker() {
     final long maxHeapSizeSubmit = 2 * 1024 * 1024;
     SimpleRequestController.SubmittedSizeChecker checker
@@ -224,7 +273,7 @@ public class TestSimpleRequestController {
 
   @Test
   public void testTaskCountChecker() throws InterruptedIOException {
-    long rowSize = 12345;
+    long heapSizeOfRow = 12345;
     int maxTotalConcurrentTasks = 100;
     int maxConcurrentTasksPerServer = 2;
     int maxConcurrentTasksPerRegion = 1;
@@ -239,13 +288,13 @@ public class TestSimpleRequestController {
 
     // inner state is unchanged.
     for (int i = 0; i != 10; ++i) {
-      ReturnCode code = checker.canTakeOperation(LOC1, rowSize);
+      ReturnCode code = checker.canTakeOperation(LOC1, heapSizeOfRow);
       assertEquals(ReturnCode.INCLUDE, code);
     }
     // add LOC1 region.
-    ReturnCode code = checker.canTakeOperation(LOC1, rowSize);
+    ReturnCode code = checker.canTakeOperation(LOC1, heapSizeOfRow);
     assertEquals(ReturnCode.INCLUDE, code);
-    checker.notifyFinal(code, LOC1, rowSize);
+    checker.notifyFinal(code, LOC1, heapSizeOfRow);
 
     // fill the task slots for LOC1.
     taskCounterPerRegion.put(LOC1.getRegionInfo().getRegionName(), new AtomicInteger(100));
@@ -253,9 +302,9 @@ public class TestSimpleRequestController {
 
     // the region was previously accepted, so it must be accpted now.
     for (int i = 0; i != maxConcurrentTasksPerRegion * 5; ++i) {
-      ReturnCode includeCode = checker.canTakeOperation(LOC1, rowSize);
+      ReturnCode includeCode = checker.canTakeOperation(LOC1, heapSizeOfRow);
       assertEquals(ReturnCode.INCLUDE, includeCode);
-      checker.notifyFinal(includeCode, LOC1, rowSize);
+      checker.notifyFinal(includeCode, LOC1, heapSizeOfRow);
     }
 
     // fill the task slots for LOC3.
@@ -264,9 +313,9 @@ public class TestSimpleRequestController {
 
     // no task slots.
     for (int i = 0; i != maxConcurrentTasksPerRegion * 5; ++i) {
-      ReturnCode excludeCode = checker.canTakeOperation(LOC3, rowSize);
+      ReturnCode excludeCode = checker.canTakeOperation(LOC3, heapSizeOfRow);
       assertNotEquals(ReturnCode.INCLUDE, excludeCode);
-      checker.notifyFinal(excludeCode, LOC3, rowSize);
+      checker.notifyFinal(excludeCode, LOC3, heapSizeOfRow);
     }
 
     // release the tasks for LOC3.
@@ -274,15 +323,15 @@ public class TestSimpleRequestController {
     taskCounterPerServer.put(LOC3.getServerName(), new AtomicInteger(0));
 
     // add LOC3 region.
-    ReturnCode code3 = checker.canTakeOperation(LOC3, rowSize);
+    ReturnCode code3 = checker.canTakeOperation(LOC3, heapSizeOfRow);
     assertEquals(ReturnCode.INCLUDE, code3);
-    checker.notifyFinal(code3, LOC3, rowSize);
+    checker.notifyFinal(code3, LOC3, heapSizeOfRow);
 
     // the region was previously accepted, so it must be accpted now.
     for (int i = 0; i != maxConcurrentTasksPerRegion * 5; ++i) {
-      ReturnCode includeCode = checker.canTakeOperation(LOC3, rowSize);
+      ReturnCode includeCode = checker.canTakeOperation(LOC3, heapSizeOfRow);
       assertEquals(ReturnCode.INCLUDE, includeCode);
-      checker.notifyFinal(includeCode, LOC3, rowSize);
+      checker.notifyFinal(includeCode, LOC3, heapSizeOfRow);
     }
 
     checker.reset();
@@ -290,9 +339,9 @@ public class TestSimpleRequestController {
     // but checker have reseted and task slots for LOC1 is full.
     // So it must be rejected now.
     for (int i = 0; i != maxConcurrentTasksPerRegion * 5; ++i) {
-      ReturnCode includeCode = checker.canTakeOperation(LOC1, rowSize);
+      ReturnCode includeCode = checker.canTakeOperation(LOC1, heapSizeOfRow);
       assertNotEquals(ReturnCode.INCLUDE, includeCode);
-      checker.notifyFinal(includeCode, LOC1, rowSize);
+      checker.notifyFinal(includeCode, LOC1, heapSizeOfRow);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b2a9be02/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 e205381..77d9d57 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
@@ -41,7 +41,7 @@ public interface RegionServerObserver extends Coprocessor {
     throws IOException;
 
   /**
-   * Called before the regions merge. 
+   * Called before the regions merge.
    * Call {@link org.apache.hadoop.hbase.coprocessor.ObserverContext#bypass()} to skip the merge.
    * @throws IOException if an error occurred on the coprocessor
    * @param ctx


[05/16] hbase git commit: HBASE-16710 Add ZStandard Codec to Compression.java

Posted by sy...@apache.org.
HBASE-16710 Add ZStandard Codec to Compression.java

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


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

Branch: refs/heads/hbase-12439
Commit: 0f6c79eb123e43133df4f4ba2a123029d62580dc
Parents: df98d8d
Author: rahul gidwani <rg...@salesforce.com>
Authored: Tue Sep 27 10:18:09 2016 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Thu Jan 5 10:46:55 2017 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/io/compress/Compression.java   | 27 ++++++++++++++++++++
 .../hadoop/hbase/util/TestCompressionTest.java  |  2 ++
 2 files changed, 29 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0f6c79eb/hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/Compression.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/Compression.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/Compression.java
index 6dc4190..8dfab44 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/Compression.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/compress/Compression.java
@@ -262,6 +262,33 @@ public final class Compression {
           throw new RuntimeException(e);
         }
       }
+    },
+    ZSTD("zstd") {
+      // Use base type to avoid compile-time dependencies.
+      private volatile transient CompressionCodec zStandardCodec;
+      private transient Object lock = new Object();
+
+      @Override
+      CompressionCodec getCodec(Configuration conf) {
+        if (zStandardCodec == null) {
+          synchronized (lock) {
+            if (zStandardCodec == null) {
+              zStandardCodec = buildCodec(conf);
+            }
+          }
+        }
+        return zStandardCodec;
+      }
+
+      private CompressionCodec buildCodec(Configuration conf) {
+        try {
+          Class<?> externalCodec =
+              getClassLoaderForCodec().loadClass("org.apache.hadoop.io.compress.ZStandardCodec");
+          return (CompressionCodec) ReflectionUtils.newInstance(externalCodec, conf);
+        } catch (ClassNotFoundException e) {
+          throw new RuntimeException(e);
+        }
+      }
     };
 
     private final Configuration conf;

http://git-wip-us.apache.org/repos/asf/hbase/blob/0f6c79eb/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCompressionTest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCompressionTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCompressionTest.java
index 398f3f0..395c04d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCompressionTest.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCompressionTest.java
@@ -76,6 +76,7 @@ public class TestCompressionTest {
       nativeCodecTest("LZ4", null, "org.apache.hadoop.io.compress.Lz4Codec");
       nativeCodecTest("SNAPPY", "snappy", "org.apache.hadoop.io.compress.SnappyCodec");
       nativeCodecTest("BZIP2", "bzip2", "org.apache.hadoop.io.compress.BZip2Codec");
+      nativeCodecTest("ZSTD", "zstd", "org.apache.hadoop.io.compress.ZStandardCodec");
     } else {
       // Hadoop nativelib is not available
       LOG.debug("Native code not loaded");
@@ -83,6 +84,7 @@ public class TestCompressionTest {
       assertFalse(CompressionTest.testCompression("LZ4"));
       assertFalse(CompressionTest.testCompression("SNAPPY"));
       assertFalse(CompressionTest.testCompression("BZIP2"));
+      assertFalse(CompressionTest.testCompression("ZSTD"));
     }
   }
 


[04/16] hbase git commit: HBASE-17410 Changed size() == 0 to isEmpty in hbase-client

Posted by sy...@apache.org.
HBASE-17410 Changed size() == 0 to isEmpty in hbase-client

Signed-off-by: Josh Elser <el...@apache.org>


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

Branch: refs/heads/hbase-12439
Commit: df98d8dcd76835e59fe6df43197308215028a41e
Parents: dba103e
Author: Jan Hentschel <ja...@ultratendency.com>
Authored: Wed Jan 4 18:29:33 2017 +0100
Committer: Josh Elser <el...@apache.org>
Committed: Thu Jan 5 11:45:18 2017 -0500

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hbase/client/ClientScanner.java  | 4 ++--
 .../apache/hadoop/hbase/client/ClientSmallReversedScanner.java   | 4 ++--
 .../java/org/apache/hadoop/hbase/client/ClientSmallScanner.java  | 4 ++--
 .../java/org/apache/hadoop/hbase/client/HTableMultiplexer.java   | 2 +-
 .../src/main/java/org/apache/hadoop/hbase/client/Increment.java  | 2 +-
 .../src/main/java/org/apache/hadoop/hbase/client/Put.java        | 2 +-
 .../src/main/java/org/apache/hadoop/hbase/client/Scan.java       | 2 +-
 .../java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java  | 2 +-
 .../main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java  | 2 +-
 .../java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java | 4 ++--
 .../apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java   | 2 +-
 .../apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java | 4 ++--
 .../test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java  | 4 ++--
 13 files changed, 19 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/df98d8dc/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 c4c86a6..283272a 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
@@ -350,10 +350,10 @@ public abstract class ClientScanner extends AbstractClientScanner {
 
   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.size() == 0 && this.closed) {
+    if (cache.isEmpty() && this.closed) {
       return null;
     }
-    if (cache.size() == 0) {
+    if (cache.isEmpty()) {
       loadCache();
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/df98d8dc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallReversedScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallReversedScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallReversedScanner.java
index ffa8af7..8f0c2f8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallReversedScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallReversedScanner.java
@@ -193,10 +193,10 @@ public class ClientSmallReversedScanner extends ReversedClientScanner {
   public Result next() throws IOException {
     // If the scanner is closed and there's nothing left in the cache, next is a
     // no-op.
-    if (cache.size() == 0 && this.closed) {
+    if (cache.isEmpty() && this.closed) {
       return null;
     }
-    if (cache.size() == 0) {
+    if (cache.isEmpty()) {
       loadCache();
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/df98d8dc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java
index b5f373a..52a291b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSmallScanner.java
@@ -216,10 +216,10 @@ public class ClientSmallScanner extends ClientSimpleScanner {
   public Result next() throws IOException {
     // If the scanner is closed and there's nothing left in the cache, next is a
     // no-op.
-    if (cache.size() == 0 && this.closed) {
+    if (cache.isEmpty() && this.closed) {
       return null;
     }
-    if (cache.size() == 0) {
+    if (cache.isEmpty()) {
       loadCache();
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/df98d8dc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
index c03b969..27393ba 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java
@@ -569,7 +569,7 @@ public class HTableMultiplexer {
         // drain all the queued puts into the tmp list
         processingList.clear();
         queue.drainTo(processingList);
-        if (processingList.size() == 0) {
+        if (processingList.isEmpty()) {
           // Nothing to flush
           return;
         }

http://git-wip-us.apache.org/repos/asf/hbase/blob/df98d8dc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java
index f4eede4..9538361 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Increment.java
@@ -225,7 +225,7 @@ public class Increment extends Mutation implements Comparable<Row> {
     StringBuilder sb = new StringBuilder();
     sb.append("row=");
     sb.append(Bytes.toStringBinary(this.row));
-    if(this.familyMap.size() == 0) {
+    if(this.familyMap.isEmpty()) {
       sb.append(", no columns set to be incremented");
       return sb.toString();
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/df98d8dc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
index 54480d1..a6ebd03 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Put.java
@@ -413,7 +413,7 @@ public class Put extends Mutation implements HeapSize, Comparable<Row> {
   private boolean has(byte[] family, byte[] qualifier, long ts, byte[] value,
                       boolean ignoreTS, boolean ignoreValue) {
     List<Cell> list = getCellList(family);
-    if (list.size() == 0) {
+    if (list.isEmpty()) {
       return false;
     }
     // Boolean analysis of ignoreTS/ignoreValue.

http://git-wip-us.apache.org/repos/asf/hbase/blob/df98d8dc/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java
index 2c69924..a2d9037 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Scan.java
@@ -871,7 +871,7 @@ public class Scan extends Query {
   public Map<String, Object> getFingerprint() {
     Map<String, Object> map = new HashMap<String, Object>();
     List<String> families = new ArrayList<String>();
-    if(this.familyMap.size() == 0) {
+    if(this.familyMap.isEmpty()) {
       map.put("families", "ALL");
       return map;
     } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/df98d8dc/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java
index 3549304..14d23d4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java
@@ -61,7 +61,7 @@ public class FirstKeyOnlyFilter extends FilterBase {
   }
 
   public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
-    Preconditions.checkArgument(filterArguments.size() == 0,
+    Preconditions.checkArgument(filterArguments.isEmpty(),
                                 "Expected 0 but got: %s", filterArguments.size());
     return new FirstKeyOnlyFilter();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/df98d8dc/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
index a22750d..adbf304 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
@@ -75,7 +75,7 @@ public class KeyOnlyFilter extends FilterBase {
   }
   
   public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
-    Preconditions.checkArgument((filterArguments.size() == 0 || filterArguments.size() == 1),
+    Preconditions.checkArgument((filterArguments.isEmpty() || filterArguments.size() == 1),
                                 "Expected: 0 or 1 but got: %s", filterArguments.size());
     KeyOnlyFilter filter = new KeyOnlyFilter();
     if (filterArguments.size() == 1) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/df98d8dc/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java
index 66556b0..768088b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java
@@ -248,7 +248,7 @@ public class MultiRowRangeFilter extends FilterBase {
    * @return the ranges after sort and merge.
    */
   public static List<RowRange> sortAndMerge(List<RowRange> ranges) {
-    if (ranges.size() == 0) {
+    if (ranges.isEmpty()) {
       throw new IllegalArgumentException("No ranges found.");
     }
     List<RowRange> invalidRanges = new ArrayList<RowRange>();
@@ -391,7 +391,7 @@ public class MultiRowRangeFilter extends FilterBase {
       throwExceptionForInvalidRanges(invalidRanges, true);
     }
     // If no valid ranges found, throw the exception
-    if(newRanges.size() == 0) {
+    if(newRanges.isEmpty()) {
       throw new IllegalArgumentException("No valid ranges found.");
     }
     return newRanges;

http://git-wip-us.apache.org/repos/asf/hbase/blob/df98d8dc/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java
index 9909d1f..bc26812 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java
@@ -70,7 +70,7 @@ public class MultipleColumnPrefixFilter extends FilterBase {
 
   @Override
   public ReturnCode filterKeyValue(Cell kv) {
-    if (sortedPrefixes.size() == 0) {
+    if (sortedPrefixes.isEmpty()) {
       return ReturnCode.INCLUDE;
     } else {
       return filterColumn(kv);

http://git-wip-us.apache.org/repos/asf/hbase/blob/df98d8dc/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
index dcbc0f0..7c548d9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
@@ -204,7 +204,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
     String rsPath = ZKUtil.joinZNode(this.queuesZNode, regionserver);
     try {
       List<String> list = ZKUtil.listChildrenNoWatch(this.zookeeper, rsPath);
-      if (list != null && list.size() == 0){
+      if (list != null && list.isEmpty()){
         ZKUtil.deleteNode(this.zookeeper, rsPath);
       }
     } catch (KeeperException e) {
@@ -282,7 +282,7 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
       }
 
       SortedSet<String> logQueue = new TreeSet<>();
-      if (wals == null || wals.size() == 0) {
+      if (wals == null || wals.isEmpty()) {
         listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
       } else {
         // create the new cluster znode

http://git-wip-us.apache.org/repos/asf/hbase/blob/df98d8dc/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java
index 9f92f4f..f44fe9f 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java
@@ -132,7 +132,7 @@ public class TestHTableDescriptor {
     String className1 = "org.apache.hadoop.hbase.coprocessor.BaseRegionObserver";
     String className2 = "org.apache.hadoop.hbase.coprocessor.SampleRegionWALObserver";
     // Check that any coprocessor is present.
-    assertTrue(desc.getCoprocessors().size() == 0);
+    assertTrue(desc.getCoprocessors().isEmpty());
 
     // Add the 1 coprocessor and check if present.
     desc.addCoprocessor(className1);
@@ -153,7 +153,7 @@ public class TestHTableDescriptor {
 
     // Remove the last and check
     desc.removeCoprocessor(className2);
-    assertTrue(desc.getCoprocessors().size() == 0);
+    assertTrue(desc.getCoprocessors().isEmpty());
     assertFalse(desc.getCoprocessors().contains(className1));
     assertFalse(desc.getCoprocessors().contains(className2));
   }


[12/16] hbase git commit: HBASE-17351 Unable to generate tar ball for master branch

Posted by sy...@apache.org.
HBASE-17351 Unable to generate tar ball for master branch


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

Branch: refs/heads/hbase-12439
Commit: 910e885a75e2092e49ae577d1481d58845fffae5
Parents: 5f631b9
Author: Esteban Gutierrez <es...@apache.org>
Authored: Tue Dec 20 15:18:28 2016 -0800
Committer: Esteban Gutierrez <es...@apache.org>
Committed: Fri Jan 6 08:42:20 2017 -0800

----------------------------------------------------------------------
 pom.xml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/910e885a/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 02320dc..285e358 100644
--- a/pom.xml
+++ b/pom.xml
@@ -876,6 +876,7 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-enforcer-plugin</artifactId>
+        <version>1.4</version>
         <dependencies>
           <dependency>
             <groupId>org.codehaus.mojo</groupId>


[09/16] hbase git commit: HBASE-17388 Move ReplicationPeer and other replication related PB messages to the replication.proto

Posted by sy...@apache.org.
HBASE-17388 Move ReplicationPeer and other replication related PB messages to the replication.proto


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

Branch: refs/heads/hbase-12439
Commit: e02ae7724ddaa147a7cf41dc398e09e456e0dad6
Parents: 0f6c79e
Author: Guanghao Zhang <zg...@apache.org>
Authored: Wed Jan 4 17:48:09 2017 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Fri Jan 6 10:01:22 2017 +0800

----------------------------------------------------------------------
 .../replication/ReplicationSerDeHelper.java     |   50 +-
 .../replication/ReplicationPeerZKImpl.java      |   14 +-
 .../replication/ReplicationPeersZKImpl.java     |   10 +-
 .../replication/ReplicationStateZKBase.java     |   14 +-
 .../apache/hadoop/hbase/zookeeper/ZKUtil.java   |   18 +-
 .../protobuf/generated/ReplicationProtos.java   | 4234 +++++++++++++++++-
 .../protobuf/generated/ZooKeeperProtos.java     | 4048 +----------------
 .../src/main/protobuf/Replication.proto         |   40 +-
 .../src/main/protobuf/ZooKeeper.proto           |   38 -
 .../replication/master/TableCFsUpdater.java     |    4 +-
 .../replication/TestPerTableCFReplication.java  |    4 +-
 11 files changed, 4237 insertions(+), 4237 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e02ae772/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationSerDeHelper.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationSerDeHelper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationSerDeHelper.java
index dd83fb1..9e04c9b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationSerDeHelper.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationSerDeHelper.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Strings;
@@ -62,13 +62,13 @@ public final class ReplicationSerDeHelper {
   }
 
   /** convert map to TableCFs Object */
-  public static ZooKeeperProtos.TableCF[] convert(
+  public static ReplicationProtos.TableCF[] convert(
       Map<TableName, ? extends Collection<String>> tableCfs) {
     if (tableCfs == null) {
       return null;
     }
-    List<ZooKeeperProtos.TableCF> tableCFList = new ArrayList<>();
-    ZooKeeperProtos.TableCF.Builder tableCFBuilder =  ZooKeeperProtos.TableCF.newBuilder();
+    List<ReplicationProtos.TableCF> tableCFList = new ArrayList<>();
+    ReplicationProtos.TableCF.Builder tableCFBuilder =  ReplicationProtos.TableCF.newBuilder();
     for (Map.Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
       tableCFBuilder.clear();
       tableCFBuilder.setTableName(ProtobufUtil.toProtoTableName(entry.getKey()));
@@ -80,7 +80,7 @@ public final class ReplicationSerDeHelper {
       }
       tableCFList.add(tableCFBuilder.build());
     }
-    return tableCFList.toArray(new ZooKeeperProtos.TableCF[tableCFList.size()]);
+    return tableCFList.toArray(new ReplicationProtos.TableCF[tableCFList.size()]);
   }
 
   public static String convertToString(Map<TableName, ? extends Collection<String>> tableCfs) {
@@ -95,12 +95,12 @@ public final class ReplicationSerDeHelper {
    *  This is only for read TableCFs information from TableCF node.
    *  Input String Format: ns1.table1:cf1,cf2;ns2.table2:cfA,cfB;ns3.table3.
    * */
-  public static ZooKeeperProtos.TableCF[] convert(String tableCFsConfig) {
+  public static ReplicationProtos.TableCF[] convert(String tableCFsConfig) {
     if (tableCFsConfig == null || tableCFsConfig.trim().length() == 0) {
       return null;
     }
-    List<ZooKeeperProtos.TableCF> tableCFList = new ArrayList<>();
-    ZooKeeperProtos.TableCF.Builder tableCFBuilder = ZooKeeperProtos.TableCF.newBuilder();
+    List<ReplicationProtos.TableCF> tableCFList = new ArrayList<>();
+    ReplicationProtos.TableCF.Builder tableCFBuilder = ReplicationProtos.TableCF.newBuilder();
 
     String[] tables = tableCFsConfig.split(";");
     for (String tab : tables) {
@@ -142,17 +142,17 @@ public final class ReplicationSerDeHelper {
       }
       tableCFList.add(tableCFBuilder.build());
     }
-    return tableCFList.toArray(new ZooKeeperProtos.TableCF[tableCFList.size()]);
+    return tableCFList.toArray(new ReplicationProtos.TableCF[tableCFList.size()]);
   }
 
   /**
    *  Convert TableCFs Object to String.
    *  Output String Format: ns1.table1:cf1,cf2;ns2.table2:cfA,cfB;table3
    * */
-  public static String convert(ZooKeeperProtos.TableCF[] tableCFs) {
+  public static String convert(ReplicationProtos.TableCF[] tableCFs) {
     StringBuilder sb = new StringBuilder();
     for (int i = 0, n = tableCFs.length; i < n; i++) {
-      ZooKeeperProtos.TableCF tableCF = tableCFs[i];
+      ReplicationProtos.TableCF tableCF = tableCFs[i];
       String namespace = tableCF.getTableName().getNamespace().toStringUtf8();
       if (!Strings.isEmpty(namespace)) {
         sb.append(namespace).append(".").
@@ -175,10 +175,10 @@ public final class ReplicationSerDeHelper {
   /**
    *  Get TableCF in TableCFs, if not exist, return null.
    * */
-  public static ZooKeeperProtos.TableCF getTableCF(ZooKeeperProtos.TableCF[] tableCFs,
+  public static ReplicationProtos.TableCF getTableCF(ReplicationProtos.TableCF[] tableCFs,
                                            String table) {
     for (int i = 0, n = tableCFs.length; i < n; i++) {
-      ZooKeeperProtos.TableCF tableCF = tableCFs[i];
+      ReplicationProtos.TableCF tableCF = tableCFs[i];
       if (tableCF.getTableName().getQualifier().toStringUtf8().equals(table)) {
         return tableCF;
       }
@@ -191,7 +191,7 @@ public final class ReplicationSerDeHelper {
    *  It is used for backward compatibility.
    *  Old format bytes have no PB_MAGIC Header
    * */
-  public static ZooKeeperProtos.TableCF[] parseTableCFs(byte[] bytes) throws IOException {
+  public static ReplicationProtos.TableCF[] parseTableCFs(byte[] bytes) throws IOException {
     if (bytes == null) {
       return null;
     }
@@ -202,20 +202,20 @@ public final class ReplicationSerDeHelper {
    *  Convert tableCFs string into Map.
    * */
   public static Map<TableName, List<String>> parseTableCFsFromConfig(String tableCFsConfig) {
-    ZooKeeperProtos.TableCF[] tableCFs = convert(tableCFsConfig);
+    ReplicationProtos.TableCF[] tableCFs = convert(tableCFsConfig);
     return convert2Map(tableCFs);
   }
 
   /**
    *  Convert tableCFs Object to Map.
    * */
-  public static Map<TableName, List<String>> convert2Map(ZooKeeperProtos.TableCF[] tableCFs) {
+  public static Map<TableName, List<String>> convert2Map(ReplicationProtos.TableCF[] tableCFs) {
     if (tableCFs == null || tableCFs.length == 0) {
       return null;
     }
     Map<TableName, List<String>> tableCFsMap = new HashMap<TableName, List<String>>();
     for (int i = 0, n = tableCFs.length; i < n; i++) {
-      ZooKeeperProtos.TableCF tableCF = tableCFs[i];
+      ReplicationProtos.TableCF tableCF = tableCFs[i];
       List<String> families = new ArrayList<>();
       for (int j = 0, m = tableCF.getFamiliesCount(); j < m; j++) {
         families.add(tableCF.getFamilies(j).toStringUtf8());
@@ -239,9 +239,9 @@ public final class ReplicationSerDeHelper {
       throws DeserializationException {
     if (ProtobufUtil.isPBMagicPrefix(bytes)) {
       int pblen = ProtobufUtil.lengthOfPBMagic();
-      ZooKeeperProtos.ReplicationPeer.Builder builder =
-          ZooKeeperProtos.ReplicationPeer.newBuilder();
-      ZooKeeperProtos.ReplicationPeer peer;
+      ReplicationProtos.ReplicationPeer.Builder builder =
+          ReplicationProtos.ReplicationPeer.newBuilder();
+      ReplicationProtos.ReplicationPeer peer;
       try {
         ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
         peer = builder.build();
@@ -257,7 +257,7 @@ public final class ReplicationSerDeHelper {
     }
   }
 
-  public static ReplicationPeerConfig convert(ZooKeeperProtos.ReplicationPeer peer) {
+  public static ReplicationPeerConfig convert(ReplicationProtos.ReplicationPeer peer) {
     ReplicationPeerConfig peerConfig = new ReplicationPeerConfig();
     if (peer.hasClusterkey()) {
       peerConfig.setClusterKey(peer.getClusterkey());
@@ -275,7 +275,7 @@ public final class ReplicationSerDeHelper {
     }
 
     Map<TableName, ? extends Collection<String>> tableCFsMap = convert2Map(
-      peer.getTableCfsList().toArray(new ZooKeeperProtos.TableCF[peer.getTableCfsCount()]));
+      peer.getTableCfsList().toArray(new ReplicationProtos.TableCF[peer.getTableCfsCount()]));
     if (tableCFsMap != null) {
       peerConfig.setTableCFsMap(tableCFsMap);
     }
@@ -293,8 +293,8 @@ public final class ReplicationSerDeHelper {
     return peerConfig;
   }
 
-  public static ZooKeeperProtos.ReplicationPeer convert(ReplicationPeerConfig  peerConfig) {
-    ZooKeeperProtos.ReplicationPeer.Builder builder = ZooKeeperProtos.ReplicationPeer.newBuilder();
+  public static ReplicationProtos.ReplicationPeer convert(ReplicationPeerConfig  peerConfig) {
+    ReplicationProtos.ReplicationPeer.Builder builder = ReplicationProtos.ReplicationPeer.newBuilder();
     if (peerConfig.getClusterKey() != null) {
       builder.setClusterkey(peerConfig.getClusterKey());
     }
@@ -316,7 +316,7 @@ public final class ReplicationSerDeHelper {
           .build());
     }
 
-    ZooKeeperProtos.TableCF[] tableCFs = convert(peerConfig.getTableCFsMap());
+    ReplicationProtos.TableCF[] tableCFs = convert(peerConfig.getTableCFsMap());
     if (tableCFs != null) {
       for (int i = 0; i < tableCFs.length; i++) {
         builder.addTableCfs(tableCFs[i]);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e02ae772/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
index c58bd71..8b13f75 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@@ -210,8 +210,8 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase
    * @throws DeserializationException
    */
   public static boolean isStateEnabled(final byte[] bytes) throws DeserializationException {
-    ZooKeeperProtos.ReplicationState.State state = parseStateFrom(bytes);
-    return ZooKeeperProtos.ReplicationState.State.ENABLED == state;
+    ReplicationProtos.ReplicationState.State state = parseStateFrom(bytes);
+    return ReplicationProtos.ReplicationState.State.ENABLED == state;
   }
 
   /**
@@ -219,13 +219,13 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase
    * @return State parsed from the passed bytes.
    * @throws DeserializationException
    */
-  private static ZooKeeperProtos.ReplicationState.State parseStateFrom(final byte[] bytes)
+  private static ReplicationProtos.ReplicationState.State parseStateFrom(final byte[] bytes)
       throws DeserializationException {
     ProtobufUtil.expectPBMagicPrefix(bytes);
     int pblen = ProtobufUtil.lengthOfPBMagic();
-    ZooKeeperProtos.ReplicationState.Builder builder =
-        ZooKeeperProtos.ReplicationState.newBuilder();
-    ZooKeeperProtos.ReplicationState state;
+    ReplicationProtos.ReplicationState.Builder builder =
+        ReplicationProtos.ReplicationState.newBuilder();
+    ReplicationProtos.ReplicationState state;
     try {
       ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
       state = builder.build();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e02ae772/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
index 9a617a7..a4b09c3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
@@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.ZKConfig;
@@ -160,13 +160,13 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
 
   @Override
   public void enablePeer(String id) throws ReplicationException {
-    changePeerState(id, ZooKeeperProtos.ReplicationState.State.ENABLED);
+    changePeerState(id, ReplicationProtos.ReplicationState.State.ENABLED);
     LOG.info("peer " + id + " is enabled");
   }
 
   @Override
   public void disablePeer(String id) throws ReplicationException {
-    changePeerState(id, ZooKeeperProtos.ReplicationState.State.DISABLED);
+    changePeerState(id, ReplicationProtos.ReplicationState.State.DISABLED);
     LOG.info("peer " + id + " is disabled");
   }
 
@@ -462,7 +462,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
    * @param id
    * @param state
    */
-  private void changePeerState(String id, ZooKeeperProtos.ReplicationState.State state)
+  private void changePeerState(String id, ReplicationProtos.ReplicationState.State state)
       throws ReplicationException {
     try {
       if (!peerExists(id)) {
@@ -471,7 +471,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
       }
       String peerStateZNode = getPeerStateNode(id);
       byte[] stateBytes =
-          (state == ZooKeeperProtos.ReplicationState.State.ENABLED) ? ENABLED_ZNODE_BYTES
+          (state == ReplicationProtos.ReplicationState.State.ENABLED) ? ENABLED_ZNODE_BYTES
               : DISABLED_ZNODE_BYTES;
       if (ZKUtil.checkExists(this.zookeeper, peerStateZNode) != -1) {
         ZKUtil.setData(this.zookeeper, peerStateZNode, stateBytes);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e02ae772/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
index 2bfe757..c80822e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
@@ -23,13 +23,13 @@ import java.io.IOException;
 import java.util.List;
 
 import com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream;
 
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedOutputStream;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
+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.ZooKeeperWatcher;
@@ -66,9 +66,9 @@ public abstract class ReplicationStateZKBase {
 
   // Public for testing
   public static final byte[] ENABLED_ZNODE_BYTES =
-      toByteArray(ZooKeeperProtos.ReplicationState.State.ENABLED);
+      toByteArray(ReplicationProtos.ReplicationState.State.ENABLED);
   public static final byte[] DISABLED_ZNODE_BYTES =
-      toByteArray(ZooKeeperProtos.ReplicationState.State.DISABLED);
+      toByteArray(ReplicationProtos.ReplicationState.State.DISABLED);
   public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_KEY =
       "zookeeper.znode.replication.hfile.refs";
   public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT = "hfile-refs";
@@ -110,9 +110,9 @@ public abstract class ReplicationStateZKBase {
    *         use as content of a peer-state znode under a peer cluster id as in
    *         /hbase/replication/peers/PEER_ID/peer-state.
    */
-  protected static byte[] toByteArray(final ZooKeeperProtos.ReplicationState.State state) {
-    ZooKeeperProtos.ReplicationState msg =
-        ZooKeeperProtos.ReplicationState.newBuilder().setState(state).build();
+  protected static byte[] toByteArray(final ReplicationProtos.ReplicationState.State state) {
+    ReplicationProtos.ReplicationState msg =
+        ReplicationProtos.ReplicationState.newBuilder().setState(state).build();
     // There is no toByteArray on this pb Message?
     // 32 bytes is default which seems fair enough here.
     try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e02ae772/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
index 2f6e1cf..98c96ec 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
@@ -51,7 +51,7 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.CreateAndFailSilent;
@@ -1860,8 +1860,8 @@ public class ZKUtil {
       }
       // parse the data of the above peer znode.
       try {
-        ZooKeeperProtos.ReplicationPeer.Builder builder =
-          ZooKeeperProtos.ReplicationPeer.newBuilder();
+        ReplicationProtos.ReplicationPeer.Builder builder =
+          ReplicationProtos.ReplicationPeer.newBuilder();
         ProtobufUtil.mergeFrom(builder, data, pblen, data.length - pblen);
         String clusterKey = builder.getClusterkey();
         sb.append("\n").append(znodeToProcess).append(": ").append(clusterKey);
@@ -1885,8 +1885,8 @@ public class ZKUtil {
       byte[] peerStateData;
       try {
         peerStateData = ZKUtil.getData(zkw, peerStateZnode);
-        ZooKeeperProtos.ReplicationState.Builder builder =
-            ZooKeeperProtos.ReplicationState.newBuilder();
+        ReplicationProtos.ReplicationState.Builder builder =
+            ReplicationProtos.ReplicationState.newBuilder();
         ProtobufUtil.mergeFrom(builder, peerStateData, pblen, peerStateData.length - pblen);
         sb.append(builder.getState().name());
       } catch (IOException ipbe) {
@@ -2054,7 +2054,7 @@ public class ZKUtil {
    *         for use as content of an wal position in a replication queue.
    */
   public static byte[] positionToByteArray(final long position) {
-    byte[] bytes = ZooKeeperProtos.ReplicationHLogPosition.newBuilder().setPosition(position)
+    byte[] bytes = ReplicationProtos.ReplicationHLogPosition.newBuilder().setPosition(position)
         .build().toByteArray();
     return ProtobufUtil.prependPBMagic(bytes);
   }
@@ -2070,9 +2070,9 @@ public class ZKUtil {
     }
     if (ProtobufUtil.isPBMagicPrefix(bytes)) {
       int pblen = ProtobufUtil.lengthOfPBMagic();
-      ZooKeeperProtos.ReplicationHLogPosition.Builder builder =
-          ZooKeeperProtos.ReplicationHLogPosition.newBuilder();
-      ZooKeeperProtos.ReplicationHLogPosition position;
+      ReplicationProtos.ReplicationHLogPosition.Builder builder =
+          ReplicationProtos.ReplicationHLogPosition.newBuilder();
+      ReplicationProtos.ReplicationHLogPosition position;
       try {
         ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
         position = builder.build();


[06/16] hbase git commit: HBASE-17388 Move ReplicationPeer and other replication related PB messages to the replication.proto

Posted by sy...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/e02ae772/hbase-protocol-shaded/src/main/protobuf/Replication.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Replication.proto b/hbase-protocol-shaded/src/main/protobuf/Replication.proto
index 05b6e59..c748514 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Replication.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Replication.proto
@@ -24,7 +24,45 @@ option java_generic_services = true;
 option java_generate_equals_and_hash = true;
 option optimize_for = SPEED;
 
-import "ZooKeeper.proto";
+import "HBase.proto";
+
+message TableCF {
+  optional TableName table_name = 1;
+  repeated bytes families = 2;
+}
+
+/**
+ * Used by replication. Holds a replication peer key.
+ */
+message ReplicationPeer {
+  // clusterkey is the concatenation of the slave cluster's
+  // hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
+  required string clusterkey = 1;
+  optional string replicationEndpointImpl = 2;
+  repeated BytesBytesPair data = 3;
+  repeated NameStringPair configuration = 4;
+  repeated TableCF table_cfs = 5;
+  repeated bytes namespaces = 6;
+  optional int64 bandwidth = 7;
+}
+
+/**
+ * Used by replication. Holds whether enabled or disabled
+ */
+message ReplicationState {
+  enum State {
+    ENABLED = 0;
+    DISABLED = 1;
+  }
+  required State state = 1;
+}
+
+/**
+ * Used by replication. Holds the current position in an WAL file.
+ */
+message ReplicationHLogPosition {
+  required int64 position = 1;
+}
 
 message AddReplicationPeerRequest {
   required string peer_id = 1;

http://git-wip-us.apache.org/repos/asf/hbase/blob/e02ae772/hbase-protocol-shaded/src/main/protobuf/ZooKeeper.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/ZooKeeper.proto b/hbase-protocol-shaded/src/main/protobuf/ZooKeeper.proto
index 323862c..b3bd2ec 100644
--- a/hbase-protocol-shaded/src/main/protobuf/ZooKeeper.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/ZooKeeper.proto
@@ -105,44 +105,6 @@ message DeprecatedTableState {
   required State state = 1 [default = ENABLED];
 }
 
-message TableCF {
-  optional TableName table_name = 1;
-  repeated bytes families = 2;
-}
-
-/**
- * Used by replication. Holds a replication peer key.
- */
-message ReplicationPeer {
-  // clusterkey is the concatenation of the slave cluster's
-  // hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent
-  required string clusterkey = 1;
-  optional string replicationEndpointImpl = 2;
-  repeated BytesBytesPair data = 3;
-  repeated NameStringPair configuration = 4;
-  repeated TableCF table_cfs = 5;
-  repeated bytes namespaces = 6;
-  optional int64 bandwidth = 7;
-}
-
-/**
- * Used by replication. Holds whether enabled or disabled
- */
-message ReplicationState {
-  enum State {
-    ENABLED = 0;
-    DISABLED = 1;
-  }
-  required State state = 1;
-}
-
-/**
- * Used by replication. Holds the current position in an WAL file.
- */
-message ReplicationHLogPosition {
-  required int64 position = 1;
-}
-
 /**
  * Metadata associated with a table lock in zookeeper
  */

http://git-wip-us.apache.org/repos/asf/hbase/blob/e02ae772/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java
index 1494892..08888f8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@@ -78,7 +78,7 @@ public class TableCFsUpdater extends ReplicationStateZKBase {
         if (rpc.getTableCFsMap() == null || rpc.getTableCFsMap().size() == 0) {
           // we copy TableCFs node into PeerNode
           LOG.info("copy tableCFs into peerNode:" + peerId);
-          ZooKeeperProtos.TableCF[] tableCFs =
+          ReplicationProtos.TableCF[] tableCFs =
                   ReplicationSerDeHelper.parseTableCFs(
                           ZKUtil.getData(this.zookeeper, tableCFsNode));
           if (tableCFs != null && tableCFs.length > 0) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e02ae772/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java
index 8f52cbe..bd70be0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestPerTableCFReplication.java
@@ -45,7 +45,7 @@ import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
 import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 import org.apache.hadoop.hbase.testclassification.FlakeyTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -271,7 +271,7 @@ public class TestPerTableCFReplication {
   @Test
   public void testTableCFsHelperConverter() {
 
-    ZooKeeperProtos.TableCF[] tableCFs = null;
+    ReplicationProtos.TableCF[] tableCFs = null;
     Map<TableName, List<String>> tabCFsMap = null;
 
     // 1. null or empty string, result should be null


[13/16] hbase git commit: HBASE-17431 Incorrect precheck condition in RoundRobinPool#get() (Jan Hentschel)

Posted by sy...@apache.org.
HBASE-17431 Incorrect precheck condition in RoundRobinPool#get() (Jan Hentschel)


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

Branch: refs/heads/hbase-12439
Commit: 6d48eb06c9cd50259e3137eea8e3554d190f678d
Parents: 910e885
Author: tedyu <yu...@gmail.com>
Authored: Fri Jan 6 09:07:02 2017 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Fri Jan 6 09:07:02 2017 -0800

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/6d48eb06/hbase-client/src/main/java/org/apache/hadoop/hbase/util/PoolMap.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/util/PoolMap.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/util/PoolMap.java
index b683fcc..f89215b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/util/PoolMap.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/util/PoolMap.java
@@ -360,7 +360,7 @@ public class PoolMap<K, V> implements Map<K, V> {
 
     @Override
     public R get() {
-      if (super.size() < maxSize) {
+      if (super.size() <= 0) {
         return null;
       }
       nextResource %= super.size();