Skip to content

Commit a370099

Browse files
authored
HBASE-27778 Incorrect ReplicationSourceWALReader.totalBufferUsed may … (#5158)
HBASE-27778 Incorrect ReplicationSourceWALReader.totalBufferUsed may cause replication hang up Signed-off-by: Duo Zhang <zhangduo@apache.org>
1 parent f27823e commit a370099

File tree

3 files changed

+130
-13
lines changed

3 files changed

+130
-13
lines changed

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

Lines changed: 34 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -140,11 +140,9 @@ protected final int sleep(int sleepMultiplier) {
140140
public void run() {
141141
int sleepMultiplier = 1;
142142
while (isReaderRunning()) { // we only loop back here if something fatal happened to our stream
143-
WALEntryBatch batch = null;
144143
try (WALEntryStream entryStream = new WALEntryStream(logQueue, fs, conf, currentPosition,
145144
source.getWALFileLengthProvider(), source.getSourceMetrics(), walGroupId)) {
146145
while (isReaderRunning()) { // loop here to keep reusing stream while we can
147-
batch = null;
148146
if (!source.isPeerEnabled()) {
149147
Threads.sleep(sleepForRetries);
150148
continue;
@@ -174,14 +172,25 @@ public void run() {
174172
continue;
175173
}
176174
// below are all for hasNext == YES
177-
batch = createBatch(entryStream);
178-
readWALEntries(entryStream, batch);
179-
currentPosition = entryStream.getPosition();
180-
// need to propagate the batch even it has no entries since it may carry the last
181-
// sequence id information for serial replication.
182-
LOG.debug("Read {} WAL entries eligible for replication", batch.getNbEntries());
183-
entryBatchQueue.put(batch);
184-
sleepMultiplier = 1;
175+
WALEntryBatch batch = createBatch(entryStream);
176+
boolean successAddToQueue = false;
177+
try {
178+
readWALEntries(entryStream, batch);
179+
currentPosition = entryStream.getPosition();
180+
// need to propagate the batch even it has no entries since it may carry the last
181+
// sequence id information for serial replication.
182+
LOG.debug("Read {} WAL entries eligible for replication", batch.getNbEntries());
183+
entryBatchQueue.put(batch);
184+
successAddToQueue = true;
185+
sleepMultiplier = 1;
186+
} finally {
187+
if (!successAddToQueue) {
188+
// batch is not put to ReplicationSourceWALReader#entryBatchQueue,so we should
189+
// decrease ReplicationSourceWALReader.totalBufferUsed by the byte size which
190+
// acquired in ReplicationSourceWALReader.acquireBufferQuota.
191+
this.releaseBufferQuota(batch);
192+
}
193+
}
185194
}
186195
} catch (WALEntryFilterRetryableException e) {
187196
// here we have to recreate the WALEntryStream, as when filtering, we have already called
@@ -212,7 +221,7 @@ protected final boolean addEntryToBatch(WALEntryBatch batch, Entry entry) {
212221
long entrySizeExcludeBulkLoad = getEntrySizeExcludeBulkLoad(entry);
213222
batch.addEntry(entry, entrySize);
214223
updateBatchStats(batch, entry, entrySize);
215-
boolean totalBufferTooLarge = acquireBufferQuota(entrySizeExcludeBulkLoad);
224+
boolean totalBufferTooLarge = acquireBufferQuota(batch, entrySizeExcludeBulkLoad);
216225

217226
// Stop if too many entries or too big
218227
return totalBufferTooLarge || batch.getHeapSize() >= replicationBatchSizeCapacity
@@ -430,13 +439,26 @@ private void updateReplicationMarkerEdit(Entry entry, long offset) {
430439
* @param size delta size for grown buffer
431440
* @return true if we should clear buffer and push all
432441
*/
433-
private boolean acquireBufferQuota(long size) {
442+
private boolean acquireBufferQuota(WALEntryBatch walEntryBatch, long size) {
434443
long newBufferUsed = totalBufferUsed.addAndGet(size);
435444
// Record the new buffer usage
436445
this.source.getSourceManager().getGlobalMetrics().setWALReaderEditsBufferBytes(newBufferUsed);
446+
walEntryBatch.incrementUsedBufferSize(size);
437447
return newBufferUsed >= totalBufferQuota;
438448
}
439449

450+
/**
451+
* To release the buffer quota of {@link WALEntryBatch} which acquired by
452+
* {@link ReplicationSourceWALReader#acquireBufferQuota}
453+
*/
454+
private void releaseBufferQuota(WALEntryBatch walEntryBatch) {
455+
long usedBufferSize = walEntryBatch.getUsedBufferSize();
456+
if (usedBufferSize > 0) {
457+
long newBufferUsed = totalBufferUsed.addAndGet(-usedBufferSize);
458+
this.source.getSourceManager().getGlobalMetrics().setWALReaderEditsBufferBytes(newBufferUsed);
459+
}
460+
}
461+
440462
/** Returns whether the reader thread is running */
441463
public boolean isReaderRunning() {
442464
return isReaderRunning && !isInterrupted();

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

Lines changed: 12 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -52,6 +52,9 @@ class WALEntryBatch {
5252
private Map<String, Long> lastSeqIds = new HashMap<>();
5353
// indicate that this is the end of the current file
5454
private boolean endOfFile;
55+
// indicate the buffer size used, which is added to
56+
// ReplicationSourceWALReader.totalBufferUsed
57+
private long usedBufferSize;
5558

5659
/**
5760
* @param lastWalPath Path of the WAL the last entry in this batch was read from
@@ -153,11 +156,19 @@ public void setLastSeqId(String region, long sequenceId) {
153156
lastSeqIds.put(region, sequenceId);
154157
}
155158

159+
public void incrementUsedBufferSize(long increment) {
160+
usedBufferSize += increment;
161+
}
162+
163+
public long getUsedBufferSize() {
164+
return this.usedBufferSize;
165+
}
166+
156167
@Override
157168
public String toString() {
158169
return "WALEntryBatch [walEntries=" + walEntriesWithSize + ", lastWalPath=" + lastWalPath
159170
+ ", lastWalPosition=" + lastWalPosition + ", nbRowKeys=" + nbRowKeys + ", nbHFiles="
160171
+ nbHFiles + ", heapSize=" + heapSize + ", lastSeqIds=" + lastSeqIds + ", endOfFile="
161-
+ endOfFile + "]";
172+
+ endOfFile + ",usedBufferSize=" + usedBufferSize + "]";
162173
}
163174
}

hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestBasicWALEntryStream.java

Lines changed: 84 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -308,6 +308,14 @@ private ReplicationSource mockReplicationSource(boolean recovered, Configuration
308308
when(source.isRecovered()).thenReturn(recovered);
309309
MetricsReplicationGlobalSourceSource globalMetrics =
310310
Mockito.mock(MetricsReplicationGlobalSourceSource.class);
311+
final AtomicLong bufferUsedCounter = new AtomicLong(0);
312+
Mockito.doAnswer((invocationOnMock) -> {
313+
bufferUsedCounter.set(invocationOnMock.getArgument(0, Long.class));
314+
return null;
315+
}).when(globalMetrics).setWALReaderEditsBufferBytes(Mockito.anyLong());
316+
when(globalMetrics.getWALReaderEditsBufferBytes())
317+
.then(invocationOnMock -> bufferUsedCounter.get());
318+
311319
when(mockSourceManager.getGlobalMetrics()).thenReturn(globalMetrics);
312320
return source;
313321
}
@@ -791,4 +799,80 @@ public void testEOFExceptionInOldWALsDirectory() throws Exception {
791799
Waiter.waitFor(localConf, 10000,
792800
(Waiter.Predicate<Exception>) () -> logQueue.getQueueSize(fakeWalGroupId) == 1);
793801
}
802+
803+
/**
804+
* This test is for HBASE-27778, when {@link WALEntryFilter#filter} throws exception for some
805+
* entries in {@link WALEntryBatch},{@link ReplicationSourceWALReader#totalBufferUsed} should be
806+
* decreased because {@link WALEntryBatch} is not put to
807+
* {@link ReplicationSourceWALReader#entryBatchQueue}.
808+
*/
809+
@Test
810+
public void testReplicationSourceWALReaderWithPartialWALEntryFailingFilter() throws Exception {
811+
appendEntriesToLogAndSync(3);
812+
// get ending position
813+
long position;
814+
try (WALEntryStream entryStream =
815+
new WALEntryStream(logQueue, fs, CONF, 0, log, new MetricsSource("1"), fakeWalGroupId)) {
816+
for (int i = 0; i < 3; i++) {
817+
assertNotNull(next(entryStream));
818+
}
819+
position = entryStream.getPosition();
820+
}
821+
822+
Path walPath = getQueue().peek();
823+
int maxThrowExceptionCount = 3;
824+
825+
ReplicationSource source = mockReplicationSource(false, CONF);
826+
when(source.isPeerEnabled()).thenReturn(true);
827+
PartialWALEntryFailingWALEntryFilter walEntryFilter =
828+
new PartialWALEntryFailingWALEntryFilter(maxThrowExceptionCount, 3);
829+
ReplicationSourceWALReader reader =
830+
new ReplicationSourceWALReader(fs, CONF, logQueue, 0, walEntryFilter, source, fakeWalGroupId);
831+
reader.start();
832+
WALEntryBatch entryBatch = reader.take();
833+
834+
assertNotNull(entryBatch);
835+
assertEquals(3, entryBatch.getWalEntries().size());
836+
long sum = entryBatch.getWalEntries().stream()
837+
.mapToLong(ReplicationSourceWALReader::getEntrySizeExcludeBulkLoad).sum();
838+
assertEquals(position, entryBatch.getLastWalPosition());
839+
assertEquals(walPath, entryBatch.getLastWalPath());
840+
assertEquals(3, entryBatch.getNbRowKeys());
841+
assertEquals(sum, source.getSourceManager().getTotalBufferUsed().get());
842+
assertEquals(sum, source.getSourceManager().getGlobalMetrics().getWALReaderEditsBufferBytes());
843+
assertEquals(maxThrowExceptionCount, walEntryFilter.getThrowExceptionCount());
844+
assertNull(reader.poll(10));
845+
}
846+
847+
private static class PartialWALEntryFailingWALEntryFilter implements WALEntryFilter {
848+
private int filteredWALEntryCount = -1;
849+
private int walEntryCount = 0;
850+
private int throwExceptionCount = -1;
851+
private int maxThrowExceptionCount;
852+
853+
public PartialWALEntryFailingWALEntryFilter(int throwExceptionLimit, int walEntryCount) {
854+
this.maxThrowExceptionCount = throwExceptionLimit;
855+
this.walEntryCount = walEntryCount;
856+
}
857+
858+
@Override
859+
public Entry filter(Entry entry) {
860+
filteredWALEntryCount++;
861+
if (filteredWALEntryCount < walEntryCount - 1) {
862+
return entry;
863+
}
864+
865+
filteredWALEntryCount = -1;
866+
throwExceptionCount++;
867+
if (throwExceptionCount <= maxThrowExceptionCount - 1) {
868+
throw new WALEntryFilterRetryableException("failing filter");
869+
}
870+
return entry;
871+
}
872+
873+
public int getThrowExceptionCount() {
874+
return throwExceptionCount;
875+
}
876+
}
877+
794878
}

0 commit comments

Comments
 (0)