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:48 UTC

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

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