HBASE-16653 Backport HBASE-11393 to branches which support namespace

Signed-off-by: chenheng <chenheng@apache.org>
This commit is contained in:
Guanghao Zhang 2016-09-26 19:33:43 +08:00 committed by chenheng
parent 6df7554d29
commit 66941910bd
36 changed files with 2178 additions and 382 deletions

View File

@ -55,6 +55,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.ReplicationPeerZKImpl;
import org.apache.hadoop.hbase.replication.ReplicationPeers;
import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
import org.apache.hadoop.hbase.replication.ReplicationSerDeHelper;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@ -184,8 +185,8 @@ public class ReplicationAdmin implements Closeable {
@Deprecated
public void addPeer(String id, String clusterKey, String tableCFs)
throws ReplicationException {
this.replicationPeers.addPeer(id,
new ReplicationPeerConfig().setClusterKey(clusterKey), tableCFs);
this.addPeer(id, new ReplicationPeerConfig().setClusterKey(clusterKey),
parseTableCFsFromConfig(tableCFs));
}
/**
@ -199,7 +200,19 @@ public class ReplicationAdmin implements Closeable {
*/
public void addPeer(String id, ReplicationPeerConfig peerConfig,
Map<TableName, ? extends Collection<String>> tableCfs) throws ReplicationException {
this.replicationPeers.addPeer(id, peerConfig, getTableCfsStr(tableCfs));
if (tableCfs != null) {
peerConfig.setTableCFsMap(tableCfs);
}
this.replicationPeers.addPeer(id, peerConfig);
}
/**
* 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
*/
public void addPeer(String id, ReplicationPeerConfig peerConfig) throws ReplicationException {
this.replicationPeers.addPeer(id, peerConfig);
}
public void updatePeerConfig(String id, ReplicationPeerConfig peerConfig)
@ -208,52 +221,7 @@ public class ReplicationAdmin implements Closeable {
}
public static Map<TableName, List<String>> parseTableCFsFromConfig(String tableCFsConfig) {
if (tableCFsConfig == null || tableCFsConfig.trim().length() == 0) {
return null;
}
Map<TableName, List<String>> 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<cf>
List<String> 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<String>();
}
cfs.add(cfName);
}
}
}
// 4 put <table, List<cf>> to map
if (tableCFsMap == null) {
tableCFsMap = new HashMap<TableName, List<String>>();
}
tableCFsMap.put(TableName.valueOf(tabName), cfs);
}
return tableCFsMap;
return ReplicationSerDeHelper.parseTableCFsFromConfig(tableCFsConfig);
}
@VisibleForTesting
@ -338,7 +306,7 @@ public class ReplicationAdmin implements Closeable {
* @param id a short name that identifies the cluster
*/
public String getPeerTableCFs(String id) throws ReplicationException {
return this.replicationPeers.getPeerTableCFsConfig(id);
return ReplicationSerDeHelper.convertToString(this.replicationPeers.getPeerTableCFsConfig(id));
}
/**
@ -348,7 +316,7 @@ public class ReplicationAdmin implements Closeable {
*/
@Deprecated
public void setPeerTableCFs(String id, String tableCFs) throws ReplicationException {
this.replicationPeers.setPeerTableCFsConfig(id, tableCFs);
this.setPeerTableCFs(id, parseTableCFsFromConfig(tableCFs));
}
/**
@ -357,7 +325,7 @@ public class ReplicationAdmin implements Closeable {
* @param tableCfs table-cfs config str
*/
public void appendPeerTableCFs(String id, String tableCfs) throws ReplicationException {
appendPeerTableCFs(id, parseTableCFsFromConfig(tableCfs));
appendPeerTableCFs(id, ReplicationSerDeHelper.parseTableCFsFromConfig(tableCfs));
}
/**
@ -370,7 +338,7 @@ public class ReplicationAdmin implements Closeable {
if (tableCfs == null) {
throw new ReplicationException("tableCfs is null");
}
Map<TableName, List<String>> preTableCfs = parseTableCFsFromConfig(getPeerTableCFs(id));
Map<TableName, List<String>> preTableCfs = this.replicationPeers.getPeerTableCFsConfig(id);
if (preTableCfs == null) {
setPeerTableCFs(id, tableCfs);
return;
@ -406,7 +374,7 @@ public class ReplicationAdmin implements Closeable {
* @throws ReplicationException
*/
public void removePeerTableCFs(String id, String tableCf) throws ReplicationException {
removePeerTableCFs(id, parseTableCFsFromConfig(tableCf));
removePeerTableCFs(id, ReplicationSerDeHelper.parseTableCFsFromConfig(tableCf));
}
/**
@ -421,7 +389,7 @@ public class ReplicationAdmin implements Closeable {
throw new ReplicationException("tableCfs is null");
}
Map<TableName, List<String>> preTableCfs = parseTableCFsFromConfig(getPeerTableCFs(id));
Map<TableName, List<String>> preTableCfs = this.replicationPeers.getPeerTableCFsConfig(id);
if (preTableCfs == null) {
throw new ReplicationException("Table-Cfs for peer" + id + " is null");
}
@ -464,7 +432,7 @@ public class ReplicationAdmin implements Closeable {
*/
public void setPeerTableCFs(String id, Map<TableName, ? extends Collection<String>> tableCfs)
throws ReplicationException {
this.replicationPeers.setPeerTableCFsConfig(id, getTableCfsStr(tableCfs));
this.replicationPeers.setPeerTableCFsConfig(id, tableCfs);
}
/**
@ -658,8 +626,8 @@ public class ReplicationAdmin implements Closeable {
try {
Pair<ReplicationPeerConfig, Configuration> pair = this.replicationPeers.getPeerConf(peerId);
Configuration peerConf = pair.getSecond();
ReplicationPeer peer = new ReplicationPeerZKImpl(peerConf, peerId, pair.getFirst(),
parseTableCFsFromConfig(this.getPeerTableCFs(peerId)));
ReplicationPeer peer = new ReplicationPeerZKImpl(zkw, pair.getSecond(),
peerId, pair.getFirst(), this.connection);
listOfPeers.add(peer);
} catch (ReplicationException e) {
LOG.warn("Failed to get valid replication peers. "

View File

@ -18,10 +18,13 @@
package org.apache.hadoop.hbase.replication;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
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.util.Bytes;
@ -37,7 +40,7 @@ public class ReplicationPeerConfig {
private String replicationEndpointImpl;
private final Map<byte[], byte[]> peerData;
private final Map<String, String> configuration;
private Map<TableName, ? extends Collection<String>> tableCFsMap = null;
public ReplicationPeerConfig() {
this.peerData = new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
@ -78,10 +81,21 @@ public class ReplicationPeerConfig {
return configuration;
}
public Map<TableName, List<String>> getTableCFsMap() {
return (Map<TableName, List<String>>) tableCFsMap;
}
public void setTableCFsMap(Map<TableName, ? extends Collection<String>> tableCFsMap) {
this.tableCFsMap = tableCFsMap;
}
@Override
public String toString() {
StringBuilder builder = new StringBuilder("clusterKey=").append(clusterKey).append(",");
builder.append("replicationEndpointImpl=").append(replicationEndpointImpl);
if (tableCFsMap != null) {
builder.append(tableCFsMap.toString());
}
return builder.toString();
}
}

View File

@ -42,7 +42,8 @@ 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 ReplicationPeerConfig peerConfig;
@ -52,8 +53,8 @@ public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closea
private final Configuration conf;
private PeerStateTracker peerStateTracker;
private TableCFsTracker tableCFsTracker;
private PeerConfigTracker peerConfigTracker;
/**
* Constructor that takes all the objects required to communicate with the specified peer, except
* for the region server addresses.
@ -61,39 +62,23 @@ 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)
throws ReplicationException {
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;
}
/**
* Constructor that takes all the objects required to communicate with the specified peer, except
* for the region server addresses.
* @param conf configuration object to this peer
* @param id string representation of this peer's identifier
* @param peerConfig configuration for the replication peer
* @param tableCFs table-cf configuration for this peer
*/
public ReplicationPeerZKImpl(Configuration conf, String id, ReplicationPeerConfig peerConfig,
Map<TableName, List<String>> tableCFs) throws ReplicationException {
this.conf = conf;
this.peerConfig = peerConfig;
this.id = id;
this.tableCFs = tableCFs;
}
/**
* 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 {
@ -110,25 +95,6 @@ public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closea
: PeerState.DISABLED;
}
/**
* 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
* @throws KeeperException
*/
public void startTableCFsTracker(ZooKeeperWatcher zookeeper, String tableCFsNode)
throws KeeperException {
this.tableCFsTracker = new TableCFsTracker(tableCFsNode, zookeeper,
this);
this.tableCFsTracker.start();
this.readTableCFsZnode();
}
private void readTableCFsZnode() {
String currentTableCFs = Bytes.toString(tableCFsTracker.getData(false));
this.tableCFs = ReplicationAdmin.parseTableCFsFromConfig(currentTableCFs);
}
/**
* start a table-cfs tracker to listen the (table, cf-list) map change
* @param zookeeper
@ -154,6 +120,7 @@ public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closea
}
return this.peerConfig;
}
@Override
public PeerState getPeerState() {
return peerState;
@ -192,6 +159,7 @@ public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closea
*/
@Override
public Map<TableName, List<String>> getTableCFs() {
this.tableCFs = peerConfig.getTableCFsMap();
return this.tableCFs;
}
@ -260,7 +228,7 @@ public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closea
* @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
@ -296,32 +264,6 @@ public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closea
}
}
/**
* Tracker for (table, cf-list) map of this peer
*/
public class TableCFsTracker extends ZooKeeperNodeTracker {
public TableCFsTracker(String tableCFsZNode, ZooKeeperWatcher watcher,
Abortable abortable) {
super(watcher, tableCFsZNode, abortable);
}
@Override
public synchronized void nodeCreated(String path) {
if (path.equals(node)) {
super.nodeCreated(path);
readTableCFsZnode();
}
}
@Override
public synchronized void nodeDataChanged(String path) {
if (path.equals(node)) {
super.nodeDataChanged(path);
}
}
}
/**
* Tracker for PeerConfigNode of this peer
*/

