1   /*
2    * Copyright The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.io.hfile;
21  
22  import static org.junit.Assert.*;
23  
24  import java.io.ByteArrayInputStream;
25  import java.io.DataOutputStream;
26  import java.io.DataInputStream;
27  import java.io.IOException;
28  import java.nio.ByteBuffer;
29  
30  import org.apache.commons.logging.Log;
31  import org.apache.commons.logging.LogFactory;
32  import org.apache.hadoop.fs.FSDataInputStream;
33  import org.apache.hadoop.fs.FSDataOutputStream;
34  import org.apache.hadoop.fs.FileSystem;
35  import org.apache.hadoop.fs.Path;
36  import org.apache.hadoop.hbase.HBaseTestingUtility;
37  import org.apache.hadoop.hbase.MediumTests;
38  import org.apache.hadoop.hbase.fs.HFileSystem;
39  import org.apache.hadoop.hbase.io.hfile.Compression.Algorithm;
40  import org.apache.hadoop.hbase.util.ChecksumType;
41  
42  import static org.apache.hadoop.hbase.io.hfile.Compression.Algorithm.*;
43  import org.junit.Before;
44  import org.junit.Test;
45  import org.junit.experimental.categories.Category;
46  
47  @Category(MediumTests.class)
48  public class TestChecksum {
49    // change this value to activate more logs
50    private static final boolean detailedLogging = true;
51    private static final boolean[] BOOLEAN_VALUES = new boolean[] { false, true };
52  
53    private static final Log LOG = LogFactory.getLog(TestHFileBlock.class);
54  
55    static final Compression.Algorithm[] COMPRESSION_ALGORITHMS = {
56        NONE, GZ };
57  
58    static final int[] BYTES_PER_CHECKSUM = {
59        50, 500, 688, 16*1024, (16*1024+980), 64 * 1024};
60  
61    private static final HBaseTestingUtility TEST_UTIL =
62      new HBaseTestingUtility();
63    private FileSystem fs;
64    private HFileSystem hfs;
65  
66    @Before
67    public void setUp() throws Exception {
68      fs = HFileSystem.get(TEST_UTIL.getConfiguration());
69      hfs = (HFileSystem)fs;
70    }
71  
72    /**
73     * Introduce checksum failures and check that we can still read
74     * the data
75     */
76    @Test
77    public void testChecksumCorruption() throws IOException {
78      for (Compression.Algorithm algo : COMPRESSION_ALGORITHMS) {
79        for (boolean pread : new boolean[] { false, true }) {
80          LOG.info("testChecksumCorruption: Compression algorithm: " + algo +
81                     ", pread=" + pread);
82          Path path = new Path(TEST_UTIL.getDataTestDir(), "blocks_v2_"
83              + algo);
84          FSDataOutputStream os = fs.create(path);
85          HFileBlock.Writer hbw = new HFileBlock.Writer(algo, null,
86              true, 1, HFile.DEFAULT_CHECKSUM_TYPE,
87              HFile.DEFAULT_BYTES_PER_CHECKSUM);
88          long totalSize = 0;
89          for (int blockId = 0; blockId < 2; ++blockId) {
90            DataOutputStream dos = hbw.startWriting(BlockType.DATA);
91            for (int i = 0; i < 1234; ++i)
92              dos.writeInt(i);
93            hbw.writeHeaderAndData(os);
94            totalSize += hbw.getOnDiskSizeWithHeader();
95          }
96          os.close();
97  
98          // Use hbase checksums. 
99          assertEquals(true, hfs.useHBaseChecksum());
100 
101         // Do a read that purposely introduces checksum verification failures.
102         FSDataInputStream is = fs.open(path);
103         HFileBlock.FSReader hbr = new FSReaderV2Test(is, algo,
104             totalSize, HFile.MAX_FORMAT_VERSION, fs, path);
105         HFileBlock b = hbr.readBlockData(0, -1, -1, pread);
106         b.sanityCheck();
107         assertEquals(4936, b.getUncompressedSizeWithoutHeader());
108         assertEquals(algo == GZ ? 2173 : 4936, 
109                      b.getOnDiskSizeWithoutHeader() - b.totalChecksumBytes());
110         // read data back from the hfile, exclude header and checksum
111         ByteBuffer bb = b.getBufferWithoutHeader(); // read back data
112         DataInputStream in = new DataInputStream(
113                                new ByteArrayInputStream(
114                                  bb.array(), bb.arrayOffset(), bb.limit()));
115 
116         // assert that we encountered hbase checksum verification failures
117         // but still used hdfs checksums and read data successfully.
118         assertEquals(1, HFile.getChecksumFailuresCount());
119         validateData(in);
120 
121         // A single instance of hbase checksum failure causes the reader to
122         // switch off hbase checksum verification for the next 100 read
123         // requests. Verify that this is correct.
124         for (int i = 0; i < 
125              HFileBlock.CHECKSUM_VERIFICATION_NUM_IO_THRESHOLD + 1; i++) {
126           b = hbr.readBlockData(0, -1, -1, pread);
127           assertEquals(0, HFile.getChecksumFailuresCount());
128         }
129         // The next read should have hbase checksum verification reanabled,
130         // we verify this by assertng that there was a hbase-checksum failure.
131         b = hbr.readBlockData(0, -1, -1, pread);
132         assertEquals(1, HFile.getChecksumFailuresCount());
133 
134         // Since the above encountered a checksum failure, we switch
135         // back to not checking hbase checksums.
136         b = hbr.readBlockData(0, -1, -1, pread);
137         assertEquals(0, HFile.getChecksumFailuresCount());
138         is.close();
139 
140         // Now, use a completely new reader. Switch off hbase checksums in 
141         // the configuration. In this case, we should not detect
142         // any retries within hbase. 
143         HFileSystem newfs = new HFileSystem(TEST_UTIL.getConfiguration(), false);
144         assertEquals(false, newfs.useHBaseChecksum());
145         is = newfs.open(path);
146         hbr = new FSReaderV2Test(is, algo,
147             totalSize, HFile.MAX_FORMAT_VERSION, newfs, path);
148         b = hbr.readBlockData(0, -1, -1, pread);
149         is.close();
150         b.sanityCheck();
151         assertEquals(4936, b.getUncompressedSizeWithoutHeader());
152         assertEquals(algo == GZ ? 2173 : 4936, 
153                      b.getOnDiskSizeWithoutHeader() - b.totalChecksumBytes());
154         // read data back from the hfile, exclude header and checksum
155         bb = b.getBufferWithoutHeader(); // read back data
156         in = new DataInputStream(new ByteArrayInputStream(
157                                  bb.array(), bb.arrayOffset(), bb.limit()));
158 
159         // assert that we did not encounter hbase checksum verification failures
160         // but still used hdfs checksums and read data successfully.
161         assertEquals(0, HFile.getChecksumFailuresCount());
162         validateData(in);
163       }
164     }
165   }
166 
167   /** 
168    * Test different values of bytesPerChecksum
169    */
170   @Test
171   public void testChecksumChunks() throws IOException {
172     Compression.Algorithm algo = NONE;
173     for (boolean pread : new boolean[] { false, true }) {
174       for (int bytesPerChecksum : BYTES_PER_CHECKSUM) {
175         Path path = new Path(TEST_UTIL.getDataTestDir(), "checksumChunk_" + 
176                              algo + bytesPerChecksum);
177         FSDataOutputStream os = fs.create(path);
178         HFileBlock.Writer hbw = new HFileBlock.Writer(algo, null,
179           true, 1,HFile.DEFAULT_CHECKSUM_TYPE, bytesPerChecksum);
180 
181         // write one block. The block has data
182         // that is at least 6 times more than the checksum chunk size
183         long dataSize = 0;
184         DataOutputStream dos = hbw.startWriting(BlockType.DATA);
185         for (; dataSize < 6 * bytesPerChecksum;) {
186           for (int i = 0; i < 1234; ++i) {
187             dos.writeInt(i);
188             dataSize += 4;
189           }
190         }
191         hbw.writeHeaderAndData(os);
192         long totalSize = hbw.getOnDiskSizeWithHeader();
193         os.close();
194 
195         long expectedChunks = ChecksumUtil.numChunks(
196                                dataSize + HFileBlock.HEADER_SIZE_WITH_CHECKSUMS,
197                                bytesPerChecksum);
198         LOG.info("testChecksumChunks: pread=" + pread +
199                    ", bytesPerChecksum=" + bytesPerChecksum +
200                    ", fileSize=" + totalSize +
201                    ", dataSize=" + dataSize +
202                    ", expectedChunks=" + expectedChunks);
203 
204         // Verify hbase checksums. 
205         assertEquals(true, hfs.useHBaseChecksum());
206 
207         // Read data back from file.
208         FSDataInputStream is = fs.open(path);
209         FSDataInputStream nochecksum = hfs.getNoChecksumFs().open(path);
210         HFileBlock.FSReader hbr = new HFileBlock.FSReaderV2(is, nochecksum, 
211             algo, totalSize, HFile.MAX_FORMAT_VERSION, hfs, path);
212         HFileBlock b = hbr.readBlockData(0, -1, -1, pread);
213         is.close();
214         b.sanityCheck();
215         assertEquals(dataSize, b.getUncompressedSizeWithoutHeader());
216 
217         // verify that we have the expected number of checksum chunks
218         assertEquals(totalSize, HFileBlock.HEADER_SIZE_WITH_CHECKSUMS + dataSize +
219                      expectedChunks * HFileBlock.CHECKSUM_SIZE);
220 
221         // assert that we did not encounter hbase checksum verification failures
222         assertEquals(0, HFile.getChecksumFailuresCount());
223       }
224     }
225   }
226 
227   /** 
228    * Test to ensure that these is at least one valid checksum implementation
229    */
230   @Test
231   public void testChecksumAlgorithm() throws IOException {
232     ChecksumType type = ChecksumType.CRC32;
233     assertEquals(ChecksumType.nameToType(type.getName()), type);
234     assertEquals(ChecksumType.valueOf(type.toString()), type);
235   }
236 
237   private void validateData(DataInputStream in) throws IOException {
238     // validate data
239     for (int i = 0; i < 1234; i++) {
240       int val = in.readInt();
241       if (val != i) {
242         String msg = "testChecksumCorruption: data mismatch at index " +
243                      i + " expected " + i + " found " + val;
244         LOG.warn(msg);
245         assertEquals(i, val);
246       }
247     }
248   }
249 
250   @org.junit.Rule
251   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
252     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
253 
254   /**
255    * A class that introduces hbase-checksum failures while 
256    * reading  data from hfiles. This should trigger the hdfs level
257    * checksum validations.
258    */
259   static private class FSReaderV2Test extends HFileBlock.FSReaderV2 {
260 
261     FSReaderV2Test(FSDataInputStream istream, Algorithm algo,
262                    long fileSize, int minorVersion, FileSystem fs,
263                    Path path) throws IOException {
264       super(istream, istream, algo, fileSize, minorVersion, 
265             (HFileSystem)fs, path);
266     }
267 
268     @Override
269     protected boolean validateBlockChecksum(HFileBlock block, 
270       byte[] data, int hdrSize) throws IOException {
271       return false;  // checksum validation failure
272     }
273   }
274 }
275