HBASE-6617 ReplicationSourceManager should be able to track multiple WAL paths (Yu Li)
This commit is contained in:
parent
84dbe39f5d
commit
be96bb6adf
|
@ -3313,4 +3313,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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -197,4 +197,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;
|
||||
}
|
||||
|
||||
|
|
|
@ -22,11 +22,17 @@ import java.io.EOFException;
|
|||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.PriorityBlockingQueue;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.commons.lang.StringUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -77,8 +83,12 @@ public class ReplicationSource extends Thread
|
|||
implements ReplicationSourceInterface {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(ReplicationSource.class);
|
||||
// Queue of logs to process
|
||||
private PriorityBlockingQueue<Path> queue;
|
||||
// Queues of logs to process, entry in format of walGroupId->queue,
|
||||
// each presents a queue for one wal group
|
||||
private Map<String, PriorityBlockingQueue<Path>> queues =
|
||||
new HashMap<String, PriorityBlockingQueue<Path>>();
|
||||
// per group queue size, keep no more than this number of logs in each wal group
|
||||
private int queueSizePerGroup;
|
||||
private ReplicationQueues replicationQueues;
|
||||
private ReplicationPeers replicationPeers;
|
||||
|
||||
|
@ -96,35 +106,23 @@ public class ReplicationSource extends Thread
|
|||
private long replicationQueueSizeCapacity;
|
||||
// Max number of entries in entriesArray
|
||||
private int replicationQueueNbCapacity;
|
||||
// Our reader for the current log. open/close handled by repLogReader
|
||||
private WAL.Reader reader;
|
||||
// Last position in the log that we sent to ZooKeeper
|
||||
private long lastLoggedPosition = -1;
|
||||
// Path of the current log
|
||||
private volatile Path currentPath;
|
||||
private FileSystem fs;
|
||||
// id of this cluster
|
||||
private UUID clusterId;
|
||||
// id of the other cluster
|
||||
private UUID peerClusterId;
|
||||
// total number of edits we replicated
|
||||
private long totalReplicatedEdits = 0;
|
||||
private AtomicLong totalReplicatedEdits = new AtomicLong(0);
|
||||
// total number of edits we replicated
|
||||
private long totalReplicatedOperations = 0;
|
||||
private AtomicLong totalReplicatedOperations = new AtomicLong(0);
|
||||
// The znode we currently play with
|
||||
private String peerClusterZnode;
|
||||
// Maximum number of retries before taking bold actions
|
||||
private int maxRetriesMultiplier;
|
||||
// Current number of operations (Put/Delete) that we need to replicate
|
||||
private int currentNbOperations = 0;
|
||||
// Current size of data we need to replicate
|
||||
private int currentSize = 0;
|
||||
// Indicates if this particular source is running
|
||||
private volatile boolean running = true;
|
||||
private volatile boolean sourceRunning = false;
|
||||
// Metrics for this source
|
||||
private MetricsSource metrics;
|
||||
// Handle on the log reader helper
|
||||
private ReplicationWALReaderManager repLogReader;
|
||||
//WARN threshold for the number of queued logs, defaults to 2
|
||||
private int logQueueWarnThreshold;
|
||||
// ReplicationEndpoint which will handle the actual replication
|
||||
|
@ -133,6 +131,9 @@ public class ReplicationSource extends Thread
|
|||
private WALEntryFilter walEntryFilter;
|
||||
// throttler
|
||||
private ReplicationThrottler throttler;
|
||||
private AtomicInteger logQueueSize = new AtomicInteger(0);
|
||||
private ConcurrentHashMap<String, ReplicationSourceWorkerThread> workerThreads =
|
||||
new ConcurrentHashMap<String, ReplicationSourceWorkerThread>();
|
||||
|
||||
/**
|
||||
* Instantiation method used by region servers
|
||||
|
@ -165,10 +166,7 @@ public class ReplicationSource extends Thread
|
|||
this.conf.getLong("replication.source.sleepforretries", 1000); // 1 second
|
||||
this.maxRetriesMultiplier =
|
||||
this.conf.getInt("replication.source.maxretriesmultiplier", 300); // 5 minutes @ 1 sec per
|
||||
this.queue =
|
||||
new PriorityBlockingQueue<Path>(
|
||||
this.conf.getInt("hbase.regionserver.maxlogs", 32),
|
||||
new LogsComparator());
|
||||
this.queueSizePerGroup = this.conf.getInt("hbase.regionserver.maxlogs", 32);
|
||||
long bandwidth = this.conf.getLong("replication.source.per.peer.node.bandwidth", 0);
|
||||
this.throttler = new ReplicationThrottler((double)bandwidth/10.0);
|
||||
this.replicationQueues = replicationQueues;
|
||||
|
@ -176,7 +174,6 @@ public class ReplicationSource extends Thread
|
|||
this.manager = manager;
|
||||
this.fs = fs;
|
||||
this.metrics = metrics;
|
||||
this.repLogReader = new ReplicationWALReaderManager(this.fs, this.conf);
|
||||
this.clusterId = clusterId;
|
||||
|
||||
this.peerClusterZnode = peerClusterZnode;
|
||||
|
@ -196,13 +193,33 @@ public class ReplicationSource extends Thread
|
|||
|
||||
@Override
|
||||
public void enqueueLog(Path log) {
|
||||
this.queue.put(log);
|
||||
int queueSize = queue.size();
|
||||
String logPrefix = DefaultWALProvider.getWALPrefixFromWALName(log.getName());
|
||||
PriorityBlockingQueue<Path> queue = queues.get(logPrefix);
|
||||
if (queue == null) {
|
||||
queue = new PriorityBlockingQueue<Path>(queueSizePerGroup, new LogsComparator());
|
||||
queues.put(logPrefix, queue);
|
||||
if (this.sourceRunning) {
|
||||
// new wal group observed after source startup, start a new worker thread to track it
|
||||
// notice: it's possible that log enqueued when this.running is set but worker thread
|
||||
// still not launched, so it's necessary to check workerThreads before start the worker
|
||||
final ReplicationSourceWorkerThread worker =
|
||||
new ReplicationSourceWorkerThread(logPrefix, queue, replicationQueueInfo, this);
|
||||
ReplicationSourceWorkerThread extant = workerThreads.putIfAbsent(logPrefix, worker);
|
||||
if (extant != null) {
|
||||
LOG.debug("Someone has beat us to start a worker thread for wal group " + logPrefix);
|
||||
} else {
|
||||
LOG.debug("Starting up worker for wal group " + logPrefix);
|
||||
worker.startup();
|
||||
}
|
||||
}
|
||||
}
|
||||
queue.put(log);
|
||||
int queueSize = logQueueSize.incrementAndGet();
|
||||
this.metrics.setSizeOfLogQueue(queueSize);
|
||||
// This will log a warning for each new log that gets created above the warn threshold
|
||||
if (queueSize > this.logQueueWarnThreshold) {
|
||||
LOG.warn("Queue size: " + queueSize +
|
||||
" exceeds value of replication.source.log.queue.warn: " + logQueueWarnThreshold);
|
||||
if (queue.size() > this.logQueueWarnThreshold) {
|
||||
LOG.warn("WAL group " + logPrefix + " queue size: " + queueSize
|
||||
+ " exceeds value of replication.source.log.queue.warn: " + logQueueWarnThreshold);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -217,12 +234,8 @@ public class ReplicationSource extends Thread
|
|||
|
||||
@Override
|
||||
public void run() {
|
||||
// We were stopped while looping to connect to sinks, just abort
|
||||
if (!this.isActive()) {
|
||||
uninitialize();
|
||||
return;
|
||||
}
|
||||
|
||||
// mark we are running now
|
||||
this.sourceRunning = true;
|
||||
try {
|
||||
// start the endpoint, connect to the cluster
|
||||
Service.State state = replicationEndpoint.start().get();
|
||||
|
@ -247,22 +260,14 @@ public class ReplicationSource extends Thread
|
|||
|
||||
int sleepMultiplier = 1;
|
||||
// delay this until we are in an asynchronous thread
|
||||
while (this.isActive() && this.peerClusterId == null) {
|
||||
while (this.isSourceActive() && this.peerClusterId == null) {
|
||||
this.peerClusterId = replicationEndpoint.getPeerUUID();
|
||||
if (this.isActive() && this.peerClusterId == null) {
|
||||
if (this.isSourceActive() && this.peerClusterId == null) {
|
||||
if (sleepForRetries("Cannot contact the peer's zk ensemble", sleepMultiplier)) {
|
||||
sleepMultiplier++;
|
||||
}
|
||||
}
|
||||
}
|
||||
// We were stopped while looping to contact peer's zk ensemble, just abort
|
||||
if (!this.isActive()) {
|
||||
uninitialize();
|
||||
return;
|
||||
}
|
||||
|
||||
// resetting to 1 to reuse later
|
||||
sleepMultiplier = 1;
|
||||
|
||||
// In rare case, zookeeper setting may be messed up. That leads to the incorrect
|
||||
// peerClusterId value, which is the same as the source clusterId
|
||||
|
@ -272,24 +277,229 @@ public class ReplicationSource extends Thread
|
|||
+ replicationEndpoint.getClass().getName(), null, false);
|
||||
}
|
||||
LOG.info("Replicating " + clusterId + " -> " + peerClusterId);
|
||||
// start workers
|
||||
for (Map.Entry<String, PriorityBlockingQueue<Path>> entry : queues.entrySet()) {
|
||||
String walGroupId = entry.getKey();
|
||||
PriorityBlockingQueue<Path> queue = entry.getValue();
|
||||
final ReplicationSourceWorkerThread worker =
|
||||
new ReplicationSourceWorkerThread(walGroupId, queue, replicationQueueInfo, this);
|
||||
ReplicationSourceWorkerThread extant = workerThreads.putIfAbsent(walGroupId, worker);
|
||||
if (extant != null) {
|
||||
LOG.debug("Someone has beat us to start a worker thread for wal group " + walGroupId);
|
||||
} else {
|
||||
LOG.debug("Starting up worker for wal group " + walGroupId);
|
||||
worker.startup();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Do the sleeping logic
|
||||
* @param msg Why we sleep
|
||||
* @param sleepMultiplier by how many times the default sleeping time is augmented
|
||||
* @return True if <code>sleepMultiplier</code> is < <code>maxRetriesMultiplier</code>
|
||||
*/
|
||||
protected boolean sleepForRetries(String msg, int sleepMultiplier) {
|
||||
try {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(msg + ", sleeping " + sleepForRetries + " times " + sleepMultiplier);
|
||||
}
|
||||
Thread.sleep(this.sleepForRetries * sleepMultiplier);
|
||||
} catch (InterruptedException e) {
|
||||
LOG.debug("Interrupted while sleeping between retries");
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
return sleepMultiplier < maxRetriesMultiplier;
|
||||
}
|
||||
|
||||
/**
|
||||
* check whether the peer is enabled or not
|
||||
*
|
||||
* @return true if the peer is enabled, otherwise false
|
||||
*/
|
||||
protected boolean isPeerEnabled() {
|
||||
return this.replicationPeers.getStatusOfPeer(this.peerId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void startup() {
|
||||
String n = Thread.currentThread().getName();
|
||||
Thread.UncaughtExceptionHandler handler = new Thread.UncaughtExceptionHandler() {
|
||||
@Override
|
||||
public void uncaughtException(final Thread t, final Throwable e) {
|
||||
LOG.error("Unexpected exception in ReplicationSource", e);
|
||||
}
|
||||
};
|
||||
Threads
|
||||
.setDaemonThreadRunning(this, n + ".replicationSource," + this.peerClusterZnode, handler);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void terminate(String reason) {
|
||||
terminate(reason, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void terminate(String reason, Exception cause) {
|
||||
terminate(reason, cause, true);
|
||||
}
|
||||
|
||||
public void terminate(String reason, Exception cause, boolean join) {
|
||||
if (cause == null) {
|
||||
LOG.info("Closing source "
|
||||
+ this.peerClusterZnode + " because: " + reason);
|
||||
|
||||
} else {
|
||||
LOG.error("Closing source " + this.peerClusterZnode
|
||||
+ " because an error occurred: " + reason, cause);
|
||||
}
|
||||
this.sourceRunning = false;
|
||||
Collection<ReplicationSourceWorkerThread> workers = workerThreads.values();
|
||||
for (ReplicationSourceWorkerThread worker : workers) {
|
||||
worker.setWorkerRunning(false);
|
||||
worker.interrupt();
|
||||
}
|
||||
ListenableFuture<Service.State> future = null;
|
||||
if (this.replicationEndpoint != null) {
|
||||
future = this.replicationEndpoint.stop();
|
||||
}
|
||||
if (join) {
|
||||
for (ReplicationSourceWorkerThread worker : workers) {
|
||||
Threads.shutdown(worker, this.sleepForRetries);
|
||||
LOG.info("ReplicationSourceWorker " + worker.getName() + " terminated");
|
||||
}
|
||||
if (future != null) {
|
||||
try {
|
||||
future.get();
|
||||
} catch (Exception e) {
|
||||
LOG.warn("Got exception:" + e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPeerClusterZnode() {
|
||||
return this.peerClusterZnode;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPeerClusterId() {
|
||||
return this.peerId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Path getCurrentPath() {
|
||||
// only for testing
|
||||
for (ReplicationSourceWorkerThread worker : workerThreads.values()) {
|
||||
if (worker.getCurrentPath() != null) return worker.getCurrentPath();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
private boolean isSourceActive() {
|
||||
return !this.stopper.isStopped() && this.sourceRunning;
|
||||
}
|
||||
|
||||
/**
|
||||
* Comparator used to compare logs together based on their start time
|
||||
*/
|
||||
public static class LogsComparator implements Comparator<Path> {
|
||||
|
||||
@Override
|
||||
public int compare(Path o1, Path o2) {
|
||||
return Long.valueOf(getTS(o1)).compareTo(getTS(o2));
|
||||
}
|
||||
|
||||
/**
|
||||
* Split a path to get the start time
|
||||
* For example: 10.20.20.171%3A60020.1277499063250
|
||||
* @param p path to split
|
||||
* @return start time
|
||||
*/
|
||||
private long getTS(Path p) {
|
||||
String[] parts = p.getName().split("\\.");
|
||||
return Long.parseLong(parts[parts.length-1]);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getStats() {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
sb.append("Total replicated edits: ").append(totalReplicatedEdits)
|
||||
.append(", current progress: \n");
|
||||
for (Map.Entry<String, ReplicationSourceWorkerThread> entry : workerThreads.entrySet()) {
|
||||
String walGroupId = entry.getKey();
|
||||
ReplicationSourceWorkerThread worker = entry.getValue();
|
||||
long position = worker.getCurrentPosition();
|
||||
Path currentPath = worker.getCurrentPath();
|
||||
sb.append("walGroup [").append(walGroupId).append("]: ");
|
||||
if (currentPath != null) {
|
||||
sb.append("currently replicating from: ").append(currentPath).append(" at position: ")
|
||||
.append(position).append("\n");
|
||||
} else {
|
||||
sb.append("no replication ongoing, waiting for new log");
|
||||
}
|
||||
}
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Replication Source Metrics
|
||||
* @return sourceMetrics
|
||||
*/
|
||||
public MetricsSource getSourceMetrics() {
|
||||
return this.metrics;
|
||||
}
|
||||
|
||||
public class ReplicationSourceWorkerThread extends Thread {
|
||||
private ReplicationSource source;
|
||||
private String walGroupId;
|
||||
private PriorityBlockingQueue<Path> queue;
|
||||
private ReplicationQueueInfo replicationQueueInfo;
|
||||
// Our reader for the current log. open/close handled by repLogReader
|
||||
private WAL.Reader reader;
|
||||
// Last position in the log that we sent to ZooKeeper
|
||||
private long lastLoggedPosition = -1;
|
||||
// Path of the current log
|
||||
private volatile Path currentPath;
|
||||
// Handle on the log reader helper
|
||||
private ReplicationWALReaderManager repLogReader;
|
||||
// Current number of operations (Put/Delete) that we need to replicate
|
||||
private int currentNbOperations = 0;
|
||||
// Current size of data we need to replicate
|
||||
private int currentSize = 0;
|
||||
// Indicates whether this particular worker is running
|
||||
private boolean workerRunning = true;
|
||||
|
||||
public ReplicationSourceWorkerThread(String walGroupId, PriorityBlockingQueue<Path> queue,
|
||||
ReplicationQueueInfo replicationQueueInfo, ReplicationSource source) {
|
||||
this.walGroupId = walGroupId;
|
||||
this.queue = queue;
|
||||
this.replicationQueueInfo = replicationQueueInfo;
|
||||
this.repLogReader = new ReplicationWALReaderManager(fs, conf);
|
||||
this.source = source;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
// If this is recovered, the queue is already full and the first log
|
||||
// normally has a position (unless the RS failed between 2 logs)
|
||||
if (this.replicationQueueInfo.isQueueRecovered()) {
|
||||
try {
|
||||
this.repLogReader.setPosition(this.replicationQueues.getLogPosition(this.peerClusterZnode,
|
||||
this.repLogReader.setPosition(replicationQueues.getLogPosition(peerClusterZnode,
|
||||
this.queue.peek().getName()));
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Recovered queue started with log " + this.queue.peek() +
|
||||
" at position " + this.repLogReader.getPosition());
|
||||
LOG.trace("Recovered queue started with log " + this.queue.peek() + " at position "
|
||||
+ this.repLogReader.getPosition());
|
||||
}
|
||||
} catch (ReplicationException e) {
|
||||
this.terminate("Couldn't get the position of this recovered queue " +
|
||||
this.peerClusterZnode, e);
|
||||
terminate("Couldn't get the position of this recovered queue " + peerClusterZnode, e);
|
||||
}
|
||||
}
|
||||
// Loop until we close down
|
||||
while (isActive()) {
|
||||
while (isWorkerActive()) {
|
||||
int sleepMultiplier = 1;
|
||||
// Sleep until replication is enabled again
|
||||
if (!isPeerEnabled()) {
|
||||
if (sleepForRetries("Replication is disabled", sleepMultiplier)) {
|
||||
|
@ -347,25 +557,25 @@ public class ReplicationSource extends Thread
|
|||
continue;
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
LOG.warn(this.peerClusterZnode + " Got: ", ioe);
|
||||
LOG.warn(peerClusterZnode + " Got: ", ioe);
|
||||
gotIOE = true;
|
||||
if (ioe.getCause() instanceof EOFException) {
|
||||
|
||||
boolean considerDumping = false;
|
||||
if (this.replicationQueueInfo.isQueueRecovered()) {
|
||||
try {
|
||||
FileStatus stat = this.fs.getFileStatus(this.currentPath);
|
||||
FileStatus stat = fs.getFileStatus(this.currentPath);
|
||||
if (stat.getLen() == 0) {
|
||||
LOG.warn(this.peerClusterZnode + " Got EOF and the file was empty");
|
||||
LOG.warn(peerClusterZnode + " Got EOF and the file was empty");
|
||||
}
|
||||
considerDumping = true;
|
||||
} catch (IOException e) {
|
||||
LOG.warn(this.peerClusterZnode + " Got while getting file size: ", e);
|
||||
LOG.warn(peerClusterZnode + " Got while getting file size: ", e);
|
||||
}
|
||||
}
|
||||
|
||||
if (considerDumping &&
|
||||
sleepMultiplier == this.maxRetriesMultiplier &&
|
||||
sleepMultiplier == maxRetriesMultiplier &&
|
||||
processEndOfFile()) {
|
||||
continue;
|
||||
}
|
||||
|
@ -383,10 +593,10 @@ public class ReplicationSource extends Thread
|
|||
// If we didn't get anything to replicate, or if we hit a IOE,
|
||||
// wait a bit and retry.
|
||||
// But if we need to stop, don't bother sleeping
|
||||
if (this.isActive() && (gotIOE || entries.isEmpty())) {
|
||||
if (isWorkerActive() && (gotIOE || entries.isEmpty())) {
|
||||
if (this.lastLoggedPosition != this.repLogReader.getPosition()) {
|
||||
this.manager.logPositionAndCleanOldLogs(this.currentPath,
|
||||
this.peerClusterZnode, this.repLogReader.getPosition(),
|
||||
manager.logPositionAndCleanOldLogs(this.currentPath,
|
||||
peerClusterZnode, this.repLogReader.getPosition(),
|
||||
this.replicationQueueInfo.isQueueRecovered(), currentWALisBeingWrittenTo);
|
||||
this.lastLoggedPosition = this.repLogReader.getPosition();
|
||||
}
|
||||
|
@ -395,7 +605,7 @@ public class ReplicationSource extends Thread
|
|||
sleepMultiplier = 1;
|
||||
// if there was nothing to ship and it's not an error
|
||||
// set "ageOfLastShippedOp" to <now> to indicate that we're current
|
||||
this.metrics.setAgeOfLastShippedOp(System.currentTimeMillis());
|
||||
metrics.setAgeOfLastShippedOp(System.currentTimeMillis(), walGroupId);
|
||||
}
|
||||
if (sleepForRetries("Nothing to replicate", sleepMultiplier)) {
|
||||
sleepMultiplier++;
|
||||
|
@ -405,16 +615,32 @@ public class ReplicationSource extends Thread
|
|||
sleepMultiplier = 1;
|
||||
shipEdits(currentWALisBeingWrittenTo, entries);
|
||||
}
|
||||
uninitialize();
|
||||
if (replicationQueueInfo.isQueueRecovered()) {
|
||||
// use synchronize to make sure one last thread will clean the queue
|
||||
synchronized (workerThreads) {
|
||||
Threads.sleep(100);// wait a short while for other worker thread to fully exit
|
||||
boolean allOtherTaskDone = true;
|
||||
for (ReplicationSourceWorkerThread worker : workerThreads.values()) {
|
||||
if (!worker.equals(this) && worker.isAlive()) {
|
||||
allOtherTaskDone = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (allOtherTaskDone) {
|
||||
manager.closeRecoveredQueue(this.source);
|
||||
LOG.info("Finished recovering queue " + peerClusterZnode
|
||||
+ " with the following stats: " + getStats());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Read all the entries from the current log files and retain those
|
||||
* that need to be replicated. Else, process the end of the current file.
|
||||
* Read all the entries from the current log files and retain those that need to be replicated.
|
||||
* Else, process the end of the current file.
|
||||
* @param currentWALisBeingWrittenTo is the current WAL being written to
|
||||
* @param entries resulting entries to be replicated
|
||||
* @return true if we got nothing and went to the next file, false if we got
|
||||
* entries
|
||||
* @return true if we got nothing and went to the next file, false if we got entries
|
||||
* @throws IOException
|
||||
*/
|
||||
protected boolean readAllEntriesToReplicateOrNextFile(boolean currentWALisBeingWrittenTo,
|
||||
|
@ -426,10 +652,9 @@ public class ReplicationSource extends Thread
|
|||
}
|
||||
this.repLogReader.seek();
|
||||
long positionBeforeRead = this.repLogReader.getPosition();
|
||||
WAL.Entry entry =
|
||||
this.repLogReader.readNextAndSetPosition();
|
||||
WAL.Entry entry = this.repLogReader.readNextAndSetPosition();
|
||||
while (entry != null) {
|
||||
this.metrics.incrLogEditsRead();
|
||||
metrics.incrLogEditsRead();
|
||||
seenEntries++;
|
||||
|
||||
// don't replicate if the log entries have already been consumed by the cluster
|
||||
|
@ -451,12 +676,13 @@ public class ReplicationSource extends Thread
|
|||
entries.add(entry);
|
||||
currentSize += entry.getEdit().heapSize();
|
||||
} else {
|
||||
this.metrics.incrLogEditsFiltered();
|
||||
metrics.incrLogEditsFiltered();
|
||||
}
|
||||
}
|
||||
// Stop if too many entries or too big
|
||||
if (currentSize >= this.replicationQueueSizeCapacity ||
|
||||
entries.size() >= this.replicationQueueNbCapacity) {
|
||||
// FIXME check the relationship between single wal group and overall
|
||||
if (currentSize >= replicationQueueSizeCapacity
|
||||
|| entries.size() >= replicationQueueNbCapacity) {
|
||||
break;
|
||||
}
|
||||
try {
|
||||
|
@ -482,11 +708,12 @@ public class ReplicationSource extends Thread
|
|||
protected boolean getNextPath() {
|
||||
try {
|
||||
if (this.currentPath == null) {
|
||||
this.currentPath = queue.poll(this.sleepForRetries, TimeUnit.MILLISECONDS);
|
||||
this.metrics.setSizeOfLogQueue(queue.size());
|
||||
this.currentPath = queue.poll(sleepForRetries, TimeUnit.MILLISECONDS);
|
||||
int queueSize = logQueueSize.decrementAndGet();
|
||||
metrics.setSizeOfLogQueue(queueSize);
|
||||
if (this.currentPath != null) {
|
||||
this.manager.cleanOldLogs(this.currentPath.getName(),
|
||||
this.peerId,
|
||||
// For recovered queue: must use peerClusterZnode since peerId is a parsed value
|
||||
manager.cleanOldLogs(this.currentPath.getName(), peerClusterZnode,
|
||||
this.replicationQueueInfo.isQueueRecovered());
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("New log: " + this.currentPath);
|
||||
|
@ -506,6 +733,7 @@ public class ReplicationSource extends Thread
|
|||
* @return true if we should continue with that file, false if we are over with it
|
||||
*/
|
||||
protected boolean openReader(int sleepMultiplier) {
|
||||
|
||||
try {
|
||||
try {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
|
@ -519,7 +747,7 @@ public class ReplicationSource extends Thread
|
|||
// to look at)
|
||||
List<String> deadRegionServers = this.replicationQueueInfo.getDeadRegionServers();
|
||||
LOG.info("NB dead servers : " + deadRegionServers.size());
|
||||
final Path rootDir = FSUtils.getRootDir(this.conf);
|
||||
final Path rootDir = FSUtils.getRootDir(conf);
|
||||
for (String curDeadServerName : deadRegionServers) {
|
||||
final Path deadRsDirectory = new Path(rootDir,
|
||||
DefaultWALProvider.getWALDirectoryName(curDeadServerName));
|
||||
|
@ -530,7 +758,7 @@ public class ReplicationSource extends Thread
|
|||
};
|
||||
for (Path possibleLogLocation : locs) {
|
||||
LOG.info("Possible location " + possibleLogLocation.toUri().toString());
|
||||
if (this.manager.getFs().exists(possibleLogLocation)) {
|
||||
if (manager.getFs().exists(possibleLogLocation)) {
|
||||
// We found the right new location
|
||||
LOG.info("Log " + this.currentPath + " still exists at " +
|
||||
possibleLogLocation);
|
||||
|
@ -575,7 +803,7 @@ public class ReplicationSource extends Thread
|
|||
// If the log was archived, continue reading from there
|
||||
Path archivedLogLocation =
|
||||
new Path(manager.getOldLogDir(), currentPath.getName());
|
||||
if (this.manager.getFs().exists(archivedLogLocation)) {
|
||||
if (manager.getFs().exists(archivedLogLocation)) {
|
||||
currentPath = archivedLogLocation;
|
||||
LOG.info("Log " + this.currentPath + " was moved to " +
|
||||
archivedLogLocation);
|
||||
|
@ -588,14 +816,14 @@ public class ReplicationSource extends Thread
|
|||
}
|
||||
} catch (IOException ioe) {
|
||||
if (ioe instanceof EOFException && isCurrentLogEmpty()) return true;
|
||||
LOG.warn(this.peerClusterZnode + " Got: ", ioe);
|
||||
LOG.warn(peerClusterZnode + " Got: ", ioe);
|
||||
this.reader = null;
|
||||
if (ioe.getCause() instanceof NullPointerException) {
|
||||
// Workaround for race condition in HDFS-4380
|
||||
// which throws a NPE if we open a file before any data node has the most recent block
|
||||
// Just sleep and retry. Will require re-reading compressed WALs for compressionContext.
|
||||
LOG.warn("Got NPE opening reader, will retry.");
|
||||
} else if (sleepMultiplier == this.maxRetriesMultiplier) {
|
||||
} else if (sleepMultiplier == maxRetriesMultiplier) {
|
||||
// TODO Need a better way to determine if a file is really gone but
|
||||
// TODO without scanning all logs dir
|
||||
LOG.warn("Waited too long for this file, considering dumping");
|
||||
|
@ -617,27 +845,8 @@ public class ReplicationSource extends Thread
|
|||
}
|
||||
|
||||
/**
|
||||
* Do the sleeping logic
|
||||
* @param msg Why we sleep
|
||||
* @param sleepMultiplier by how many times the default sleeping time is augmented
|
||||
* @return True if <code>sleepMultiplier</code> is < <code>maxRetriesMultiplier</code>
|
||||
*/
|
||||
protected boolean sleepForRetries(String msg, int sleepMultiplier) {
|
||||
try {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(msg + ", sleeping " + sleepForRetries + " times " + sleepMultiplier);
|
||||
}
|
||||
Thread.sleep(this.sleepForRetries * sleepMultiplier);
|
||||
} catch (InterruptedException e) {
|
||||
LOG.debug("Interrupted while sleeping between retries");
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
return sleepMultiplier < maxRetriesMultiplier;
|
||||
}
|
||||
|
||||
/**
|
||||
* Count the number of different row keys in the given edit because of
|
||||
* mini-batching. We assume that there's at least one Cell in the WALEdit.
|
||||
* Count the number of different row keys in the given edit because of mini-batching. We assume
|
||||
* that there's at least one Cell in the WALEdit.
|
||||
* @param edit edit to count row keys from
|
||||
* @return number of different row keys
|
||||
*/
|
||||
|
@ -664,10 +873,10 @@ public class ReplicationSource extends Thread
|
|||
LOG.warn("Was given 0 edits to ship");
|
||||
return;
|
||||
}
|
||||
while (this.isActive()) {
|
||||
while (isWorkerActive()) {
|
||||
try {
|
||||
if (this.throttler.isEnabled()) {
|
||||
long sleepTicks = this.throttler.getNextSleepInterval(currentSize);
|
||||
if (throttler.isEnabled()) {
|
||||
long sleepTicks = throttler.getNextSleepInterval(currentSize);
|
||||
if (sleepTicks > 0) {
|
||||
try {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
|
@ -682,12 +891,15 @@ public class ReplicationSource extends Thread
|
|||
continue;
|
||||
}
|
||||
// reset throttler's cycle start tick when sleep for throttling occurs
|
||||
this.throttler.resetStartTick();
|
||||
throttler.resetStartTick();
|
||||
}
|
||||
}
|
||||
// create replicateContext here, so the entries can be GC'd upon return from this call stack
|
||||
ReplicationEndpoint.ReplicateContext replicateContext = new ReplicationEndpoint.ReplicateContext();
|
||||
// create replicateContext here, so the entries can be GC'd upon return from this call
|
||||
// stack
|
||||
ReplicationEndpoint.ReplicateContext replicateContext =
|
||||
new ReplicationEndpoint.ReplicateContext();
|
||||
replicateContext.setEntries(entries).setSize(currentSize);
|
||||
replicateContext.setWalGroupId(walGroupId);
|
||||
|
||||
long startTimeNs = System.nanoTime();
|
||||
// send the edits to the endpoint. Will block until the edits are shipped and acknowledged
|
||||
|
@ -701,27 +913,29 @@ public class ReplicationSource extends Thread
|
|||
}
|
||||
|
||||
if (this.lastLoggedPosition != this.repLogReader.getPosition()) {
|
||||
this.manager.logPositionAndCleanOldLogs(this.currentPath,
|
||||
this.peerClusterZnode, this.repLogReader.getPosition(),
|
||||
this.replicationQueueInfo.isQueueRecovered(), currentWALisBeingWrittenTo);
|
||||
manager.logPositionAndCleanOldLogs(this.currentPath, peerClusterZnode,
|
||||
this.repLogReader.getPosition(), this.replicationQueueInfo.isQueueRecovered(),
|
||||
currentWALisBeingWrittenTo);
|
||||
this.lastLoggedPosition = this.repLogReader.getPosition();
|
||||
}
|
||||
if (this.throttler.isEnabled()) {
|
||||
this.throttler.addPushSize(currentSize);
|
||||
if (throttler.isEnabled()) {
|
||||
throttler.addPushSize(currentSize);
|
||||
}
|
||||
this.totalReplicatedEdits += entries.size();
|
||||
this.totalReplicatedOperations += currentNbOperations;
|
||||
this.metrics.shipBatch(this.currentNbOperations, this.currentSize/1024);
|
||||
this.metrics.setAgeOfLastShippedOp(entries.get(entries.size()-1).getKey().getWriteTime());
|
||||
totalReplicatedEdits.addAndGet(entries.size());
|
||||
totalReplicatedOperations.addAndGet(currentNbOperations);
|
||||
// FIXME check relationship between wal group and overall
|
||||
metrics.shipBatch(currentNbOperations, currentSize / 1024);
|
||||
metrics.setAgeOfLastShippedOp(entries.get(entries.size() - 1).getKey().getWriteTime(),
|
||||
walGroupId);
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Replicated " + this.totalReplicatedEdits + " entries in total, or "
|
||||
+ this.totalReplicatedOperations + " operations in " +
|
||||
((endTimeNs - startTimeNs)/1000000) + " ms");
|
||||
LOG.trace("Replicated " + totalReplicatedEdits + " entries in total, or "
|
||||
+ totalReplicatedOperations + " operations in "
|
||||
+ ((endTimeNs - startTimeNs) / 1000000) + " ms");
|
||||
}
|
||||
break;
|
||||
} catch (Exception ex) {
|
||||
LOG.warn(replicationEndpoint.getClass().getName() + " threw unknown exception:" +
|
||||
org.apache.hadoop.util.StringUtils.stringifyException(ex));
|
||||
LOG.warn(replicationEndpoint.getClass().getName() + " threw unknown exception:"
|
||||
+ org.apache.hadoop.util.StringUtils.stringifyException(ex));
|
||||
if (sleepForRetries("ReplicationEndpoint threw exception", sleepMultiplier)) {
|
||||
sleepMultiplier++;
|
||||
}
|
||||
|
@ -730,20 +944,10 @@ public class ReplicationSource extends Thread
|
|||
}
|
||||
|
||||
/**
|
||||
* check whether the peer is enabled or not
|
||||
*
|
||||
* @return true if the peer is enabled, otherwise false
|
||||
*/
|
||||
protected boolean isPeerEnabled() {
|
||||
return this.replicationPeers.getStatusOfPeer(this.peerId);
|
||||
}
|
||||
|
||||
/**
|
||||
* If the queue isn't empty, switch to the next one
|
||||
* Else if this is a recovered queue, it means we're done!
|
||||
* Else we'll just continue to try reading the log file
|
||||
* @return true if we're done with the current file, false if we should
|
||||
* continue trying to read from it
|
||||
* If the queue isn't empty, switch to the next one Else if this is a recovered queue, it means
|
||||
* we're done! Else we'll just continue to try reading the log file
|
||||
* @return true if we're done with the current file, false if we should continue trying to read
|
||||
* from it
|
||||
*/
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "DE_MIGHT_IGNORE",
|
||||
justification = "Yeah, this is how it works")
|
||||
|
@ -752,132 +956,67 @@ public class ReplicationSource extends Thread
|
|||
if (LOG.isTraceEnabled()) {
|
||||
String filesize = "N/A";
|
||||
try {
|
||||
FileStatus stat = this.fs.getFileStatus(this.currentPath);
|
||||
FileStatus stat = fs.getFileStatus(this.currentPath);
|
||||
filesize = stat.getLen() + "";
|
||||
} catch (IOException ex) {}
|
||||
LOG.trace("Reached the end of a log, stats: " + getStats() +
|
||||
", and the length of the file is " + filesize);
|
||||
} catch (IOException ex) {
|
||||
}
|
||||
LOG.trace("Reached the end of log " + this.currentPath + ", stats: " + getStats()
|
||||
+ ", and the length of the file is " + filesize);
|
||||
}
|
||||
this.currentPath = null;
|
||||
this.repLogReader.finishCurrentFile();
|
||||
this.reader = null;
|
||||
return true;
|
||||
} else if (this.replicationQueueInfo.isQueueRecovered()) {
|
||||
this.manager.closeRecoveredQueue(this);
|
||||
LOG.info("Finished recovering the queue with the following stats " + getStats());
|
||||
this.running = false;
|
||||
LOG.debug("Finished recovering queue for group " + walGroupId + " of peer "
|
||||
+ peerClusterZnode);
|
||||
workerRunning = false;
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void startup() {
|
||||
String n = Thread.currentThread().getName();
|
||||
Thread.UncaughtExceptionHandler handler =
|
||||
new Thread.UncaughtExceptionHandler() {
|
||||
Thread.UncaughtExceptionHandler handler = new Thread.UncaughtExceptionHandler() {
|
||||
@Override
|
||||
public void uncaughtException(final Thread t, final Throwable e) {
|
||||
LOG.error("Unexpected exception in ReplicationSource," +
|
||||
" currentPath=" + currentPath, e);
|
||||
LOG.error("Unexpected exception in ReplicationSourceWorkerThread," + " currentPath="
|
||||
+ getCurrentPath(), e);
|
||||
}
|
||||
};
|
||||
Threads.setDaemonThreadRunning(
|
||||
this, n + ".replicationSource," +
|
||||
this.peerClusterZnode, handler);
|
||||
Threads.setDaemonThreadRunning(this, n + ".replicationSource." + walGroupId + ","
|
||||
+ peerClusterZnode, handler);
|
||||
workerThreads.put(walGroupId, this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void terminate(String reason) {
|
||||
terminate(reason, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void terminate(String reason, Exception cause) {
|
||||
terminate(reason, cause, true);
|
||||
}
|
||||
|
||||
public void terminate(String reason, Exception cause, boolean join) {
|
||||
if (cause == null) {
|
||||
LOG.info("Closing source "
|
||||
+ this.peerClusterZnode + " because: " + reason);
|
||||
|
||||
} else {
|
||||
LOG.error("Closing source " + this.peerClusterZnode
|
||||
+ " because an error occurred: " + reason, cause);
|
||||
}
|
||||
this.running = false;
|
||||
this.interrupt();
|
||||
ListenableFuture<Service.State> future = null;
|
||||
if (this.replicationEndpoint != null) {
|
||||
future = this.replicationEndpoint.stop();
|
||||
}
|
||||
if (join) {
|
||||
Threads.shutdown(this, this.sleepForRetries);
|
||||
if (future != null) {
|
||||
try {
|
||||
future.get();
|
||||
} catch (Exception e) {
|
||||
LOG.warn("Got exception:" + e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPeerClusterZnode() {
|
||||
return this.peerClusterZnode;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPeerClusterId() {
|
||||
return this.peerId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Path getCurrentPath() {
|
||||
return this.currentPath;
|
||||
}
|
||||
|
||||
private boolean isActive() {
|
||||
return !this.stopper.isStopped() && this.running && !isInterrupted();
|
||||
public long getCurrentPosition() {
|
||||
return this.repLogReader.getPosition();
|
||||
}
|
||||
|
||||
/**
|
||||
* Comparator used to compare logs together based on their start time
|
||||
*/
|
||||
public static class LogsComparator implements Comparator<Path> {
|
||||
|
||||
@Override
|
||||
public int compare(Path o1, Path o2) {
|
||||
return Long.valueOf(getTS(o1)).compareTo(getTS(o2));
|
||||
private boolean isWorkerActive() {
|
||||
return !stopper.isStopped() && workerRunning && !isInterrupted();
|
||||
}
|
||||
|
||||
/**
|
||||
* Split a path to get the start time
|
||||
* For example: 10.20.20.171%3A60020.1277499063250
|
||||
* @param p path to split
|
||||
* @return start time
|
||||
*/
|
||||
private long getTS(Path p) {
|
||||
String[] parts = p.getName().split("\\.");
|
||||
return Long.parseLong(parts[parts.length-1]);
|
||||
private void terminate(String reason, Exception cause) {
|
||||
if (cause == null) {
|
||||
LOG.info("Closing worker for wal group " + this.walGroupId + " because: " + reason);
|
||||
|
||||
} else {
|
||||
LOG.error("Closing worker for wal group " + this.walGroupId
|
||||
+ " because an error occurred: " + reason, cause);
|
||||
}
|
||||
this.interrupt();
|
||||
Threads.shutdown(this, sleepForRetries);
|
||||
LOG.info("ReplicationSourceWorker " + this.getName() + " terminated");
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getStats() {
|
||||
long position = this.repLogReader.getPosition();
|
||||
return "Total replicated edits: " + totalReplicatedEdits +
|
||||
", currently replicating from: " + this.currentPath +
|
||||
" at position: " + position;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Replication Source Metrics
|
||||
* @return sourceMetrics
|
||||
*/
|
||||
public MetricsSource getSourceMetrics() {
|
||||
return this.metrics;
|
||||
public void setWorkerRunning(boolean workerRunning) {
|
||||
this.workerRunning = workerRunning;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
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(src.getPeerClusterZnode(), name);
|
||||
this.replicationQueues.addLog(id, name);
|
||||
} catch (ReplicationException e) {
|
||||
String message =
|
||||
"Cannot add log to queue when creating a new source, queueId="
|
||||
+ src.getPeerClusterZnode() + ", filename=" + name;
|
||||
"Cannot add log to queue when creating a new source, queueId=" + id
|
||||
+ ", filename=" + name;
|
||||
server.stop(message);
|
||||
throw e;
|
||||
}
|
||||
src.enqueueLog(this.latestPath);
|
||||
src.enqueueLog(logPath);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
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) {
|
||||
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;
|
||||
}
|
||||
}
|
||||
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(source.getPeerClusterZnode(), name);
|
||||
this.replicationQueues.addLog(id, logName);
|
||||
} catch (ReplicationException e) {
|
||||
throw new IOException("Cannot add log to replication queue with id="
|
||||
+ source.getPeerClusterZnode() + ", filename=" + name, e);
|
||||
throw new IOException("Cannot add log to replication queue"
|
||||
+ " when creating a new source, queueId=" + id + ", filename=" + logName, e);
|
||||
}
|
||||
}
|
||||
for (SortedSet<String> wals : this.walsById.values()) {
|
||||
}
|
||||
// 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();
|
||||
}
|
||||
wals.add(name);
|
||||
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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
this.latestPath = newLog;
|
||||
}
|
||||
|
||||
void postLogRoll(Path newLog) throws IOException {
|
||||
|
@ -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,20 +545,24 @@ public class ReplicationSourceManager implements ReplicationListener {
|
|||
LOG.info("Number of deleted recovered sources for " + id + ": "
|
||||
+ oldSourcesToDelete.size());
|
||||
// Now look for the one on this cluster
|
||||
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 = src;
|
||||
break;
|
||||
srcToRemove.add(src);
|
||||
}
|
||||
}
|
||||
if (srcToRemove == null) {
|
||||
if (srcToRemove.size() == 0) {
|
||||
LOG.error("The queue we wanted to close is missing " + id);
|
||||
return;
|
||||
}
|
||||
srcToRemove.terminate(terminateMessage);
|
||||
this.sources.remove(srcToRemove);
|
||||
for (ReplicationSourceInterface toRemove : srcToRemove) {
|
||||
toRemove.terminate(terminateMessage);
|
||||
this.sources.remove(toRemove);
|
||||
}
|
||||
deleteSource(id, true);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void regionServerRemoved(String regionserver) {
|
||||
|
@ -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);
|
||||
|
|
|
@ -366,4 +366,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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -84,7 +84,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";
|
||||
|
|
|
@ -73,15 +73,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,33 @@
|
|||
/**
|
||||
* 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.wal.WALFactory;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
@Category(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,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.TestReplicationKillMasterRSCompressed;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.wal.WALFactory;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
@Category(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,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.TestReplicationBase;
|
||||
import org.apache.hadoop.hbase.replication.TestReplicationSyncUpTool;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.wal.WALFactory;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
@Category(LargeTests.class)
|
||||
public class TestReplicationSyncUpToolWithMultipleWAL extends TestReplicationSyncUpTool {
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
conf1.set(WALFactory.WAL_PROVIDER, "multiwal");
|
||||
TestReplicationBase.setUpBeforeClass();
|
||||
}
|
||||
}
|
|
@ -260,7 +260,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;
|
||||
|
@ -228,7 +229,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();
|
||||
|
||||
|
@ -297,8 +302,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);
|
||||
}
|
||||
|
@ -316,10 +324,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