View File

@ -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;
/**
@ -78,17 +77,19 @@ public interface ReplicationPeers {
void disablePeer(String peerId) throws ReplicationException;
/**
* Get the table and column-family list string of the peer from ZK.
* Get the table and column-family list of the peer from ZK.
* @param peerId a short that identifies the cluster
*/
public String getPeerTableCFsConfig(String peerId) throws ReplicationException;
public Map<TableName, List<String>> getPeerTableCFsConfig(String peerId)
throws ReplicationException;
/**
* Set the table and column-family list string of the peer to ZK.
* Set the table and column-family list 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;
public void setPeerTableCFsConfig(String peerId,
Map<TableName, ? extends Collection<String>> tableCFs) throws ReplicationException;
/**
* Get the table and column-family-list map of the peer.

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.replication;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Set;
@ -78,15 +79,15 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
// Map of peer clusters keyed by their id
private Map<String, ReplicationPeerZKImpl> 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<String, ReplicationPeerZKImpl>();
this.queuesClient = queuesClient;
}
@ -104,7 +105,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)) {
@ -129,18 +130,15 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
ZKUtil.createWithParents(this.zookeeper, this.peersZNode);
List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>();
ZKUtilOp op1 = ZKUtilOp.createAndFailSilent(ZKUtil.joinZNode(this.peersZNode, id),
ZKUtilOp op1 = ZKUtilOp.createAndFailSilent(getPeerNode(id),
ReplicationSerDeHelper.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) {
@ -175,13 +173,17 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
}
@Override
public String getPeerTableCFsConfig(String id) throws ReplicationException {
public Map<TableName, List<String>> 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 rpc.getTableCFsMap();
} catch (Exception e) {
throw new ReplicationException(e);
}
@ -191,20 +193,22 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
}
@Override
public void setPeerTableCFsConfig(String id, String tableCFsStr) throws ReplicationException {
public void setPeerTableCFsConfig(String id,
Map<TableName, ? extends Collection<String>> 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.setTableCFsMap(tableCFs);
ZKUtil.setData(this.zookeeper, getPeerNode(id),
ReplicationSerDeHelper.toByteArray(rpc));
LOG.info("Peer tableCFs with id= " + id + " is now "
+ ReplicationSerDeHelper.convertToString(tableCFs));
} catch (KeeperException e) {
throw new ReplicationException("Unable to change tableCFs of the peer with id=" + id, e);
}
@ -289,7 +293,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);
@ -458,14 +462,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
@ -506,21 +502,15 @@ 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(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));
} catch (KeeperException e) {
throw new ReplicationException("Error starting the peer tableCFs tracker for peerId=" +
peerId, e);
}
try {
peer.startPeerConfigTracker(this.zookeeper, this.getPeerNode(peerId));
}

View File

@ -19,8 +19,10 @@
package org.apache.hadoop.hbase.replication;
import com.google.protobuf.ByteString;
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.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
@ -28,8 +30,13 @@ 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.util.Bytes;
import org.apache.hadoop.hbase.util.Strings;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@InterfaceAudience.Private
@ -39,6 +46,175 @@ public final class ReplicationSerDeHelper {
private ReplicationSerDeHelper() {}
/** convert map to TableCFs Object */
public static ZooKeeperProtos.TableCF[] convert(
Map<TableName, ? extends Collection<String>> tableCfs) {
if (tableCfs == null) {
return null;
}
List<ZooKeeperProtos.TableCF> tableCFList = new ArrayList<>();
ZooKeeperProtos.TableCF.Builder tableCFBuilder = ZooKeeperProtos.TableCF.newBuilder();
for (Map.Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
tableCFBuilder.clear();
tableCFBuilder.setTableName(ProtobufUtil.toProtoTableName(entry.getKey()));
Collection<String> v = entry.getValue();
if (v != null && !v.isEmpty()) {
for (String value : entry.getValue()) {
tableCFBuilder.addFamilies(ByteString.copyFromUtf8(value));
}
}
tableCFList.add(tableCFBuilder.build());
}
return tableCFList.toArray(new ZooKeeperProtos.TableCF[tableCFList.size()]);
}
public static String convertToString(Map<TableName, ? extends Collection<String>> tableCfs) {
if (tableCfs == null) {
return null;
}
return convert(convert(tableCfs));
}
/**
* Convert string to TableCFs Object.
* This is only for read TableCFs information from TableCF node.
* Input String Format: ns1.table1:cf1,cf2;ns2.table2:cfA,cfB;ns3.table3.
* */
public static ZooKeeperProtos.TableCF[] convert(String tableCFsConfig) {
if (tableCFsConfig == null || tableCFsConfig.trim().length() == 0) {
return null;
}
List<ZooKeeperProtos.TableCF> tableCFList = new ArrayList<>();
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) {
LOG.info("incorrect format:" + tableCFsConfig);
continue;
}
tableCFBuilder.clear();
// split namespace from tableName
String ns = "default";
String tName = tabName;
String[] dbs = tabName.split("\\.");
if (dbs != null && dbs.length == 2) {
ns = dbs[0];
tName = dbs[1];
}
tableCFBuilder.setTableName(
ProtobufUtil.toProtoTableName(TableName.valueOf(ns, tName)));
// 3 parse "cf1,cf2" part to List<cf>
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));
}
}
}
tableCFList.add(tableCFBuilder.build());
}
return tableCFList.toArray(new ZooKeeperProtos.TableCF[tableCFList.size()]);
}
/**
* Convert TableCFs Object to String.
* Output String Format: ns1.table1:cf1,cf2;ns2.table2:cfA,cfB;table3
* */
public static String convert(ZooKeeperProtos.TableCF[] tableCFs) {
StringBuilder sb = new StringBuilder();
for (int i = 0, n = tableCFs.length; i < n; i++) {
ZooKeeperProtos.TableCF tableCF = tableCFs[i];
String namespace = tableCF.getTableName().getNamespace().toStringUtf8();
if (!Strings.isEmpty(namespace)) {
sb.append(namespace).append(".").
append(tableCF.getTableName().getQualifier().toStringUtf8())
.append(":");
} else {
sb.append(tableCF.getTableName().toString()).append(":");
}
for (int j = 0; j < tableCF.getFamiliesCount(); j++) {
sb.append(tableCF.getFamilies(j).toStringUtf8()).append(",");
}
sb.deleteCharAt(sb.length() - 1).append(";");
}
if (sb.length() > 0) {
sb.deleteCharAt(sb.length() - 1);
}
return sb.toString();
}
/**
* Get TableCF in TableCFs, if not exist, return null.
* */
public static ZooKeeperProtos.TableCF getTableCF(ZooKeeperProtos.TableCF[] tableCFs,
String table) {
for (int i = 0, n = tableCFs.length; i < n; i++) {
ZooKeeperProtos.TableCF tableCF = tableCFs[i];
if (tableCF.getTableName().getQualifier().toStringUtf8().equals(table)) {
return tableCF;
}
}
return null;
}
/**
* Parse bytes into TableCFs.
* It is used for backward compatibility.
* Old format bytes have no PB_MAGIC Header
* */
public static ZooKeeperProtos.TableCF[] parseTableCFs(byte[] bytes) throws IOException {
if (bytes == null) {
return null;
}
return ReplicationSerDeHelper.convert(Bytes.toString(bytes));
}
/**
* Convert tableCFs string into Map.
* */
public static Map<TableName, List<String>> parseTableCFsFromConfig(String tableCFsConfig) {
ZooKeeperProtos.TableCF[] tableCFs = convert(tableCFsConfig);
return convert2Map(tableCFs);
}
/**
* Convert tableCFs Object to Map.
* */
public static Map<TableName, List<String>> convert2Map(ZooKeeperProtos.TableCF[] tableCFs) {
if (tableCFs == null || tableCFs.length == 0) {
return null;
}
Map<TableName, List<String>> tableCFsMap = new HashMap<TableName, List<String>>();
for (int i = 0, n = tableCFs.length; i < n; i++) {
ZooKeeperProtos.TableCF tableCF = tableCFs[i];
List<String> families = new ArrayList<>();
for (int j = 0, m = tableCF.getFamiliesCount(); j < m; 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.
@ -82,6 +258,12 @@ public final class ReplicationSerDeHelper {
for (HBaseProtos.NameStringPair pair : peer.getConfigurationList()) {
peerConfig.getConfiguration().put(pair.getName(), pair.getValue());
}
Map<TableName, ? extends Collection<String>> tableCFsMap = convert2Map(
peer.getTableCfsList().toArray(new ZooKeeperProtos.TableCF[peer.getTableCfsCount()]));
if (tableCFsMap != null) {
peerConfig.setTableCFsMap(tableCFsMap);
}
return peerConfig;
}
@ -119,6 +301,13 @@ public final class ReplicationSerDeHelper {
.build());
}
ZooKeeperProtos.TableCF[] tableCFs = convert(peerConfig.getTableCFsMap());
if (tableCFs != null) {
for (int i = 0; i < tableCFs.length; i++) {
builder.addTableCfs(tableCFs[i]);
}
}
return builder.build();
}
}

