View Javadoc

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.HConstants;
38  import org.apache.hadoop.hbase.SmallTests;
39  import org.apache.hadoop.hbase.fs.HFileSystem;
40  import org.apache.hadoop.hbase.io.compress.Compression;
41  import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
42  import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
43  import org.apache.hadoop.hbase.util.ChecksumType;
44  
45  import static org.apache.hadoop.hbase.io.compress.Compression.Algorithm.*;
46  
47  import org.junit.Before;
48  import org.junit.Test;
49  import org.junit.experimental.categories.Category;
50  
51  @Category(SmallTests.class)
52  public class TestChecksum {
53    // change this value to activate more logs
54    private static final boolean detailedLogging = true;
55    private static final boolean[] BOOLEAN_VALUES = new boolean[] { false, true };
56  
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    /**
77     * Introduce checksum failures and check that we can still read
78     * the data
79     */
80    @Test
81    public void testChecksumCorruption() throws IOException {
82      for (Compression.Algorithm algo : COMPRESSION_ALGORITHMS) {
83        for (boolean pread : new boolean[] { false, true }) {
84          LOG.info("testChecksumCorruption: Compression algorithm: " + algo +
85                     ", pread=" + pread);
86          Path path = new Path(TEST_UTIL.getDataTestDir(), "blocks_v2_"
87              + algo);
88          FSDataOutputStream os = fs.create(path);
89          HFileBlock.Writer hbw = new HFileBlock.Writer(algo, null,
90              true, HFile.DEFAULT_CHECKSUM_TYPE,
91              HFile.DEFAULT_BYTES_PER_CHECKSUM);
92          long totalSize = 0;
93          for (int blockId = 0; blockId < 2; ++blockId) {
94            DataOutputStream dos = hbw.startWriting(BlockType.DATA);
95            for (int i = 0; i < 1234; ++i)
96              dos.writeInt(i);
97            hbw.writeHeaderAndData(os);
98            totalSize += hbw.getOnDiskSizeWithHeader();
99          }
100         os.close();
101 
102         // Use hbase checksums. 
103         assertEquals(true, hfs.useHBaseChecksum());
104 
105         // Do a read that purposely introduces checksum verification failures.
106         FSDataInputStreamWrapper is = new FSDataInputStreamWrapper(fs, path);
107         HFileBlock.FSReader hbr = new FSReaderV2Test(is, algo,
108             totalSize, HFile.MAX_FORMAT_VERSION, fs, path);
109         HFileBlock b = hbr.readBlockData(0, -1, -1, pread);
110         b.sanityCheck();
111         assertEquals(4936, b.getUncompressedSizeWithoutHeader());
112         assertEquals(algo == GZ ? 2173 : 4936, 
113                      b.getOnDiskSizeWithoutHeader() - b.totalChecksumBytes());
114         // read data back from the hfile, exclude header and checksum
115         ByteBuffer bb = b.getBufferWithoutHeader(); // read back data
116         DataInputStream in = new DataInputStream(
117                                new ByteArrayInputStream(
118                                  bb.array(), bb.arrayOffset(), bb.limit()));
119 
120         // assert that we encountered hbase checksum verification failures
121         // but still used hdfs checksums and read data successfully.
122         assertEquals(1, HFile.getChecksumFailuresCount());
123         validateData(in);
124 
125         // A single instance of hbase checksum failure causes the reader to
126         // switch off hbase checksum verification for the next 100 read
127         // requests. Verify that this is correct.
128         for (int i = 0; i < 
129              HFileBlock.CHECKSUM_VERIFICATION_NUM_IO_THRESHOLD + 1; i++) {
130           b = hbr.readBlockData(0, -1, -1, pread);
131           assertEquals(0, HFile.getChecksumFailuresCount());
132         }
133         // The next read should have hbase checksum verification reanabled,
134         // we verify this by assertng that there was a hbase-checksum failure.
135         b = hbr.readBlockData(0, -1, -1, pread);
136         assertEquals(1, HFile.getChecksumFailuresCount());
137 
138         // Since the above encountered a checksum failure, we switch
139         // back to not checking hbase checksums.
140         b = hbr.readBlockData(0, -1, -1, pread);
141         assertEquals(0, HFile.getChecksumFailuresCount());
142         is.close();
143 
144         // Now, use a completely new reader. Switch off hbase checksums in 
145         // the configuration. In this case, we should not detect
146         // any retries within hbase. 
147         HFileSystem newfs = new HFileSystem(TEST_UTIL.getConfiguration(), false);
148         assertEquals(false, newfs.useHBaseChecksum());
149         is = new FSDataInputStreamWrapper(newfs, path);
150         hbr = new FSReaderV2Test(is, algo,
151             totalSize, HFile.MAX_FORMAT_VERSION, newfs, path);
152         b = hbr.readBlockData(0, -1, -1, pread);
153         is.close();
154         b.sanityCheck();
155         assertEquals(4936, b.getUncompressedSizeWithoutHeader());
156         assertEquals(algo == GZ ? 2173 : 4936, 
157                      b.getOnDiskSizeWithoutHeader() - b.totalChecksumBytes());
158         // read data back from the hfile, exclude header and checksum
159         bb = b.getBufferWithoutHeader(); // read back data
160         in = new DataInputStream(new ByteArrayInputStream(
161                                  bb.array(), bb.arrayOffset(), bb.limit()));
162 
163         // assert that we did not encounter hbase checksum verification failures
164         // but still used hdfs checksums and read data successfully.
165         assertEquals(0, HFile.getChecksumFailuresCount());
166         validateData(in);
167       }
168     }
169   }
170 
171   /** 
172    * Test different values of bytesPerChecksum
173    */
174   @Test
175   public void testChecksumChunks() throws IOException {
176     Compression.Algorithm algo = NONE;
177     for (boolean pread : new boolean[] { false, true }) {
178       for (int bytesPerChecksum : BYTES_PER_CHECKSUM) {
179         Path path = new Path(TEST_UTIL.getDataTestDir(), "checksumChunk_" + 
180                              algo + bytesPerChecksum);
181         FSDataOutputStream os = fs.create(path);
182         HFileBlock.Writer hbw = new HFileBlock.Writer(algo, null,
183           true, HFile.DEFAULT_CHECKSUM_TYPE, bytesPerChecksum);
184 
185         // write one block. The block has data
186         // that is at least 6 times more than the checksum chunk size
187         long dataSize = 0;
188         DataOutputStream dos = hbw.startWriting(BlockType.DATA);
189         for (; dataSize < 6 * bytesPerChecksum;) {
190           for (int i = 0; i < 1234; ++i) {
191             dos.writeInt(i);
192             dataSize += 4;
193           }
194         }
195         hbw.writeHeaderAndData(os);
196         long totalSize = hbw.getOnDiskSizeWithHeader();
197         os.close();
198 
199         long expectedChunks = ChecksumUtil.numChunks(
200                                dataSize + HConstants.HFILEBLOCK_HEADER_SIZE,
201                                bytesPerChecksum);
202         LOG.info("testChecksumChunks: pread=" + pread +
203                    ", bytesPerChecksum=" + bytesPerChecksum +
204                    ", fileSize=" + totalSize +
205                    ", dataSize=" + dataSize +
206                    ", expectedChunks=" + expectedChunks);
207 
208         // Verify hbase checksums. 
209         assertEquals(true, hfs.useHBaseChecksum());
210 
211         // Read data back from file.
212         FSDataInputStream is = fs.open(path);
213         FSDataInputStream nochecksum = hfs.getNoChecksumFs().open(path);
214         HFileBlock.FSReader hbr = new HFileBlock.FSReaderV2(new FSDataInputStreamWrapper(
215             is, nochecksum), algo, totalSize, HFile.MAX_FORMAT_VERSION, hfs, path);
216         HFileBlock b = hbr.readBlockData(0, -1, -1, pread);
217         is.close();
218         b.sanityCheck();
219         assertEquals(dataSize, b.getUncompressedSizeWithoutHeader());
220 
221         // verify that we have the expected number of checksum chunks
222         assertEquals(totalSize, HConstants.HFILEBLOCK_HEADER_SIZE + dataSize +
223                      expectedChunks * HFileBlock.CHECKSUM_SIZE);
224 
225         // assert that we did not encounter hbase checksum verification failures
226         assertEquals(0, HFile.getChecksumFailuresCount());
227       }
228     }
229   }
230 
231   /** 
232    * Test to ensure that these is at least one valid checksum implementation
233    */
234   @Test
235   public void testChecksumAlgorithm() throws IOException {
236     ChecksumType type = ChecksumType.CRC32;
237     assertEquals(ChecksumType.nameToType(type.getName()), type);
238     assertEquals(ChecksumType.valueOf(type.toString()), type);
239   }
240 
241   private void validateData(DataInputStream in) throws IOException {
242     // validate data
243     for (int i = 0; i < 1234; i++) {
244       int val = in.readInt();
245       if (val != i) {
246         String msg = "testChecksumCorruption: data mismatch at index " +
247                      i + " expected " + i + " found " + val;
248         LOG.warn(msg);
249         assertEquals(i, val);
250       }
251     }
252   }
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     public FSReaderV2Test(FSDataInputStreamWrapper istream, Algorithm algo, long fileSize,
261         int minorVersion, FileSystem fs,Path path) throws IOException {
262       super(istream, algo, fileSize, minorVersion, (HFileSystem)fs, path);
263     }
264 
265     @Override
266     protected boolean validateBlockChecksum(HFileBlock block, 
267       byte[] data, int hdrSize) throws IOException {
268       return false;  // checksum validation failure
269     }
270   }
271 }
272