Skip to content

Commit d63ca4f

Browse files
authored
HBASE-26674 Should modify filesCompacting under storeWriteLock (#4040)
Signed-off-by: Josh Elser <[email protected]>
1 parent 4a94cfc commit d63ca4f

File tree

3 files changed

+12
-9
lines changed

3 files changed

+12
-9
lines changed

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

Lines changed: 6 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -1230,13 +1230,14 @@ private void writeCompactionWalRecord(Collection<HStoreFile> filesCompacted,
12301230
allowedOnPath = ".*/(HStore|TestHStore).java")
12311231
void replaceStoreFiles(Collection<HStoreFile> compactedFiles, Collection<HStoreFile> result,
12321232
boolean writeCompactionMarker) throws IOException {
1233-
storeEngine.replaceStoreFiles(compactedFiles, result);
1233+
storeEngine.replaceStoreFiles(compactedFiles, result, () -> {
1234+
synchronized(filesCompacting) {
1235+
filesCompacting.removeAll(compactedFiles);
1236+
}
1237+
});
12341238
if (writeCompactionMarker) {
12351239
writeCompactionWalRecord(compactedFiles, result);
12361240
}
1237-
synchronized (filesCompacting) {
1238-
filesCompacting.removeAll(compactedFiles);
1239-
}
12401241
// These may be null when the RS is shutting down. The space quota Chores will fix the Region
12411242
// sizes later so it's not super-critical if we miss these.
12421243
RegionServerServices rsServices = region.getRegionServerServices();
@@ -1567,7 +1568,7 @@ public void cancelRequestedCompaction(CompactionContext compaction) {
15671568
finishCompactionRequest(compaction.getRequest());
15681569
}
15691570

1570-
protected void finishCompactionRequest(CompactionRequestImpl cr) {
1571+
private void finishCompactionRequest(CompactionRequestImpl cr) {
15711572
this.region.reportCompactionRequestEnd(cr.isMajor(), cr.getFiles().size(), cr.getSize());
15721573
if (cr.isOffPeak()) {
15731574
offPeakCompactionTracker.set(false);

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

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -410,7 +410,8 @@ private void refreshStoreFilesInternal(Collection<StoreFileInfo> newFiles) throw
410410
List<HStoreFile> openedFiles = openStoreFiles(toBeAddedFiles, false);
411411

412412
// propogate the file changes to the underlying store file manager
413-
replaceStoreFiles(toBeRemovedStoreFiles, openedFiles); // won't throw an exception
413+
replaceStoreFiles(toBeRemovedStoreFiles, openedFiles, () -> {
414+
}); // won't throw an exception
414415
}
415416

416417
/**
@@ -493,12 +494,13 @@ public void addStoreFiles(Collection<HStoreFile> storeFiles,
493494
}
494495

495496
public void replaceStoreFiles(Collection<HStoreFile> compactedFiles,
496-
Collection<HStoreFile> newFiles) throws IOException {
497+
Collection<HStoreFile> newFiles, Runnable actionUnderLock) throws IOException {
497498
storeFileTracker.replace(StoreUtils.toStoreFileInfo(compactedFiles),
498499
StoreUtils.toStoreFileInfo(newFiles));
499500
writeLock();
500501
try {
501502
storeFileManager.addCompactionResults(compactedFiles, newFiles);
503+
actionUnderLock.run();
502504
} finally {
503505
writeUnlock();
504506
}

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1033,14 +1033,14 @@ public void testRefreshStoreFilesNotChanged() throws IOException {
10331033
// call first time after files changed
10341034
spiedStoreEngine.refreshStoreFiles();
10351035
assertEquals(2, this.store.getStorefilesCount());
1036-
verify(spiedStoreEngine, times(1)).replaceStoreFiles(any(), any());
1036+
verify(spiedStoreEngine, times(1)).replaceStoreFiles(any(), any(), any());
10371037

10381038
// call second time
10391039
spiedStoreEngine.refreshStoreFiles();
10401040

10411041
// ensure that replaceStoreFiles is not called, i.e, the times does not change, if files are not
10421042
// refreshed,
1043-
verify(spiedStoreEngine, times(1)).replaceStoreFiles(any(), any());
1043+
verify(spiedStoreEngine, times(1)).replaceStoreFiles(any(), any(), any());
10441044
}
10451045

10461046
private long countMemStoreScanner(StoreScanner scanner) {

0 commit comments

Comments
 (0)