Skip to content

Commit

Permalink
HBASE-26675 Data race on Compactor.writer (#4035)
Browse files Browse the repository at this point in the history
Signed-off-by: Xin Sun <[email protected]>
  • Loading branch information
Apache9 authored Jan 24, 2022
1 parent 6cd092b commit 97f3c1c
Show file tree
Hide file tree
Showing 2 changed files with 16 additions and 16 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -96,7 +96,10 @@ public abstract class Compactor<T extends CellSink> {
private final boolean dropCacheMajor;
private final boolean dropCacheMinor;

protected T writer = null;
// In compaction process only a single thread will access and write to this field, and
// getCompactionTargets is the only place we will access it other than the compaction thread, so
// make it volatile.
protected volatile T writer = null;

//TODO: depending on Store is not good but, realistically, all compactors currently do.
Compactor(Configuration conf, HStore store) {
Expand Down Expand Up @@ -547,17 +550,16 @@ protected InternalScanner createScanner(HStore store, ScanInfo scanInfo,
dropDeletesFromRow, dropDeletesToRow);
}

public List<Path> getCompactionTargets(){
if (writer == null){
public List<Path> getCompactionTargets() {
T writer = this.writer;
if (writer == null) {
return Collections.emptyList();
}
synchronized (writer){
if (writer instanceof StoreFileWriter){
return Arrays.asList(((StoreFileWriter)writer).getPath());
}
return ((AbstractMultiFileWriter)writer).writers().stream().map(sfw -> sfw.getPath()).collect(
Collectors.toList());
if (writer instanceof StoreFileWriter) {
return Arrays.asList(((StoreFileWriter) writer).getPath());
}
return ((AbstractMultiFileWriter) writer).writers().stream().map(sfw -> sfw.getPath())
.collect(Collectors.toList());
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -74,24 +74,22 @@ protected List<Path> commitWriter(FileDetails fd,
@Override
protected void abortWriter() throws IOException {
abortWriter(writer);
// this step signals that the target file is no longer written and can be cleaned up
writer = null;
}

protected void abortWriter(StoreFileWriter writer) throws IOException {
protected final void abortWriter(StoreFileWriter writer) throws IOException {
Path leftoverFile = writer.getPath();
try {
writer.close();
} catch (IOException e) {
LOG.warn("Failed to close the writer after an unfinished compaction.", e);
} finally {
//this step signals that the target file is no longer writen and can be cleaned up
writer = null;
}
try {
store.getFileSystem().delete(leftoverFile, false);
} catch (IOException e) {
LOG.warn(
"Failed to delete the leftover file " + leftoverFile + " after an unfinished compaction.",
e);
LOG.warn("Failed to delete the leftover file {} after an unfinished compaction.",
leftoverFile, e);
}
}
}

0 comments on commit 97f3c1c

Please sign in to comment.