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

[07/14] hbase git commit: HBASE-18431 Mitigate compatibility concerns between branch-1.3 and branch-1.4

HBASE-18431 Mitigate compatibility concerns between branch-1.3 and branch-1.4

- Refactor TableName into its own proto module and fix up users

- Move SnapshotDescription from Snapshot.proto back to HBase.proto

- Restore FastLongHistogram and TestFastLongHistogram;
  deprecate FastLongHistogram

- Move DeleteQueryTracker back to o.a.h.h.regionserver


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

Branch: refs/heads/branch-1.4
Commit: ea5789a7f2953c4cbfa6ce29132e882186d81799
Parents: 3552c70
Author: Andrew Purtell <ap...@apache.org>
Authored: Wed Aug 16 10:52:18 2017 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Wed Aug 16 11:33:49 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Admin.java   |    3 +-
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |    5 +-
 .../hadoop/hbase/protobuf/ProtobufUtil.java     |    9 +-
 .../ClientSnapshotDescriptionUtils.java         |    2 +-
 .../snapshot/CorruptedSnapshotException.java    |    2 +-
 .../hbase/snapshot/HBaseSnapshotException.java  |    2 +-
 .../snapshot/RestoreSnapshotException.java      |    2 +-
 .../snapshot/SnapshotCreationException.java     |    2 +-
 .../snapshot/SnapshotDoesNotExistException.java |    2 +-
 .../hbase/snapshot/SnapshotExistsException.java |    2 +-
 .../hbase/client/TestSnapshotFromAdmin.java     |    2 +-
 .../hadoop/hbase/util/FastLongHistogram.java    |  319 +
 .../hbase/util/TestFastLongHistogram.java       |  132 +
 hbase-protocol/pom.xml                          |    1 +
 .../protobuf/generated/AccessControlProtos.java |   92 +-
 .../hbase/protobuf/generated/ClientProtos.java  |    2 +-
 .../hbase/protobuf/generated/HBaseProtos.java   | 5603 ++++++++++--------
 .../generated/MasterProcedureProtos.java        |  586 +-
 .../hbase/protobuf/generated/MasterProtos.java  | 1689 +++---
 .../generated/SecureBulkLoadProtos.java         |   94 +-
 .../protobuf/generated/SnapshotProtos.java      | 1551 +----
 .../hbase/protobuf/generated/TableProtos.java   |  607 ++
 .../hbase/protobuf/generated/WALProtos.java     |  145 +-
 .../protobuf/generated/ZooKeeperProtos.java     |  171 +-
 .../src/main/protobuf/AccessControl.proto       |    2 +-
 hbase-protocol/src/main/protobuf/HBase.proto    |   28 +-
 hbase-protocol/src/main/protobuf/Master.proto   |    1 +
 .../src/main/protobuf/MasterProcedure.proto     |    1 +
 .../src/main/protobuf/SecureBulkLoad.proto      |    1 +
 hbase-protocol/src/main/protobuf/Snapshot.proto |   19 -
 hbase-protocol/src/main/protobuf/Table.proto    |   33 +
 hbase-protocol/src/main/protobuf/WAL.proto      |    1 +
 .../src/main/protobuf/ZooKeeper.proto           |    1 +
 .../hbase/tmpl/master/MasterStatusTmpl.jamon    |    2 +-
 .../BaseMasterAndRegionObserver.java            |    2 +-
 .../hbase/coprocessor/BaseMasterObserver.java   |    2 +-
 .../hbase/coprocessor/BaseRegionObserver.java   |    2 +-
 .../hbase/coprocessor/MasterObserver.java       |    2 +-
 .../hbase/coprocessor/RegionObserver.java       |    2 +-
 .../mapreduce/TableSnapshotInputFormatImpl.java |    2 +-
 .../hbase/master/MasterCoprocessorHost.java     |    2 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |    4 +-
 .../hadoop/hbase/master/SnapshotSentinel.java   |    2 +-
 .../master/snapshot/CloneSnapshotHandler.java   |    2 +-
 .../snapshot/DisabledTableSnapshotHandler.java  |    2 +-
 .../snapshot/EnabledTableSnapshotHandler.java   |    2 +-
 .../master/snapshot/MasterSnapshotVerifier.java |    2 +-
 .../master/snapshot/RestoreSnapshotHandler.java |    2 +-
 .../hbase/master/snapshot/SnapshotManager.java  |    4 +-
 .../master/snapshot/TakeSnapshotHandler.java    |    2 +-
 .../hbase/regionserver/DeleteTracker.java       |  101 +
 .../hadoop/hbase/regionserver/HRegion.java      |    2 +-
 .../regionserver/RegionCoprocessorHost.java     |    2 +-
 .../CompactionScanQueryMatcher.java             |    1 +
 .../querymatcher/DeleteTracker.java             |  101 -
 .../DropDeletesCompactionScanQueryMatcher.java  |    1 +
 .../querymatcher/LegacyScanQueryMatcher.java    |    3 +-
 .../MajorCompactionScanQueryMatcher.java        |    1 +
 .../MinorCompactionScanQueryMatcher.java        |    1 +
 .../NormalUserScanQueryMatcher.java             |    1 +
 .../querymatcher/ScanDeleteTracker.java         |    1 +
 .../querymatcher/ScanQueryMatcher.java          |    3 +-
 .../StripeCompactionScanQueryMatcher.java       |    1 +
 .../snapshot/FlushSnapshotSubprocedure.java     |    2 +-
 .../snapshot/RegionServerSnapshotManager.java   |    2 +-
 .../hbase/security/access/AccessController.java |    2 +-
 .../visibility/VisibilityController.java        |    2 +-
 .../hadoop/hbase/snapshot/CreateSnapshot.java   |    2 +-
 .../hadoop/hbase/snapshot/ExportSnapshot.java   |    2 +-
 .../hbase/snapshot/RestoreSnapshotHelper.java   |    2 +-
 .../snapshot/SnapshotDescriptionUtils.java      |    2 +-
 .../hadoop/hbase/snapshot/SnapshotInfo.java     |    2 +-
 .../hadoop/hbase/snapshot/SnapshotManifest.java |    2 +-
 .../hbase/snapshot/SnapshotManifestV1.java      |    2 +-
 .../hbase/snapshot/SnapshotManifestV2.java      |    2 +-
 .../hbase/snapshot/SnapshotReferenceUtil.java   |    2 +-
 .../resources/hbase-webapps/master/snapshot.jsp |    2 +-
 .../hbase-webapps/master/snapshotsStats.jsp     |    2 +-
 .../hbase/client/TestSnapshotFromClient.java    |    2 +-
 .../hbase/coprocessor/TestMasterObserver.java   |    2 +-
 .../master/cleaner/TestSnapshotFromMaster.java  |    2 +-
 .../hbase/regionserver/TestHRegionInfo.java     |    4 +-
 .../querymatcher/TestScanDeleteTracker.java     |    2 +-
 .../security/access/TestAccessController.java   |    2 +-
 .../access/TestWithDisabledAuthorization.java   |    2 +-
 .../hbase/snapshot/SnapshotTestingUtils.java    |    2 +-
 .../hbase/snapshot/TestExportSnapshot.java      |    2 +-
 .../snapshot/TestFlushSnapshotFromClient.java   |    2 +-
 .../hbase/snapshot/TestRegionSnapshotTask.java  |    9 +-
 .../TestRestoreFlushSnapshotFromClient.java     |    2 +-
 .../snapshot/TestRestoreSnapshotHelper.java     |    2 +-
 .../snapshot/TestSnapshotClientRetries.java     |    2 +-
 .../snapshot/TestSnapshotDescriptionUtils.java  |    2 +-
 .../hbase/snapshot/TestSnapshotManifest.java    |    2 +-
 94 files changed, 5991 insertions(+), 5449 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ea5789a7/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index 5b3744a..fa9594a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -43,9 +43,8 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
