Skip to content

Commit 4b984e1

Browse files
JeongDaeKimJeongdae Kim
authored andcommitted
HBASE-23205 Correctly update the position of WALs currently being replicated
1 parent 1451063 commit 4b984e1

File tree

5 files changed

+500
-151
lines changed

5 files changed

+500
-151
lines changed

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

Lines changed: 40 additions & 24 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@
1818
*/
1919
package org.apache.hadoop.hbase.replication.regionserver;
2020

21+
import com.google.common.annotations.VisibleForTesting;
2122
import com.google.common.collect.Lists;
2223
import com.google.common.util.concurrent.ListenableFuture;
2324
import com.google.common.util.concurrent.Service;
@@ -439,14 +440,30 @@ public String getPeerClusterId() {
439440
}
440441

441442
@Override
443+
@VisibleForTesting
442444
public Path getCurrentPath() {
443-
// only for testing
444445
for (ReplicationSourceShipperThread worker : workerThreads.values()) {
445446
if (worker.getCurrentPath() != null) return worker.getCurrentPath();
446447
}
447448
return null;
448449
}
449450

451+
@VisibleForTesting
452+
public Path getLastLoggedPath() {
453+
for (ReplicationSourceShipperThread worker : workerThreads.values()) {
454+
return worker.getLastLoggedPath();
455+
}
456+
return null;
457+
}
458+
459+
@VisibleForTesting
460+
public long getLastLoggedPosition() {
461+
for (ReplicationSourceShipperThread worker : workerThreads.values()) {
462+
return worker.getLastLoggedPosition();
463+
}
464+
return 0;
465+
}
466+
450467
private boolean isSourceActive() {
451468
return !this.stopper.isStopped() && this.sourceRunning;
452469
}
@@ -481,8 +498,8 @@ public String getStats() {
481498
for (Map.Entry<String, ReplicationSourceShipperThread> entry : workerThreads.entrySet()) {
482499
String walGroupId = entry.getKey();
483500
ReplicationSourceShipperThread worker = entry.getValue();
484-
long position = worker.getCurrentPosition();
485-
Path currentPath = worker.getCurrentPath();
501+
long position = worker.getLastLoggedPosition();
502+
Path currentPath = worker.getLastLoggedPath();
486503
sb.append("walGroup [").append(walGroupId).append("]: ");
487504
if (currentPath != null) {
488505
sb.append("currently replicating from: ").append(currentPath).append(" at position: ")
@@ -517,7 +534,7 @@ public Map<String, ReplicationStatus> getWalGroupStatus() {
517534
int queueSize = queues.get(walGroupId).size();
518535
replicationDelay =
519536
ReplicationLoad.calculateReplicationDelay(ageOfLastShippedOp, lastTimeStamp, queueSize);
520-
Path currentPath = worker.getCurrentPath();
537+
Path currentPath = worker.getLastLoggedPath();
521538
fileSize = -1;
522539
if (currentPath != null) {
523540
try {
@@ -535,7 +552,7 @@ public Map<String, ReplicationStatus> getWalGroupStatus() {
535552
.withQueueSize(queueSize)
536553
.withWalGroup(walGroupId)
537554
.withCurrentPath(currentPath)
538-
.withCurrentPosition(worker.getCurrentPosition())
555+
.withCurrentPosition(worker.getLastLoggedPosition())
539556
.withFileSize(fileSize)
540557
.withAgeOfLastShippedOp(ageOfLastShippedOp)
541558
.withReplicationDelay(replicationDelay);
@@ -555,7 +572,7 @@ public class ReplicationSourceShipperThread extends Thread {
555572
// Last position in the log that we sent to ZooKeeper
556573
private long lastLoggedPosition = -1;
557574
// Path of the current log
558-
private volatile Path currentPath;
575+
private volatile Path lastLoggedPath;
559576
// Current state of the worker thread
560577
private WorkerState state;
561578
ReplicationSourceWALReaderThread entryReader;
@@ -600,21 +617,19 @@ public void run() {
600617
WALEntryBatch entryBatch = entryReader.take();
601618
shipEdits(entryBatch);
602619
releaseBufferQuota((int) entryBatch.getHeapSize());
603-
if (replicationQueueInfo.isQueueRecovered() && entryBatch.getWalEntries().isEmpty()
604-
&& entryBatch.getLastSeqIds().isEmpty()) {
605-
LOG.debug("Finished recovering queue for group " + walGroupId + " of peer "
606-
+ peerClusterZnode);
620+
if (!entryBatch.hasMoreEntries()) {
621+
LOG.debug("Finished recovering queue for group "
622+
+ walGroupId + " of peer " + peerClusterZnode);
607623
metrics.incrCompletedRecoveryQueue();
608624
setWorkerState(WorkerState.FINISHED);
609-
continue;
610625
}
611626
} catch (InterruptedException e) {
612627
LOG.trace("Interrupted while waiting for next replication entry batch", e);
613628
Thread.currentThread().interrupt();
614629
}
615630
}
616631

617-
if (replicationQueueInfo.isQueueRecovered() && getWorkerState() == WorkerState.FINISHED) {
632+
if (getWorkerState() == WorkerState.FINISHED) {
618633
// use synchronize to make sure one last thread will clean the queue
619634
synchronized (this) {
620635
Threads.sleep(100);// wait a short while for other worker thread to fully exit
@@ -694,15 +709,13 @@ private int getBatchEntrySizeExcludeBulkLoad(WALEntryBatch entryBatch) {
694709
protected void shipEdits(WALEntryBatch entryBatch) {
695710
List<Entry> entries = entryBatch.getWalEntries();
696711
long lastReadPosition = entryBatch.getLastWalPosition();
697-
currentPath = entryBatch.getLastWalPath();
712+
lastLoggedPath = entryBatch.getLastWalPath();
698713
int sleepMultiplier = 0;
699714
if (entries.isEmpty()) {
700-
if (lastLoggedPosition != lastReadPosition) {
701-
updateLogPosition(lastReadPosition);
702-
// if there was nothing to ship and it's not an error
703-
// set "ageOfLastShippedOp" to <now> to indicate that we're current
704-
metrics.setAgeOfLastShippedOp(EnvironmentEdgeManager.currentTime(), walGroupId);
705-
}
715+
updateLogPosition(lastReadPosition);
716+
// if there was nothing to ship and it's not an error
717+
// set "ageOfLastShippedOp" to <now> to indicate that we're current
718+
metrics.setAgeOfLastShippedOp(EnvironmentEdgeManager.currentTime(), walGroupId);
706719
return;
707720
}
708721
int currentSize = (int) entryBatch.getHeapSize();
@@ -787,8 +800,7 @@ protected void shipEdits(WALEntryBatch entryBatch) {
787800
}
788801

789802
private void updateLogPosition(long lastReadPosition) {
790-
manager.setPendingShipment(false);
791-
manager.logPositionAndCleanOldLogs(currentPath, peerClusterZnode, lastReadPosition,
803+
manager.logPositionAndCleanOldLogs(lastLoggedPath, peerClusterZnode, lastReadPosition,
792804
this.replicationQueueInfo.isQueueRecovered(), false);
793805
lastLoggedPosition = lastReadPosition;
794806
}
@@ -800,7 +812,7 @@ public void startup() {
800812
public void uncaughtException(final Thread t, final Throwable e) {
801813
RSRpcServices.exitIfOOME(e);
802814
LOG.error("Unexpected exception in ReplicationSourceWorkerThread," + " currentPath="
803-
+ getCurrentPath(), e);
815+
+ getLastLoggedPath(), e);
804816
stopper.stop("Unexpected exception in ReplicationSourceWorkerThread");
805817
}
806818
};
@@ -941,8 +953,12 @@ public Path getCurrentPath() {
941953
return this.entryReader.getCurrentPath();
942954
}
943955

944-
public long getCurrentPosition() {
945-
return this.lastLoggedPosition;
956+
public Path getLastLoggedPath() {
957+
return lastLoggedPath;
958+
}
959+
960+
public long getLastLoggedPosition() {
961+
return lastLoggedPosition;
946962
}
947963

948964
private boolean isWorkerActive() {

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

Lines changed: 6 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -123,8 +123,6 @@ public class ReplicationSourceManager implements ReplicationListener {
123123

124124
private AtomicLong totalBufferUsed = new AtomicLong();
125125

126-
private boolean pendingShipment;
127-
128126
/**
129127
* Creates a replication manager and sets the watch on all the other registered region servers
130128
* @param replicationQueues the interface for manipulating replication queues
@@ -191,19 +189,13 @@ public ReplicationSourceManager(final ReplicationQueues replicationQueues,
191189
* @param holdLogInZK if true then the log is retained in ZK
192190
*/
193191
public synchronized void logPositionAndCleanOldLogs(Path log, String id, long position,
194-
boolean queueRecovered, boolean holdLogInZK) {
195-
if (!this.pendingShipment) {
196-
String fileName = log.getName();
197-
this.replicationQueues.setLogPosition(id, fileName, position);
198-
if (holdLogInZK) {
199-
return;
200-
}
201-
cleanOldLogs(fileName, id, queueRecovered);
192+
boolean queueRecovered, boolean holdLogInZK) {
193+
String fileName = log.getName();
194+
this.replicationQueues.setLogPosition(id, fileName, position);
195+
if (holdLogInZK) {
196+
return;
202197
}
203-
}
204-
205-
public synchronized void setPendingShipment(boolean pendingShipment) {
206-
this.pendingShipment = pendingShipment;
198+
cleanOldLogs(fileName, id, queueRecovered);
207199
}
208200

209201
/**

0 commit comments

Comments
 (0)