View File

@ -30,6 +30,8 @@ import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import com.google.common.annotations.VisibleForTesting;
/**
* This is a base class for maintaining replication state in zookeeper.
@ -52,6 +54,9 @@ public abstract class ReplicationStateZKBase {
protected final String hfileRefsZNode;
/** The cluster key of the local cluster */
protected final String ourClusterKey;
/** The name of the znode that contains tableCFs */
protected final String tableCFsNodeName;
protected final ZooKeeperWatcher zookeeper;
protected final Configuration conf;
protected final Abortable abortable;
@ -77,6 +82,7 @@ public abstract class ReplicationStateZKBase {
String hfileRefsZNodeName = conf.get(ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_KEY,
ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT);
this.peerStateNodeName = conf.get("zookeeper.znode.replication.peers.state", "peer-state");
this.tableCFsNodeName = conf.get("zookeeper.znode.replication.peers.tableCFs", "tableCFs");
this.ourClusterKey = ZKConfig.getZooKeeperClusterKey(this.conf);
this.replicationZNode = ZKUtil.joinZNode(this.zookeeper.baseZNode, replicationZNodeName);
this.peersZNode = ZKUtil.joinZNode(replicationZNode, peersZNodeName);
@ -119,6 +125,17 @@ public abstract class ReplicationStateZKBase {
return path.split("/").length == peersZNode.split("/").length + 1;
}
@VisibleForTesting
protected String getTableCFsNode(String id) {
return ZKUtil.joinZNode(this.peersZNode, ZKUtil.joinZNode(id, this.tableCFsNodeName));
}
@VisibleForTesting
protected String getPeerStateNode(String id) {
return ZKUtil.joinZNode(this.peersZNode, ZKUtil.joinZNode(id, this.peerStateNodeName));
}
@VisibleForTesting
protected String getPeerNode(String id) {
return ZKUtil.joinZNode(this.peersZNode, id);
}

View File

@ -119,6 +119,11 @@ message Table {
required State state = 1 [default = ENABLED];
}
message TableCF {
optional TableName table_name = 1;
repeated bytes families = 2;
}
/**
* Used by replication. Holds a replication peer key.
*/
@ -129,6 +134,7 @@ message ReplicationPeer {
optional string replicationEndpointImpl = 2;
repeated BytesBytesPair data = 3;
repeated NameStringPair configuration = 4;
repeated TableCF table_cfs = 5;
}
/**
@ -173,4 +179,4 @@ message TableLock {
*/
message SwitchState {
optional bool enabled = 1;
}
}

View File

@ -143,6 +143,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.User;
import org.apache.hadoop.hbase.security.UserProvider;
@ -712,6 +713,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);

View File

@ -0,0 +1,120 @@
/*
*
* 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.replication.ReplicationSerDeHelper;
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.
* It will be removed in HBase 3.x. See HBASE-11393
*/
@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<String> 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);
// We only need to copy data from tableCFs node to rpc Node the first time hmaster start.
if (rpc.getTableCFsMap() == null || rpc.getTableCFsMap().size() == 0) {
// we copy TableCFs node into PeerNode
LOG.info("copy tableCFs into peerNode:" + peerId);
ZooKeeperProtos.TableCF[] tableCFs =
ReplicationSerDeHelper.parseTableCFs(
ZKUtil.getData(this.zookeeper, tableCFsNode));
rpc.setTableCFsMap(ReplicationSerDeHelper.convert2Map(tableCFs));
ZKUtil.setData(this.zookeeper, peerNode,
ReplicationSerDeHelper.toByteArray(rpc));
} else {
LOG.info("No tableCFs in peerNode:" + peerId);
}
}
} catch (KeeperException e) {
LOG.warn("NOTICE!! Update peerId failed, peerId=" + peerId, e);
return false;
} catch (InterruptedException e) {
LOG.warn("NOTICE!! Update peerId failed, peerId=" + peerId, e);
return false;
} catch (IOException e) {
LOG.warn("NOTICE!! Update peerId failed, peerId=" + peerId, 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 ReplicationSerDeHelper.parsePeerFrom(data);
} catch (DeserializationException e) {
LOG.warn("Failed to parse cluster key from peer=" + peerNode);
return null;
}
}
}

