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

This commit is contained in:
tedyu 2015-09-11 09:26:27 -07:00
parent a8730c2839
commit c94d10952f
16 changed files with 1029 additions and 612 deletions

View File

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

View File

@ -191,4 +191,18 @@ public class LogRoller extends HasThread {
requester);
}
}
/**
* For testing only
* @return true if all WAL roll finished
*/
@VisibleForTesting
public boolean walRollFinished() {
for (boolean needRoll : walNeedsRoll.values()) {
if (needRoll) {
return false;
}
}
return true;
}
}

View File

@ -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;
}
}
/**

View File

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

View File

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

View File

@ -23,9 +23,12 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.SortedMap;
import java.util.SortedSet;
import java.util.TreeSet;
@ -56,6 +59,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeers;
import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
import org.apache.hadoop.hbase.replication.ReplicationQueues;
import org.apache.hadoop.hbase.replication.ReplicationTracker;
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
@ -91,13 +95,14 @@ public class ReplicationSourceManager implements ReplicationListener {
// All about stopping
private final Server server;
// All logs we are currently tracking
private final Map<String, SortedSet<String>> walsById;
// Index structure of the map is: peer_id->logPrefix/logGroup->logs
private final Map<String, Map<String, SortedSet<String>>> walsById;
// Logs for recovered sources we are currently tracking
private final Map<String, SortedSet<String>> walsByIdRecoveredQueues;
private final Map<String, Map<String, SortedSet<String>>> walsByIdRecoveredQueues;
private final Configuration conf;
private final FileSystem fs;
// The path to the latest log we saw, for new coming sources
private Path latestPath;
// The paths to the latest log of each wal group, for new coming peers
private Set<Path> latestPaths;
// Path to the wals directories
private final Path logDir;
// Path to the wal archive
@ -133,8 +138,8 @@ public class ReplicationSourceManager implements ReplicationListener {
this.replicationPeers = replicationPeers;
this.replicationTracker = replicationTracker;
this.server = server;
this.walsById = new HashMap<String, SortedSet<String>>();
this.walsByIdRecoveredQueues = new ConcurrentHashMap<String, SortedSet<String>>();
this.walsById = new HashMap<String, Map<String, SortedSet<String>>>();
this.walsByIdRecoveredQueues = new ConcurrentHashMap<String, Map<String, SortedSet<String>>>();
this.oldsources = new CopyOnWriteArrayList<ReplicationSourceInterface>();
this.conf = conf;
this.fs = fs;
@ -158,6 +163,7 @@ public class ReplicationSourceManager implements ReplicationListener {
tfb.setDaemon(true);
this.executor.setThreadFactory(tfb.build());
this.rand = new Random();
this.latestPaths = Collections.synchronizedSet(new HashSet<Path>());
}
/**
@ -189,15 +195,16 @@ public class ReplicationSourceManager implements ReplicationListener {
* @param queueRecovered Whether this is a recovered queue
*/
public void cleanOldLogs(String key, String id, boolean queueRecovered) {
String logPrefix = DefaultWALProvider.getWALPrefixFromWALName(key);
if (queueRecovered) {
SortedSet<String> wals = walsByIdRecoveredQueues.get(id);
SortedSet<String> wals = walsByIdRecoveredQueues.get(id).get(logPrefix);
if (wals != null && !wals.first().equals(key)) {
cleanOldLogs(wals, key, id);
}
} else {
synchronized (this.walsById) {
SortedSet<String> wals = walsById.get(id);
if (!wals.first().equals(key)) {
SortedSet<String> wals = walsById.get(id).get(logPrefix);
if (wals != null && !wals.first().equals(key)) {
cleanOldLogs(wals, key, id);
}
}
@ -238,36 +245,44 @@ public class ReplicationSourceManager implements ReplicationListener {
}
/**
* Add a new normal source to this region server
* Add sources for the given peer cluster on this region server. For the newly added peer, we only
* need to enqueue the latest log of each wal group and do replication
* @param id the id of the peer cluster
* @return the source that was created
* @throws IOException
*/
protected ReplicationSourceInterface addSource(String id) throws IOException,
ReplicationException {
ReplicationPeerConfig peerConfig
= replicationPeers.getReplicationPeerConfig(id);
ReplicationPeerConfig peerConfig = replicationPeers.getReplicationPeerConfig(id);
ReplicationPeer peer = replicationPeers.getPeer(id);
ReplicationSourceInterface src =
getReplicationSource(this.conf, this.fs, this, this.replicationQueues,
this.replicationPeers, server, id, this.clusterId, peerConfig, peer);
synchronized (this.walsById) {
this.sources.add(src);
this.walsById.put(id, new TreeSet<String>());
Map<String, SortedSet<String>> walsByGroup = new HashMap<String, SortedSet<String>>();
this.walsById.put(id, walsByGroup);
// Add the latest wal to that source's queue
if (this.latestPath != null) {
String name = this.latestPath.getName();
this.walsById.get(id).add(name);
try {
this.replicationQueues.addLog(src.getPeerClusterZnode(), name);
} catch (ReplicationException e) {
String message =
"Cannot add log to queue when creating a new source, queueId="
+ src.getPeerClusterZnode() + ", filename=" + name;
server.stop(message);
throw e;
synchronized (latestPaths) {
if (this.latestPaths.size() > 0) {
for (Path logPath : latestPaths) {
String name = logPath.getName();
String walPrefix = DefaultWALProvider.getWALPrefixFromWALName(name);
SortedSet<String> logs = new TreeSet<String>();
logs.add(name);
walsByGroup.put(walPrefix, logs);
try {
this.replicationQueues.addLog(id, name);
} catch (ReplicationException e) {
String message =
"Cannot add log to queue when creating a new source, queueId=" + id
+ ", filename=" + name;
server.stop(message);
throw e;
}
src.enqueueLog(logPath);
}
}
src.enqueueLog(this.latestPath);
}
}
src.startup();
@ -302,7 +317,7 @@ public class ReplicationSourceManager implements ReplicationListener {
* Get a copy of the wals of the first source on this rs
* @return a sorted set of wal names
*/
protected Map<String, SortedSet<String>> getWALs() {
protected Map<String, Map<String, SortedSet<String>>> getWALs() {
return Collections.unmodifiableMap(walsById);
}
@ -310,7 +325,7 @@ public class ReplicationSourceManager implements ReplicationListener {
* Get a copy of the wals of the recovered sources on this rs
* @return a sorted set of wal names
*/
protected Map<String, SortedSet<String>> getWalsByIdRecoveredQueues() {
protected Map<String, Map<String, SortedSet<String>>> getWalsByIdRecoveredQueues() {
return Collections.unmodifiableMap(walsByIdRecoveredQueues);
}
@ -331,27 +346,70 @@ public class ReplicationSourceManager implements ReplicationListener {
}
void preLogRoll(Path newLog) throws IOException {
synchronized (this.walsById) {
String name = newLog.getName();
for (ReplicationSourceInterface source : this.sources) {
try {
this.replicationQueues.addLog(source.getPeerClusterZnode(), name);
} catch (ReplicationException e) {
throw new IOException("Cannot add log to replication queue with id="
+ source.getPeerClusterZnode() + ", filename=" + name, e);
recordLog(newLog);
String logName = newLog.getName();
String logPrefix = DefaultWALProvider.getWALPrefixFromWALName(logName);
synchronized (latestPaths) {
Iterator<Path> iterator = latestPaths.iterator();
while (iterator.hasNext()) {
Path path = iterator.next();
if (path.getName().contains(logPrefix)) {
iterator.remove();
break;
}
}
for (SortedSet<String> wals : this.walsById.values()) {
if (this.sources.isEmpty()) {
// If there's no slaves, don't need to keep the old wals since
// we only consider the last one when a new slave comes in
wals.clear();
this.latestPaths.add(newLog);
}
}
/**
* Check and enqueue the given log to the correct source. If there's still no source for the
* group to which the given log belongs, create one
* @param logPath the log path to check and enqueue
* @throws IOException
*/
private void recordLog(Path logPath) throws IOException {
String logName = logPath.getName();
String logPrefix = DefaultWALProvider.getWALPrefixFromWALName(logName);
// update replication queues on ZK
synchronized (replicationPeers) {// synchronize on replicationPeers to avoid adding source for
// the to-be-removed peer
for (String id : replicationPeers.getPeerIds()) {
try {
this.replicationQueues.addLog(id, logName);
} catch (ReplicationException e) {
throw new IOException("Cannot add log to replication queue"
+ " when creating a new source, queueId=" + id + ", filename=" + logName, e);
}
}
}
// update walsById map
synchronized (walsById) {
for (Map.Entry<String, Map<String, SortedSet<String>>> entry : this.walsById.entrySet()) {
String peerId = entry.getKey();
Map<String, SortedSet<String>> walsByPrefix = entry.getValue();
boolean existingPrefix = false;
for (Map.Entry<String, SortedSet<String>> walsEntry : walsByPrefix.entrySet()) {
SortedSet<String> wals = walsEntry.getValue();
if (this.sources.isEmpty()) {
// If there's no slaves, don't need to keep the old wals since
// we only consider the last one when a new slave comes in
wals.clear();
}
if (logPrefix.equals(walsEntry.getKey())) {
wals.add(logName);
existingPrefix = true;
}
}
if (!existingPrefix) {
// The new log belongs to a new group, add it into this peer
LOG.debug("Start tracking logs for wal group " + logPrefix + " for peer " + peerId);
SortedSet<String> wals = new TreeSet<String>();
wals.add(logName);
walsByPrefix.put(logPrefix, wals);
}
wals.add(name);
}
}
this.latestPath = newLog;
}
void postLogRoll(Path newLog) throws IOException {
@ -376,7 +434,7 @@ public class ReplicationSourceManager implements ReplicationListener {
final ReplicationQueues replicationQueues, final ReplicationPeers replicationPeers,
final Server server, final String peerId, final UUID clusterId,
final ReplicationPeerConfig peerConfig, final ReplicationPeer replicationPeer)
throws IOException {
throws IOException {
RegionServerCoprocessorHost rsServerHost = null;
TableDescriptors tableDescriptors = null;
if (server instanceof HRegionServer) {
@ -414,7 +472,8 @@ public class ReplicationSourceManager implements ReplicationListener {
}
}
} catch (Exception e) {
LOG.warn("Passed replication endpoint implementation throws errors", e);
LOG.warn("Passed replication endpoint implementation throws errors"
+ " while initializing ReplicationSource for peer: " + peerId, e);
throw new IOException(e);
}
@ -470,7 +529,7 @@ public class ReplicationSourceManager implements ReplicationListener {
+ sources.size() + " and another "
+ oldsources.size() + " that were recovered");
String terminateMessage = "Replication stream was removed by a user";
ReplicationSourceInterface srcToRemove = null;
List<ReplicationSourceInterface> srcToRemove = new ArrayList<ReplicationSourceInterface>();
List<ReplicationSourceInterface> oldSourcesToDelete =
new ArrayList<ReplicationSourceInterface>();
// First close all the recovered sources for this peer
@ -486,19 +545,23 @@ public class ReplicationSourceManager implements ReplicationListener {
LOG.info("Number of deleted recovered sources for " + id + ": "
+ oldSourcesToDelete.size());
// Now look for the one on this cluster
for (ReplicationSourceInterface src : this.sources) {
if (id.equals(src.getPeerClusterId())) {
srcToRemove = src;
break;
synchronized (this.replicationPeers) {// synchronize on replicationPeers to avoid adding source
// for the to-be-removed peer
for (ReplicationSourceInterface src : this.sources) {
if (id.equals(src.getPeerClusterId())) {
srcToRemove.add(src);
}
}
if (srcToRemove.size() == 0) {
LOG.error("The queue we wanted to close is missing " + id);
return;
}
for (ReplicationSourceInterface toRemove : srcToRemove) {
toRemove.terminate(terminateMessage);
this.sources.remove(toRemove);
}
deleteSource(id, true);
}
if (srcToRemove == null) {
LOG.error("The queue we wanted to close is missing " + id);
return;
}
srcToRemove.terminate(terminateMessage);
this.sources.remove(srcToRemove);
deleteSource(id, true);
}
@Override
@ -580,6 +643,7 @@ public class ReplicationSourceManager implements ReplicationListener {
for (Map.Entry<String, SortedSet<String>> entry : newQueues.entrySet()) {
String peerId = entry.getKey();
SortedSet<String> walsSet = entry.getValue();
try {
// there is not an actual peer defined corresponding to peerId for the failover.
ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(peerId);
@ -596,7 +660,20 @@ public class ReplicationSourceManager implements ReplicationListener {
LOG.warn("Skipping failover for peer:" + actualPeerId + " of node" + rsZnode);
continue;
}
// track sources in walsByIdRecoveredQueues
Map<String, SortedSet<String>> walsByGroup = new HashMap<String, SortedSet<String>>();
walsByIdRecoveredQueues.put(peerId, walsByGroup);
for (String wal : walsSet) {
String walPrefix = DefaultWALProvider.getWALPrefixFromWALName(wal);
SortedSet<String> wals = walsByGroup.get(walPrefix);
if (wals == null) {
wals = new TreeSet<String>();
walsByGroup.put(walPrefix, wals);
}
wals.add(wal);
}
// enqueue sources
ReplicationSourceInterface src =
getReplicationSource(conf, fs, ReplicationSourceManager.this, this.rq, this.rp,
server, peerId, this.clusterId, peerConfig, peer);
@ -605,12 +682,10 @@ public class ReplicationSourceManager implements ReplicationListener {
break;
}
oldsources.add(src);
SortedSet<String> walsSet = entry.getValue();
for (String wal : walsSet) {
src.enqueueLog(new Path(oldLogDir, wal));
}
src.startup();
walsByIdRecoveredQueues.put(peerId, walsSet);
} catch (IOException e) {
// TODO manage it
LOG.error("Failed creating a source", e);

View File

@ -363,4 +363,15 @@ public class DefaultWALProvider implements WALProvider {
}
}
/**
* Get prefix of the log from its name, assuming WAL name in format of
* log_prefix.filenumber.log_suffix @see {@link FSHLog#getCurrentFileName()}
* @param name Name of the WAL to parse
* @return prefix of the log
*/
public static String getWALPrefixFromWALName(String name) {
int endIndex = name.replaceAll(META_WAL_PROVIDER_ID, "").lastIndexOf(".");
return name.substring(0, endIndex);
}
}

View File

@ -83,7 +83,7 @@ public class WALFactory {
}
}
static final String WAL_PROVIDER = "hbase.wal.provider";
public static final String WAL_PROVIDER = "hbase.wal.provider";
static final String DEFAULT_WAL_PROVIDER = Providers.defaultProvider.name();
static final String META_WAL_PROVIDER = "hbase.wal.meta_provider";

View File

@ -74,15 +74,40 @@ public class TestReplicationEndpoint extends TestReplicationBase {
}
@Before
public void setup() throws FailedLogCloseException, IOException {
public void setup() throws Exception {
ReplicationEndpointForTest.contructedCount.set(0);
ReplicationEndpointForTest.startedCount.set(0);
ReplicationEndpointForTest.replicateCount.set(0);
ReplicationEndpointReturningFalse.replicated.set(false);
ReplicationEndpointForTest.lastEntries = null;
for (RegionServerThread rs : utility1.getMiniHBaseCluster().getRegionServerThreads()) {
final List<RegionServerThread> rsThreads =
utility1.getMiniHBaseCluster().getRegionServerThreads();
for (RegionServerThread rs : rsThreads) {
utility1.getHBaseAdmin().rollWALWriter(rs.getRegionServer().getServerName());
}
// Wait for all log roll to finish
utility1.waitFor(3000, new Waiter.ExplainingPredicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
for (RegionServerThread rs : rsThreads) {
if (!rs.getRegionServer().walRollRequestFinished()) {
return false;
}
}
return true;
}
@Override
public String explainFailure() throws Exception {
List<String> logRollInProgressRsList = new ArrayList<String>();
for (RegionServerThread rs : rsThreads) {
if (!rs.getRegionServer().walRollRequestFinished()) {
logRollInProgressRsList.add(rs.getRegionServer().toString());
}
}
return "Still waiting for log roll on regionservers: " + logRollInProgressRsList;
}
});
}
@Test (timeout=120000)

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.TestReplicationEndpoint;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.junit.BeforeClass;
import org.junit.experimental.categories.Category;
@Category({ ReplicationTests.class, MediumTests.class })
public class TestReplicationEndpointWithMultipleWAL extends TestReplicationEndpoint {
@BeforeClass
public static void setUpBeforeClass() throws Exception {
conf1.set(WALFactory.WAL_PROVIDER, "multiwal");
TestReplicationEndpoint.setUpBeforeClass();
}
}

View File

@ -0,0 +1,35 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.replication.multiwal;
import org.apache.hadoop.hbase.replication.TestReplicationKillMasterRSCompressed;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.junit.BeforeClass;
import org.junit.experimental.categories.Category;
@Category({ReplicationTests.class, LargeTests.class})
public class TestReplicationKillMasterRSCompressedWithMultipleWAL extends
TestReplicationKillMasterRSCompressed {
@BeforeClass
public static void setUpBeforeClass() throws Exception {
conf1.set(WALFactory.WAL_PROVIDER, "multiwal");
TestReplicationKillMasterRSCompressed.setUpBeforeClass();
}
}

View File

@ -0,0 +1,35 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.replication.multiwal;
import org.apache.hadoop.hbase.replication.TestReplicationBase;
import org.apache.hadoop.hbase.replication.TestReplicationSyncUpTool;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.junit.BeforeClass;
import org.junit.experimental.categories.Category;
@Category({ ReplicationTests.class, LargeTests.class })
public class TestReplicationSyncUpToolWithMultipleWAL extends TestReplicationSyncUpTool {
@BeforeClass
public static void setUpBeforeClass() throws Exception {
conf1.set(WALFactory.WAL_PROVIDER, "multiwal");
TestReplicationBase.setUpBeforeClass();
}
}

View File

@ -257,7 +257,9 @@ public class TestRegionReplicaReplicationEndpointNoMaster {
Assert.assertEquals(1000, entries.size());
// replay the edits to the secondary using replay callable
replicator.replicate(new ReplicateContext().setEntries(Lists.newArrayList(entries)));
final String fakeWalGroupId = "fakeWALGroup";
replicator.replicate(new ReplicateContext().setEntries(Lists.newArrayList(entries))
.setWalGroupId(fakeWalGroupId));
Region region = rs0.getFromOnlineRegions(hriSecondary.getEncodedName());
HTU.verifyNumericRows(region, f, 0, 1000);

View File

@ -25,6 +25,7 @@ import java.net.URLEncoder;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.SortedMap;
import java.util.SortedSet;
import java.util.TreeSet;
@ -229,7 +230,11 @@ public class TestReplicationSourceManager {
}
wal.sync();
assertEquals(6, manager.getWALs().get(slaveId).size());
int logNumber = 0;
for (Map.Entry<String, SortedSet<String>> entry : manager.getWALs().get(slaveId).entrySet()) {
logNumber += entry.getValue().size();
}
assertEquals(6, logNumber);
wal.rollWriter();
@ -298,8 +303,11 @@ public class TestReplicationSourceManager {
rq.init(server.getServerName().toString());
// populate some znodes in the peer znode
SortedSet<String> files = new TreeSet<String>();
files.add("log1");
files.add("log2");
String group = "testgroup";
String file1 = group + ".log1";
String file2 = group + ".log2";
files.add(file1);
files.add(file2);
for (String file : files) {
rq.addLog("1", file);
}
@ -317,10 +325,10 @@ public class TestReplicationSourceManager {
w1.join(5000);
assertEquals(1, manager.getWalsByIdRecoveredQueues().size());
String id = "1-" + server.getServerName().getServerName();
assertEquals(files, manager.getWalsByIdRecoveredQueues().get(id));
manager.cleanOldLogs("log2", id, true);
assertEquals(files, manager.getWalsByIdRecoveredQueues().get(id).get(group));
manager.cleanOldLogs(file2, id, true);
// log1 should be deleted
assertEquals(Sets.newHashSet("log2"), manager.getWalsByIdRecoveredQueues().get(id));
assertEquals(Sets.newHashSet(file2), manager.getWalsByIdRecoveredQueues().get(id).get(group));
}
@Test

View File

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