1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.io.hfile;
21
22 import static org.apache.hadoop.hbase.io.compress.Compression.Algorithm.GZ;
23 import static org.apache.hadoop.hbase.io.compress.Compression.Algorithm.NONE;
24 import static org.junit.Assert.assertEquals;
25 import static org.junit.Assert.assertTrue;
26
27 import java.io.ByteArrayInputStream;
28 import java.io.DataInputStream;
29 import java.io.DataOutputStream;
30 import java.io.IOException;
31 import java.nio.BufferUnderflowException;
32 import java.nio.ByteBuffer;
33 import java.util.ArrayList;
34 import java.util.Arrays;
35 import java.util.Iterator;
36 import java.util.List;
37
38 import org.apache.commons.logging.Log;
39 import org.apache.commons.logging.LogFactory;
40 import org.apache.hadoop.fs.FSDataInputStream;
41 import org.apache.hadoop.fs.FSDataOutputStream;
42 import org.apache.hadoop.fs.FileSystem;
43 import org.apache.hadoop.fs.Path;
44 import org.apache.hadoop.hbase.HBaseTestingUtility;
45 import org.apache.hadoop.hbase.HConstants;
46 import org.apache.hadoop.hbase.testclassification.SmallTests;
47 import org.apache.hadoop.hbase.fs.HFileSystem;
48 import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
49 import org.apache.hadoop.hbase.io.compress.Compression;
50 import org.apache.hadoop.hbase.util.ChecksumType;
51 import org.junit.Before;
52 import org.junit.Test;
53 import org.junit.experimental.categories.Category;
54
55 @Category(SmallTests.class)
56 public class TestChecksum {
57 private static final Log LOG = LogFactory.getLog(TestHFileBlock.class);
58
59 static final Compression.Algorithm[] COMPRESSION_ALGORITHMS = {
60 NONE, GZ };
61
62 static final int[] BYTES_PER_CHECKSUM = {
63 50, 500, 688, 16*1024, (16*1024+980), 64 * 1024};
64
65 private static final HBaseTestingUtility TEST_UTIL =
66 new HBaseTestingUtility();
67 private FileSystem fs;
68 private HFileSystem hfs;
69
70 @Before
71 public void setUp() throws Exception {
72 fs = HFileSystem.get(TEST_UTIL.getConfiguration());
73 hfs = (HFileSystem)fs;
74 }
75
76 @Test
77 public void testNewBlocksHaveDefaultChecksum() throws IOException {
78 Path path = new Path(TEST_UTIL.getDataTestDir(), "default_checksum");
79 FSDataOutputStream os = fs.create(path);
80 HFileContext meta = new HFileContextBuilder().build();
81 HFileBlock.Writer hbw = new HFileBlock.Writer(null, meta);
82 DataOutputStream dos = hbw.startWriting(BlockType.DATA);
83 for (int i = 0; i < 1000; ++i)
84 dos.writeInt(i);
85 hbw.writeHeaderAndData(os);
86 int totalSize = hbw.getOnDiskSizeWithHeader();
87 os.close();
88
89
90 assertEquals(true, hfs.useHBaseChecksum());
91
92 FSDataInputStreamWrapper is = new FSDataInputStreamWrapper(fs, path);
93 meta = new HFileContextBuilder().withHBaseCheckSum(true).build();
94 HFileBlock.FSReader hbr = new HFileBlock.FSReaderImpl(
95 is, totalSize, (HFileSystem) fs, path, meta);
96 HFileBlock b = hbr.readBlockData(0, -1, false, false);
97 assertEquals(b.getChecksumType(), ChecksumType.getDefaultChecksumType().getCode());
98 }
99
100
101
102
103 @Test
104 public void testAllChecksumTypes() throws IOException {
105 List<ChecksumType> cktypes = new ArrayList<>(Arrays.asList(ChecksumType.values()));
106 for (Iterator<ChecksumType> itr = cktypes.iterator(); itr.hasNext(); ) {
107 ChecksumType cktype = itr.next();
108 Path path = new Path(TEST_UTIL.getDataTestDir(), "checksum" + cktype.getName());
109 FSDataOutputStream os = fs.create(path);
110 HFileContext meta = new HFileContextBuilder()
111 .withChecksumType(cktype)
112 .build();
113 HFileBlock.Writer hbw = new HFileBlock.Writer(null, meta);
114 DataOutputStream dos = hbw.startWriting(BlockType.DATA);
115 for (int i = 0; i < 1000; ++i) {
116 dos.writeInt(i);
117 }
118 hbw.writeHeaderAndData(os);
119 int totalSize = hbw.getOnDiskSizeWithHeader();
120 os.close();
121
122
123 assertEquals(true, hfs.useHBaseChecksum());
124
125 FSDataInputStreamWrapper is = new FSDataInputStreamWrapper(fs, path);
126 meta = new HFileContextBuilder().withHBaseCheckSum(true).build();
127 HFileBlock.FSReader hbr = new HFileBlock.FSReaderImpl(
128 is, totalSize, (HFileSystem) fs, path, meta);
129 HFileBlock b = hbr.readBlockData(0, -1, false, false);
130 ByteBuffer data = b.getBufferWithoutHeader();
131 for (int i = 0; i < 1000; i++) {
132 assertEquals(i, data.getInt());
133 }
134 boolean exception_thrown = false;
135 try {
136 data.getInt();
137 } catch (BufferUnderflowException e) {
138 exception_thrown = true;
139 }
140 assertTrue(exception_thrown);
141 assertEquals(0, HFile.getAndResetChecksumFailuresCount());
142 }
143 }
144
145
146
147
148
149 @Test
150 public void testChecksumCorruption() throws IOException {
151 testChecksumCorruptionInternals(false);
152 testChecksumCorruptionInternals(true);
153 }
154
155 protected void testChecksumCorruptionInternals(boolean useTags) throws IOException {
156 for (Compression.Algorithm algo : COMPRESSION_ALGORITHMS) {
157 for (boolean pread : new boolean[] { false, true }) {
158 LOG.info("testChecksumCorruption: Compression algorithm: " + algo +
159 ", pread=" + pread);
160 Path path = new Path(TEST_UTIL.getDataTestDir(), "blocks_v2_"
161 + algo);
162 FSDataOutputStream os = fs.create(path);
163 HFileContext meta = new HFileContextBuilder()
164 .withCompression(algo)
165 .withIncludesMvcc(true)
166 .withIncludesTags(useTags)
167 .withBytesPerCheckSum(HFile.DEFAULT_BYTES_PER_CHECKSUM)
168 .build();
169 HFileBlock.Writer hbw = new HFileBlock.Writer(null, meta);
170 long totalSize = 0;
171 for (int blockId = 0; blockId < 2; ++blockId) {
172 DataOutputStream dos = hbw.startWriting(BlockType.DATA);
173 for (int i = 0; i < 1234; ++i)
174 dos.writeInt(i);
175 hbw.writeHeaderAndData(os);
176 totalSize += hbw.getOnDiskSizeWithHeader();
177 }
178 os.close();
179
180
181 assertEquals(true, hfs.useHBaseChecksum());
182
183
184 FSDataInputStreamWrapper is = new FSDataInputStreamWrapper(fs, path);
185 meta = new HFileContextBuilder()
186 .withCompression(algo)
187 .withIncludesMvcc(true)
188 .withIncludesTags(useTags)
189 .withHBaseCheckSum(true)
190 .build();
191 HFileBlock.FSReader hbr = new CorruptedFSReaderImpl(is, totalSize, fs, path, meta);
192 HFileBlock b = hbr.readBlockData(0, -1, pread, false);
193 b.sanityCheck();
194 assertEquals(4936, b.getUncompressedSizeWithoutHeader());
195 assertEquals(algo == GZ ? 2173 : 4936,
196 b.getOnDiskSizeWithoutHeader() - b.totalChecksumBytes());
197
198 ByteBuffer bb = b.unpack(meta, hbr).getBufferWithoutHeader();
199 DataInputStream in = new DataInputStream(
200 new ByteArrayInputStream(
201 bb.array(), bb.arrayOffset(), bb.limit()));
202
203
204
205 assertEquals(1, HFile.getAndResetChecksumFailuresCount());
206 validateData(in);
207
208
209
210
211 for (int i = 0; i <
212 HFileBlock.CHECKSUM_VERIFICATION_NUM_IO_THRESHOLD + 1; i++) {
213 b = hbr.readBlockData(0, -1, pread, false);
214 assertEquals(0, HFile.getAndResetChecksumFailuresCount());
215 }
216
217
218 b = hbr.readBlockData(0, -1, pread, false);
219 assertEquals(1, HFile.getAndResetChecksumFailuresCount());
220
221
222
223 b = hbr.readBlockData(0, -1, pread, false);
224 assertEquals(0, HFile.getAndResetChecksumFailuresCount());
225 is.close();
226
227
228
229
230 HFileSystem newfs = new HFileSystem(TEST_UTIL.getConfiguration(), false);
231 assertEquals(false, newfs.useHBaseChecksum());
232 is = new FSDataInputStreamWrapper(newfs, path);
233 hbr = new CorruptedFSReaderImpl(is, totalSize, newfs, path, meta);
234 b = hbr.readBlockData(0, -1, pread, false);
235 is.close();
236 b.sanityCheck();
237 b = b.unpack(meta, hbr);
238 assertEquals(4936, b.getUncompressedSizeWithoutHeader());
239 assertEquals(algo == GZ ? 2173 : 4936,
240 b.getOnDiskSizeWithoutHeader() - b.totalChecksumBytes());
241
242 bb = b.getBufferWithoutHeader();
243 in = new DataInputStream(new ByteArrayInputStream(
244 bb.array(), bb.arrayOffset(), bb.limit()));
245
246
247
248 assertEquals(0, HFile.getAndResetChecksumFailuresCount());
249 validateData(in);
250 }
251 }
252 }
253
254
255
256
257 @Test
258 public void testChecksumChunks() throws IOException {
259 testChecksumInternals(false);
260 testChecksumInternals(true);
261 }
262
263 protected void testChecksumInternals(boolean useTags) throws IOException {
264 Compression.Algorithm algo = NONE;
265 for (boolean pread : new boolean[] { false, true }) {
266 for (int bytesPerChecksum : BYTES_PER_CHECKSUM) {
267 Path path = new Path(TEST_UTIL.getDataTestDir(), "checksumChunk_" +
268 algo + bytesPerChecksum);
269 FSDataOutputStream os = fs.create(path);
270 HFileContext meta = new HFileContextBuilder()
271 .withCompression(algo)
272 .withIncludesMvcc(true)
273 .withIncludesTags(useTags)
274 .withHBaseCheckSum(true)
275 .withBytesPerCheckSum(bytesPerChecksum)
276 .build();
277 HFileBlock.Writer hbw = new HFileBlock.Writer(null,
278 meta);
279
280
281
282 long dataSize = 0;
283 DataOutputStream dos = hbw.startWriting(BlockType.DATA);
284 for (; dataSize < 6 * bytesPerChecksum;) {
285 for (int i = 0; i < 1234; ++i) {
286 dos.writeInt(i);
287 dataSize += 4;
288 }
289 }
290 hbw.writeHeaderAndData(os);
291 long totalSize = hbw.getOnDiskSizeWithHeader();
292 os.close();
293
294 long expectedChunks = ChecksumUtil.numChunks(
295 dataSize + HConstants.HFILEBLOCK_HEADER_SIZE,
296 bytesPerChecksum);
297 LOG.info("testChecksumChunks: pread=" + pread +
298 ", bytesPerChecksum=" + bytesPerChecksum +
299 ", fileSize=" + totalSize +
300 ", dataSize=" + dataSize +
301 ", expectedChunks=" + expectedChunks +
302 ", compression=" + algo);
303
304
305 assertEquals(true, hfs.useHBaseChecksum());
306
307
308 FSDataInputStream is = fs.open(path);
309 FSDataInputStream nochecksum = hfs.getNoChecksumFs().open(path);
310 meta = new HFileContextBuilder()
311 .withCompression(algo)
312 .withIncludesMvcc(true)
313 .withIncludesTags(useTags)
314 .withHBaseCheckSum(true)
315 .withBytesPerCheckSum(bytesPerChecksum)
316 .build();
317 HFileBlock.FSReader hbr = new HFileBlock.FSReaderImpl(new FSDataInputStreamWrapper(
318 is, nochecksum), totalSize, hfs, path, meta);
319 HFileBlock b = hbr.readBlockData(0, -1, pread, false);
320 is.close();
321 b.sanityCheck();
322 assertEquals(dataSize, b.getUncompressedSizeWithoutHeader());
323
324
325 assertEquals(totalSize, HConstants.HFILEBLOCK_HEADER_SIZE + dataSize +
326 expectedChunks * HFileBlock.CHECKSUM_SIZE);
327
328
329 assertEquals(0, HFile.getAndResetChecksumFailuresCount());
330 }
331 }
332 }
333
334 private void validateData(DataInputStream in) throws IOException {
335
336 for (int i = 0; i < 1234; i++) {
337 int val = in.readInt();
338 assertEquals("testChecksumCorruption: data mismatch at index " + i, i, val);
339 }
340 }
341
342
343
344
345
346
347
348
349
350
351 static private class CorruptedFSReaderImpl extends HFileBlock.FSReaderImpl {
352
353
354
355 boolean corruptDataStream = false;
356
357 public CorruptedFSReaderImpl(FSDataInputStreamWrapper istream, long fileSize, FileSystem fs,
358 Path path, HFileContext meta) throws IOException {
359 super(istream, fileSize, (HFileSystem) fs, path, meta);
360 }
361
362 @Override
363 protected HFileBlock readBlockDataInternal(FSDataInputStream is, long offset,
364 long onDiskSizeWithHeaderL, boolean pread, boolean verifyChecksum, boolean updateMetrics)
365 throws IOException {
366 if (verifyChecksum) {
367 corruptDataStream = true;
368 }
369 HFileBlock b = super.readBlockDataInternal(is, offset, onDiskSizeWithHeaderL, pread,
370 verifyChecksum, updateMetrics);
371 corruptDataStream = false;
372 return b;
373 }
374
375 @Override
376 protected int readAtOffset(FSDataInputStream istream, byte [] dest, int destOffset, int size,
377 boolean peekIntoNextBlock, long fileOffset, boolean pread) throws IOException {
378 int returnValue = super.readAtOffset(istream, dest, destOffset, size, peekIntoNextBlock,
379 fileOffset, pread);
380 if (!corruptDataStream) {
381 return returnValue;
382 }
383
384 if (peekIntoNextBlock) {
385 dest[destOffset + size + 3] = 0b00000000;
386 }
387
388 dest[destOffset + 1] = 0b00000000;
389
390 if (size > hdrSize) {
391 dest[destOffset + hdrSize + 1] = 0b00000000;
392 }
393 return returnValue;
394 }
395 }
396 }