From 2f7cf9ccb9473fc0882df2d02ab1c373f951e8c1 Mon Sep 17 00:00:00 2001 From: chenheng Date: Sun, 22 Nov 2015 13:25:46 +0800 Subject: [PATCH] HBASE-11393 Replication TableCfs should be a PB object rather than a string --- .../hbase/client/replication/ReplicationAdmin.java | 169 +- .../hbase/client/replication/TableCFsHelper.java | 327 ++++ .../hadoop/hbase/replication/ReplicationPeer.java | 5 +- .../hbase/replication/ReplicationPeerConfig.java | 10 + .../hbase/replication/ReplicationPeerZKImpl.java | 58 +- .../hadoop/hbase/replication/ReplicationPeers.java | 20 +- .../hbase/replication/ReplicationPeersZKImpl.java | 168 +- .../hbase/replication/ReplicationStateZKBase.java | 16 + .../hbase/protobuf/generated/ZooKeeperProtos.java | 1713 +++++++++++++++++++- hbase-protocol/src/main/protobuf/ZooKeeper.proto | 10 + .../org/apache/hadoop/hbase/master/HMaster.java | 10 +- .../hbase/replication/TableCfWALEntryFilter.java | 9 +- .../hbase/replication/master/TableCFsUpdater.java | 118 ++ .../hbase/client/TestReplicaWithCluster.java | 6 +- .../client/replication/TestReplicationAdmin.java | 196 ++- .../hbase/replication/TestMasterReplication.java | 5 +- .../replication/TestMultiSlaveReplication.java | 10 +- .../replication/TestPerTableCFReplication.java | 190 ++- .../hbase/replication/TestReplicationBase.java | 4 +- .../replication/TestReplicationSmallTests.java | 5 +- .../replication/TestReplicationStateBasic.java | 8 +- .../replication/TestReplicationSyncUpTool.java | 4 +- .../replication/TestReplicationTrackerZKImpl.java | 10 +- .../hbase/replication/TestReplicationWithTags.java | 6 +- ...tVisibilityLabelReplicationWithExpAsString.java | 9 +- .../TestVisibilityLabelsReplication.java | 5 +- .../hadoop/hbase/util/TestHBaseFsckOneRS.java | 5 +- .../src/main/ruby/hbase/replication_admin.rb | 10 +- .../ruby/shell/commands/append_peer_tableCFs.rb | 2 +- .../ruby/shell/commands/remove_peer_tableCFs.rb | 4 +- .../main/ruby/shell/commands/set_peer_tableCFs.rb | 2 +- 31 files changed, 2627 insertions(+), 487 deletions(-) create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/TableCFsHelper.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java index 8bd1267..fa88c42 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java @@ -28,8 +28,7 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Set; - -import org.apache.commons.lang.StringUtils; +import com.google.common.collect.Lists; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -45,6 +44,7 @@ import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.RegionLocator; +import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationFactory; import org.apache.hadoop.hbase.replication.ReplicationPeer; @@ -58,9 +58,6 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.data.Stat; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.Lists; - /** *

* This class provides the administrative interface to HBase cluster @@ -169,27 +166,6 @@ public class ReplicationAdmin implements Closeable { } /** - * Add a new peer cluster to replicate to. - * @param id a short name that identifies the cluster - * @param clusterKey the concatenation of the slave cluster's - * hbase.zookeeper.quorum:hbase.zookeeper.property.clientPort:zookeeper.znode.parent - * @throws IllegalStateException if there's already one slave since - * multi-slave isn't supported yet. - * @deprecated Use addPeer(String, ReplicationPeerConfig, Map) instead. - */ - @Deprecated - public void addPeer(String id, String clusterKey) throws ReplicationException { - this.addPeer(id, new ReplicationPeerConfig().setClusterKey(clusterKey), null); - } - - @Deprecated - public void addPeer(String id, String clusterKey, String tableCFs) - throws ReplicationException { - this.replicationPeers.addPeer(id, - new ReplicationPeerConfig().setClusterKey(clusterKey), tableCFs); - } - - /** * Add a new remote slave cluster for replication. * @param id a short name that identifies the cluster * @param peerConfig configuration for the replication slave cluster @@ -200,77 +176,11 @@ public class ReplicationAdmin implements Closeable { */ public void addPeer(String id, ReplicationPeerConfig peerConfig, Map> tableCfs) throws ReplicationException { - this.replicationPeers.addPeer(id, peerConfig, getTableCfsStr(tableCfs)); - } - - public static Map> parseTableCFsFromConfig(String tableCFsConfig) { - if (tableCFsConfig == null || tableCFsConfig.trim().length() == 0) { - return null; - } - - Map> tableCFsMap = null; - // TODO: This should be a PB object rather than a String to be parsed!! See HBASE-11393 - // parse out (table, cf-list) pairs from tableCFsConfig - // format: "table1:cf1,cf2;table2:cfA,cfB" - String[] tables = tableCFsConfig.split(";"); - for (String tab : tables) { - // 1 ignore empty table config - tab = tab.trim(); - if (tab.length() == 0) { - continue; - } - // 2 split to "table" and "cf1,cf2" - // for each table: "table:cf1,cf2" or "table" - String[] pair = tab.split(":"); - String tabName = pair[0].trim(); - if (pair.length > 2 || tabName.length() == 0) { - LOG.error("ignore invalid tableCFs setting: " + tab); - continue; - } - - // 3 parse "cf1,cf2" part to List - List cfs = null; - if (pair.length == 2) { - String[] cfsList = pair[1].split(","); - for (String cf : cfsList) { - String cfName = cf.trim(); - if (cfName.length() > 0) { - if (cfs == null) { - cfs = new ArrayList(); - } - cfs.add(cfName); - } - } - } - - // 4 put > to map - if (tableCFsMap == null) { - tableCFsMap = new HashMap>(); - } - tableCFsMap.put(TableName.valueOf(tabName), cfs); - } - return tableCFsMap; - } - - @VisibleForTesting - static String getTableCfsStr(Map> tableCfs) { - String tableCfsStr = null; - if (tableCfs != null) { - // Format: table1:cf1,cf2;table2:cfA,cfB;table3 - StringBuilder builder = new StringBuilder(); - for (Entry> entry : tableCfs.entrySet()) { - if (builder.length() > 0) { - builder.append(";"); - } - builder.append(entry.getKey()); - if (entry.getValue() != null && !entry.getValue().isEmpty()) { - builder.append(":"); - builder.append(StringUtils.join(entry.getValue(), ",")); - } - } - tableCfsStr = builder.toString(); + ZooKeeperProtos.TableCFs tableCFs = TableCFsHelper.convert(tableCfs); + if (tableCFs != null) { + peerConfig.setTableCFs(tableCFs); } - return tableCfsStr; + this.replicationPeers.addPeer(id, peerConfig); } /** @@ -305,22 +215,6 @@ public class ReplicationAdmin implements Closeable { return this.replicationPeers.getAllPeerIds().size(); } - /** - * Map of this cluster's peers for display. - * @return A map of peer ids to peer cluster keys - * @deprecated use {@link #listPeerConfigs()} - */ - @Deprecated - public Map listPeers() { - Map peers = this.listPeerConfigs(); - Map ret = new HashMap(peers.size()); - - for (Map.Entry entry : peers.entrySet()) { - ret.put(entry.getKey(), entry.getValue().getClusterKey()); - } - return ret; - } - public Map listPeerConfigs() { return this.replicationPeers.getAllPeerConfigs(); } @@ -333,29 +227,11 @@ public class ReplicationAdmin implements Closeable { * Get the replicable table-cf config of the specified peer. * @param id a short name that identifies the cluster */ - public String getPeerTableCFs(String id) throws ReplicationException { + public Map> getPeerTableCFsMap(String id) + throws ReplicationException { return this.replicationPeers.getPeerTableCFsConfig(id); } - /** - * Set the replicable table-cf config of the specified peer - * @param id a short name that identifies the cluster - * @deprecated use {@link #setPeerTableCFs(String, Map)} - */ - @Deprecated - public void setPeerTableCFs(String id, String tableCFs) throws ReplicationException { - this.replicationPeers.setPeerTableCFsConfig(id, tableCFs); - } - - /** - * Append the replicable table-cf config of the specified peer - * @param id a short that identifies the cluster - * @param tableCfs table-cfs config str - * @throws ReplicationException - */ - public void appendPeerTableCFs(String id, String tableCfs) throws ReplicationException { - appendPeerTableCFs(id, parseTableCFsFromConfig(tableCfs)); - } /** * Append the replicable table-cf config of the specified peer @@ -368,12 +244,11 @@ public class ReplicationAdmin implements Closeable { if (tableCfs == null) { throw new ReplicationException("tableCfs is null"); } - Map> preTableCfs = parseTableCFsFromConfig(getPeerTableCFs(id)); + Map> preTableCfs = this.replicationPeers.getPeerTableCFsConfig(id); if (preTableCfs == null) { setPeerTableCFs(id, tableCfs); return; } - for (Map.Entry> entry : tableCfs.entrySet()) { TableName table = entry.getKey(); Collection appendCfs = entry.getValue(); @@ -385,6 +260,7 @@ public class ReplicationAdmin implements Closeable { Set cfSet = new HashSet(cfs); cfSet.addAll(appendCfs); preTableCfs.put(table, Lists.newArrayList(cfSet)); + } } else { if (appendCfs == null || appendCfs.isEmpty()) { @@ -398,19 +274,9 @@ public class ReplicationAdmin implements Closeable { } /** - * Remove some table-cfs from table-cfs config of the specified peer - * @param id a short name that identifies the cluster - * @param tableCf table-cfs config str - * @throws ReplicationException - */ - public void removePeerTableCFs(String id, String tableCf) throws ReplicationException { - removePeerTableCFs(id, parseTableCFsFromConfig(tableCf)); - } - - /** * Remove some table-cfs from config of the specified peer * @param id a short name that identifies the cluster - * @param tableCfs A map from tableName to column family names + * @param cfs A map from tableName to column family names * @throws ReplicationException */ public void removePeerTableCFs(String id, Map> tableCfs) @@ -418,12 +284,12 @@ public class ReplicationAdmin implements Closeable { if (tableCfs == null) { throw new ReplicationException("tableCfs is null"); } - - Map> preTableCfs = parseTableCFsFromConfig(getPeerTableCFs(id)); + Map> preTableCfs = this.replicationPeers.getPeerTableCFsConfig(id); if (preTableCfs == null) { throw new ReplicationException("Table-Cfs for peer" + id + " is null"); } for (Map.Entry> entry: tableCfs.entrySet()) { + TableName table = entry.getKey(); Collection removeCfs = entry.getValue(); if (preTableCfs.containsKey(table)) { @@ -447,6 +313,7 @@ public class ReplicationAdmin implements Closeable { } } else { throw new ReplicationException("No table: " + table + " in table-cfs config of peer: " + id); + } } setPeerTableCFs(id, preTableCfs); @@ -462,7 +329,7 @@ public class ReplicationAdmin implements Closeable { */ public void setPeerTableCFs(String id, Map> tableCfs) throws ReplicationException { - this.replicationPeers.setPeerTableCFsConfig(id, getTableCfsStr(tableCfs)); + this.replicationPeers.setPeerTableCFsConfig(id, tableCfs); } /** @@ -640,9 +507,9 @@ public class ReplicationAdmin implements Closeable { try { ZKUtil.applyClusterKeyToConf(peerConf, clusterKey); Pair pair = this.replicationPeers.getPeerConf(peerId); - ReplicationPeer peer = new ReplicationPeerZKImpl(peerConf, peerId, pair.getFirst()); - s = - zkw.getRecoverableZooKeeper().exists(peerConf.get(HConstants.ZOOKEEPER_ZNODE_PARENT), + ReplicationPeer peer = new ReplicationPeerZKImpl(zkw, peerConf, peerId, pair.getFirst(), + this.connection); + s = zkw.getRecoverableZooKeeper().exists(peerConf.get(HConstants.ZOOKEEPER_ZNODE_PARENT), null); if (null == s) { LOG.info(peerId + ' ' + clusterKey + " is invalid now."); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/TableCFsHelper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/TableCFsHelper.java new file mode 100644 index 0000000..ddf2825 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/TableCFsHelper.java @@ -0,0 +1,327 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.client.replication; + +import com.google.protobuf.ByteString; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.exceptions.DeserializationException; +import org.apache.hadoop.hbase.protobuf.ProtobufMagic; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; +import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; +import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; +import org.apache.hadoop.hbase.util.ArrayUtils; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Strings; +import org.apache.hadoop.util.StringUtils; + +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.HashMap; +import java.util.ArrayList; + +/** + * Helper for TableCFs Operations. + */ +@InterfaceAudience.Private +@InterfaceStability.Stable +public final class TableCFsHelper { + + private TableCFsHelper() {} + + /** convert map to TableCFs Object */ + public static ZooKeeperProtos.TableCFs convert( + Map> tableCfs) { + if (tableCfs == null) { + return null; + } + ZooKeeperProtos.TableCFs.Builder tableCFsBuilder = ZooKeeperProtos.TableCFs.newBuilder(); + ZooKeeperProtos.TableCF.Builder tableCFBuilder = ZooKeeperProtos.TableCF.newBuilder(); + for (Map.Entry> entry : tableCfs.entrySet()) { + tableCFBuilder.clear(); + tableCFBuilder.setTableName(ProtobufUtil.toProtoTableName(entry.getKey())); + if (entry.getValue() != null && !entry.getValue().isEmpty()) { + for (String value : entry.getValue()) { + tableCFBuilder.addFamilies(ByteString.copyFromUtf8(value)); + } + } + tableCFsBuilder.addTableCfs(tableCFBuilder.build()); + } + return tableCFsBuilder.build(); + } + + /** + * Convert string to TableCFs Object. + * This is only for read TableCFs information from TableCF node. + * Input String Format: table1:cf1,cf2;table2:cfA,cfB;table3. + * */ + public static ZooKeeperProtos.TableCFs convert(String tableCFsConfig) { + if (tableCFsConfig == null || tableCFsConfig.trim().length() == 0) { + return null; + } + + ZooKeeperProtos.TableCFs.Builder tableCFsBuilder = ZooKeeperProtos.TableCFs.newBuilder(); + ZooKeeperProtos.TableCF.Builder tableCFBuilder = ZooKeeperProtos.TableCF.newBuilder(); + + String[] tables = tableCFsConfig.split(";"); + for (String tab : tables) { + // 1 ignore empty table config + tab = tab.trim(); + if (tab.length() == 0) { + continue; + } + // 2 split to "table" and "cf1,cf2" + // for each table: "table#cf1,cf2" or "table" + String[] pair = tab.split(":"); + String tabName = pair[0].trim(); + if (pair.length > 2 || tabName.length() == 0) { + continue; + } + tableCFBuilder.clear(); + tableCFBuilder.setTableName( + ProtobufUtil.toProtoTableName(TableName.valueOf(tabName))); + // 3 parse "cf1,cf2" part to List + if (pair.length == 2) { + String[] cfsList = pair[1].split(","); + for (String cf : cfsList) { + String cfName = cf.trim(); + if (cfName.length() > 0) { + tableCFBuilder.addFamilies(ByteString.copyFromUtf8(cfName)); + } + } + } + tableCFsBuilder.addTableCfs(tableCFBuilder.build()); + } + return tableCFsBuilder.build(); + } + + /** + * Convert TableCFs Object to String. + * Output String Format: ns1.table1:cf1,cf2;ns2.table2:cfA,cfB;table3 + * */ + public static String convert(ZooKeeperProtos.TableCFs tableCFs) { + StringBuilder sb = new StringBuilder(); + for (int i = 0; i 0) { + sb.deleteCharAt(sb.length() - 1); + } + return sb.toString(); + } + + /** + * Get TableCF in TableCFs, if not exist, return null. + * */ + public static ZooKeeperProtos.TableCF getTableCF(ZooKeeperProtos.TableCFs tableCFs, + String table) { + for (int i = 0; i < tableCFs.getTableCfsCount(); i++) { + ZooKeeperProtos.TableCF tableCF = tableCFs.getTableCfs(i); + if (tableCF.getTableName().getQualifier().toStringUtf8().equals(table)) { + return tableCF; + } + } + return null; + } + + /** + * Get index of TableCF in TableCFs, if not exist, return -1. + * */ + public static int getTableCFIndex(ZooKeeperProtos.TableCFs tableCFs, + String table) { + for (int i = 0; i < tableCFs.getTableCfsCount(); i++) { + ZooKeeperProtos.TableCF tableCF = tableCFs.getTableCfs(i); + if (tableCF.getTableName().getQualifier().toStringUtf8().equals(table)) { + return i; + } + } + return -1; + } + + /** + * Convert TableCFs to bytes, add PB_MAGIC in header. + * */ + public static byte[] toByteArray(final ZooKeeperProtos.TableCFs tableCFs) { + if (tableCFs != null) { + return ProtobufUtil.prependPBMagic(tableCFs.toByteArray()); + } else { + return ProtobufMagic.PB_MAGIC; + } + } + + /** + * Parse bytes into TableCFs. + * It is used for backward compatibility. + * Old format bytes have no PB_MAGIC Header + * */ + public static ZooKeeperProtos.TableCFs parseTableCFs(byte[] bytes) throws IOException { + if (bytes == null) { + return null; + } + if (ProtobufUtil.isPBMagicPrefix(bytes)) { + int pblen = ProtobufUtil.lengthOfPBMagic(); + ZooKeeperProtos.TableCFs.Builder builder = ZooKeeperProtos.TableCFs.newBuilder(); + ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen); + return builder.build(); + } else { + // Old format is string, we should parse it. + return TableCFsHelper.convert(Bytes.toString(bytes)); + } + } + + /** + * Convert tableCFs string into Map. + * */ + public static Map> parseTableCFsFromConfig(String tableCFsConfig) { + ZooKeeperProtos.TableCFs tableCFs = convert(tableCFsConfig); + return convert2Map(tableCFs); + } + + public static String convert2String(Map> tableNameListMap) { + return convert(convert(tableNameListMap)); + } + + /** + * Convert tableCFs Object to Map. + * */ + public static Map> convert2Map(ZooKeeperProtos.TableCFs tableCFs) { + if (tableCFs == null || tableCFs.getTableCfsCount() == 0) { + return null; + } + Map> tableCFsMap = new HashMap>(); + for (int i = 0; i < tableCFs.getTableCfsCount(); i++) { + ZooKeeperProtos.TableCF tableCF = tableCFs.getTableCfs(i); + List families = new ArrayList<>(); + for (int j = 0; j < tableCF.getFamiliesCount(); j++) { + families.add(tableCF.getFamilies(j).toStringUtf8()); + } + if (families.size() > 0) { + tableCFsMap.put(ProtobufUtil.toTableName(tableCF.getTableName()), families); + } else { + tableCFsMap.put(ProtobufUtil.toTableName(tableCF.getTableName()), null); + } + } + + return tableCFsMap; + } + + /** + * @param bytes Content of a peer znode. + * @return ClusterKey parsed from the passed bytes. + * @throws DeserializationException + */ + public static ReplicationPeerConfig parsePeerFrom(final byte[] bytes) + throws DeserializationException { + if (ProtobufUtil.isPBMagicPrefix(bytes)) { + int pblen = ProtobufUtil.lengthOfPBMagic(); + ZooKeeperProtos.ReplicationPeer.Builder builder = + ZooKeeperProtos.ReplicationPeer.newBuilder(); + ZooKeeperProtos.ReplicationPeer peer; + try { + ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen); + peer = builder.build(); + } catch (IOException e) { + throw new DeserializationException(e); + } + return convert(peer); + } else { + if (bytes.length > 0) { + return new ReplicationPeerConfig().setClusterKey(Bytes.toString(bytes)); + } + return new ReplicationPeerConfig().setClusterKey(""); + } + } + + public static ReplicationPeerConfig convert(ZooKeeperProtos.ReplicationPeer peer) { + ReplicationPeerConfig peerConfig = new ReplicationPeerConfig(); + if (peer.hasClusterkey()) { + peerConfig.setClusterKey(peer.getClusterkey()); + } + if (peer.hasReplicationEndpointImpl()) { + peerConfig.setReplicationEndpointImpl(peer.getReplicationEndpointImpl()); + } + + for (HBaseProtos.BytesBytesPair pair : peer.getDataList()) { + peerConfig.getPeerData().put(pair.getFirst().toByteArray(), pair.getSecond().toByteArray()); + } + + for (HBaseProtos.NameStringPair pair : peer.getConfigurationList()) { + peerConfig.getConfiguration().put(pair.getName(), pair.getValue()); + } + if (peer.getTableCFs() != null) { + peerConfig.setTableCFs(peer.getTableCFs()); + } + return peerConfig; + } + + public static ZooKeeperProtos.ReplicationPeer convert(ReplicationPeerConfig peerConfig) { + ZooKeeperProtos.ReplicationPeer.Builder builder = ZooKeeperProtos.ReplicationPeer.newBuilder(); + if (peerConfig.getClusterKey() != null) { + builder.setClusterkey(peerConfig.getClusterKey()); + } + if (peerConfig.getReplicationEndpointImpl() != null) { + builder.setReplicationEndpointImpl(peerConfig.getReplicationEndpointImpl()); + } + + for (Map.Entry entry : peerConfig.getPeerData().entrySet()) { + builder.addData(HBaseProtos.BytesBytesPair.newBuilder() + .setFirst(ByteString.copyFrom(entry.getKey())) + .setSecond(ByteString.copyFrom(entry.getValue())) + .build()); + } + + for (Map.Entry entry : peerConfig.getConfiguration().entrySet()) { + builder.addConfiguration(HBaseProtos.NameStringPair.newBuilder() + .setName(entry.getKey()) + .setValue(entry.getValue()) + .build()); + } + + builder.setTableCFs(peerConfig.getTableCFs()); + return builder.build(); + } + + /** + * @param peerConfig + * @return Serialized protobuf of peerConfig with pb magic prefix prepended suitable + * for use as content of a this.peersZNode; i.e. the content of PEER_ID znode under + * /hbase/replication/peers/PEER_ID + */ + public static byte[] toByteArray(final ReplicationPeerConfig peerConfig) { + byte[] bytes = convert(peerConfig).toByteArray(); + return ProtobufUtil.prependPBMagic(bytes); + } + +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java index b8b5b22..95fe978 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java @@ -17,13 +17,12 @@ */ package org.apache.hadoop.hbase.replication; -import java.util.List; -import java.util.Map; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.classification.InterfaceAudience; +import java.util.List; +import java.util.Map; /** * ReplicationPeer manages enabled / disabled state for the peer. diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java index 043b38f..f79c626 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java @@ -24,6 +24,7 @@ import java.util.TreeMap; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.util.Bytes; /** @@ -37,11 +38,13 @@ public class ReplicationPeerConfig { private String replicationEndpointImpl; private final Map peerData; private final Map configuration; + private ZooKeeperProtos.TableCFs tableCFs; public ReplicationPeerConfig() { this.peerData = new TreeMap(Bytes.BYTES_COMPARATOR); this.configuration = new HashMap(0); + this.tableCFs = ZooKeeperProtos.TableCFs.newBuilder().build(); } /** @@ -78,6 +81,13 @@ public class ReplicationPeerConfig { return configuration; } + public ZooKeeperProtos.TableCFs getTableCFs() { + return tableCFs; + } + + public void setTableCFs(ZooKeeperProtos.TableCFs tableCFs) { + this.tableCFs = tableCFs; + } @Override public String toString() { StringBuilder builder = new StringBuilder("clusterKey=").append(clusterKey).append(","); 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 3ac8007..ecea380 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 @@ -30,11 +30,10 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; +import org.apache.hadoop.hbase.client.replication.TableCFsHelper; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; -import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; @@ -42,17 +41,18 @@ import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.NodeExistsException; @InterfaceAudience.Private -public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closeable { +public class ReplicationPeerZKImpl extends ReplicationStateZKBase + implements ReplicationPeer, Abortable, Closeable { private static final Log LOG = LogFactory.getLog(ReplicationPeerZKImpl.class); - private final ReplicationPeerConfig peerConfig; + private ReplicationPeerConfig peerConfig; private final String id; private volatile PeerState peerState; private volatile Map> tableCFs = new HashMap>(); private final Configuration conf; - private PeerStateTracker peerStateTracker; - private TableCFsTracker tableCFsTracker; + private PeerConfigTracker peerConfigTracker; + /** * Constructor that takes all the objects required to communicate with the @@ -61,8 +61,11 @@ public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closea * @param id string representation of this peer's identifier * @param peerConfig configuration for the replication peer */ - public ReplicationPeerZKImpl(Configuration conf, String id, ReplicationPeerConfig peerConfig) + public ReplicationPeerZKImpl(ZooKeeperWatcher zkWatcher, Configuration conf, + String id, ReplicationPeerConfig peerConfig, + Abortable abortable) throws ReplicationException { + super(zkWatcher, conf, abortable); this.conf = conf; this.peerConfig = peerConfig; this.id = id; @@ -71,13 +74,12 @@ public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closea /** * start a state tracker to check whether this peer is enabled or not * - * @param zookeeper zk watcher for the local cluster * @param peerStateNode path to zk node which stores peer state * @throws KeeperException */ - public void startStateTracker(ZooKeeperWatcher zookeeper, String peerStateNode) + public void startStateTracker(String peerStateNode) throws KeeperException { - ensurePeerEnabled(zookeeper, peerStateNode); + ensurePeerEnabled(peerStateNode); this.peerStateTracker = new PeerStateTracker(peerStateNode, zookeeper, this); this.peerStateTracker.start(); try { @@ -96,22 +98,26 @@ public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closea /** * start a table-cfs tracker to listen the (table, cf-list) map change - * - * @param zookeeper zk watcher for the local cluster - * @param tableCFsNode path to zk node which stores table-cfs + * @param peerConfigNode path to zk node which stores table-cfs * @throws KeeperException */ - public void startTableCFsTracker(ZooKeeperWatcher zookeeper, String tableCFsNode) + public void startPeerConfigTracker(String peerConfigNode) throws KeeperException { - this.tableCFsTracker = new TableCFsTracker(tableCFsNode, zookeeper, + this.peerConfigTracker = new PeerConfigTracker(peerConfigNode, zookeeper, this); - this.tableCFsTracker.start(); - this.readTableCFsZnode(); + this.peerConfigTracker.start(); + this.readPeerConfig(); } - private void readTableCFsZnode() { - String currentTableCFs = Bytes.toString(tableCFsTracker.getData(false)); - this.tableCFs = ReplicationAdmin.parseTableCFsFromConfig(currentTableCFs); + private void readPeerConfig() { + try { + byte[] data = peerConfigTracker.getData(false); + if (data != null) { + this.peerConfig = TableCFsHelper.parsePeerFrom(data); + } + } catch (DeserializationException e) { + LOG.error("", e); + } } @Override @@ -152,6 +158,7 @@ public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closea */ @Override public Map> getTableCFs() { + this.tableCFs = TableCFsHelper.convert2Map(peerConfig.getTableCFs()); return this.tableCFs; } @@ -207,13 +214,12 @@ public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closea /** * Utility method to ensure an ENABLED znode is in place; if not present, we create it. - * @param zookeeper * @param path Path to znode to check * @return True if we created the znode. * @throws NodeExistsException * @throws KeeperException */ - private static boolean ensurePeerEnabled(final ZooKeeperWatcher zookeeper, final String path) + private boolean ensurePeerEnabled(final String path) throws NodeExistsException, KeeperException { if (ZKUtil.checkExists(zookeeper, path) == -1) { // There is a race b/w PeerWatcher and ReplicationZookeeper#add method to create the @@ -252,18 +258,18 @@ public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closea /** * Tracker for (table, cf-list) map of this peer */ - public class TableCFsTracker extends ZooKeeperNodeTracker { + public class PeerConfigTracker extends ZooKeeperNodeTracker { - public TableCFsTracker(String tableCFsZNode, ZooKeeperWatcher watcher, + public PeerConfigTracker(String peerConfigNode, ZooKeeperWatcher watcher, Abortable abortable) { - super(watcher, tableCFsZNode, abortable); + super(watcher, peerConfigNode, abortable); } @Override public synchronized void nodeCreated(String path) { if (path.equals(node)) { super.nodeCreated(path); - readTableCFsZnode(); + readPeerConfig(); } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java index 8e80e06..9891906 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java @@ -18,6 +18,7 @@ */ package org.apache.hadoop.hbase.replication; +import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Set; @@ -49,10 +50,8 @@ public interface ReplicationPeers { * Add a new remote slave cluster for replication. * @param peerId a short that identifies the cluster * @param peerConfig configuration for the replication slave cluster - * @param tableCFs the table and column-family list which will be replicated for this peer or null - * for all table and column families */ - void addPeer(String peerId, ReplicationPeerConfig peerConfig, String tableCFs) + void addPeer(String peerId, ReplicationPeerConfig peerConfig) throws ReplicationException; /** @@ -63,6 +62,7 @@ public interface ReplicationPeers { boolean peerAdded(String peerId) throws ReplicationException; + @Deprecated void peerRemoved(String peerId); /** @@ -81,21 +81,17 @@ public interface ReplicationPeers { * Get the table and column-family list string of the peer from ZK. * @param peerId a short that identifies the cluster */ - public String getPeerTableCFsConfig(String peerId) throws ReplicationException; + public Map> getPeerTableCFsConfig(String peerId) + throws ReplicationException; /** * Set the table and column-family list string of the peer to ZK. * @param peerId a short that identifies the cluster * @param tableCFs the table and column-family list which will be replicated for this peer */ - public void setPeerTableCFsConfig(String peerId, String tableCFs) throws ReplicationException; - - /** - * Get the table and column-family-list map of the peer. - * @param peerId a short that identifies the cluster - * @return the table and column-family list which will be replicated for this peer - */ - public Map> getTableCFs(String peerId); + public void setPeerTableCFsConfig(String peerId, + Map> tableCFs) + throws ReplicationException; /** * Returns the ReplicationPeer 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 1884469..eab8ec7 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 @@ -19,36 +19,27 @@ package org.apache.hadoop.hbase.replication; import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.TreeMap; +import java.util.*; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.CompoundConfiguration; -import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.replication.TableCFsHelper; import org.apache.hadoop.hbase.exceptions.DeserializationException; -import org.apache.hadoop.hbase.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair; -import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState; -import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.zookeeper.KeeperException; -import com.google.protobuf.ByteString; - /** * This class provides an implementation of the ReplicationPeers interface using Zookeeper. The * peers znode contains a list of all peer replication clusters and the current replication state of @@ -80,15 +71,15 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re // Map of peer clusters keyed by their id private Map peerClusters; - private final String tableCFsNodeName; private final ReplicationQueuesClient queuesClient; + private Abortable abortable; private static final Log LOG = LogFactory.getLog(ReplicationPeersZKImpl.class); public ReplicationPeersZKImpl(final ZooKeeperWatcher zk, final Configuration conf, final ReplicationQueuesClient queuesClient, Abortable abortable) { super(zk, conf, abortable); - this.tableCFsNodeName = conf.get("zookeeper.znode.replication.peers.tableCFs", "tableCFs"); + this.abortable = abortable; this.peerClusters = new ConcurrentHashMap(); this.queuesClient = queuesClient; } @@ -106,7 +97,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re } @Override - public void addPeer(String id, ReplicationPeerConfig peerConfig, String tableCFs) + public void addPeer(String id, ReplicationPeerConfig peerConfig) throws ReplicationException { try { if (peerExists(id)) { @@ -119,21 +110,17 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re } checkQueuesDeleted(id); - ZKUtil.createWithParents(this.zookeeper, this.peersZNode); List listOfOps = new ArrayList(); - ZKUtilOp op1 = ZKUtilOp.createAndFailSilent(ZKUtil.joinZNode(this.peersZNode, id), - toByteArray(peerConfig)); + ZKUtilOp op1 = ZKUtilOp.createAndFailSilent(getPeerNode(id), + TableCFsHelper.toByteArray(peerConfig)); // There is a race (if hbase.zookeeper.useMulti is false) // b/w PeerWatcher and ReplicationZookeeper#add method to create the // peer-state znode. This happens while adding a peer // The peer state data is set as "ENABLED" by default. ZKUtilOp op2 = ZKUtilOp.createAndFailSilent(getPeerStateNode(id), ENABLED_ZNODE_BYTES); - String tableCFsStr = (tableCFs == null) ? "" : tableCFs; - ZKUtilOp op3 = ZKUtilOp.createAndFailSilent(getTableCFsNode(id), Bytes.toBytes(tableCFsStr)); listOfOps.add(op1); listOfOps.add(op2); - listOfOps.add(op3); ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false); // A peer is enabled by default } catch (KeeperException e) { @@ -149,7 +136,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re throw new IllegalArgumentException("Cannot remove peer with id=" + id + " because that id does not exist."); } - ZKUtil.deleteNodeRecursively(this.zookeeper, ZKUtil.joinZNode(this.peersZNode, id)); + ZKUtil.deleteNodeRecursively(this.zookeeper, getPeerNode(id)); } catch (KeeperException e) { throw new ReplicationException("Could not remove peer with id=" + id, e); } @@ -168,13 +155,17 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re } @Override - public String getPeerTableCFsConfig(String id) throws ReplicationException { + public Map> getPeerTableCFsConfig(String id) throws ReplicationException { try { if (!peerExists(id)) { throw new IllegalArgumentException("peer " + id + " doesn't exist"); } try { - return Bytes.toString(ZKUtil.getData(this.zookeeper, getTableCFsNode(id))); + ReplicationPeerConfig rpc = getReplicationPeerConfig(id); + if (rpc == null) { + throw new ReplicationException("Unable to get tableCFs of the peer with id=" + id); + } + return TableCFsHelper.convert2Map(rpc.getTableCFs()); } catch (Exception e) { throw new ReplicationException(e); } @@ -184,35 +175,28 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re } @Override - public void setPeerTableCFsConfig(String id, String tableCFsStr) throws ReplicationException { + public void setPeerTableCFsConfig(String id, + Map> tableCFs) + throws ReplicationException { try { if (!peerExists(id)) { throw new IllegalArgumentException("Cannot set peer tableCFs because id=" + id + " does not exist."); } - String tableCFsZKNode = getTableCFsNode(id); - byte[] tableCFs = Bytes.toBytes(tableCFsStr); - if (ZKUtil.checkExists(this.zookeeper, tableCFsZKNode) != -1) { - ZKUtil.setData(this.zookeeper, tableCFsZKNode, tableCFs); - } else { - ZKUtil.createAndWatch(this.zookeeper, tableCFsZKNode, tableCFs); + ReplicationPeerConfig rpc = getReplicationPeerConfig(id); + if (rpc == null) { + throw new ReplicationException("Unable to get tableCFs of the peer with id=" + id); } - LOG.info("Peer tableCFs with id= " + id + " is now " + tableCFsStr); + rpc.setTableCFs(TableCFsHelper.convert(tableCFs)); + ZKUtil.setData(this.zookeeper, getPeerNode(id), + TableCFsHelper.toByteArray(rpc)); + LOG.info("Peer tableCFs with id= " + id + " is now " + TableCFsHelper.convert(tableCFs)); } catch (KeeperException e) { throw new ReplicationException("Unable to change tableCFs of the peer with id=" + id, e); } } @Override - public Map> getTableCFs(String id) throws IllegalArgumentException { - ReplicationPeer replicationPeer = this.peerClusters.get(id); - if (replicationPeer == null) { - throw new IllegalArgumentException("Peer with id= " + id + " is not connected"); - } - return replicationPeer.getTableCFs(); - } - - @Override public boolean getStatusOfPeer(String id) { ReplicationPeer replicationPeer = this.peerClusters.get(id); if (replicationPeer == null) { @@ -282,7 +266,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re @Override public ReplicationPeerConfig getReplicationPeerConfig(String peerId) throws ReplicationException { - String znode = ZKUtil.joinZNode(this.peersZNode, peerId); + String znode = getPeerNode(peerId); byte[] data = null; try { data = ZKUtil.getData(this.zookeeper, znode); @@ -301,7 +285,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re } try { - return parsePeerFrom(data); + return TableCFsHelper.parsePeerFrom(data); } catch (DeserializationException e) { LOG.warn("Failed to parse cluster key from peerId=" + peerId + ", specifically the content from the following znode: " + znode); @@ -416,14 +400,6 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re return true; } - private String getTableCFsNode(String id) { - return ZKUtil.joinZNode(this.peersZNode, ZKUtil.joinZNode(id, this.tableCFsNodeName)); - } - - private String getPeerStateNode(String id) { - return ZKUtil.joinZNode(this.peersZNode, ZKUtil.joinZNode(id, this.peerStateNodeName)); - } - /** * Update the state znode of a peer cluster. * @param id @@ -464,16 +440,17 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re } Configuration peerConf = pair.getSecond(); - ReplicationPeerZKImpl peer = new ReplicationPeerZKImpl(peerConf, peerId, pair.getFirst()); + ReplicationPeerZKImpl peer = new ReplicationPeerZKImpl(zookeeper, + peerConf, peerId, pair.getFirst(), abortable); try { - peer.startStateTracker(this.zookeeper, this.getPeerStateNode(peerId)); + peer.startStateTracker(this.getPeerStateNode(peerId)); } catch (KeeperException e) { throw new ReplicationException("Error starting the peer state tracker for peerId=" + peerId, e); } try { - peer.startTableCFsTracker(this.zookeeper, this.getTableCFsNode(peerId)); + peer.startPeerConfigTracker(this.getPeerNode(peerId)); } catch (KeeperException e) { throw new ReplicationException("Error starting the peer tableCFs tracker for peerId=" + peerId, e); @@ -482,89 +459,6 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re return peer; } - /** - * @param bytes Content of a peer znode. - * @return ClusterKey parsed from the passed bytes. - * @throws DeserializationException - */ - private static ReplicationPeerConfig parsePeerFrom(final byte[] bytes) - throws DeserializationException { - if (ProtobufUtil.isPBMagicPrefix(bytes)) { - int pblen = ProtobufUtil.lengthOfPBMagic(); - ZooKeeperProtos.ReplicationPeer.Builder builder = - ZooKeeperProtos.ReplicationPeer.newBuilder(); - ZooKeeperProtos.ReplicationPeer peer; - try { - ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen); - peer = builder.build(); - } catch (IOException e) { - throw new DeserializationException(e); - } - return convert(peer); - } else { - if (bytes.length > 0) { - return new ReplicationPeerConfig().setClusterKey(Bytes.toString(bytes)); - } - return new ReplicationPeerConfig().setClusterKey(""); - } - } - - private static ReplicationPeerConfig convert(ZooKeeperProtos.ReplicationPeer peer) { - ReplicationPeerConfig peerConfig = new ReplicationPeerConfig(); - if (peer.hasClusterkey()) { - peerConfig.setClusterKey(peer.getClusterkey()); - } - if (peer.hasReplicationEndpointImpl()) { - peerConfig.setReplicationEndpointImpl(peer.getReplicationEndpointImpl()); - } - - for (BytesBytesPair pair : peer.getDataList()) { - peerConfig.getPeerData().put(pair.getFirst().toByteArray(), pair.getSecond().toByteArray()); - } - - for (NameStringPair pair : peer.getConfigurationList()) { - peerConfig.getConfiguration().put(pair.getName(), pair.getValue()); - } - return peerConfig; - } - - private static ZooKeeperProtos.ReplicationPeer convert(ReplicationPeerConfig peerConfig) { - ZooKeeperProtos.ReplicationPeer.Builder builder = ZooKeeperProtos.ReplicationPeer.newBuilder(); - if (peerConfig.getClusterKey() != null) { - builder.setClusterkey(peerConfig.getClusterKey()); - } - if (peerConfig.getReplicationEndpointImpl() != null) { - builder.setReplicationEndpointImpl(peerConfig.getReplicationEndpointImpl()); - } - - for (Map.Entry entry : peerConfig.getPeerData().entrySet()) { - builder.addData(BytesBytesPair.newBuilder() - .setFirst(ByteString.copyFrom(entry.getKey())) - .setSecond(ByteString.copyFrom(entry.getValue())) - .build()); - } - - for (Map.Entry entry : peerConfig.getConfiguration().entrySet()) { - builder.addConfiguration(NameStringPair.newBuilder() - .setName(entry.getKey()) - .setValue(entry.getValue()) - .build()); - } - - return builder.build(); - } - - /** - * @param peerConfig - * @return Serialized protobuf of peerConfig with pb magic prefix prepended suitable - * for use as content of a this.peersZNode; i.e. the content of PEER_ID znode under - * /hbase/replication/peers/PEER_ID - */ - private static byte[] toByteArray(final ReplicationPeerConfig peerConfig) { - byte[] bytes = convert(peerConfig).toByteArray(); - return ProtobufUtil.prependPBMagic(bytes); - } - private void checkQueuesDeleted(String peerId) throws ReplicationException { if (queuesClient == null) return; try { 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 1691b3f..aef123e 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 @@ -47,6 +47,8 @@ public abstract class ReplicationStateZKBase { protected final String peersZNode; /** The name of the znode that contains all replication queues */ protected final String queuesZNode; + /** The name of the znode that contains tableCFs */ + protected final String tableCFsNodeName; /** The cluster key of the local cluster */ protected final String ourClusterKey; protected final ZooKeeperWatcher zookeeper; @@ -69,6 +71,7 @@ public abstract class ReplicationStateZKBase { String peersZNodeName = conf.get("zookeeper.znode.replication.peers", "peers"); String queuesZNodeName = conf.get("zookeeper.znode.replication.rs", "rs"); this.peerStateNodeName = conf.get("zookeeper.znode.replication.peers.state", "peer-state"); + this.tableCFsNodeName = conf.get("zookeeper.znode.replication.peers.tableCFs", "tableCFs"); this.ourClusterKey = ZKUtil.getZooKeeperClusterKey(this.conf); this.replicationZNode = ZKUtil.joinZNode(this.zookeeper.baseZNode, replicationZNodeName); this.peersZNode = ZKUtil.joinZNode(replicationZNode, peersZNodeName); @@ -109,4 +112,17 @@ public abstract class ReplicationStateZKBase { protected boolean isPeerPath(String path) { return path.split("/").length == peersZNode.split("/").length + 1; } + + + protected String getTableCFsNode(String id) { + return ZKUtil.joinZNode(this.peersZNode, ZKUtil.joinZNode(id, this.tableCFsNodeName)); + } + + protected String getPeerStateNode(String id) { + return ZKUtil.joinZNode(this.peersZNode, ZKUtil.joinZNode(id, this.peerStateNodeName)); + } + + protected String getPeerNode(String id) { + return ZKUtil.joinZNode(this.peersZNode, id); + } } diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java index 4371739..901bf77 100644 --- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java +++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ZooKeeperProtos.java @@ -3947,6 +3947,1440 @@ public final class ZooKeeperProtos { // @@protoc_insertion_point(class_scope:hbase.pb.DeprecatedTableState) } + public interface TableCFOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // optional .hbase.pb.TableName table_name = 1; + /** + * optional .hbase.pb.TableName table_name = 1; + */ + boolean hasTableName(); + /** + * optional .hbase.pb.TableName table_name = 1; + */ + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName(); + /** + * optional .hbase.pb.TableName table_name = 1; + */ + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder(); + + // repeated bytes families = 2; + /** + * repeated bytes families = 2; + */ + java.util.List getFamiliesList(); + /** + * repeated bytes families = 2; + */ + int getFamiliesCount(); + /** + * repeated bytes families = 2; + */ + com.google.protobuf.ByteString getFamilies(int index); + } + /** + * Protobuf type {@code hbase.pb.TableCF} + */ + public static final class TableCF extends + com.google.protobuf.GeneratedMessage + implements TableCFOrBuilder { + // Use TableCF.newBuilder() to construct. + private TableCF(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private TableCF(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final TableCF defaultInstance; + public static TableCF getDefaultInstance() { + return defaultInstance; + } + + public TableCF getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private TableCF( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + 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.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + subBuilder = tableName_.toBuilder(); + } + tableName_ = input.readMessage(org.apache.hadoop.hbase.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(); + mutable_bitField0_ |= 0x00000002; + } + families_.add(input.readBytes()); + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) { + families_ = java.util.Collections.unmodifiableList(families_); + } + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCF_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCF_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.class, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public TableCF parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new TableCF(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + 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_; + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public boolean hasTableName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() { + return tableName_; + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() { + return tableName_; + } + + // repeated bytes families = 2; + public static final int FAMILIES_FIELD_NUMBER = 2; + private java.util.List families_; + /** + * repeated bytes families = 2; + */ + public java.util.List + getFamiliesList() { + return families_; + } + /** + * repeated bytes families = 2; + */ + public int getFamiliesCount() { + return families_.size(); + } + /** + * repeated bytes families = 2; + */ + public com.google.protobuf.ByteString getFamilies(int index) { + return families_.get(index); + } + + private void initFields() { + tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance(); + families_ = java.util.Collections.emptyList(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (hasTableName()) { + if (!getTableName().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, tableName_); + } + for (int i = 0; i < families_.size(); i++) { + output.writeBytes(2, families_.get(i)); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, tableName_); + } + { + int dataSize = 0; + for (int i = 0; i < families_.size(); i++) { + dataSize += com.google.protobuf.CodedOutputStream + .computeBytesSizeNoTag(families_.get(i)); + } + size += dataSize; + size += 1 * getFamiliesList().size(); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF other = (org.apache.hadoop.hbase.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 && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + private int memoizedHashCode = 0; + @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) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code hbase.pb.TableCF} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCF_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCF_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.class, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getTableNameFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (tableNameBuilder_ == null) { + tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance(); + } else { + tableNameBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + families_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCF_descriptor; + } + + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF build() { + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF result = new org.apache.hadoop.hbase.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 mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF other) { + if (other == org.apache.hadoop.hbase.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.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (hasTableName()) { + if (!getTableName().isInitialized()) { + + return false; + } + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + 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 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_; + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public boolean hasTableName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() { + if (tableNameBuilder_ == null) { + return tableName_; + } else { + return tableNameBuilder_.getMessage(); + } + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public Builder setTableName(org.apache.hadoop.hbase.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; + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public Builder setTableName( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) { + if (tableNameBuilder_ == null) { + tableName_ = builderForValue.build(); + onChanged(); + } else { + tableNameBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public Builder mergeTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.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.HBaseProtos.TableName.newBuilder(tableName_).mergeFrom(value).buildPartial(); + } else { + tableName_ = value; + } + onChanged(); + } else { + tableNameBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public Builder clearTableName() { + if (tableNameBuilder_ == null) { + tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance(); + onChanged(); + } else { + tableNameBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder getTableNameBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getTableNameFieldBuilder().getBuilder(); + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() { + if (tableNameBuilder_ != null) { + return tableNameBuilder_.getMessageOrBuilder(); + } else { + return tableName_; + } + } + /** + * optional .hbase.pb.TableName table_name = 1; + */ + 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> + 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>( + tableName_, + getParentForChildren(), + isClean()); + tableName_ = null; + } + return tableNameBuilder_; + } + + // repeated bytes families = 2; + private java.util.List families_ = java.util.Collections.emptyList(); + private void ensureFamiliesIsMutable() { + if (!((bitField0_ & 0x00000002) == 0x00000002)) { + families_ = new java.util.ArrayList(families_); + bitField0_ |= 0x00000002; + } + } + /** + * repeated bytes families = 2; + */ + public java.util.List + getFamiliesList() { + return java.util.Collections.unmodifiableList(families_); + } + /** + * repeated bytes families = 2; + */ + public int getFamiliesCount() { + return families_.size(); + } + /** + * repeated bytes families = 2; + */ + public com.google.protobuf.ByteString getFamilies(int index) { + return families_.get(index); + } + /** + * repeated bytes families = 2; + */ + public Builder setFamilies( + int index, com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + ensureFamiliesIsMutable(); + families_.set(index, value); + onChanged(); + return this; + } + /** + * repeated bytes families = 2; + */ + public Builder addFamilies(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + ensureFamiliesIsMutable(); + families_.add(value); + onChanged(); + return this; + } + /** + * repeated bytes families = 2; + */ + public Builder addAllFamilies( + java.lang.Iterable values) { + ensureFamiliesIsMutable(); + super.addAll(values, families_); + onChanged(); + return this; + } + /** + * repeated bytes families = 2; + */ + public Builder clearFamilies() { + families_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000002); + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:hbase.pb.TableCF) + } + + static { + defaultInstance = new TableCF(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:hbase.pb.TableCF) + } + + public interface TableCFsOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // repeated .hbase.pb.TableCF table_cfs = 1; + /** + * repeated .hbase.pb.TableCF table_cfs = 1; + */ + java.util.List + getTableCfsList(); + /** + * repeated .hbase.pb.TableCF table_cfs = 1; + */ + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF getTableCfs(int index); + /** + * repeated .hbase.pb.TableCF table_cfs = 1; + */ + int getTableCfsCount(); + /** + * repeated .hbase.pb.TableCF table_cfs = 1; + */ + java.util.List + getTableCfsOrBuilderList(); + /** + * repeated .hbase.pb.TableCF table_cfs = 1; + */ + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder getTableCfsOrBuilder( + int index); + } + /** + * Protobuf type {@code hbase.pb.TableCFs} + */ + public static final class TableCFs extends + com.google.protobuf.GeneratedMessage + implements TableCFsOrBuilder { + // Use TableCFs.newBuilder() to construct. + private TableCFs(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private TableCFs(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final TableCFs defaultInstance; + public static TableCFs getDefaultInstance() { + return defaultInstance; + } + + public TableCFs getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private TableCFs( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + 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: { + if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) { + tableCfs_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000001; + } + tableCfs_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.PARSER, extensionRegistry)); + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) { + tableCfs_ = java.util.Collections.unmodifiableList(tableCfs_); + } + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCFs_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCFs_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs.class, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public TableCFs parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new TableCFs(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + // repeated .hbase.pb.TableCF table_cfs = 1; + public static final int TABLE_CFS_FIELD_NUMBER = 1; + private java.util.List tableCfs_; + /** + * repeated .hbase.pb.TableCF table_cfs = 1; + */ + public java.util.List getTableCfsList() { + return tableCfs_; + } + /** + * repeated .hbase.pb.TableCF table_cfs = 1; + */ + public java.util.List + getTableCfsOrBuilderList() { + return tableCfs_; + } + /** + * repeated .hbase.pb.TableCF table_cfs = 1; + */ + public int getTableCfsCount() { + return tableCfs_.size(); + } + /** + * repeated .hbase.pb.TableCF table_cfs = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF getTableCfs(int index) { + return tableCfs_.get(index); + } + /** + * repeated .hbase.pb.TableCF table_cfs = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder getTableCfsOrBuilder( + int index) { + return tableCfs_.get(index); + } + + private void initFields() { + tableCfs_ = java.util.Collections.emptyList(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + for (int i = 0; i < getTableCfsCount(); i++) { + if (!getTableCfs(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + for (int i = 0; i < tableCfs_.size(); i++) { + output.writeMessage(1, tableCfs_.get(i)); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + for (int i = 0; i < tableCfs_.size(); i++) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, tableCfs_.get(i)); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs other = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs) obj; + + boolean result = true; + result = result && getTableCfsList() + .equals(other.getTableCfsList()); + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + private int memoizedHashCode = 0; + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (getTableCfsCount() > 0) { + hash = (37 * hash) + TABLE_CFS_FIELD_NUMBER; + hash = (53 * hash) + getTableCfsList().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code hbase.pb.TableCFs} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFsOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCFs_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCFs_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs.class, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getTableCfsFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (tableCfsBuilder_ == null) { + tableCfs_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + } else { + tableCfsBuilder_.clear(); + } + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.internal_static_hbase_pb_TableCFs_descriptor; + } + + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs build() { + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs result = new org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs(this); + int from_bitField0_ = bitField0_; + if (tableCfsBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + tableCfs_ = java.util.Collections.unmodifiableList(tableCfs_); + bitField0_ = (bitField0_ & ~0x00000001); + } + result.tableCfs_ = tableCfs_; + } else { + result.tableCfs_ = tableCfsBuilder_.build(); + } + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs.getDefaultInstance()) return this; + if (tableCfsBuilder_ == null) { + if (!other.tableCfs_.isEmpty()) { + if (tableCfs_.isEmpty()) { + tableCfs_ = other.tableCfs_; + bitField0_ = (bitField0_ & ~0x00000001); + } else { + ensureTableCfsIsMutable(); + tableCfs_.addAll(other.tableCfs_); + } + onChanged(); + } + } else { + if (!other.tableCfs_.isEmpty()) { + if (tableCfsBuilder_.isEmpty()) { + tableCfsBuilder_.dispose(); + tableCfsBuilder_ = null; + tableCfs_ = other.tableCfs_; + bitField0_ = (bitField0_ & ~0x00000001); + tableCfsBuilder_ = + com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? + getTableCfsFieldBuilder() : null; + } else { + tableCfsBuilder_.addAllMessages(other.tableCfs_); + } + } + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + for (int i = 0; i < getTableCfsCount(); i++) { + if (!getTableCfs(i).isInitialized()) { + + return false; + } + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // repeated .hbase.pb.TableCF table_cfs = 1; + private java.util.List tableCfs_ = + java.util.Collections.emptyList(); + private void ensureTableCfsIsMutable() { + if (!((bitField0_ & 0x00000001) == 0x00000001)) { + tableCfs_ = new java.util.ArrayList(tableCfs_); + bitField0_ |= 0x00000001; + } + } + + private com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder> tableCfsBuilder_; + + /** + * repeated .hbase.pb.TableCF table_cfs = 1; + */ + public java.util.List getTableCfsList() { + if (tableCfsBuilder_ == null) { + return java.util.Collections.unmodifiableList(tableCfs_); + } else { + return tableCfsBuilder_.getMessageList(); + } + } + /** + * repeated .hbase.pb.TableCF table_cfs = 1; + */ + public int getTableCfsCount() { + if (tableCfsBuilder_ == null) { + return tableCfs_.size(); + } else { + return tableCfsBuilder_.getCount(); + } + } + /** + * repeated .hbase.pb.TableCF table_cfs = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF getTableCfs(int index) { + if (tableCfsBuilder_ == null) { + return tableCfs_.get(index); + } else { + return tableCfsBuilder_.getMessage(index); + } + } + /** + * repeated .hbase.pb.TableCF table_cfs = 1; + */ + public Builder setTableCfs( + int index, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF value) { + if (tableCfsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureTableCfsIsMutable(); + tableCfs_.set(index, value); + onChanged(); + } else { + tableCfsBuilder_.setMessage(index, value); + } + return this; + } + /** + * repeated .hbase.pb.TableCF table_cfs = 1; + */ + public Builder setTableCfs( + int index, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder builderForValue) { + if (tableCfsBuilder_ == null) { + ensureTableCfsIsMutable(); + tableCfs_.set(index, builderForValue.build()); + onChanged(); + } else { + tableCfsBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.TableCF table_cfs = 1; + */ + public Builder addTableCfs(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF value) { + if (tableCfsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureTableCfsIsMutable(); + tableCfs_.add(value); + onChanged(); + } else { + tableCfsBuilder_.addMessage(value); + } + return this; + } + /** + * repeated .hbase.pb.TableCF table_cfs = 1; + */ + public Builder addTableCfs( + int index, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF value) { + if (tableCfsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureTableCfsIsMutable(); + tableCfs_.add(index, value); + onChanged(); + } else { + tableCfsBuilder_.addMessage(index, value); + } + return this; + } + /** + * repeated .hbase.pb.TableCF table_cfs = 1; + */ + public Builder addTableCfs( + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder builderForValue) { + if (tableCfsBuilder_ == null) { + ensureTableCfsIsMutable(); + tableCfs_.add(builderForValue.build()); + onChanged(); + } else { + tableCfsBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.TableCF table_cfs = 1; + */ + public Builder addTableCfs( + int index, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder builderForValue) { + if (tableCfsBuilder_ == null) { + ensureTableCfsIsMutable(); + tableCfs_.add(index, builderForValue.build()); + onChanged(); + } else { + tableCfsBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .hbase.pb.TableCF table_cfs = 1; + */ + public Builder addAllTableCfs( + java.lang.Iterable values) { + if (tableCfsBuilder_ == null) { + ensureTableCfsIsMutable(); + super.addAll(values, tableCfs_); + onChanged(); + } else { + tableCfsBuilder_.addAllMessages(values); + } + return this; + } + /** + * repeated .hbase.pb.TableCF table_cfs = 1; + */ + public Builder clearTableCfs() { + if (tableCfsBuilder_ == null) { + tableCfs_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + } else { + tableCfsBuilder_.clear(); + } + return this; + } + /** + * repeated .hbase.pb.TableCF table_cfs = 1; + */ + public Builder removeTableCfs(int index) { + if (tableCfsBuilder_ == null) { + ensureTableCfsIsMutable(); + tableCfs_.remove(index); + onChanged(); + } else { + tableCfsBuilder_.remove(index); + } + return this; + } + /** + * repeated .hbase.pb.TableCF table_cfs = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder getTableCfsBuilder( + int index) { + return getTableCfsFieldBuilder().getBuilder(index); + } + /** + * repeated .hbase.pb.TableCF table_cfs = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder getTableCfsOrBuilder( + int index) { + if (tableCfsBuilder_ == null) { + return tableCfs_.get(index); } else { + return tableCfsBuilder_.getMessageOrBuilder(index); + } + } + /** + * repeated .hbase.pb.TableCF table_cfs = 1; + */ + public java.util.List + getTableCfsOrBuilderList() { + if (tableCfsBuilder_ != null) { + return tableCfsBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(tableCfs_); + } + } + /** + * repeated .hbase.pb.TableCF table_cfs = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder addTableCfsBuilder() { + return getTableCfsFieldBuilder().addBuilder( + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.getDefaultInstance()); + } + /** + * repeated .hbase.pb.TableCF table_cfs = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder addTableCfsBuilder( + int index) { + return getTableCfsFieldBuilder().addBuilder( + index, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.getDefaultInstance()); + } + /** + * repeated .hbase.pb.TableCF table_cfs = 1; + */ + public java.util.List + getTableCfsBuilderList() { + return getTableCfsFieldBuilder().getBuilderList(); + } + private com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder> + getTableCfsFieldBuilder() { + if (tableCfsBuilder_ == null) { + tableCfsBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCF.Builder, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFOrBuilder>( + tableCfs_, + ((bitField0_ & 0x00000001) == 0x00000001), + getParentForChildren(), + isClean()); + tableCfs_ = null; + } + return tableCfsBuilder_; + } + + // @@protoc_insertion_point(builder_scope:hbase.pb.TableCFs) + } + + static { + defaultInstance = new TableCFs(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:hbase.pb.TableCFs) + } + public interface ReplicationPeerOrBuilder extends com.google.protobuf.MessageOrBuilder { @@ -4044,6 +5478,20 @@ public final class ZooKeeperProtos { */ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPairOrBuilder getConfigurationOrBuilder( int index); + + // optional .hbase.pb.TableCFs tableCFs = 5; + /** + * optional .hbase.pb.TableCFs tableCFs = 5; + */ + boolean hasTableCFs(); + /** + * optional .hbase.pb.TableCFs tableCFs = 5; + */ + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs getTableCFs(); + /** + * optional .hbase.pb.TableCFs tableCFs = 5; + */ + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFsOrBuilder getTableCFsOrBuilder(); } /** * Protobuf type {@code hbase.pb.ReplicationPeer} @@ -4127,6 +5575,19 @@ public final class ZooKeeperProtos { configuration_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair.PARSER, extensionRegistry)); break; } + case 42: { + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs.Builder subBuilder = null; + if (((bitField0_ & 0x00000004) == 0x00000004)) { + subBuilder = tableCFs_.toBuilder(); + } + tableCFs_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(tableCFs_); + tableCFs_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000004; + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -4346,11 +5807,34 @@ public final class ZooKeeperProtos { return configuration_.get(index); } + // optional .hbase.pb.TableCFs tableCFs = 5; + public static final int TABLECFS_FIELD_NUMBER = 5; + private org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs tableCFs_; + /** + * optional .hbase.pb.TableCFs tableCFs = 5; + */ + public boolean hasTableCFs() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional .hbase.pb.TableCFs tableCFs = 5; + */ + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs getTableCFs() { + return tableCFs_; + } + /** + * optional .hbase.pb.TableCFs tableCFs = 5; + */ + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFsOrBuilder getTableCFsOrBuilder() { + return tableCFs_; + } + private void initFields() { clusterkey_ = ""; replicationEndpointImpl_ = ""; data_ = java.util.Collections.emptyList(); configuration_ = java.util.Collections.emptyList(); + tableCFs_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs.getDefaultInstance(); } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -4373,6 +5857,12 @@ public final class ZooKeeperProtos { return false; } } + if (hasTableCFs()) { + if (!getTableCFs().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } memoizedIsInitialized = 1; return true; } @@ -4392,6 +5882,9 @@ public final class ZooKeeperProtos { for (int i = 0; i < configuration_.size(); i++) { output.writeMessage(4, configuration_.get(i)); } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeMessage(5, tableCFs_); + } getUnknownFields().writeTo(output); } @@ -4417,6 +5910,10 @@ public final class ZooKeeperProtos { size += com.google.protobuf.CodedOutputStream .computeMessageSize(4, configuration_.get(i)); } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(5, tableCFs_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -4454,6 +5951,11 @@ public final class ZooKeeperProtos { .equals(other.getDataList()); result = result && getConfigurationList() .equals(other.getConfigurationList()); + result = result && (hasTableCFs() == other.hasTableCFs()); + if (hasTableCFs()) { + result = result && getTableCFs() + .equals(other.getTableCFs()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -4483,6 +5985,10 @@ public final class ZooKeeperProtos { hash = (37 * hash) + CONFIGURATION_FIELD_NUMBER; hash = (53 * hash) + getConfigurationList().hashCode(); } + if (hasTableCFs()) { + hash = (37 * hash) + TABLECFS_FIELD_NUMBER; + hash = (53 * hash) + getTableCFs().hashCode(); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -4591,6 +6097,7 @@ public final class ZooKeeperProtos { if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { getDataFieldBuilder(); getConfigurationFieldBuilder(); + getTableCFsFieldBuilder(); } } private static Builder create() { @@ -4615,6 +6122,12 @@ public final class ZooKeeperProtos { } else { configurationBuilder_.clear(); } + if (tableCFsBuilder_ == null) { + tableCFs_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs.getDefaultInstance(); + } else { + tableCFsBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000010); return this; } @@ -4669,6 +6182,14 @@ public final class ZooKeeperProtos { } else { result.configuration_ = configurationBuilder_.build(); } + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000004; + } + if (tableCFsBuilder_ == null) { + result.tableCFs_ = tableCFs_; + } else { + result.tableCFs_ = tableCFsBuilder_.build(); + } result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -4747,6 +6268,9 @@ public final class ZooKeeperProtos { } } } + if (other.hasTableCFs()) { + mergeTableCFs(other.getTableCFs()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -4768,6 +6292,12 @@ public final class ZooKeeperProtos { return false; } } + if (hasTableCFs()) { + if (!getTableCFs().isInitialized()) { + + return false; + } + } return true; } @@ -5448,6 +6978,123 @@ public final class ZooKeeperProtos { return configurationBuilder_; } + // optional .hbase.pb.TableCFs tableCFs = 5; + private org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs tableCFs_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs.Builder, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFsOrBuilder> tableCFsBuilder_; + /** + * optional .hbase.pb.TableCFs tableCFs = 5; + */ + public boolean hasTableCFs() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + /** + * optional .hbase.pb.TableCFs tableCFs = 5; + */ + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs getTableCFs() { + if (tableCFsBuilder_ == null) { + return tableCFs_; + } else { + return tableCFsBuilder_.getMessage(); + } + } + /** + * optional .hbase.pb.TableCFs tableCFs = 5; + */ + public Builder setTableCFs(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs value) { + if (tableCFsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + tableCFs_ = value; + onChanged(); + } else { + tableCFsBuilder_.setMessage(value); + } + bitField0_ |= 0x00000010; + return this; + } + /** + * optional .hbase.pb.TableCFs tableCFs = 5; + */ + public Builder setTableCFs( + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs.Builder builderForValue) { + if (tableCFsBuilder_ == null) { + tableCFs_ = builderForValue.build(); + onChanged(); + } else { + tableCFsBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000010; + return this; + } + /** + * optional .hbase.pb.TableCFs tableCFs = 5; + */ + public Builder mergeTableCFs(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs value) { + if (tableCFsBuilder_ == null) { + if (((bitField0_ & 0x00000010) == 0x00000010) && + tableCFs_ != org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs.getDefaultInstance()) { + tableCFs_ = + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs.newBuilder(tableCFs_).mergeFrom(value).buildPartial(); + } else { + tableCFs_ = value; + } + onChanged(); + } else { + tableCFsBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000010; + return this; + } + /** + * optional .hbase.pb.TableCFs tableCFs = 5; + */ + public Builder clearTableCFs() { + if (tableCFsBuilder_ == null) { + tableCFs_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs.getDefaultInstance(); + onChanged(); + } else { + tableCFsBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000010); + return this; + } + /** + * optional .hbase.pb.TableCFs tableCFs = 5; + */ + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs.Builder getTableCFsBuilder() { + bitField0_ |= 0x00000010; + onChanged(); + return getTableCFsFieldBuilder().getBuilder(); + } + /** + * optional .hbase.pb.TableCFs tableCFs = 5; + */ + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFsOrBuilder getTableCFsOrBuilder() { + if (tableCFsBuilder_ != null) { + return tableCFsBuilder_.getMessageOrBuilder(); + } else { + return tableCFs_; + } + } + /** + * optional .hbase.pb.TableCFs tableCFs = 5; + */ + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs.Builder, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFsOrBuilder> + getTableCFsFieldBuilder() { + if (tableCFsBuilder_ == null) { + tableCFsBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFs.Builder, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableCFsOrBuilder>( + tableCFs_, + getParentForChildren(), + isClean()); + tableCFs_ = null; + } + return tableCFsBuilder_; + } + // @@protoc_insertion_point(builder_scope:hbase.pb.ReplicationPeer) } @@ -8222,6 +9869,16 @@ public final class ZooKeeperProtos { com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_hbase_pb_DeprecatedTableState_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor + internal_static_hbase_pb_TableCF_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_hbase_pb_TableCF_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_hbase_pb_TableCFs_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_hbase_pb_TableCFs_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor internal_static_hbase_pb_ReplicationPeer_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable @@ -8273,22 +9930,26 @@ public final class ZooKeeperProtos { "Y\020\002\"\225\001\n\024DeprecatedTableState\022<\n\005state\030\001 " + "\002(\0162$.hbase.pb.DeprecatedTableState.Stat" + "e:\007ENABLED\"?\n\005State\022\013\n\007ENABLED\020\000\022\014\n\010DISA" + - "BLED\020\001\022\r\n\tDISABLING\020\002\022\014\n\010ENABLING\020\003\"\237\001\n\017" + - "ReplicationPeer\022\022\n\nclusterkey\030\001 \002(\t\022\037\n\027r", + "BLED\020\001\022\r\n\tDISABLING\020\002\022\014\n\010ENABLING\020\003\"D\n\007T" + + "ableCF\022\'\n\ntable_name\030\001 \001(\0132\023.hbase.pb.Ta", + "bleName\022\020\n\010families\030\002 \003(\014\"0\n\010TableCFs\022$\n" + + "\ttable_cfs\030\001 \003(\0132\021.hbase.pb.TableCF\"\305\001\n\017" + + "ReplicationPeer\022\022\n\nclusterkey\030\001 \002(\t\022\037\n\027r" + "eplicationEndpointImpl\030\002 \001(\t\022&\n\004data\030\003 \003" + "(\0132\030.hbase.pb.BytesBytesPair\022/\n\rconfigur" + - "ation\030\004 \003(\0132\030.hbase.pb.NameStringPair\"g\n" + - "\020ReplicationState\022/\n\005state\030\001 \002(\0162 .hbase" + - ".pb.ReplicationState.State\"\"\n\005State\022\013\n\007E" + - "NABLED\020\000\022\014\n\010DISABLED\020\001\"+\n\027ReplicationHLo" + - "gPosition\022\020\n\010position\030\001 \002(\003\"%\n\017Replicati" + - "onLock\022\022\n\nlock_owner\030\001 \002(\t\"\252\001\n\tTableLock" + - "\022\'\n\ntable_name\030\001 \001(\0132\023.hbase.pb.TableNam" + - "e\022(\n\nlock_owner\030\002 \001(\0132\024.hbase.pb.ServerN", - "ame\022\021\n\tthread_id\030\003 \001(\003\022\021\n\tis_shared\030\004 \001(" + - "\010\022\017\n\007purpose\030\005 \001(\t\022\023\n\013create_time\030\006 \001(\003B" + - "E\n*org.apache.hadoop.hbase.protobuf.gene" + - "ratedB\017ZooKeeperProtosH\001\210\001\001\240\001\001" + "ation\030\004 \003(\0132\030.hbase.pb.NameStringPair\022$\n" + + "\010tableCFs\030\005 \001(\0132\022.hbase.pb.TableCFs\"g\n\020R" + + "eplicationState\022/\n\005state\030\001 \002(\0162 .hbase.p" + + "b.ReplicationState.State\"\"\n\005State\022\013\n\007ENA" + + "BLED\020\000\022\014\n\010DISABLED\020\001\"+\n\027ReplicationHLogP", + "osition\022\020\n\010position\030\001 \002(\003\"%\n\017Replication" + + "Lock\022\022\n\nlock_owner\030\001 \002(\t\"\252\001\n\tTableLock\022\'" + + "\n\ntable_name\030\001 \001(\0132\023.hbase.pb.TableName\022" + + "(\n\nlock_owner\030\002 \001(\0132\024.hbase.pb.ServerNam" + + "e\022\021\n\tthread_id\030\003 \001(\003\022\021\n\tis_shared\030\004 \001(\010\022" + + "\017\n\007purpose\030\005 \001(\t\022\023\n\013create_time\030\006 \001(\003BE\n" + + "*org.apache.hadoop.hbase.protobuf.genera" + + "tedB\017ZooKeeperProtosH\001\210\001\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -8325,32 +9986,44 @@ public final class ZooKeeperProtos { com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_DeprecatedTableState_descriptor, new java.lang.String[] { "State", }); - internal_static_hbase_pb_ReplicationPeer_descriptor = + internal_static_hbase_pb_TableCF_descriptor = getDescriptor().getMessageTypes().get(5); + internal_static_hbase_pb_TableCF_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_hbase_pb_TableCF_descriptor, + new java.lang.String[] { "TableName", "Families", }); + internal_static_hbase_pb_TableCFs_descriptor = + getDescriptor().getMessageTypes().get(6); + internal_static_hbase_pb_TableCFs_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_hbase_pb_TableCFs_descriptor, + new java.lang.String[] { "TableCfs", }); + internal_static_hbase_pb_ReplicationPeer_descriptor = + getDescriptor().getMessageTypes().get(7); internal_static_hbase_pb_ReplicationPeer_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_ReplicationPeer_descriptor, - new java.lang.String[] { "Clusterkey", "ReplicationEndpointImpl", "Data", "Configuration", }); + new java.lang.String[] { "Clusterkey", "ReplicationEndpointImpl", "Data", "Configuration", "TableCFs", }); internal_static_hbase_pb_ReplicationState_descriptor = - getDescriptor().getMessageTypes().get(6); + getDescriptor().getMessageTypes().get(8); internal_static_hbase_pb_ReplicationState_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_ReplicationState_descriptor, new java.lang.String[] { "State", }); internal_static_hbase_pb_ReplicationHLogPosition_descriptor = - getDescriptor().getMessageTypes().get(7); + getDescriptor().getMessageTypes().get(9); internal_static_hbase_pb_ReplicationHLogPosition_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_ReplicationHLogPosition_descriptor, new java.lang.String[] { "Position", }); internal_static_hbase_pb_ReplicationLock_descriptor = - getDescriptor().getMessageTypes().get(8); + getDescriptor().getMessageTypes().get(10); internal_static_hbase_pb_ReplicationLock_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_ReplicationLock_descriptor, new java.lang.String[] { "LockOwner", }); internal_static_hbase_pb_TableLock_descriptor = - getDescriptor().getMessageTypes().get(9); + getDescriptor().getMessageTypes().get(11); internal_static_hbase_pb_TableLock_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_TableLock_descriptor, diff --git a/hbase-protocol/src/main/protobuf/ZooKeeper.proto b/hbase-protocol/src/main/protobuf/ZooKeeper.proto index 54652af..70c9bbf 100644 --- a/hbase-protocol/src/main/protobuf/ZooKeeper.proto +++ b/hbase-protocol/src/main/protobuf/ZooKeeper.proto @@ -105,6 +105,15 @@ message DeprecatedTableState { required State state = 1 [default = ENABLED]; } +message TableCF { + optional TableName table_name = 1; + repeated bytes families = 2; +} + +message TableCFs { + repeated TableCF table_cfs = 1; +} + /** * Used by replication. Holds a replication peer key. */ @@ -115,6 +124,7 @@ message ReplicationPeer { optional string replicationEndpointImpl = 2; repeated BytesBytesPair data = 3; repeated NameStringPair configuration = 4; + optional TableCFs tableCFs = 5; } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index b30de7c..7cc2432 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -132,6 +132,7 @@ import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost; import org.apache.hadoop.hbase.regionserver.RegionSplitPolicy; import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy; import org.apache.hadoop.hbase.regionserver.compactions.FIFOCompactionPolicy; +import org.apache.hadoop.hbase.replication.master.TableCFsUpdater; import org.apache.hadoop.hbase.replication.regionserver.Replication; import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.util.Addressing; @@ -667,6 +668,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server { // publish cluster ID status.setStatus("Publishing Cluster ID in ZooKeeper"); ZKClusterId.setClusterId(this.zooKeeper, fileSystemManager.getClusterId()); + this.serverManager = createServerManager(this, this); // Invalidate all write locks held previously @@ -676,6 +678,13 @@ public class HMaster extends HRegionServer implements MasterServices, Server { status.setStatus("Initializing ZK system trackers"); initializeZKBasedSystemTrackers(); + // This is for backwards compatibility + // See HBASE-11393 + status.setStatus("Update TableCFs node in ZNode"); + TableCFsUpdater tableCFsUpdater = new TableCFsUpdater(zooKeeper, + conf, this.clusterConnection); + tableCFsUpdater.update(); + // initialize master side coprocessors before we start handling requests status.setStatus("Initializing master coprocessors"); this.cpHost = new MasterCoprocessorHost(this, this.conf); @@ -843,7 +852,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server { zombieDetector.interrupt(); } - /** * Create a {@link ServerManager} instance. * @param master diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/TableCfWALEntryFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/TableCfWALEntryFilter.java index 642ee8a..ef8562e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/TableCfWALEntryFilter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/TableCfWALEntryFilter.java @@ -43,7 +43,6 @@ public class TableCfWALEntryFilter implements WALEntryFilter { TableName tabName = entry.getKey().getTablename(); ArrayList cells = entry.getEdit().getCells(); Map> tableCFs = null; - try { tableCFs = this.peer.getTableCFs(); } catch (IllegalArgumentException e) { @@ -51,7 +50,6 @@ public class TableCfWALEntryFilter implements WALEntryFilter { ", degenerate as if it's not configured by keeping tableCFs==null"); } int size = cells.size(); - // return null(prevent replicating) if logKey's table isn't in this peer's // replicable table list (empty tableCFs means all table are replicable) if (tableCFs != null && !tableCFs.containsKey(tabName)) { @@ -62,9 +60,9 @@ public class TableCfWALEntryFilter implements WALEntryFilter { Cell cell = cells.get(i); // ignore(remove) kv if its cf isn't in the replicable cf list // (empty cfs means all cfs of this table are replicable) - if ((cfs != null) && !cfs.contains(Bytes.toString( - cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()))) { - cells.remove(i); + if ((cfs != null && !cfs.contains(Bytes.toString(cell.getFamilyArray(), + cell.getFamilyOffset(), cell.getFamilyLength())))) { + cells.remove(i); } } } @@ -73,5 +71,4 @@ public class TableCfWALEntryFilter implements WALEntryFilter { } return entry; } - } 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 new file mode 100644 index 0000000..40c42cf --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/TableCFsUpdater.java @@ -0,0 +1,118 @@ +/* + * + * 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.master; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Abortable; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.client.replication.TableCFsHelper; +import org.apache.hadoop.hbase.exceptions.DeserializationException; +import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; +import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; +import org.apache.hadoop.hbase.replication.ReplicationStateZKBase; +import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; +import org.apache.zookeeper.KeeperException; + +import java.io.IOException; +import java.util.List; + +/** + * This class is used to upgrade TableCFs from HBase 1.x to HBase 2.x + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public class TableCFsUpdater extends ReplicationStateZKBase { + + private static final Log LOG = LogFactory.getLog(TableCFsUpdater.class); + + public TableCFsUpdater(ZooKeeperWatcher zookeeper, + Configuration conf, Abortable abortable) { + super(zookeeper, conf, abortable); + } + + public void update() { + List znodes = null; + try { + znodes = ZKUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode); + } catch (KeeperException e) { + LOG.warn("", e); + } + if (znodes != null) { + for (String peerId : znodes) { + if (!update(peerId)) { + LOG.error("upgrade tableCFs failed for peerId=" + peerId); + } + } + } + } + + public boolean update(String peerId) { + String tableCFsNode = getTableCFsNode(peerId); + try { + if (ZKUtil.checkExists(zookeeper, tableCFsNode) != -1) { + String peerNode = getPeerNode(peerId); + ReplicationPeerConfig rpc = getReplicationPeerConig(peerNode); + if (rpc.getTableCFs() == null) { + // we copy TableCFs node into PeerNode + LOG.info("copy tableCFs into peerNode:" + peerId); + ZooKeeperProtos.TableCFs tableCFs = + TableCFsHelper.parseTableCFs( + ZKUtil.getData(this.zookeeper, tableCFsNode)); + rpc.setTableCFs(tableCFs); + ZKUtil.setData(this.zookeeper, peerNode, + TableCFsHelper.toByteArray(rpc)); + } else { + LOG.info("TableCFs node has been copied into peerNode:" + peerId); + } + } + } catch (KeeperException e) { + LOG.warn("", e); + return false; + } catch (InterruptedException e) { + LOG.warn("", e); + return false; + } catch (IOException e) { + LOG.warn("", e); + return false; + } + return true; + } + + private ReplicationPeerConfig getReplicationPeerConig(String peerNode) + throws KeeperException, InterruptedException { + byte[] data = null; + data = ZKUtil.getData(this.zookeeper, peerNode); + if (data == null) { + LOG.error("Could not get configuration for " + + "peer because it doesn't exist. peer=" + peerNode); + return null; + } + try { + return TableCFsHelper.parsePeerFrom(data); + } catch (DeserializationException e) { + LOG.warn("Failed to parse cluster key from peer=" + peerNode); + return null; + } + } + +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java index b773b46..a79aa0a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java @@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequ import org.apache.hadoop.hbase.protobuf.RequestConverter; import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore; import org.apache.hadoop.hbase.regionserver.TestHRegionServerBulkLoad; +import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -249,7 +250,10 @@ public class TestReplicaWithCluster { HTU2.getHBaseAdmin().createTable(hdt, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE); ReplicationAdmin admin = new ReplicationAdmin(HTU.getConfiguration()); - admin.addPeer("2", HTU2.getClusterKey()); + + ReplicationPeerConfig rpc = new ReplicationPeerConfig(); + rpc.setClusterKey(HTU2.getClusterKey()); + admin.addPeer("2", rpc, null); admin.close(); Put p = new Put(row); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java index e187b9b..d22aa15 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java @@ -17,9 +17,7 @@ */ package org.apache.hadoop.hbase.client.replication; -import java.util.List; -import java.util.Map; -import java.util.TreeMap; +import java.util.*; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -27,6 +25,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationFactory; +import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationQueues; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -38,10 +37,7 @@ import org.junit.experimental.categories.Category; import com.google.common.collect.Lists; -import static org.junit.Assert.fail; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertFalse; +import static org.junit.Assert.*; /** * Unit testing of ReplicationAdmin @@ -87,11 +83,15 @@ public class TestReplicationAdmin { */ @Test public void testAddRemovePeer() throws Exception { + ReplicationPeerConfig rpc1 = new ReplicationPeerConfig(); + rpc1.setClusterKey(KEY_ONE); + ReplicationPeerConfig rpc2 = new ReplicationPeerConfig(); + rpc2.setClusterKey(KEY_SECOND); // Add a valid peer - admin.addPeer(ID_ONE, KEY_ONE); + admin.addPeer(ID_ONE, rpc1, null); // try adding the same (fails) try { - admin.addPeer(ID_ONE, KEY_ONE); + admin.addPeer(ID_ONE, rpc1, null); } catch (IllegalArgumentException iae) { // OK! } @@ -106,7 +106,7 @@ public class TestReplicationAdmin { assertEquals(1, admin.getPeersCount()); // Add a second since multi-slave is supported try { - admin.addPeer(ID_SECOND, KEY_SECOND); + admin.addPeer(ID_SECOND, rpc2, null); } catch (IllegalStateException iae) { fail(); } @@ -120,6 +120,10 @@ public class TestReplicationAdmin { @Test public void testAddPeerWithUnDeletedQueues() throws Exception { + ReplicationPeerConfig rpc1 = new ReplicationPeerConfig(); + rpc1.setClusterKey(KEY_ONE); + ReplicationPeerConfig rpc2 = new ReplicationPeerConfig(); + rpc2.setClusterKey(KEY_SECOND); Configuration conf = TEST_UTIL.getConfiguration(); ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "Test HBaseAdmin", null); ReplicationQueues repQueues = @@ -129,7 +133,7 @@ public class TestReplicationAdmin { // add queue for ID_ONE repQueues.addLog(ID_ONE, "file1"); try { - admin.addPeer(ID_ONE, KEY_ONE); + admin.addPeer(ID_ONE, rpc1, null); fail(); } catch (ReplicationException e) { // OK! @@ -140,7 +144,7 @@ public class TestReplicationAdmin { // add recovered queue for ID_ONE repQueues.addLog(ID_ONE + "-server2", "file1"); try { - admin.addPeer(ID_ONE, KEY_ONE); + admin.addPeer(ID_ONE, rpc2, null); fail(); } catch (ReplicationException e) { // OK! @@ -155,7 +159,9 @@ public class TestReplicationAdmin { */ @Test public void testEnableDisable() throws Exception { - admin.addPeer(ID_ONE, KEY_ONE); + ReplicationPeerConfig rpc1 = new ReplicationPeerConfig(); + rpc1.setClusterKey(KEY_ONE); + admin.addPeer(ID_ONE, rpc1, null); assertEquals(1, admin.getPeersCount()); assertTrue(admin.getPeerState(ID_ONE)); admin.disablePeer(ID_ONE); @@ -170,100 +176,140 @@ public class TestReplicationAdmin { } @Test - public void testGetTableCfsStr() { - // opposite of TestPerTableCFReplication#testParseTableCFsFromConfig() - - Map> tabCFsMap = null; - - // 1. null or empty string, result should be null - assertEquals(null, ReplicationAdmin.getTableCfsStr(tabCFsMap)); - - - // 2. single table: "tab1" / "tab2:cf1" / "tab3:cf1,cf3" - tabCFsMap = new TreeMap>(); - tabCFsMap.put(TableName.valueOf("tab1"), null); // its table name is "tab1" - assertEquals("tab1", ReplicationAdmin.getTableCfsStr(tabCFsMap)); - - tabCFsMap = new TreeMap>(); - tabCFsMap.put(TableName.valueOf("tab1"), Lists.newArrayList("cf1")); - assertEquals("tab1:cf1", ReplicationAdmin.getTableCfsStr(tabCFsMap)); - - tabCFsMap = new TreeMap>(); - tabCFsMap.put(TableName.valueOf("tab1"), Lists.newArrayList("cf1", "cf3")); - assertEquals("tab1:cf1,cf3", ReplicationAdmin.getTableCfsStr(tabCFsMap)); - - // 3. multiple tables: "tab1 ; tab2:cf1 ; tab3:cf1,cf3" - tabCFsMap = new TreeMap>(); - tabCFsMap.put(TableName.valueOf("tab1"), null); - tabCFsMap.put(TableName.valueOf("tab2"), Lists.newArrayList("cf1")); - tabCFsMap.put(TableName.valueOf("tab3"), Lists.newArrayList("cf1", "cf3")); - assertEquals("tab1;tab2:cf1;tab3:cf1,cf3", ReplicationAdmin.getTableCfsStr(tabCFsMap)); - } - - @Test public void testAppendPeerTableCFs() throws Exception { + ReplicationPeerConfig rpc1 = new ReplicationPeerConfig(); + rpc1.setClusterKey(KEY_ONE); + TableName tab1 = TableName.valueOf("t1"); + TableName tab2 = TableName.valueOf("t2"); + TableName tab3 = TableName.valueOf("t3"); + TableName tab4 = TableName.valueOf("t4"); + // Add a valid peer - admin.addPeer(ID_ONE, KEY_ONE); + admin.addPeer(ID_ONE, rpc1, null); - admin.appendPeerTableCFs(ID_ONE, "t1"); - assertEquals("t1", admin.getPeerTableCFs(ID_ONE)); + Map> tableCFs = new HashMap<>(); - // append table t2 to replication - admin.appendPeerTableCFs(ID_ONE, "t2"); - String peerTablesOne = admin.getPeerTableCFs(ID_ONE); + tableCFs.put(tab1, null); + admin.appendPeerTableCFs(ID_ONE, tableCFs); + Map> result = admin.getPeerTableCFsMap(ID_ONE); + assertEquals(1, result.size()); + assertEquals(true, result.containsKey(tab1)); + assertNull(result.get(tab1)); - // Different jdk's return different sort order for the tables. ( Not sure on why exactly ) - // - // So instead of asserting that the string is exactly we - // assert that the string contains all tables and the needed separator. - assertTrue("Should contain t1", peerTablesOne.contains("t1")); - assertTrue("Should contain t2", peerTablesOne.contains("t2")); - assertTrue("Should contain ; as the seperator", peerTablesOne.contains(";")); + // append table t2 to replication + tableCFs.clear(); + tableCFs.put(tab2, null); + admin.appendPeerTableCFs(ID_ONE, tableCFs); + result = admin.getPeerTableCFsMap(ID_ONE); + assertEquals(2, result.size()); + assertTrue("Should contain t1", result.containsKey(tab1)); + assertTrue("Should contain t2", result.containsKey(tab2)); + assertNull(result.get(tab1)); + assertNull(result.get(tab2)); // append table column family: f1 of t3 to replication - admin.appendPeerTableCFs(ID_ONE, "t3:f1"); - String peerTablesTwo = admin.getPeerTableCFs(ID_ONE); - assertTrue("Should contain t1", peerTablesTwo.contains("t1")); - assertTrue("Should contain t2", peerTablesTwo.contains("t2")); - assertTrue("Should contain t3:f1", peerTablesTwo.contains("t3:f1")); - assertTrue("Should contain ; as the seperator", peerTablesTwo.contains(";")); + tableCFs.clear(); + tableCFs.put(tab3, new ArrayList()); + tableCFs.get(tab3).add("f1"); + admin.appendPeerTableCFs(ID_ONE, tableCFs); + result = admin.getPeerTableCFsMap(ID_ONE); + assertEquals(3, result.size()); + assertTrue("Should contain t1", result.containsKey(tab1)); + assertTrue("Should contain t2", result.containsKey(tab2)); + assertTrue("Should contain t3", result.containsKey(tab3)); + assertNull(result.get(tab1)); + assertNull(result.get(tab2)); + assertEquals(1, result.get(tab3).size()); + assertEquals("f1", result.get(tab3).get(0)); + + tableCFs.clear(); + tableCFs.put(tab4, new ArrayList()); + tableCFs.get(tab4).add("f1"); + tableCFs.get(tab4).add("f2"); + admin.appendPeerTableCFs(ID_ONE, tableCFs); + result = admin.getPeerTableCFsMap(ID_ONE); + assertEquals(4, result.size()); + assertTrue("Should contain t1", result.containsKey(tab1)); + assertTrue("Should contain t2", result.containsKey(tab2)); + assertTrue("Should contain t3", result.containsKey(tab3)); + assertTrue("Should contain t4", result.containsKey(tab4)); + assertNull(result.get(tab1)); + assertNull(result.get(tab2)); + assertEquals(1, result.get(tab3).size()); + assertEquals("f1", result.get(tab3).get(0)); + assertEquals(2, result.get(tab4).size()); + assertEquals("f1", result.get(tab4).get(0)); + assertEquals("f2", result.get(tab4).get(1)); + admin.removePeer(ID_ONE); } @Test public void testRemovePeerTableCFs() throws Exception { + ReplicationPeerConfig rpc1 = new ReplicationPeerConfig(); + rpc1.setClusterKey(KEY_ONE); + TableName tab1 = TableName.valueOf("t1"); + TableName tab2 = TableName.valueOf("t2"); + TableName tab3 = TableName.valueOf("t3"); // Add a valid peer - admin.addPeer(ID_ONE, KEY_ONE); + admin.addPeer(ID_ONE, rpc1, null); + Map> tableCFs = new HashMap<>(); try { - admin.removePeerTableCFs(ID_ONE, "t3"); + tableCFs.put(tab3, null); + admin.removePeerTableCFs(ID_ONE, tableCFs); assertTrue(false); } catch (ReplicationException e) { } - assertEquals("", admin.getPeerTableCFs(ID_ONE)); + assertNull(admin.getPeerTableCFsMap(ID_ONE)); - admin.setPeerTableCFs(ID_ONE, "t1;t2:cf1"); + tableCFs.clear(); + tableCFs.put(tab1, null); + tableCFs.put(tab2, new ArrayList()); + tableCFs.get(tab2).add("cf1"); + admin.setPeerTableCFs(ID_ONE, tableCFs); try { - admin.removePeerTableCFs(ID_ONE, "t3"); + tableCFs.clear(); + tableCFs.put(tab3, null); + admin.removePeerTableCFs(ID_ONE, tableCFs); assertTrue(false); } catch (ReplicationException e) { } - assertEquals("t1;t2:cf1", admin.getPeerTableCFs(ID_ONE)); + Map> result = admin.getPeerTableCFsMap(ID_ONE); + assertEquals(2, result.size()); + assertTrue("Should contain t1", result.containsKey(tab1)); + assertTrue("Should contain t2", result.containsKey(tab2)); + assertNull(result.get(tab1)); + assertEquals(1, result.get(tab2).size()); + assertEquals("cf1", result.get(tab2).get(0)); try { - admin.removePeerTableCFs(ID_ONE, "t1:f1"); + tableCFs.clear(); + tableCFs.put(tab1, new ArrayList()); + tableCFs.get(tab1).add("f1"); + admin.removePeerTableCFs(ID_ONE, tableCFs); assertTrue(false); } catch (ReplicationException e) { } - admin.removePeerTableCFs(ID_ONE, "t1"); - assertEquals("t2:cf1", admin.getPeerTableCFs(ID_ONE)); + tableCFs.clear(); + tableCFs.put(tab1, null); + admin.removePeerTableCFs(ID_ONE, tableCFs); + result = admin.getPeerTableCFsMap(ID_ONE); + assertEquals(1, result.size()); + assertEquals(1, result.get(tab2).size()); + assertEquals("cf1", result.get(tab2).get(0)); try { - admin.removePeerTableCFs(ID_ONE, "t2"); + tableCFs.clear(); + tableCFs.put(tab2, null); + admin.removePeerTableCFs(ID_ONE, tableCFs); assertTrue(false); } catch (ReplicationException e) { } - admin.removePeerTableCFs(ID_ONE, "t2:cf1"); - assertEquals("", admin.getPeerTableCFs(ID_ONE)); + tableCFs.clear(); + tableCFs.put(tab2, new ArrayList()); + tableCFs.get(tab2).add("cf1"); + admin.removePeerTableCFs(ID_ONE, tableCFs); + assertNull(admin.getPeerTableCFsMap(ID_ONE)); admin.removePeer(ID_ONE); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java index 455a790..8d712a6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMasterReplication.java @@ -322,8 +322,9 @@ public class TestMasterReplication { try { replicationAdmin = new ReplicationAdmin( configurations[masterClusterNumber]); - replicationAdmin.addPeer(id, - utilities[slaveClusterNumber].getClusterKey()); + ReplicationPeerConfig rpc = new ReplicationPeerConfig(); + rpc.setClusterKey(utilities[slaveClusterNumber].getClusterKey()); + replicationAdmin.addPeer(id, rpc, null); } finally { close(replicationAdmin); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java index 82a0680..b778864 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestMultiSlaveReplication.java @@ -138,8 +138,10 @@ public class TestMultiSlaveReplication { htable2.setWriteBufferSize(1024); Table htable3 = utility3.getConnection().getTable(tableName); htable3.setWriteBufferSize(1024); - - admin1.addPeer("1", utility2.getClusterKey()); + + ReplicationPeerConfig rpc = new ReplicationPeerConfig(); + rpc.setClusterKey(utility2.getClusterKey()); + admin1.addPeer("1", rpc, null); // put "row" and wait 'til it got around, then delete putAndWait(row, famName, htable1, htable2); @@ -155,7 +157,9 @@ public class TestMultiSlaveReplication { // after the log was rolled put a new row putAndWait(row3, famName, htable1, htable2); - admin1.addPeer("2", utility3.getClusterKey()); + rpc = new ReplicationPeerConfig(); + rpc.setClusterKey(utility3.getClusterKey()); + admin1.addPeer("2", rpc, null); // put a row, check it was replicated to all clusters putAndWait(row1, famName, htable1, htable2, htable3); 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 29a052b..8772ed4 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 @@ -19,13 +19,9 @@ */ package org.apache.hadoop.hbase.replication; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -47,7 +43,10 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; +import org.apache.hadoop.hbase.client.replication.TableCFsHelper; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.testclassification.FlakeyTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; @@ -58,6 +57,8 @@ import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; +import static org.junit.Assert.*; + @Category({FlakeyTests.class, LargeTests.class}) public class TestPerTableCFReplication { @@ -184,13 +185,13 @@ public class TestPerTableCFReplication { Map> tabCFsMap = null; // 1. null or empty string, result should be null - tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig(null); + tabCFsMap = TableCFsHelper.parseTableCFsFromConfig(null); assertEquals(null, tabCFsMap); - tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig(""); + tabCFsMap = TableCFsHelper.parseTableCFsFromConfig(""); assertEquals(null, tabCFsMap); - tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig(" "); + tabCFsMap = TableCFsHelper.parseTableCFsFromConfig(" "); assertEquals(null, tabCFsMap); TableName tab1 = TableName.valueOf("tab1"); @@ -198,20 +199,20 @@ public class TestPerTableCFReplication { TableName tab3 = TableName.valueOf("tab3"); // 2. single table: "tab1" / "tab2:cf1" / "tab3:cf1,cf3" - tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig("tab1"); + tabCFsMap = TableCFsHelper.parseTableCFsFromConfig("tab1"); assertEquals(1, tabCFsMap.size()); // only one table assertTrue(tabCFsMap.containsKey(tab1)); // its table name is "tab1" assertFalse(tabCFsMap.containsKey(tab2)); // not other table assertEquals(null, tabCFsMap.get(tab1)); // null cf-list, - tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig("tab2:cf1"); + tabCFsMap = TableCFsHelper.parseTableCFsFromConfig("tab2:cf1"); assertEquals(1, tabCFsMap.size()); // only one table assertTrue(tabCFsMap.containsKey(tab2)); // its table name is "tab2" assertFalse(tabCFsMap.containsKey(tab1)); // not other table assertEquals(1, tabCFsMap.get(tab2).size()); // cf-list contains only 1 cf assertEquals("cf1", tabCFsMap.get(tab2).get(0));// the only cf is "cf1" - tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig("tab3 : cf1 , cf3"); + tabCFsMap = TableCFsHelper.parseTableCFsFromConfig("tab3 : cf1 , cf3"); assertEquals(1, tabCFsMap.size()); // only one table assertTrue(tabCFsMap.containsKey(tab3)); // its table name is "tab2" assertFalse(tabCFsMap.containsKey(tab1)); // not other table @@ -220,7 +221,7 @@ public class TestPerTableCFReplication { assertTrue(tabCFsMap.get(tab3).contains("cf3"));// contains "cf3" // 3. multiple tables: "tab1 ; tab2:cf1 ; tab3:cf1,cf3" - tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig("tab1 ; tab2:cf1 ; tab3:cf1,cf3"); + tabCFsMap = TableCFsHelper.parseTableCFsFromConfig("tab1 ; tab2:cf1 ; tab3:cf1,cf3"); // 3.1 contains 3 tables : "tab1", "tab2" and "tab3" assertEquals(3, tabCFsMap.size()); assertTrue(tabCFsMap.containsKey(tab1)); @@ -238,7 +239,7 @@ public class TestPerTableCFReplication { // 4. contiguous or additional ";"(table delimiter) or ","(cf delimiter) can be tolerated // still use the example of multiple tables: "tab1 ; tab2:cf1 ; tab3:cf1,cf3" - tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig( + tabCFsMap = TableCFsHelper.parseTableCFsFromConfig( "tab1 ; ; tab2:cf1 ; tab3:cf1,,cf3 ;"); // 4.1 contains 3 tables : "tab1", "tab2" and "tab3" assertEquals(3, tabCFsMap.size()); @@ -257,7 +258,7 @@ public class TestPerTableCFReplication { // 5. invalid format "tab1:tt:cf1 ; tab2::cf1 ; tab3:cf1,cf3" // "tab1:tt:cf1" and "tab2::cf1" are invalid and will be ignored totally - tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig( + tabCFsMap = TableCFsHelper.parseTableCFsFromConfig( "tab1:tt:cf1 ; tab2::cf1 ; tab3:cf1,cf3"); // 5.1 no "tab1" and "tab2", only "tab3" assertEquals(1, tabCFsMap.size()); // only one table @@ -270,6 +271,130 @@ public class TestPerTableCFReplication { assertTrue(tabCFsMap.get(tab3).contains("cf3")); } + @Test + public void testTableCFsHelperConverter() { + + ZooKeeperProtos.TableCFs tableCFs = null; + Map> tabCFsMap = null; + + // 1. null or empty string, result should be null + assertNull(TableCFsHelper.convert(tabCFsMap)); + + tabCFsMap = new HashMap>(); + tableCFs = TableCFsHelper.convert(tabCFsMap); + assertEquals(0, tableCFs.getTableCfsCount()); + + TableName tab1 = TableName.valueOf("tab1"); + TableName tab2 = TableName.valueOf("tab2"); + TableName tab3 = TableName.valueOf("tab3"); + + // 2. single table: "tab1" / "tab2:cf1" / "tab3:cf1,cf3" + tabCFsMap.clear(); + tabCFsMap.put(tab1, null); + tableCFs = TableCFsHelper.convert(tabCFsMap); + assertEquals(1, tableCFs.getTableCfsCount()); // only one table + assertEquals(tab1.toString(), + tableCFs.getTableCfs(0).getTableName().getQualifier().toStringUtf8()); + assertEquals(0, tableCFs.getTableCfs(0).getFamiliesCount()); + + tabCFsMap.clear(); + tabCFsMap.put(tab2, new ArrayList()); + tabCFsMap.get(tab2).add("cf1"); + tableCFs = TableCFsHelper.convert(tabCFsMap); + assertEquals(1, tableCFs.getTableCfsCount()); // only one table + assertEquals(tab2.toString(), + tableCFs.getTableCfs(0).getTableName().getQualifier().toStringUtf8()); + assertEquals(1, tableCFs.getTableCfs(0).getFamiliesCount()); + assertEquals("cf1", tableCFs.getTableCfs(0).getFamilies(0).toStringUtf8()); + + tabCFsMap.clear(); + tabCFsMap.put(tab3, new ArrayList()); + tabCFsMap.get(tab3).add("cf1"); + tabCFsMap.get(tab3).add("cf3"); + tableCFs = TableCFsHelper.convert(tabCFsMap); + assertEquals(1, tableCFs.getTableCfsCount()); + assertEquals(tab3.toString(), + tableCFs.getTableCfs(0).getTableName().getQualifier().toStringUtf8()); + assertEquals(2, tableCFs.getTableCfs(0).getFamiliesCount()); + assertEquals("cf1", tableCFs.getTableCfs(0).getFamilies(0).toStringUtf8()); + assertEquals("cf3", tableCFs.getTableCfs(0).getFamilies(1).toStringUtf8()); + + tabCFsMap.clear(); + tabCFsMap.put(tab1, null); + tabCFsMap.put(tab2, new ArrayList()); + tabCFsMap.get(tab2).add("cf1"); + tabCFsMap.put(tab3, new ArrayList()); + tabCFsMap.get(tab3).add("cf1"); + tabCFsMap.get(tab3).add("cf3"); + + tableCFs = TableCFsHelper.convert(tabCFsMap); + assertEquals(3, tableCFs.getTableCfsCount()); + assertNotNull(TableCFsHelper.getTableCF(tableCFs, tab1.toString())); + assertNotNull(TableCFsHelper.getTableCF(tableCFs, tab2.toString())); + assertNotNull(TableCFsHelper.getTableCF(tableCFs, tab3.toString())); + + assertEquals(0, + TableCFsHelper.getTableCF(tableCFs, tab1.toString()).getFamiliesCount()); + + assertEquals(1, + TableCFsHelper.getTableCF(tableCFs, tab2.toString()).getFamiliesCount()); + assertEquals("cf1", + TableCFsHelper.getTableCF(tableCFs, tab2.toString()).getFamilies(0).toStringUtf8()); + + assertEquals(2, + TableCFsHelper.getTableCF(tableCFs, tab3.toString()).getFamiliesCount()); + assertEquals("cf1", + TableCFsHelper.getTableCF(tableCFs, tab3.toString()).getFamilies(0).toStringUtf8()); + assertEquals("cf3", + TableCFsHelper.getTableCF(tableCFs, tab3.toString()).getFamilies(1).toStringUtf8()); + + tabCFsMap = TableCFsHelper.convert2Map(tableCFs); + assertEquals(3, tabCFsMap.size()); + assertTrue(tabCFsMap.containsKey(tab1)); + assertTrue(tabCFsMap.containsKey(tab2)); + assertTrue(tabCFsMap.containsKey(tab3)); + // 3.2 table "tab1" : null cf-list + assertEquals(null, tabCFsMap.get(tab1)); + // 3.3 table "tab2" : cf-list contains a single cf "cf1" + assertEquals(1, tabCFsMap.get(tab2).size()); + assertEquals("cf1", tabCFsMap.get(tab2).get(0)); + // 3.4 table "tab3" : cf-list contains "cf1" and "cf3" + assertEquals(2, tabCFsMap.get(tab3).size()); + assertTrue(tabCFsMap.get(tab3).contains("cf1")); + assertTrue(tabCFsMap.get(tab3).contains("cf3")); + } + + @Test + public void testOldFormatTableCFsReading() throws IOException { + TableName tab1 = TableName.valueOf("tab1"); + TableName tab2 = TableName.valueOf("tab2"); + TableName tab3 = TableName.valueOf("tab3"); + + byte[] bytes = Bytes.toBytes("tab1 ; tab2:cf1 ; tab3:cf1,cf3"); + ZooKeeperProtos.TableCFs tableCFs = TableCFsHelper.parseTableCFs(bytes); + assertEquals(3, tableCFs.getTableCfsCount()); + assertNotNull(TableCFsHelper.getTableCF(tableCFs, tab1.toString())); + assertNotNull(TableCFsHelper.getTableCF(tableCFs, tab2.toString())); + assertNotNull(TableCFsHelper.getTableCF(tableCFs, tab3.toString())); + assertEquals(0, + TableCFsHelper.getTableCF(tableCFs, tab1.toString()).getFamiliesCount()); + assertEquals(1, + TableCFsHelper.getTableCF(tableCFs, tab2.toString()).getFamiliesCount()); + assertEquals("cf1", + TableCFsHelper.getTableCF(tableCFs, tab2.toString()).getFamilies(0).toStringUtf8()); + assertEquals(2, + TableCFsHelper.getTableCF(tableCFs, tab3.toString()).getFamiliesCount()); + assertEquals("cf1", + TableCFsHelper.getTableCF(tableCFs, tab3.toString()).getFamilies(0).toStringUtf8()); + assertEquals("cf3", + TableCFsHelper.getTableCF(tableCFs, tab3.toString()).getFamilies(1).toStringUtf8()); + + assertFalse(ProtobufUtil.isPBMagicPrefix(bytes)); + bytes = TableCFsHelper.toByteArray(tableCFs); + assertTrue(ProtobufUtil.isPBMagicPrefix(bytes)); + } + + @Test(timeout=300000) public void testPerTableCFReplication() throws Exception { LOG.info("testPerTableCFReplication"); @@ -305,8 +430,23 @@ public class TestPerTableCFReplication { Table htab3C = connection3.getTable(tabCName); // A. add cluster2/cluster3 as peers to cluster1 - replicationAdmin.addPeer("2", utility2.getClusterKey(), "TC;TB:f1,f3"); - replicationAdmin.addPeer("3", utility3.getClusterKey(), "TA;TB:f1,f2"); + ReplicationPeerConfig rpc2 = new ReplicationPeerConfig(); + rpc2.setClusterKey(utility2.getClusterKey()); + Map> tableCFs = new HashMap<>(); + tableCFs.put(tabCName, null); + tableCFs.put(tabBName, new ArrayList()); + tableCFs.get(tabBName).add("f1"); + tableCFs.get(tabBName).add("f3"); + replicationAdmin.addPeer("2", rpc2, tableCFs); + + ReplicationPeerConfig rpc3 = new ReplicationPeerConfig(); + rpc3.setClusterKey(utility3.getClusterKey()); + tableCFs.clear(); + tableCFs.put(tabAName, null); + tableCFs.put(tabBName, new ArrayList()); + tableCFs.get(tabBName).add("f1"); + tableCFs.get(tabBName).add("f2"); + replicationAdmin.addPeer("3", rpc3, tableCFs); // A1. tableA can only replicated to cluster3 putAndWaitWithFamily(row1, f1Name, htab1A, htab3A); @@ -349,8 +489,20 @@ public class TestPerTableCFReplication { deleteAndWaitWithFamily(row1, f3Name, htab1C, htab2C); // B. change peers' replicable table-cf config - replicationAdmin.setPeerTableCFs("2", "TA:f1,f2; TC:f2,f3"); - replicationAdmin.setPeerTableCFs("3", "TB; TC:f3"); + tableCFs.clear(); + tableCFs.put(tabAName, new ArrayList()); + tableCFs.get(tabAName).add("f1"); + tableCFs.get(tabAName).add("f2"); + tableCFs.put(tabCName, new ArrayList()); + tableCFs.get(tabCName).add("f2"); + tableCFs.get(tabCName).add("f3"); + replicationAdmin.setPeerTableCFs("2", tableCFs); + + tableCFs.clear(); + tableCFs.put(tabBName, null); + tableCFs.put(tabCName, new ArrayList()); + tableCFs.get(tabCName).add("f3"); + replicationAdmin.setPeerTableCFs("3", tableCFs); // B1. cf 'f1' of tableA can only replicated to cluster2 putAndWaitWithFamily(row2, f1Name, htab1A, htab2A); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java index ac87269..b54b01a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java @@ -124,7 +124,9 @@ public class TestReplicationBase { utility2.setZkCluster(miniZK); zkw2 = new ZooKeeperWatcher(conf2, "cluster2", null, true); - admin.addPeer("2", utility2.getClusterKey()); + ReplicationPeerConfig rpc = new ReplicationPeerConfig(); + rpc.setClusterKey(utility2.getClusterKey()); + admin.addPeer("2", rpc, null); LOG.info("Setup second Zk"); CONF_WITH_LOCALFS = HBaseConfiguration.create(conf1); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java index b050f49..535000f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java @@ -353,8 +353,9 @@ public class TestReplicationSmallTests extends TestReplicationBase { Thread.sleep(SLEEP_TIME); } } - - admin.addPeer("2", utility2.getClusterKey()); + ReplicationPeerConfig rpc = new ReplicationPeerConfig(); + rpc.setClusterKey(utility2.getClusterKey()); + admin.addPeer("2", rpc, null); Thread.sleep(SLEEP_TIME); rowKey = Bytes.toBytes("do rep"); put = new Put(rowKey); 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 f05eceb..254331b 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 @@ -191,9 +191,9 @@ public abstract class TestReplicationStateBasic { assertNumberOfPeers(0); // Add some peers - rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE), null); + rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE)); assertNumberOfPeers(1); - rp.addPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO), null); + rp.addPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO)); assertNumberOfPeers(2); // Test methods with a peer that is added but not connected @@ -208,7 +208,7 @@ public abstract class TestReplicationStateBasic { assertNumberOfPeers(1); // Add one peer - rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE), null); + rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE)); rp.peerAdded(ID_ONE); assertNumberOfPeers(2); assertTrue(rp.getStatusOfPeer(ID_ONE)); @@ -268,7 +268,7 @@ public abstract class TestReplicationStateBasic { rq3.addLog("qId" + i, "filename" + j); } //Add peers for the corresponding queues so they are not orphans - rp.addPeer("qId" + i, new ReplicationPeerConfig().setClusterKey("bogus" + i), null); + rp.addPeer("qId" + i, new ReplicationPeerConfig().setClusterKey("bogus" + i)); } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java index 13545b5..b220b6b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java @@ -211,7 +211,9 @@ public class TestReplicationSyncUpTool extends TestReplicationBase { /** * set M-S : Master: utility1 Slave1: utility2 */ - admin1.addPeer("1", utility2.getClusterKey()); + ReplicationPeerConfig rpc = new ReplicationPeerConfig(); + rpc.setClusterKey(utility2.getClusterKey()); + admin1.addPeer("1", rpc, null); admin1.close(); admin2.close(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java index a082b19..e74f1db 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java @@ -146,7 +146,7 @@ public class TestReplicationTrackerZKImpl { @Test(timeout = 30000) public void testPeerRemovedEvent() throws Exception { - rp.addPeer("5", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey()), null); + rp.addPeer("5", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey())); rt.registerListener(new DummyReplicationListener()); rp.removePeer("5"); // wait for event @@ -159,7 +159,7 @@ public class TestReplicationTrackerZKImpl { @Test(timeout = 30000) public void testPeerListChangedEvent() throws Exception { // add a peer - rp.addPeer("5", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey()), null); + rp.addPeer("5", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey())); zkw.getRecoverableZooKeeper().getZooKeeper().getChildren("/hbase/replication/peers/5", true); rt.registerListener(new DummyReplicationListener()); rp.disablePeer("5"); @@ -183,16 +183,16 @@ public class TestReplicationTrackerZKImpl { public void testPeerNameControl() throws Exception { int exists = 0; int hyphen = 0; - rp.addPeer("6", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey()), null); + rp.addPeer("6", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey())); try{ - rp.addPeer("6", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey()), null); + rp.addPeer("6", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey())); }catch(IllegalArgumentException e){ exists++; } try{ - rp.addPeer("6-ec2", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey()), null); + rp.addPeer("6-ec2", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey())); }catch(IllegalArgumentException e){ hyphen++; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java index 988373f..275e7da 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java @@ -125,12 +125,14 @@ public class TestReplicationWithTags { conf2.setBoolean("hbase.tests.use.shortcircuit.reads", false); conf2.setStrings(HConstants.REPLICATION_CODEC_CONF_KEY, KeyValueCodecWithTags.class.getName()); conf2.setStrings(CoprocessorHost.USER_REGION_COPROCESSOR_CONF_KEY, - TestCoprocessorForTagsAtSink.class.getName()); + TestCoprocessorForTagsAtSink.class.getName()); utility2 = new HBaseTestingUtility(conf2); utility2.setZkCluster(miniZK); - replicationAdmin.addPeer("2", utility2.getClusterKey()); + ReplicationPeerConfig rpc = new ReplicationPeerConfig(); + rpc.setClusterKey(utility2.getClusterKey()); + replicationAdmin.addPeer("2", rpc, null); LOG.info("Setup second Zk"); utility1.startMiniCluster(2); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java index fecff07..e777e1b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelReplicationWithExpAsString.java @@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; import org.apache.hadoop.hbase.codec.KeyValueCodecWithTags; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; +import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.visibility.VisibilityController.VisibilityReplication; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -102,7 +103,7 @@ public class TestVisibilityLabelReplicationWithExpAsString extends TestVisibilit // Have to reset conf1 in case zk cluster location different // than default conf.setClass(VisibilityUtils.VISIBILITY_LABEL_GENERATOR_CLASS, SimpleScanLabelGenerator.class, - ScanLabelGenerator.class); + ScanLabelGenerator.class); conf.set("hbase.superuser", "admin"); conf.set("hbase.superuser", User.getCurrent().getShortName()); SUPERUSER = User.createUserForTesting(conf, User.getCurrent().getShortName(), @@ -126,12 +127,14 @@ public class TestVisibilityLabelReplicationWithExpAsString extends TestVisibilit conf.setBoolean(HConstants.REPLICATION_ENABLE_KEY, HConstants.REPLICATION_ENABLE_DEFAULT); conf1.setStrings(HConstants.REPLICATION_CODEC_CONF_KEY, KeyValueCodecWithTags.class.getName()); conf1.setStrings(CoprocessorHost.USER_REGION_COPROCESSOR_CONF_KEY, - TestCoprocessorForTagsAtSink.class.getName()); + TestCoprocessorForTagsAtSink.class.getName()); setVisibilityLabelServiceImpl(conf1, ExpAsStringVisibilityLabelServiceImpl.class); TEST_UTIL1 = new HBaseTestingUtility(conf1); TEST_UTIL1.setZkCluster(miniZK); zkw2 = new ZooKeeperWatcher(conf1, "cluster2", null, true); - replicationAdmin.addPeer("2", TEST_UTIL1.getClusterKey()); + ReplicationPeerConfig rpc = new ReplicationPeerConfig(); + rpc.setClusterKey(TEST_UTIL1.getClusterKey()); + replicationAdmin.addPeer("2", rpc, null); TEST_UTIL.startMiniCluster(1); // Wait for the labels table to become available diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java index 8414813..c6aca0a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsReplication.java @@ -64,6 +64,7 @@ import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.replication.ReplicationEndpoint; +import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.visibility.VisibilityController.VisibilityReplication; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -180,7 +181,9 @@ public class TestVisibilityLabelsReplication { TEST_UTIL1 = new HBaseTestingUtility(conf1); TEST_UTIL1.setZkCluster(miniZK); zkw2 = new ZooKeeperWatcher(conf1, "cluster2", null, true); - replicationAdmin.addPeer("2", TEST_UTIL1.getClusterKey()); + ReplicationPeerConfig rpc = new ReplicationPeerConfig(); + rpc.setClusterKey(TEST_UTIL1.getClusterKey()); + replicationAdmin.addPeer("2", rpc, null); TEST_UTIL.startMiniCluster(1); // Wait for the labels table to become available diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java index c1c49e2..0bed9f5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java @@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.TestEndToEndSplitTransaction; import org.apache.hadoop.hbase.replication.ReplicationFactory; +import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationQueues; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MiscTests; @@ -1492,7 +1493,9 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck { Assert.assertEquals(0, replicationAdmin.getPeersCount()); int zkPort = conf.getInt(HConstants.ZOOKEEPER_CLIENT_PORT, HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT); - replicationAdmin.addPeer("1", "127.0.0.1:" + zkPort + ":/hbase"); + ReplicationPeerConfig rpc = new ReplicationPeerConfig(); + rpc.setClusterKey("127.0.0.1:" + zkPort + ":/hbase"); + replicationAdmin.addPeer("1", rpc, null); replicationAdmin.getPeersCount(); Assert.assertEquals(1, replicationAdmin.getPeersCount()); diff --git a/hbase-shell/src/main/ruby/hbase/replication_admin.rb b/hbase-shell/src/main/ruby/hbase/replication_admin.rb index 617073b..9b92e75 100644 --- a/hbase-shell/src/main/ruby/hbase/replication_admin.rb +++ b/hbase-shell/src/main/ruby/hbase/replication_admin.rb @@ -40,11 +40,7 @@ module Hbase #---------------------------------------------------------------------------------------------- # Add a new peer cluster to replicate to def add_peer(id, args = {}, peer_tableCFs = nil) - # make add_peer backwards compatible to take in string for clusterKey and peer_tableCFs - if args.is_a?(String) - cluster_key = args - @replication_admin.addPeer(id, cluster_key, peer_tableCFs) - elsif args.is_a?(Hash) + if args.is_a?(Hash) unless peer_tableCFs.nil? raise(ArgumentError, "peer_tableCFs should be specified as TABLE_CFS in args") end @@ -114,7 +110,7 @@ module Hbase #---------------------------------------------------------------------------------------------- # List all peer clusters def list_peers - @replication_admin.listPeers + @replication_admin.listPeerConfigs end #---------------------------------------------------------------------------------------------- @@ -138,7 +134,7 @@ module Hbase #---------------------------------------------------------------------------------------------- # Show the current tableCFs config for the specified peer def show_peer_tableCFs(id) - @replication_admin.getPeerTableCFs(id) + @replication_admin.getPeerTableCFsMap(id) end #---------------------------------------------------------------------------------------------- diff --git a/hbase-shell/src/main/ruby/shell/commands/append_peer_tableCFs.rb b/hbase-shell/src/main/ruby/shell/commands/append_peer_tableCFs.rb index 3919b20..b4d13fb 100644 --- a/hbase-shell/src/main/ruby/shell/commands/append_peer_tableCFs.rb +++ b/hbase-shell/src/main/ruby/shell/commands/append_peer_tableCFs.rb @@ -26,7 +26,7 @@ Append a replicable table-cf config for the specified peer Examples: # append a table / table-cf to be replicable for a peer - hbase> append_peer_tableCFs '2', "table4:cfA,cfB" + hbase> append_peer_tableCFs '2', {ns1.table4 => ['cfA', 'cfB']} EOF end diff --git a/hbase-shell/src/main/ruby/shell/commands/remove_peer_tableCFs.rb b/hbase-shell/src/main/ruby/shell/commands/remove_peer_tableCFs.rb index 5b15b52..e06c441 100644 --- a/hbase-shell/src/main/ruby/shell/commands/remove_peer_tableCFs.rb +++ b/hbase-shell/src/main/ruby/shell/commands/remove_peer_tableCFs.rb @@ -26,8 +26,8 @@ Remove a table / table-cf from the table-cfs config for the specified peer Examples: # Remove a table / table-cf from the replicable table-cfs for a peer - hbase> remove_peer_tableCFs '2', "table1" - hbase> remove_peer_tableCFs '2', "table1:cf1" + hbase> remove_peer_tableCFs '2', {ns1.table1} + hbase> remove_peer_tableCFs '2', {ns1.table1 => ['cf1']} EOF end diff --git a/hbase-shell/src/main/ruby/shell/commands/set_peer_tableCFs.rb b/hbase-shell/src/main/ruby/shell/commands/set_peer_tableCFs.rb index 3a88dbb..ed87262 100644 --- a/hbase-shell/src/main/ruby/shell/commands/set_peer_tableCFs.rb +++ b/hbase-shell/src/main/ruby/shell/commands/set_peer_tableCFs.rb @@ -32,7 +32,7 @@ module Shell # set table / table-cf to be replicable for a peer, for a table without # an explicit column-family list, all replicable column-families (with # replication_scope == 1) will be replicated - hbase> set_peer_tableCFs '2', "table1; table2:cf1,cf2; table3:cfA,cfB" + hbase> set_peer_tableCFs '2', {ns1.table1 => [], ns2.table2 => ['cf1', 'cf2'], ns3.table3 => ['cfA', 'cfB']} EOF end -- 2.3.8 (Apple Git-58)