View File

@ -40,6 +40,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.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
@ -249,7 +250,9 @@ 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);
admin.close();
Put p = new Put(row);

View File

@ -17,6 +17,8 @@
*/
package org.apache.hadoop.hbase.client.replication;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
@ -44,6 +46,7 @@ import com.google.common.collect.Lists;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@ -92,11 +95,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);
// try adding the same (fails)
try {
admin.addPeer(ID_ONE, KEY_ONE);
admin.addPeer(ID_ONE, rpc1);
} catch (IllegalArgumentException iae) {
// OK!
}
@ -111,7 +118,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);
} catch (IllegalStateException iae) {
fail();
}
@ -125,6 +132,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 =
@ -134,7 +145,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);
fail();
} catch (ReplicationException e) {
// OK!
@ -145,7 +156,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);
fail();
} catch (ReplicationException e) {
// OK!
@ -182,7 +193,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);
assertEquals(1, admin.getPeersCount());
assertTrue(admin.getPeerState(ID_ONE));
admin.disablePeer(ID_ONE);
@ -196,101 +209,141 @@ public class TestReplicationAdmin {
admin.removePeer(ID_ONE);
}
@Test
public void testGetTableCfsStr() {
// opposite of TestPerTableCFReplication#testParseTableCFsFromConfig()
Map<TableName, List<String>> 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<TableName, List<String>>();
tabCFsMap.put(TableName.valueOf("tab1"), null); // its table name is "tab1"
assertEquals("tab1", ReplicationAdmin.getTableCfsStr(tabCFsMap));
tabCFsMap = new TreeMap<TableName, List<String>>();
tabCFsMap.put(TableName.valueOf("tab1"), Lists.newArrayList("cf1"));
assertEquals("tab1:cf1", ReplicationAdmin.getTableCfsStr(tabCFsMap));
tabCFsMap = new TreeMap<TableName, List<String>>();
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<TableName, List<String>>();
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 {
// Add a valid peer
admin.addPeer(ID_ONE, KEY_ONE);
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");
admin.appendPeerTableCFs(ID_ONE, "t1");
assertEquals("t1", admin.getPeerTableCFs(ID_ONE));
// Add a valid peer
admin.addPeer(ID_ONE, rpc1);
Map<TableName, List<String>> tableCFs = new HashMap<>();
tableCFs.put(tab1, null);
admin.appendPeerTableCFs(ID_ONE, tableCFs);
Map<TableName, List<String>> result = admin.getPeerConfig(ID_ONE).getTableCFsMap();
assertEquals(1, result.size());
assertEquals(true, result.containsKey(tab1));
assertNull(result.get(tab1));
// append table t2 to replication
admin.appendPeerTableCFs(ID_ONE, "t2");
String peerTablesOne = admin.getPeerTableCFs(ID_ONE);
// 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(";"));
tableCFs.clear();
tableCFs.put(tab2, null);
admin.appendPeerTableCFs(ID_ONE, tableCFs);
result = admin.getPeerConfig(ID_ONE).getTableCFsMap();
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<String>());
tableCFs.get(tab3).add("f1");
admin.appendPeerTableCFs(ID_ONE, tableCFs);
result = admin.getPeerConfig(ID_ONE).getTableCFsMap();
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<String>());
tableCFs.get(tab4).add("f1");
tableCFs.get(tab4).add("f2");
admin.appendPeerTableCFs(ID_ONE, tableCFs);
result = admin.getPeerConfig(ID_ONE).getTableCFsMap();
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);
Map<TableName, List<String>> 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.getPeerConfig(ID_ONE).getTableCFsMap());
admin.setPeerTableCFs(ID_ONE, "t1;t2:cf1");
tableCFs.clear();
tableCFs.put(tab1, null);
tableCFs.put(tab2, new ArrayList<String>());
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<TableName, List<String>> result = admin.getPeerConfig(ID_ONE).getTableCFsMap();
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));
tableCFs.clear();
tableCFs.put(tab1, new ArrayList<String>());
tableCFs.get(tab1).add("f1");
try {
admin.removePeerTableCFs(ID_ONE, "t1: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.getPeerConfig(ID_ONE).getTableCFsMap();
assertEquals(1, result.size());
assertEquals(1, result.get(tab2).size());
assertEquals("cf1", result.get(tab2).get(0));
tableCFs.clear();
tableCFs.put(tab2, null);
try {
admin.removePeerTableCFs(ID_ONE, "t2");
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<String>());
tableCFs.get(tab2).add("cf1");
admin.removePeerTableCFs(ID_ONE, tableCFs);
assertNull(admin.getPeerConfig(ID_ONE).getTableCFsMap());
admin.removePeer(ID_ONE);
}
}

View File

