HBASE-6617 ReplicationSourceManager should be able to track multiple WAL paths (Yu Li)
This commit is contained in:
parent
a8730c2839
commit
c94d10952f
|
@ -3327,4 +3327,13 @@ public class HRegionServer extends HasThread implements
|
|||
}
|
||||
return max;
|
||||
}
|
||||
|
||||
/**
|
||||
* For testing
|
||||
* @return whether all wal roll request finished for this regionserver
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public boolean walRollRequestFinished() {
|
||||
return this.walRoller.walRollFinished();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -191,4 +191,18 @@ public class LogRoller extends HasThread {
|
|||
requester);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* For testing only
|
||||
* @return true if all WAL roll finished
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public boolean walRollFinished() {
|
||||
for (boolean needRoll : walNeedsRoll.values()) {
|
||||
if (needRoll) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
}
|
|
@ -137,6 +137,7 @@ public interface ReplicationEndpoint extends Service {
|
|||
static class ReplicateContext {
|
||||
List<Entry> entries;
|
||||
int size;
|
||||
String walGroupId;
|
||||
@InterfaceAudience.Private
|
||||
public ReplicateContext() {
|
||||
}
|
||||
|
@ -149,12 +150,19 @@ public interface ReplicationEndpoint extends Service {
|
|||
this.size = size;
|
||||
return this;
|
||||
}
|
||||
public ReplicateContext setWalGroupId(String walGroupId) {
|
||||
this.walGroupId = walGroupId;
|
||||
return this;
|
||||
}
|
||||
public List<Entry> getEntries() {
|
||||
return entries;
|
||||
}
|
||||
public int getSize() {
|
||||
return size;
|
||||
}
|
||||
public String getWalGroupId(){
|
||||
return walGroupId;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -154,6 +154,7 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
|
|||
@Override
|
||||
public boolean replicate(ReplicateContext replicateContext) {
|
||||
List<Entry> entries = replicateContext.getEntries();
|
||||
String walGroupId = replicateContext.getWalGroupId();
|
||||
int sleepMultiplier = 1;
|
||||
|
||||
if (!peersSelected && this.isRunning()) {
|
||||
|
@ -219,12 +220,13 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
|
|||
throw iox;
|
||||
}
|
||||
// update metrics
|
||||
this.metrics.setAgeOfLastShippedOp(entries.get(entries.size()-1).getKey().getWriteTime());
|
||||
this.metrics.setAgeOfLastShippedOp(entries.get(entries.size() - 1).getKey().getWriteTime(),
|
||||
walGroupId);
|
||||
return true;
|
||||
|
||||
} catch (IOException ioe) {
|
||||
// Didn't ship anything, but must still age the last time we did
|
||||
this.metrics.refreshAgeOfLastShippedOp();
|
||||
this.metrics.refreshAgeOfLastShippedOp(walGroupId);
|
||||
if (ioe instanceof RemoteException) {
|
||||
ioe = ((RemoteException) ioe).unwrapRemoteException();
|
||||
LOG.warn("Can't replicate because of an error on the remote cluster: ", ioe);
|
||||
|
|
|
@ -18,6 +18,9 @@
|
|||
|
||||
package org.apache.hadoop.hbase.replication.regionserver;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
@ -34,7 +37,8 @@ public class MetricsSource {
|
|||
|
||||
private static final Log LOG = LogFactory.getLog(MetricsSource.class);
|
||||
|
||||
private long lastTimestamp = 0;
|
||||
// tracks last shipped timestamp for each wal group
|
||||
private Map<String, Long> lastTimeStamps = new HashMap<String, Long>();
|
||||
private int lastQueueSize = 0;
|
||||
private String id;
|
||||
|
||||
|
@ -56,23 +60,29 @@ public class MetricsSource {
|
|||
|
||||
/**
|
||||
* Set the age of the last edit that was shipped
|
||||
*
|
||||
* @param timestamp write time of the edit
|
||||
* @param walGroup which group we are setting
|
||||
*/
|
||||
public void setAgeOfLastShippedOp(long timestamp) {
|
||||
public void setAgeOfLastShippedOp(long timestamp, String walGroup) {
|
||||
long age = EnvironmentEdgeManager.currentTime() - timestamp;
|
||||
singleSourceSource.setLastShippedAge(age);
|
||||
globalSourceSource.setLastShippedAge(age);
|
||||
this.lastTimestamp = timestamp;
|
||||
this.lastTimeStamps.put(walGroup, timestamp);
|
||||
}
|
||||
|
||||
/**
|
||||
* Convenience method to use the last given timestamp to refresh the age of the last edit. Used
|
||||
* when replication fails and need to keep that metric accurate.
|
||||
* @param walGroupId id of the group to update
|
||||
*/
|
||||
public void refreshAgeOfLastShippedOp() {
|
||||
if (this.lastTimestamp > 0) {
|
||||
setAgeOfLastShippedOp(this.lastTimestamp);
|
||||
public void refreshAgeOfLastShippedOp(String walGroupId) {
|
||||
Long lastTimestamp = this.lastTimeStamps.get(walGroupId);
|
||||
if (lastTimestamp == null) {
|
||||
this.lastTimeStamps.put(walGroupId, 0L);
|
||||
lastTimestamp = 0L;
|
||||
}
|
||||
if (lastTimestamp > 0) {
|
||||
setAgeOfLastShippedOp(lastTimestamp, walGroupId);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -143,6 +153,7 @@ public class MetricsSource {
|
|||
public void clear() {
|
||||
singleSourceSource.clear();
|
||||
globalSourceSource.decrSizeOfLogQueue(lastQueueSize);
|
||||
lastTimeStamps.clear();
|
||||
lastQueueSize = 0;
|
||||
}
|
||||
|
||||
|
@ -163,10 +174,16 @@ public class MetricsSource {
|
|||
}
|
||||
|
||||
/**
|
||||
* Get the timeStampsOfLastShippedOp
|
||||
* Get the timeStampsOfLastShippedOp, if there are multiple groups, return the latest one
|
||||
* @return lastTimestampForAge
|
||||
*/
|
||||
public long getTimeStampOfLastShippedOp() {
|
||||
long lastTimestamp = 0L;
|
||||
for (long ts : lastTimeStamps.values()) {
|
||||
if (ts > lastTimestamp) {
|
||||
lastTimestamp = ts;
|
||||
}
|
||||
}
|
||||
return lastTimestamp;
|
||||
}
|
||||
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -23,9 +23,12 @@ import java.io.IOException;
|
|||
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.SortedMap;
|
||||
import java.util.SortedSet;
|
||||
import java.util.TreeSet;
|
||||
|
@ -56,6 +59,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeers;
|
|||
import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationQueues;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationTracker;
|
||||
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
|
||||
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
|
||||
|
@ -91,13 +95,14 @@ public class ReplicationSourceManager implements ReplicationListener {
|
|||
// All about stopping
|
||||
private final Server server;
|
||||
// All logs we are currently tracking
|
||||
private final Map<String, SortedSet<String>> walsById;
|
||||
// Index structure of the map is: peer_id->logPrefix/logGroup->logs
|
||||
private final Map<String, Map<String, SortedSet<String>>> walsById;
|
||||
// Logs for recovered sources we are currently tracking
|
||||
private final Map<String, SortedSet<String>> walsByIdRecoveredQueues;
|
||||
private final Map<String, Map<String, SortedSet<String>>> walsByIdRecoveredQueues;
|
||||
private final Configuration conf;
|
||||
private final FileSystem fs;
|
||||
// The path to the latest log we saw, for new coming sources
|
||||
private Path latestPath;
|
||||
// The paths to the latest log of each wal group, for new coming peers
|
||||
private Set<Path> latestPaths;
|
||||
// Path to the wals directories
|
||||
private final Path logDir;
|
||||
// Path to the wal archive
|
||||
|
@ -133,8 +138,8 @@ public class ReplicationSourceManager implements ReplicationListener {
|
|||
this.replicationPeers = replicationPeers;
|
||||
this.replicationTracker = replicationTracker;
|
||||
this.server = server;
|
||||
this.walsById = new HashMap<String, SortedSet<String>>();
|
||||
this.walsByIdRecoveredQueues = new ConcurrentHashMap<String, SortedSet<String>>();
|
||||
this.walsById = new HashMap<String, Map<String, SortedSet<String>>>();
|
||||
this.walsByIdRecoveredQueues = new ConcurrentHashMap<String, Map<String, SortedSet<String>>>();
|
||||
this.oldsources = new CopyOnWriteArrayList<ReplicationSourceInterface>();
|
||||
this.conf = conf;
|
||||
this.fs = fs;
|
||||
|
@ -158,6 +163,7 @@ public class ReplicationSourceManager implements ReplicationListener {
|
|||
tfb.setDaemon(true);
|
||||
this.executor.setThreadFactory(tfb.build());
|
||||
this.rand = new Random();
|
||||
this.latestPaths = Collections.synchronizedSet(new HashSet<Path>());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -189,15 +195,16 @@ public class ReplicationSourceManager implements ReplicationListener {
|
|||
* @param queueRecovered Whether this is a recovered queue
|
||||
*/
|
||||
public void cleanOldLogs(String key, String id, boolean queueRecovered) {
|
||||
String logPrefix = DefaultWALProvider.getWALPrefixFromWALName(key);
|
||||
if (queueRecovered) {
|
||||
SortedSet<String> wals = walsByIdRecoveredQueues.get(id);
|
||||
SortedSet<String> wals = walsByIdRecoveredQueues.get(id).get(logPrefix);
|
||||
if (wals != null && !wals.first().equals(key)) {
|
||||
cleanOldLogs(wals, key, id);
|
||||
}
|
||||
} else {
|
||||
synchronized (this.walsById) {
|
||||
SortedSet<String> wals = walsById.get(id);
|
||||
if (!wals.first().equals(key)) {
|
||||
SortedSet<String> wals = walsById.get(id).get(logPrefix);
|
||||
if (wals != null && !wals.first().equals(key)) {
|
||||
cleanOldLogs(wals, key, id);
|
||||
}
|
||||
}
|
||||
|
@ -238,36 +245,44 @@ public class ReplicationSourceManager implements ReplicationListener {
|
|||
}
|
||||
|
||||
/**
|
||||
* Add a new normal source to this region server
|
||||
* Add sources for the given peer cluster on this region server. For the newly added peer, we only
|
||||
* need to enqueue the latest log of each wal group and do replication
|
||||
* @param id the id of the peer cluster
|
||||
* @return the source that was created
|
||||
* @throws IOException
|
||||
*/
|
||||
protected ReplicationSourceInterface addSource(String id) throws IOException,
|
||||
ReplicationException {
|
||||
ReplicationPeerConfig peerConfig
|
||||
= replicationPeers.getReplicationPeerConfig(id);
|
||||
ReplicationPeerConfig peerConfig = replicationPeers.getReplicationPeerConfig(id);
|
||||
ReplicationPeer peer = replicationPeers.getPeer(id);
|
||||
ReplicationSourceInterface src =
|
||||
getReplicationSource(this.conf, this.fs, this, this.replicationQueues,
|
||||
this.replicationPeers, server, id, this.clusterId, peerConfig, peer);
|
||||
synchronized (this.walsById) {
|
||||
this.sources.add(src);
|
||||
this.walsById.put(id, new TreeSet<String>());
|
||||
Map<String, SortedSet<String>> walsByGroup = new HashMap<String, SortedSet<String>>();
|
||||
this.walsById.put(id, walsByGroup);
|
||||
// Add the latest wal to that source's queue
|
||||
if (this.latestPath != null) {
|
||||
String name = this.latestPath.getName();
|
||||
this.walsById.get(id).add(name);
|
||||
try {
|
||||
this.replicationQueues.addLog(src.getPeerClusterZnode(), name);
|
||||
} catch (ReplicationException e) {
|
||||
String message =
|
||||
"Cannot add log to queue when creating a new source, queueId="
|
||||
+ src.getPeerClusterZnode() + ", filename=" + name;
|
||||
server.stop(message);
|
||||
throw e;
|
||||
synchronized (latestPaths) {
|
||||
if (this.latestPaths.size() > 0) {
|
||||
for (Path logPath : latestPaths) {
|
||||
String name = logPath.getName();
|
||||
String walPrefix = DefaultWALProvider.getWALPrefixFromWALName(name);
|
||||
SortedSet<String> logs = new TreeSet<String>();
|
||||
logs.add(name);
|
||||
walsByGroup.put(walPrefix, logs);
|
||||
try {
|
||||
this.replicationQueues.addLog(id, name);
|
||||
} catch (ReplicationException e) {
|
||||
String message =
|
||||
"Cannot add log to queue when creating a new source, queueId=" + id
|
||||
+ ", filename=" + name;
|
||||
server.stop(message);
|
||||
throw e;
|
||||
}
|
||||
src.enqueueLog(logPath);
|
||||
}
|
||||
}
|
||||
src.enqueueLog(this.latestPath);
|
||||
}
|
||||
}
|
||||
src.startup();
|
||||
|
@ -302,7 +317,7 @@ public class ReplicationSourceManager implements ReplicationListener {
|
|||
* Get a copy of the wals of the first source on this rs
|
||||
* @return a sorted set of wal names
|
||||
*/
|
||||
protected Map<String, SortedSet<String>> getWALs() {
|
||||
protected Map<String, Map<String, SortedSet<String>>> getWALs() {
|
||||
return Collections.unmodifiableMap(walsById);
|
||||
}
|
||||
|
||||
|
@ -310,7 +325,7 @@ public class ReplicationSourceManager implements ReplicationListener {
|
|||
* Get a copy of the wals of the recovered sources on this rs
|
||||
* @return a sorted set of wal names
|
||||
*/
|
||||
protected Map<String, SortedSet<String>> getWalsByIdRecoveredQueues() {
|
||||
protected Map<String, Map<String, SortedSet<String>>> getWalsByIdRecoveredQueues() {
|
||||
return Collections.unmodifiableMap(walsByIdRecoveredQueues);
|
||||
}
|
||||
|
||||
|
@ -331,27 +346,70 @@ public class ReplicationSourceManager implements ReplicationListener {
|
|||
}
|
||||
|
||||
void preLogRoll(Path newLog) throws IOException {
|
||||
synchronized (this.walsById) {
|
||||
String name = newLog.getName();
|
||||
for (ReplicationSourceInterface source : this.sources) {
|
||||
try {
|
||||
this.replicationQueues.addLog(source.getPeerClusterZnode(), name);
|
||||
} catch (ReplicationException e) {
|
||||
throw new IOException("Cannot add log to replication queue with id="
|
||||
+ source.getPeerClusterZnode() + ", filename=" + name, e);
|
||||
recordLog(newLog);
|
||||
String logName = newLog.getName();
|
||||
String logPrefix = DefaultWALProvider.getWALPrefixFromWALName(logName);
|
||||
synchronized (latestPaths) {
|
||||
Iterator<Path> iterator = latestPaths.iterator();
|
||||
while (iterator.hasNext()) {
|
||||
Path path = iterator.next();
|
||||
if (path.getName().contains(logPrefix)) {
|
||||
iterator.remove();
|
||||
break;
|
||||
}
|
||||
}
|
||||
for (SortedSet<String> wals : this.walsById.values()) {
|
||||
if (this.sources.isEmpty()) {
|
||||
// If there's no slaves, don't need to keep the old wals since
|
||||
// we only consider the last one when a new slave comes in
|
||||
wals.clear();
|
||||
this.latestPaths.add(newLog);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Check and enqueue the given log to the correct source. If there's still no source for the
|
||||
* group to which the given log belongs, create one
|
||||
* @param logPath the log path to check and enqueue
|
||||
* @throws IOException
|
||||
*/
|
||||
private void recordLog(Path logPath) throws IOException {
|
||||
String logName = logPath.getName();
|
||||
String logPrefix = DefaultWALProvider.getWALPrefixFromWALName(logName);
|
||||
// update replication queues on ZK
|
||||
synchronized (replicationPeers) {// synchronize on replicationPeers to avoid adding source for
|
||||
// the to-be-removed peer
|
||||
for (String id : replicationPeers.getPeerIds()) {
|
||||
try {
|
||||
this.replicationQueues.addLog(id, logName);
|
||||
} catch (ReplicationException e) {
|
||||
throw new IOException("Cannot add log to replication queue"
|
||||
+ " when creating a new source, queueId=" + id + ", filename=" + logName, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
// update walsById map
|
||||
synchronized (walsById) {
|
||||
for (Map.Entry<String, Map<String, SortedSet<String>>> entry : this.walsById.entrySet()) {
|
||||
String peerId = entry.getKey();
|
||||
Map<String, SortedSet<String>> walsByPrefix = entry.getValue();
|
||||
boolean existingPrefix = false;
|
||||
for (Map.Entry<String, SortedSet<String>> walsEntry : walsByPrefix.entrySet()) {
|
||||
SortedSet<String> wals = walsEntry.getValue();
|
||||
if (this.sources.isEmpty()) {
|
||||
// If there's no slaves, don't need to keep the old wals since
|
||||
// we only consider the last one when a new slave comes in
|
||||
wals.clear();
|
||||
}
|
||||
if (logPrefix.equals(walsEntry.getKey())) {
|
||||
wals.add(logName);
|
||||
existingPrefix = true;
|
||||
}
|
||||
}
|
||||
if (!existingPrefix) {
|
||||
// The new log belongs to a new group, add it into this peer
|
||||
LOG.debug("Start tracking logs for wal group " + logPrefix + " for peer " + peerId);
|
||||
SortedSet<String> wals = new TreeSet<String>();
|
||||
wals.add(logName);
|
||||
walsByPrefix.put(logPrefix, wals);
|
||||
}
|
||||
wals.add(name);
|
||||
}
|
||||
}
|
||||
|
||||
this.latestPath = newLog;
|
||||
}
|
||||
|
||||
void postLogRoll(Path newLog) throws IOException {
|
||||
|
@ -376,7 +434,7 @@ public class ReplicationSourceManager implements ReplicationListener {
|
|||
final ReplicationQueues replicationQueues, final ReplicationPeers replicationPeers,
|
||||
final Server server, final String peerId, final UUID clusterId,
|
||||
final ReplicationPeerConfig peerConfig, final ReplicationPeer replicationPeer)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
RegionServerCoprocessorHost rsServerHost = null;
|
||||
TableDescriptors tableDescriptors = null;
|
||||
if (server instanceof HRegionServer) {
|
||||
|
@ -414,7 +472,8 @@ public class ReplicationSourceManager implements ReplicationListener {
|
|||
}
|
||||
}
|
||||
} catch (Exception e) {
|
||||
LOG.warn("Passed replication endpoint implementation throws errors", e);
|
||||
LOG.warn("Passed replication endpoint implementation throws errors"
|
||||
+ " while initializing ReplicationSource for peer: " + peerId, e);
|
||||
throw new IOException(e);
|
||||
}
|
||||
|
||||
|
@ -470,7 +529,7 @@ public class ReplicationSourceManager implements ReplicationListener {
|
|||
+ sources.size() + " and another "
|
||||
+ oldsources.size() + " that were recovered");
|
||||
String terminateMessage = "Replication stream was removed by a user";
|
||||
ReplicationSourceInterface srcToRemove = null;
|
||||
List<ReplicationSourceInterface> srcToRemove = new ArrayList<ReplicationSourceInterface>();
|
||||
List<ReplicationSourceInterface> oldSourcesToDelete =
|
||||
new ArrayList<ReplicationSourceInterface>();
|
||||
// First close all the recovered sources for this peer
|
||||
|
@ -486,19 +545,23 @@ public class ReplicationSourceManager implements ReplicationListener {
|
|||
LOG.info("Number of deleted recovered sources for " + id + ": "
|
||||
+ oldSourcesToDelete.size());
|
||||
// Now look for the one on this cluster
|
||||
for (ReplicationSourceInterface src : this.sources) {
|
||||
if (id.equals(src.getPeerClusterId())) {
|
||||
srcToRemove = src;
|
||||
break;
|
||||
synchronized (this.replicationPeers) {// synchronize on replicationPeers to avoid adding source
|
||||
// for the to-be-removed peer
|
||||
for (ReplicationSourceInterface src : this.sources) {
|
||||
if (id.equals(src.getPeerClusterId())) {
|
||||
srcToRemove.add(src);
|
||||
}
|
||||
}
|
||||
if (srcToRemove.size() == 0) {
|
||||
LOG.error("The queue we wanted to close is missing " + id);
|
||||
return;
|
||||
}
|
||||
for (ReplicationSourceInterface toRemove : srcToRemove) {
|
||||
toRemove.terminate(terminateMessage);
|
||||
this.sources.remove(toRemove);
|
||||
}
|
||||
deleteSource(id, true);
|
||||
}
|
||||
if (srcToRemove == null) {
|
||||
LOG.error("The queue we wanted to close is missing " + id);
|
||||
return;
|
||||
}
|
||||
srcToRemove.terminate(terminateMessage);
|
||||
this.sources.remove(srcToRemove);
|
||||
deleteSource(id, true);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -580,6 +643,7 @@ public class ReplicationSourceManager implements ReplicationListener {
|
|||
|
||||
for (Map.Entry<String, SortedSet<String>> entry : newQueues.entrySet()) {
|
||||
String peerId = entry.getKey();
|
||||
SortedSet<String> walsSet = entry.getValue();
|
||||
try {
|
||||
// there is not an actual peer defined corresponding to peerId for the failover.
|
||||
ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(peerId);
|
||||
|
@ -596,7 +660,20 @@ public class ReplicationSourceManager implements ReplicationListener {
|
|||
LOG.warn("Skipping failover for peer:" + actualPeerId + " of node" + rsZnode);
|
||||
continue;
|
||||
}
|
||||
// track sources in walsByIdRecoveredQueues
|
||||
Map<String, SortedSet<String>> walsByGroup = new HashMap<String, SortedSet<String>>();
|
||||
walsByIdRecoveredQueues.put(peerId, walsByGroup);
|
||||
for (String wal : walsSet) {
|
||||
String walPrefix = DefaultWALProvider.getWALPrefixFromWALName(wal);
|
||||
SortedSet<String> wals = walsByGroup.get(walPrefix);
|
||||
if (wals == null) {
|
||||
wals = new TreeSet<String>();
|
||||
walsByGroup.put(walPrefix, wals);
|
||||
}
|
||||
wals.add(wal);
|
||||
}
|
||||
|
||||
// enqueue sources
|
||||
ReplicationSourceInterface src =
|
||||
getReplicationSource(conf, fs, ReplicationSourceManager.this, this.rq, this.rp,
|
||||
server, peerId, this.clusterId, peerConfig, peer);
|
||||
|
@ -605,12 +682,10 @@ public class ReplicationSourceManager implements ReplicationListener {
|
|||
break;
|
||||
}
|
||||
oldsources.add(src);
|
||||
SortedSet<String> walsSet = entry.getValue();
|
||||
for (String wal : walsSet) {
|
||||
src.enqueueLog(new Path(oldLogDir, wal));
|
||||
}
|
||||
src.startup();
|
||||
walsByIdRecoveredQueues.put(peerId, walsSet);
|
||||
} catch (IOException e) {
|
||||
// TODO manage it
|
||||
LOG.error("Failed creating a source", e);
|
||||
|
|
|
@ -363,4 +363,15 @@ public class DefaultWALProvider implements WALProvider {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get prefix of the log from its name, assuming WAL name in format of
|
||||
* log_prefix.filenumber.log_suffix @see {@link FSHLog#getCurrentFileName()}
|
||||
* @param name Name of the WAL to parse
|
||||
* @return prefix of the log
|
||||
*/
|
||||
public static String getWALPrefixFromWALName(String name) {
|
||||
int endIndex = name.replaceAll(META_WAL_PROVIDER_ID, "").lastIndexOf(".");
|
||||
return name.substring(0, endIndex);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -83,7 +83,7 @@ public class WALFactory {
|
|||
}
|
||||
}
|
||||
|
||||
static final String WAL_PROVIDER = "hbase.wal.provider";
|
||||
public static final String WAL_PROVIDER = "hbase.wal.provider";
|
||||
static final String DEFAULT_WAL_PROVIDER = Providers.defaultProvider.name();
|
||||
|
||||
static final String META_WAL_PROVIDER = "hbase.wal.meta_provider";
|
||||
|
|
|
@ -74,15 +74,40 @@ public class TestReplicationEndpoint extends TestReplicationBase {
|
|||
}
|
||||
|
||||
@Before
|
||||
public void setup() throws FailedLogCloseException, IOException {
|
||||
public void setup() throws Exception {
|
||||
ReplicationEndpointForTest.contructedCount.set(0);
|
||||
ReplicationEndpointForTest.startedCount.set(0);
|
||||
ReplicationEndpointForTest.replicateCount.set(0);
|
||||
ReplicationEndpointReturningFalse.replicated.set(false);
|
||||
ReplicationEndpointForTest.lastEntries = null;
|
||||
for (RegionServerThread rs : utility1.getMiniHBaseCluster().getRegionServerThreads()) {
|
||||
final List<RegionServerThread> rsThreads =
|
||||
utility1.getMiniHBaseCluster().getRegionServerThreads();
|
||||
for (RegionServerThread rs : rsThreads) {
|
||||
utility1.getHBaseAdmin().rollWALWriter(rs.getRegionServer().getServerName());
|
||||
}
|
||||
// Wait for all log roll to finish
|
||||
utility1.waitFor(3000, new Waiter.ExplainingPredicate<Exception>() {
|
||||
@Override
|
||||
public boolean evaluate() throws Exception {
|
||||
for (RegionServerThread rs : rsThreads) {
|
||||
if (!rs.getRegionServer().walRollRequestFinished()) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String explainFailure() throws Exception {
|
||||
List<String> logRollInProgressRsList = new ArrayList<String>();
|
||||
for (RegionServerThread rs : rsThreads) {
|
||||
if (!rs.getRegionServer().walRollRequestFinished()) {
|
||||
logRollInProgressRsList.add(rs.getRegionServer().toString());
|
||||
}
|
||||
}
|
||||
return "Still waiting for log roll on regionservers: " + logRollInProgressRsList;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Test (timeout=120000)
|
||||
|
|
|
@ -0,0 +1,34 @@
|
|||
/**
|
||||
* 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.multiwal;
|
||||
|
||||
import org.apache.hadoop.hbase.replication.TestReplicationEndpoint;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
|
||||
import org.apache.hadoop.hbase.wal.WALFactory;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
@Category({ ReplicationTests.class, MediumTests.class })
|
||||
public class TestReplicationEndpointWithMultipleWAL extends TestReplicationEndpoint {
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
conf1.set(WALFactory.WAL_PROVIDER, "multiwal");
|
||||
TestReplicationEndpoint.setUpBeforeClass();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,35 @@
|
|||
/**
|
||||
* 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.multiwal;
|
||||
|
||||
import org.apache.hadoop.hbase.replication.TestReplicationKillMasterRSCompressed;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
|
||||
import org.apache.hadoop.hbase.wal.WALFactory;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
@Category({ReplicationTests.class, LargeTests.class})
|
||||
public class TestReplicationKillMasterRSCompressedWithMultipleWAL extends
|
||||
TestReplicationKillMasterRSCompressed {
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
conf1.set(WALFactory.WAL_PROVIDER, "multiwal");
|
||||
TestReplicationKillMasterRSCompressed.setUpBeforeClass();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,35 @@
|
|||
/**
|
||||
* 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.multiwal;
|
||||
|
||||
import org.apache.hadoop.hbase.replication.TestReplicationBase;
|
||||
import org.apache.hadoop.hbase.replication.TestReplicationSyncUpTool;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
|
||||
import org.apache.hadoop.hbase.wal.WALFactory;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
@Category({ ReplicationTests.class, LargeTests.class })
|
||||
public class TestReplicationSyncUpToolWithMultipleWAL extends TestReplicationSyncUpTool {
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
conf1.set(WALFactory.WAL_PROVIDER, "multiwal");
|
||||
TestReplicationBase.setUpBeforeClass();
|
||||
}
|
||||
}
|
|
@ -257,7 +257,9 @@ public class TestRegionReplicaReplicationEndpointNoMaster {
|
|||
|
||||
Assert.assertEquals(1000, entries.size());
|
||||
// replay the edits to the secondary using replay callable
|
||||
replicator.replicate(new ReplicateContext().setEntries(Lists.newArrayList(entries)));
|
||||
final String fakeWalGroupId = "fakeWALGroup";
|
||||
replicator.replicate(new ReplicateContext().setEntries(Lists.newArrayList(entries))
|
||||
.setWalGroupId(fakeWalGroupId));
|
||||
|
||||
Region region = rs0.getFromOnlineRegions(hriSecondary.getEncodedName());
|
||||
HTU.verifyNumericRows(region, f, 0, 1000);
|
||||
|
|
|
@ -25,6 +25,7 @@ import java.net.URLEncoder;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.SortedMap;
|
||||
import java.util.SortedSet;
|
||||
import java.util.TreeSet;
|
||||
|
@ -229,7 +230,11 @@ public class TestReplicationSourceManager {
|
|||
}
|
||||
wal.sync();
|
||||
|
||||
assertEquals(6, manager.getWALs().get(slaveId).size());
|
||||
int logNumber = 0;
|
||||
for (Map.Entry<String, SortedSet<String>> entry : manager.getWALs().get(slaveId).entrySet()) {
|
||||
logNumber += entry.getValue().size();
|
||||
}
|
||||
assertEquals(6, logNumber);
|
||||
|
||||
wal.rollWriter();
|
||||
|
||||
|
@ -298,8 +303,11 @@ public class TestReplicationSourceManager {
|
|||
rq.init(server.getServerName().toString());
|
||||
// populate some znodes in the peer znode
|
||||
SortedSet<String> files = new TreeSet<String>();
|
||||
files.add("log1");
|
||||
files.add("log2");
|
||||
String group = "testgroup";
|
||||
String file1 = group + ".log1";
|
||||
String file2 = group + ".log2";
|
||||
files.add(file1);
|
||||
files.add(file2);
|
||||
for (String file : files) {
|
||||
rq.addLog("1", file);
|
||||
}
|
||||
|
@ -317,10 +325,10 @@ public class TestReplicationSourceManager {
|
|||
w1.join(5000);
|
||||
assertEquals(1, manager.getWalsByIdRecoveredQueues().size());
|
||||
String id = "1-" + server.getServerName().getServerName();
|
||||
assertEquals(files, manager.getWalsByIdRecoveredQueues().get(id));
|
||||
manager.cleanOldLogs("log2", id, true);
|
||||
assertEquals(files, manager.getWalsByIdRecoveredQueues().get(id).get(group));
|
||||
manager.cleanOldLogs(file2, id, true);
|
||||
// log1 should be deleted
|
||||
assertEquals(Sets.newHashSet("log2"), manager.getWalsByIdRecoveredQueues().get(id));
|
||||
assertEquals(Sets.newHashSet(file2), manager.getWalsByIdRecoveredQueues().get(id).get(group));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -39,8 +39,10 @@ import org.junit.After;
|
|||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
import org.junit.runners.Parameterized.Parameters;
|
||||
|
@ -75,6 +77,7 @@ public class TestReplicationWALReaderManager {
|
|||
private int nbRows;
|
||||
private int walEditKVs;
|
||||
private final AtomicLong sequenceId = new AtomicLong(1);
|
||||
@Rule public TestName tn = new TestName();
|
||||
|
||||
@Parameters
|
||||
public static Collection<Object[]> parameters() {
|
||||
|
@ -127,7 +130,7 @@ public class TestReplicationWALReaderManager {
|
|||
List<WALActionsListener> listeners = new ArrayList<WALActionsListener>();
|
||||
pathWatcher = new PathWatcher();
|
||||
listeners.add(pathWatcher);
|
||||
final WALFactory wals = new WALFactory(conf, listeners, "some server");
|
||||
final WALFactory wals = new WALFactory(conf, listeners, tn.getMethodName());
|
||||
log = wals.getWAL(info.getEncodedNameAsBytes());
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue