HBASE-6617 ReplicationSourceManager should be able to track multiple WAL paths (Yu Li)

This commit is contained in:
tedyu 2015-09-11 09:30:58 -07:00
parent 84dbe39f5d
commit be96bb6adf
16 changed files with 1027 additions and 613 deletions

View File

@ -3313,4 +3313,13 @@ public class HRegionServer extends HasThread implements
} }
return max; return max;
} }
/**
* For testing
* @return whether all wal roll request finished for this regionserver
*/
@VisibleForTesting
public boolean walRollRequestFinished() {
return this.walRoller.walRollFinished();
}
} }

View File

@ -197,4 +197,18 @@ public class LogRoller extends HasThread {
requester); 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;
}
} }

View File

@ -137,6 +137,7 @@ public interface ReplicationEndpoint extends Service {
static class ReplicateContext { static class ReplicateContext {
List<Entry> entries; List<Entry> entries;
int size; int size;
String walGroupId;
@InterfaceAudience.Private @InterfaceAudience.Private
public ReplicateContext() { public ReplicateContext() {
} }
@ -149,12 +150,19 @@ public interface ReplicationEndpoint extends Service {
this.size = size; this.size = size;
return this; return this;
} }
public ReplicateContext setWalGroupId(String walGroupId) {
this.walGroupId = walGroupId;
return this;
}
public List<Entry> getEntries() { public List<Entry> getEntries() {
return entries; return entries;
} }
public int getSize() { public int getSize() {
return size; return size;
} }
public String getWalGroupId(){
return walGroupId;
}
} }
/** /**

View File

@ -154,6 +154,7 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
@Override @Override
public boolean replicate(ReplicateContext replicateContext) { public boolean replicate(ReplicateContext replicateContext) {
List<Entry> entries = replicateContext.getEntries(); List<Entry> entries = replicateContext.getEntries();
String walGroupId = replicateContext.getWalGroupId();
int sleepMultiplier = 1; int sleepMultiplier = 1;
if (!peersSelected && this.isRunning()) { if (!peersSelected && this.isRunning()) {
@ -219,12 +220,13 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
throw iox; throw iox;
} }
// update metrics // 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; return true;
} catch (IOException ioe) { } catch (IOException ioe) {
// Didn't ship anything, but must still age the last time we did // Didn't ship anything, but must still age the last time we did
this.metrics.refreshAgeOfLastShippedOp(); this.metrics.refreshAgeOfLastShippedOp(walGroupId);
if (ioe instanceof RemoteException) { if (ioe instanceof RemoteException) {
ioe = ((RemoteException) ioe).unwrapRemoteException(); ioe = ((RemoteException) ioe).unwrapRemoteException();
LOG.warn("Can't replicate because of an error on the remote cluster: ", ioe); LOG.warn("Can't replicate because of an error on the remote cluster: ", ioe);

View File

@ -18,6 +18,9 @@
package org.apache.hadoop.hbase.replication.regionserver; 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.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
@ -34,7 +37,8 @@ public class MetricsSource {
private static final Log LOG = LogFactory.getLog(MetricsSource.class); 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 int lastQueueSize = 0;
private String id; private String id;
@ -56,23 +60,29 @@ public class MetricsSource {
/** /**
* Set the age of the last edit that was shipped * Set the age of the last edit that was shipped
*
* @param timestamp write time of the edit * @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; long age = EnvironmentEdgeManager.currentTime() - timestamp;
singleSourceSource.setLastShippedAge(age); singleSourceSource.setLastShippedAge(age);
globalSourceSource.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 * 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. * when replication fails and need to keep that metric accurate.
* @param walGroupId id of the group to update
*/ */
public void refreshAgeOfLastShippedOp() { public void refreshAgeOfLastShippedOp(String walGroupId) {
if (this.lastTimestamp > 0) { Long lastTimestamp = this.lastTimeStamps.get(walGroupId);
setAgeOfLastShippedOp(this.lastTimestamp); 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() { public void clear() {
singleSourceSource.clear(); singleSourceSource.clear();
globalSourceSource.decrSizeOfLogQueue(lastQueueSize); globalSourceSource.decrSizeOfLogQueue(lastQueueSize);
lastTimeStamps.clear();
lastQueueSize = 0; 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 * @return lastTimestampForAge
*/ */
public long getTimeStampOfLastShippedOp() { public long getTimeStampOfLastShippedOp() {
long lastTimestamp = 0L;
for (long ts : lastTimeStamps.values()) {
if (ts > lastTimestamp) {
lastTimestamp = ts;
}
}
return lastTimestamp; return lastTimestamp;
} }

View File

@ -23,9 +23,12 @@ import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Random; import java.util.Random;
import java.util.Set;
import java.util.SortedMap; import java.util.SortedMap;
import java.util.SortedSet; import java.util.SortedSet;
import java.util.TreeSet; 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.ReplicationQueueInfo;
import org.apache.hadoop.hbase.replication.ReplicationQueues; import org.apache.hadoop.hbase.replication.ReplicationQueues;
import org.apache.hadoop.hbase.replication.ReplicationTracker; import org.apache.hadoop.hbase.replication.ReplicationTracker;
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.common.util.concurrent.ThreadFactoryBuilder;
@ -91,13 +95,14 @@ public class ReplicationSourceManager implements ReplicationListener {
// All about stopping // All about stopping
private final Server server; private final Server server;
// All logs we are currently tracking // 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 // 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 Configuration conf;
private final FileSystem fs; private final FileSystem fs;
// The path to the latest log we saw, for new coming sources // The paths to the latest log of each wal group, for new coming peers
private Path latestPath; private Set<Path> latestPaths;
// Path to the wals directories // Path to the wals directories
private final Path logDir; private final Path logDir;
// Path to the wal archive // Path to the wal archive
@ -133,8 +138,8 @@ public class ReplicationSourceManager implements ReplicationListener {
this.replicationPeers = replicationPeers; this.replicationPeers = replicationPeers;
this.replicationTracker = replicationTracker; this.replicationTracker = replicationTracker;
this.server = server; this.server = server;
this.walsById = new HashMap<String, SortedSet<String>>(); this.walsById = new HashMap<String, Map<String, SortedSet<String>>>();
this.walsByIdRecoveredQueues = new ConcurrentHashMap<String, SortedSet<String>>(); this.walsByIdRecoveredQueues = new ConcurrentHashMap<String, Map<String, SortedSet<String>>>();
this.oldsources = new CopyOnWriteArrayList<ReplicationSourceInterface>(); this.oldsources = new CopyOnWriteArrayList<ReplicationSourceInterface>();
this.conf = conf; this.conf = conf;
this.fs = fs; this.fs = fs;
@ -158,6 +163,7 @@ public class ReplicationSourceManager implements ReplicationListener {
tfb.setDaemon(true); tfb.setDaemon(true);
this.executor.setThreadFactory(tfb.build()); this.executor.setThreadFactory(tfb.build());
this.rand = new Random(); 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 * @param queueRecovered Whether this is a recovered queue
*/ */
public void cleanOldLogs(String key, String id, boolean queueRecovered) { public void cleanOldLogs(String key, String id, boolean queueRecovered) {
String logPrefix = DefaultWALProvider.getWALPrefixFromWALName(key);
if (queueRecovered) { if (queueRecovered) {
SortedSet<String> wals = walsByIdRecoveredQueues.get(id); SortedSet<String> wals = walsByIdRecoveredQueues.get(id).get(logPrefix);
if (wals != null && !wals.first().equals(key)) { if (wals != null && !wals.first().equals(key)) {
cleanOldLogs(wals, key, id); cleanOldLogs(wals, key, id);
} }
} else { } else {
synchronized (this.walsById) { synchronized (this.walsById) {
SortedSet<String> wals = walsById.get(id); SortedSet<String> wals = walsById.get(id).get(logPrefix);
if (!wals.first().equals(key)) { if (wals != null && !wals.first().equals(key)) {
cleanOldLogs(wals, key, id); 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 * @param id the id of the peer cluster
* @return the source that was created * @return the source that was created
* @throws IOException * @throws IOException
*/ */
protected ReplicationSourceInterface addSource(String id) throws IOException, protected ReplicationSourceInterface addSource(String id) throws IOException,
ReplicationException { ReplicationException {
ReplicationPeerConfig peerConfig ReplicationPeerConfig peerConfig = replicationPeers.getReplicationPeerConfig(id);
= replicationPeers.getReplicationPeerConfig(id);
ReplicationPeer peer = replicationPeers.getPeer(id); ReplicationPeer peer = replicationPeers.getPeer(id);
ReplicationSourceInterface src = ReplicationSourceInterface src =
getReplicationSource(this.conf, this.fs, this, this.replicationQueues, getReplicationSource(this.conf, this.fs, this, this.replicationQueues,
this.replicationPeers, server, id, this.clusterId, peerConfig, peer); this.replicationPeers, server, id, this.clusterId, peerConfig, peer);
synchronized (this.walsById) { synchronized (this.walsById) {
this.sources.add(src); 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 // Add the latest wal to that source's queue
if (this.latestPath != null) { synchronized (latestPaths) {
String name = this.latestPath.getName(); if (this.latestPaths.size() > 0) {
this.walsById.get(id).add(name); for (Path logPath : latestPaths) {
try { String name = logPath.getName();
this.replicationQueues.addLog(src.getPeerClusterZnode(), name); String walPrefix = DefaultWALProvider.getWALPrefixFromWALName(name);
} catch (ReplicationException e) { SortedSet<String> logs = new TreeSet<String>();
String message = logs.add(name);
"Cannot add log to queue when creating a new source, queueId=" walsByGroup.put(walPrefix, logs);
+ src.getPeerClusterZnode() + ", filename=" + name; try {
server.stop(message); this.replicationQueues.addLog(id, name);
throw e; } 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(); src.startup();
@ -302,7 +317,7 @@ public class ReplicationSourceManager implements ReplicationListener {
* Get a copy of the wals of the first source on this rs * Get a copy of the wals of the first source on this rs
* @return a sorted set of wal names * @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); 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 * Get a copy of the wals of the recovered sources on this rs
* @return a sorted set of wal names * @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); return Collections.unmodifiableMap(walsByIdRecoveredQueues);
} }
@ -331,27 +346,70 @@ public class ReplicationSourceManager implements ReplicationListener {
} }
void preLogRoll(Path newLog) throws IOException { void preLogRoll(Path newLog) throws IOException {
synchronized (this.walsById) { recordLog(newLog);
String name = newLog.getName(); String logName = newLog.getName();
for (ReplicationSourceInterface source : this.sources) { String logPrefix = DefaultWALProvider.getWALPrefixFromWALName(logName);
try { synchronized (latestPaths) {
this.replicationQueues.addLog(source.getPeerClusterZnode(), name); Iterator<Path> iterator = latestPaths.iterator();
} catch (ReplicationException e) { while (iterator.hasNext()) {
throw new IOException("Cannot add log to replication queue with id=" Path path = iterator.next();
+ source.getPeerClusterZnode() + ", filename=" + name, e); if (path.getName().contains(logPrefix)) {
iterator.remove();
break;
} }
} }
for (SortedSet<String> wals : this.walsById.values()) { this.latestPaths.add(newLog);
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(); /**
* 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 { void postLogRoll(Path newLog) throws IOException {
@ -376,7 +434,7 @@ public class ReplicationSourceManager implements ReplicationListener {
final ReplicationQueues replicationQueues, final ReplicationPeers replicationPeers, final ReplicationQueues replicationQueues, final ReplicationPeers replicationPeers,
final Server server, final String peerId, final UUID clusterId, final Server server, final String peerId, final UUID clusterId,
final ReplicationPeerConfig peerConfig, final ReplicationPeer replicationPeer) final ReplicationPeerConfig peerConfig, final ReplicationPeer replicationPeer)
throws IOException { throws IOException {
RegionServerCoprocessorHost rsServerHost = null; RegionServerCoprocessorHost rsServerHost = null;
TableDescriptors tableDescriptors = null; TableDescriptors tableDescriptors = null;
if (server instanceof HRegionServer) { if (server instanceof HRegionServer) {
@ -414,7 +472,8 @@ public class ReplicationSourceManager implements ReplicationListener {
} }
} }
} catch (Exception e) { } 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); throw new IOException(e);
} }
@ -470,7 +529,7 @@ public class ReplicationSourceManager implements ReplicationListener {
+ sources.size() + " and another " + sources.size() + " and another "
+ oldsources.size() + " that were recovered"); + oldsources.size() + " that were recovered");
String terminateMessage = "Replication stream was removed by a user"; String terminateMessage = "Replication stream was removed by a user";
ReplicationSourceInterface srcToRemove = null; List<ReplicationSourceInterface> srcToRemove = new ArrayList<ReplicationSourceInterface>();
List<ReplicationSourceInterface> oldSourcesToDelete = List<ReplicationSourceInterface> oldSourcesToDelete =
new ArrayList<ReplicationSourceInterface>(); new ArrayList<ReplicationSourceInterface>();
// First close all the recovered sources for this peer // 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 + ": " LOG.info("Number of deleted recovered sources for " + id + ": "
+ oldSourcesToDelete.size()); + oldSourcesToDelete.size());
// Now look for the one on this cluster // Now look for the one on this cluster
for (ReplicationSourceInterface src : this.sources) { synchronized (this.replicationPeers) {// synchronize on replicationPeers to avoid adding source
if (id.equals(src.getPeerClusterId())) { // for the to-be-removed peer
srcToRemove = src; for (ReplicationSourceInterface src : this.sources) {
break; 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 @Override
@ -580,6 +643,7 @@ public class ReplicationSourceManager implements ReplicationListener {
for (Map.Entry<String, SortedSet<String>> entry : newQueues.entrySet()) { for (Map.Entry<String, SortedSet<String>> entry : newQueues.entrySet()) {
String peerId = entry.getKey(); String peerId = entry.getKey();
SortedSet<String> walsSet = entry.getValue();
try { try {
// there is not an actual peer defined corresponding to peerId for the failover. // there is not an actual peer defined corresponding to peerId for the failover.
ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(peerId); ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(peerId);
@ -596,7 +660,20 @@ public class ReplicationSourceManager implements ReplicationListener {
LOG.warn("Skipping failover for peer:" + actualPeerId + " of node" + rsZnode); LOG.warn("Skipping failover for peer:" + actualPeerId + " of node" + rsZnode);
continue; 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 = ReplicationSourceInterface src =
getReplicationSource(conf, fs, ReplicationSourceManager.this, this.rq, this.rp, getReplicationSource(conf, fs, ReplicationSourceManager.this, this.rq, this.rp,
server, peerId, this.clusterId, peerConfig, peer); server, peerId, this.clusterId, peerConfig, peer);
@ -605,12 +682,10 @@ public class ReplicationSourceManager implements ReplicationListener {
break; break;
} }
oldsources.add(src); oldsources.add(src);
SortedSet<String> walsSet = entry.getValue();
for (String wal : walsSet) { for (String wal : walsSet) {
src.enqueueLog(new Path(oldLogDir, wal)); src.enqueueLog(new Path(oldLogDir, wal));
} }
src.startup(); src.startup();
walsByIdRecoveredQueues.put(peerId, walsSet);
} catch (IOException e) { } catch (IOException e) {
// TODO manage it // TODO manage it
LOG.error("Failed creating a source", e); LOG.error("Failed creating a source", e);

View File

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

View File

@ -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 DEFAULT_WAL_PROVIDER = Providers.defaultProvider.name();
static final String META_WAL_PROVIDER = "hbase.wal.meta_provider"; static final String META_WAL_PROVIDER = "hbase.wal.meta_provider";

View File

@ -73,15 +73,40 @@ public class TestReplicationEndpoint extends TestReplicationBase {
} }
@Before @Before
public void setup() throws FailedLogCloseException, IOException { public void setup() throws Exception {
ReplicationEndpointForTest.contructedCount.set(0); ReplicationEndpointForTest.contructedCount.set(0);
ReplicationEndpointForTest.startedCount.set(0); ReplicationEndpointForTest.startedCount.set(0);
ReplicationEndpointForTest.replicateCount.set(0); ReplicationEndpointForTest.replicateCount.set(0);
ReplicationEndpointReturningFalse.replicated.set(false); ReplicationEndpointReturningFalse.replicated.set(false);
ReplicationEndpointForTest.lastEntries = null; 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()); 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) @Test (timeout=120000)

View File

@ -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();
}
}

View File

@ -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();
}
}

View File

@ -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();
}
}

View File

@ -260,7 +260,9 @@ public class TestRegionReplicaReplicationEndpointNoMaster {
Assert.assertEquals(1000, entries.size()); Assert.assertEquals(1000, entries.size());
// replay the edits to the secondary using replay callable // 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()); Region region = rs0.getFromOnlineRegions(hriSecondary.getEncodedName());
HTU.verifyNumericRows(region, f, 0, 1000); HTU.verifyNumericRows(region, f, 0, 1000);

View File

@ -25,6 +25,7 @@ import java.net.URLEncoder;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.SortedMap; import java.util.SortedMap;
import java.util.SortedSet; import java.util.SortedSet;
import java.util.TreeSet; import java.util.TreeSet;
@ -228,7 +229,11 @@ public class TestReplicationSourceManager {
} }
wal.sync(); 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(); wal.rollWriter();
@ -297,8 +302,11 @@ public class TestReplicationSourceManager {
rq.init(server.getServerName().toString()); rq.init(server.getServerName().toString());
// populate some znodes in the peer znode // populate some znodes in the peer znode
SortedSet<String> files = new TreeSet<String>(); SortedSet<String> files = new TreeSet<String>();
files.add("log1"); String group = "testgroup";
files.add("log2"); String file1 = group + ".log1";
String file2 = group + ".log2";
files.add(file1);
files.add(file2);
for (String file : files) { for (String file : files) {
rq.addLog("1", file); rq.addLog("1", file);
} }
@ -316,10 +324,10 @@ public class TestReplicationSourceManager {
w1.join(5000); w1.join(5000);
assertEquals(1, manager.getWalsByIdRecoveredQueues().size()); assertEquals(1, manager.getWalsByIdRecoveredQueues().size());
String id = "1-" + server.getServerName().getServerName(); String id = "1-" + server.getServerName().getServerName();
assertEquals(files, manager.getWalsByIdRecoveredQueues().get(id)); assertEquals(files, manager.getWalsByIdRecoveredQueues().get(id).get(group));
manager.cleanOldLogs("log2", id, true); manager.cleanOldLogs(file2, id, true);
// log1 should be deleted // log1 should be deleted
assertEquals(Sets.newHashSet("log2"), manager.getWalsByIdRecoveredQueues().get(id)); assertEquals(Sets.newHashSet(file2), manager.getWalsByIdRecoveredQueues().get(id).get(group));
} }
@Test @Test

View File

@ -39,8 +39,10 @@ import org.junit.After;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Before; import org.junit.Before;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.junit.runner.RunWith; import org.junit.runner.RunWith;
import org.junit.runners.Parameterized; import org.junit.runners.Parameterized;
import org.junit.runners.Parameterized.Parameters; import org.junit.runners.Parameterized.Parameters;
@ -75,6 +77,7 @@ public class TestReplicationWALReaderManager {
private int nbRows; private int nbRows;
private int walEditKVs; private int walEditKVs;
private final AtomicLong sequenceId = new AtomicLong(1); private final AtomicLong sequenceId = new AtomicLong(1);
@Rule public TestName tn = new TestName();
@Parameters @Parameters
public static Collection<Object[]> parameters() { public static Collection<Object[]> parameters() {
@ -127,7 +130,7 @@ public class TestReplicationWALReaderManager {
List<WALActionsListener> listeners = new ArrayList<WALActionsListener>(); List<WALActionsListener> listeners = new ArrayList<WALActionsListener>();
pathWatcher = new PathWatcher(); pathWatcher = new PathWatcher();
listeners.add(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()); log = wals.getWAL(info.getEncodedNameAsBytes());
} }