@ -110,7 +110,7 @@ public class TestReplicationHFileCleaner {
@Before
public void setup() throws ReplicationException, IOException {
root = TEST_UTIL.getDataTestDirOnTestFS();
rp.addPeer(peerId, new ReplicationPeerConfig().setClusterKey(TEST_UTIL.getClusterKey()), null);
rp.addPeer(peerId, new ReplicationPeerConfig().setClusterKey(TEST_UTIL.getClusterKey()));
rq.addPeerToHFileRefs(peerId);
}

View File

@ -508,8 +508,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);
} finally {
close(replicationAdmin);
}
@ -520,7 +521,9 @@ public class TestMasterReplication {
ReplicationAdmin replicationAdmin = null;
try {
replicationAdmin = new ReplicationAdmin(configurations[masterClusterNumber]);
replicationAdmin.addPeer(id, utilities[slaveClusterNumber].getClusterKey(), tableCfs);
ReplicationPeerConfig rpc = new ReplicationPeerConfig();
rpc.setClusterKey(utilities[slaveClusterNumber].getClusterKey());
replicationAdmin.addPeer(id, rpc, ReplicationSerDeHelper.parseTableCFsFromConfig(tableCfs));
} finally {
close(replicationAdmin);
}

View File

@ -152,7 +152,9 @@ public class TestMultiSlaveReplication {
Table htable3 = new HTable(conf3, tableName);
htable3.setWriteBufferSize(1024);
admin1.addPeer("1", utility2.getClusterKey());
ReplicationPeerConfig rpc = new ReplicationPeerConfig();
rpc.setClusterKey(utility2.getClusterKey());
admin1.addPeer("1", rpc);
// put "row" and wait 'til it got around, then delete
putAndWait(row, famName, htable1, htable2);
@ -168,7 +170,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);
// put a row, check it was replicated to all clusters
putAndWait(row1, famName, htable1, htable2, htable3);

View File

@ -22,10 +22,14 @@ 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.assertNotNull;
import static org.junit.Assert.assertNull;
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;
@ -37,6 +41,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
@ -183,13 +188,13 @@ public class TestPerTableCFReplication {
Map<TableName, List<String>> tabCFsMap = null;
// 1. null or empty string, result should be null
tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig(null);
tabCFsMap = ReplicationSerDeHelper.parseTableCFsFromConfig(null);
assertEquals(null, tabCFsMap);
tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig("");
tabCFsMap = ReplicationSerDeHelper.parseTableCFsFromConfig("");
assertEquals(null, tabCFsMap);
tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig(" ");
tabCFsMap = ReplicationSerDeHelper.parseTableCFsFromConfig(" ");
assertEquals(null, tabCFsMap);
TableName tab1 = TableName.valueOf("tab1");
@ -197,20 +202,20 @@ public class TestPerTableCFReplication {
TableName tab3 = TableName.valueOf("tab3");
// 2. single table: "tab1" / "tab2:cf1" / "tab3:cf1,cf3"
tabCFsMap = ReplicationAdmin.parseTableCFsFromConfig("tab1");
tabCFsMap = ReplicationSerDeHelper.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 = ReplicationSerDeHelper.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 = ReplicationSerDeHelper.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
@ -219,7 +224,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 = ReplicationSerDeHelper.parseTableCFsFromConfig("tab1 ; tab2:cf1 ; tab3:cf1,cf3");
// 3.1 contains 3 tables : "tab1", "tab2" and "tab3"
assertEquals(3, tabCFsMap.size());
assertTrue(tabCFsMap.containsKey(tab1));
@ -237,7 +242,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 = ReplicationSerDeHelper.parseTableCFsFromConfig(
"tab1 ; ; tab2:cf1 ; tab3:cf1,,cf3 ;");
// 4.1 contains 3 tables : "tab1", "tab2" and "tab3"
assertEquals(3, tabCFsMap.size());
@ -256,7 +261,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 = ReplicationSerDeHelper.parseTableCFsFromConfig(
"tab1:tt:cf1 ; tab2::cf1 ; tab3:cf1,cf3");
// 5.1 no "tab1" and "tab2", only "tab3"
assertEquals(1, tabCFsMap.size()); // only one table
@ -267,7 +272,100 @@ public class TestPerTableCFReplication {
assertEquals(2, tabCFsMap.get(tab3).size());
assertTrue(tabCFsMap.get(tab3).contains("cf1"));
assertTrue(tabCFsMap.get(tab3).contains("cf3"));
}
}
@Test
public void testTableCFsHelperConverter() {
ZooKeeperProtos.TableCF[] tableCFs = null;
Map<TableName, List<String>> tabCFsMap = null;
// 1. null or empty string, result should be null
assertNull(ReplicationSerDeHelper.convert(tabCFsMap));
tabCFsMap = new HashMap<TableName, List<String>>();
tableCFs = ReplicationSerDeHelper.convert(tabCFsMap);
assertEquals(0, tableCFs.length);
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 = ReplicationSerDeHelper.convert(tabCFsMap);
assertEquals(1, tableCFs.length); // only one table
assertEquals(tab1.toString(),
tableCFs[0].getTableName().getQualifier().toStringUtf8());
assertEquals(0, tableCFs[0].getFamiliesCount());
tabCFsMap.clear();
tabCFsMap.put(tab2, new ArrayList<String>());
tabCFsMap.get(tab2).add("cf1");
tableCFs = ReplicationSerDeHelper.convert(tabCFsMap);
assertEquals(1, tableCFs.length); // only one table
assertEquals(tab2.toString(),
tableCFs[0].getTableName().getQualifier().toStringUtf8());
assertEquals(1, tableCFs[0].getFamiliesCount());
assertEquals("cf1", tableCFs[0].getFamilies(0).toStringUtf8());
tabCFsMap.clear();
tabCFsMap.put(tab3, new ArrayList<String>());
tabCFsMap.get(tab3).add("cf1");
tabCFsMap.get(tab3).add("cf3");
tableCFs = ReplicationSerDeHelper.convert(tabCFsMap);
assertEquals(1, tableCFs.length);
assertEquals(tab3.toString(),
tableCFs[0].getTableName().getQualifier().toStringUtf8());
assertEquals(2, tableCFs[0].getFamiliesCount());
assertEquals("cf1", tableCFs[0].getFamilies(0).toStringUtf8());
assertEquals("cf3", tableCFs[0].getFamilies(1).toStringUtf8());
tabCFsMap.clear();
tabCFsMap.put(tab1, null);
tabCFsMap.put(tab2, new ArrayList<String>());
tabCFsMap.get(tab2).add("cf1");
tabCFsMap.put(tab3, new ArrayList<String>());
tabCFsMap.get(tab3).add("cf1");
tabCFsMap.get(tab3).add("cf3");
tableCFs = ReplicationSerDeHelper.convert(tabCFsMap);
assertEquals(3, tableCFs.length);
assertNotNull(ReplicationSerDeHelper.getTableCF(tableCFs, tab1.toString()));
assertNotNull(ReplicationSerDeHelper.getTableCF(tableCFs, tab2.toString()));
assertNotNull(ReplicationSerDeHelper.getTableCF(tableCFs, tab3.toString()));
assertEquals(0,
ReplicationSerDeHelper.getTableCF(tableCFs, tab1.toString()).getFamiliesCount());
assertEquals(1,
ReplicationSerDeHelper.getTableCF(tableCFs, tab2.toString()).getFamiliesCount());
assertEquals("cf1",
ReplicationSerDeHelper.getTableCF(tableCFs, tab2.toString()).getFamilies(0).toStringUtf8());
assertEquals(2,
ReplicationSerDeHelper.getTableCF(tableCFs, tab3.toString()).getFamiliesCount());
assertEquals("cf1",
ReplicationSerDeHelper.getTableCF(tableCFs, tab3.toString()).getFamilies(0).toStringUtf8());
assertEquals("cf3",
ReplicationSerDeHelper.getTableCF(tableCFs, tab3.toString()).getFamilies(1).toStringUtf8());
tabCFsMap = ReplicationSerDeHelper.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(timeout=300000)
public void testPerTableCFReplication() throws Exception {
@ -304,8 +402,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<TableName, List<String>> tableCFs = new HashMap<>();
tableCFs.put(tabCName, null);
tableCFs.put(tabBName, new ArrayList<String>());
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<String>());
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);
@ -348,8 +461,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<String>());
tableCFs.get(tabAName).add("f1");
tableCFs.get(tabAName).add("f2");
tableCFs.put(tabCName, new ArrayList<String>());
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<String>());
tableCFs.get(tabCName).add("f3");
replicationAdmin.setPeerTableCFs("3", tableCFs);
// B1. cf 'f1' of tableA can only replicated to cluster2
putAndWaitWithFamily(row2, f1Name, htab1A, htab2A);

View File

@ -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);
LOG.info("Setup second Zk");
CONF_WITH_LOCALFS = HBaseConfiguration.create(conf1);

View File

@ -369,7 +369,9 @@ public class TestReplicationSmallTests extends TestReplicationBase {
}
}
admin.addPeer("2", utility2.getClusterKey());
ReplicationPeerConfig rpc = new ReplicationPeerConfig();
rpc.setClusterKey(utility2.getClusterKey());
admin.addPeer("2", rpc);
Thread.sleep(SLEEP_TIME);
rowKey = Bytes.toBytes("do rep");
put = new Put(rowKey);

View File

