HBASE-13153 Bulk Loaded HFile Replication (Ashish Singhi)
This commit is contained in:
parent
9647fee3f0
commit
26ac60b03f
|
@ -47,6 +47,7 @@ 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 org.apache.zookeeper.KeeperException.NoNodeException;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
|
||||
|
@ -122,6 +123,19 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
|
|||
checkQueuesDeleted(id);
|
||||
|
||||
ZKUtil.createWithParents(this.zookeeper, this.peersZNode);
|
||||
|
||||
// If only bulk load hfile replication is enabled then add peerId node to hfile-refs node
|
||||
if (replicationForBulkLoadEnabled) {
|
||||
try {
|
||||
String peerId = ZKUtil.joinZNode(this.hfileRefsZNode, id);
|
||||
LOG.info("Adding peer " + peerId + " to hfile reference queue.");
|
||||
ZKUtil.createWithParents(this.zookeeper, peerId);
|
||||
} catch (KeeperException e) {
|
||||
throw new ReplicationException("Failed to add peer with id=" + id
|
||||
+ ", node under hfile references node.", e);
|
||||
}
|
||||
}
|
||||
|
||||
List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>();
|
||||
ZKUtilOp op1 = ZKUtilOp.createAndFailSilent(ZKUtil.joinZNode(this.peersZNode, id),
|
||||
toByteArray(peerConfig));
|
||||
|
@ -151,6 +165,16 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
|
|||
+ " because that id does not exist.");
|
||||
}
|
||||
ZKUtil.deleteNodeRecursively(this.zookeeper, ZKUtil.joinZNode(this.peersZNode, id));
|
||||
// Delete peerId node from hfile-refs node irrespective of whether bulk loaded hfile
|
||||
// replication is enabled or not
|
||||
|
||||
String peerId = ZKUtil.joinZNode(this.hfileRefsZNode, id);
|
||||
try {
|
||||
LOG.info("Removing peer " + peerId + " from hfile reference queue.");
|
||||
ZKUtil.deleteNodeRecursively(this.zookeeper, peerId);
|
||||
} catch (NoNodeException e) {
|
||||
LOG.info("Did not find node " + peerId + " to delete.", e);
|
||||
}
|
||||
} catch (KeeperException e) {
|
||||
throw new ReplicationException("Could not remove peer with id=" + id, e);
|
||||
}
|
||||
|
|
|
@ -26,7 +26,8 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
|
||||
/**
|
||||
* This provides an interface for maintaining a region server's replication queues. These queues
|
||||
* keep track of the WALs that still need to be replicated to remote clusters.
|
||||
* keep track of the WALs and HFile references (if hbase.replication.bulkload.enabled is enabled)
|
||||
* that still need to be replicated to remote clusters.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public interface ReplicationQueues {
|
||||
|
@ -113,4 +114,26 @@ public interface ReplicationQueues {
|
|||
* @return if this is this rs's znode
|
||||
*/
|
||||
boolean isThisOurZnode(String znode);
|
||||
|
||||
/**
|
||||
* Add a peer to hfile reference queue if peer does not exist.
|
||||
* @param peerId peer cluster id to be added
|
||||
* @throws ReplicationException if fails to add a peer id to hfile reference queue
|
||||
*/
|
||||
void addPeerToHFileRefs(String peerId) throws ReplicationException;
|
||||
|
||||
/**
|
||||
* Add new hfile references to the queue.
|
||||
* @param peerId peer cluster id to which the hfiles need to be replicated
|
||||
* @param files list of hfile references to be added
|
||||
* @throws ReplicationException if fails to add a hfile reference
|
||||
*/
|
||||
void addHFileRefs(String peerId, List<String> files) throws ReplicationException;
|
||||
|
||||
/**
|
||||
* Remove hfile references from the queue.
|
||||
* @param peerId peer cluster id from which this hfile references needs to be removed
|
||||
* @param files list of hfile references to be removed
|
||||
*/
|
||||
void removeHFileRefs(String peerId, List<String> files);
|
||||
}
|
||||
|
|
|
@ -25,7 +25,8 @@ import org.apache.zookeeper.KeeperException;
|
|||
|
||||
/**
|
||||
* This provides an interface for clients of replication to view replication queues. These queues
|
||||
* keep track of the WALs that still need to be replicated to remote clusters.
|
||||
* keep track of the sources(WALs/HFile references) that still need to be replicated to remote
|
||||
* clusters.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public interface ReplicationQueuesClient {
|
||||
|
@ -65,4 +66,26 @@ public interface ReplicationQueuesClient {
|
|||
* @return cversion of replication rs node
|
||||
*/
|
||||
int getQueuesZNodeCversion() throws KeeperException;
|
||||
|
||||
/**
|
||||
* Get the change version number of replication hfile references node. This can be used as
|
||||
* optimistic locking to get a consistent snapshot of the replication queues of hfile references.
|
||||
* @return change version number of hfile references node
|
||||
*/
|
||||
int getHFileRefsNodeChangeVersion() throws KeeperException;
|
||||
|
||||
/**
|
||||
* Get list of all peers from hfile reference queue.
|
||||
* @return a list of peer ids
|
||||
* @throws KeeperException zookeeper exception
|
||||
*/
|
||||
List<String> getAllPeersFromHFileRefsQueue() throws KeeperException;
|
||||
|
||||
/**
|
||||
* Get a list of all hfile references in the given peer.
|
||||
* @param peerId a String that identifies the peer
|
||||
* @return a list of hfile references, null if not found any
|
||||
* @throws KeeperException zookeeper exception
|
||||
*/
|
||||
List<String> getReplicableHFiles(String peerId) throws KeeperException;
|
||||
}
|
||||
|
|
|
@ -84,4 +84,41 @@ public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implem
|
|||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getHFileRefsNodeChangeVersion() throws KeeperException {
|
||||
Stat stat = new Stat();
|
||||
try {
|
||||
ZKUtil.getDataNoWatch(this.zookeeper, this.hfileRefsZNode, stat);
|
||||
} catch (KeeperException e) {
|
||||
this.abortable.abort("Failed to get stat of replication hfile references node.", e);
|
||||
throw e;
|
||||
}
|
||||
return stat.getCversion();
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getAllPeersFromHFileRefsQueue() throws KeeperException {
|
||||
List<String> result = null;
|
||||
try {
|
||||
result = ZKUtil.listChildrenNoWatch(this.zookeeper, this.hfileRefsZNode);
|
||||
} catch (KeeperException e) {
|
||||
this.abortable.abort("Failed to get list of all peers in hfile references node.", e);
|
||||
throw e;
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getReplicableHFiles(String peerId) throws KeeperException {
|
||||
String znode = ZKUtil.joinZNode(this.hfileRefsZNode, peerId);
|
||||
List<String> result = null;
|
||||
try {
|
||||
result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
|
||||
} catch (KeeperException e) {
|
||||
this.abortable.abort("Failed to get list of hfile references for peerId=" + peerId, e);
|
||||
throw e;
|
||||
}
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -84,6 +84,15 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
|
|||
} catch (KeeperException e) {
|
||||
throw new ReplicationException("Could not initialize replication queues.", e);
|
||||
}
|
||||
// If only bulk load hfile replication is enabled then create the hfile-refs znode
|
||||
if (replicationForBulkLoadEnabled) {
|
||||
try {
|
||||
ZKUtil.createWithParents(this.zookeeper, this.hfileRefsZNode);
|
||||
} catch (KeeperException e) {
|
||||
throw new ReplicationException("Could not initialize hfile references replication queue.",
|
||||
e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -431,4 +440,65 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
|
|||
ZooKeeperProtos.ReplicationLock.newBuilder().setLockOwner(lockOwner).build().toByteArray();
|
||||
return ProtobufUtil.prependPBMagic(bytes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addHFileRefs(String peerId, List<String> files) throws ReplicationException {
|
||||
String peerZnode = ZKUtil.joinZNode(this.hfileRefsZNode, peerId);
|
||||
boolean debugEnabled = LOG.isDebugEnabled();
|
||||
if (debugEnabled) {
|
||||
LOG.debug("Adding hfile references " + files + " in queue " + peerZnode);
|
||||
}
|
||||
List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>();
|
||||
int size = files.size();
|
||||
for (int i = 0; i < size; i++) {
|
||||
listOfOps.add(ZKUtilOp.createAndFailSilent(ZKUtil.joinZNode(peerZnode, files.get(i)),
|
||||
HConstants.EMPTY_BYTE_ARRAY));
|
||||
}
|
||||
if (debugEnabled) {
|
||||
LOG.debug(" The multi list size for adding hfile references in zk for node " + peerZnode
|
||||
+ " is " + listOfOps.size());
|
||||
}
|
||||
try {
|
||||
ZKUtil.multiOrSequential(this.zookeeper, listOfOps, true);
|
||||
} catch (KeeperException e) {
|
||||
throw new ReplicationException("Failed to create hfile reference znode=" + e.getPath(), e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeHFileRefs(String peerId, List<String> files) {
|
||||
String peerZnode = ZKUtil.joinZNode(this.hfileRefsZNode, peerId);
|
||||
boolean debugEnabled = LOG.isDebugEnabled();
|
||||
if (debugEnabled) {
|
||||
LOG.debug("Removing hfile references " + files + " from queue " + peerZnode);
|
||||
}
|
||||
List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>();
|
||||
int size = files.size();
|
||||
for (int i = 0; i < size; i++) {
|
||||
listOfOps.add(ZKUtilOp.deleteNodeFailSilent(ZKUtil.joinZNode(peerZnode, files.get(i))));
|
||||
}
|
||||
if (debugEnabled) {
|
||||
LOG.debug(" The multi list size for removing hfile references in zk for node " + peerZnode
|
||||
+ " is " + listOfOps.size());
|
||||
}
|
||||
try {
|
||||
ZKUtil.multiOrSequential(this.zookeeper, listOfOps, true);
|
||||
} catch (KeeperException e) {
|
||||
LOG.error("Failed to remove hfile reference znode=" + e.getPath(), e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addPeerToHFileRefs(String peerId) throws ReplicationException {
|
||||
String peerZnode = ZKUtil.joinZNode(this.hfileRefsZNode, peerId);
|
||||
try {
|
||||
if (ZKUtil.checkExists(this.zookeeper, peerZnode) == -1) {
|
||||
LOG.info("Adding peer " + peerId + " to hfile reference queue.");
|
||||
ZKUtil.createWithParents(this.zookeeper, peerZnode);
|
||||
}
|
||||
} catch (KeeperException e) {
|
||||
throw new ReplicationException("Failed to add peer " + peerId + " to hfile reference queue.",
|
||||
e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,9 +20,10 @@ package org.apache.hadoop.hbase.replication;
|
|||
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Abortable;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKConfig;
|
||||
|
@ -48,32 +49,43 @@ 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 queues of hfile references to be replicated */
|
||||
protected final String hfileRefsZNode;
|
||||
/** The cluster key of the local cluster */
|
||||
protected final String ourClusterKey;
|
||||
protected final ZooKeeperWatcher zookeeper;
|
||||
protected final Configuration conf;
|
||||
protected final Abortable abortable;
|
||||
protected final boolean replicationForBulkLoadEnabled;
|
||||
|
||||
// Public for testing
|
||||
public static final byte[] ENABLED_ZNODE_BYTES =
|
||||
toByteArray(ZooKeeperProtos.ReplicationState.State.ENABLED);
|
||||
public static final byte[] DISABLED_ZNODE_BYTES =
|
||||
toByteArray(ZooKeeperProtos.ReplicationState.State.DISABLED);
|
||||
public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_KEY =
|
||||
"zookeeper.znode.replication.hfile.refs";
|
||||
public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT = "hfile-refs";
|
||||
|
||||
public ReplicationStateZKBase(ZooKeeperWatcher zookeeper, Configuration conf,
|
||||
Abortable abortable) {
|
||||
this.zookeeper = zookeeper;
|
||||
this.conf = conf;
|
||||
this.abortable = abortable;
|
||||
this.replicationForBulkLoadEnabled = conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
|
||||
HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT);
|
||||
|
||||
String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication");
|
||||
String peersZNodeName = conf.get("zookeeper.znode.replication.peers", "peers");
|
||||
String queuesZNodeName = conf.get("zookeeper.znode.replication.rs", "rs");
|
||||
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.ourClusterKey = ZKConfig.getZooKeeperClusterKey(this.conf);
|
||||
this.replicationZNode = ZKUtil.joinZNode(this.zookeeper.baseZNode, replicationZNodeName);
|
||||
this.peersZNode = ZKUtil.joinZNode(replicationZNode, peersZNodeName);
|
||||
this.queuesZNode = ZKUtil.joinZNode(replicationZNode, queuesZNodeName);
|
||||
this.hfileRefsZNode = ZKUtil.joinZNode(replicationZNode, hfileRefsZNodeName);
|
||||
}
|
||||
|
||||
public List<String> getListOfReplicators() {
|
||||
|
|
|
@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
|
||||
import org.apache.hadoop.hbase.security.Superusers;
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -1797,6 +1798,27 @@ public class ZKUtil {
|
|||
} else if (child.equals(zkw.getConfiguration().
|
||||
get("zookeeper.znode.replication.rs", "rs"))) {
|
||||
appendRSZnodes(zkw, znode, sb);
|
||||
} else if (child.equals(zkw.getConfiguration().get(
|
||||
ReplicationStateZKBase.ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_KEY,
|
||||
ReplicationStateZKBase.ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT))) {
|
||||
appendHFileRefsZnodes(zkw, znode, sb);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static void appendHFileRefsZnodes(ZooKeeperWatcher zkw, String hfileRefsZnode,
|
||||
StringBuilder sb) throws KeeperException {
|
||||
sb.append("\n").append(hfileRefsZnode).append(": ");
|
||||
for (String peerIdZnode : ZKUtil.listChildrenNoWatch(zkw, hfileRefsZnode)) {
|
||||
String znodeToProcess = ZKUtil.joinZNode(hfileRefsZnode, peerIdZnode);
|
||||
sb.append("\n").append(znodeToProcess).append(": ");
|
||||
List<String> peerHFileRefsZnodes = ZKUtil.listChildrenNoWatch(zkw, znodeToProcess);
|
||||
int size = peerHFileRefsZnodes.size();
|
||||
for (int i = 0; i < size; i++) {
|
||||
sb.append(peerHFileRefsZnodes.get(i));
|
||||
if (i != size - 1) {
|
||||
sb.append(", ");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -850,6 +850,18 @@ public final class HConstants {
|
|||
REPLICATION_SINK_SERVICE_CLASSNAME = "hbase.replication.sink.service";
|
||||
public static final String REPLICATION_SERVICE_CLASSNAME_DEFAULT =
|
||||
"org.apache.hadoop.hbase.replication.regionserver.Replication";
|
||||
public static final String REPLICATION_BULKLOAD_ENABLE_KEY = "hbase.replication.bulkload.enabled";
|
||||
public static final boolean REPLICATION_BULKLOAD_ENABLE_DEFAULT = false;
|
||||
/** Replication cluster id of source cluster which uniquely identifies itself with peer cluster */
|
||||
public static final String REPLICATION_CLUSTER_ID = "hbase.replication.cluster.id";
|
||||
/**
|
||||
* Directory where the source cluster file system client configuration are placed which is used by
|
||||
* sink cluster to copy HFiles from source cluster file system
|
||||
*/
|
||||
public static final String REPLICATION_CONF_DIR = "hbase.replication.conf.dir";
|
||||
|
||||
/** Maximum time to retry for a failed bulk load request */
|
||||
public static final String BULKLOAD_MAX_RETRIES_NUMBER = "hbase.bulkload.retries.number";
|
||||
|
||||
/** HBCK special code name used as server name when manipulating ZK nodes */
|
||||
public static final String HBCK_CODE_NAME = "HBCKServerName";
|
||||
|
|
|
@ -22,9 +22,11 @@ public interface MetricsReplicationSinkSource {
|
|||
public static final String SINK_AGE_OF_LAST_APPLIED_OP = "sink.ageOfLastAppliedOp";
|
||||
public static final String SINK_APPLIED_BATCHES = "sink.appliedBatches";
|
||||
public static final String SINK_APPLIED_OPS = "sink.appliedOps";
|
||||
public static final String SINK_APPLIED_HFILES = "sink.appliedHFiles";
|
||||
|
||||
void setLastAppliedOpAge(long age);
|
||||
void incrAppliedBatches(long batches);
|
||||
void incrAppliedOps(long batchsize);
|
||||
long getLastAppliedOpAge();
|
||||
void incrAppliedHFiles(long hfileSize);
|
||||
}
|
||||
|
|
|
@ -32,6 +32,9 @@ public interface MetricsReplicationSourceSource {
|
|||
|
||||
public static final String SOURCE_LOG_EDITS_FILTERED = "source.logEditsFiltered";
|
||||
|
||||
public static final String SOURCE_SHIPPED_HFILES = "source.shippedHFiles";
|
||||
public static final String SOURCE_SIZE_OF_HFILE_REFS_QUEUE = "source.sizeOfHFileRefsQueue";
|
||||
|
||||
void setLastShippedAge(long age);
|
||||
void setSizeOfLogQueue(int size);
|
||||
void incrSizeOfLogQueue(int size);
|
||||
|
@ -44,4 +47,7 @@ public interface MetricsReplicationSourceSource {
|
|||
void incrLogReadInEdits(long size);
|
||||
void clear();
|
||||
long getLastShippedAge();
|
||||
void incrHFilesShipped(long hfiles);
|
||||
void incrSizeOfHFileRefsQueue(long size);
|
||||
void decrSizeOfHFileRefsQueue(long size);
|
||||
}
|
||||
|
|
|
@ -32,6 +32,8 @@ public class MetricsReplicationGlobalSourceSource implements MetricsReplicationS
|
|||
private final MutableCounterLong shippedOpsCounter;
|
||||
private final MutableCounterLong shippedKBsCounter;
|
||||
private final MutableCounterLong logReadInBytesCounter;
|
||||
private final MutableCounterLong shippedHFilesCounter;
|
||||
private final MutableGaugeLong sizeOfHFileRefsQueueGauge;
|
||||
|
||||
public MetricsReplicationGlobalSourceSource(MetricsReplicationSourceImpl rms) {
|
||||
this.rms = rms;
|
||||
|
@ -51,6 +53,11 @@ public class MetricsReplicationGlobalSourceSource implements MetricsReplicationS
|
|||
logReadInEditsCounter = rms.getMetricsRegistry().getLongCounter(SOURCE_LOG_READ_IN_EDITS, 0L);
|
||||
|
||||
logEditsFilteredCounter = rms.getMetricsRegistry().getLongCounter(SOURCE_LOG_EDITS_FILTERED, 0L);
|
||||
|
||||
shippedHFilesCounter = rms.getMetricsRegistry().getLongCounter(SOURCE_SHIPPED_HFILES, 0L);
|
||||
|
||||
sizeOfHFileRefsQueueGauge =
|
||||
rms.getMetricsRegistry().getLongGauge(SOURCE_SIZE_OF_HFILE_REFS_QUEUE, 0L);
|
||||
}
|
||||
|
||||
@Override public void setLastShippedAge(long age) {
|
||||
|
@ -100,4 +107,18 @@ public class MetricsReplicationGlobalSourceSource implements MetricsReplicationS
|
|||
public long getLastShippedAge() {
|
||||
return ageOfLastShippedOpGauge.value();
|
||||
}
|
||||
|
||||
@Override public void incrHFilesShipped(long hfiles) {
|
||||
shippedHFilesCounter.incr(hfiles);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void incrSizeOfHFileRefsQueue(long size) {
|
||||
sizeOfHFileRefsQueueGauge.incr(size);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void decrSizeOfHFileRefsQueue(long size) {
|
||||
sizeOfHFileRefsQueueGauge.decr(size);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,11 +26,13 @@ public class MetricsReplicationSinkSourceImpl implements MetricsReplicationSinkS
|
|||
private final MutableGaugeLong ageGauge;
|
||||
private final MutableCounterLong batchesCounter;
|
||||
private final MutableCounterLong opsCounter;
|
||||
private final MutableCounterLong hfilesCounter;
|
||||
|
||||
public MetricsReplicationSinkSourceImpl(MetricsReplicationSourceImpl rms) {
|
||||
ageGauge = rms.getMetricsRegistry().getLongGauge(SINK_AGE_OF_LAST_APPLIED_OP, 0L);
|
||||
batchesCounter = rms.getMetricsRegistry().getLongCounter(SINK_APPLIED_BATCHES, 0L);
|
||||
opsCounter = rms.getMetricsRegistry().getLongCounter(SINK_APPLIED_OPS, 0L);
|
||||
hfilesCounter = rms.getMetricsRegistry().getLongCounter(SINK_APPLIED_HFILES, 0L);
|
||||
}
|
||||
|
||||
@Override public void setLastAppliedOpAge(long age) {
|
||||
|
@ -49,4 +51,9 @@ public class MetricsReplicationSinkSourceImpl implements MetricsReplicationSinkS
|
|||
public long getLastAppliedOpAge() {
|
||||
return ageGauge.value();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void incrAppliedHFiles(long hfiles) {
|
||||
hfilesCounter.incr(hfiles);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,6 +32,8 @@ public class MetricsReplicationSourceSourceImpl implements MetricsReplicationSou
|
|||
private final String shippedOpsKey;
|
||||
private final String shippedKBsKey;
|
||||
private final String logReadInBytesKey;
|
||||
private final String shippedHFilesKey;
|
||||
private final String sizeOfHFileRefsQueueKey;
|
||||
|
||||
private final MutableGaugeLong ageOfLastShippedOpGauge;
|
||||
private final MutableGaugeLong sizeOfLogQueueGauge;
|
||||
|
@ -41,6 +43,8 @@ public class MetricsReplicationSourceSourceImpl implements MetricsReplicationSou
|
|||
private final MutableCounterLong shippedOpsCounter;
|
||||
private final MutableCounterLong shippedKBsCounter;
|
||||
private final MutableCounterLong logReadInBytesCounter;
|
||||
private final MutableCounterLong shippedHFilesCounter;
|
||||
private final MutableGaugeLong sizeOfHFileRefsQueueGauge;
|
||||
|
||||
public MetricsReplicationSourceSourceImpl(MetricsReplicationSourceImpl rms, String id) {
|
||||
this.rms = rms;
|
||||
|
@ -69,6 +73,12 @@ public class MetricsReplicationSourceSourceImpl implements MetricsReplicationSou
|
|||
|
||||
logEditsFilteredKey = "source." + id + ".logEditsFiltered";
|
||||
logEditsFilteredCounter = rms.getMetricsRegistry().getLongCounter(logEditsFilteredKey, 0L);
|
||||
|
||||
shippedHFilesKey = "source." + this.id + ".shippedHFiles";
|
||||
shippedHFilesCounter = rms.getMetricsRegistry().getLongCounter(shippedHFilesKey, 0L);
|
||||
|
||||
sizeOfHFileRefsQueueKey = "source." + id + ".sizeOfHFileRefsQueue";
|
||||
sizeOfHFileRefsQueueGauge = rms.getMetricsRegistry().getLongGauge(sizeOfHFileRefsQueueKey, 0L);
|
||||
}
|
||||
|
||||
@Override public void setLastShippedAge(long age) {
|
||||
|
@ -124,10 +134,28 @@ public class MetricsReplicationSourceSourceImpl implements MetricsReplicationSou
|
|||
rms.removeMetric(logReadInEditsKey);
|
||||
|
||||
rms.removeMetric(logEditsFilteredKey);
|
||||
|
||||
rms.removeMetric(shippedHFilesKey);
|
||||
rms.removeMetric(sizeOfHFileRefsQueueKey);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLastShippedAge() {
|
||||
return ageOfLastShippedOpGauge.value();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void incrHFilesShipped(long hfiles) {
|
||||
shippedHFilesCounter.incr(hfiles);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void incrSizeOfHFileRefsQueue(long size) {
|
||||
sizeOfHFileRefsQueueGauge.incr(size);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void decrSizeOfHFileRefsQueue(long size) {
|
||||
sizeOfHFileRefsQueueGauge.decr(size);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -16896,6 +16896,51 @@ public final class AdminProtos {
|
|||
*/
|
||||
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntryOrBuilder getEntryOrBuilder(
|
||||
int index);
|
||||
|
||||
// optional string replicationClusterId = 2;
|
||||
/**
|
||||
* <code>optional string replicationClusterId = 2;</code>
|
||||
*/
|
||||
boolean hasReplicationClusterId();
|
||||
/**
|
||||
* <code>optional string replicationClusterId = 2;</code>
|
||||
*/
|
||||
java.lang.String getReplicationClusterId();
|
||||
/**
|
||||
* <code>optional string replicationClusterId = 2;</code>
|
||||
*/
|
||||
com.google.protobuf.ByteString
|
||||
getReplicationClusterIdBytes();
|
||||
|
||||
// optional string sourceBaseNamespaceDirPath = 3;
|
||||
/**
|
||||
* <code>optional string sourceBaseNamespaceDirPath = 3;</code>
|
||||
*/
|
||||
boolean hasSourceBaseNamespaceDirPath();
|
||||
/**
|
||||
* <code>optional string sourceBaseNamespaceDirPath = 3;</code>
|
||||
*/
|
||||
java.lang.String getSourceBaseNamespaceDirPath();
|
||||
/**
|
||||
* <code>optional string sourceBaseNamespaceDirPath = 3;</code>
|
||||
*/
|
||||
com.google.protobuf.ByteString
|
||||
getSourceBaseNamespaceDirPathBytes();
|
||||
|
||||
// optional string sourceHFileArchiveDirPath = 4;
|
||||
/**
|
||||
* <code>optional string sourceHFileArchiveDirPath = 4;</code>
|
||||
*/
|
||||
boolean hasSourceHFileArchiveDirPath();
|
||||
/**
|
||||
* <code>optional string sourceHFileArchiveDirPath = 4;</code>
|
||||
*/
|
||||
java.lang.String getSourceHFileArchiveDirPath();
|
||||
/**
|
||||
* <code>optional string sourceHFileArchiveDirPath = 4;</code>
|
||||
*/
|
||||
com.google.protobuf.ByteString
|
||||
getSourceHFileArchiveDirPathBytes();
|
||||
}
|
||||
/**
|
||||
* Protobuf type {@code hbase.pb.ReplicateWALEntryRequest}
|
||||
|
@ -16963,6 +17008,21 @@ public final class AdminProtos {
|
|||
entry_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry.PARSER, extensionRegistry));
|
||||
break;
|
||||
}
|
||||
case 18: {
|
||||
bitField0_ |= 0x00000001;
|
||||
replicationClusterId_ = input.readBytes();
|
||||
break;
|
||||
}
|
||||
case 26: {
|
||||
bitField0_ |= 0x00000002;
|
||||
sourceBaseNamespaceDirPath_ = input.readBytes();
|
||||
break;
|
||||
}
|
||||
case 34: {
|
||||
bitField0_ |= 0x00000004;
|
||||
sourceHFileArchiveDirPath_ = input.readBytes();
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
|
||||
|
@ -17005,6 +17065,7 @@ public final class AdminProtos {
|
|||
return PARSER;
|
||||
}
|
||||
|
||||
private int bitField0_;
|
||||
// repeated .hbase.pb.WALEntry entry = 1;
|
||||
public static final int ENTRY_FIELD_NUMBER = 1;
|
||||
private java.util.List<org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry> entry_;
|
||||
|
@ -17041,8 +17102,140 @@ public final class AdminProtos {
|
|||
return entry_.get(index);
|
||||
}
|
||||
|
||||
// optional string replicationClusterId = 2;
|
||||
public static final int REPLICATIONCLUSTERID_FIELD_NUMBER = 2;
|
||||
private java.lang.Object replicationClusterId_;
|
||||
/**
|
||||
* <code>optional string replicationClusterId = 2;</code>
|
||||
*/
|
||||
public boolean hasReplicationClusterId() {
|
||||
return ((bitField0_ & 0x00000001) == 0x00000001);
|
||||
}
|
||||
/**
|
||||
* <code>optional string replicationClusterId = 2;</code>
|
||||
*/
|
||||
public java.lang.String getReplicationClusterId() {
|
||||
java.lang.Object ref = replicationClusterId_;
|
||||
if (ref instanceof java.lang.String) {
|
||||
return (java.lang.String) ref;
|
||||
} else {
|
||||
com.google.protobuf.ByteString bs =
|
||||
(com.google.protobuf.ByteString) ref;
|
||||
java.lang.String s = bs.toStringUtf8();
|
||||
if (bs.isValidUtf8()) {
|
||||
replicationClusterId_ = s;
|
||||
}
|
||||
return s;
|
||||
}
|
||||
}
|
||||
/**
|
||||
* <code>optional string replicationClusterId = 2;</code>
|
||||
*/
|
||||
public com.google.protobuf.ByteString
|
||||
getReplicationClusterIdBytes() {
|
||||
java.lang.Object ref = replicationClusterId_;
|
||||
if (ref instanceof java.lang.String) {
|
||||
com.google.protobuf.ByteString b =
|
||||
com.google.protobuf.ByteString.copyFromUtf8(
|
||||
(java.lang.String) ref);
|
||||
replicationClusterId_ = b;
|
||||
return b;
|
||||
} else {
|
||||
return (com.google.protobuf.ByteString) ref;
|
||||
}
|
||||
}
|
||||
|
||||
// optional string sourceBaseNamespaceDirPath = 3;
|
||||
public static final int SOURCEBASENAMESPACEDIRPATH_FIELD_NUMBER = 3;
|
||||
private java.lang.Object sourceBaseNamespaceDirPath_;
|
||||
/**
|
||||
* <code>optional string sourceBaseNamespaceDirPath = 3;</code>
|
||||
*/
|
||||
public boolean hasSourceBaseNamespaceDirPath() {
|
||||
return ((bitField0_ & 0x00000002) == 0x00000002);
|
||||
}
|
||||
/**
|
||||
* <code>optional string sourceBaseNamespaceDirPath = 3;</code>
|
||||
*/
|
||||
public java.lang.String getSourceBaseNamespaceDirPath() {
|
||||
java.lang.Object ref = sourceBaseNamespaceDirPath_;
|
||||
if (ref instanceof java.lang.String) {
|
||||
return (java.lang.String) ref;
|
||||
} else {
|
||||
com.google.protobuf.ByteString bs =
|
||||
(com.google.protobuf.ByteString) ref;
|
||||
java.lang.String s = bs.toStringUtf8();
|
||||
if (bs.isValidUtf8()) {
|
||||
sourceBaseNamespaceDirPath_ = s;
|
||||
}
|
||||
return s;
|
||||
}
|
||||
}
|
||||
/**
|
||||
* <code>optional string sourceBaseNamespaceDirPath = 3;</code>
|
||||
*/
|
||||
public com.google.protobuf.ByteString
|
||||
getSourceBaseNamespaceDirPathBytes() {
|
||||
java.lang.Object ref = sourceBaseNamespaceDirPath_;
|
||||
if (ref instanceof java.lang.String) {
|
||||
com.google.protobuf.ByteString b =
|
||||
com.google.protobuf.ByteString.copyFromUtf8(
|
||||
(java.lang.String) ref);
|
||||
sourceBaseNamespaceDirPath_ = b;
|
||||
return b;
|
||||
} else {
|
||||
return (com.google.protobuf.ByteString) ref;
|
||||
}
|
||||
}
|
||||
|
||||
// optional string sourceHFileArchiveDirPath = 4;
|
||||
public static final int SOURCEHFILEARCHIVEDIRPATH_FIELD_NUMBER = 4;
|
||||
private java.lang.Object sourceHFileArchiveDirPath_;
|
||||
/**
|
||||
* <code>optional string sourceHFileArchiveDirPath = 4;</code>
|
||||
*/
|
||||
public boolean hasSourceHFileArchiveDirPath() {
|
||||
return ((bitField0_ & 0x00000004) == 0x00000004);
|
||||
}
|
||||
/**
|
||||
* <code>optional string sourceHFileArchiveDirPath = 4;</code>
|
||||
*/
|
||||
public java.lang.String getSourceHFileArchiveDirPath() {
|
||||
java.lang.Object ref = sourceHFileArchiveDirPath_;
|
||||
if (ref instanceof java.lang.String) {
|
||||
return (java.lang.String) ref;
|
||||
} else {
|
||||
com.google.protobuf.ByteString bs =
|
||||
(com.google.protobuf.ByteString) ref;
|
||||
java.lang.String s = bs.toStringUtf8();
|
||||
if (bs.isValidUtf8()) {
|
||||
sourceHFileArchiveDirPath_ = s;
|
||||
}
|
||||
return s;
|
||||
}
|
||||
}
|
||||
/**
|
||||
* <code>optional string sourceHFileArchiveDirPath = 4;</code>
|
||||
*/
|
||||
public com.google.protobuf.ByteString
|
||||
getSourceHFileArchiveDirPathBytes() {
|
||||
java.lang.Object ref = sourceHFileArchiveDirPath_;
|
||||
if (ref instanceof java.lang.String) {
|
||||
com.google.protobuf.ByteString b =
|
||||
com.google.protobuf.ByteString.copyFromUtf8(
|
||||
(java.lang.String) ref);
|
||||
sourceHFileArchiveDirPath_ = b;
|
||||
return b;
|
||||
} else {
|
||||
return (com.google.protobuf.ByteString) ref;
|
||||
}
|
||||
}
|
||||
|
||||
private void initFields() {
|
||||
entry_ = java.util.Collections.emptyList();
|
||||
replicationClusterId_ = "";
|
||||
sourceBaseNamespaceDirPath_ = "";
|
||||
sourceHFileArchiveDirPath_ = "";
|
||||
}
|
||||
private byte memoizedIsInitialized = -1;
|
||||
public final boolean isInitialized() {
|
||||
|
@ -17065,6 +17258,15 @@ public final class AdminProtos {
|
|||
for (int i = 0; i < entry_.size(); i++) {
|
||||
output.writeMessage(1, entry_.get(i));
|
||||
}
|
||||
if (((bitField0_ & 0x00000001) == 0x00000001)) {
|
||||
output.writeBytes(2, getReplicationClusterIdBytes());
|
||||
}
|
||||
if (((bitField0_ & 0x00000002) == 0x00000002)) {
|
||||
output.writeBytes(3, getSourceBaseNamespaceDirPathBytes());
|
||||
}
|
||||
if (((bitField0_ & 0x00000004) == 0x00000004)) {
|
||||
output.writeBytes(4, getSourceHFileArchiveDirPathBytes());
|
||||
}
|
||||
getUnknownFields().writeTo(output);
|
||||
}
|
||||
|
||||
|
@ -17078,6 +17280,18 @@ public final class AdminProtos {
|
|||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeMessageSize(1, entry_.get(i));
|
||||
}
|
||||
if (((bitField0_ & 0x00000001) == 0x00000001)) {
|
||||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeBytesSize(2, getReplicationClusterIdBytes());
|
||||
}
|
||||
if (((bitField0_ & 0x00000002) == 0x00000002)) {
|
||||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeBytesSize(3, getSourceBaseNamespaceDirPathBytes());
|
||||
}
|
||||
if (((bitField0_ & 0x00000004) == 0x00000004)) {
|
||||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeBytesSize(4, getSourceHFileArchiveDirPathBytes());
|
||||
}
|
||||
size += getUnknownFields().getSerializedSize();
|
||||
memoizedSerializedSize = size;
|
||||
return size;
|
||||
|
@ -17103,6 +17317,21 @@ public final class AdminProtos {
|
|||
boolean result = true;
|
||||
result = result && getEntryList()
|
||||
.equals(other.getEntryList());
|
||||
result = result && (hasReplicationClusterId() == other.hasReplicationClusterId());
|
||||
if (hasReplicationClusterId()) {
|
||||
result = result && getReplicationClusterId()
|
||||
.equals(other.getReplicationClusterId());
|
||||
}
|
||||
result = result && (hasSourceBaseNamespaceDirPath() == other.hasSourceBaseNamespaceDirPath());
|
||||
if (hasSourceBaseNamespaceDirPath()) {
|
||||
result = result && getSourceBaseNamespaceDirPath()
|
||||
.equals(other.getSourceBaseNamespaceDirPath());
|
||||
}
|
||||
result = result && (hasSourceHFileArchiveDirPath() == other.hasSourceHFileArchiveDirPath());
|
||||
if (hasSourceHFileArchiveDirPath()) {
|
||||
result = result && getSourceHFileArchiveDirPath()
|
||||
.equals(other.getSourceHFileArchiveDirPath());
|
||||
}
|
||||
result = result &&
|
||||
getUnknownFields().equals(other.getUnknownFields());
|
||||
return result;
|
||||
|
@ -17120,6 +17349,18 @@ public final class AdminProtos {
|
|||
hash = (37 * hash) + ENTRY_FIELD_NUMBER;
|
||||
hash = (53 * hash) + getEntryList().hashCode();
|
||||
}
|
||||
if (hasReplicationClusterId()) {
|
||||
hash = (37 * hash) + REPLICATIONCLUSTERID_FIELD_NUMBER;
|
||||
hash = (53 * hash) + getReplicationClusterId().hashCode();
|
||||
}
|
||||
if (hasSourceBaseNamespaceDirPath()) {
|
||||
hash = (37 * hash) + SOURCEBASENAMESPACEDIRPATH_FIELD_NUMBER;
|
||||
hash = (53 * hash) + getSourceBaseNamespaceDirPath().hashCode();
|
||||
}
|
||||
if (hasSourceHFileArchiveDirPath()) {
|
||||
hash = (37 * hash) + SOURCEHFILEARCHIVEDIRPATH_FIELD_NUMBER;
|
||||
hash = (53 * hash) + getSourceHFileArchiveDirPath().hashCode();
|
||||
}
|
||||
hash = (29 * hash) + getUnknownFields().hashCode();
|
||||
memoizedHashCode = hash;
|
||||
return hash;
|
||||
|
@ -17243,6 +17484,12 @@ public final class AdminProtos {
|
|||
} else {
|
||||
entryBuilder_.clear();
|
||||
}
|
||||
replicationClusterId_ = "";
|
||||
bitField0_ = (bitField0_ & ~0x00000002);
|
||||
sourceBaseNamespaceDirPath_ = "";
|
||||
bitField0_ = (bitField0_ & ~0x00000004);
|
||||
sourceHFileArchiveDirPath_ = "";
|
||||
bitField0_ = (bitField0_ & ~0x00000008);
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -17270,6 +17517,7 @@ public final class AdminProtos {
|
|||
public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest buildPartial() {
|
||||
org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest result = new org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest(this);
|
||||
int from_bitField0_ = bitField0_;
|
||||
int to_bitField0_ = 0;
|
||||
if (entryBuilder_ == null) {
|
||||
if (((bitField0_ & 0x00000001) == 0x00000001)) {
|
||||
entry_ = java.util.Collections.unmodifiableList(entry_);
|
||||
|
@ -17279,6 +17527,19 @@ public final class AdminProtos {
|
|||
} else {
|
||||
result.entry_ = entryBuilder_.build();
|
||||
}
|
||||
if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
|
||||
to_bitField0_ |= 0x00000001;
|
||||
}
|
||||
result.replicationClusterId_ = replicationClusterId_;
|
||||
if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
|
||||
to_bitField0_ |= 0x00000002;
|
||||
}
|
||||
result.sourceBaseNamespaceDirPath_ = sourceBaseNamespaceDirPath_;
|
||||
if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
|
||||
to_bitField0_ |= 0x00000004;
|
||||
}
|
||||
result.sourceHFileArchiveDirPath_ = sourceHFileArchiveDirPath_;
|
||||
result.bitField0_ = to_bitField0_;
|
||||
onBuilt();
|
||||
return result;
|
||||
}
|
||||
|
@ -17320,6 +17581,21 @@ public final class AdminProtos {
|
|||
}
|
||||
}
|
||||
}
|
||||
if (other.hasReplicationClusterId()) {
|
||||
bitField0_ |= 0x00000002;
|
||||
replicationClusterId_ = other.replicationClusterId_;
|
||||
onChanged();
|
||||
}
|
||||
if (other.hasSourceBaseNamespaceDirPath()) {
|
||||
bitField0_ |= 0x00000004;
|
||||
sourceBaseNamespaceDirPath_ = other.sourceBaseNamespaceDirPath_;
|
||||
onChanged();
|
||||
}
|
||||
if (other.hasSourceHFileArchiveDirPath()) {
|
||||
bitField0_ |= 0x00000008;
|
||||
sourceHFileArchiveDirPath_ = other.sourceHFileArchiveDirPath_;
|
||||
onChanged();
|
||||
}
|
||||
this.mergeUnknownFields(other.getUnknownFields());
|
||||
return this;
|
||||
}
|
||||
|
@ -17593,6 +17869,228 @@ public final class AdminProtos {
|
|||
return entryBuilder_;
|
||||
}
|
||||
|
||||
// optional string replicationClusterId = 2;
|
||||
private java.lang.Object replicationClusterId_ = "";
|
||||
/**
|
||||
* <code>optional string replicationClusterId = 2;</code>
|
||||
*/
|
||||
public boolean hasReplicationClusterId() {
|
||||
return ((bitField0_ & 0x00000002) == 0x00000002);
|
||||
}
|
||||
/**
|
||||
* <code>optional string replicationClusterId = 2;</code>
|
||||
*/
|
||||
public java.lang.String getReplicationClusterId() {
|
||||
java.lang.Object ref = replicationClusterId_;
|
||||
if (!(ref instanceof java.lang.String)) {
|
||||
java.lang.String s = ((com.google.protobuf.ByteString) ref)
|
||||
.toStringUtf8();
|
||||
replicationClusterId_ = s;
|
||||
return s;
|
||||
} else {
|
||||
return (java.lang.String) ref;
|
||||
}
|
||||
}
|
||||
/**
|
||||
* <code>optional string replicationClusterId = 2;</code>
|
||||
*/
|
||||
public com.google.protobuf.ByteString
|
||||
getReplicationClusterIdBytes() {
|
||||
java.lang.Object ref = replicationClusterId_;
|
||||
if (ref instanceof String) {
|
||||
com.google.protobuf.ByteString b =
|
||||
com.google.protobuf.ByteString.copyFromUtf8(
|
||||
(java.lang.String) ref);
|
||||
replicationClusterId_ = b;
|
||||
return b;
|
||||
} else {
|
||||
return (com.google.protobuf.ByteString) ref;
|
||||
}
|
||||
}
|
||||
/**
|
||||
* <code>optional string replicationClusterId = 2;</code>
|
||||
*/
|
||||
public Builder setReplicationClusterId(
|
||||
java.lang.String value) {
|
||||
if (value == null) {
|
||||
throw new NullPointerException();
|
||||
}
|
||||
bitField0_ |= 0x00000002;
|
||||
replicationClusterId_ = value;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
/**
|
||||
* <code>optional string replicationClusterId = 2;</code>
|
||||
*/
|
||||
public Builder clearReplicationClusterId() {
|
||||
bitField0_ = (bitField0_ & ~0x00000002);
|
||||
replicationClusterId_ = getDefaultInstance().getReplicationClusterId();
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
/**
|
||||
* <code>optional string replicationClusterId = 2;</code>
|
||||
*/
|
||||
public Builder setReplicationClusterIdBytes(
|
||||
com.google.protobuf.ByteString value) {
|
||||
if (value == null) {
|
||||
throw new NullPointerException();
|
||||
}
|
||||
bitField0_ |= 0x00000002;
|
||||
replicationClusterId_ = value;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
|
||||
// optional string sourceBaseNamespaceDirPath = 3;
|
||||
private java.lang.Object sourceBaseNamespaceDirPath_ = "";
|
||||
/**
|
||||
* <code>optional string sourceBaseNamespaceDirPath = 3;</code>
|
||||
*/
|
||||
public boolean hasSourceBaseNamespaceDirPath() {
|
||||
return ((bitField0_ & 0x00000004) == 0x00000004);
|
||||
}
|
||||
/**
|
||||
* <code>optional string sourceBaseNamespaceDirPath = 3;</code>
|
||||
*/
|
||||
public java.lang.String getSourceBaseNamespaceDirPath() {
|
||||
java.lang.Object ref = sourceBaseNamespaceDirPath_;
|
||||
if (!(ref instanceof java.lang.String)) {
|
||||
java.lang.String s = ((com.google.protobuf.ByteString) ref)
|
||||
.toStringUtf8();
|
||||
sourceBaseNamespaceDirPath_ = s;
|
||||
return s;
|
||||
} else {
|
||||
return (java.lang.String) ref;
|
||||
}
|
||||
}
|
||||
/**
|
||||
* <code>optional string sourceBaseNamespaceDirPath = 3;</code>
|
||||
*/
|
||||
public com.google.protobuf.ByteString
|
||||
getSourceBaseNamespaceDirPathBytes() {
|
||||
java.lang.Object ref = sourceBaseNamespaceDirPath_;
|
||||
if (ref instanceof String) {
|
||||
com.google.protobuf.ByteString b =
|
||||
com.google.protobuf.ByteString.copyFromUtf8(
|
||||
(java.lang.String) ref);
|
||||
sourceBaseNamespaceDirPath_ = b;
|
||||
return b;
|
||||
} else {
|
||||
return (com.google.protobuf.ByteString) ref;
|
||||
}
|
||||
}
|
||||
/**
|
||||
* <code>optional string sourceBaseNamespaceDirPath = 3;</code>
|
||||
*/
|
||||
public Builder setSourceBaseNamespaceDirPath(
|
||||
java.lang.String value) {
|
||||
if (value == null) {
|
||||
throw new NullPointerException();
|
||||
}
|
||||
bitField0_ |= 0x00000004;
|
||||
sourceBaseNamespaceDirPath_ = value;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
/**
|
||||
* <code>optional string sourceBaseNamespaceDirPath = 3;</code>
|
||||
*/
|
||||
public Builder clearSourceBaseNamespaceDirPath() {
|
||||
bitField0_ = (bitField0_ & ~0x00000004);
|
||||
sourceBaseNamespaceDirPath_ = getDefaultInstance().getSourceBaseNamespaceDirPath();
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
/**
|
||||
* <code>optional string sourceBaseNamespaceDirPath = 3;</code>
|
||||
*/
|
||||
public Builder setSourceBaseNamespaceDirPathBytes(
|
||||
com.google.protobuf.ByteString value) {
|
||||
if (value == null) {
|
||||
throw new NullPointerException();
|
||||
}
|
||||
bitField0_ |= 0x00000004;
|
||||
sourceBaseNamespaceDirPath_ = value;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
|
||||
// optional string sourceHFileArchiveDirPath = 4;
|
||||
private java.lang.Object sourceHFileArchiveDirPath_ = "";
|
||||
/**
|
||||
* <code>optional string sourceHFileArchiveDirPath = 4;</code>
|
||||
*/
|
||||
public boolean hasSourceHFileArchiveDirPath() {
|
||||
return ((bitField0_ & 0x00000008) == 0x00000008);
|
||||
}
|
||||
/**
|
||||
* <code>optional string sourceHFileArchiveDirPath = 4;</code>
|
||||
*/
|
||||
public java.lang.String getSourceHFileArchiveDirPath() {
|
||||
java.lang.Object ref = sourceHFileArchiveDirPath_;
|
||||
if (!(ref instanceof java.lang.String)) {
|
||||
java.lang.String s = ((com.google.protobuf.ByteString) ref)
|
||||
.toStringUtf8();
|
||||
sourceHFileArchiveDirPath_ = s;
|
||||
return s;
|
||||
} else {
|
||||
return (java.lang.String) ref;
|
||||
}
|
||||
}
|
||||
/**
|
||||
* <code>optional string sourceHFileArchiveDirPath = 4;</code>
|
||||
*/
|
||||
public com.google.protobuf.ByteString
|
||||
getSourceHFileArchiveDirPathBytes() {
|
||||
java.lang.Object ref = sourceHFileArchiveDirPath_;
|
||||
if (ref instanceof String) {
|
||||
com.google.protobuf.ByteString b =
|
||||
com.google.protobuf.ByteString.copyFromUtf8(
|
||||
(java.lang.String) ref);
|
||||
sourceHFileArchiveDirPath_ = b;
|
||||
return b;
|
||||
} else {
|
||||
return (com.google.protobuf.ByteString) ref;
|
||||
}
|
||||
}
|
||||
/**
|
||||
* <code>optional string sourceHFileArchiveDirPath = 4;</code>
|
||||
*/
|
||||
public Builder setSourceHFileArchiveDirPath(
|
||||
java.lang.String value) {
|
||||
if (value == null) {
|
||||
throw new NullPointerException();
|
||||
}
|
||||
bitField0_ |= 0x00000008;
|
||||
sourceHFileArchiveDirPath_ = value;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
/**
|
||||
* <code>optional string sourceHFileArchiveDirPath = 4;</code>
|
||||
*/
|
||||
public Builder clearSourceHFileArchiveDirPath() {
|
||||
bitField0_ = (bitField0_ & ~0x00000008);
|
||||
sourceHFileArchiveDirPath_ = getDefaultInstance().getSourceHFileArchiveDirPath();
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
/**
|
||||
* <code>optional string sourceHFileArchiveDirPath = 4;</code>
|
||||
*/
|
||||
public Builder setSourceHFileArchiveDirPathBytes(
|
||||
com.google.protobuf.ByteString value) {
|
||||
if (value == null) {
|
||||
throw new NullPointerException();
|
||||
}
|
||||
bitField0_ |= 0x00000008;
|
||||
sourceHFileArchiveDirPath_ = value;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
|
||||
// @@protoc_insertion_point(builder_scope:hbase.pb.ReplicateWALEntryRequest)
|
||||
}
|
||||
|
||||
|
@ -23539,56 +24037,58 @@ public final class AdminProtos {
|
|||
"ster_system_time\030\004 \001(\004\"\026\n\024MergeRegionsRe" +
|
||||
"sponse\"a\n\010WALEntry\022\035\n\003key\030\001 \002(\0132\020.hbase." +
|
||||
"pb.WALKey\022\027\n\017key_value_bytes\030\002 \003(\014\022\035\n\025as",
|
||||
"sociated_cell_count\030\003 \001(\005\"=\n\030ReplicateWA" +
|
||||
"LEntryRequest\022!\n\005entry\030\001 \003(\0132\022.hbase.pb." +
|
||||
"WALEntry\"\033\n\031ReplicateWALEntryResponse\"\026\n" +
|
||||
"\024RollWALWriterRequest\"0\n\025RollWALWriterRe" +
|
||||
"sponse\022\027\n\017region_to_flush\030\001 \003(\014\"#\n\021StopS" +
|
||||
"erverRequest\022\016\n\006reason\030\001 \002(\t\"\024\n\022StopServ" +
|
||||
"erResponse\"\026\n\024GetServerInfoRequest\"K\n\nSe" +
|
||||
"rverInfo\022)\n\013server_name\030\001 \002(\0132\024.hbase.pb" +
|
||||
".ServerName\022\022\n\nwebui_port\030\002 \001(\r\"B\n\025GetSe" +
|
||||
"rverInfoResponse\022)\n\013server_info\030\001 \002(\0132\024.",
|
||||
"hbase.pb.ServerInfo\"\034\n\032UpdateConfigurati" +
|
||||
"onRequest\"\035\n\033UpdateConfigurationResponse" +
|
||||
"2\207\013\n\014AdminService\022P\n\rGetRegionInfo\022\036.hba" +
|
||||
"se.pb.GetRegionInfoRequest\032\037.hbase.pb.Ge" +
|
||||
"tRegionInfoResponse\022M\n\014GetStoreFile\022\035.hb" +
|
||||
"ase.pb.GetStoreFileRequest\032\036.hbase.pb.Ge" +
|
||||
"tStoreFileResponse\022V\n\017GetOnlineRegion\022 ." +
|
||||
"hbase.pb.GetOnlineRegionRequest\032!.hbase." +
|
||||
"pb.GetOnlineRegionResponse\022G\n\nOpenRegion" +
|
||||
"\022\033.hbase.pb.OpenRegionRequest\032\034.hbase.pb",
|
||||
".OpenRegionResponse\022M\n\014WarmupRegion\022\035.hb" +
|
||||
"ase.pb.WarmupRegionRequest\032\036.hbase.pb.Wa" +
|
||||
"rmupRegionResponse\022J\n\013CloseRegion\022\034.hbas" +
|
||||
"e.pb.CloseRegionRequest\032\035.hbase.pb.Close" +
|
||||
"RegionResponse\022J\n\013FlushRegion\022\034.hbase.pb" +
|
||||
".FlushRegionRequest\032\035.hbase.pb.FlushRegi" +
|
||||
"onResponse\022J\n\013SplitRegion\022\034.hbase.pb.Spl" +
|
||||
"itRegionRequest\032\035.hbase.pb.SplitRegionRe" +
|
||||
"sponse\022P\n\rCompactRegion\022\036.hbase.pb.Compa" +
|
||||
"ctRegionRequest\032\037.hbase.pb.CompactRegion",
|
||||
"Response\022M\n\014MergeRegions\022\035.hbase.pb.Merg" +
|
||||
"eRegionsRequest\032\036.hbase.pb.MergeRegionsR" +
|
||||
"esponse\022\\\n\021ReplicateWALEntry\022\".hbase.pb." +
|
||||
"ReplicateWALEntryRequest\032#.hbase.pb.Repl" +
|
||||
"icateWALEntryResponse\022Q\n\006Replay\022\".hbase." +
|
||||
"pb.ReplicateWALEntryRequest\032#.hbase.pb.R" +
|
||||
"eplicateWALEntryResponse\022P\n\rRollWALWrite" +
|
||||
"r\022\036.hbase.pb.RollWALWriterRequest\032\037.hbas" +
|
||||
"e.pb.RollWALWriterResponse\022P\n\rGetServerI" +
|
||||
"nfo\022\036.hbase.pb.GetServerInfoRequest\032\037.hb",
|
||||
"ase.pb.GetServerInfoResponse\022G\n\nStopServ" +
|
||||
"er\022\033.hbase.pb.StopServerRequest\032\034.hbase." +
|
||||
"pb.StopServerResponse\022_\n\022UpdateFavoredNo" +
|
||||
"des\022#.hbase.pb.UpdateFavoredNodesRequest" +
|
||||
"\032$.hbase.pb.UpdateFavoredNodesResponse\022b" +
|
||||
"\n\023UpdateConfiguration\022$.hbase.pb.UpdateC" +
|
||||
"onfigurationRequest\032%.hbase.pb.UpdateCon" +
|
||||
"figurationResponseBA\n*org.apache.hadoop." +
|
||||
"hbase.protobuf.generatedB\013AdminProtosH\001\210" +
|
||||
"\001\001\240\001\001"
|
||||
"sociated_cell_count\030\003 \001(\005\"\242\001\n\030ReplicateW" +
|
||||
"ALEntryRequest\022!\n\005entry\030\001 \003(\0132\022.hbase.pb" +
|
||||
".WALEntry\022\034\n\024replicationClusterId\030\002 \001(\t\022" +
|
||||
"\"\n\032sourceBaseNamespaceDirPath\030\003 \001(\t\022!\n\031s" +
|
||||
"ourceHFileArchiveDirPath\030\004 \001(\t\"\033\n\031Replic" +
|
||||
"ateWALEntryResponse\"\026\n\024RollWALWriterRequ" +
|
||||
"est\"0\n\025RollWALWriterResponse\022\027\n\017region_t" +
|
||||
"o_flush\030\001 \003(\014\"#\n\021StopServerRequest\022\016\n\006re" +
|
||||
"ason\030\001 \002(\t\"\024\n\022StopServerResponse\"\026\n\024GetS" +
|
||||
"erverInfoRequest\"K\n\nServerInfo\022)\n\013server",
|
||||
"_name\030\001 \002(\0132\024.hbase.pb.ServerName\022\022\n\nweb" +
|
||||
"ui_port\030\002 \001(\r\"B\n\025GetServerInfoResponse\022)" +
|
||||
"\n\013server_info\030\001 \002(\0132\024.hbase.pb.ServerInf" +
|
||||
"o\"\034\n\032UpdateConfigurationRequest\"\035\n\033Updat" +
|
||||
"eConfigurationResponse2\207\013\n\014AdminService\022" +
|
||||
"P\n\rGetRegionInfo\022\036.hbase.pb.GetRegionInf" +
|
||||
"oRequest\032\037.hbase.pb.GetRegionInfoRespons" +
|
||||
"e\022M\n\014GetStoreFile\022\035.hbase.pb.GetStoreFil" +
|
||||
"eRequest\032\036.hbase.pb.GetStoreFileResponse" +
|
||||
"\022V\n\017GetOnlineRegion\022 .hbase.pb.GetOnline",
|
||||
"RegionRequest\032!.hbase.pb.GetOnlineRegion" +
|
||||
"Response\022G\n\nOpenRegion\022\033.hbase.pb.OpenRe" +
|
||||
"gionRequest\032\034.hbase.pb.OpenRegionRespons" +
|
||||
"e\022M\n\014WarmupRegion\022\035.hbase.pb.WarmupRegio" +
|
||||
"nRequest\032\036.hbase.pb.WarmupRegionResponse" +
|
||||
"\022J\n\013CloseRegion\022\034.hbase.pb.CloseRegionRe" +
|
||||
"quest\032\035.hbase.pb.CloseRegionResponse\022J\n\013" +
|
||||
"FlushRegion\022\034.hbase.pb.FlushRegionReques" +
|
||||
"t\032\035.hbase.pb.FlushRegionResponse\022J\n\013Spli" +
|
||||
"tRegion\022\034.hbase.pb.SplitRegionRequest\032\035.",
|
||||
"hbase.pb.SplitRegionResponse\022P\n\rCompactR" +
|
||||
"egion\022\036.hbase.pb.CompactRegionRequest\032\037." +
|
||||
"hbase.pb.CompactRegionResponse\022M\n\014MergeR" +
|
||||
"egions\022\035.hbase.pb.MergeRegionsRequest\032\036." +
|
||||
"hbase.pb.MergeRegionsResponse\022\\\n\021Replica" +
|
||||
"teWALEntry\022\".hbase.pb.ReplicateWALEntryR" +
|
||||
"equest\032#.hbase.pb.ReplicateWALEntryRespo" +
|
||||
"nse\022Q\n\006Replay\022\".hbase.pb.ReplicateWALEnt" +
|
||||
"ryRequest\032#.hbase.pb.ReplicateWALEntryRe" +
|
||||
"sponse\022P\n\rRollWALWriter\022\036.hbase.pb.RollW",
|
||||
"ALWriterRequest\032\037.hbase.pb.RollWALWriter" +
|
||||
"Response\022P\n\rGetServerInfo\022\036.hbase.pb.Get" +
|
||||
"ServerInfoRequest\032\037.hbase.pb.GetServerIn" +
|
||||
"foResponse\022G\n\nStopServer\022\033.hbase.pb.Stop" +
|
||||
"ServerRequest\032\034.hbase.pb.StopServerRespo" +
|
||||
"nse\022_\n\022UpdateFavoredNodes\022#.hbase.pb.Upd" +
|
||||
"ateFavoredNodesRequest\032$.hbase.pb.Update" +
|
||||
"FavoredNodesResponse\022b\n\023UpdateConfigurat" +
|
||||
"ion\022$.hbase.pb.UpdateConfigurationReques" +
|
||||
"t\032%.hbase.pb.UpdateConfigurationResponse",
|
||||
"BA\n*org.apache.hadoop.hbase.protobuf.gen" +
|
||||
"eratedB\013AdminProtosH\001\210\001\001\240\001\001"
|
||||
};
|
||||
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
||||
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
||||
|
@ -23750,7 +24250,7 @@ public final class AdminProtos {
|
|||
internal_static_hbase_pb_ReplicateWALEntryRequest_fieldAccessorTable = new
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||
internal_static_hbase_pb_ReplicateWALEntryRequest_descriptor,
|
||||
new java.lang.String[] { "Entry", });
|
||||
new java.lang.String[] { "Entry", "ReplicationClusterId", "SourceBaseNamespaceDirPath", "SourceHFileArchiveDirPath", });
|
||||
internal_static_hbase_pb_ReplicateWALEntryResponse_descriptor =
|
||||
getDescriptor().getMessageTypes().get(24);
|
||||
internal_static_hbase_pb_ReplicateWALEntryResponse_fieldAccessorTable = new
|
||||
|
|
|
@ -211,6 +211,9 @@ message WALEntry {
|
|||
*/
|
||||
message ReplicateWALEntryRequest {
|
||||
repeated WALEntry entry = 1;
|
||||
optional string replicationClusterId = 2;
|
||||
optional string sourceBaseNamespaceDirPath = 3;
|
||||
optional string sourceHFileArchiveDirPath = 4;
|
||||
}
|
||||
|
||||
message ReplicateWALEntryResponse {
|
||||
|
|
|
@ -67,6 +67,7 @@ import org.apache.hadoop.hbase.regionserver.HStore;
|
|||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
|
||||
import org.apache.hadoop.hbase.security.UserProvider;
|
||||
import org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint;
|
||||
import org.apache.hadoop.hbase.security.token.FsDelegationToken;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSHDFSUtils;
|
||||
|
@ -125,6 +126,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
|
|||
private FsDelegationToken fsDelegationToken;
|
||||
private String bulkToken;
|
||||
private UserProvider userProvider;
|
||||
private int nrThreads;
|
||||
|
||||
private LoadIncrementalHFiles() {}
|
||||
|
||||
|
@ -146,6 +148,8 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
|
|||
this.fsDelegationToken = new FsDelegationToken(userProvider, "renewer");
|
||||
assignSeqIds = conf.getBoolean(ASSIGN_SEQ_IDS, true);
|
||||
maxFilesPerRegionPerFamily = conf.getInt(MAX_FILES_PER_REGION_PER_FAMILY, 32);
|
||||
nrThreads = conf.getInt("hbase.loadincremental.threads.max",
|
||||
Runtime.getRuntime().availableProcessors());
|
||||
initalized = true;
|
||||
}
|
||||
|
||||
|
@ -246,7 +250,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
|
|||
* region boundary, and each part is added back into the queue.
|
||||
* The import process finishes when the queue is empty.
|
||||
*/
|
||||
static class LoadQueueItem {
|
||||
public static class LoadQueueItem {
|
||||
final byte[] family;
|
||||
final Path hfilePath;
|
||||
|
||||
|
@ -313,7 +317,6 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
|
|||
* @param table the table to load into
|
||||
* @throws TableNotFoundException if table does not yet exist
|
||||
*/
|
||||
@SuppressWarnings("deprecation")
|
||||
public void doBulkLoad(Path hfofDir, final Admin admin, Table table,
|
||||
RegionLocator regionLocator) throws TableNotFoundException, IOException {
|
||||
|
||||
|
@ -321,16 +324,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
|
|||
throw new TableNotFoundException("Table " + table.getName() + "is not currently available.");
|
||||
}
|
||||
|
||||
// initialize thread pools
|
||||
int nrThreads = getConf().getInt("hbase.loadincremental.threads.max",
|
||||
Runtime.getRuntime().availableProcessors());
|
||||
ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
|
||||
builder.setNameFormat("LoadIncrementalHFiles-%1$d");
|
||||
ExecutorService pool = new ThreadPoolExecutor(nrThreads, nrThreads,
|
||||
60, TimeUnit.SECONDS,
|
||||
new LinkedBlockingQueue<Runnable>(),
|
||||
builder.build());
|
||||
((ThreadPoolExecutor)pool).allowCoreThreadTimeOut(true);
|
||||
ExecutorService pool = createExecutorService();
|
||||
|
||||
// LQI queue does not need to be threadsafe -- all operations on this queue
|
||||
// happen in this thread
|
||||
|
@ -347,30 +341,8 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
|
|||
"option, consider removing the files and bulkload again without this option. " +
|
||||
"See HBASE-13985");
|
||||
}
|
||||
discoverLoadQueue(queue, hfofDir, validateHFile);
|
||||
// check whether there is invalid family name in HFiles to be bulkloaded
|
||||
Collection<HColumnDescriptor> families = table.getTableDescriptor().getFamilies();
|
||||
ArrayList<String> familyNames = new ArrayList<String>(families.size());
|
||||
for (HColumnDescriptor family : families) {
|
||||
familyNames.add(family.getNameAsString());
|
||||
}
|
||||
ArrayList<String> unmatchedFamilies = new ArrayList<String>();
|
||||
Iterator<LoadQueueItem> queueIter = queue.iterator();
|
||||
while (queueIter.hasNext()) {
|
||||
LoadQueueItem lqi = queueIter.next();
|
||||
String familyNameInHFile = Bytes.toString(lqi.family);
|
||||
if (!familyNames.contains(familyNameInHFile)) {
|
||||
unmatchedFamilies.add(familyNameInHFile);
|
||||
}
|
||||
}
|
||||
if (unmatchedFamilies.size() > 0) {
|
||||
String msg =
|
||||
"Unmatched family names found: unmatched family names in HFiles to be bulkloaded: "
|
||||
+ unmatchedFamilies + "; valid family names of table "
|
||||
+ table.getName() + " are: " + familyNames;
|
||||
LOG.error(msg);
|
||||
throw new IOException(msg);
|
||||
}
|
||||
prepareHFileQueue(hfofDir, table, queue, validateHFile);
|
||||
|
||||
int count = 0;
|
||||
|
||||
if (queue.isEmpty()) {
|
||||
|
@ -397,7 +369,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
|
|||
+ count + " with " + queue.size() + " files remaining to group or split");
|
||||
}
|
||||
|
||||
int maxRetries = getConf().getInt("hbase.bulkload.retries.number", 10);
|
||||
int maxRetries = getConf().getInt(HConstants.BULKLOAD_MAX_RETRIES_NUMBER, 10);
|
||||
maxRetries = Math.max(maxRetries, startEndKeys.getFirst().length + 1);
|
||||
if (maxRetries != 0 && count >= maxRetries) {
|
||||
throw new IOException("Retry attempted " + count +
|
||||
|
@ -446,6 +418,85 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Prepare a collection of {@link LoadQueueItem} from list of source hfiles contained in the
|
||||
* passed directory and validates whether the prepared queue has all the valid table column
|
||||
* families in it.
|
||||
* @param hfilesDir directory containing list of hfiles to be loaded into the table
|
||||
* @param table table to which hfiles should be loaded
|
||||
* @param queue queue which needs to be loaded into the table
|
||||
* @throws IOException If any I/O or network error occurred
|
||||
*/
|
||||
public void prepareHFileQueue(Path hfofDir, Table table, Deque<LoadQueueItem> queue,
|
||||
boolean validateHFile) throws IOException {
|
||||
discoverLoadQueue(queue, hfofDir, validateHFile);
|
||||
validateFamiliesInHFiles(table, queue);
|
||||
}
|
||||
|
||||
// Initialize a thread pool
|
||||
private ExecutorService createExecutorService() {
|
||||
ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
|
||||
builder.setNameFormat("LoadIncrementalHFiles-%1$d");
|
||||
ExecutorService pool = new ThreadPoolExecutor(nrThreads, nrThreads, 60, TimeUnit.SECONDS,
|
||||
new LinkedBlockingQueue<Runnable>(), builder.build());
|
||||
((ThreadPoolExecutor) pool).allowCoreThreadTimeOut(true);
|
||||
return pool;
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks whether there is any invalid family name in HFiles to be bulk loaded.
|
||||
*/
|
||||
private void validateFamiliesInHFiles(Table table, Deque<LoadQueueItem> queue)
|
||||
throws IOException {
|
||||
Collection<HColumnDescriptor> families = table.getTableDescriptor().getFamilies();
|
||||
List<String> familyNames = new ArrayList<String>(families.size());
|
||||
for (HColumnDescriptor family : families) {
|
||||
familyNames.add(family.getNameAsString());
|
||||
}
|
||||
List<String> unmatchedFamilies = new ArrayList<String>();
|
||||
Iterator<LoadQueueItem> queueIter = queue.iterator();
|
||||
while (queueIter.hasNext()) {
|
||||
LoadQueueItem lqi = queueIter.next();
|
||||
String familyNameInHFile = Bytes.toString(lqi.family);
|
||||
if (!familyNames.contains(familyNameInHFile)) {
|
||||
unmatchedFamilies.add(familyNameInHFile);
|
||||
}
|
||||
}
|
||||
if (unmatchedFamilies.size() > 0) {
|
||||
String msg =
|
||||
"Unmatched family names found: unmatched family names in HFiles to be bulkloaded: "
|
||||
+ unmatchedFamilies + "; valid family names of table " + table.getName() + " are: "
|
||||
+ familyNames;
|
||||
LOG.error(msg);
|
||||
throw new IOException(msg);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Used by the replication sink to load the hfiles from the source cluster. It does the following,
|
||||
* 1. {@link LoadIncrementalHFiles#groupOrSplitPhase(Table, ExecutorService, Deque, Pair)} 2.
|
||||
* {@link
|
||||
* LoadIncrementalHFiles#bulkLoadPhase(Table, Connection, ExecutorService, Deque, Multimap)}
|
||||
* @param table Table to which these hfiles should be loaded to
|
||||
* @param conn Connection to use
|
||||
* @param queue {@link LoadQueueItem} has hfiles yet to be loaded
|
||||
* @param startEndKeys starting and ending row keys of the region
|
||||
*/
|
||||
public void loadHFileQueue(final Table table, final Connection conn, Deque<LoadQueueItem> queue,
|
||||
Pair<byte[][], byte[][]> startEndKeys) throws IOException {
|
||||
ExecutorService pool = null;
|
||||
try {
|
||||
pool = createExecutorService();
|
||||
Multimap<ByteBuffer, LoadQueueItem> regionGroups =
|
||||
groupOrSplitPhase(table, pool, queue, startEndKeys);
|
||||
bulkLoadPhase(table, conn, pool, queue, regionGroups);
|
||||
} finally {
|
||||
if (pool != null) {
|
||||
pool.shutdown();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This takes the LQI's grouped by likely regions and attempts to bulk load
|
||||
* them. Any failures are re-queued for another pass with the
|
||||
|
@ -592,10 +643,10 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
|
|||
|
||||
String uniqueName = getUniqueName();
|
||||
HColumnDescriptor familyDesc = table.getTableDescriptor().getFamily(item.family);
|
||||
|
||||
Path botOut = new Path(tmpDir, uniqueName + ".bottom");
|
||||
Path topOut = new Path(tmpDir, uniqueName + ".top");
|
||||
splitStoreFile(getConf(), hfilePath, familyDesc, splitKey,
|
||||
botOut, topOut);
|
||||
splitStoreFile(getConf(), hfilePath, familyDesc, splitKey, botOut, topOut);
|
||||
|
||||
FileSystem fs = tmpDir.getFileSystem(getConf());
|
||||
fs.setPermission(tmpDir, FsPermission.valueOf("-rwxrwxrwx"));
|
||||
|
@ -626,6 +677,10 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
|
|||
final Pair<byte[][], byte[][]> startEndKeys)
|
||||
throws IOException {
|
||||
final Path hfilePath = item.hfilePath;
|
||||
// fs is the source filesystem
|
||||
if (fs == null) {
|
||||
fs = hfilePath.getFileSystem(getConf());
|
||||
}
|
||||
HFile.Reader hfr = HFile.createReader(fs, hfilePath,
|
||||
new CacheConfig(getConf()), getConf());
|
||||
final byte[] first, last;
|
||||
|
@ -712,7 +767,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
|
|||
* failure
|
||||
*/
|
||||
protected List<LoadQueueItem> tryAtomicRegionLoad(final Connection conn,
|
||||
final TableName tableName, final byte[] first, Collection<LoadQueueItem> lqis)
|
||||
final TableName tableName, final byte[] first, final Collection<LoadQueueItem> lqis)
|
||||
throws IOException {
|
||||
final List<Pair<byte[], String>> famPaths =
|
||||
new ArrayList<Pair<byte[], String>>(lqis.size());
|
||||
|
@ -747,6 +802,10 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
|
|||
//in user directory
|
||||
if(secureClient != null && !success) {
|
||||
FileSystem targetFs = FileSystem.get(getConf());
|
||||
// fs is the source filesystem
|
||||
if(fs == null) {
|
||||
fs = lqis.iterator().next().hfilePath.getFileSystem(getConf());
|
||||
}
|
||||
// Check to see if the source and target filesystems are the same
|
||||
// If they are the same filesystem, we will try move the files back
|
||||
// because previously we moved them to the staging directory.
|
||||
|
@ -1000,4 +1059,17 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
|
|||
System.exit(ret);
|
||||
}
|
||||
|
||||
/**
|
||||
* Called from replication sink, where it manages bulkToken(staging directory) by itself. This is
|
||||
* used only when {@link SecureBulkLoadEndpoint} is configured in hbase.coprocessor.region.classes
|
||||
* property. This directory is used as a temporary directory where all files are initially
|
||||
* copied/moved from user given directory, set all the required file permissions and then from
|
||||
* their it is finally loaded into a table. This should be set only when, one would like to manage
|
||||
* the staging directory by itself. Otherwise this tool will handle this by itself.
|
||||
* @param stagingDir staging directory path
|
||||
*/
|
||||
public void setBulkToken(String stagingDir) {
|
||||
this.bulkToken = stagingDir;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -28,22 +28,23 @@ import java.util.Map;
|
|||
import java.util.NavigableMap;
|
||||
import java.util.UUID;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.io.SizedCellScanner;
|
||||
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Entry;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Entry;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
|
@ -51,15 +52,20 @@ import com.google.protobuf.ServiceException;
|
|||
public class ReplicationProtbufUtil {
|
||||
/**
|
||||
* A helper to replicate a list of WAL entries using admin protocol.
|
||||
*
|
||||
* @param admin
|
||||
* @param entries
|
||||
* @param admin Admin service
|
||||
* @param entries Array of WAL entries to be replicated
|
||||
* @param replicationClusterId Id which will uniquely identify source cluster FS client
|
||||
* configurations in the replication configuration directory
|
||||
* @param sourceBaseNamespaceDir Path to source cluster base namespace directory
|
||||
* @param sourceHFileArchiveDir Path to the source cluster hfile archive directory
|
||||
* @throws java.io.IOException
|
||||
*/
|
||||
public static void replicateWALEntry(final AdminService.BlockingInterface admin,
|
||||
final Entry[] entries) throws IOException {
|
||||
final Entry[] entries, String replicationClusterId, Path sourceBaseNamespaceDir,
|
||||
Path sourceHFileArchiveDir) throws IOException {
|
||||
Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner> p =
|
||||
buildReplicateWALEntryRequest(entries, null);
|
||||
buildReplicateWALEntryRequest(entries, null, replicationClusterId, sourceBaseNamespaceDir,
|
||||
sourceHFileArchiveDir);
|
||||
PayloadCarryingRpcController controller = new PayloadCarryingRpcController(p.getSecond());
|
||||
try {
|
||||
admin.replicateWALEntry(controller, p.getFirst());
|
||||
|
@ -77,19 +83,22 @@ public class ReplicationProtbufUtil {
|
|||
*/
|
||||
public static Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner>
|
||||
buildReplicateWALEntryRequest(final Entry[] entries) {
|
||||
return buildReplicateWALEntryRequest(entries, null);
|
||||
return buildReplicateWALEntryRequest(entries, null, null, null, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new ReplicateWALEntryRequest from a list of WAL entries
|
||||
*
|
||||
* @param entries the WAL entries to be replicated
|
||||
* @param encodedRegionName alternative region name to use if not null
|
||||
* @return a pair of ReplicateWALEntryRequest and a CellScanner over all the WALEdit values
|
||||
* found.
|
||||
* @param replicationClusterId Id which will uniquely identify source cluster FS client
|
||||
* configurations in the replication configuration directory
|
||||
* @param sourceBaseNamespaceDir Path to source cluster base namespace directory
|
||||
* @param sourceHFileArchiveDir Path to the source cluster hfile archive directory
|
||||
* @return a pair of ReplicateWALEntryRequest and a CellScanner over all the WALEdit values found.
|
||||
*/
|
||||
public static Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner>
|
||||
buildReplicateWALEntryRequest(final Entry[] entries, byte[] encodedRegionName) {
|
||||
buildReplicateWALEntryRequest(final Entry[] entries, byte[] encodedRegionName,
|
||||
String replicationClusterId, Path sourceBaseNamespaceDir, Path sourceHFileArchiveDir) {
|
||||
// Accumulate all the Cells seen in here.
|
||||
List<List<? extends Cell>> allCells = new ArrayList<List<? extends Cell>>(entries.length);
|
||||
int size = 0;
|
||||
|
@ -146,6 +155,17 @@ public class ReplicationProtbufUtil {
|
|||
entryBuilder.setAssociatedCellCount(cells.size());
|
||||
builder.addEntry(entryBuilder.build());
|
||||
}
|
||||
|
||||
if (replicationClusterId != null) {
|
||||
builder.setReplicationClusterId(replicationClusterId);
|
||||
}
|
||||
if (sourceBaseNamespaceDir != null) {
|
||||
builder.setSourceBaseNamespaceDirPath(sourceBaseNamespaceDir.toString());
|
||||
}
|
||||
if (sourceHFileArchiveDir != null) {
|
||||
builder.setSourceHFileArchiveDirPath(sourceHFileArchiveDir.toString());
|
||||
}
|
||||
|
||||
return new Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner>(builder.build(),
|
||||
getCellScanner(allCells, size));
|
||||
}
|
||||
|
|
|
@ -1800,7 +1800,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
List<WALEntry> entries = request.getEntryList();
|
||||
CellScanner cellScanner = ((PayloadCarryingRpcController)controller).cellScanner();
|
||||
regionServer.getRegionServerCoprocessorHost().preReplicateLogEntries(entries, cellScanner);
|
||||
regionServer.replicationSinkHandler.replicateLogEntries(entries, cellScanner);
|
||||
regionServer.replicationSinkHandler.replicateLogEntries(entries, cellScanner,
|
||||
request.getReplicationClusterId(), request.getSourceBaseNamespaceDirPath(),
|
||||
request.getSourceHFileArchiveDirPath());
|
||||
regionServer.getRegionServerCoprocessorHost().postReplicateLogEntries(entries, cellScanner);
|
||||
return ReplicateWALEntryResponse.newBuilder().build();
|
||||
} else {
|
||||
|
|
|
@ -36,7 +36,13 @@ public interface ReplicationSinkService extends ReplicationService {
|
|||
* Carry on the list of log entries down to the sink
|
||||
* @param entries list of WALEntries to replicate
|
||||
* @param cells Cells that the WALEntries refer to (if cells is non-null)
|
||||
* @param replicationClusterId Id which will uniquely identify source cluster FS client
|
||||
* configurations in the replication configuration directory
|
||||
* @param sourceBaseNamespaceDirPath Path that point to the source cluster base namespace
|
||||
* directory required for replicating hfiles
|
||||
* @param sourceHFileArchiveDirPath Path that point to the source cluster hfile archive directory
|
||||
* @throws IOException
|
||||
*/
|
||||
void replicateLogEntries(List<WALEntry> entries, CellScanner cells) throws IOException;
|
||||
void replicateLogEntries(List<WALEntry> entries, CellScanner cells, String replicationClusterId,
|
||||
String sourceBaseNamespaceDirPath, String sourceHFileArchiveDirPath) throws IOException;
|
||||
}
|
||||
|
|
|
@ -85,17 +85,16 @@ public interface WALActionsListener {
|
|||
);
|
||||
|
||||
/**
|
||||
*
|
||||
* @param htd
|
||||
* @param logKey
|
||||
* @param logEdit
|
||||
* TODO: Retire this in favor of {@link #visitLogEntryBeforeWrite(HRegionInfo, WALKey, WALEdit)}
|
||||
* It only exists to get scope when replicating. Scope should be in the WALKey and not need
|
||||
* us passing in a <code>htd</code>.
|
||||
* @param logEdit TODO: Retire this in favor of
|
||||
* {@link #visitLogEntryBeforeWrite(HRegionInfo, WALKey, WALEdit)} It only exists to get
|
||||
* scope when replicating. Scope should be in the WALKey and not need us passing in a
|
||||
* <code>htd</code>.
|
||||
* @throws IOException If failed to parse the WALEdit
|
||||
*/
|
||||
void visitLogEntryBeforeWrite(
|
||||
HTableDescriptor htd, WALKey logKey, WALEdit logEdit
|
||||
);
|
||||
void visitLogEntryBeforeWrite(HTableDescriptor htd, WALKey logKey, WALEdit logEdit)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* For notification post append to the writer. Used by metrics system at least.
|
||||
|
@ -136,7 +135,9 @@ public interface WALActionsListener {
|
|||
public void visitLogEntryBeforeWrite(HRegionInfo info, WALKey logKey, WALEdit logEdit) {}
|
||||
|
||||
@Override
|
||||
public void visitLogEntryBeforeWrite(HTableDescriptor htd, WALKey logKey, WALEdit logEdit) {}
|
||||
public void visitLogEntryBeforeWrite(HTableDescriptor htd, WALKey logKey, WALEdit logEdit)
|
||||
throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postAppend(final long entryLen, final long elapsedTimeMillis) {}
|
||||
|
|
|
@ -18,13 +18,21 @@
|
|||
|
||||
package org.apache.hadoop.hbase.replication;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.NavigableMap;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Entry;
|
||||
|
||||
/**
|
||||
|
@ -32,6 +40,7 @@ import org.apache.hadoop.hbase.wal.WAL.Entry;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ScopeWALEntryFilter implements WALEntryFilter {
|
||||
private static final Log LOG = LogFactory.getLog(ScopeWALEntryFilter.class);
|
||||
|
||||
@Override
|
||||
public Entry filter(Entry entry) {
|
||||
|
@ -41,19 +50,70 @@ public class ScopeWALEntryFilter implements WALEntryFilter {
|
|||
}
|
||||
ArrayList<Cell> cells = entry.getEdit().getCells();
|
||||
int size = cells.size();
|
||||
byte[] fam;
|
||||
for (int i = size - 1; i >= 0; i--) {
|
||||
Cell cell = cells.get(i);
|
||||
// If a bulk load entry has a scope then that means user has enabled replication for bulk load
|
||||
// hfiles.
|
||||
// TODO There is a similar logic in TableCfWALEntryFilter but data structures are different so
|
||||
// cannot refactor into one now, can revisit and see if any way to unify them.
|
||||
if (CellUtil.matchingColumn(cell, WALEdit.METAFAMILY, WALEdit.BULK_LOAD)) {
|
||||
Cell filteredBulkLoadEntryCell = filterBulkLoadEntries(scopes, cell);
|
||||
if (filteredBulkLoadEntryCell != null) {
|
||||
cells.set(i, filteredBulkLoadEntryCell);
|
||||
} else {
|
||||
cells.remove(i);
|
||||
}
|
||||
} else {
|
||||
// The scope will be null or empty if
|
||||
// there's nothing to replicate in that WALEdit
|
||||
byte[] fam = CellUtil.cloneFamily(cell);
|
||||
fam = CellUtil.cloneFamily(cell);
|
||||
if (!scopes.containsKey(fam) || scopes.get(fam) == HConstants.REPLICATION_SCOPE_LOCAL) {
|
||||
cells.remove(i);
|
||||
}
|
||||
}
|
||||
}
|
||||
if (cells.size() < size / 2) {
|
||||
cells.trimToSize();
|
||||
}
|
||||
return entry;
|
||||
}
|
||||
|
||||
private Cell filterBulkLoadEntries(NavigableMap<byte[], Integer> scopes, Cell cell) {
|
||||
byte[] fam;
|
||||
BulkLoadDescriptor bld = null;
|
||||
try {
|
||||
bld = WALEdit.getBulkLoadDescriptor(cell);
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Failed to get bulk load events information from the WAL file.", e);
|
||||
return cell;
|
||||
}
|
||||
List<StoreDescriptor> storesList = bld.getStoresList();
|
||||
// Copy the StoreDescriptor list and update it as storesList is a unmodifiableList
|
||||
List<StoreDescriptor> copiedStoresList = new ArrayList<StoreDescriptor>(storesList);
|
||||
Iterator<StoreDescriptor> copiedStoresListIterator = copiedStoresList.iterator();
|
||||
boolean anyStoreRemoved = false;
|
||||
while (copiedStoresListIterator.hasNext()) {
|
||||
StoreDescriptor sd = copiedStoresListIterator.next();
|
||||
fam = sd.getFamilyName().toByteArray();
|
||||
if (!scopes.containsKey(fam) || scopes.get(fam) == HConstants.REPLICATION_SCOPE_LOCAL) {
|
||||
copiedStoresListIterator.remove();
|
||||
anyStoreRemoved = true;
|
||||
}
|
||||
}
|
||||
|
||||
if (!anyStoreRemoved) {
|
||||
return cell;
|
||||
} else if (copiedStoresList.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
BulkLoadDescriptor.Builder newDesc =
|
||||
BulkLoadDescriptor.newBuilder().setTableName(bld.getTableName())
|
||||
.setEncodedRegionName(bld.getEncodedRegionName())
|
||||
.setBulkloadSeqNum(bld.getBulkloadSeqNum());
|
||||
newDesc.addAllStores(copiedStoresList);
|
||||
BulkLoadDescriptor newBulkLoadDescriptor = newDesc.build();
|
||||
return CellUtil.createCell(CellUtil.cloneRow(cell), WALEdit.METAFAMILY, WALEdit.BULK_LOAD,
|
||||
cell.getTimestamp(), cell.getTypeByte(), newBulkLoadDescriptor.toByteArray());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,14 +18,20 @@
|
|||
|
||||
package org.apache.hadoop.hbase.replication;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Entry;
|
||||
|
||||
|
@ -52,26 +58,80 @@ public class TableCfWALEntryFilter implements WALEntryFilter {
|
|||
}
|
||||
int size = cells.size();
|
||||
|
||||
// If null means user has explicitly not configured any table CFs so all the tables data are
|
||||
// applicable for replication
|
||||
if (tableCFs == null) {
|
||||
return entry;
|
||||
}
|
||||
// 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)) {
|
||||
// replicable table list
|
||||
if (!tableCFs.containsKey(tabName)) {
|
||||
return null;
|
||||
} else {
|
||||
List<String> cfs = (tableCFs == null) ? null : tableCFs.get(tabName);
|
||||
List<String> cfs = tableCFs.get(tabName);
|
||||
for (int i = size - 1; i >= 0; i--) {
|
||||
Cell cell = cells.get(i);
|
||||
// TODO There is a similar logic in ScopeWALEntryFilter but data structures are different so
|
||||
// cannot refactor into one now, can revisit and see if any way to unify them.
|
||||
// Filter bulk load entries separately
|
||||
if (CellUtil.matchingColumn(cell, WALEdit.METAFAMILY, WALEdit.BULK_LOAD)) {
|
||||
Cell filteredBulkLoadEntryCell = filterBulkLoadEntries(cfs, cell);
|
||||
if (filteredBulkLoadEntryCell != null) {
|
||||
cells.set(i, filteredBulkLoadEntryCell);
|
||||
} else {
|
||||
cells.remove(i);
|
||||
}
|
||||
} else {
|
||||
// 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()))) {
|
||||
if ((cfs != null) && !cfs.contains(Bytes.toString(cell.getFamilyArray(),
|
||||
cell.getFamilyOffset(), cell.getFamilyLength()))) {
|
||||
cells.remove(i);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (cells.size() < size/2) {
|
||||
cells.trimToSize();
|
||||
}
|
||||
return entry;
|
||||
}
|
||||
|
||||
private Cell filterBulkLoadEntries(List<String> cfs, Cell cell) {
|
||||
byte[] fam;
|
||||
BulkLoadDescriptor bld = null;
|
||||
try {
|
||||
bld = WALEdit.getBulkLoadDescriptor(cell);
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Failed to get bulk load events information from the WAL file.", e);
|
||||
return cell;
|
||||
}
|
||||
List<StoreDescriptor> storesList = bld.getStoresList();
|
||||
// Copy the StoreDescriptor list and update it as storesList is a unmodifiableList
|
||||
List<StoreDescriptor> copiedStoresList = new ArrayList<StoreDescriptor>(storesList);
|
||||
Iterator<StoreDescriptor> copiedStoresListIterator = copiedStoresList.iterator();
|
||||
boolean anyStoreRemoved = false;
|
||||
while (copiedStoresListIterator.hasNext()) {
|
||||
StoreDescriptor sd = copiedStoresListIterator.next();
|
||||
fam = sd.getFamilyName().toByteArray();
|
||||
if (cfs != null && !cfs.contains(Bytes.toString(fam))) {
|
||||
copiedStoresListIterator.remove();
|
||||
anyStoreRemoved = true;
|
||||
}
|
||||
}
|
||||
|
||||
if (!anyStoreRemoved) {
|
||||
return cell;
|
||||
} else if (copiedStoresList.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
BulkLoadDescriptor.Builder newDesc =
|
||||
BulkLoadDescriptor.newBuilder().setTableName(bld.getTableName())
|
||||
.setEncodedRegionName(bld.getEncodedRegionName())
|
||||
.setBulkloadSeqNum(bld.getBulkloadSeqNum());
|
||||
newDesc.addAllStores(copiedStoresList);
|
||||
BulkLoadDescriptor newBulkLoadDescriptor = newDesc.build();
|
||||
return CellUtil.createCell(CellUtil.cloneRow(cell), WALEdit.METAFAMILY, WALEdit.BULK_LOAD,
|
||||
cell.getTimestamp(), cell.getTypeByte(), newBulkLoadDescriptor.toByteArray());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,193 @@
|
|||
/*
|
||||
* 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 com.google.common.base.Predicate;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Sets;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.hbase.Abortable;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate;
|
||||
import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationFactory;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
/**
|
||||
* Implementation of a file cleaner that checks if a hfile is still scheduled for replication before
|
||||
* deleting it from hfile archive directory.
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
|
||||
public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate implements Abortable {
|
||||
private static final Log LOG = LogFactory.getLog(ReplicationHFileCleaner.class);
|
||||
private ZooKeeperWatcher zkw;
|
||||
private ReplicationQueuesClient rqc;
|
||||
private boolean stopped = false;
|
||||
private boolean aborted;
|
||||
|
||||
@Override
|
||||
public Iterable<FileStatus> getDeletableFiles(Iterable<FileStatus> files) {
|
||||
// all members of this class are null if replication is disabled,
|
||||
// so we cannot filter the files
|
||||
if (this.getConf() == null) {
|
||||
return files;
|
||||
}
|
||||
|
||||
final Set<String> hfileRefs;
|
||||
try {
|
||||
// The concurrently created new hfile entries in ZK may not be included in the return list,
|
||||
// but they won't be deleted because they're not in the checking set.
|
||||
hfileRefs = loadHFileRefsFromPeers();
|
||||
} catch (KeeperException e) {
|
||||
LOG.warn("Failed to read hfile references from zookeeper, skipping checking deletable files");
|
||||
return Collections.emptyList();
|
||||
}
|
||||
return Iterables.filter(files, new Predicate<FileStatus>() {
|
||||
@Override
|
||||
public boolean apply(FileStatus file) {
|
||||
String hfile = file.getPath().getName();
|
||||
boolean foundHFileRefInQueue = hfileRefs.contains(hfile);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
if (foundHFileRefInQueue) {
|
||||
LOG.debug("Found hfile reference in ZK, keeping: " + hfile);
|
||||
} else {
|
||||
LOG.debug("Did not find hfile reference in ZK, deleting: " + hfile);
|
||||
}
|
||||
}
|
||||
return !foundHFileRefInQueue;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Load all hfile references in all replication queues from ZK. This method guarantees to return a
|
||||
* snapshot which contains all hfile references in the zookeeper at the start of this call.
|
||||
* However, some newly created hfile references during the call may not be included.
|
||||
*/
|
||||
private Set<String> loadHFileRefsFromPeers() throws KeeperException {
|
||||
Set<String> hfileRefs = Sets.newHashSet();
|
||||
List<String> listOfPeers;
|
||||
for (int retry = 0;; retry++) {
|
||||
int v0 = rqc.getHFileRefsNodeChangeVersion();
|
||||
hfileRefs.clear();
|
||||
listOfPeers = rqc.getAllPeersFromHFileRefsQueue();
|
||||
if (listOfPeers == null) {
|
||||
LOG.debug("Didn't find any peers with hfile references, won't prevent any deletions.");
|
||||
return ImmutableSet.of();
|
||||
}
|
||||
for (String id : listOfPeers) {
|
||||
List<String> peerHFileRefs = rqc.getReplicableHFiles(id);
|
||||
if (peerHFileRefs != null) {
|
||||
hfileRefs.addAll(peerHFileRefs);
|
||||
}
|
||||
}
|
||||
int v1 = rqc.getHFileRefsNodeChangeVersion();
|
||||
if (v0 == v1) {
|
||||
return hfileRefs;
|
||||
}
|
||||
LOG.debug(String.format("Replication hfile references node cversion changed from "
|
||||
+ "%d to %d, retry = %d", v0, v1, retry));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setConf(Configuration config) {
|
||||
// If either replication or replication of bulk load hfiles is disabled, keep all members null
|
||||
if (!(config.getBoolean(HConstants.REPLICATION_ENABLE_KEY,
|
||||
HConstants.REPLICATION_ENABLE_DEFAULT) && config.getBoolean(
|
||||
HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
|
||||
HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT))) {
|
||||
LOG.warn(HConstants.REPLICATION_ENABLE_KEY
|
||||
+ " is not enabled so allowing all hfile references to be deleted. Better to remove "
|
||||
+ ReplicationHFileCleaner.class + " from " + HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS
|
||||
+ " configuration.");
|
||||
return;
|
||||
}
|
||||
// Make my own Configuration. Then I'll have my own connection to zk that
|
||||
// I can close myself when time comes.
|
||||
Configuration conf = new Configuration(config);
|
||||
super.setConf(conf);
|
||||
try {
|
||||
initReplicationQueuesClient(conf);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Error while configuring " + this.getClass().getName(), e);
|
||||
}
|
||||
}
|
||||
|
||||
private void initReplicationQueuesClient(Configuration conf)
|
||||
throws ZooKeeperConnectionException, IOException {
|
||||
this.zkw = new ZooKeeperWatcher(conf, "replicationHFileCleaner", null);
|
||||
this.rqc = ReplicationFactory.getReplicationQueuesClient(zkw, conf, this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop(String why) {
|
||||
if (this.stopped) {
|
||||
return;
|
||||
}
|
||||
this.stopped = true;
|
||||
if (this.zkw != null) {
|
||||
LOG.info("Stopping " + this.zkw);
|
||||
this.zkw.close();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isStopped() {
|
||||
return this.stopped;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort(String why, Throwable e) {
|
||||
LOG.warn("Aborting ReplicationHFileCleaner because " + why, e);
|
||||
this.aborted = true;
|
||||
stop(why);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isAborted() {
|
||||
return this.aborted;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isFileDeletable(FileStatus fStat) {
|
||||
Set<String> hfileRefsFromQueue;
|
||||
// all members of this class are null if replication is disabled,
|
||||
// so do not stop from deleting the file
|
||||
if (getConf() == null) {
|
||||
return true;
|
||||
}
|
||||
|
||||
try {
|
||||
hfileRefsFromQueue = loadHFileRefsFromPeers();
|
||||
} catch (KeeperException e) {
|
||||
LOG.warn("Failed to read hfile references from zookeeper, skipping checking deletable "
|
||||
+ "file for " + fStat.getPath());
|
||||
return false;
|
||||
}
|
||||
return !hfileRefsFromQueue.contains(fStat.getPath().getName());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,78 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license
|
||||
* agreements. See the NOTICE file distributed with this work for additional information regarding
|
||||
* copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable
|
||||
* law or agreed to in writing, software distributed under the License is distributed on an "AS IS"
|
||||
* BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License
|
||||
* for the specific language governing permissions and limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.replication.regionserver;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.net.URL;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileUtil;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* This will load all the xml configuration files for the source cluster replication ID from
|
||||
* user configured replication configuration directory.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class DefaultSourceFSConfigurationProvider implements SourceFSConfigurationProvider {
|
||||
private static final Log LOG = LogFactory.getLog(DefaultSourceFSConfigurationProvider.class);
|
||||
// Map containing all the source clusters configurations against their replication cluster id
|
||||
private Map<String, Configuration> sourceClustersConfs = new HashMap<>();
|
||||
private static final String XML = ".xml";
|
||||
|
||||
@Override
|
||||
public Configuration getConf(Configuration sinkConf, String replicationClusterId)
|
||||
throws IOException {
|
||||
if (sourceClustersConfs.get(replicationClusterId) == null) {
|
||||
synchronized (this.sourceClustersConfs) {
|
||||
if (sourceClustersConfs.get(replicationClusterId) == null) {
|
||||
LOG.info("Loading source cluster FS client conf for cluster " + replicationClusterId);
|
||||
// Load only user provided client configurations.
|
||||
Configuration sourceClusterConf = new Configuration(false);
|
||||
|
||||
String replicationConfDir = sinkConf.get(HConstants.REPLICATION_CONF_DIR);
|
||||
if (replicationConfDir == null) {
|
||||
LOG.debug(HConstants.REPLICATION_CONF_DIR + " is not configured.");
|
||||
URL resource = HBaseConfiguration.class.getClassLoader().getResource("hbase-site.xml");
|
||||
if (resource != null) {
|
||||
String path = resource.getPath();
|
||||
replicationConfDir = path.substring(0, path.lastIndexOf("/"));
|
||||
} else {
|
||||
replicationConfDir = System.getenv("HBASE_CONF_DIR");
|
||||
}
|
||||
}
|
||||
|
||||
LOG.info("Loading source cluster " + replicationClusterId
|
||||
+ " file system configurations from xml files under directory " + replicationConfDir);
|
||||
File confDir = new File(replicationConfDir, replicationClusterId);
|
||||
String[] listofConfFiles = FileUtil.list(confDir);
|
||||
for (String confFile : listofConfFiles) {
|
||||
if (new File(confDir, confFile).isFile() && confFile.endsWith(XML)) {
|
||||
// Add all the user provided client conf files
|
||||
sourceClusterConf.addResource(new Path(confDir.getPath(), confFile));
|
||||
}
|
||||
}
|
||||
this.sourceClustersConfs.put(replicationClusterId, sourceClusterConf);
|
||||
}
|
||||
}
|
||||
}
|
||||
return this.sourceClustersConfs.get(replicationClusterId);
|
||||
}
|
||||
|
||||
}
|
|
@ -37,20 +37,22 @@ import com.google.common.annotations.VisibleForTesting;
|
|||
import org.apache.commons.lang.StringUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.HConnection;
|
||||
import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Entry;
|
||||
import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
|
||||
import org.apache.hadoop.hbase.replication.regionserver.ReplicationSinkManager.SinkPeer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Entry;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
|
||||
/**
|
||||
|
@ -84,8 +86,12 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
|
|||
// Handles connecting to peer region servers
|
||||
private ReplicationSinkManager replicationSinkMgr;
|
||||
private boolean peersSelected = false;
|
||||
private String replicationClusterId = "";
|
||||
private ThreadPoolExecutor exec;
|
||||
private int maxThreads;
|
||||
private Path baseNamespaceDir;
|
||||
private Path hfileArchiveDir;
|
||||
private boolean replicationBulkLoadDataEnabled;
|
||||
|
||||
@Override
|
||||
public void init(Context context) throws IOException {
|
||||
|
@ -109,6 +115,18 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
|
|||
HConstants.REPLICATION_SOURCE_MAXTHREADS_DEFAULT);
|
||||
this.exec = new ThreadPoolExecutor(1, maxThreads, 60, TimeUnit.SECONDS,
|
||||
new SynchronousQueue<Runnable>());
|
||||
|
||||
this.replicationBulkLoadDataEnabled =
|
||||
conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
|
||||
HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT);
|
||||
if (this.replicationBulkLoadDataEnabled) {
|
||||
replicationClusterId = this.conf.get(HConstants.REPLICATION_CLUSTER_ID);
|
||||
}
|
||||
// Construct base namespace directory and hfile archive directory path
|
||||
Path rootDir = FSUtils.getRootDir(conf);
|
||||
Path baseNSDir = new Path(HConstants.BASE_NAMESPACE_DIR);
|
||||
baseNamespaceDir = new Path(rootDir, baseNSDir);
|
||||
hfileArchiveDir = new Path(rootDir, new Path(HConstants.HFILE_ARCHIVE_DIRECTORY, baseNSDir));
|
||||
}
|
||||
|
||||
private void decorateConf() {
|
||||
|
@ -317,8 +335,8 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
|
|||
try {
|
||||
sinkPeer = replicationSinkMgr.getReplicationSink();
|
||||
BlockingInterface rrs = sinkPeer.getRegionServer();
|
||||
ReplicationProtbufUtil.replicateWALEntry(rrs,
|
||||
entries.toArray(new Entry[entries.size()]));
|
||||
ReplicationProtbufUtil.replicateWALEntry(rrs, entries.toArray(new Entry[entries.size()]),
|
||||
replicationClusterId, baseNamespaceDir, hfileArchiveDir);
|
||||
replicationSinkMgr.reportSinkSuccess(sinkPeer);
|
||||
return ordinal;
|
||||
|
||||
|
|
|
@ -0,0 +1,393 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license
|
||||
* agreements. See the NOTICE file distributed with this work for additional information regarding
|
||||
* copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable
|
||||
* law or agreed to in writing, software distributed under the License is distributed on an "AS IS"
|
||||
* BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License
|
||||
* for the specific language governing permissions and limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.replication.regionserver;
|
||||
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.math.BigInteger;
|
||||
import java.security.SecureRandom;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Deque;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FileUtil;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
|
||||
import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles.LoadQueueItem;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.security.UserProvider;
|
||||
import org.apache.hadoop.hbase.security.token.FsDelegationToken;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
||||
/**
|
||||
* It is used for replicating HFile entries. It will first copy parallely all the hfiles to a local
|
||||
* staging directory and then it will use ({@link LoadIncrementalHFiles} to prepare a collection of
|
||||
* {@link LoadQueueItem} which will finally be loaded(replicated) into the table of this cluster.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class HFileReplicator {
|
||||
/** Maximum number of threads to allow in pool to copy hfiles during replication */
|
||||
public static final String REPLICATION_BULKLOAD_COPY_MAXTHREADS_KEY =
|
||||
"hbase.replication.bulkload.copy.maxthreads";
|
||||
public static final int REPLICATION_BULKLOAD_COPY_MAXTHREADS_DEFAULT = 10;
|
||||
/** Number of hfiles to copy per thread during replication */
|
||||
public static final String REPLICATION_BULKLOAD_COPY_HFILES_PERTHREAD_KEY =
|
||||
"hbase.replication.bulkload.copy.hfiles.perthread";
|
||||
public static final int REPLICATION_BULKLOAD_COPY_HFILES_PERTHREAD_DEFAULT = 10;
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(HFileReplicator.class);
|
||||
private final String UNDERSCORE = "_";
|
||||
private final static FsPermission PERM_ALL_ACCESS = FsPermission.valueOf("-rwxrwxrwx");
|
||||
|
||||
private Configuration sourceClusterConf;
|
||||
private String sourceBaseNamespaceDirPath;
|
||||
private String sourceHFileArchiveDirPath;
|
||||
private Map<String, List<Pair<byte[], List<String>>>> bulkLoadHFileMap;
|
||||
private FileSystem sinkFs;
|
||||
private FsDelegationToken fsDelegationToken;
|
||||
private UserProvider userProvider;
|
||||
private Configuration conf;
|
||||
private Connection connection;
|
||||
private String hbaseStagingDir;
|
||||
private ThreadPoolExecutor exec;
|
||||
private int maxCopyThreads;
|
||||
private int copiesPerThread;
|
||||
|
||||
public HFileReplicator(Configuration sourceClusterConf,
|
||||
String sourceBaseNamespaceDirPath, String sourceHFileArchiveDirPath,
|
||||
Map<String, List<Pair<byte[], List<String>>>> tableQueueMap, Configuration conf,
|
||||
Connection connection) throws IOException {
|
||||
this.sourceClusterConf = sourceClusterConf;
|
||||
this.sourceBaseNamespaceDirPath = sourceBaseNamespaceDirPath;
|
||||
this.sourceHFileArchiveDirPath = sourceHFileArchiveDirPath;
|
||||
this.bulkLoadHFileMap = tableQueueMap;
|
||||
this.conf = conf;
|
||||
this.connection = connection;
|
||||
|
||||
userProvider = UserProvider.instantiate(conf);
|
||||
fsDelegationToken = new FsDelegationToken(userProvider, "renewer");
|
||||
this.hbaseStagingDir = conf.get("hbase.bulkload.staging.dir");
|
||||
this.maxCopyThreads =
|
||||
this.conf.getInt(REPLICATION_BULKLOAD_COPY_MAXTHREADS_KEY,
|
||||
REPLICATION_BULKLOAD_COPY_MAXTHREADS_DEFAULT);
|
||||
ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
|
||||
builder.setNameFormat("HFileReplicationCallable-%1$d");
|
||||
this.exec =
|
||||
new ThreadPoolExecutor(1, maxCopyThreads, 60, TimeUnit.SECONDS,
|
||||
new LinkedBlockingQueue<Runnable>(), builder.build());
|
||||
this.exec.allowCoreThreadTimeOut(true);
|
||||
this.copiesPerThread =
|
||||
conf.getInt(REPLICATION_BULKLOAD_COPY_HFILES_PERTHREAD_KEY,
|
||||
REPLICATION_BULKLOAD_COPY_HFILES_PERTHREAD_DEFAULT);
|
||||
|
||||
sinkFs = FileSystem.get(conf);
|
||||
}
|
||||
|
||||
public Void replicate() throws IOException {
|
||||
// Copy all the hfiles to the local file system
|
||||
Map<String, Path> tableStagingDirsMap = copyHFilesToStagingDir();
|
||||
|
||||
int maxRetries = conf.getInt(HConstants.BULKLOAD_MAX_RETRIES_NUMBER, 10);
|
||||
|
||||
for (Entry<String, Path> tableStagingDir : tableStagingDirsMap.entrySet()) {
|
||||
String tableNameString = tableStagingDir.getKey();
|
||||
Path stagingDir = tableStagingDir.getValue();
|
||||
|
||||
LoadIncrementalHFiles loadHFiles = null;
|
||||
try {
|
||||
loadHFiles = new LoadIncrementalHFiles(conf);
|
||||
} catch (Exception e) {
|
||||
LOG.error("Failed to initialize LoadIncrementalHFiles for replicating bulk loaded"
|
||||
+ " data.", e);
|
||||
throw new IOException(e);
|
||||
}
|
||||
Configuration newConf = HBaseConfiguration.create(conf);
|
||||
newConf.set(LoadIncrementalHFiles.CREATE_TABLE_CONF_KEY, "no");
|
||||
loadHFiles.setConf(newConf);
|
||||
|
||||
TableName tableName = TableName.valueOf(tableNameString);
|
||||
Table table = this.connection.getTable(tableName);
|
||||
|
||||
// Prepare collection of queue of hfiles to be loaded(replicated)
|
||||
Deque<LoadQueueItem> queue = new LinkedList<LoadQueueItem>();
|
||||
loadHFiles.prepareHFileQueue(stagingDir, table, queue, false);
|
||||
|
||||
if (queue.isEmpty()) {
|
||||
LOG.warn("Replication process did not find any files to replicate in directory "
|
||||
+ stagingDir.toUri());
|
||||
return null;
|
||||
}
|
||||
|
||||
try (RegionLocator locator = connection.getRegionLocator(tableName)) {
|
||||
|
||||
fsDelegationToken.acquireDelegationToken(sinkFs);
|
||||
|
||||
// Set the staging directory which will be used by LoadIncrementalHFiles for loading the
|
||||
// data
|
||||
loadHFiles.setBulkToken(stagingDir.toString());
|
||||
|
||||
doBulkLoad(loadHFiles, table, queue, locator, maxRetries);
|
||||
} finally {
|
||||
cleanup(stagingDir.toString(), table);
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
private void doBulkLoad(LoadIncrementalHFiles loadHFiles, Table table,
|
||||
Deque<LoadQueueItem> queue, RegionLocator locator, int maxRetries) throws IOException {
|
||||
int count = 0;
|
||||
Pair<byte[][], byte[][]> startEndKeys;
|
||||
while (!queue.isEmpty()) {
|
||||
// need to reload split keys each iteration.
|
||||
startEndKeys = locator.getStartEndKeys();
|
||||
if (count != 0) {
|
||||
LOG.warn("Error occured while replicating HFiles, retry attempt " + count + " with "
|
||||
+ queue.size() + " files still remaining to replicate.");
|
||||
}
|
||||
|
||||
if (maxRetries != 0 && count >= maxRetries) {
|
||||
throw new IOException("Retry attempted " + count
|
||||
+ " times without completing, bailing out.");
|
||||
}
|
||||
count++;
|
||||
|
||||
// Try bulk load
|
||||
loadHFiles.loadHFileQueue(table, connection, queue, startEndKeys);
|
||||
}
|
||||
}
|
||||
|
||||
private void cleanup(String stagingDir, Table table) {
|
||||
// Release the file system delegation token
|
||||
fsDelegationToken.releaseDelegationToken();
|
||||
// Delete the staging directory
|
||||
if (stagingDir != null) {
|
||||
try {
|
||||
sinkFs.delete(new Path(stagingDir), true);
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Failed to delete the staging directory " + stagingDir, e);
|
||||
}
|
||||
}
|
||||
// Do not close the file system
|
||||
|
||||
/*
|
||||
* if (sinkFs != null) { try { sinkFs.close(); } catch (IOException e) { LOG.warn(
|
||||
* "Failed to close the file system"); } }
|
||||
*/
|
||||
|
||||
// Close the table
|
||||
if (table != null) {
|
||||
try {
|
||||
table.close();
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Failed to close the table.", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private Map<String, Path> copyHFilesToStagingDir() throws IOException {
|
||||
Map<String, Path> mapOfCopiedHFiles = new HashMap<String, Path>();
|
||||
Pair<byte[], List<String>> familyHFilePathsPair;
|
||||
List<String> hfilePaths;
|
||||
byte[] family;
|
||||
Path familyStagingDir;
|
||||
int familyHFilePathsPairsListSize;
|
||||
int totalNoOfHFiles;
|
||||
List<Pair<byte[], List<String>>> familyHFilePathsPairsList;
|
||||
FileSystem sourceFs = null;
|
||||
|
||||
try {
|
||||
Path sourceClusterPath = new Path(sourceBaseNamespaceDirPath);
|
||||
/*
|
||||
* Path#getFileSystem will by default get the FS from cache. If both source and sink cluster
|
||||
* has same FS name service then it will return peer cluster FS. To avoid this we explicitly
|
||||
* disable the loading of FS from cache, so that a new FS is created with source cluster
|
||||
* configuration.
|
||||
*/
|
||||
String sourceScheme = sourceClusterPath.toUri().getScheme();
|
||||
String disableCacheName =
|
||||
String.format("fs.%s.impl.disable.cache", new Object[] { sourceScheme });
|
||||
sourceClusterConf.setBoolean(disableCacheName, true);
|
||||
|
||||
sourceFs = sourceClusterPath.getFileSystem(sourceClusterConf);
|
||||
|
||||
User user = userProvider.getCurrent();
|
||||
// For each table name in the map
|
||||
for (Entry<String, List<Pair<byte[], List<String>>>> tableEntry : bulkLoadHFileMap
|
||||
.entrySet()) {
|
||||
String tableName = tableEntry.getKey();
|
||||
|
||||
// Create staging directory for each table
|
||||
Path stagingDir =
|
||||
createStagingDir(new Path(hbaseStagingDir), user, TableName.valueOf(tableName));
|
||||
|
||||
familyHFilePathsPairsList = tableEntry.getValue();
|
||||
familyHFilePathsPairsListSize = familyHFilePathsPairsList.size();
|
||||
|
||||
// For each list of family hfile paths pair in the table
|
||||
for (int i = 0; i < familyHFilePathsPairsListSize; i++) {
|
||||
familyHFilePathsPair = familyHFilePathsPairsList.get(i);
|
||||
|
||||
family = familyHFilePathsPair.getFirst();
|
||||
hfilePaths = familyHFilePathsPair.getSecond();
|
||||
|
||||
familyStagingDir = new Path(stagingDir, Bytes.toString(family));
|
||||
totalNoOfHFiles = hfilePaths.size();
|
||||
|
||||
// For each list of hfile paths for the family
|
||||
List<Future<Void>> futures = new ArrayList<Future<Void>>();
|
||||
Callable<Void> c;
|
||||
Future<Void> future;
|
||||
int currentCopied = 0;
|
||||
// Copy the hfiles parallely
|
||||
while (totalNoOfHFiles > currentCopied + this.copiesPerThread) {
|
||||
c =
|
||||
new Copier(sourceFs, familyStagingDir, hfilePaths.subList(currentCopied,
|
||||
currentCopied + this.copiesPerThread));
|
||||
future = exec.submit(c);
|
||||
futures.add(future);
|
||||
currentCopied += this.copiesPerThread;
|
||||
}
|
||||
|
||||
int remaining = totalNoOfHFiles - currentCopied;
|
||||
if (remaining > 0) {
|
||||
c =
|
||||
new Copier(sourceFs, familyStagingDir, hfilePaths.subList(currentCopied,
|
||||
currentCopied + remaining));
|
||||
future = exec.submit(c);
|
||||
futures.add(future);
|
||||
}
|
||||
|
||||
for (Future<Void> f : futures) {
|
||||
try {
|
||||
f.get();
|
||||
} catch (InterruptedException e) {
|
||||
InterruptedIOException iioe =
|
||||
new InterruptedIOException(
|
||||
"Failed to copy HFiles to local file system. This will be retried again "
|
||||
+ "by the source cluster.");
|
||||
iioe.initCause(e);
|
||||
throw iioe;
|
||||
} catch (ExecutionException e) {
|
||||
throw new IOException("Failed to copy HFiles to local file system. This will "
|
||||
+ "be retried again by the source cluster.", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
// Add the staging directory to this table. Staging directory contains all the hfiles
|
||||
// belonging to this table
|
||||
mapOfCopiedHFiles.put(tableName, stagingDir);
|
||||
}
|
||||
return mapOfCopiedHFiles;
|
||||
} finally {
|
||||
if (sourceFs != null) {
|
||||
sourceFs.close();
|
||||
}
|
||||
if(exec != null) {
|
||||
exec.shutdown();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private Path createStagingDir(Path baseDir, User user, TableName tableName) throws IOException {
|
||||
String tblName = tableName.getNameAsString().replace(":", UNDERSCORE);
|
||||
int RANDOM_WIDTH = 320;
|
||||
int RANDOM_RADIX = 32;
|
||||
String doubleUnderScore = UNDERSCORE + UNDERSCORE;
|
||||
String randomDir = user.getShortName() + doubleUnderScore + tblName + doubleUnderScore
|
||||
+ (new BigInteger(RANDOM_WIDTH, new SecureRandom()).toString(RANDOM_RADIX));
|
||||
return createStagingDir(baseDir, user, randomDir);
|
||||
}
|
||||
|
||||
private Path createStagingDir(Path baseDir, User user, String randomDir) throws IOException {
|
||||
Path p = new Path(baseDir, randomDir);
|
||||
sinkFs.mkdirs(p, PERM_ALL_ACCESS);
|
||||
sinkFs.setPermission(p, PERM_ALL_ACCESS);
|
||||
return p;
|
||||
}
|
||||
|
||||
/**
|
||||
* This class will copy the given hfiles from the given source file system to the given local file
|
||||
* system staging directory.
|
||||
*/
|
||||
private class Copier implements Callable<Void> {
|
||||
private FileSystem sourceFs;
|
||||
private Path stagingDir;
|
||||
private List<String> hfiles;
|
||||
|
||||
public Copier(FileSystem sourceFs, final Path stagingDir, final List<String> hfiles)
|
||||
throws IOException {
|
||||
this.sourceFs = sourceFs;
|
||||
this.stagingDir = stagingDir;
|
||||
this.hfiles = hfiles;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Void call() throws IOException {
|
||||
Path sourceHFilePath;
|
||||
Path localHFilePath;
|
||||
int totalHFiles = hfiles.size();
|
||||
for (int i = 0; i < totalHFiles; i++) {
|
||||
sourceHFilePath = new Path(sourceBaseNamespaceDirPath, hfiles.get(i));
|
||||
localHFilePath = new Path(stagingDir, sourceHFilePath.getName());
|
||||
try {
|
||||
FileUtil.copy(sourceFs, sourceHFilePath, sinkFs, localHFilePath, false, conf);
|
||||
// If any other exception other than FNFE then we will fail the replication requests and
|
||||
// source will retry to replicate these data.
|
||||
} catch (FileNotFoundException e) {
|
||||
LOG.info("Failed to copy hfile from " + sourceHFilePath + " to " + localHFilePath
|
||||
+ ". Trying to copy from hfile archive directory.",
|
||||
e);
|
||||
sourceHFilePath = new Path(sourceHFileArchiveDirPath, hfiles.get(i));
|
||||
|
||||
try {
|
||||
FileUtil.copy(sourceFs, sourceHFilePath, sinkFs, localHFilePath, false, conf);
|
||||
} catch (FileNotFoundException e1) {
|
||||
// This will mean that the hfile does not exists any where in source cluster FS. So we
|
||||
// cannot do anything here just log and return.
|
||||
LOG.error("Failed to copy hfile from " + sourceHFilePath + " to " + localHFilePath
|
||||
+ ". Hence ignoring this hfile from replication..",
|
||||
e1);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
sinkFs.setPermission(localHFilePath, PERM_ALL_ACCESS);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -71,6 +71,17 @@ public class MetricsSink {
|
|||
mss.incrAppliedOps(batchSize);
|
||||
}
|
||||
|
||||
/**
|
||||
* Convience method to change metrics when a batch of operations are applied.
|
||||
*
|
||||
* @param batchSize total number of mutations that are applied/replicated
|
||||
* @param hfileSize total number of hfiles that are applied/replicated
|
||||
*/
|
||||
public void applyBatch(long batchSize, long hfileSize) {
|
||||
applyBatch(batchSize);
|
||||
mss.incrAppliedHFiles(hfileSize);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the Age of Last Applied Op
|
||||
* @return ageOfLastAppliedOp
|
||||
|
|
|
@ -40,11 +40,13 @@ public class MetricsSource {
|
|||
// tracks last shipped timestamp for each wal group
|
||||
private Map<String, Long> lastTimeStamps = new HashMap<String, Long>();
|
||||
private int lastQueueSize = 0;
|
||||
private long lastHFileRefsQueueSize = 0;
|
||||
private String id;
|
||||
|
||||
private final MetricsReplicationSourceSource singleSourceSource;
|
||||
private final MetricsReplicationSourceSource globalSourceSource;
|
||||
|
||||
|
||||
/**
|
||||
* Constructor used to register the metrics
|
||||
*
|
||||
|
@ -143,6 +145,18 @@ public class MetricsSource {
|
|||
globalSourceSource.incrShippedKBs(sizeInKB);
|
||||
}
|
||||
|
||||
/**
|
||||
* Convience method to apply changes to metrics do to shipping a batch of logs.
|
||||
*
|
||||
* @param batchSize the size of the batch that was shipped to sinks.
|
||||
* @param hfiles total number of hfiles shipped to sinks.
|
||||
*/
|
||||
public void shipBatch(long batchSize, int sizeInKB, long hfiles) {
|
||||
shipBatch(batchSize, sizeInKB);
|
||||
singleSourceSource.incrHFilesShipped(hfiles);
|
||||
globalSourceSource.incrHFilesShipped(hfiles);
|
||||
}
|
||||
|
||||
/** increase the byte number read by source from log file */
|
||||
public void incrLogReadInBytes(long readInBytes) {
|
||||
singleSourceSource.incrLogReadInBytes(readInBytes);
|
||||
|
@ -153,8 +167,10 @@ public class MetricsSource {
|
|||
public void clear() {
|
||||
singleSourceSource.clear();
|
||||
globalSourceSource.decrSizeOfLogQueue(lastQueueSize);
|
||||
globalSourceSource.decrSizeOfHFileRefsQueue(lastHFileRefsQueueSize);
|
||||
lastTimeStamps.clear();
|
||||
lastQueueSize = 0;
|
||||
lastHFileRefsQueueSize = 0;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -194,4 +210,19 @@ public class MetricsSource {
|
|||
public String getPeerID() {
|
||||
return id;
|
||||
}
|
||||
|
||||
public void incrSizeOfHFileRefsQueue(long size) {
|
||||
singleSourceSource.incrSizeOfHFileRefsQueue(size);
|
||||
globalSourceSource.incrSizeOfHFileRefsQueue(size);
|
||||
lastHFileRefsQueueSize = size;
|
||||
}
|
||||
|
||||
public void decrSizeOfHFileRefsQueue(int size) {
|
||||
singleSourceSource.decrSizeOfHFileRefsQueue(size);
|
||||
globalSourceSource.decrSizeOfHFileRefsQueue(size);
|
||||
lastHFileRefsQueueSize -= size;
|
||||
if (lastHFileRefsQueueSize < 0) {
|
||||
lastHFileRefsQueueSize = 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -649,8 +649,8 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint {
|
|||
|
||||
// set the region name for the target region replica
|
||||
Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner> p =
|
||||
ReplicationProtbufUtil.buildReplicateWALEntryRequest(
|
||||
entriesArray, location.getRegionInfo().getEncodedNameAsBytes());
|
||||
ReplicationProtbufUtil.buildReplicateWALEntryRequest(entriesArray, location
|
||||
.getRegionInfo().getEncodedNameAsBytes(), null, null, null);
|
||||
try {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController(p.getSecond());
|
||||
controller.setCallTimeout(timeout);
|
||||
|
|
|
@ -35,6 +35,7 @@ import java.util.concurrent.TimeUnit;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
@ -44,7 +45,10 @@ import org.apache.hadoop.hbase.CellUtil;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
|
||||
import org.apache.hadoop.hbase.regionserver.ReplicationSinkService;
|
||||
import org.apache.hadoop.hbase.regionserver.ReplicationSourceService;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
|
@ -55,6 +59,7 @@ import org.apache.hadoop.hbase.replication.ReplicationFactory;
|
|||
import org.apache.hadoop.hbase.replication.ReplicationPeers;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationQueues;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationTracker;
|
||||
import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner;
|
||||
import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
|
||||
|
@ -71,6 +76,7 @@ public class Replication extends WALActionsListener.Base implements
|
|||
private static final Log LOG =
|
||||
LogFactory.getLog(Replication.class);
|
||||
private boolean replication;
|
||||
private boolean replicationForBulkLoadData;
|
||||
private ReplicationSourceManager replicationManager;
|
||||
private ReplicationQueues replicationQueues;
|
||||
private ReplicationPeers replicationPeers;
|
||||
|
@ -84,7 +90,6 @@ public class Replication extends WALActionsListener.Base implements
|
|||
private int statsThreadPeriod;
|
||||
// ReplicationLoad to access replication metrics
|
||||
private ReplicationLoad replicationLoad;
|
||||
|
||||
/**
|
||||
* Instantiate the replication management (if rep is enabled).
|
||||
* @param server Hosting server
|
||||
|
@ -109,11 +114,20 @@ public class Replication extends WALActionsListener.Base implements
|
|||
this.server = server;
|
||||
this.conf = this.server.getConfiguration();
|
||||
this.replication = isReplication(this.conf);
|
||||
this.replicationForBulkLoadData = isReplicationForBulkLoadDataEnabled(this.conf);
|
||||
this.scheduleThreadPool = Executors.newScheduledThreadPool(1,
|
||||
new ThreadFactoryBuilder()
|
||||
.setNameFormat(server.getServerName().toShortString() + "Replication Statistics #%d")
|
||||
.setDaemon(true)
|
||||
.build());
|
||||
if (this.replicationForBulkLoadData) {
|
||||
if (conf.get(HConstants.REPLICATION_CLUSTER_ID) == null
|
||||
|| conf.get(HConstants.REPLICATION_CLUSTER_ID).isEmpty()) {
|
||||
throw new IllegalArgumentException(HConstants.REPLICATION_CLUSTER_ID
|
||||
+ " cannot be null/empty when " + HConstants.REPLICATION_BULKLOAD_ENABLE_KEY
|
||||
+ " is set to true.");
|
||||
}
|
||||
}
|
||||
if (replication) {
|
||||
try {
|
||||
this.replicationQueues =
|
||||
|
@ -158,6 +172,15 @@ public class Replication extends WALActionsListener.Base implements
|
|||
return c.getBoolean(REPLICATION_ENABLE_KEY, HConstants.REPLICATION_ENABLE_DEFAULT);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param c Configuration to look at
|
||||
* @return True if replication for bulk load data is enabled.
|
||||
*/
|
||||
public static boolean isReplicationForBulkLoadDataEnabled(final Configuration c) {
|
||||
return c.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
|
||||
HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT);
|
||||
}
|
||||
|
||||
/*
|
||||
* Returns an object to listen to new wal changes
|
||||
**/
|
||||
|
@ -187,14 +210,22 @@ public class Replication extends WALActionsListener.Base implements
|
|||
/**
|
||||
* Carry on the list of log entries down to the sink
|
||||
* @param entries list of entries to replicate
|
||||
* @param cells The data -- the cells -- that <code>entries</code> describes (the entries
|
||||
* do not contain the Cells we are replicating; they are passed here on the side in this
|
||||
* @param cells The data -- the cells -- that <code>entries</code> describes (the entries do not
|
||||
* contain the Cells we are replicating; they are passed here on the side in this
|
||||
* CellScanner).
|
||||
* @param replicationClusterId Id which will uniquely identify source cluster FS client
|
||||
* configurations in the replication configuration directory
|
||||
* @param sourceBaseNamespaceDirPath Path that point to the source cluster base namespace
|
||||
* directory required for replicating hfiles
|
||||
* @param sourceHFileArchiveDirPath Path that point to the source cluster hfile archive directory
|
||||
* @throws IOException
|
||||
*/
|
||||
public void replicateLogEntries(List<WALEntry> entries, CellScanner cells) throws IOException {
|
||||
public void replicateLogEntries(List<WALEntry> entries, CellScanner cells,
|
||||
String replicationClusterId, String sourceBaseNamespaceDirPath,
|
||||
String sourceHFileArchiveDirPath) throws IOException {
|
||||
if (this.replication) {
|
||||
this.replicationSink.replicateEntries(entries, cells);
|
||||
this.replicationSink.replicateEntries(entries, cells, replicationClusterId,
|
||||
sourceBaseNamespaceDirPath, sourceHFileArchiveDirPath);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -226,41 +257,85 @@ public class Replication extends WALActionsListener.Base implements
|
|||
}
|
||||
|
||||
@Override
|
||||
public void visitLogEntryBeforeWrite(HTableDescriptor htd, WALKey logKey,
|
||||
WALEdit logEdit) {
|
||||
scopeWALEdits(htd, logKey, logEdit);
|
||||
public void visitLogEntryBeforeWrite(HTableDescriptor htd, WALKey logKey, WALEdit logEdit)
|
||||
throws IOException {
|
||||
scopeWALEdits(htd, logKey, logEdit, this.conf, this.getReplicationManager());
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility method used to set the correct scopes on each log key. Doesn't set a scope on keys
|
||||
* from compaction WAL edits and if the scope is local.
|
||||
* Utility method used to set the correct scopes on each log key. Doesn't set a scope on keys from
|
||||
* compaction WAL edits and if the scope is local.
|
||||
* @param htd Descriptor used to find the scope to use
|
||||
* @param logKey Key that may get scoped according to its edits
|
||||
* @param logEdit Edits used to lookup the scopes
|
||||
* @param replicationManager Manager used to add bulk load events hfile references
|
||||
* @throws IOException If failed to parse the WALEdit
|
||||
*/
|
||||
public static void scopeWALEdits(HTableDescriptor htd, WALKey logKey,
|
||||
WALEdit logEdit) {
|
||||
NavigableMap<byte[], Integer> scopes =
|
||||
new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
|
||||
public static void scopeWALEdits(HTableDescriptor htd, WALKey logKey, WALEdit logEdit,
|
||||
Configuration conf, ReplicationSourceManager replicationManager) throws IOException {
|
||||
NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
|
||||
byte[] family;
|
||||
boolean replicationForBulkLoadEnabled = isReplicationForBulkLoadDataEnabled(conf);
|
||||
for (Cell cell : logEdit.getCells()) {
|
||||
if (CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
|
||||
if (replicationForBulkLoadEnabled && CellUtil.matchingQualifier(cell, WALEdit.BULK_LOAD)) {
|
||||
scopeBulkLoadEdits(htd, replicationManager, scopes, logKey.getTablename(), cell);
|
||||
} else {
|
||||
// Skip the flush/compaction/region events
|
||||
continue;
|
||||
}
|
||||
} else {
|
||||
family = CellUtil.cloneFamily(cell);
|
||||
// This is expected and the KV should not be replicated
|
||||
if (CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) continue;
|
||||
// Unexpected, has a tendency to happen in unit tests
|
||||
assert htd.getFamily(family) != null;
|
||||
|
||||
if (!scopes.containsKey(family)) {
|
||||
int scope = htd.getFamily(family).getScope();
|
||||
if (scope != REPLICATION_SCOPE_LOCAL &&
|
||||
!scopes.containsKey(family)) {
|
||||
if (scope != REPLICATION_SCOPE_LOCAL) {
|
||||
scopes.put(family, scope);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (!scopes.isEmpty()) {
|
||||
logKey.setScopes(scopes);
|
||||
}
|
||||
}
|
||||
|
||||
private static void scopeBulkLoadEdits(HTableDescriptor htd,
|
||||
ReplicationSourceManager replicationManager, NavigableMap<byte[], Integer> scopes,
|
||||
TableName tableName, Cell cell) throws IOException {
|
||||
byte[] family;
|
||||
try {
|
||||
BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cell);
|
||||
for (StoreDescriptor s : bld.getStoresList()) {
|
||||
family = s.getFamilyName().toByteArray();
|
||||
if (!scopes.containsKey(family)) {
|
||||
int scope = htd.getFamily(family).getScope();
|
||||
if (scope != REPLICATION_SCOPE_LOCAL) {
|
||||
scopes.put(family, scope);
|
||||
addHFileRefsToQueue(replicationManager, tableName, family, s);
|
||||
}
|
||||
} else {
|
||||
addHFileRefsToQueue(replicationManager, tableName, family, s);
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.error("Failed to get bulk load events information from the wal file.", e);
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
private static void addHFileRefsToQueue(ReplicationSourceManager replicationManager,
|
||||
TableName tableName, byte[] family, StoreDescriptor s) throws IOException {
|
||||
try {
|
||||
replicationManager.addHFileRefs(tableName, family, s.getStoreFileList());
|
||||
} catch (ReplicationException e) {
|
||||
LOG.error("Failed to create hfile references in ZK.", e);
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void preLogRoll(Path oldPath, Path newPath) throws IOException {
|
||||
getReplicationManager().preLogRoll(newPath);
|
||||
|
@ -272,8 +347,7 @@ public class Replication extends WALActionsListener.Base implements
|
|||
}
|
||||
|
||||
/**
|
||||
* This method modifies the master's configuration in order to inject
|
||||
* replication-related features
|
||||
* This method modifies the master's configuration in order to inject replication-related features
|
||||
* @param conf
|
||||
*/
|
||||
public static void decorateMasterConfiguration(Configuration conf) {
|
||||
|
@ -285,6 +359,13 @@ public class Replication extends WALActionsListener.Base implements
|
|||
if (!plugins.contains(cleanerClass)) {
|
||||
conf.set(HBASE_MASTER_LOGCLEANER_PLUGINS, plugins + "," + cleanerClass);
|
||||
}
|
||||
if (isReplicationForBulkLoadDataEnabled(conf)) {
|
||||
plugins = conf.get(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS);
|
||||
cleanerClass = ReplicationHFileCleaner.class.getCanonicalName();
|
||||
if (!plugins.contains(cleanerClass)) {
|
||||
conf.set(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS, plugins + "," + cleanerClass);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
|
|
|
@ -33,15 +33,16 @@ import java.util.concurrent.atomic.AtomicLong;
|
|||
import org.apache.commons.lang.StringUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.Stoppable;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
|
@ -51,6 +52,11 @@ import org.apache.hadoop.hbase.client.Row;
|
|||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
|
@ -78,6 +84,9 @@ public class ReplicationSink {
|
|||
private final MetricsSink metrics;
|
||||
private final AtomicLong totalReplicatedEdits = new AtomicLong();
|
||||
private final Object sharedHtableConLock = new Object();
|
||||
// Number of hfiles that we successfully replicated
|
||||
private long hfilesReplicated = 0;
|
||||
private SourceFSConfigurationProvider provider;
|
||||
|
||||
/**
|
||||
* Create a sink for replication
|
||||
|
@ -91,6 +100,18 @@ public class ReplicationSink {
|
|||
this.conf = HBaseConfiguration.create(conf);
|
||||
decorateConf();
|
||||
this.metrics = new MetricsSink();
|
||||
|
||||
String className =
|
||||
conf.get("hbase.replication.source.fs.conf.provider",
|
||||
DefaultSourceFSConfigurationProvider.class.getCanonicalName());
|
||||
try {
|
||||
@SuppressWarnings("rawtypes")
|
||||
Class c = Class.forName(className);
|
||||
this.provider = (SourceFSConfigurationProvider) c.newInstance();
|
||||
} catch (Exception e) {
|
||||
throw new IllegalArgumentException("Configured source fs configuration provider class "
|
||||
+ className + " throws error.", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -113,9 +134,16 @@ public class ReplicationSink {
|
|||
* operates against raw protobuf type saving on a conversion from pb to pojo.
|
||||
* @param entries
|
||||
* @param cells
|
||||
* @throws IOException
|
||||
* @param replicationClusterId Id which will uniquely identify source cluster FS client
|
||||
* configurations in the replication configuration directory
|
||||
* @param sourceBaseNamespaceDirPath Path that point to the source cluster base namespace
|
||||
* directory
|
||||
* @param sourceHFileArchiveDirPath Path that point to the source cluster hfile archive directory
|
||||
* @throws IOException If failed to replicate the data
|
||||
*/
|
||||
public void replicateEntries(List<WALEntry> entries, final CellScanner cells) throws IOException {
|
||||
public void replicateEntries(List<WALEntry> entries, final CellScanner cells,
|
||||
String replicationClusterId, String sourceBaseNamespaceDirPath,
|
||||
String sourceHFileArchiveDirPath) throws IOException {
|
||||
if (entries.isEmpty()) return;
|
||||
if (cells == null) throw new NullPointerException("TODO: Add handling of null CellScanner");
|
||||
// Very simple optimization where we batch sequences of rows going
|
||||
|
@ -126,6 +154,10 @@ public class ReplicationSink {
|
|||
// invocation of this method per table and cluster id.
|
||||
Map<TableName, Map<List<UUID>, List<Row>>> rowMap =
|
||||
new TreeMap<TableName, Map<List<UUID>, List<Row>>>();
|
||||
|
||||
// Map of table name Vs list of pair of family and list of hfile paths from its namespace
|
||||
Map<String, List<Pair<byte[], List<String>>>> bulkLoadHFileMap = null;
|
||||
|
||||
for (WALEntry entry : entries) {
|
||||
TableName table =
|
||||
TableName.valueOf(entry.getKey().getTableName().toByteArray());
|
||||
|
@ -138,11 +170,20 @@ public class ReplicationSink {
|
|||
throw new ArrayIndexOutOfBoundsException("Expected=" + count + ", index=" + i);
|
||||
}
|
||||
Cell cell = cells.current();
|
||||
// Handle bulk load hfiles replication
|
||||
if (CellUtil.matchingQualifier(cell, WALEdit.BULK_LOAD)) {
|
||||
if (bulkLoadHFileMap == null) {
|
||||
bulkLoadHFileMap = new HashMap<String, List<Pair<byte[], List<String>>>>();
|
||||
}
|
||||
buildBulkLoadHFileMap(bulkLoadHFileMap, table, cell);
|
||||
} else {
|
||||
// Handle wal replication
|
||||
if (isNewRowOrType(previousCell, cell)) {
|
||||
// Create new mutation
|
||||
m = CellUtil.isDelete(cell)?
|
||||
new Delete(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()):
|
||||
new Put(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
|
||||
m =
|
||||
CellUtil.isDelete(cell) ? new Delete(cell.getRowArray(), cell.getRowOffset(),
|
||||
cell.getRowLength()) : new Put(cell.getRowArray(), cell.getRowOffset(),
|
||||
cell.getRowLength());
|
||||
List<UUID> clusterIds = new ArrayList<UUID>();
|
||||
for (HBaseProtos.UUID clusterId : entry.getKey().getClusterIdsList()) {
|
||||
clusterIds.add(toUUID(clusterId));
|
||||
|
@ -157,14 +198,32 @@ public class ReplicationSink {
|
|||
}
|
||||
previousCell = cell;
|
||||
}
|
||||
}
|
||||
totalReplicated++;
|
||||
}
|
||||
|
||||
// TODO Replicating mutations and bulk loaded data can be made parallel
|
||||
if (!rowMap.isEmpty()) {
|
||||
LOG.debug("Started replicating mutations.");
|
||||
for (Entry<TableName, Map<List<UUID>, List<Row>>> entry : rowMap.entrySet()) {
|
||||
batch(entry.getKey(), entry.getValue().values());
|
||||
}
|
||||
LOG.debug("Finished replicating mutations.");
|
||||
}
|
||||
|
||||
if (bulkLoadHFileMap != null && !bulkLoadHFileMap.isEmpty()) {
|
||||
LOG.debug("Started replicating bulk loaded data.");
|
||||
HFileReplicator hFileReplicator =
|
||||
new HFileReplicator(this.provider.getConf(this.conf, replicationClusterId),
|
||||
sourceBaseNamespaceDirPath, sourceHFileArchiveDirPath, bulkLoadHFileMap, conf,
|
||||
getConnection());
|
||||
hFileReplicator.replicate();
|
||||
LOG.debug("Finished replicating bulk loaded data.");
|
||||
}
|
||||
|
||||
int size = entries.size();
|
||||
this.metrics.setAgeOfLastAppliedOp(entries.get(size - 1).getKey().getWriteTime());
|
||||
this.metrics.applyBatch(size);
|
||||
this.metrics.applyBatch(size + hfilesReplicated, hfilesReplicated);
|
||||
this.totalReplicatedEdits.addAndGet(totalReplicated);
|
||||
} catch (IOException ex) {
|
||||
LOG.error("Unable to accept edit because:", ex);
|
||||
|
@ -172,6 +231,76 @@ public class ReplicationSink {
|
|||
}
|
||||
}
|
||||
|
||||
private void buildBulkLoadHFileMap(
|
||||
final Map<String, List<Pair<byte[], List<String>>>> bulkLoadHFileMap, TableName table,
|
||||
Cell cell) throws IOException {
|
||||
BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cell);
|
||||
List<StoreDescriptor> storesList = bld.getStoresList();
|
||||
int storesSize = storesList.size();
|
||||
for (int j = 0; j < storesSize; j++) {
|
||||
StoreDescriptor storeDescriptor = storesList.get(j);
|
||||
List<String> storeFileList = storeDescriptor.getStoreFileList();
|
||||
int storeFilesSize = storeFileList.size();
|
||||
hfilesReplicated += storeFilesSize;
|
||||
for (int k = 0; k < storeFilesSize; k++) {
|
||||
byte[] family = storeDescriptor.getFamilyName().toByteArray();
|
||||
|
||||
// Build hfile relative path from its namespace
|
||||
String pathToHfileFromNS = getHFilePath(table, bld, storeFileList.get(k), family);
|
||||
|
||||
String tableName = table.getNameWithNamespaceInclAsString();
|
||||
if (bulkLoadHFileMap.containsKey(tableName)) {
|
||||
List<Pair<byte[], List<String>>> familyHFilePathsList = bulkLoadHFileMap.get(tableName);
|
||||
boolean foundFamily = false;
|
||||
for (int i = 0; i < familyHFilePathsList.size(); i++) {
|
||||
Pair<byte[], List<String>> familyHFilePathsPair = familyHFilePathsList.get(i);
|
||||
if (Bytes.equals(familyHFilePathsPair.getFirst(), family)) {
|
||||
// Found family already present, just add the path to the existing list
|
||||
familyHFilePathsPair.getSecond().add(pathToHfileFromNS);
|
||||
foundFamily = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (!foundFamily) {
|
||||
// Family not found, add this family and its hfile paths pair to the list
|
||||
addFamilyAndItsHFilePathToTableInMap(family, pathToHfileFromNS, familyHFilePathsList);
|
||||
}
|
||||
} else {
|
||||
// Add this table entry into the map
|
||||
addNewTableEntryInMap(bulkLoadHFileMap, family, pathToHfileFromNS, tableName);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void addFamilyAndItsHFilePathToTableInMap(byte[] family, String pathToHfileFromNS,
|
||||
List<Pair<byte[], List<String>>> familyHFilePathsList) {
|
||||
List<String> hfilePaths = new ArrayList<String>();
|
||||
hfilePaths.add(pathToHfileFromNS);
|
||||
familyHFilePathsList.add(new Pair<byte[], List<String>>(family, hfilePaths));
|
||||
}
|
||||
|
||||
private void addNewTableEntryInMap(
|
||||
final Map<String, List<Pair<byte[], List<String>>>> bulkLoadHFileMap, byte[] family,
|
||||
String pathToHfileFromNS, String tableName) {
|
||||
List<String> hfilePaths = new ArrayList<String>();
|
||||
hfilePaths.add(pathToHfileFromNS);
|
||||
Pair<byte[], List<String>> newFamilyHFilePathsPair =
|
||||
new Pair<byte[], List<String>>(family, hfilePaths);
|
||||
List<Pair<byte[], List<String>>> newFamilyHFilePathsList =
|
||||
new ArrayList<Pair<byte[], List<String>>>();
|
||||
newFamilyHFilePathsList.add(newFamilyHFilePathsPair);
|
||||
bulkLoadHFileMap.put(tableName, newFamilyHFilePathsList);
|
||||
}
|
||||
|
||||
private String getHFilePath(TableName table, BulkLoadDescriptor bld, String storeFile,
|
||||
byte[] family) {
|
||||
return new StringBuilder(100).append(table.getNamespaceAsString()).append(Path.SEPARATOR)
|
||||
.append(table.getQualifierAsString()).append(Path.SEPARATOR)
|
||||
.append(Bytes.toString(bld.getEncodedRegionName().toByteArray())).append(Path.SEPARATOR)
|
||||
.append(Bytes.toString(family)).append(Path.SEPARATOR).append(storeFile).toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* @param previousCell
|
||||
* @param cell
|
||||
|
@ -241,16 +370,7 @@ public class ReplicationSink {
|
|||
}
|
||||
Table table = null;
|
||||
try {
|
||||
// See https://en.wikipedia.org/wiki/Double-checked_locking
|
||||
Connection connection = this.sharedHtableCon;
|
||||
if (connection == null) {
|
||||
synchronized (sharedHtableConLock) {
|
||||
connection = this.sharedHtableCon;
|
||||
if (connection == null) {
|
||||
connection = this.sharedHtableCon = ConnectionFactory.createConnection(this.conf);
|
||||
}
|
||||
}
|
||||
}
|
||||
Connection connection = getConnection();
|
||||
table = connection.getTable(tableName);
|
||||
for (List<Row> rows : allRows) {
|
||||
table.batch(rows, null);
|
||||
|
@ -264,6 +384,20 @@ public class ReplicationSink {
|
|||
}
|
||||
}
|
||||
|
||||
private Connection getConnection() throws IOException {
|
||||
// See https://en.wikipedia.org/wiki/Double-checked_locking
|
||||
Connection connection = sharedHtableCon;
|
||||
if (connection == null) {
|
||||
synchronized (sharedHtableConLock) {
|
||||
connection = sharedHtableCon;
|
||||
if (connection == null) {
|
||||
connection = sharedHtableCon = ConnectionFactory.createConnection(conf);
|
||||
}
|
||||
}
|
||||
}
|
||||
return connection;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a string representation of this sink's metrics
|
||||
* @return string with the total replicated edits count and the date
|
||||
|
|
|
@ -37,7 +37,6 @@ import java.util.concurrent.atomic.AtomicLong;
|
|||
import org.apache.commons.lang.StringUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
@ -47,9 +46,10 @@ import org.apache.hadoop.hbase.CellUtil;
|
|||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.Stoppable;
|
||||
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.replication.ChainWALEntryFilter;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
|
||||
|
@ -59,8 +59,12 @@ import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
|
|||
import org.apache.hadoop.hbase.replication.ReplicationQueues;
|
||||
import org.apache.hadoop.hbase.replication.SystemTableWALEntryFilter;
|
||||
import org.apache.hadoop.hbase.replication.WALEntryFilter;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
|
@ -223,6 +227,34 @@ public class ReplicationSource extends Thread
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addHFileRefs(TableName tableName, byte[] family, List<String> files)
|
||||
throws ReplicationException {
|
||||
String peerId = peerClusterZnode;
|
||||
if (peerId.contains("-")) {
|
||||
// peerClusterZnode will be in the form peerId + "-" + rsZNode.
|
||||
// A peerId will not have "-" in its name, see HBASE-11394
|
||||
peerId = peerClusterZnode.split("-")[0];
|
||||
}
|
||||
Map<TableName, List<String>> tableCFMap = replicationPeers.getPeer(peerId).getTableCFs();
|
||||
if (tableCFMap != null) {
|
||||
List<String> tableCfs = tableCFMap.get(tableName);
|
||||
if (tableCFMap.containsKey(tableName)
|
||||
&& (tableCfs == null || tableCfs.contains(Bytes.toString(family)))) {
|
||||
this.replicationQueues.addHFileRefs(peerId, files);
|
||||
metrics.incrSizeOfHFileRefsQueue(files.size());
|
||||
} else {
|
||||
LOG.debug("HFiles will not be replicated belonging to the table " + tableName + " family "
|
||||
+ Bytes.toString(family) + " to peer id " + peerId);
|
||||
}
|
||||
} else {
|
||||
// user has explicitly not defined any table cfs for replication, means replicate all the
|
||||
// data
|
||||
this.replicationQueues.addHFileRefs(peerId, files);
|
||||
metrics.incrSizeOfHFileRefsQueue(files.size());
|
||||
}
|
||||
}
|
||||
|
||||
private void uninitialize() {
|
||||
LOG.debug("Source exiting " + this.peerId);
|
||||
metrics.clear();
|
||||
|
@ -471,6 +503,8 @@ public class ReplicationSource extends Thread
|
|||
private int currentSize = 0;
|
||||
// Indicates whether this particular worker is running
|
||||
private boolean workerRunning = true;
|
||||
// Current number of hfiles that we need to replicate
|
||||
private long currentNbHFiles = 0;
|
||||
|
||||
public ReplicationSourceWorkerThread(String walGroupId,
|
||||
PriorityBlockingQueue<Path> queue, ReplicationQueueInfo replicationQueueInfo, ReplicationSource source) {
|
||||
|
@ -550,6 +584,7 @@ public class ReplicationSource extends Thread
|
|||
|
||||
boolean gotIOE = false;
|
||||
currentNbOperations = 0;
|
||||
currentNbHFiles = 0;
|
||||
List<WAL.Entry> entries = new ArrayList<WAL.Entry>(1);
|
||||
currentSize = 0;
|
||||
try {
|
||||
|
@ -701,6 +736,28 @@ public class ReplicationSource extends Thread
|
|||
return seenEntries == 0 && processEndOfFile();
|
||||
}
|
||||
|
||||
private void cleanUpHFileRefs(WALEdit edit) throws IOException {
|
||||
String peerId = peerClusterZnode;
|
||||
if (peerId.contains("-")) {
|
||||
// peerClusterZnode will be in the form peerId + "-" + rsZNode.
|
||||
// A peerId will not have "-" in its name, see HBASE-11394
|
||||
peerId = peerClusterZnode.split("-")[0];
|
||||
}
|
||||
List<Cell> cells = edit.getCells();
|
||||
for (int i = 0; i < cells.size(); i++) {
|
||||
Cell cell = cells.get(i);
|
||||
if (CellUtil.matchingQualifier(cell, WALEdit.BULK_LOAD)) {
|
||||
BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cell);
|
||||
List<StoreDescriptor> stores = bld.getStoresList();
|
||||
for (int j = 0; j < stores.size(); j++) {
|
||||
List<String> storeFileList = stores.get(j).getStoreFileList();
|
||||
manager.cleanUpHFileRefs(peerId, storeFileList);
|
||||
metrics.decrSizeOfHFileRefsQueue(storeFileList.size());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Poll for the next path
|
||||
* @return true if a path was obtained, false if not
|
||||
|
@ -853,14 +910,31 @@ public class ReplicationSource extends Thread
|
|||
private int countDistinctRowKeys(WALEdit edit) {
|
||||
List<Cell> cells = edit.getCells();
|
||||
int distinctRowKeys = 1;
|
||||
int totalHFileEntries = 0;
|
||||
Cell lastCell = cells.get(0);
|
||||
|
||||
for (int i = 0; i < edit.size(); i++) {
|
||||
// Count HFiles to be replicated
|
||||
if (CellUtil.matchingQualifier(cells.get(i), WALEdit.BULK_LOAD)) {
|
||||
try {
|
||||
BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cells.get(i));
|
||||
List<StoreDescriptor> stores = bld.getStoresList();
|
||||
for (int j = 0; j < stores.size(); j++) {
|
||||
totalHFileEntries += stores.get(j).getStoreFileList().size();
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.error("Failed to deserialize bulk load entry from wal edit. "
|
||||
+ "This its hfiles count will not be added into metric.");
|
||||
}
|
||||
}
|
||||
|
||||
if (!CellUtil.matchingRows(cells.get(i), lastCell)) {
|
||||
distinctRowKeys++;
|
||||
}
|
||||
lastCell = cells.get(i);
|
||||
}
|
||||
return distinctRowKeys;
|
||||
currentNbHFiles += totalHFileEntries;
|
||||
return distinctRowKeys + totalHFileEntries;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -914,6 +988,12 @@ public class ReplicationSource extends Thread
|
|||
}
|
||||
|
||||
if (this.lastLoggedPosition != this.repLogReader.getPosition()) {
|
||||
//Clean up hfile references
|
||||
int size = entries.size();
|
||||
for (int i = 0; i < size; i++) {
|
||||
cleanUpHFileRefs(entries.get(i).getEdit());
|
||||
}
|
||||
//Log and clean up WAL logs
|
||||
manager.logPositionAndCleanOldLogs(this.currentPath, peerClusterZnode,
|
||||
this.repLogReader.getPosition(), this.replicationQueueInfo.isQueueRecovered(),
|
||||
currentWALisBeingWrittenTo);
|
||||
|
@ -925,7 +1005,7 @@ public class ReplicationSource extends Thread
|
|||
totalReplicatedEdits.addAndGet(entries.size());
|
||||
totalReplicatedOperations.addAndGet(currentNbOperations);
|
||||
// FIXME check relationship between wal group and overall
|
||||
metrics.shipBatch(currentNbOperations, currentSize / 1024);
|
||||
metrics.shipBatch(currentNbOperations, currentSize / 1024, currentNbHFiles);
|
||||
metrics.setAgeOfLastShippedOp(entries.get(entries.size() - 1).getKey().getWriteTime(),
|
||||
walGroupId);
|
||||
if (LOG.isTraceEnabled()) {
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.hbase.replication.regionserver;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
@ -26,7 +27,9 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Stoppable;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationException;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationPeers;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationQueues;
|
||||
|
||||
|
@ -105,4 +108,14 @@ public interface ReplicationSourceInterface {
|
|||
*/
|
||||
String getStats();
|
||||
|
||||
/**
|
||||
* Add hfile names to the queue to be replicated.
|
||||
* @param tableName Name of the table these files belongs to
|
||||
* @param family Name of the family these files belong to
|
||||
* @param files files whose names needs to be added to the queue to be replicated
|
||||
* @throws ReplicationException If failed to add hfile references
|
||||
*/
|
||||
void addHFileRefs(TableName tableName, byte[] family, List<String> files)
|
||||
throws ReplicationException;
|
||||
|
||||
}
|
||||
|
|
|
@ -45,8 +45,10 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.TableDescriptors;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost;
|
||||
|
@ -225,8 +227,16 @@ public class ReplicationSourceManager implements ReplicationListener {
|
|||
* old region server wal queues
|
||||
*/
|
||||
protected void init() throws IOException, ReplicationException {
|
||||
boolean replicationForBulkLoadDataEnabled =
|
||||
conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
|
||||
HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT);
|
||||
for (String id : this.replicationPeers.getPeerIds()) {
|
||||
addSource(id);
|
||||
if (replicationForBulkLoadDataEnabled) {
|
||||
// Check if peer exists in hfile-refs queue, if not add it. This can happen in the case
|
||||
// when a peer was added before replication for bulk loaded data was enabled.
|
||||
this.replicationQueues.addPeerToHFileRefs(id);
|
||||
}
|
||||
}
|
||||
List<String> currentReplicators = this.replicationQueues.getListOfReplicators();
|
||||
if (currentReplicators == null || currentReplicators.size() == 0) {
|
||||
|
@ -733,4 +743,15 @@ public class ReplicationSourceManager implements ReplicationListener {
|
|||
}
|
||||
return stats.toString();
|
||||
}
|
||||
|
||||
public void addHFileRefs(TableName tableName, byte[] family, List<String> files)
|
||||
throws ReplicationException {
|
||||
for (ReplicationSourceInterface source : this.sources) {
|
||||
source.addHFileRefs(tableName, family, files);
|
||||
}
|
||||
}
|
||||
|
||||
public void cleanUpHFileRefs(String peerId, List<String> files) {
|
||||
this.replicationQueues.removeHFileRefs(peerId, files);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,40 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license
|
||||
* agreements. See the NOTICE file distributed with this work for additional information regarding
|
||||
* copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable
|
||||
* law or agreed to in writing, software distributed under the License is distributed on an "AS IS"
|
||||
* BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License
|
||||
* for the specific language governing permissions and limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.replication.regionserver;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* Interface that defines how a region server in peer cluster will get source cluster file system
|
||||
* configurations. User can configure their custom implementation implementing this interface by
|
||||
* setting the value of their custom implementation's fully qualified class name to
|
||||
* hbase.replication.source.fs.conf.provider property in RegionServer configuration. Default is
|
||||
* {@link DefaultSourceFSConfigurationProvider}
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION)
|
||||
public interface SourceFSConfigurationProvider {
|
||||
|
||||
/**
|
||||
* Returns the source cluster file system configuration for the given source cluster replication
|
||||
* ID.
|
||||
* @param sinkConf sink cluster configuration
|
||||
* @param replicationClusterId unique ID which identifies the source cluster
|
||||
* @return source cluster file system configuration
|
||||
* @throws IOException for invalid directory or for a bad disk.
|
||||
*/
|
||||
public Configuration getConf(Configuration sinkConf, String replicationClusterId)
|
||||
throws IOException;
|
||||
|
||||
}
|
|
@ -375,6 +375,14 @@ public class SecureBulkLoadEndpoint extends SecureBulkLoadService
|
|||
public String prepareBulkLoad(final byte[] family, final String srcPath) throws IOException {
|
||||
Path p = new Path(srcPath);
|
||||
Path stageP = new Path(stagingDir, new Path(Bytes.toString(family), p.getName()));
|
||||
|
||||
// In case of Replication for bulk load files, hfiles are already copied in staging directory
|
||||
if (p.equals(stageP)) {
|
||||
LOG.debug(p.getName()
|
||||
+ " is already available in staging directory. Skipping copy or rename.");
|
||||
return stageP.toString();
|
||||
}
|
||||
|
||||
if (srcFs == null) {
|
||||
srcFs = FileSystem.get(p.toUri(), conf);
|
||||
}
|
||||
|
@ -414,6 +422,14 @@ public class SecureBulkLoadEndpoint extends SecureBulkLoadService
|
|||
Path p = new Path(srcPath);
|
||||
Path stageP = new Path(stagingDir,
|
||||
new Path(Bytes.toString(family), p.getName()));
|
||||
|
||||
// In case of Replication for bulk load files, hfiles are not renamed by end point during
|
||||
// prepare stage, so no need of rename here again
|
||||
if (p.equals(stageP)) {
|
||||
LOG.debug(p.getName() + " is already available in source directory. Skipping rename.");
|
||||
return;
|
||||
}
|
||||
|
||||
LOG.debug("Moving " + stageP + " back to " + p);
|
||||
if(!fs.rename(stageP, p))
|
||||
throw new IOException("Failed to move HFile: " + stageP + " to " + p);
|
||||
|
|
|
@ -0,0 +1,264 @@
|
|||
/**
|
||||
* 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.master.cleaner;
|
||||
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.Field;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.ChoreService;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||
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.ReplicationPeers;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationQueues;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationQueuesZKImpl;
|
||||
import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner;
|
||||
import org.apache.hadoop.hbase.replication.regionserver.Replication;
|
||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
@Category({ MasterTests.class, SmallTests.class })
|
||||
public class TestReplicationHFileCleaner {
|
||||
private static final Log LOG = LogFactory.getLog(ReplicationQueuesZKImpl.class);
|
||||
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
private static Server server;
|
||||
private static ReplicationQueues rq;
|
||||
private static ReplicationPeers rp;
|
||||
private static final String peerId = "TestReplicationHFileCleaner";
|
||||
private static Configuration conf = TEST_UTIL.getConfiguration();
|
||||
static FileSystem fs = null;
|
||||
Path root;
|
||||
|
||||
/**
|
||||
* @throws java.lang.Exception
|
||||
*/
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
TEST_UTIL.startMiniZKCluster();
|
||||
server = new DummyServer();
|
||||
conf.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true);
|
||||
Replication.decorateMasterConfiguration(conf);
|
||||
rp = ReplicationFactory.getReplicationPeers(server.getZooKeeper(), conf, server);
|
||||
rp.init();
|
||||
|
||||
rq = ReplicationFactory.getReplicationQueues(server.getZooKeeper(), conf, server);
|
||||
rq.init(server.getServerName().toString());
|
||||
try {
|
||||
fs = FileSystem.get(conf);
|
||||
} finally {
|
||||
if (fs != null) {
|
||||
fs.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @throws java.lang.Exception
|
||||
*/
|
||||
@AfterClass
|
||||
public static void tearDownAfterClass() throws Exception {
|
||||
TEST_UTIL.shutdownMiniZKCluster();
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setup() throws ReplicationException, IOException {
|
||||
root = TEST_UTIL.getDataTestDirOnTestFS();
|
||||
rp.addPeer(peerId, new ReplicationPeerConfig().setClusterKey(TEST_UTIL.getClusterKey()), null);
|
||||
}
|
||||
|
||||
@After
|
||||
public void cleanup() throws ReplicationException {
|
||||
try {
|
||||
fs.delete(root, true);
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Failed to delete files recursively from path " + root);
|
||||
}
|
||||
rp.removePeer(peerId);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIsFileDeletable() throws IOException, ReplicationException {
|
||||
// 1. Create a file
|
||||
Path file = new Path(root, "testIsFileDeletableWithNoHFileRefs");
|
||||
fs.createNewFile(file);
|
||||
// 2. Assert file is successfully created
|
||||
assertTrue("Test file not created!", fs.exists(file));
|
||||
ReplicationHFileCleaner cleaner = new ReplicationHFileCleaner();
|
||||
cleaner.setConf(conf);
|
||||
// 3. Assert that file as is should be deletable
|
||||
assertTrue("Cleaner should allow to delete this file as there is no hfile reference node "
|
||||
+ "for it in the queue.",
|
||||
cleaner.isFileDeletable(fs.getFileStatus(file)));
|
||||
|
||||
List<String> files = new ArrayList<String>(1);
|
||||
files.add(file.getName());
|
||||
// 4. Add the file to hfile-refs queue
|
||||
rq.addHFileRefs(peerId, files);
|
||||
// 5. Assert file should not be deletable
|
||||
assertFalse("Cleaner should not allow to delete this file as there is a hfile reference node "
|
||||
+ "for it in the queue.",
|
||||
cleaner.isFileDeletable(fs.getFileStatus(file)));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetDeletableFiles() throws Exception {
|
||||
// 1. Create two files and assert that they do not exist
|
||||
Path notDeletablefile = new Path(root, "testGetDeletableFiles_1");
|
||||
fs.createNewFile(notDeletablefile);
|
||||
assertTrue("Test file not created!", fs.exists(notDeletablefile));
|
||||
Path deletablefile = new Path(root, "testGetDeletableFiles_2");
|
||||
fs.createNewFile(deletablefile);
|
||||
assertTrue("Test file not created!", fs.exists(deletablefile));
|
||||
|
||||
List<FileStatus> files = new ArrayList<FileStatus>(2);
|
||||
FileStatus f = new FileStatus();
|
||||
f.setPath(deletablefile);
|
||||
files.add(f);
|
||||
f = new FileStatus();
|
||||
f.setPath(notDeletablefile);
|
||||
files.add(f);
|
||||
|
||||
List<String> hfiles = new ArrayList<>(1);
|
||||
hfiles.add(notDeletablefile.getName());
|
||||
// 2. Add one file to hfile-refs queue
|
||||
rq.addHFileRefs(peerId, hfiles);
|
||||
|
||||
ReplicationHFileCleaner cleaner = new ReplicationHFileCleaner();
|
||||
cleaner.setConf(conf);
|
||||
Iterator<FileStatus> deletableFilesIterator = cleaner.getDeletableFiles(files).iterator();
|
||||
int i = 0;
|
||||
while (deletableFilesIterator.hasNext() && i < 2) {
|
||||
i++;
|
||||
}
|
||||
// 5. Assert one file should not be deletable and it is present in the list returned
|
||||
if (i > 2) {
|
||||
fail("File " + notDeletablefile
|
||||
+ " should not be deletable as its hfile reference node is not added.");
|
||||
}
|
||||
assertTrue(deletableFilesIterator.next().getPath().equals(deletablefile));
|
||||
}
|
||||
|
||||
/*
|
||||
* Test for HBASE-14621. This test will not assert directly anything. Without the fix the test
|
||||
* will end up in a infinite loop, so it will timeout.
|
||||
*/
|
||||
@Test(timeout = 15000)
|
||||
public void testForDifferntHFileRefsZnodeVersion() throws Exception {
|
||||
// 1. Create a file
|
||||
Path file = new Path(root, "testForDifferntHFileRefsZnodeVersion");
|
||||
fs.createNewFile(file);
|
||||
// 2. Assert file is successfully created
|
||||
assertTrue("Test file not created!", fs.exists(file));
|
||||
ReplicationHFileCleaner cleaner = new ReplicationHFileCleaner();
|
||||
cleaner.setConf(conf);
|
||||
|
||||
ReplicationQueuesClient replicationQueuesClient = Mockito.mock(ReplicationQueuesClient.class);
|
||||
//Return different znode version for each call
|
||||
Mockito.when(replicationQueuesClient.getHFileRefsNodeChangeVersion()).thenReturn(1, 2);
|
||||
|
||||
Class<? extends ReplicationHFileCleaner> cleanerClass = cleaner.getClass();
|
||||
Field rqc = cleanerClass.getDeclaredField("rqc");
|
||||
rqc.setAccessible(true);
|
||||
rqc.set(cleaner, replicationQueuesClient);
|
||||
|
||||
cleaner.isFileDeletable(fs.getFileStatus(file));
|
||||
}
|
||||
|
||||
static class DummyServer implements Server {
|
||||
|
||||
@Override
|
||||
public Configuration getConfiguration() {
|
||||
return TEST_UTIL.getConfiguration();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ZooKeeperWatcher getZooKeeper() {
|
||||
try {
|
||||
return new ZooKeeperWatcher(getConfiguration(), "dummy server", this);
|
||||
} catch (IOException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CoordinatedStateManager getCoordinatedStateManager() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ClusterConnection getConnection() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public MetaTableLocator getMetaTableLocator() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ServerName getServerName() {
|
||||
return ServerName.valueOf("regionserver,60020,000000");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort(String why, Throwable e) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isAborted() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop(String why) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isStopped() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChoreService getChoreService() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -19,12 +19,14 @@
|
|||
package org.apache.hadoop.hbase.replication;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Stoppable;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.replication.regionserver.MetricsSource;
|
||||
import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceInterface;
|
||||
import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager;
|
||||
|
@ -89,4 +91,10 @@ public class ReplicationSourceDummy implements ReplicationSourceInterface {
|
|||
public String getStats() {
|
||||
return "";
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addHFileRefs(TableName tableName, byte[] family, List<String> files)
|
||||
throws ReplicationException {
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,15 +19,21 @@ package org.apache.hadoop.hbase.replication;
|
|||
|
||||
import static org.junit.Assert.assertArrayEquals;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
|
@ -35,7 +41,9 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Durability;
|
||||
|
@ -48,12 +56,17 @@ import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
|
|||
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
||||
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.regionserver.RSRpcServices;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.replication.regionserver.TestSourceFSConfigurationProvider;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.HFileTestUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.junit.After;
|
||||
|
@ -79,6 +92,7 @@ public class TestMasterReplication {
|
|||
|
||||
private static final TableName tableName = TableName.valueOf("test");
|
||||
private static final byte[] famName = Bytes.toBytes("f");
|
||||
private static final byte[] famName1 = Bytes.toBytes("f1");
|
||||
private static final byte[] row = Bytes.toBytes("row");
|
||||
private static final byte[] row1 = Bytes.toBytes("row1");
|
||||
private static final byte[] row2 = Bytes.toBytes("row2");
|
||||
|
@ -104,6 +118,10 @@ public class TestMasterReplication {
|
|||
baseConfiguration.setLong("hbase.master.logcleaner.ttl", 10);
|
||||
baseConfiguration.setBoolean(HConstants.REPLICATION_ENABLE_KEY,
|
||||
HConstants.REPLICATION_ENABLE_DEFAULT);
|
||||
baseConfiguration.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true);
|
||||
baseConfiguration.set("hbase.replication.source.fs.conf.provider",
|
||||
TestSourceFSConfigurationProvider.class.getCanonicalName());
|
||||
baseConfiguration.set(HConstants.REPLICATION_CLUSTER_ID, "12345");
|
||||
baseConfiguration.setBoolean("dfs.support.append", true);
|
||||
baseConfiguration.setLong(HConstants.THREAD_WAKE_FREQUENCY, 100);
|
||||
baseConfiguration.setStrings(
|
||||
|
@ -114,6 +132,9 @@ public class TestMasterReplication {
|
|||
HColumnDescriptor fam = new HColumnDescriptor(famName);
|
||||
fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL);
|
||||
table.addFamily(fam);
|
||||
fam = new HColumnDescriptor(famName1);
|
||||
fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL);
|
||||
table.addFamily(fam);
|
||||
fam = new HColumnDescriptor(noRepfamName);
|
||||
table.addFamily(fam);
|
||||
}
|
||||
|
@ -130,14 +151,7 @@ public class TestMasterReplication {
|
|||
int numClusters = 2;
|
||||
Table[] htables = null;
|
||||
try {
|
||||
startMiniClusters(numClusters);
|
||||
createTableOnClusters(table);
|
||||
|
||||
htables = getHTablesOnClusters(tableName);
|
||||
|
||||
// Test the replication scenarios of 0 -> 1 -> 0
|
||||
addPeer("1", 0, 1);
|
||||
addPeer("1", 1, 0);
|
||||
htables = setUpClusterTablesAndPeers(numClusters);
|
||||
|
||||
int[] expectedCounts = new int[] { 2, 2 };
|
||||
|
||||
|
@ -157,12 +171,64 @@ public class TestMasterReplication {
|
|||
}
|
||||
|
||||
/**
|
||||
* Tests the cyclic replication scenario of 0 -> 1 -> 2 -> 0 by adding and
|
||||
* deleting rows to a table in each clusters and ensuring that the each of
|
||||
* these clusters get the appropriate mutations. It also tests the grouping
|
||||
* scenario where a cluster needs to replicate the edits originating from
|
||||
* itself and also the edits that it received using replication from a
|
||||
* different cluster. The scenario is explained in HBASE-9158
|
||||
* It tests the replication scenario involving 0 -> 1 -> 0. It does it by bulk loading a set of
|
||||
* HFiles to a table in each cluster, checking if it's replicated.
|
||||
*/
|
||||
@Test(timeout = 300000)
|
||||
public void testHFileCyclicReplication() throws Exception {
|
||||
LOG.info("testHFileCyclicReplication");
|
||||
int numClusters = 2;
|
||||
Table[] htables = null;
|
||||
try {
|
||||
htables = setUpClusterTablesAndPeers(numClusters);
|
||||
|
||||
// Load 100 rows for each hfile range in cluster '0' and validate whether its been replicated
|
||||
// to cluster '1'.
|
||||
byte[][][] hfileRanges =
|
||||
new byte[][][] { new byte[][] { Bytes.toBytes("aaaa"), Bytes.toBytes("cccc") },
|
||||
new byte[][] { Bytes.toBytes("ddd"), Bytes.toBytes("fff") }, };
|
||||
int numOfRows = 100;
|
||||
int[] expectedCounts =
|
||||
new int[] { hfileRanges.length * numOfRows, hfileRanges.length * numOfRows };
|
||||
|
||||
loadAndValidateHFileReplication("testHFileCyclicReplication_01", 0, new int[] { 1 }, row,
|
||||
famName, htables, hfileRanges, numOfRows, expectedCounts, true);
|
||||
|
||||
// Load 200 rows for each hfile range in cluster '1' and validate whether its been replicated
|
||||
// to cluster '0'.
|
||||
hfileRanges = new byte[][][] { new byte[][] { Bytes.toBytes("gggg"), Bytes.toBytes("iiii") },
|
||||
new byte[][] { Bytes.toBytes("jjj"), Bytes.toBytes("lll") }, };
|
||||
numOfRows = 200;
|
||||
int[] newExpectedCounts = new int[] { hfileRanges.length * numOfRows + expectedCounts[0],
|
||||
hfileRanges.length * numOfRows + expectedCounts[1] };
|
||||
|
||||
loadAndValidateHFileReplication("testHFileCyclicReplication_10", 1, new int[] { 0 }, row,
|
||||
famName, htables, hfileRanges, numOfRows, newExpectedCounts, true);
|
||||
|
||||
} finally {
|
||||
close(htables);
|
||||
shutDownMiniClusters();
|
||||
}
|
||||
}
|
||||
|
||||
private Table[] setUpClusterTablesAndPeers(int numClusters) throws Exception {
|
||||
Table[] htables;
|
||||
startMiniClusters(numClusters);
|
||||
createTableOnClusters(table);
|
||||
|
||||
htables = getHTablesOnClusters(tableName);
|
||||
// Test the replication scenarios of 0 -> 1 -> 0
|
||||
addPeer("1", 0, 1);
|
||||
addPeer("1", 1, 0);
|
||||
return htables;
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests the cyclic replication scenario of 0 -> 1 -> 2 -> 0 by adding and deleting rows to a
|
||||
* table in each clusters and ensuring that the each of these clusters get the appropriate
|
||||
* mutations. It also tests the grouping scenario where a cluster needs to replicate the edits
|
||||
* originating from itself and also the edits that it received using replication from a different
|
||||
* cluster. The scenario is explained in HBASE-9158
|
||||
*/
|
||||
@Test(timeout = 300000)
|
||||
public void testCyclicReplication2() throws Exception {
|
||||
|
@ -212,6 +278,119 @@ public class TestMasterReplication {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* It tests the multi slave hfile replication scenario involving 0 -> 1, 2. It does it by bulk
|
||||
* loading a set of HFiles to a table in master cluster, checking if it's replicated in its peers.
|
||||
*/
|
||||
@Test(timeout = 300000)
|
||||
public void testHFileMultiSlaveReplication() throws Exception {
|
||||
LOG.info("testHFileMultiSlaveReplication");
|
||||
int numClusters = 3;
|
||||
Table[] htables = null;
|
||||
try {
|
||||
startMiniClusters(numClusters);
|
||||
createTableOnClusters(table);
|
||||
|
||||
// Add a slave, 0 -> 1
|
||||
addPeer("1", 0, 1);
|
||||
|
||||
htables = getHTablesOnClusters(tableName);
|
||||
|
||||
// Load 100 rows for each hfile range in cluster '0' and validate whether its been replicated
|
||||
// to cluster '1'.
|
||||
byte[][][] hfileRanges =
|
||||
new byte[][][] { new byte[][] { Bytes.toBytes("mmmm"), Bytes.toBytes("oooo") },
|
||||
new byte[][] { Bytes.toBytes("ppp"), Bytes.toBytes("rrr") }, };
|
||||
int numOfRows = 100;
|
||||
|
||||
int[] expectedCounts =
|
||||
new int[] { hfileRanges.length * numOfRows, hfileRanges.length * numOfRows };
|
||||
|
||||
loadAndValidateHFileReplication("testHFileCyclicReplication_0", 0, new int[] { 1 }, row,
|
||||
famName, htables, hfileRanges, numOfRows, expectedCounts, true);
|
||||
|
||||
// Validate data is not replicated to cluster '2'.
|
||||
assertEquals(0, utilities[2].countRows(htables[2]));
|
||||
|
||||
rollWALAndWait(utilities[0], htables[0].getName(), row);
|
||||
|
||||
// Add one more slave, 0 -> 2
|
||||
addPeer("2", 0, 2);
|
||||
|
||||
// Load 200 rows for each hfile range in cluster '0' and validate whether its been replicated
|
||||
// to cluster '1' and '2'. Previous data should be replicated to cluster '2'.
|
||||
hfileRanges = new byte[][][] { new byte[][] { Bytes.toBytes("ssss"), Bytes.toBytes("uuuu") },
|
||||
new byte[][] { Bytes.toBytes("vvv"), Bytes.toBytes("xxx") }, };
|
||||
numOfRows = 200;
|
||||
|
||||
int[] newExpectedCounts = new int[] { hfileRanges.length * numOfRows + expectedCounts[0],
|
||||
hfileRanges.length * numOfRows + expectedCounts[1], hfileRanges.length * numOfRows };
|
||||
|
||||
loadAndValidateHFileReplication("testHFileCyclicReplication_1", 0, new int[] { 1, 2 }, row,
|
||||
famName, htables, hfileRanges, numOfRows, newExpectedCounts, true);
|
||||
|
||||
} finally {
|
||||
close(htables);
|
||||
shutDownMiniClusters();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* It tests the bulk loaded hfile replication scenario to only explicitly specified table column
|
||||
* families. It does it by bulk loading a set of HFiles belonging to both the CFs of table and set
|
||||
* only one CF data to replicate.
|
||||
*/
|
||||
@Test(timeout = 300000)
|
||||
public void testHFileReplicationForConfiguredTableCfs() throws Exception {
|
||||
LOG.info("testHFileReplicationForConfiguredTableCfs");
|
||||
int numClusters = 2;
|
||||
Table[] htables = null;
|
||||
try {
|
||||
startMiniClusters(numClusters);
|
||||
createTableOnClusters(table);
|
||||
|
||||
htables = getHTablesOnClusters(tableName);
|
||||
// Test the replication scenarios only 'f' is configured for table data replication not 'f1'
|
||||
addPeer("1", 0, 1, tableName.getNameAsString() + ":" + Bytes.toString(famName));
|
||||
|
||||
// Load 100 rows for each hfile range in cluster '0' for table CF 'f'
|
||||
byte[][][] hfileRanges =
|
||||
new byte[][][] { new byte[][] { Bytes.toBytes("aaaa"), Bytes.toBytes("cccc") },
|
||||
new byte[][] { Bytes.toBytes("ddd"), Bytes.toBytes("fff") }, };
|
||||
int numOfRows = 100;
|
||||
int[] expectedCounts =
|
||||
new int[] { hfileRanges.length * numOfRows, hfileRanges.length * numOfRows };
|
||||
|
||||
loadAndValidateHFileReplication("load_f", 0, new int[] { 1 }, row, famName, htables,
|
||||
hfileRanges, numOfRows, expectedCounts, true);
|
||||
|
||||
// Load 100 rows for each hfile range in cluster '0' for table CF 'f1'
|
||||
hfileRanges = new byte[][][] { new byte[][] { Bytes.toBytes("gggg"), Bytes.toBytes("iiii") },
|
||||
new byte[][] { Bytes.toBytes("jjj"), Bytes.toBytes("lll") }, };
|
||||
numOfRows = 100;
|
||||
|
||||
int[] newExpectedCounts =
|
||||
new int[] { hfileRanges.length * numOfRows + expectedCounts[0], expectedCounts[1] };
|
||||
|
||||
loadAndValidateHFileReplication("load_f1", 0, new int[] { 1 }, row, famName1, htables,
|
||||
hfileRanges, numOfRows, newExpectedCounts, false);
|
||||
|
||||
// Validate data replication for CF 'f1'
|
||||
|
||||
// Source cluster table should contain data for the families
|
||||
wait(0, htables[0], hfileRanges.length * numOfRows + expectedCounts[0]);
|
||||
|
||||
// Sleep for enough time so that the data is still not replicated for the CF which is not
|
||||
// configured for replication
|
||||
Thread.sleep((NB_RETRIES / 2) * SLEEP_TIME);
|
||||
// Peer cluster should have only configured CF data
|
||||
wait(1, htables[1], expectedCounts[1]);
|
||||
} finally {
|
||||
close(htables);
|
||||
shutDownMiniClusters();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests cyclic replication scenario of 0 -> 1 -> 2 -> 1.
|
||||
*/
|
||||
|
@ -329,6 +508,17 @@ public class TestMasterReplication {
|
|||
}
|
||||
}
|
||||
|
||||
private void addPeer(String id, int masterClusterNumber, int slaveClusterNumber, String tableCfs)
|
||||
throws Exception {
|
||||
ReplicationAdmin replicationAdmin = null;
|
||||
try {
|
||||
replicationAdmin = new ReplicationAdmin(configurations[masterClusterNumber]);
|
||||
replicationAdmin.addPeer(id, utilities[slaveClusterNumber].getClusterKey(), tableCfs);
|
||||
} finally {
|
||||
close(replicationAdmin);
|
||||
}
|
||||
}
|
||||
|
||||
private void disablePeer(String id, int masterClusterNumber) throws Exception {
|
||||
ReplicationAdmin replicationAdmin = null;
|
||||
try {
|
||||
|
@ -405,8 +595,56 @@ public class TestMasterReplication {
|
|||
wait(row, target, false);
|
||||
}
|
||||
|
||||
private void wait(byte[] row, Table target, boolean isDeleted)
|
||||
throws Exception {
|
||||
private void loadAndValidateHFileReplication(String testName, int masterNumber,
|
||||
int[] slaveNumbers, byte[] row, byte[] fam, Table[] tables, byte[][][] hfileRanges,
|
||||
int numOfRows, int[] expectedCounts, boolean toValidate) throws Exception {
|
||||
HBaseTestingUtility util = utilities[masterNumber];
|
||||
|
||||
Path dir = util.getDataTestDirOnTestFS(testName);
|
||||
FileSystem fs = util.getTestFileSystem();
|
||||
dir = dir.makeQualified(fs);
|
||||
Path familyDir = new Path(dir, Bytes.toString(fam));
|
||||
|
||||
int hfileIdx = 0;
|
||||
for (byte[][] range : hfileRanges) {
|
||||
byte[] from = range[0];
|
||||
byte[] to = range[1];
|
||||
HFileTestUtil.createHFile(util.getConfiguration(), fs,
|
||||
new Path(familyDir, "hfile_" + hfileIdx++), fam, row, from, to, numOfRows);
|
||||
}
|
||||
|
||||
Table source = tables[masterNumber];
|
||||
final TableName tableName = source.getName();
|
||||
LoadIncrementalHFiles loader = new LoadIncrementalHFiles(util.getConfiguration());
|
||||
String[] args = { dir.toString(), tableName.toString() };
|
||||
loader.run(args);
|
||||
|
||||
if (toValidate) {
|
||||
for (int slaveClusterNumber : slaveNumbers) {
|
||||
wait(slaveClusterNumber, tables[slaveClusterNumber], expectedCounts[slaveClusterNumber]);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void wait(int slaveNumber, Table target, int expectedCount)
|
||||
throws IOException, InterruptedException {
|
||||
int count = 0;
|
||||
for (int i = 0; i < NB_RETRIES; i++) {
|
||||
if (i == NB_RETRIES - 1) {
|
||||
fail("Waited too much time for bulkloaded data replication. Current count=" + count
|
||||
+ ", expected count=" + expectedCount);
|
||||
}
|
||||
count = utilities[slaveNumber].countRows(target);
|
||||
if (count != expectedCount) {
|
||||
LOG.info("Waiting more time for bulkloaded data replication.");
|
||||
Thread.sleep(SLEEP_TIME);
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void wait(byte[] row, Table target, boolean isDeleted) throws Exception {
|
||||
Get get = new Get(row);
|
||||
for (int i = 0; i < NB_RETRIES; i++) {
|
||||
if (i == NB_RETRIES - 1) {
|
||||
|
@ -430,6 +668,47 @@ public class TestMasterReplication {
|
|||
}
|
||||
}
|
||||
|
||||
private void rollWALAndWait(final HBaseTestingUtility utility, final TableName table,
|
||||
final byte[] row) throws IOException {
|
||||
final Admin admin = utility.getHBaseAdmin();
|
||||
final MiniHBaseCluster cluster = utility.getMiniHBaseCluster();
|
||||
|
||||
// find the region that corresponds to the given row.
|
||||
HRegion region = null;
|
||||
for (HRegion candidate : cluster.getRegions(table)) {
|
||||
if (HRegion.rowIsInRange(candidate.getRegionInfo(), row)) {
|
||||
region = candidate;
|
||||
break;
|
||||
}
|
||||
}
|
||||
assertNotNull("Couldn't find the region for row '" + Arrays.toString(row) + "'", region);
|
||||
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
|
||||
// listen for successful log rolls
|
||||
final WALActionsListener listener = new WALActionsListener.Base() {
|
||||
@Override
|
||||
public void postLogRoll(final Path oldPath, final Path newPath) throws IOException {
|
||||
latch.countDown();
|
||||
}
|
||||
};
|
||||
region.getWAL().registerWALActionsListener(listener);
|
||||
|
||||
// request a roll
|
||||
admin.rollWALWriter(cluster.getServerHoldingRegion(region.getTableDesc().getTableName(),
|
||||
region.getRegionInfo().getRegionName()));
|
||||
|
||||
// wait
|
||||
try {
|
||||
latch.await();
|
||||
} catch (InterruptedException exception) {
|
||||
LOG.warn("Interrupted while waiting for the wal of '" + region + "' to roll. If later " +
|
||||
"replication tests fail, it's probably because we should still be waiting.");
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
region.getWAL().unregisterWALActionsListener(listener);
|
||||
}
|
||||
|
||||
/**
|
||||
* Use a coprocessor to count puts and deletes. as KVs would be replicated back with the same
|
||||
* timestamp there is otherwise no way to count them.
|
||||
|
|
|
@ -658,7 +658,8 @@ public class TestReplicationSmallTests extends TestReplicationBase {
|
|||
HRegionInfo hri = new HRegionInfo(htable1.getName(),
|
||||
HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
|
||||
WALEdit edit = WALEdit.createCompaction(hri, compactionDescriptor);
|
||||
Replication.scopeWALEdits(htable1.getTableDescriptor(), new WALKey(), edit);
|
||||
Replication.scopeWALEdits(htable1.getTableDescriptor(), new WALKey(), edit,
|
||||
htable1.getConfiguration(), null);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.replication;
|
|||
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.SortedMap;
|
||||
import java.util.SortedSet;
|
||||
|
@ -159,6 +160,62 @@ public abstract class TestReplicationStateBasic {
|
|||
assertEquals(0, rq2.getListOfReplicators().size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHfileRefsReplicationQueues() throws ReplicationException, KeeperException {
|
||||
rp.init();
|
||||
rq1.init(server1);
|
||||
rqc.init();
|
||||
|
||||
List<String> files1 = new ArrayList<String>(3);
|
||||
files1.add("file_1");
|
||||
files1.add("file_2");
|
||||
files1.add("file_3");
|
||||
assertNull(rqc.getReplicableHFiles(ID_ONE));
|
||||
assertEquals(0, rqc.getAllPeersFromHFileRefsQueue().size());
|
||||
rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE), null);
|
||||
rq1.addHFileRefs(ID_ONE, files1);
|
||||
assertEquals(1, rqc.getAllPeersFromHFileRefsQueue().size());
|
||||
assertEquals(3, rqc.getReplicableHFiles(ID_ONE).size());
|
||||
List<String> files2 = new ArrayList<>(files1);
|
||||
String removedString = files2.remove(0);
|
||||
rq1.removeHFileRefs(ID_ONE, files2);
|
||||
assertEquals(1, rqc.getReplicableHFiles(ID_ONE).size());
|
||||
files2 = new ArrayList<>(1);
|
||||
files2.add(removedString);
|
||||
rq1.removeHFileRefs(ID_ONE, files2);
|
||||
assertEquals(0, rqc.getReplicableHFiles(ID_ONE).size());
|
||||
rp.removePeer(ID_ONE);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRemovePeerForHFileRefs() throws ReplicationException, KeeperException {
|
||||
rq1.init(server1);
|
||||
rqc.init();
|
||||
|
||||
rp.init();
|
||||
rp.addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE), null);
|
||||
rp.addPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO), null);
|
||||
|
||||
List<String> files1 = new ArrayList<String>(3);
|
||||
files1.add("file_1");
|
||||
files1.add("file_2");
|
||||
files1.add("file_3");
|
||||
rq1.addHFileRefs(ID_ONE, files1);
|
||||
rq1.addHFileRefs(ID_TWO, files1);
|
||||
assertEquals(2, rqc.getAllPeersFromHFileRefsQueue().size());
|
||||
assertEquals(3, rqc.getReplicableHFiles(ID_ONE).size());
|
||||
assertEquals(3, rqc.getReplicableHFiles(ID_TWO).size());
|
||||
|
||||
rp.removePeer(ID_ONE);
|
||||
assertEquals(1, rqc.getAllPeersFromHFileRefsQueue().size());
|
||||
assertNull(rqc.getReplicableHFiles(ID_ONE));
|
||||
assertEquals(3, rqc.getReplicableHFiles(ID_TWO).size());
|
||||
|
||||
rp.removePeer(ID_TWO);
|
||||
assertEquals(0, rqc.getAllPeersFromHFileRefsQueue().size());
|
||||
assertNull(rqc.getReplicableHFiles(ID_TWO));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReplicationPeers() throws Exception {
|
||||
rp.init();
|
||||
|
|
|
@ -64,6 +64,7 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
|
|||
utility = new HBaseTestingUtility();
|
||||
utility.startMiniZKCluster();
|
||||
conf = utility.getConfiguration();
|
||||
conf.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true);
|
||||
zkw = HBaseTestingUtility.getZooKeeperWatcher(utility);
|
||||
String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication");
|
||||
replicationZNode = ZKUtil.joinZNode(zkw.baseZNode, replicationZNodeName);
|
||||
|
|
|
@ -52,15 +52,15 @@ public class TestReplicationSyncUpTool extends TestReplicationBase {
|
|||
private static final TableName t1_su = TableName.valueOf("t1_syncup");
|
||||
private static final TableName t2_su = TableName.valueOf("t2_syncup");
|
||||
|
||||
private static final byte[] famName = Bytes.toBytes("cf1");
|
||||
protected static final byte[] famName = Bytes.toBytes("cf1");
|
||||
private static final byte[] qualName = Bytes.toBytes("q1");
|
||||
|
||||
private static final byte[] noRepfamName = Bytes.toBytes("norep");
|
||||
protected static final byte[] noRepfamName = Bytes.toBytes("norep");
|
||||
|
||||
private HTableDescriptor t1_syncupSource, t1_syncupTarget;
|
||||
private HTableDescriptor t2_syncupSource, t2_syncupTarget;
|
||||
|
||||
private Table ht1Source, ht2Source, ht1TargetAtPeer1, ht2TargetAtPeer1;
|
||||
protected Table ht1Source, ht2Source, ht1TargetAtPeer1, ht2TargetAtPeer1;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
|
@ -179,7 +179,7 @@ public class TestReplicationSyncUpTool extends TestReplicationBase {
|
|||
|
||||
}
|
||||
|
||||
private void setupReplication() throws Exception {
|
||||
protected void setupReplication() throws Exception {
|
||||
ReplicationAdmin admin1 = new ReplicationAdmin(conf1);
|
||||
ReplicationAdmin admin2 = new ReplicationAdmin(conf2);
|
||||
|
||||
|
@ -418,7 +418,7 @@ public class TestReplicationSyncUpTool extends TestReplicationBase {
|
|||
}
|
||||
}
|
||||
|
||||
private void syncUp(HBaseTestingUtility ut) throws Exception {
|
||||
protected void syncUp(HBaseTestingUtility ut) throws Exception {
|
||||
ReplicationSyncUp.setConfigure(ut.getConfiguration());
|
||||
String[] arguments = new String[] { null };
|
||||
new ReplicationSyncUp().run(arguments);
|
||||
|
|
|
@ -0,0 +1,235 @@
|
|||
/**
|
||||
* 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;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
||||
import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
|
||||
import org.apache.hadoop.hbase.replication.regionserver.TestSourceFSConfigurationProvider;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.HFileTestUtil;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
@Category({ ReplicationTests.class, LargeTests.class })
|
||||
public class TestReplicationSyncUpToolWithBulkLoadedData extends TestReplicationSyncUpTool {
|
||||
|
||||
private static final Log LOG = LogFactory
|
||||
.getLog(TestReplicationSyncUpToolWithBulkLoadedData.class);
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
conf1.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true);
|
||||
conf1.set(HConstants.REPLICATION_CLUSTER_ID, "12345");
|
||||
conf1.set("hbase.replication.source.fs.conf.provider",
|
||||
TestSourceFSConfigurationProvider.class.getCanonicalName());
|
||||
String classes = conf1.get(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, "");
|
||||
if (!classes.contains("org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint")) {
|
||||
classes = classes + ",org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint";
|
||||
conf1.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, classes);
|
||||
}
|
||||
|
||||
TestReplicationBase.setUpBeforeClass();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void testSyncUpTool() throws Exception {
|
||||
/**
|
||||
* Set up Replication: on Master and one Slave Table: t1_syncup and t2_syncup columnfamily:
|
||||
* 'cf1' : replicated 'norep': not replicated
|
||||
*/
|
||||
setupReplication();
|
||||
|
||||
/**
|
||||
* Prepare 16 random hfile ranges required for creating hfiles
|
||||
*/
|
||||
Iterator<String> randomHFileRangeListIterator = null;
|
||||
Set<String> randomHFileRanges = new HashSet<String>(16);
|
||||
for (int i = 0; i < 16; i++) {
|
||||
randomHFileRanges.add(UUID.randomUUID().toString());
|
||||
}
|
||||
List<String> randomHFileRangeList = new ArrayList<>(randomHFileRanges);
|
||||
Collections.sort(randomHFileRangeList);
|
||||
randomHFileRangeListIterator = randomHFileRangeList.iterator();
|
||||
|
||||
/**
|
||||
* at Master: t1_syncup: Load 100 rows into cf1, and 3 rows into norep t2_syncup: Load 200 rows
|
||||
* into cf1, and 3 rows into norep verify correctly replicated to slave
|
||||
*/
|
||||
loadAndReplicateHFiles(true, randomHFileRangeListIterator);
|
||||
|
||||
/**
|
||||
* Verify hfile load works step 1: stop hbase on Slave step 2: at Master: t1_syncup: Load
|
||||
* another 100 rows into cf1 and 3 rows into norep t2_syncup: Load another 200 rows into cf1 and
|
||||
* 3 rows into norep step 3: stop hbase on master, restart hbase on Slave step 4: verify Slave
|
||||
* still has the rows before load t1_syncup: 100 rows from cf1 t2_syncup: 200 rows from cf1 step
|
||||
* 5: run syncup tool on Master step 6: verify that hfiles show up on Slave and 'norep' does not
|
||||
* t1_syncup: 200 rows from cf1 t2_syncup: 400 rows from cf1 verify correctly replicated to
|
||||
* Slave
|
||||
*/
|
||||
mimicSyncUpAfterBulkLoad(randomHFileRangeListIterator);
|
||||
|
||||
}
|
||||
|
||||
private void mimicSyncUpAfterBulkLoad(Iterator<String> randomHFileRangeListIterator)
|
||||
throws Exception {
|
||||
LOG.debug("mimicSyncUpAfterBulkLoad");
|
||||
utility2.shutdownMiniHBaseCluster();
|
||||
|
||||
loadAndReplicateHFiles(false, randomHFileRangeListIterator);
|
||||
|
||||
int rowCount_ht1Source = utility1.countRows(ht1Source);
|
||||
assertEquals("t1_syncup has 206 rows on source, after bulk load of another 103 hfiles", 206,
|
||||
rowCount_ht1Source);
|
||||
|
||||
int rowCount_ht2Source = utility1.countRows(ht2Source);
|
||||
assertEquals("t2_syncup has 406 rows on source, after bulk load of another 203 hfiles", 406,
|
||||
rowCount_ht2Source);
|
||||
|
||||
utility1.shutdownMiniHBaseCluster();
|
||||
utility2.restartHBaseCluster(1);
|
||||
|
||||
Thread.sleep(SLEEP_TIME);
|
||||
|
||||
// Before sync up
|
||||
int rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1);
|
||||
int rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1);
|
||||
assertEquals("@Peer1 t1_syncup should still have 100 rows", 100, rowCount_ht1TargetAtPeer1);
|
||||
assertEquals("@Peer1 t2_syncup should still have 200 rows", 200, rowCount_ht2TargetAtPeer1);
|
||||
|
||||
// Run sync up tool
|
||||
syncUp(utility1);
|
||||
|
||||
// After syun up
|
||||
for (int i = 0; i < NB_RETRIES; i++) {
|
||||
syncUp(utility1);
|
||||
rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1);
|
||||
rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1);
|
||||
if (i == NB_RETRIES - 1) {
|
||||
if (rowCount_ht1TargetAtPeer1 != 200 || rowCount_ht2TargetAtPeer1 != 400) {
|
||||
// syncUP still failed. Let's look at the source in case anything wrong there
|
||||
utility1.restartHBaseCluster(1);
|
||||
rowCount_ht1Source = utility1.countRows(ht1Source);
|
||||
LOG.debug("t1_syncup should have 206 rows at source, and it is " + rowCount_ht1Source);
|
||||
rowCount_ht2Source = utility1.countRows(ht2Source);
|
||||
LOG.debug("t2_syncup should have 406 rows at source, and it is " + rowCount_ht2Source);
|
||||
}
|
||||
assertEquals("@Peer1 t1_syncup should be sync up and have 200 rows", 200,
|
||||
rowCount_ht1TargetAtPeer1);
|
||||
assertEquals("@Peer1 t2_syncup should be sync up and have 400 rows", 400,
|
||||
rowCount_ht2TargetAtPeer1);
|
||||
}
|
||||
if (rowCount_ht1TargetAtPeer1 == 200 && rowCount_ht2TargetAtPeer1 == 400) {
|
||||
LOG.info("SyncUpAfterBulkLoad succeeded at retry = " + i);
|
||||
break;
|
||||
} else {
|
||||
LOG.debug("SyncUpAfterBulkLoad failed at retry = " + i + ", with rowCount_ht1TargetPeer1 ="
|
||||
+ rowCount_ht1TargetAtPeer1 + " and rowCount_ht2TargetAtPeer1 ="
|
||||
+ rowCount_ht2TargetAtPeer1);
|
||||
}
|
||||
Thread.sleep(SLEEP_TIME);
|
||||
}
|
||||
}
|
||||
|
||||
private void loadAndReplicateHFiles(boolean verifyReplicationOnSlave,
|
||||
Iterator<String> randomHFileRangeListIterator) throws Exception {
|
||||
LOG.debug("loadAndReplicateHFiles");
|
||||
|
||||
// Load 100 + 3 hfiles to t1_syncup.
|
||||
byte[][][] hfileRanges =
|
||||
new byte[][][] { new byte[][] { Bytes.toBytes(randomHFileRangeListIterator.next()),
|
||||
Bytes.toBytes(randomHFileRangeListIterator.next()) } };
|
||||
loadAndValidateHFileReplication("HFileReplication_1", row, famName, ht1Source, hfileRanges,
|
||||
100);
|
||||
|
||||
hfileRanges =
|
||||
new byte[][][] { new byte[][] { Bytes.toBytes(randomHFileRangeListIterator.next()),
|
||||
Bytes.toBytes(randomHFileRangeListIterator.next()) } };
|
||||
loadAndValidateHFileReplication("HFileReplication_1", row, noRepfamName, ht1Source,
|
||||
hfileRanges, 3);
|
||||
|
||||
// Load 200 + 3 hfiles to t2_syncup.
|
||||
hfileRanges =
|
||||
new byte[][][] { new byte[][] { Bytes.toBytes(randomHFileRangeListIterator.next()),
|
||||
Bytes.toBytes(randomHFileRangeListIterator.next()) } };
|
||||
loadAndValidateHFileReplication("HFileReplication_1", row, famName, ht2Source, hfileRanges,
|
||||
200);
|
||||
|
||||
hfileRanges =
|
||||
new byte[][][] { new byte[][] { Bytes.toBytes(randomHFileRangeListIterator.next()),
|
||||
Bytes.toBytes(randomHFileRangeListIterator.next()) } };
|
||||
loadAndValidateHFileReplication("HFileReplication_1", row, noRepfamName, ht2Source,
|
||||
hfileRanges, 3);
|
||||
|
||||
if (verifyReplicationOnSlave) {
|
||||
// ensure replication completed
|
||||
wait(ht1TargetAtPeer1, utility1.countRows(ht1Source) - 3,
|
||||
"t1_syncup has 103 rows on source, and 100 on slave1");
|
||||
|
||||
wait(ht2TargetAtPeer1, utility1.countRows(ht2Source) - 3,
|
||||
"t2_syncup has 203 rows on source, and 200 on slave1");
|
||||
}
|
||||
}
|
||||
|
||||
private void loadAndValidateHFileReplication(String testName, byte[] row, byte[] fam,
|
||||
Table source, byte[][][] hfileRanges, int numOfRows) throws Exception {
|
||||
Path dir = utility1.getDataTestDirOnTestFS(testName);
|
||||
FileSystem fs = utility1.getTestFileSystem();
|
||||
dir = dir.makeQualified(fs);
|
||||
Path familyDir = new Path(dir, Bytes.toString(fam));
|
||||
|
||||
int hfileIdx = 0;
|
||||
for (byte[][] range : hfileRanges) {
|
||||
byte[] from = range[0];
|
||||
byte[] to = range[1];
|
||||
HFileTestUtil.createHFile(utility1.getConfiguration(), fs, new Path(familyDir, "hfile_"
|
||||
+ hfileIdx++), fam, row, from, to, numOfRows);
|
||||
}
|
||||
|
||||
final TableName tableName = source.getName();
|
||||
LoadIncrementalHFiles loader = new LoadIncrementalHFiles(utility1.getConfiguration());
|
||||
String[] args = { dir.toString(), tableName.toString() };
|
||||
loader.run(args);
|
||||
}
|
||||
|
||||
private void wait(Table target, int expectedCount, String msg) throws IOException,
|
||||
InterruptedException {
|
||||
for (int i = 0; i < NB_RETRIES; i++) {
|
||||
int rowCount_ht2TargetAtPeer1 = utility2.countRows(target);
|
||||
if (i == NB_RETRIES - 1) {
|
||||
assertEquals(msg, expectedCount, rowCount_ht2TargetAtPeer1);
|
||||
}
|
||||
if (expectedCount == rowCount_ht2TargetAtPeer1) {
|
||||
break;
|
||||
}
|
||||
Thread.sleep(SLEEP_TIME);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -21,32 +21,52 @@ package org.apache.hadoop.hbase.replication.regionserver;
|
|||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.security.SecureRandom;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FileUtil;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.Stoppable;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.UUID;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.HFileTestUtil;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
|
@ -58,21 +78,18 @@ public class TestReplicationSink {
|
|||
private static final Log LOG = LogFactory.getLog(TestReplicationSink.class);
|
||||
private static final int BATCH_SIZE = 10;
|
||||
|
||||
private final static HBaseTestingUtility TEST_UTIL =
|
||||
new HBaseTestingUtility();
|
||||
protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
|
||||
private static ReplicationSink SINK;
|
||||
protected static ReplicationSink SINK;
|
||||
|
||||
private static final TableName TABLE_NAME1 =
|
||||
TableName.valueOf("table1");
|
||||
private static final TableName TABLE_NAME2 =
|
||||
TableName.valueOf("table2");
|
||||
protected static final TableName TABLE_NAME1 = TableName.valueOf("table1");
|
||||
protected static final TableName TABLE_NAME2 = TableName.valueOf("table2");
|
||||
|
||||
private static final byte[] FAM_NAME1 = Bytes.toBytes("info1");
|
||||
private static final byte[] FAM_NAME2 = Bytes.toBytes("info2");
|
||||
protected static final byte[] FAM_NAME1 = Bytes.toBytes("info1");
|
||||
protected static final byte[] FAM_NAME2 = Bytes.toBytes("info2");
|
||||
|
||||
private static Table table1;
|
||||
private static Stoppable STOPPABLE = new Stoppable() {
|
||||
protected static Table table1;
|
||||
protected static Stoppable STOPPABLE = new Stoppable() {
|
||||
final AtomicBoolean stop = new AtomicBoolean(false);
|
||||
|
||||
@Override
|
||||
|
@ -88,7 +105,10 @@ public class TestReplicationSink {
|
|||
|
||||
};
|
||||
|
||||
private static Table table2;
|
||||
protected static Table table2;
|
||||
protected static String baseNamespaceDir;
|
||||
protected static String hfileArchiveDir;
|
||||
protected static String replicationClusterId;
|
||||
|
||||
/**
|
||||
* @throws java.lang.Exception
|
||||
|
@ -98,11 +118,18 @@ public class TestReplicationSink {
|
|||
TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true);
|
||||
TEST_UTIL.getConfiguration().setBoolean(HConstants.REPLICATION_ENABLE_KEY,
|
||||
HConstants.REPLICATION_ENABLE_DEFAULT);
|
||||
TEST_UTIL.getConfiguration().set("hbase.replication.source.fs.conf.provider",
|
||||
TestSourceFSConfigurationProvider.class.getCanonicalName());
|
||||
|
||||
TEST_UTIL.startMiniCluster(3);
|
||||
SINK =
|
||||
new ReplicationSink(new Configuration(TEST_UTIL.getConfiguration()), STOPPABLE);
|
||||
table1 = TEST_UTIL.createTable(TABLE_NAME1, FAM_NAME1);
|
||||
table2 = TEST_UTIL.createTable(TABLE_NAME2, FAM_NAME2);
|
||||
Path rootDir = FSUtils.getRootDir(TEST_UTIL.getConfiguration());
|
||||
baseNamespaceDir = new Path(rootDir, new Path(HConstants.BASE_NAMESPACE_DIR)).toString();
|
||||
hfileArchiveDir = new Path(rootDir, new Path(HConstants.HFILE_ARCHIVE_DIRECTORY)).toString();
|
||||
replicationClusterId = "12345";
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -134,7 +161,8 @@ public class TestReplicationSink {
|
|||
for(int i = 0; i < BATCH_SIZE; i++) {
|
||||
entries.add(createEntry(TABLE_NAME1, i, KeyValue.Type.Put, cells));
|
||||
}
|
||||
SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()));
|
||||
SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()),
|
||||
replicationClusterId, baseNamespaceDir, hfileArchiveDir);
|
||||
Scan scan = new Scan();
|
||||
ResultScanner scanRes = table1.getScanner(scan);
|
||||
assertEquals(BATCH_SIZE, scanRes.next(BATCH_SIZE).length);
|
||||
|
@ -151,7 +179,8 @@ public class TestReplicationSink {
|
|||
for(int i = 0; i < BATCH_SIZE/2; i++) {
|
||||
entries.add(createEntry(TABLE_NAME1, i, KeyValue.Type.Put, cells));
|
||||
}
|
||||
SINK.replicateEntries(entries, CellUtil.createCellScanner(cells));
|
||||
SINK.replicateEntries(entries, CellUtil.createCellScanner(cells), replicationClusterId,
|
||||
baseNamespaceDir, hfileArchiveDir);
|
||||
|
||||
entries = new ArrayList<WALEntry>(BATCH_SIZE);
|
||||
cells = new ArrayList<Cell>();
|
||||
|
@ -160,7 +189,8 @@ public class TestReplicationSink {
|
|||
i % 2 != 0 ? KeyValue.Type.Put: KeyValue.Type.DeleteColumn, cells));
|
||||
}
|
||||
|
||||
SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()));
|
||||
SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()),
|
||||
replicationClusterId, baseNamespaceDir, hfileArchiveDir);
|
||||
Scan scan = new Scan();
|
||||
ResultScanner scanRes = table1.getScanner(scan);
|
||||
assertEquals(BATCH_SIZE/2, scanRes.next(BATCH_SIZE).length);
|
||||
|
@ -179,7 +209,8 @@ public class TestReplicationSink {
|
|||
i, KeyValue.Type.Put, cells));
|
||||
}
|
||||
|
||||
SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()));
|
||||
SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()),
|
||||
replicationClusterId, baseNamespaceDir, hfileArchiveDir);
|
||||
Scan scan = new Scan();
|
||||
ResultScanner scanRes = table2.getScanner(scan);
|
||||
for(Result res : scanRes) {
|
||||
|
@ -198,14 +229,16 @@ public class TestReplicationSink {
|
|||
for(int i = 0; i < 3; i++) {
|
||||
entries.add(createEntry(TABLE_NAME1, i, KeyValue.Type.Put, cells));
|
||||
}
|
||||
SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()));
|
||||
SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()),
|
||||
replicationClusterId, baseNamespaceDir, hfileArchiveDir);
|
||||
entries = new ArrayList<WALEntry>(3);
|
||||
cells = new ArrayList<Cell>();
|
||||
entries.add(createEntry(TABLE_NAME1, 0, KeyValue.Type.DeleteColumn, cells));
|
||||
entries.add(createEntry(TABLE_NAME1, 1, KeyValue.Type.DeleteFamily, cells));
|
||||
entries.add(createEntry(TABLE_NAME1, 2, KeyValue.Type.DeleteColumn, cells));
|
||||
|
||||
SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()));
|
||||
SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()),
|
||||
replicationClusterId, baseNamespaceDir, hfileArchiveDir);
|
||||
|
||||
Scan scan = new Scan();
|
||||
ResultScanner scanRes = table1.getScanner(scan);
|
||||
|
@ -228,12 +261,96 @@ public class TestReplicationSink {
|
|||
for(int i = 3; i < 5; i++) {
|
||||
entries.add(createEntry(TABLE_NAME1, i, KeyValue.Type.Put, cells));
|
||||
}
|
||||
SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()));
|
||||
SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()),
|
||||
replicationClusterId, baseNamespaceDir, hfileArchiveDir);
|
||||
Get get = new Get(Bytes.toBytes(1));
|
||||
Result res = table1.get(get);
|
||||
assertEquals(0, res.size());
|
||||
}
|
||||
|
||||
/**
|
||||
* Test replicateEntries with a bulk load entry for 25 HFiles
|
||||
*/
|
||||
@Test
|
||||
public void testReplicateEntriesForHFiles() throws Exception {
|
||||
Path dir = TEST_UTIL.getDataTestDirOnTestFS("testReplicateEntries");
|
||||
Path familyDir = new Path(dir, Bytes.toString(FAM_NAME1));
|
||||
int numRows = 10;
|
||||
|
||||
List<Path> p = new ArrayList<>(1);
|
||||
|
||||
// 1. Generate 25 hfile ranges
|
||||
Random rng = new SecureRandom();
|
||||
Set<Integer> numbers = new HashSet<>();
|
||||
while (numbers.size() < 50) {
|
||||
numbers.add(rng.nextInt(1000));
|
||||
}
|
||||
List<Integer> numberList = new ArrayList<>(numbers);
|
||||
Collections.sort(numberList);
|
||||
|
||||
// 2. Create 25 hfiles
|
||||
Configuration conf = TEST_UTIL.getConfiguration();
|
||||
FileSystem fs = dir.getFileSystem(conf);
|
||||
Iterator<Integer> numbersItr = numberList.iterator();
|
||||
for (int i = 0; i < 25; i++) {
|
||||
Path hfilePath = new Path(familyDir, "hfile_" + i);
|
||||
HFileTestUtil.createHFile(conf, fs, hfilePath, FAM_NAME1, FAM_NAME1,
|
||||
Bytes.toBytes(numbersItr.next()), Bytes.toBytes(numbersItr.next()), numRows);
|
||||
p.add(hfilePath);
|
||||
}
|
||||
|
||||
// 3. Create a BulkLoadDescriptor and a WALEdit
|
||||
Map<byte[], List<Path>> storeFiles = new HashMap<>(1);
|
||||
storeFiles.put(FAM_NAME1, p);
|
||||
WALEdit edit = null;
|
||||
WALProtos.BulkLoadDescriptor loadDescriptor = null;
|
||||
|
||||
try (Connection c = ConnectionFactory.createConnection(conf);
|
||||
RegionLocator l = c.getRegionLocator(TABLE_NAME1)) {
|
||||
HRegionInfo regionInfo = l.getAllRegionLocations().get(0).getRegionInfo();
|
||||
loadDescriptor =
|
||||
ProtobufUtil.toBulkLoadDescriptor(TABLE_NAME1,
|
||||
ByteStringer.wrap(regionInfo.getEncodedNameAsBytes()), storeFiles, 1);
|
||||
edit = WALEdit.createBulkLoadEvent(regionInfo, loadDescriptor);
|
||||
}
|
||||
List<WALEntry> entries = new ArrayList<WALEntry>(1);
|
||||
|
||||
// 4. Create a WALEntryBuilder
|
||||
WALEntry.Builder builder = createWALEntryBuilder(TABLE_NAME1);
|
||||
|
||||
// 5. Copy the hfile to the path as it is in reality
|
||||
for (int i = 0; i < 25; i++) {
|
||||
String pathToHfileFromNS =
|
||||
new StringBuilder(100).append(TABLE_NAME1.getNamespaceAsString()).append(Path.SEPARATOR)
|
||||
.append(Bytes.toString(TABLE_NAME1.getName())).append(Path.SEPARATOR)
|
||||
.append(Bytes.toString(loadDescriptor.getEncodedRegionName().toByteArray()))
|
||||
.append(Path.SEPARATOR).append(Bytes.toString(FAM_NAME1)).append(Path.SEPARATOR)
|
||||
.append("hfile_" + i).toString();
|
||||
String dst = baseNamespaceDir + Path.SEPARATOR + pathToHfileFromNS;
|
||||
|
||||
FileUtil.copy(fs, p.get(0), fs, new Path(dst), false, conf);
|
||||
}
|
||||
|
||||
entries.add(builder.build());
|
||||
ResultScanner scanRes = null;
|
||||
try {
|
||||
Scan scan = new Scan();
|
||||
scanRes = table1.getScanner(scan);
|
||||
// 6. Assert no existing data in table
|
||||
assertEquals(0, scanRes.next(numRows).length);
|
||||
// 7. Replicate the bulk loaded entry
|
||||
SINK.replicateEntries(entries, CellUtil.createCellScanner(edit.getCells().iterator()),
|
||||
replicationClusterId, baseNamespaceDir, hfileArchiveDir);
|
||||
scanRes = table1.getScanner(scan);
|
||||
// 8. Assert data is replicated
|
||||
assertEquals(numRows, scanRes.next(numRows).length);
|
||||
} finally {
|
||||
if (scanRes != null) {
|
||||
scanRes.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private WALEntry createEntry(TableName table, int row, KeyValue.Type type, List<Cell> cells) {
|
||||
byte[] fam = table.equals(TABLE_NAME1) ? FAM_NAME1 : FAM_NAME2;
|
||||
byte[] rowBytes = Bytes.toBytes(row);
|
||||
|
@ -256,6 +373,13 @@ public class TestReplicationSink {
|
|||
kv = new KeyValue(rowBytes, fam, null,
|
||||
now, KeyValue.Type.DeleteFamily);
|
||||
}
|
||||
WALEntry.Builder builder = createWALEntryBuilder(table);
|
||||
cells.add(kv);
|
||||
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
private WALEntry.Builder createWALEntryBuilder(TableName table) {
|
||||
WALEntry.Builder builder = WALEntry.newBuilder();
|
||||
builder.setAssociatedCellCount(1);
|
||||
WALKey.Builder keyBuilder = WALKey.newBuilder();
|
||||
|
@ -264,13 +388,10 @@ public class TestReplicationSink {
|
|||
uuidBuilder.setMostSigBits(HConstants.DEFAULT_CLUSTER_ID.getMostSignificantBits());
|
||||
keyBuilder.setClusterId(uuidBuilder.build());
|
||||
keyBuilder.setTableName(ByteStringer.wrap(table.getName()));
|
||||
keyBuilder.setWriteTime(now);
|
||||
keyBuilder.setWriteTime(System.currentTimeMillis());
|
||||
keyBuilder.setEncodedRegionName(ByteStringer.wrap(HConstants.EMPTY_BYTE_ARRAY));
|
||||
keyBuilder.setLogSequenceNumber(-1);
|
||||
builder.setKey(keyBuilder.build());
|
||||
cells.add(kv);
|
||||
|
||||
return builder.build();
|
||||
return builder;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -19,13 +19,17 @@
|
|||
package org.apache.hadoop.hbase.replication.regionserver;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.net.URLEncoder;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.NavigableMap;
|
||||
import java.util.SortedMap;
|
||||
import java.util.SortedSet;
|
||||
import java.util.TreeSet;
|
||||
|
@ -51,6 +55,8 @@ import org.apache.hadoop.hbase.Server;
|
|||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
|
||||
import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
|
@ -64,6 +70,7 @@ import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
|
|||
import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.NodeFailoverWorker;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
|
@ -108,6 +115,8 @@ public class TestReplicationSourceManager {
|
|||
|
||||
private static final byte[] f1 = Bytes.toBytes("f1");
|
||||
|
||||
private static final byte[] f2 = Bytes.toBytes("f2");
|
||||
|
||||
private static final TableName test =
|
||||
TableName.valueOf("test");
|
||||
|
||||
|
@ -161,10 +170,10 @@ public class TestReplicationSourceManager {
|
|||
manager.addSource(slaveId);
|
||||
|
||||
htd = new HTableDescriptor(test);
|
||||
HColumnDescriptor col = new HColumnDescriptor("f1");
|
||||
HColumnDescriptor col = new HColumnDescriptor(f1);
|
||||
col.setScope(HConstants.REPLICATION_SCOPE_GLOBAL);
|
||||
htd.addFamily(col);
|
||||
col = new HColumnDescriptor("f2");
|
||||
col = new HColumnDescriptor(f2);
|
||||
col.setScope(HConstants.REPLICATION_SCOPE_LOCAL);
|
||||
htd.addFamily(col);
|
||||
|
||||
|
@ -416,6 +425,63 @@ public class TestReplicationSourceManager {
|
|||
s0.abort("", null);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBulkLoadWALEditsWithoutBulkLoadReplicationEnabled() throws Exception {
|
||||
// 1. Create wal key
|
||||
WALKey logKey = new WALKey();
|
||||
// 2. Get the bulk load wal edit event
|
||||
WALEdit logEdit = getBulkLoadWALEdit();
|
||||
|
||||
// 3. Get the scopes for the key
|
||||
Replication.scopeWALEdits(htd, logKey, logEdit, conf, manager);
|
||||
|
||||
// 4. Assert that no bulk load entry scopes are added if bulk load hfile replication is disabled
|
||||
assertNull("No bulk load entries scope should be added if bulk load replication is diabled.",
|
||||
logKey.getScopes());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBulkLoadWALEdits() throws Exception {
|
||||
// 1. Create wal key
|
||||
WALKey logKey = new WALKey();
|
||||
// 2. Get the bulk load wal edit event
|
||||
WALEdit logEdit = getBulkLoadWALEdit();
|
||||
// 3. Enable bulk load hfile replication
|
||||
Configuration bulkLoadConf = HBaseConfiguration.create(conf);
|
||||
bulkLoadConf.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true);
|
||||
|
||||
// 4. Get the scopes for the key
|
||||
Replication.scopeWALEdits(htd, logKey, logEdit, bulkLoadConf, manager);
|
||||
|
||||
NavigableMap<byte[], Integer> scopes = logKey.getScopes();
|
||||
// Assert family with replication scope global is present in the key scopes
|
||||
assertTrue("This family scope is set to global, should be part of replication key scopes.",
|
||||
scopes.containsKey(f1));
|
||||
// Assert family with replication scope local is not present in the key scopes
|
||||
assertFalse("This family scope is set to local, should not be part of replication key scopes",
|
||||
scopes.containsKey(f2));
|
||||
}
|
||||
|
||||
private WALEdit getBulkLoadWALEdit() {
|
||||
// 1. Create store files for the families
|
||||
Map<byte[], List<Path>> storeFiles = new HashMap<>(1);
|
||||
List<Path> p = new ArrayList<>(1);
|
||||
p.add(new Path(Bytes.toString(f1)));
|
||||
storeFiles.put(f1, p);
|
||||
|
||||
p = new ArrayList<>(1);
|
||||
p.add(new Path(Bytes.toString(f2)));
|
||||
storeFiles.put(f2, p);
|
||||
|
||||
// 2. Create bulk load descriptor
|
||||
BulkLoadDescriptor desc = ProtobufUtil.toBulkLoadDescriptor(hri.getTable(),
|
||||
ByteStringer.wrap(hri.getEncodedNameAsBytes()), storeFiles, 1);
|
||||
|
||||
// 3. create bulk load wal edit event
|
||||
WALEdit logEdit = WALEdit.createBulkLoadEvent(hri, desc);
|
||||
return logEdit;
|
||||
}
|
||||
|
||||
static class DummyNodeFailoverWorker extends Thread {
|
||||
private SortedMap<String, SortedSet<String>> logZnodesMap;
|
||||
Server server;
|
||||
|
|
|
@ -0,0 +1,25 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license
|
||||
* agreements. See the NOTICE file distributed with this work for additional information regarding
|
||||
* copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License. You may obtain a
|
||||
* copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable
|
||||
* law or agreed to in writing, software distributed under the License is distributed on an "AS IS"
|
||||
* BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License
|
||||
* for the specific language governing permissions and limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.replication.regionserver;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class TestSourceFSConfigurationProvider implements SourceFSConfigurationProvider {
|
||||
@Override
|
||||
public Configuration getConf(Configuration sinkConf, String replicationClusterId)
|
||||
throws IOException {
|
||||
return sinkConf;
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue