joshelser commented on code in PR #4414:
URL: https://github.com/apache/hbase/pull/4414#discussion_r870885671
##########
hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBlockIOUtils.java:
##########
@@ -92,6 +108,102 @@ public void testReadFully() throws IOException {
assertArrayEquals(Bytes.toBytes(s), heapBuf);
}
+ @Test
+ public void testPreadWithReadFullBytes() throws IOException {
+ testPreadReadFullBytesInternal(true);
+ }
+
+ @Test
+ public void testPreadWithoutReadFullBytes() throws IOException {
+ testPreadReadFullBytesInternal(false);
+ }
+
+ private void testPreadReadFullBytesInternal(boolean readAllBytes) throws
IOException {
+ Configuration conf = TEST_UTIL.getConfiguration();
+ conf.setBoolean(HConstants.HFILE_PREAD_ALL_BYTES_ENABLED_KEY,
readAllBytes);
+ FileSystem fs = TEST_UTIL.getTestFileSystem();
+ Path path = new Path(TEST_UTIL.getDataTestDirOnTestFS(),
testName.getMethodName());
+ // give a fixed seed such we can see failure easily.
+ Random rand = new Random(5685632);
Review Comment:
nit: What about choosing a random seed (maybe `System.currentTimeMillis()`)
in `testPreadWithoutReadFullBytes()` and `testPreadWithReadFullBytes()` and
pass that seed into `testPreadReadFullBytesInternal(..)`. As long as we log the
seed we get the benefit of randomly choosing different data every time but a
human can come back and replay the same exact seed (as we'd have the seed
logged in surefire-reports).
##########
hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java:
##########
@@ -1453,7 +1453,12 @@ protected boolean readAtOffset(FSDataInputStream
istream, ByteBuff dest, int siz
} else {
// Positional read. Better for random reads; or when the streamLock is
already locked.
int extraSize = peekIntoNextBlock ? hdrSize : 0;
- if (!BlockIOUtils.preadWithExtra(dest, istream, fileOffset, size,
extraSize)) {
+ boolean readAllBytes =
+
hfs.getConf().getBoolean(HConstants.HFILE_PREAD_ALL_BYTES_ENABLED_KEY,
+ HConstants.HFILE_PREAD_ALL_BYTES_ENABLED_DEFAULT);
Review Comment:
Thinking some more, we should really do this once and not for every call to
`readAtOffset(..)`. What about moving this logic into the `ReaderContext` and
cache it there to avoid the expensive lookup into the Hadoop Configuration
object.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]