@ -170,7 +170,7 @@ public abstract class TestReplicationStateBasic {
try {
rp.addPeer(ID_ONE,
new ReplicationPeerConfig().setClusterKey("hostname1.example.org:1234:hbase"), null);
new ReplicationPeerConfig().setClusterKey("hostname1.example.org:1234:hbase"));
fail("Should throw an IllegalArgumentException because "
+ "zookeeper.znode.parent is missing leading '/'.");
} catch (IllegalArgumentException e) {
@ -179,7 +179,7 @@ public abstract class TestReplicationStateBasic {
try {
rp.addPeer(ID_ONE,
new ReplicationPeerConfig().setClusterKey("hostname1.example.org:1234:/"), null);
new ReplicationPeerConfig().setClusterKey("hostname1.example.org:1234:/"));
fail("Should throw an IllegalArgumentException because zookeeper.znode.parent is missing.");
} catch (IllegalArgumentException e) {
// Expected.
@ -187,7 +187,7 @@ public abstract class TestReplicationStateBasic {
try {
rp.addPeer(ID_ONE,
new ReplicationPeerConfig().setClusterKey("hostname1.example.org::/hbase"), null);
new ReplicationPeerConfig().setClusterKey("hostname1.example.org::/hbase"));
fail("Should throw an IllegalArgumentException because "
+ "hbase.zookeeper.property.clientPort is missing.");
} catch (IllegalArgumentException e) {
@ -207,7 +207,7 @@ public abstract class TestReplicationStateBasic {
files1.add("file_3");
assertNull(rqc.getReplicableHFiles(ID_ONE));
assertEquals(0, rqc.getAllPeersFromHFileRefsQueue().size());
rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE), null);
rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
rq1.addPeerToHFileRefs(ID_ONE);
rq1.addHFileRefs(ID_ONE, files1);
assertEquals(1, rqc.getAllPeersFromHFileRefsQueue().size());
@ -229,8 +229,8 @@ public abstract class TestReplicationStateBasic {
rqc.init();
rp.init();
rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE), null);
rp.addPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO), null);
rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
rp.addPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO));
List<String> files1 = new ArrayList<String>(3);
files1.add("file_1");
@ -288,9 +288,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
@ -305,7 +305,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));
@ -365,7 +365,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("localhost:2818:/bogus" + i), null);
rp.addPeer("qId" + i, new ReplicationPeerConfig().setClusterKey("localhost:2818:/bogus" + i));
}
}
}

View File

@ -203,7 +203,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);
admin1.close();
admin2.close();

View File

@ -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++;
}

View File

