Skip to content

Commit 717e4c6

Browse files
committed
HBASE-24735: Refactor ReplicationSourceManager: move logPositionAndCleanOldLogs/cleanUpHFileRefs to ReplicationSource inside
1 parent 9c61d14 commit 717e4c6

12 files changed

Lines changed: 270 additions & 255 deletions

hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java

Lines changed: 11 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@
2121
import java.util.List;
2222
import java.util.UUID;
2323
import java.util.concurrent.PriorityBlockingQueue;
24+
2425
import org.apache.hadoop.conf.Configuration;
2526
import org.apache.hadoop.fs.FileStatus;
2627
import org.apache.hadoop.fs.FileSystem;
@@ -44,15 +45,18 @@ public class RecoveredReplicationSource extends ReplicationSource {
4445

4546
private static final Logger LOG = LoggerFactory.getLogger(RecoveredReplicationSource.class);
4647

48+
private Path walDir;
49+
4750
private String actualPeerId;
4851

4952
@Override
50-
public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
51-
ReplicationQueueStorage queueStorage, ReplicationPeer replicationPeer, Server server,
52-
String peerClusterZnode, UUID clusterId, WALFileLengthProvider walFileLengthProvider,
53-
MetricsSource metrics) throws IOException {
54-
super.init(conf, fs, manager, queueStorage, replicationPeer, server, peerClusterZnode,
53+
public void init(Configuration conf, FileSystem fs, Path walDir, ReplicationSourceManager manager,
54+
ReplicationQueueStorage queueStorage, ReplicationPeer replicationPeer, Server server,
55+
String peerClusterZnode, UUID clusterId, WALFileLengthProvider walFileLengthProvider,
56+
MetricsSource metrics) throws IOException {
57+
super.init(conf, fs, walDir, manager, queueStorage, replicationPeer, server, peerClusterZnode,
5558
clusterId, walFileLengthProvider, metrics);
59+
this.walDir = walDir;
5660
this.actualPeerId = this.replicationQueueInfo.getPeerId();
5761
}
5862

@@ -93,7 +97,7 @@ public void locateRecoveredPaths(PriorityBlockingQueue<Path> queue) throws IOExc
9397
deadRsDirectory.suffix(AbstractFSWALProvider.SPLITTING_EXT), path.getName()) };
9498
for (Path possibleLogLocation : locs) {
9599
LOG.info("Possible location " + possibleLogLocation.toUri().toString());
96-
if (manager.getFs().exists(possibleLogLocation)) {
100+
if (this.fs.exists(possibleLogLocation)) {
97101
// We found the right new location
98102
LOG.info("Log " + path + " still exists at " + possibleLogLocation);
99103
newPaths.add(possibleLogLocation);
@@ -126,7 +130,7 @@ public void locateRecoveredPaths(PriorityBlockingQueue<Path> queue) throws IOExc
126130
// N.B. the ReplicationSyncUp tool sets the manager.getWALDir to the root of the wal
127131
// area rather than to the wal area for a particular region server.
128132
private Path getReplSyncUpPath(Path path) throws IOException {
129-
FileStatus[] rss = fs.listStatus(manager.getLogDir());
133+
FileStatus[] rss = fs.listStatus(walDir);
130134
for (FileStatus rs : rss) {
131135
Path p = rs.getPath();
132136
FileStatus[] logs = fs.listStatus(p);

hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java

Lines changed: 147 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -28,14 +28,17 @@
2828
import java.util.HashMap;
2929
import java.util.List;
3030
import java.util.Map;
31+
import java.util.NavigableSet;
3132
import java.util.TreeMap;
33+
import java.util.TreeSet;
3234
import java.util.UUID;
3335
import java.util.concurrent.ConcurrentHashMap;
3436
import java.util.concurrent.PriorityBlockingQueue;
3537
import java.util.concurrent.TimeUnit;
3638
import java.util.concurrent.TimeoutException;
3739
import java.util.concurrent.atomic.AtomicLong;
3840
import java.util.function.Predicate;
41+
import java.util.stream.Collectors;
3942

4043
import org.apache.commons.lang3.StringUtils;
4144
import org.apache.hadoop.conf.Configuration;
@@ -52,16 +55,20 @@
5255
import org.apache.hadoop.hbase.replication.ChainWALEntryFilter;
5356
import org.apache.hadoop.hbase.replication.ClusterMarkingEntryFilter;
5457
import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
58+
import org.apache.hadoop.hbase.replication.ReplicationException;
5559
import org.apache.hadoop.hbase.replication.ReplicationPeer;
5660
import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
5761
import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
62+
import org.apache.hadoop.hbase.replication.ReplicationUtils;
5863
import org.apache.hadoop.hbase.replication.SystemTableWALEntryFilter;
5964
import org.apache.hadoop.hbase.replication.WALEntryFilter;
6065
import org.apache.hadoop.hbase.util.Threads;
6166
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
67+
import org.apache.hadoop.hbase.wal.SyncReplicationWALProvider;
6268
import org.apache.hadoop.hbase.wal.WAL.Entry;
6369
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
6470
import org.apache.yetus.audience.InterfaceAudience;
71+
import org.apache.zookeeper.KeeperException;
6572
import org.slf4j.Logger;
6673
import org.slf4j.LoggerFactory;
6774
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
@@ -130,8 +137,6 @@ public class ReplicationSource implements ReplicationSourceInterface {
130137
protected final ConcurrentHashMap<String, ReplicationSourceShipper> workerThreads =
131138
new ConcurrentHashMap<>();
132139

133-
private AtomicLong totalBufferUsed;
134-
135140
public static final String WAIT_ON_ENDPOINT_SECONDS =
136141
"hbase.replication.wait.on.endpoint.seconds";
137142
public static final int DEFAULT_WAIT_ON_ENDPOINT_SECONDS = 30;
@@ -183,7 +188,7 @@ public class ReplicationSource implements ReplicationSourceInterface {
183188
* @param metrics metrics for replication source
184189
*/
185190
@Override
186-
public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
191+
public void init(Configuration conf, FileSystem fs, Path walDir, ReplicationSourceManager manager,
187192
ReplicationQueueStorage queueStorage, ReplicationPeer replicationPeer, Server server,
188193
String queueId, UUID clusterId, WALFileLengthProvider walFileLengthProvider,
189194
MetricsSource metrics) throws IOException {
@@ -211,7 +216,6 @@ public void init(Configuration conf, FileSystem fs, ReplicationSourceManager man
211216
defaultBandwidth = this.conf.getLong("replication.source.per.peer.node.bandwidth", 0);
212217
currentBandwidth = getCurrentBandwidth();
213218
this.throttler = new ReplicationThrottler((double) currentBandwidth / 10.0);
214-
this.totalBufferUsed = manager.getTotalBufferUsed();
215219
this.walFileLengthProvider = walFileLengthProvider;
216220
LOG.info("queueId={}, ReplicationSource: {}, currentBandwidth={}", queueId,
217221
replicationPeer.getId(), this.currentBandwidth);
@@ -398,10 +402,11 @@ protected ReplicationSourceShipper createNewShipper(String walGroupId,
398402
}
399403

400404
private ReplicationSourceWALReader createNewWALReader(String walGroupId,
401-
PriorityBlockingQueue<Path> queue, long startPosition) {
402-
return replicationPeer.getPeerConfig().isSerial()
403-
? new SerialReplicationSourceWALReader(fs, conf, queue, startPosition, walEntryFilter, this)
404-
: new ReplicationSourceWALReader(fs, conf, queue, startPosition, walEntryFilter, this);
405+
PriorityBlockingQueue<Path> queue, long startPosition) {
406+
return replicationPeer.getPeerConfig().isSerial() ?
407+
new SerialReplicationSourceWALReader(fs, conf, queue, startPosition, walEntryFilter, this,
408+
manager) :
409+
new ReplicationSourceWALReader(fs, conf, queue, startPosition, walEntryFilter, this, manager);
405410
}
406411

407412
/**
@@ -426,11 +431,6 @@ public ReplicationEndpoint getReplicationEndpoint() {
426431
return this.replicationEndpoint;
427432
}
428433

429-
@Override
430-
public ReplicationSourceManager getSourceManager() {
431-
return this.manager;
432-
}
433-
434434
@Override
435435
public void tryThrottle(int batchSize) throws InterruptedException {
436436
checkBandwidthChangeAndResetThrottler();
@@ -735,7 +735,7 @@ public void postShipEdits(List<Entry> entries, int batchSize) {
735735
throttler.addPushSize(batchSize);
736736
}
737737
totalReplicatedEdits.addAndGet(entries.size());
738-
long newBufferUsed = totalBufferUsed.addAndGet(-batchSize);
738+
long newBufferUsed = manager.getTotalBufferUsed().addAndGet(-batchSize);
739739
// Record the new buffer usage
740740
this.manager.getGlobalMetrics().setWALReaderEditsBufferBytes(newBufferUsed);
741741
}
@@ -770,4 +770,137 @@ void removeWorker(ReplicationSourceShipper worker) {
770770
private String logPeerId(){
771771
return "[Source for peer " + this.getPeer().getId() + "]:";
772772
}
773+
774+
@VisibleForTesting
775+
public void setWALPosition(WALEntryBatch entryBatch) {
776+
String fileName = entryBatch.getLastWalPath().getName();
777+
interruptOrAbortWhenFail(() -> this.queueStorage
778+
.setWALPosition(server.getServerName(), getQueueId(), fileName,
779+
entryBatch.getLastWalPosition(), entryBatch.getLastSeqIds()));
780+
}
781+
782+
@VisibleForTesting
783+
public void cleanOldWALs(String log, boolean inclusive) {
784+
NavigableSet<String> walsToRemove = getWalsToRemove(log, inclusive);
785+
if (walsToRemove.isEmpty()) {
786+
return;
787+
}
788+
// cleanOldWALs may spend some time, especially for sync replication where we may want to
789+
// remove remote wals as the remote cluster may have already been down, so we do it outside
790+
// the lock to avoid block preLogRoll
791+
cleanOldWALs(walsToRemove);
792+
}
793+
794+
private NavigableSet<String> getWalsToRemove(String log, boolean inclusive) {
795+
NavigableSet<String> walsToRemove = new TreeSet<>();
796+
String logPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(log);
797+
try {
798+
this.queueStorage.getWALsInQueue(this.server.getServerName(), getQueueId()).forEach(wal -> {
799+
LOG.debug("getWalsToRemove wal {}", wal);
800+
String walPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(wal);
801+
if (walPrefix.equals(logPrefix)) {
802+
walsToRemove.add(wal);
803+
}
804+
});
805+
} catch (ReplicationException e) {
806+
// Just log the exception here, as the recovered replication source will try to cleanup again.
807+
LOG.warn("Failed to read wals in queue {}", getQueueId(), e);
808+
}
809+
return walsToRemove.headSet(log, inclusive);
810+
}
811+
812+
private void removeRemoteWALs(String peerId, String remoteWALDir, Collection<String> wals)
813+
throws IOException {
814+
Path remoteWALDirForPeer = ReplicationUtils.getPeerRemoteWALDir(remoteWALDir, peerId);
815+
FileSystem fs = ReplicationUtils.getRemoteWALFileSystem(conf, remoteWALDir);
816+
for (String wal : wals) {
817+
Path walFile = new Path(remoteWALDirForPeer, wal);
818+
try {
819+
if (!fs.delete(walFile, false) && fs.exists(walFile)) {
820+
throw new IOException("Can not delete " + walFile);
821+
}
822+
} catch (FileNotFoundException e) {
823+
// Just ignore since this means the file has already been deleted.
824+
// The javadoc of the FileSystem.delete methods does not specify the behavior of deleting an
825+
// inexistent file, so here we deal with both, i.e, check the return value of the
826+
// FileSystem.delete, and also catch FNFE.
827+
LOG.debug("The remote wal {} has already been deleted?", walFile, e);
828+
}
829+
}
830+
}
831+
832+
private void cleanOldWALs(NavigableSet<String> wals) {
833+
LOG.debug("Removing {} logs in the list: {}", wals.size(), wals);
834+
// The intention here is that, we want to delete the remote wal files ASAP as it may effect the
835+
// failover time if you want to transit the remote cluster from S to A. And the infinite retry
836+
// is not a problem, as if we can not contact with the remote HDFS cluster, then usually we can
837+
// not contact with the HBase cluster either, so the replication will be blocked either.
838+
if (isSyncReplication()) {
839+
String peerId = getPeerId();
840+
String remoteWALDir = replicationPeer.getPeerConfig().getRemoteWALDir();
841+
// Filter out the wals need to be removed from the remote directory. Its name should be the
842+
// special format, and also, the peer id in its name should match the peer id for the
843+
// replication source.
844+
List<String> remoteWals = wals.stream().filter(w -> SyncReplicationWALProvider
845+
.getSyncReplicationPeerIdFromWALName(w).map(peerId::equals).orElse(false))
846+
.collect(Collectors.toList());
847+
LOG.debug("Removing {} logs from remote dir {} in the list: {}", remoteWals.size(),
848+
remoteWALDir, remoteWals);
849+
if (!remoteWals.isEmpty()) {
850+
for (int sleepMultiplier = 0;;) {
851+
try {
852+
removeRemoteWALs(peerId, remoteWALDir, remoteWals);
853+
break;
854+
} catch (IOException e) {
855+
LOG.warn("Failed to delete remote wals from remote dir {} for peer {}", remoteWALDir,
856+
peerId);
857+
}
858+
if (!isSourceActive()) {
859+
// skip the following operations
860+
return;
861+
}
862+
if (ReplicationUtils.sleepForRetries("Failed to delete remote wals", sleepForRetries,
863+
sleepMultiplier, maxRetriesMultiplier)) {
864+
sleepMultiplier++;
865+
}
866+
}
867+
}
868+
}
869+
for (String wal : wals) {
870+
interruptOrAbortWhenFail(
871+
() -> this.queueStorage.removeWAL(server.getServerName(), getQueueId(), wal));
872+
}
873+
}
874+
875+
public void cleanUpHFileRefs(List<String> files) {
876+
interruptOrAbortWhenFail(() -> this.queueStorage.removeHFileRefs(getPeerId(), files));
877+
}
878+
879+
@FunctionalInterface
880+
private interface ReplicationQueueOperation {
881+
void exec() throws ReplicationException;
882+
}
883+
884+
/**
885+
* Refresh replication source will terminate the old source first, then the source thread will be
886+
* interrupted. Need to handle it instead of abort the region server.
887+
*/
888+
private void interruptOrAbortWhenFail(ReplicationQueueOperation op) {
889+
try {
890+
op.exec();
891+
} catch (ReplicationException e) {
892+
if (e.getCause() != null && e.getCause() instanceof KeeperException.SystemErrorException
893+
&& e.getCause().getCause() != null && e.getCause()
894+
.getCause() instanceof InterruptedException) {
895+
// ReplicationRuntimeException(a RuntimeException) is thrown out here. The reason is
896+
// that thread is interrupted deep down in the stack, it should pass the following
897+
// processing logic and propagate to the most top layer which can handle this exception
898+
// properly. In this specific case, the top layer is ReplicationSourceShipper#run().
899+
throw new ReplicationRuntimeException(
900+
"Thread is interrupted, the replication source may be terminated",
901+
e.getCause().getCause());
902+
}
903+
server.abort("Failed to operate on replication queue", e);
904+
}
905+
}
773906
}

hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java

Lines changed: 22 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@
2323
import java.util.List;
2424
import java.util.Map;
2525
import java.util.UUID;
26+
import java.util.concurrent.atomic.AtomicLong;
2627

2728
import org.apache.hadoop.conf.Configuration;
2829
import org.apache.hadoop.fs.FileSystem;
@@ -43,15 +44,15 @@ public interface ReplicationSourceInterface {
4344

4445
/**
4546
* Initializer for the source
46-
* @param conf the configuration to use
47-
* @param fs the file system to use
48-
* @param manager the manager to use
47+
*
48+
* @param conf the configuration to use
49+
* @param fs the file system to use
4950
* @param server the server for this region server
5051
*/
51-
void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
52-
ReplicationQueueStorage queueStorage, ReplicationPeer replicationPeer, Server server,
53-
String queueId, UUID clusterId, WALFileLengthProvider walFileLengthProvider,
54-
MetricsSource metrics) throws IOException;
52+
void init(Configuration conf, FileSystem fs, Path walDir, ReplicationSourceManager manager,
53+
ReplicationQueueStorage queueStorage, ReplicationPeer replicationPeer, Server server,
54+
String queueId, UUID clusterId, WALFileLengthProvider walFileLengthProvider,
55+
MetricsSource metrics) throws IOException;
5556

5657
/**
5758
* Add a log to the list of logs to replicate
@@ -147,11 +148,6 @@ default boolean isSyncReplication() {
147148
*/
148149
ReplicationEndpoint getReplicationEndpoint();
149150

150-
/**
151-
* @return the replication source manager
152-
*/
153-
ReplicationSourceManager getSourceManager();
154-
155151
/**
156152
* @return the wal file length provider
157153
*/
@@ -192,4 +188,18 @@ default Map<String, ReplicationStatus> getWalGroupStatus() {
192188
default boolean isRecovered() {
193189
return false;
194190
}
191+
192+
/**
193+
* Set the current position of WAL to {@link ReplicationQueueStorage}
194+
* @param entryBatch a batch of WAL entries to replicate
195+
*/
196+
void setWALPosition(WALEntryBatch entryBatch);
197+
198+
/**
199+
* Cleans a WAL and all older WALs from replication queue. Called when we are sure that a WAL is
200+
* closed and has no more entries.
201+
* @param walName the name of WAL
202+
* @param inclusive whether we should also remove the given WAL
203+
*/
204+
void cleanOldWALs(String walName, boolean inclusive);
195205
}

0 commit comments

Comments
 (0)