HBASE-12411 Optionally enable p-reads and private readers for compactions.

This commit is contained in:
Lars Hofhansl 2014-11-18 12:29:06 -08:00
parent bb0a3ce914
commit 2ed5ac9ca9
3 changed files with 48 additions and 11 deletions

View File

@ -219,6 +219,18 @@ public class StoreFile {
this.modificationTimeStamp = fileInfo.getModificationTime();
}
/**
* Clone
* @param other The StoreFile to clone from
*/
public StoreFile(final StoreFile other) {
this.fs = other.fs;
this.fileInfo = other.fileInfo;
this.cacheConf = other.cacheConf;
this.cfBloomType = other.cfBloomType;
this.modificationTimeStamp = other.modificationTimeStamp;
}
/**
* @return the StoreFile object associated to this StoreFile.
* null if the StoreFile is not a reference.

View File

@ -29,6 +29,7 @@ import java.util.concurrent.locks.ReentrantLock;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
@ -127,10 +128,13 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
if (store != null && ((HStore)store).getHRegion() != null
&& ((HStore)store).getHRegion().getBaseConf() != null) {
this.maxRowSize = ((HStore) store).getHRegion().getBaseConf().getLong(
HConstants.TABLE_MAX_ROWSIZE_KEY, HConstants.TABLE_MAX_ROWSIZE_DEFAULT);
Configuration conf = ((HStore) store).getHRegion().getBaseConf();
this.maxRowSize =
conf.getLong(HConstants.TABLE_MAX_ROWSIZE_KEY, HConstants.TABLE_MAX_ROWSIZE_DEFAULT);
this.scanUsePread = conf.getBoolean("hbase.storescanner.use.pread", scan.isSmall());
} else {
this.maxRowSize = HConstants.TABLE_MAX_ROWSIZE_DEFAULT;
this.scanUsePread = scan.isSmall();
}
// We look up row-column Bloom filters for multi-column queries as part of
@ -138,7 +142,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
// for multi-row (non-"get") scans because this is not done in
// StoreFile.passesBloomFilter(Scan, SortedSet<byte[]>).
useRowColBloom = numCol > 1 || (!isGet && numCol == 1);
this.scanUsePread = scan.isSmall();
// The parallel-seeking is on :
// 1) the config value is *true*
// 2) store has more than one store file

View File

@ -21,6 +21,7 @@ import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
@ -50,7 +51,21 @@ public class DefaultCompactor extends Compactor {
// Find the smallest read point across all the Scanners.
long smallestReadPoint = getSmallestReadPoint();
List<StoreFileScanner> scanners = createFileScanners(request.getFiles(), smallestReadPoint);
List<StoreFileScanner> scanners;
Collection<StoreFile> readersToClose;
if (this.conf.getBoolean("hbase.regionserver.compaction.private.readers", false)) {
// clone all StoreFiles, so we'll do the compaction on a independent copy of StoreFiles,
// HFileFiles, and their readers
readersToClose = new ArrayList<StoreFile>(request.getFiles().size());
for (StoreFile f : request.getFiles()) {
readersToClose.add(new StoreFile(f));
}
scanners = createFileScanners(readersToClose, smallestReadPoint);
} else {
readersToClose = Collections.emptyList();
scanners = createFileScanners(request.getFiles(), smallestReadPoint);
}
StoreFile.Writer writer = null;
List<Path> newFiles = new ArrayList<Path>();
@ -99,13 +114,19 @@ public class DefaultCompactor extends Compactor {
throw ioe;
}
finally {
if (writer != null) {
if (e != null) {
writer.close();
} else {
writer.appendMetadata(fd.maxSeqId, request.isAllFiles());
writer.close();
newFiles.add(writer.getPath());
try {
if (writer != null) {
if (e != null) {
writer.close();
} else {
writer.appendMetadata(fd.maxSeqId, request.isAllFiles());
writer.close();
newFiles.add(writer.getPath());
}
}
} finally {
for (StoreFile f : readersToClose) {
f.closeReader(true);
}
}
}