-import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.quotas.QuotaFilter;
 import org.apache.hadoop.hbase.quotas.QuotaRetriever;
 import org.apache.hadoop.hbase.quotas.QuotaSettings;

http://git-wip-us.apache.org/repos/asf/hbase/blob/ea5789a7/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 2cbeb9a..84871ed 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
@@ -95,6 +95,7 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AbortProcedureRequest;
@@ -156,7 +157,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterRequest
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos;
-import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.protobuf.generated.TableProtos;
 import org.apache.hadoop.hbase.quotas.QuotaFilter;
 import org.apache.hadoop.hbase.quotas.QuotaRetriever;
 import org.apache.hadoop.hbase.quotas.QuotaSettings;
@@ -3174,7 +3175,7 @@ public class HBaseAdmin implements Admin {
           public TableName[] call(int callTimeout) throws Exception {
             HBaseRpcController controller = rpcControllerFactory.newController();
             controller.setCallTimeout(callTimeout);
-            List<HBaseProtos.TableName> tableNames =
+            List<TableProtos.TableName> tableNames =
               master.listTableNamesByNamespace(controller, ListTableNamesByNamespaceRequest.
                 newBuilder().setNamespaceName(name).build())
                 .getTableNameList();

http://git-wip-us.apache.org/repos/asf/hbase/blob/ea5789a7/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index b704f4a..fa44cdd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -145,6 +145,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
 import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
+import org.apache.hadoop.hbase.protobuf.generated.TableProtos;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
@@ -2975,18 +2976,18 @@ public final class ProtobufUtil {
         ", type=" + proto.getMutateType().toString();
   }
 
-  public static TableName toTableName(HBaseProtos.TableName tableNamePB) {
+  public static TableName toTableName(TableProtos.TableName tableNamePB) {
     return TableName.valueOf(tableNamePB.getNamespace().asReadOnlyByteBuffer(),
         tableNamePB.getQualifier().asReadOnlyByteBuffer());
   }
 
-  public static HBaseProtos.TableName toProtoTableName(TableName tableName) {
-    return HBaseProtos.TableName.newBuilder()
+  public static TableProtos.TableName toProtoTableName(TableName tableName) {
+    return TableProtos.TableName.newBuilder()
         .setNamespace(ByteStringer.wrap(tableName.getNamespace()))
         .setQualifier(ByteStringer.wrap(tableName.getQualifier())).build();
   }
 
-  public static TableName[] getTableNameArray(List<HBaseProtos.TableName> tableNamesList) {
+  public static TableName[] getTableNameArray(List<TableProtos.TableName> tableNamesList) {
     if (tableNamesList == null) {
       return new TableName[0];
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ea5789a7/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/ClientSnapshotDescriptionUtils.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/ClientSnapshotDescriptionUtils.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/ClientSnapshotDescriptionUtils.java
index 686dbb8..bba166b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/ClientSnapshotDescriptionUtils.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/ClientSnapshotDescriptionUtils.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.hbase.snapshot;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/ea5789a7/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/CorruptedSnapshotException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/CorruptedSnapshotException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/CorruptedSnapshotException.java
index a1f5911..d29c89c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/CorruptedSnapshotException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/CorruptedSnapshotException.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.hbase.snapshot;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/ea5789a7/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/HBaseSnapshotException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/HBaseSnapshotException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/HBaseSnapshotException.java
index a88d820..8889cfc 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/HBaseSnapshotException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/HBaseSnapshotException.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.snapshot;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 
 /**
  * General exception base class for when a snapshot fails

http://git-wip-us.apache.org/repos/asf/hbase/blob/ea5789a7/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotException.java
index 8ed42e0..25cd583 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotException.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.snapshot;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 
 /**
  * Thrown when a snapshot could not be restored due to a server-side error when restoring it.

http://git-wip-us.apache.org/repos/asf/hbase/blob/ea5789a7/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotCreationException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotCreationException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotCreationException.java
index d550101..324d41f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotCreationException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotCreationException.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.hbase.snapshot;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 
 /**
  * Thrown when a snapshot could not be created due to a server-side error when

http://git-wip-us.apache.org/repos/asf/hbase/blob/ea5789a7/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDoesNotExistException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDoesNotExistException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDoesNotExistException.java
index 6006136..6ba45bd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDoesNotExistException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDoesNotExistException.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.hbase.snapshot;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/ea5789a7/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotExistsException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotExistsException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotExistsException.java
index 0344aff..ff3cdcb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotExistsException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotExistsException.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.hbase.snapshot;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 
 /**
  * Thrown when a snapshot exists but should not

http://git-wip-us.apache.org/repos/asf/hbase/blob/ea5789a7/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java
index afa5892..6385c27 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java
@@ -32,11 +32,11 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotRequest;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotResponse;
-import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;

http://git-wip-us.apache.org/repos/asf/hbase/blob/ea5789a7/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FastLongHistogram.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FastLongHistogram.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FastLongHistogram.java
new file mode 100644
index 0000000..2473bb4
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FastLongHistogram.java
@@ -0,0 +1,319 @@
+/**
+ * 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.util;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * FastLongHistogram is a thread-safe class that estimate distribution of data and computes the
+ * quantiles.
+ */
+@Deprecated
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class FastLongHistogram {
+
+  /**
+   * Default number of bins.
+   */
+  public static final int DEFAULT_NBINS = 255;
+
+  public static final double[] DEFAULT_QUANTILES =
+      new double[]{0.25, 0.5, 0.75, 0.90, 0.95, 0.98, 0.99, 0.999};
+
+  /**
+   * Bins is a class containing a list of buckets(or bins) for estimation histogram of some data.
+   */
+  private static class Bins {
+    private final Counter[] counts;
+    // inclusive
+    private final long binsMin;
+    // exclusive
+    private final long binsMax;
+    private final long bins10XMax;
+    private final AtomicLong min = new AtomicLong(Long.MAX_VALUE);
+    private final AtomicLong max = new AtomicLong(0L);
+
+    private final Counter count = new Counter(0);
+    private final Counter total = new Counter(0);
+
+    // set to true when any of data has been inserted to the Bins. It is set after the counts are
+    // updated.
+    private final AtomicBoolean hasData = new AtomicBoolean(false);
+
+    /**
+     * The constructor for creating a Bins without any prior data.
+     */
+    public Bins(int numBins) {
+      counts = createCounters(numBins + 3);
+      this.binsMin = 1L;
+
+      // These two numbers are total guesses
+      // and should be treated as highly suspect.
+      this.binsMax = 1000;
+      this.bins10XMax = binsMax * 10;
+    }
+
+    /**
+     * The constructor for creating a Bins with last Bins.
+     */
+    public Bins(Bins last, int numOfBins, double minQ, double maxQ) {
+      long[] values = last.getQuantiles(new double[] { minQ, maxQ });
+      long wd = values[1] - values[0] + 1;
+      // expand minQ and maxQ in two ends back assuming uniform distribution
+      this.binsMin = Math.max(0L, (long) (values[0] - wd * minQ));
+      long binsMax = (long) (values[1] + wd * (1 - maxQ)) + 1;
+      // make sure each of bins is at least of width 1
+      this.binsMax = Math.max(binsMax, this.binsMin + numOfBins);
+      this.bins10XMax = Math.max((long) (values[1] + (binsMax - 1) * 9), this.binsMax + 1);
+
+      this.counts = createCounters(numOfBins + 3);
+    }
+
+    private Counter[] createCounters(int num) {
+      Counter[] counters = new Counter[num];
+      for (int i = 0; i < num; i++) {
+        counters[i] = new Counter();
+      }
+      return counters;
+    }
+
+    private int getIndex(long value) {
+      if (value < this.binsMin) {
+        return 0;
+      } else if (value > this.bins10XMax) {
+        return this.counts.length - 1;
+      } else if (value >= this.binsMax) {
+        return this.counts.length - 2;
+      }
+      // compute the position
+      return 1 + (int) ((value - this.binsMin) * (this.counts.length - 3) /
+          (this.binsMax - this.binsMin));
+
+    }
+
+    /**
+     * Adds a value to the histogram.
+     */
+    public void add(long value, long count) {
+      if (value < 0) {
+        // The whole computation is completely thrown off if there are negative numbers
+        //
+        // Normally we would throw an IllegalArgumentException however this is the metrics
+        // system and it should be completely safe at all times.
+        // So silently throw it away.
+        return;
+      }
+      AtomicUtils.updateMin(min, value);
+      AtomicUtils.updateMax(max, value);
+
+      this.count.add(count);
+      this.total.add(value * count);
+
+      int pos = getIndex(value);
+      this.counts[pos].add(count);
+
+      // hasData needs to be updated as last
+      this.hasData.set(true);
+    }
+
+    /**
+     * Computes the quantiles give the ratios.
+     */
+    public long[] getQuantiles(double[] quantiles) {
+      if (!this.hasData.get()) {
+        // No data yet.
+        return new long[quantiles.length];
+      }
+
+      // Make a snapshot of lowerCounter, higherCounter and bins.counts to counts.
+      // This is not synchronized, but since the counter are accumulating, the result is a good
+      // estimation of a snapshot.
+      long[] counts = new long[this.counts.length];
+      long total = 0L;
+      for (int i = 0; i < this.counts.length; i++) {
+        counts[i] = this.counts[i].get();
+        total += counts[i];
+      }
+
+      int rIndex = 0;
+      double qCount = total * quantiles[0];
+      long cum = 0L;
+
+      long[] res = new long[quantiles.length];
+      countsLoop: for (int i = 0; i < counts.length; i++) {
+        // mn and mx define a value range
+        long mn, mx;
+        if (i == 0) {
+          mn = this.min.get();
+          mx = this.binsMin;
+        } else if (i == counts.length - 1) {
+          mn = this.bins10XMax;
+          mx = this.max.get();
+        } else if (i == counts.length - 2) {
+          mn = this.binsMax;
+          mx = this.bins10XMax;
+        } else {
+          mn = this.binsMin + (i - 1) * (this.binsMax - this.binsMin) / (this.counts.length - 3);
+          mx = this.binsMin + i * (this.binsMax - this.binsMin) / (this.counts.length - 3);
+        }
+
+        if (mx < this.min.get()) {
+          continue;
+        }
+        if (mn > this.max.get()) {
+          break;
+        }
+        mn = Math.max(mn, this.min.get());
+        mx = Math.min(mx, this.max.get());
+
+        // lastCum/cum are the corresponding counts to mn/mx
+        double lastCum = cum;
+        cum += counts[i];
+
+        // fill the results for qCount is within current range.
+        while (qCount <= cum) {
+          if (cum == lastCum) {
+            res[rIndex] = mn;
+          } else {
+            res[rIndex] = (long) ((qCount - lastCum) * (mx - mn) / (cum - lastCum) + mn);
+          }
+
+          // move to next quantile
+          rIndex++;
+          if (rIndex >= quantiles.length) {
+            break countsLoop;
+          }
+          qCount = total * quantiles[rIndex];
+        }
+      }
+      // In case quantiles contains values >= 100%
+      for (; rIndex < quantiles.length; rIndex++) {
+        res[rIndex] = this.max.get();
+      }
+
+      return res;
+    }
+
+
+    long getNumAtOrBelow(long val) {
+      final int targetIndex = getIndex(val);
+      long totalToCurrentIndex = 0;
+      for (int i = 0; i <= targetIndex; i++) {
+        totalToCurrentIndex += this.counts[i].get();
+      }
+      return  totalToCurrentIndex;
+    }
+  }
+
+  // The bins counting values. It is replaced with a new one in calling of reset().
+  private volatile Bins bins;
+
+  /**
+   * Constructor.
+   */
+  public FastLongHistogram() {
+    this(DEFAULT_NBINS);
+  }
+
+  /**
+   * Constructor.
+   * @param numOfBins the number of bins for the histogram. A larger value results in more precise
+   *          results but with lower efficiency, and vice versus.
+   */
+  public FastLongHistogram(int numOfBins) {
+    this.bins = new Bins(numOfBins);
+  }
+
+  /**
+   * Constructor setting the bins assuming a uniform distribution within a range.
+   * @param numOfBins the number of bins for the histogram. A larger value results in more precise
+   *          results but with lower efficiency, and vice versus.
+   * @param min lower bound of the region, inclusive.
+   * @param max higher bound of the region, inclusive.
+   */
+  public FastLongHistogram(int numOfBins, long min, long max) {
+    this(numOfBins);
+    Bins bins = new Bins(numOfBins);
+    bins.add(min, 1);
+    bins.add(max, 1);
+    this.bins = new Bins(bins, numOfBins, 0.01, 0.999);
+  }
+
+  private FastLongHistogram(Bins bins) {
+    this.bins = bins;
+  }
+
+  /**
+   * Adds a value to the histogram.
+   */
+  public void add(long value, long count) {
+    this.bins.add(value, count);
+  }
+
+  /**
+   * Computes the quantiles give the ratios.
+   */
+  public long[] getQuantiles(double[] quantiles) {
+    return this.bins.getQuantiles(quantiles);
+  }
+
+  public long[] getQuantiles() {
+    return this.bins.getQuantiles(DEFAULT_QUANTILES);
+  }
+
+  public long getMin() {
+    long min = this.bins.min.get();
+    return min == Long.MAX_VALUE ? 0 : min; // in case it is not initialized
+  }
+
+  public long getMax() {
+    return this.bins.max.get();
+  }
+
+  public long getCount() {
+    return this.bins.count.get();
+  }
+
+  public long getMean() {
+    Bins bins = this.bins;
+    long count = bins.count.get();
+    long total = bins.total.get();
+    if (count == 0) {
+      return 0;
+    }
+    return total / count;
+  }
+
+  public long getNumAtOrBelow(long value) {
+    return this.bins.getNumAtOrBelow(value);
+  }
+
+  /**
+   * Resets the histogram for new counting.
+   */
+  public FastLongHistogram reset() {
+    Bins oldBins = this.bins;
+    this.bins = new Bins(this.bins, this.bins.counts.length - 3, 0.01, 0.99);
+    return new FastLongHistogram(oldBins);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ea5789a7/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestFastLongHistogram.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestFastLongHistogram.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestFastLongHistogram.java
new file mode 100644
index 0000000..7304e2d
--- /dev/null
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestFastLongHistogram.java
@@ -0,0 +1,132 @@
+/**
+ * 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.util;
+
+import java.util.Arrays;
+import java.util.Random;
+
+
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Testcases for FastLongHistogram.
+ */
+@Category(SmallTests.class)
+public class TestFastLongHistogram {
+
+  private static void doTestUniform(FastLongHistogram hist) {
+    long[] VALUES = { 0, 10, 20, 30, 40, 50 };
+    double[] qs = new double[VALUES.length];
+    for (int i = 0; i < qs.length; i++) {
+      qs[i] = (double) VALUES[i] / VALUES[VALUES.length - 1];
+    }
+
+    for (int i = 0; i < 10; i++) {
+      for (long v : VALUES) {
+        hist.add(v, 1);
+      }
+      long[] vals = hist.getQuantiles(qs);
+      System.out.println(Arrays.toString(vals));
+      for (int j = 0; j < qs.length; j++) {
+        Assert.assertTrue(j + "-th element org: " + VALUES[j] + ", act: " + vals[j],
+          Math.abs(vals[j] - VALUES[j]) <= 10);
+      }
+      hist.reset();
+    }
+  }
+
+  @Test
+  public void testUniform() {
+    FastLongHistogram hist = new FastLongHistogram(100, 0, 50);
+    doTestUniform(hist);
+  }
+
+  @Test
+  public void testAdaptionOfChange() {
+    // assumes the uniform distribution
+    FastLongHistogram hist = new FastLongHistogram(100, 0, 100);
+
+    Random rand = new Random();
+
+    for (int n = 0; n < 10; n++) {
+      for (int i = 0; i < 900; i++) {
+        hist.add(rand.nextInt(100), 1);
+      }
+
+      // add 10% outliers, this breaks the assumption, hope bin10xMax works
+      for (int i = 0; i < 100; i++) {
+        hist.add(1000 + rand.nextInt(100), 1);
+      }
+
+      long[] vals = hist.getQuantiles(new double[] { 0.25, 0.75, 0.95 });
+      System.out.println(Arrays.toString(vals));
+      if (n == 0) {
+        Assert.assertTrue("Out of possible value", vals[0] >= 0 && vals[0] <= 50);
+        Assert.assertTrue("Out of possible value", vals[1] >= 50 && vals[1] <= 100);
+        Assert.assertTrue("Out of possible value", vals[2] >= 900 && vals[2] <= 1100);
+      }
+
+      hist.reset();
+    }
+  }
+
+
+  @Test
+  public void testGetNumAtOrBelow() {
+    long[] VALUES = { 1, 10, 20, 30, 40, 50 };
+
+    FastLongHistogram h = new FastLongHistogram();
+    for (long v : VALUES) {
+      for (int i = 0; i < 100; i++) {
+        h.add(v, 1);
+      }
+    }
+
+    h.add(Integer.MAX_VALUE, 1);
+
+    h.reset();
+
+    for (long v : VALUES) {
+      for (int i = 0; i < 100; i++) {
+        h.add(v, 1);
+      }
+    }
+    // Add something way out there to make sure it doesn't throw off the counts.
+    h.add(Integer.MAX_VALUE, 1);
+
+    assertEquals(100, h.getNumAtOrBelow(1));
+    assertEquals(200, h.getNumAtOrBelow(11));
+    assertEquals(601, h.getNumAtOrBelow(Long.MAX_VALUE));
+  }
+
+
+  @Test
+  public void testSameValues() {
+    FastLongHistogram hist = new FastLongHistogram(100);
+
+    hist.add(50, 100);
+
+    hist.reset();
+    doTestUniform(hist);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ea5789a7/hbase-protocol/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-protocol/pom.xml b/hbase-protocol/pom.xml
index 9424691..496b3cf 100644
--- a/hbase-protocol/pom.xml
+++ b/hbase-protocol/pom.xml
@@ -197,6 +197,7 @@
                           <include>RowProcessor.proto</include>
                           <include>SecureBulkLoad.proto</include>
                           <include>Snapshot.proto</include>
+                          <include>Table.proto</include>
                           <include>Tracing.proto</include>
                           <include>VisibilityLabels.proto</include>
                           <include>WAL.proto</include>

http://git-wip-us.apache.org/repos/asf/hbase/blob/ea5789a7/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AccessControlProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AccessControlProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AccessControlProtos.java
index b72e6e5..161282c 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AccessControlProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AccessControlProtos.java
@@ -1285,11 +1285,11 @@ public final class AccessControlProtos {
     /**
      * <code>optional .hbase.pb.TableName table_name = 1;</code>
      */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName();
+    org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName getTableName();
     /**
      * <code>optional .hbase.pb.TableName table_name = 1;</code>
      */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder();
+    org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableNameOrBuilder getTableNameOrBuilder();
 
     // optional bytes family = 2;
     /**
@@ -1377,11 +1377,11 @@ public final class AccessControlProtos {
               break;
             }
             case 10: {
-              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null;
+              org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName.Builder subBuilder = null;
               if (((bitField0_ & 0x00000001) == 0x00000001)) {
                 subBuilder = tableName_.toBuilder();
               }
-              tableName_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry);
+              tableName_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName.PARSER, extensionRegistry);
               if (subBuilder != null) {
                 subBuilder.mergeFrom(tableName_);
                 tableName_ = subBuilder.buildPartial();
@@ -1477,7 +1477,7 @@ public final class AccessControlProtos {
     private int bitField0_;
     // optional .hbase.pb.TableName table_name = 1;
     public static final int TABLE_NAME_FIELD_NUMBER = 1;
-    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_;
+    private org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName tableName_;
     /**
      * <code>optional .hbase.pb.TableName table_name = 1;</code>
      */
@@ -1487,13 +1487,13 @@ public final class AccessControlProtos {
     /**
      * <code>optional .hbase.pb.TableName table_name = 1;</code>
      */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+    public org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName getTableName() {
       return tableName_;
     }
     /**
      * <code>optional .hbase.pb.TableName table_name = 1;</code>
      */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+    public org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableNameOrBuilder getTableNameOrBuilder() {
       return tableName_;
     }
 
@@ -1552,7 +1552,7 @@ public final class AccessControlProtos {
     }
 
     private void initFields() {
-      tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+      tableName_ = org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName.getDefaultInstance();
       family_ = com.google.protobuf.ByteString.EMPTY;
       qualifier_ = com.google.protobuf.ByteString.EMPTY;
       action_ = java.util.Collections.emptyList();
@@ -1797,7 +1797,7 @@ public final class AccessControlProtos {
       public Builder clear() {
         super.clear();
         if (tableNameBuilder_ == null) {
-          tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+          tableName_ = org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName.getDefaultInstance();
         } else {
           tableNameBuilder_.clear();
         }
@@ -1926,9 +1926,9 @@ public final class AccessControlProtos {
       private int bitField0_;
 
       // optional .hbase.pb.TableName table_name = 1;
-      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+      private org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName tableName_ = org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName.getDefaultInstance();
       private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
+          org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableNameOrBuilder> tableNameBuilder_;
       /**
        * <code>optional .hbase.pb.TableName table_name = 1;</code>
        */
@@ -1938,7 +1938,7 @@ public final class AccessControlProtos {
       /**
        * <code>optional .hbase.pb.TableName table_name = 1;</code>
        */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+      public org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName getTableName() {
         if (tableNameBuilder_ == null) {
           return tableName_;
         } else {
@@ -1948,7 +1948,7 @@ public final class AccessControlProtos {
       /**
        * <code>optional .hbase.pb.TableName table_name = 1;</code>
        */
-      public Builder setTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+      public Builder setTableName(org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName value) {
         if (tableNameBuilder_ == null) {
           if (value == null) {
             throw new NullPointerException();
@@ -1965,7 +1965,7 @@ public final class AccessControlProtos {
        * <code>optional .hbase.pb.TableName table_name = 1;</code>
        */
       public Builder setTableName(
-          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
+          org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName.Builder builderForValue) {
         if (tableNameBuilder_ == null) {
           tableName_ = builderForValue.build();
           onChanged();
@@ -1978,12 +1978,12 @@ public final class AccessControlProtos {
       /**
        * <code>optional .hbase.pb.TableName table_name = 1;</code>
        */
-      public Builder mergeTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+      public Builder mergeTableName(org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName value) {
         if (tableNameBuilder_ == null) {
           if (((bitField0_ & 0x00000001) == 0x00000001) &&
-              tableName_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()) {
+              tableName_ != org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName.getDefaultInstance()) {
             tableName_ =
-              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.newBuilder(tableName_).mergeFrom(value).buildPartial();
+              org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName.newBuilder(tableName_).mergeFrom(value).buildPartial();
           } else {
             tableName_ = value;
           }
@@ -1999,7 +1999,7 @@ public final class AccessControlProtos {
        */
       public Builder clearTableName() {
         if (tableNameBuilder_ == null) {
-          tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+          tableName_ = org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName.getDefaultInstance();
           onChanged();
         } else {
           tableNameBuilder_.clear();
@@ -2010,7 +2010,7 @@ public final class AccessControlProtos {
       /**
        * <code>optional .hbase.pb.TableName table_name = 1;</code>
        */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder getTableNameBuilder() {
+      public org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName.Builder getTableNameBuilder() {
         bitField0_ |= 0x00000001;
         onChanged();
         return getTableNameFieldBuilder().getBuilder();
@@ -2018,7 +2018,7 @@ public final class AccessControlProtos {
       /**
        * <code>optional .hbase.pb.TableName table_name = 1;</code>
        */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+      public org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableNameOrBuilder getTableNameOrBuilder() {
         if (tableNameBuilder_ != null) {
           return tableNameBuilder_.getMessageOrBuilder();
         } else {
@@ -2029,11 +2029,11 @@ public final class AccessControlProtos {
        * <code>optional .hbase.pb.TableName table_name = 1;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+          org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableNameOrBuilder> 
           getTableNameFieldBuilder() {
         if (tableNameBuilder_ == null) {
           tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilder<
-              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
+              org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableNameOrBuilder>(
                   tableName_,
                   getParentForChildren(),
                   isClean());
@@ -7462,11 +7462,11 @@ public final class AccessControlProtos {
     /**
      * <code>optional .hbase.pb.TableName table_name = 2;</code>
      */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName();
+    org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName getTableName();
     /**
      * <code>optional .hbase.pb.TableName table_name = 2;</code>
      */
-    org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder();
+    org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableNameOrBuilder getTableNameOrBuilder();
 
     // optional bytes namespace_name = 3;
     /**
@@ -7541,11 +7541,11 @@ public final class AccessControlProtos {
               break;
             }
             case 18: {
-              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null;
+              org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName.Builder subBuilder = null;
               if (((bitField0_ & 0x00000002) == 0x00000002)) {
                 subBuilder = tableName_.toBuilder();
               }
-              tableName_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry);
+              tableName_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName.PARSER, extensionRegistry);
               if (subBuilder != null) {
                 subBuilder.mergeFrom(tableName_);
                 tableName_ = subBuilder.buildPartial();
@@ -7616,7 +7616,7 @@ public final class AccessControlProtos {
 
     // optional .hbase.pb.TableName table_name = 2;
     public static final int TABLE_NAME_FIELD_NUMBER = 2;
-    private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_;
+    private org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName tableName_;
     /**
      * <code>optional .hbase.pb.TableName table_name = 2;</code>
      */
@@ -7626,13 +7626,13 @@ public final class AccessControlProtos {
     /**
      * <code>optional .hbase.pb.TableName table_name = 2;</code>
      */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+    public org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName getTableName() {
       return tableName_;
     }
     /**
      * <code>optional .hbase.pb.TableName table_name = 2;</code>
      */
-    public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+    public org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableNameOrBuilder getTableNameOrBuilder() {
       return tableName_;
     }
 
@@ -7654,7 +7654,7 @@ public final class AccessControlProtos {
 
     private void initFields() {
       type_ = org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.Permission.Type.Global;
-      tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+      tableName_ = org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName.getDefaultInstance();
       namespaceName_ = com.google.protobuf.ByteString.EMPTY;
     }
     private byte memoizedIsInitialized = -1;
@@ -7881,7 +7881,7 @@ public final class AccessControlProtos {
         type_ = org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.Permission.Type.Global;
         bitField0_ = (bitField0_ & ~0x00000001);
         if (tableNameBuilder_ == null) {
-          tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+          tableName_ = org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName.getDefaultInstance();
         } else {
           tableNameBuilder_.clear();
         }
@@ -8027,9 +8027,9 @@ public final class AccessControlProtos {
       }
 
       // optional .hbase.pb.TableName table_name = 2;
-      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+      private org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName tableName_ = org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName.getDefaultInstance();
       private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
+          org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableNameOrBuilder> tableNameBuilder_;
       /**
        * <code>optional .hbase.pb.TableName table_name = 2;</code>
        */
@@ -8039,7 +8039,7 @@ public final class AccessControlProtos {
       /**
        * <code>optional .hbase.pb.TableName table_name = 2;</code>
        */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+      public org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName getTableName() {
         if (tableNameBuilder_ == null) {
           return tableName_;
         } else {
@@ -8049,7 +8049,7 @@ public final class AccessControlProtos {
       /**
        * <code>optional .hbase.pb.TableName table_name = 2;</code>
        */
-      public Builder setTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+      public Builder setTableName(org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName value) {
         if (tableNameBuilder_ == null) {
           if (value == null) {
             throw new NullPointerException();
@@ -8066,7 +8066,7 @@ public final class AccessControlProtos {
        * <code>optional .hbase.pb.TableName table_name = 2;</code>
        */
       public Builder setTableName(
-          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
+          org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName.Builder builderForValue) {
         if (tableNameBuilder_ == null) {
           tableName_ = builderForValue.build();
           onChanged();
@@ -8079,12 +8079,12 @@ public final class AccessControlProtos {
       /**
        * <code>optional .hbase.pb.TableName table_name = 2;</code>
        */
-      public Builder mergeTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+      public Builder mergeTableName(org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName value) {
         if (tableNameBuilder_ == null) {
           if (((bitField0_ & 0x00000002) == 0x00000002) &&
-              tableName_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()) {
+              tableName_ != org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName.getDefaultInstance()) {
             tableName_ =
-              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.newBuilder(tableName_).mergeFrom(value).buildPartial();
+              org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName.newBuilder(tableName_).mergeFrom(value).buildPartial();
           } else {
             tableName_ = value;
           }
@@ -8100,7 +8100,7 @@ public final class AccessControlProtos {
        */
       public Builder clearTableName() {
         if (tableNameBuilder_ == null) {
-          tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+          tableName_ = org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName.getDefaultInstance();
           onChanged();
         } else {
           tableNameBuilder_.clear();
@@ -8111,7 +8111,7 @@ public final class AccessControlProtos {
       /**
        * <code>optional .hbase.pb.TableName table_name = 2;</code>
        */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder getTableNameBuilder() {
+      public org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName.Builder getTableNameBuilder() {
         bitField0_ |= 0x00000002;
         onChanged();
         return getTableNameFieldBuilder().getBuilder();
@@ -8119,7 +8119,7 @@ public final class AccessControlProtos {
       /**
        * <code>optional .hbase.pb.TableName table_name = 2;</code>
        */
-      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+      public org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableNameOrBuilder getTableNameOrBuilder() {
         if (tableNameBuilder_ != null) {
           return tableNameBuilder_.getMessageOrBuilder();
         } else {
@@ -8130,11 +8130,11 @@ public final class AccessControlProtos {
        * <code>optional .hbase.pb.TableName table_name = 2;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> 
+          org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableNameOrBuilder> 
           getTableNameFieldBuilder() {
         if (tableNameBuilder_ == null) {
           tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilder<
-              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
+              org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.TableProtos.TableNameOrBuilder>(
                   tableName_,
                   getParentForChildren(),
                   isClean());
@@ -10499,7 +10499,7 @@ public final class AccessControlProtos {
       descriptor;
   static {
     java.lang.String[] descriptorData = {
-      "\n\023AccessControl.proto\022\010hbase.pb\032\013HBase.p" +
+      "\n\023AccessControl.proto\022\010hbase.pb\032\013Table.p" +
       "roto\"\314\002\n\nPermission\022\'\n\004type\030\001 \002(\0162\031.hbas" +
       "e.pb.Permission.Type\0225\n\021global_permissio" +
       "n\030\002 \001(\0132\032.hbase.pb.GlobalPermission\022;\n\024n" +
@@ -10648,7 +10648,7 @@ public final class AccessControlProtos {
     com.google.protobuf.Descriptors.FileDescriptor
       .internalBuildGeneratedFileFrom(descriptorData,
         new com.google.protobuf.Descriptors.FileDescriptor[] {
-          org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.getDescriptor(),
+          org.apache.hadoop.hbase.protobuf.generated.TableProtos.getDescriptor(),
         }, assigner);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ea5789a7/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
index 12950f7..4bd3f5f 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
@@ -22646,7 +22646,7 @@ public final class ClientProtos {
        * </pre>
        */
       private com.google.protobuf.SingleFieldBuilder<
-          org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CursorOrBuilder>
+          org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Cursor.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CursorOrBuilder> 
           getCursorFieldBuilder() {
         if (cursorBuilder_ == null) {
           cursorBuilder_ = new com.google.protobuf.SingleFieldBuilder<