Skip to content

Commit

Permalink
HBASE-27646 Should not use pread when prefetching in HFilePreadReader
Browse files Browse the repository at this point in the history
  • Loading branch information
sunhelly committed Feb 27, 2023
1 parent f20efaf commit e1b9bd6
Showing 1 changed file with 14 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand All @@ -41,7 +42,15 @@ public HFilePreadReader(ReaderContext context, HFileInfo fileInfo, CacheConfig c
public void run() {
long offset = 0;
long end = 0;
HFile.Reader prefetchStreamReader = null;
try {
ReaderContext streamReaderContext = new ReaderContextBuilder()
.withReaderType(ReaderContext.ReaderType.STREAM).withFilePath(context.getFilePath())
.withFileSize(context.getFileSize()).withFileSystem(context.getFileSystem())
.withInputStreamWrapper(new FSDataInputStreamWrapper(context.getFileSystem(), path))
.build();
prefetchStreamReader =
new HFileStreamReader(streamReaderContext, fileInfo, cacheConf, conf);
end = getTrailer().getLoadOnOpenDataOffset();
if (LOG.isTraceEnabled()) {
LOG.trace("Prefetch start " + getPathOffsetEndStr(path, offset, end));
Expand All @@ -56,8 +65,8 @@ public void run() {
// the internal-to-hfileblock thread local which holds the overread that gets the
// next header, will not have happened...so, pass in the onDiskSize gotten from the
// cached block. This 'optimization' triggers extremely rarely I'd say.
HFileBlock block = readBlock(offset, onDiskSizeOfNextBlock, /* cacheBlock= */true,
/* pread= */true, false, false, null, null, true);
HFileBlock block = prefetchStreamReader.readBlock(offset, onDiskSizeOfNextBlock,
/* cacheBlock= */true, /* pread= */false, false, false, null, null, true);
try {
onDiskSizeOfNextBlock = block.getNextBlockOnDiskSize();
offset += block.getOnDiskSizeWithHeader();
Expand All @@ -77,6 +86,9 @@ public void run() {
// Other exceptions are interesting
LOG.warn("Prefetch " + getPathOffsetEndStr(path, offset, end), e);
} finally {
if (prefetchStreamReader != null) {
prefetchStreamReader.unbufferStream();
}
PrefetchExecutor.complete(path);
}
}
Expand Down

0 comments on commit e1b9bd6

Please sign in to comment.