@ -130,7 +130,9 @@ public class TestReplicationWithTags {
utility2 = new HBaseTestingUtility(conf2);
utility2.setZkCluster(miniZK);
replicationAdmin.addPeer("2", utility2.getClusterKey());
ReplicationPeerConfig rpc = new ReplicationPeerConfig();
rpc.setClusterKey(utility2.getClusterKey());
replicationAdmin.addPeer("2", rpc);
LOG.info("Setup second Zk");
utility1.startMiniCluster(2);

View File

@ -0,0 +1,164 @@
/**
* Copyright The Apache Software Foundation
*
* 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.HBaseTestingUtility;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.replication.ReplicationSerDeHelper;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import java.util.List;
import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
@Category({ReplicationTests.class, SmallTests.class})
public class TestTableCFsUpdater extends TableCFsUpdater {
private static final Log LOG = LogFactory.getLog(TestTableCFsUpdater.class);
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private static ZooKeeperWatcher zkw = null;
private static Abortable abortable = null;
public TestTableCFsUpdater() {
super(zkw, TEST_UTIL.getConfiguration(), abortable);
}
@BeforeClass
public static void setUpBeforeClass() throws Exception {
TEST_UTIL.startMiniZKCluster();
Configuration conf = TEST_UTIL.getConfiguration();
abortable = new Abortable() {
@Override
public void abort(String why, Throwable e) {
LOG.info(why, e);
}
@Override
public boolean isAborted() {
return false;
}
};
zkw = new ZooKeeperWatcher(conf, "TableCFs", abortable, true);
}
@AfterClass
public static void tearDownAfterClass() throws Exception {
TEST_UTIL.shutdownMiniZKCluster();
}
@Test
public void testUpgrade() throws KeeperException, InterruptedException,
DeserializationException {
String peerId = "1";
TableName tab1 = TableName.valueOf("table1");
TableName tab2 = TableName.valueOf("table2");
TableName tab3 = TableName.valueOf("table3");
ReplicationPeerConfig rpc = new ReplicationPeerConfig();
rpc.setClusterKey(zkw.getQuorum());
String peerNode = getPeerNode(peerId);
ZKUtil.createWithParents(zkw, peerNode, ReplicationSerDeHelper.toByteArray(rpc));
String tableCFs = "table1:cf1,cf2;table2:cf3;table3";
String tableCFsNode = getTableCFsNode(peerId);
LOG.info("create tableCFs :" + tableCFsNode + " for peerId=" + peerId);
ZKUtil.createWithParents(zkw, tableCFsNode , Bytes.toBytes(tableCFs));
ReplicationPeerConfig actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
String actualTableCfs = Bytes.toString(ZKUtil.getData(zkw, tableCFsNode));
assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey());
assertNull(actualRpc.getTableCFsMap());
assertEquals(tableCFs, actualTableCfs);
peerId = "2";
rpc = new ReplicationPeerConfig();
rpc.setClusterKey(zkw.getQuorum());
peerNode = getPeerNode(peerId);
ZKUtil.createWithParents(zkw, peerNode, ReplicationSerDeHelper.toByteArray(rpc));
tableCFs = "table1:cf1,cf3;table2:cf2";
tableCFsNode = getTableCFsNode(peerId);
LOG.info("create tableCFs :" + tableCFsNode + " for peerId=" + peerId);
ZKUtil.createWithParents(zkw, tableCFsNode , Bytes.toBytes(tableCFs));
actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
actualTableCfs = Bytes.toString(ZKUtil.getData(zkw, tableCFsNode));
assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey());
assertNull(actualRpc.getTableCFsMap());
assertEquals(tableCFs, actualTableCfs);
update();
peerId = "1";
peerNode = getPeerNode(peerId);
actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey());
Map<TableName, List<String>> tableNameListMap = actualRpc.getTableCFsMap();
assertEquals(3, tableNameListMap.size());
assertTrue(tableNameListMap.containsKey(tab1));
assertTrue(tableNameListMap.containsKey(tab2));
assertTrue(tableNameListMap.containsKey(tab3));
assertEquals(2, tableNameListMap.get(tab1).size());
assertEquals("cf1", tableNameListMap.get(tab1).get(0));
assertEquals("cf2", tableNameListMap.get(tab1).get(1));
assertEquals(1, tableNameListMap.get(tab2).size());
assertEquals("cf3", tableNameListMap.get(tab2).get(0));
assertNull(tableNameListMap.get(tab3));
peerId = "2";
peerNode = getPeerNode(peerId);
actualRpc = ReplicationSerDeHelper.parsePeerFrom(ZKUtil.getData(zkw, peerNode));
assertEquals(rpc.getClusterKey(), actualRpc.getClusterKey());
tableNameListMap = actualRpc.getTableCFsMap();
assertEquals(2, tableNameListMap.size());
assertTrue(tableNameListMap.containsKey(tab1));
assertTrue(tableNameListMap.containsKey(tab2));
assertEquals(2, tableNameListMap.get(tab1).size());
assertEquals("cf1", tableNameListMap.get(tab1).get(0));
assertEquals("cf3", tableNameListMap.get(tab1).get(1));
assertEquals(1, tableNameListMap.get(tab2).size());
assertEquals("cf2", tableNameListMap.get(tab2).get(0));
}
}

View File

@ -535,7 +535,7 @@ public class TestReplicationSourceManager {
FailInitializeDummyReplicationSource.class.getName());
final ReplicationPeers rp = manager.getReplicationPeers();
// Set up the znode and ReplicationPeer for the fake peer
rp.addPeer("FakePeer", new ReplicationPeerConfig().setClusterKey("localhost:1:/hbase"), null);
rp.addPeer("FakePeer", new ReplicationPeerConfig().setClusterKey("localhost:1:/hbase"));
// Wait for the peer to get created and connected
Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
@Override

View File

@ -47,6 +47,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;
@ -131,7 +132,9 @@ public class TestVisibilityLabelReplicationWithExpAsString extends TestVisibilit
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);
TEST_UTIL.startMiniCluster(1);
// Wait for the labels table to become available

View File

@ -65,6 +65,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.junit.experimental.categories.Category;
@ -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);
TEST_UTIL.startMiniCluster(1);
// Wait for the labels table to become available

View File

@ -108,6 +108,7 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.SplitTransactionImpl;
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.util.HBaseFsck.ErrorReporter;
@ -2260,7 +2261,9 @@ public class TestHBaseFsck {
ReplicationAdmin replicationAdmin = new ReplicationAdmin(conf);
Assert.assertEquals(0, replicationAdmin.getPeersCount());
String zkPort = conf.get(HConstants.ZOOKEEPER_CLIENT_PORT);
replicationAdmin.addPeer("1", "127.0.0.1:2181" + zkPort + ":/hbase");
ReplicationPeerConfig rpc = new ReplicationPeerConfig();
rpc.setClusterKey("127.0.0.1:2181" + zkPort + ":/hbase");
replicationAdmin.addPeer("1", rpc);
replicationAdmin.getPeersCount();
Assert.assertEquals(1, replicationAdmin.getPeersCount());

View File

@ -39,11 +39,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
@ -87,9 +83,18 @@ module Hbase
}
end
@replication_admin.add_peer(id, replication_peer_config, table_cfs)
unless table_cfs.nil?
# convert table_cfs to TableName
map = java.util.HashMap.new
table_cfs.each{|key, val|
map.put(org.apache.hadoop.hbase.TableName.valueOf(key), val)
}
replication_peer_config.set_table_cfs_map(map)
end
@replication_admin.add_peer(id, replication_peer_config)
else
raise(ArgumentError, "args must be either a String or Hash")
raise(ArgumentError, "args must be a Hash")
end
end
@ -111,7 +116,7 @@ module Hbase
#----------------------------------------------------------------------------------------------
# List all peer clusters
def list_peers
@replication_admin.listPeers
@replication_admin.listPeerConfigs
end
#----------------------------------------------------------------------------------------------
@ -141,20 +146,42 @@ module Hbase
#----------------------------------------------------------------------------------------------
# Set new tableCFs config for the specified peer
def set_peer_tableCFs(id, tableCFs)
@replication_admin.setPeerTableCFs(id, tableCFs)
unless tableCFs.nil?
# convert tableCFs to TableName
map = java.util.HashMap.new
tableCFs.each{|key, val|
map.put(org.apache.hadoop.hbase.TableName.valueOf(key), val)
}
end
@replication_admin.setPeerTableCFs(id, map)
end
#----------------------------------------------------------------------------------------------
# Append a tableCFs config for the specified peer
def append_peer_tableCFs(id, tableCFs)
@replication_admin.appendPeerTableCFs(id, tableCFs)
unless tableCFs.nil?
# convert tableCFs to TableName
map = java.util.HashMap.new
tableCFs.each{|key, val|
map.put(org.apache.hadoop.hbase.TableName.valueOf(key), val)
}
end
@replication_admin.appendPeerTableCFs(id, map)
end
#----------------------------------------------------------------------------------------------
# Remove some tableCFs from the tableCFs config of the specified peer
def remove_peer_tableCFs(id, tableCFs)
@replication_admin.removePeerTableCFs(id, tableCFs)
unless tableCFs.nil?
# convert tableCFs to TableName
map = java.util.HashMap.new
tableCFs.each{|key, val|
map.put(org.apache.hadoop.hbase.TableName.valueOf(key), val)
}
end
@replication_admin.removePeerTableCFs(id, map)
end
#----------------------------------------------------------------------------------------------
# Enables a table's replication switch
def enable_tablerep(table_name)

View File

@ -33,7 +33,7 @@ Examples:
hbase> add_peer '1', CLUSTER_KEY => "server1.cie.com:2181:/hbase"
hbase> add_peer '2', CLUSTER_KEY => "zk1,zk2,zk3:2182:/hbase-prod",
TABLE_CFS => { "table1" => [], "table2" => ["cf1"], "table3" => ["cf1", "cf2"] }
TABLE_CFS => { "table1" => [], "ns2:table2" => ["cf1"], "ns3:table3" => ["cf1", "cf2"] }
For a custom replication endpoint, the ENDPOINT_CLASSNAME can be provided. Two optional arguments
are DATA and CONFIG which can be specified to set different either the peer_data or configuration
@ -48,7 +48,7 @@ the key TABLE_CFS.
hbase> add_peer '9', ENDPOINT_CLASSNAME => 'org.apache.hadoop.hbase.MyReplicationEndpoint',
DATA => { "key1" => 1 }, CONFIG => { "config1" => "value1", "config2" => "value2" },
hbase> add_peer '10', ENDPOINT_CLASSNAME => 'org.apache.hadoop.hbase.MyReplicationEndpoint',
TABLE_CFS => { "table1" => [], "table2" => ["cf1"], "table3" => ["cf1", "cf2"] }
TABLE_CFS => { "table1" => [], "ns2:table2" => ["cf1"], "ns3:table3" => ["cf1", "cf2"] }
hbase> add_peer '11', ENDPOINT_CLASSNAME => 'org.apache.hadoop.hbase.MyReplicationEndpoint',
DATA => { "key1" => 1 }, CONFIG => { "config1" => "value1", "config2" => "value2" },
TABLE_CFS => { "table1" => [], "ns2:table2" => ["cf1"], "ns3:table3" => ["cf1", "cf2"] }

View File

@ -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

View File

@ -33,12 +33,14 @@ EOF
now = Time.now
peers = replication_admin.list_peers
formatter.header(["PEER_ID", "CLUSTER_KEY", "STATE", "TABLE_CFS"])
formatter.header(["PEER_ID", "CLUSTER_KEY", "ENDPOINT_CLASSNAME",
"STATE", "TABLE_CFS"])
peers.entrySet().each do |e|
state = replication_admin.get_peer_state(e.key)
tableCFs = replication_admin.show_peer_tableCFs(e.key)
formatter.row([ e.key, e.value, state, tableCFs ])
formatter.row([ e.key, e.value.getClusterKey,
e.value.getReplicationEndpointImpl, state, tableCFs ])
end
formatter.footer(now)

View File

@ -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

View File

@ -32,7 +32,9 @@ 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

View File

@ -28,7 +28,7 @@ import org.junit.experimental.categories.Category;
@Category({ ClientTests.class, LargeTests.class })
public class TestReplicationShell extends AbstractTestShell {
@Ignore ("Disabled because hangs on occasion.. about 10% of the time") @Test
@Test
public void testRunShellTests() throws IOException {
System.setProperty("shell.test.include", "replication_admin_test.rb");
// Start all ruby tests

View File

@ -62,47 +62,9 @@ module Hbase
assert_raise(ArgumentError) do
replication_admin.add_peer(@peer_id, ['test'])
end
end
define_test "add_peer: single zk cluster key" do
cluster_key = "server1.cie.com:2181:/hbase"
replication_admin.add_peer(@peer_id, cluster_key)
assert_equal(1, replication_admin.list_peers.length)
assert(replication_admin.list_peers.key?(@peer_id))
assert_equal(cluster_key, replication_admin.list_peers.fetch(@peer_id))
# cleanup for future tests
replication_admin.remove_peer(@peer_id)
end
define_test "add_peer: multiple zk cluster key" do
cluster_key = "zk1,zk2,zk3:2182:/hbase-prod"
replication_admin.add_peer(@peer_id, cluster_key)
assert_equal(1, replication_admin.list_peers.length)
assert(replication_admin.list_peers.key?(@peer_id))
assert_equal(replication_admin.list_peers.fetch(@peer_id), cluster_key)
# cleanup for future tests
replication_admin.remove_peer(@peer_id)
end
define_test "add_peer: multiple zk cluster key and table_cfs" do
cluster_key = "zk4,zk5,zk6:11000:/hbase-test"
table_cfs_str = "table1;table2:cf1;table3:cf2,cf3"
replication_admin.add_peer(@peer_id, cluster_key, table_cfs_str)
assert_equal(1, replication_admin.list_peers.length)
assert(replication_admin.list_peers.key?(@peer_id))
assert_equal(cluster_key, replication_admin.list_peers.fetch(@peer_id))
assert_equal(table_cfs_str, replication_admin.show_peer_tableCFs(@peer_id))
# cleanup for future tests
replication_admin.remove_peer(@peer_id)
assert_raise(ArgumentError) do
replication_admin.add_peer(@peer_id, 'test')
end
end
define_test "add_peer: single zk cluster key - peer config" do
@ -113,7 +75,7 @@ module Hbase
assert_equal(1, replication_admin.list_peers.length)
assert(replication_admin.list_peers.key?(@peer_id))
assert_equal(cluster_key, replication_admin.list_peers.fetch(@peer_id))
assert_equal(cluster_key, replication_admin.list_peers.fetch(@peer_id).get_cluster_key)
# cleanup for future tests
replication_admin.remove_peer(@peer_id)
@ -127,7 +89,7 @@ module Hbase
assert_equal(1, replication_admin.list_peers.length)
assert(replication_admin.list_peers.key?(@peer_id))
assert_equal(cluster_key, replication_admin.list_peers.fetch(@peer_id))
assert_equal(cluster_key, replication_admin.list_peers.fetch(@peer_id).get_cluster_key)
# cleanup for future tests
replication_admin.remove_peer(@peer_id)
@ -135,25 +97,37 @@ module Hbase
define_test "add_peer: multiple zk cluster key and table_cfs - peer config" do
cluster_key = "zk4,zk5,zk6:11000:/hbase-test"
table_cfs = { "table1" => [], "table2" => ["cf1"], "table3" => ["cf1", "cf2"] }
#table_cfs_str = "default.table1;default.table3:cf1,cf2;default.table2:cf1"
table_cfs = { "table1" => [], "ns2:table2" => ["cf1"], "ns3:table3" => ["cf1", "cf2"] }
args = { CLUSTER_KEY => cluster_key, TABLE_CFS => table_cfs }
replication_admin.add_peer(@peer_id, args)
assert_equal(1, command(:list_peers).length)
assert(command(:list_peers).key?(@peer_id))
assert_equal(cluster_key, command(:list_peers).fetch(@peer_id).get_cluster_key)
assert_equal(1, replication_admin.list_peers.length)
assert(replication_admin.list_peers.key?(@peer_id))
assert_equal(cluster_key, replication_admin.list_peers.fetch(@peer_id).get_cluster_key)
# Note: below assertion is dependent on the sort order of an unordered
# map and hence flaky depending on JVM
# Commenting out until HBASE-16274 is worked.
# assert_equal(table_cfs_str, command(:show_peer_tableCFs, @peer_id))
table_cfs_map = replication_admin.get_peer_config(@peer_id).getTableCFsMap()
assert_tablecfs_equal(table_cfs, table_cfs_map)
# cleanup for future tests
replication_admin.remove_peer(@peer_id)
end
def assert_tablecfs_equal(table_cfs, table_cfs_map)
assert_equal(table_cfs.length, table_cfs_map.length)
table_cfs_map.each{|key, value|
assert(table_cfs.has_key?(key.getNameAsString))
if table_cfs.fetch(key.getNameAsString).length == 0
assert_equal(nil, value)
else
assert_equal(table_cfs.fetch(key.getNameAsString).length, value.length)
value.each{|v|
assert(table_cfs.fetch(key.getNameAsString).include?(v))
}
end
}
end
define_test "add_peer: should fail when args is a hash and peer_tableCFs provided" do
cluster_key = "zk4,zk5,zk6:11000:/hbase-test"
table_cfs_str = "table1;table2:cf1;table3:cf1,cf2"
@ -164,6 +138,66 @@ module Hbase
end
end
define_test "set_peer_tableCFs: works with table-cfs map" do
cluster_key = "zk4,zk5,zk6:11000:/hbase-test"
args = { CLUSTER_KEY => cluster_key}
replication_admin.add_peer(@peer_id, args)
assert_equal(1, replication_admin.list_peers.length)
assert(replication_admin.list_peers.key?(@peer_id))
assert_equal(cluster_key, replication_admin.list_peers.fetch(@peer_id).get_cluster_key)
table_cfs = { "table1" => [], "table2" => ["cf1"], "ns3:table3" => ["cf1", "cf2"] }
replication_admin.set_peer_tableCFs(@peer_id, table_cfs)
table_cfs_map = replication_admin.get_peer_config(@peer_id).getTableCFsMap()
assert_tablecfs_equal(table_cfs, table_cfs_map)
# cleanup for future tests
replication_admin.remove_peer(@peer_id)
end
define_test "append_peer_tableCFs: works with table-cfs map" do
cluster_key = "zk4,zk5,zk6:11000:/hbase-test"
args = { CLUSTER_KEY => cluster_key }
replication_admin.add_peer(@peer_id, args)
assert_equal(1, replication_admin.list_peers.length)
assert(replication_admin.list_peers.key?(@peer_id))
assert_equal(cluster_key, replication_admin.list_peers.fetch(@peer_id).get_cluster_key)
table_cfs = { "table1" => [], "ns2:table2" => ["cf1"] }
replication_admin.append_peer_tableCFs(@peer_id, table_cfs)
table_cfs_map = replication_admin.get_peer_config(@peer_id).getTableCFsMap()
assert_tablecfs_equal(table_cfs, table_cfs_map)
table_cfs = { "table1" => [], "ns2:table2" => ["cf1"], "ns3:table3" => ["cf1", "cf2"] }
replication_admin.append_peer_tableCFs(@peer_id, { "ns3:table3" => ["cf1", "cf2"] })
table_cfs_map = replication_admin.get_peer_config(@peer_id).getTableCFsMap()
assert_tablecfs_equal(table_cfs, table_cfs_map)
# cleanup for future tests
replication_admin.remove_peer(@peer_id)
end
define_test "remove_peer_tableCFs: works with table-cfs map" do
cluster_key = "zk4,zk5,zk6:11000:/hbase-test"
table_cfs = { "table1" => [], "ns2:table2" => ["cf1"], "ns3:table3" => ["cf1", "cf2"] }
args = { CLUSTER_KEY => cluster_key, TABLE_CFS => table_cfs }
replication_admin.add_peer(@peer_id, args)
assert_equal(1, replication_admin.list_peers.length)
assert(replication_admin.list_peers.key?(@peer_id))
assert_equal(cluster_key, replication_admin.list_peers.fetch(@peer_id).get_cluster_key)
table_cfs = { "table1" => [], "ns2:table2" => ["cf1"] }
replication_admin.remove_peer_tableCFs(@peer_id, { "ns3:table3" => ["cf1", "cf2"] })
table_cfs_map = replication_admin.get_peer_config(@peer_id).getTableCFsMap()
assert_tablecfs_equal(table_cfs, table_cfs_map)
# cleanup for future tests
replication_admin.remove_peer(@peer_id)
end
define_test "get_peer_config: works with simple clusterKey peer" do
cluster_key = "localhost:2181:/hbase-test"
args = { CLUSTER_KEY => cluster_key }
@ -180,8 +214,8 @@ module Hbase
config_params = { "config1" => "value1", "config2" => "value2" }
args = { CLUSTER_KEY => cluster_key, ENDPOINT_CLASSNAME => repl_impl,
CONFIG => config_params }
command(:add_peer, @peer_id, args)
peer_config = command(:get_peer_config, @peer_id)
replication_admin.add_peer(@peer_id, args)
peer_config = replication_admin.get_peer_config(@peer_id)
assert_equal(cluster_key, peer_config.get_cluster_key)
assert_equal(repl_impl, peer_config.get_replication_endpoint_impl)
assert_equal(2, peer_config.get_configuration.size)