Skip to content

Commit 70a2ee1

Browse files
authored
HBASE-27053 IOException during caching of uncompressed block to the block cache (#4610)
Signed-off-by: Duo Zhang <zhangduo@apache.org> Reviewed-by: wenwj0 <wenweijian2@huawei.com>
1 parent 02f2636 commit 70a2ee1

File tree

4 files changed

+164
-66
lines changed

4 files changed

+164
-66
lines changed

hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java

Lines changed: 18 additions & 24 deletions
Original file line numberDiff line numberDiff line change
@@ -493,18 +493,8 @@ private void overwriteHeader() {
493493
* @return the buffer with header skipped and checksum omitted.
494494
*/
495495
public ByteBuff getBufferWithoutHeader() {
496-
return this.getBufferWithoutHeader(false);
497-
}
498-
499-
/**
500-
* Returns a buffer that does not include the header or checksum.
501-
* @param withChecksum to indicate whether include the checksum or not.
502-
* @return the buffer with header skipped and checksum omitted.
503-
*/
504-
public ByteBuff getBufferWithoutHeader(boolean withChecksum) {
505496
ByteBuff dup = getBufferReadOnly();
506-
int delta = withChecksum ? 0 : totalChecksumBytes();
507-
return dup.position(headerSize()).limit(buf.limit() - delta).slice();
497+
return dup.position(headerSize()).slice();
508498
}
509499

510500
/**
@@ -568,19 +558,21 @@ void sanityCheck() throws IOException {
568558
sanityCheckAssertion(dup.getInt(), onDiskDataSizeWithHeader, "onDiskDataSizeWithHeader");
569559
}
570560

571-
int cksumBytes = totalChecksumBytes();
572-
int expectedBufLimit = onDiskDataSizeWithHeader + cksumBytes;
573-
if (dup.limit() != expectedBufLimit) {
574-
throw new AssertionError("Expected limit " + expectedBufLimit + ", got " + dup.limit());
561+
if (dup.limit() != onDiskDataSizeWithHeader) {
562+
throw new AssertionError(
563+
"Expected limit " + onDiskDataSizeWithHeader + ", got " + dup.limit());
575564
}
576565

577566
// We might optionally allocate HFILEBLOCK_HEADER_SIZE more bytes to read the next
578567
// block's header, so there are two sensible values for buffer capacity.
579568
int hdrSize = headerSize();
580569
dup.rewind();
581-
if (dup.remaining() != expectedBufLimit && dup.remaining() != expectedBufLimit + hdrSize) {
570+
if (
571+
dup.remaining() != onDiskDataSizeWithHeader
572+
&& dup.remaining() != onDiskDataSizeWithHeader + hdrSize
573+
) {
582574
throw new AssertionError("Invalid buffer capacity: " + dup.remaining() + ", expected "
583-
+ expectedBufLimit + " or " + (expectedBufLimit + hdrSize));
575+
+ onDiskDataSizeWithHeader + " or " + (onDiskDataSizeWithHeader + hdrSize));
584576
}
585577
}
586578

@@ -641,12 +633,13 @@ HFileBlock unpack(HFileContext fileContext, FSReader reader) throws IOException
641633
? reader.getBlockDecodingContext()
642634
: reader.getDefaultBlockDecodingContext();
643635
// Create a duplicated buffer without the header part.
636+
int headerSize = this.headerSize();
644637
ByteBuff dup = this.buf.duplicate();
645-
dup.position(this.headerSize());
638+
dup.position(headerSize);
646639
dup = dup.slice();
647640
// Decode the dup into unpacked#buf
648-
ctx.prepareDecoding(unpacked.getOnDiskSizeWithoutHeader(),
649-
unpacked.getUncompressedSizeWithoutHeader(), unpacked.getBufferWithoutHeader(true), dup);
641+
ctx.prepareDecoding(unpacked.getOnDiskDataSizeWithHeader() - headerSize,
642+
unpacked.getUncompressedSizeWithoutHeader(), unpacked.getBufferWithoutHeader(), dup);
650643
succ = true;
651644
return unpacked;
652645
} finally {
@@ -661,9 +654,8 @@ HFileBlock unpack(HFileContext fileContext, FSReader reader) throws IOException
661654
* buffer. Does not change header fields. Reserve room to keep checksum bytes too.
662655
*/
663656
private ByteBuff allocateBufferForUnpacking() {
664-
int cksumBytes = totalChecksumBytes();
665657
int headerSize = headerSize();
666-
int capacityNeeded = headerSize + uncompressedSizeWithoutHeader + cksumBytes;
658+
int capacityNeeded = headerSize + uncompressedSizeWithoutHeader;
667659

668660
ByteBuff source = buf.duplicate();
669661
ByteBuff newBuf = allocator.allocate(capacityNeeded);
@@ -682,9 +674,8 @@ private ByteBuff allocateBufferForUnpacking() {
682674
* calculated heuristic, not tracked attribute of the block.
683675
*/
684676
public boolean isUnpacked() {
685-
final int cksumBytes = totalChecksumBytes();
686677
final int headerSize = headerSize();
687-
final int expectedCapacity = headerSize + uncompressedSizeWithoutHeader + cksumBytes;
678+
final int expectedCapacity = headerSize + uncompressedSizeWithoutHeader;
688679
final int bufCapacity = buf.remaining();
689680
return bufCapacity == expectedCapacity || bufCapacity == expectedCapacity + headerSize;
690681
}
@@ -1709,6 +1700,9 @@ protected HFileBlock readBlockDataInternal(FSDataInputStream is, long offset,
17091700
if (verifyChecksum && !validateChecksum(offset, curBlock, hdrSize)) {
17101701
return null;
17111702
}
1703+
// remove checksum from buffer now that it's verified
1704+
int sizeWithoutChecksum = curBlock.getInt(Header.ON_DISK_DATA_SIZE_WITH_HEADER_INDEX);
1705+
curBlock.limit(sizeWithoutChecksum);
17121706
long duration = EnvironmentEdgeManager.currentTime() - startTime;
17131707
if (updateMetrics) {
17141708
HFile.updateReadLatency(duration, pread);

hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestChecksum.java

Lines changed: 26 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -149,11 +149,13 @@ public void testVerifyCheckSum() throws IOException {
149149
HFileBlock b = hbr.readBlockData(0, -1, false, false, true);
150150
assertTrue(!b.isSharedMem());
151151

152+
ByteBuff bufferWithChecksum = getBufferWithChecksum(b);
153+
152154
// verify SingleByteBuff checksum.
153-
verifySBBCheckSum(b.getBufferReadOnly());
155+
verifySBBCheckSum(bufferWithChecksum);
154156

155157
// verify MultiByteBuff checksum.
156-
verifyMBBCheckSum(b.getBufferReadOnly());
158+
verifyMBBCheckSum(bufferWithChecksum);
157159

158160
ByteBuff data = b.getBufferWithoutHeader();
159161
for (int i = 0; i < intCount; i++) {
@@ -169,6 +171,28 @@ public void testVerifyCheckSum() throws IOException {
169171
}
170172
}
171173

174+
/**
175+
* HFileBlock buffer does not include checksum because it is discarded after verifying upon
176+
* reading from disk. We artificially add a checksum onto the buffer for use in testing that
177+
* ChecksumUtil.validateChecksum works for SingleByteBuff and MultiByteBuff in
178+
* {@link #verifySBBCheckSum(ByteBuff)} and {@link #verifyMBBCheckSum(ByteBuff)}
179+
*/
180+
private ByteBuff getBufferWithChecksum(HFileBlock block) throws IOException {
181+
ByteBuff buf = block.getBufferReadOnly();
182+
183+
int numBytes =
184+
(int) ChecksumUtil.numBytes(buf.remaining(), block.getHFileContext().getBytesPerChecksum());
185+
byte[] checksum = new byte[numBytes];
186+
ChecksumUtil.generateChecksums(buf.array(), 0, buf.limit(), checksum, 0,
187+
block.getHFileContext().getChecksumType(), block.getBytesPerChecksum());
188+
189+
ByteBuff bufWithChecksum = ByteBuffAllocator.HEAP.allocate(buf.limit() + numBytes);
190+
bufWithChecksum.put(buf.array(), 0, buf.limit());
191+
bufWithChecksum.put(checksum);
192+
193+
return bufWithChecksum.rewind();
194+
}
195+
172196
/**
173197
* Introduce checksum failures and check that we can still read the data
174198
*/

hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java

Lines changed: 6 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -652,14 +652,13 @@ protected void testPreviousOffsetInternals() throws IOException {
652652
// verifies that the unpacked value read back off disk matches the unpacked value
653653
// generated before writing to disk.
654654
HFileBlock newBlock = b.unpack(meta, hbr);
655-
// b's buffer has header + data + checksum while
656-
// expectedContents have header + data only
655+
// neither b's unpacked nor the expectedContents have checksum.
656+
// they should be identical
657657
ByteBuff bufRead = newBlock.getBufferReadOnly();
658658
ByteBuffer bufExpected = expectedContents.get(i);
659-
byte[] tmp = new byte[bufRead.limit() - newBlock.totalChecksumBytes()];
660-
bufRead.get(tmp, 0, tmp.length);
661-
boolean bytesAreCorrect = Bytes.compareTo(tmp, 0, tmp.length, bufExpected.array(),
662-
bufExpected.arrayOffset(), bufExpected.limit()) == 0;
659+
byte[] bytesRead = bufRead.toBytes();
660+
boolean bytesAreCorrect = Bytes.compareTo(bytesRead, 0, bytesRead.length,
661+
bufExpected.array(), bufExpected.arrayOffset(), bufExpected.limit()) == 0;
663662
String wrongBytesMsg = "";
664663

665664
if (!bytesAreCorrect) {
@@ -669,8 +668,7 @@ protected void testPreviousOffsetInternals() throws IOException {
669668
+ pread + ", cacheOnWrite=" + cacheOnWrite + "):\n";
670669
wrongBytesMsg += Bytes.toStringBinary(bufExpected.array(),
671670
bufExpected.arrayOffset(), Math.min(32 + 10, bufExpected.limit())) + ", actual:\n"
672-
+ Bytes.toStringBinary(bufRead.array(), bufRead.arrayOffset(),
673-
Math.min(32 + 10, bufRead.limit()));
671+
+ Bytes.toStringBinary(bytesRead, 0, Math.min(32 + 10, bytesRead.length));
674672
if (detailedLogging) {
675673
LOG.warn(
676674
"expected header" + HFileBlock.toStringHeader(new SingleByteBuff(bufExpected))

hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockUnpack.java

Lines changed: 114 additions & 32 deletions
Original file line numberDiff line numberDiff line change
@@ -17,10 +17,13 @@
1717
*/
1818
package org.apache.hadoop.hbase.io.hfile;
1919

20+
import static org.junit.Assert.assertEquals;
2021
import static org.junit.Assert.assertFalse;
2122
import static org.junit.Assert.assertTrue;
2223

2324
import java.io.IOException;
25+
import java.util.ArrayList;
26+
import java.util.List;
2427
import java.util.Random;
2528
import org.apache.hadoop.conf.Configuration;
2629
import org.apache.hadoop.fs.FSDataInputStream;
@@ -30,11 +33,13 @@
3033
import org.apache.hadoop.hbase.HBaseClassTestRule;
3134
import org.apache.hadoop.hbase.HBaseConfiguration;
3235
import org.apache.hadoop.hbase.HBaseTestingUtil;
36+
import org.apache.hadoop.hbase.HConstants;
3337
import org.apache.hadoop.hbase.KeyValue;
3438
import org.apache.hadoop.hbase.fs.HFileSystem;
3539
import org.apache.hadoop.hbase.io.ByteBuffAllocator;
3640
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
3741
import org.apache.hadoop.hbase.io.compress.Compression;
42+
import org.apache.hadoop.hbase.nio.ByteBuff;
3843
import org.apache.hadoop.hbase.testclassification.IOTests;
3944
import org.apache.hadoop.hbase.testclassification.MediumTests;
4045
import org.apache.hadoop.hbase.util.Bytes;
@@ -71,7 +76,62 @@ public void setUp() throws Exception {
7176
Configuration conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
7277
conf.setInt(ByteBuffAllocator.MIN_ALLOCATE_SIZE_KEY, MIN_ALLOCATION_SIZE);
7378
allocator = ByteBuffAllocator.create(conf, true);
79+
}
7480

81+
/**
82+
* It's important that if you read and unpack the same HFileBlock twice, it results in an
83+
* identical buffer each time. Otherwise we end up with validation failures in block cache, since
84+
* contents may not match if the same block is cached twice. See
85+
* https://issues.apache.org/jira/browse/HBASE-27053
86+
*/
87+
@Test
88+
public void itUnpacksIdenticallyEachTime() throws IOException {
89+
Path path = new Path(TEST_UTIL.getDataTestDir(), name.getMethodName());
90+
int totalSize = createTestBlock(path);
91+
92+
// Allocate a bunch of random buffers, so we can be sure that unpack will only have "dirty"
93+
// buffers to choose from when allocating itself.
94+
Random random = new Random();
95+
byte[] temp = new byte[HConstants.DEFAULT_BLOCKSIZE];
96+
List<ByteBuff> buffs = new ArrayList<>();
97+
for (int i = 0; i < 10; i++) {
98+
ByteBuff buff = allocator.allocate(HConstants.DEFAULT_BLOCKSIZE);
99+
random.nextBytes(temp);
100+
buff.put(temp);
101+
buffs.add(buff);
102+
}
103+
104+
buffs.forEach(ByteBuff::release);
105+
106+
// read the same block twice. we should expect the underlying buffer below to
107+
// be identical each time
108+
HFileBlockWrapper blockOne = readBlock(path, totalSize);
109+
HFileBlockWrapper blockTwo = readBlock(path, totalSize);
110+
111+
// first check size fields
112+
assertEquals(blockOne.original.getOnDiskSizeWithHeader(),
113+
blockTwo.original.getOnDiskSizeWithHeader());
114+
assertEquals(blockOne.original.getUncompressedSizeWithoutHeader(),
115+
blockTwo.original.getUncompressedSizeWithoutHeader());
116+
117+
// next check packed buffers
118+
assertBuffersEqual(blockOne.original.getBufferWithoutHeader(),
119+
blockTwo.original.getBufferWithoutHeader(),
120+
blockOne.original.getOnDiskDataSizeWithHeader() - blockOne.original.headerSize());
121+
122+
// now check unpacked buffers. prior to HBASE-27053, this would fail because
123+
// the unpacked buffer would include extra space for checksums at the end that was not written.
124+
// so the checksum space would be filled with random junk when re-using pooled buffers.
125+
assertBuffersEqual(blockOne.unpacked.getBufferWithoutHeader(),
126+
blockTwo.unpacked.getBufferWithoutHeader(),
127+
blockOne.original.getUncompressedSizeWithoutHeader());
128+
}
129+
130+
private void assertBuffersEqual(ByteBuff bufferOne, ByteBuff bufferTwo, int expectedSize) {
131+
assertEquals(expectedSize, bufferOne.limit());
132+
assertEquals(expectedSize, bufferTwo.limit());
133+
assertEquals(0,
134+
ByteBuff.compareTo(bufferOne, 0, bufferOne.limit(), bufferTwo, 0, bufferTwo.limit()));
75135
}
76136

77137
/**
@@ -83,53 +143,75 @@ public void setUp() throws Exception {
83143
*/
84144
@Test
85145
public void itUsesSharedMemoryIfUnpackedBlockExceedsMinAllocationSize() throws IOException {
86-
Configuration conf = TEST_UTIL.getConfiguration();
87-
HFileContext meta =
88-
new HFileContextBuilder().withCompression(Compression.Algorithm.GZ).withIncludesMvcc(false)
89-
.withIncludesTags(false).withBytesPerCheckSum(HFile.DEFAULT_BYTES_PER_CHECKSUM).build();
90-
91146
Path path = new Path(TEST_UTIL.getDataTestDir(), name.getMethodName());
92-
int totalSize;
93-
try (FSDataOutputStream os = fs.create(path)) {
94-
HFileBlock.Writer hbw = new HFileBlock.Writer(conf, NoOpDataBlockEncoder.INSTANCE, meta);
95-
hbw.startWriting(BlockType.DATA);
96-
writeTestKeyValues(hbw, MIN_ALLOCATION_SIZE - 1);
97-
hbw.writeHeaderAndData(os);
98-
totalSize = hbw.getOnDiskSizeWithHeader();
99-
assertTrue(
100-
"expected generated block size " + totalSize + " to be less than " + MIN_ALLOCATION_SIZE,
101-
totalSize < MIN_ALLOCATION_SIZE);
147+
int totalSize = createTestBlock(path);
148+
HFileBlockWrapper blockFromHFile = readBlock(path, totalSize);
149+
150+
assertFalse("expected hfile block to NOT be unpacked", blockFromHFile.original.isUnpacked());
151+
assertFalse("expected hfile block to NOT use shared memory",
152+
blockFromHFile.original.isSharedMem());
153+
154+
assertTrue(
155+
"expected generated block size " + blockFromHFile.original.getOnDiskSizeWithHeader()
156+
+ " to be less than " + MIN_ALLOCATION_SIZE,
157+
blockFromHFile.original.getOnDiskSizeWithHeader() < MIN_ALLOCATION_SIZE);
158+
assertTrue(
159+
"expected generated block uncompressed size "
160+
+ blockFromHFile.original.getUncompressedSizeWithoutHeader() + " to be more than "
161+
+ MIN_ALLOCATION_SIZE,
162+
blockFromHFile.original.getUncompressedSizeWithoutHeader() > MIN_ALLOCATION_SIZE);
163+
164+
assertTrue("expected unpacked block to be unpacked", blockFromHFile.unpacked.isUnpacked());
165+
assertTrue("expected unpacked block to use shared memory",
166+
blockFromHFile.unpacked.isSharedMem());
167+
}
168+
169+
private final static class HFileBlockWrapper {
170+
private final HFileBlock original;
171+
private final HFileBlock unpacked;
172+
173+
private HFileBlockWrapper(HFileBlock original, HFileBlock unpacked) {
174+
this.original = original;
175+
this.unpacked = unpacked;
102176
}
177+
}
103178

179+
private HFileBlockWrapper readBlock(Path path, int totalSize) throws IOException {
104180
try (FSDataInputStream is = fs.open(path)) {
105-
meta =
181+
HFileContext meta =
106182
new HFileContextBuilder().withHBaseCheckSum(true).withCompression(Compression.Algorithm.GZ)
107183
.withIncludesMvcc(false).withIncludesTags(false).build();
108184
ReaderContext context =
109185
new ReaderContextBuilder().withInputStreamWrapper(new FSDataInputStreamWrapper(is))
110186
.withFileSize(totalSize).withFilePath(path).withFileSystem(fs).build();
111-
HFileBlock.FSReaderImpl hbr = new HFileBlock.FSReaderImpl(context, meta, allocator, conf);
112-
hbr.setDataBlockEncoder(NoOpDataBlockEncoder.INSTANCE, conf);
187+
HFileBlock.FSReaderImpl hbr =
188+
new HFileBlock.FSReaderImpl(context, meta, allocator, TEST_UTIL.getConfiguration());
189+
hbr.setDataBlockEncoder(NoOpDataBlockEncoder.INSTANCE, TEST_UTIL.getConfiguration());
113190
hbr.setIncludesMemStoreTS(false);
114191
HFileBlock blockFromHFile = hbr.readBlockData(0, -1, false, false, false);
115192
blockFromHFile.sanityCheck();
116-
assertFalse("expected hfile block to NOT be unpacked", blockFromHFile.isUnpacked());
117-
assertFalse("expected hfile block to NOT use shared memory", blockFromHFile.isSharedMem());
193+
return new HFileBlockWrapper(blockFromHFile, blockFromHFile.unpack(meta, hbr));
194+
}
195+
}
118196

197+
private int createTestBlock(Path path) throws IOException {
198+
HFileContext meta =
199+
new HFileContextBuilder().withCompression(Compression.Algorithm.GZ).withIncludesMvcc(false)
200+
.withIncludesTags(false).withBytesPerCheckSum(HFile.DEFAULT_BYTES_PER_CHECKSUM).build();
201+
202+
int totalSize;
203+
try (FSDataOutputStream os = fs.create(path)) {
204+
HFileBlock.Writer hbw =
205+
new HFileBlock.Writer(TEST_UTIL.getConfiguration(), NoOpDataBlockEncoder.INSTANCE, meta);
206+
hbw.startWriting(BlockType.DATA);
207+
writeTestKeyValues(hbw, MIN_ALLOCATION_SIZE - 1);
208+
hbw.writeHeaderAndData(os);
209+
totalSize = hbw.getOnDiskSizeWithHeader();
119210
assertTrue(
120-
"expected generated block size " + blockFromHFile.getOnDiskSizeWithHeader()
121-
+ " to be less than " + MIN_ALLOCATION_SIZE,
122-
blockFromHFile.getOnDiskSizeWithHeader() < MIN_ALLOCATION_SIZE);
123-
assertTrue(
124-
"expected generated block uncompressed size "
125-
+ blockFromHFile.getUncompressedSizeWithoutHeader() + " to be more than "
126-
+ MIN_ALLOCATION_SIZE,
127-
blockFromHFile.getUncompressedSizeWithoutHeader() > MIN_ALLOCATION_SIZE);
128-
129-
HFileBlock blockUnpacked = blockFromHFile.unpack(meta, hbr);
130-
assertTrue("expected unpacked block to be unpacked", blockUnpacked.isUnpacked());
131-
assertTrue("expected unpacked block to use shared memory", blockUnpacked.isSharedMem());
211+
"expected generated block size " + totalSize + " to be less than " + MIN_ALLOCATION_SIZE,
212+
totalSize < MIN_ALLOCATION_SIZE);
132213
}
214+
return totalSize;
133215
}
134216

135217
static int writeTestKeyValues(HFileBlock.Writer hbw, int desiredSize) throws IOException {

0 commit comments

Comments
 (0)