001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.io.hfile; 019 020import java.io.IOException; 021import java.nio.ByteBuffer; 022import org.apache.hadoop.fs.ChecksumException; 023import org.apache.hadoop.hbase.nio.ByteBuff; 024import org.apache.hadoop.hbase.nio.SingleByteBuff; 025import org.apache.hadoop.hbase.util.ChecksumType; 026import org.apache.hadoop.util.DataChecksum; 027import org.apache.yetus.audience.InterfaceAudience; 028import org.slf4j.Logger; 029import org.slf4j.LoggerFactory; 030 031/** 032 * Utility methods to compute and validate checksums. 033 */ 034@InterfaceAudience.Private 035public class ChecksumUtil { 036 public static final Logger LOG = LoggerFactory.getLogger(ChecksumUtil.class); 037 038 public static final int CHECKSUM_BUF_SIZE = 256; 039 040 /** 041 * This is used by unit tests to make checksum failures throw an exception instead of returning 042 * null. Returning a null value from checksum validation will cause the higher layer to retry that 043 * read with hdfs-level checksums. Instead, we would like checksum failures to cause the entire 044 * unit test to fail. 045 */ 046 private static boolean generateExceptions = false; 047 048 /** 049 * Generates a checksum for all the data in indata. The checksum is written to outdata. 050 * @param indata input data stream 051 * @param startOffset starting offset in the indata stream from where to compute checkums 052 * from 053 * @param endOffset ending offset in the indata stream upto which checksums needs to be 054 * computed 055 * @param outdata the output buffer where checksum values are written 056 * @param outOffset the starting offset in the outdata where the checksum values are 057 * written 058 * @param checksumType type of checksum 059 * @param bytesPerChecksum number of bytes per checksum value 060 */ 061 static void generateChecksums(byte[] indata, int startOffset, int endOffset, byte[] outdata, 062 int outOffset, ChecksumType checksumType, int bytesPerChecksum) throws IOException { 063 064 if (checksumType == ChecksumType.NULL) { 065 return; // No checksum for this block. 066 } 067 068 DataChecksum checksum = 069 DataChecksum.newDataChecksum(checksumType.getDataChecksumType(), bytesPerChecksum); 070 071 checksum.calculateChunkedSums(ByteBuffer.wrap(indata, startOffset, endOffset - startOffset), 072 ByteBuffer.wrap(outdata, outOffset, outdata.length - outOffset)); 073 } 074 075 /** 076 * Like the hadoop's {@link DataChecksum#verifyChunkedSums(ByteBuffer, ByteBuffer, String, long)}, 077 * this method will also verify checksum of each chunk in data. the difference is: this method can 078 * accept {@link ByteBuff} as arguments, we can not add it in hadoop-common so defined here. 079 * @param dataChecksum to calculate the checksum. 080 * @param data as the input 081 * @param checksums to compare 082 * @param pathName indicate that the data is read from which file. 083 * @return a flag indicate the checksum match or mismatch. 084 * @see org.apache.hadoop.util.DataChecksum#verifyChunkedSums(ByteBuffer, ByteBuffer, String, 085 * long) 086 */ 087 private static boolean verifyChunkedSums(DataChecksum dataChecksum, ByteBuff data, 088 ByteBuff checksums, String pathName) { 089 // Almost all of the HFile Block are about 64KB, and it would be a SingleByteBuff, use the 090 // Hadoop's verify checksum directly, because it'll use the native checksum, which has no extra 091 // byte[] allocation or copying. (HBASE-21917) 092 if (data instanceof SingleByteBuff && checksums instanceof SingleByteBuff) { 093 // the checksums ByteBuff must also be an SingleByteBuff because it's duplicated from data. 094 ByteBuffer dataBB = (ByteBuffer) (data.nioByteBuffers()[0]).duplicate() 095 .position(data.position()).limit(data.limit()); 096 ByteBuffer checksumBB = (ByteBuffer) (checksums.nioByteBuffers()[0]).duplicate() 097 .position(checksums.position()).limit(checksums.limit()); 098 try { 099 dataChecksum.verifyChunkedSums(dataBB, checksumBB, pathName, 0); 100 return true; 101 } catch (ChecksumException e) { 102 return false; 103 } 104 } 105 106 // If the block is a MultiByteBuff. we use a small byte[] to update the checksum many times for 107 // reducing GC pressure. it's a rare case. 108 int checksumTypeSize = dataChecksum.getChecksumType().size; 109 if (checksumTypeSize == 0) { 110 return true; 111 } 112 // we have 5 checksum type now: NULL,DEFAULT,MIXED,CRC32,CRC32C. the former three need 0 byte, 113 // and the other two need 4 bytes. 114 assert checksumTypeSize == 4; 115 116 int bytesPerChecksum = dataChecksum.getBytesPerChecksum(); 117 int startDataPos = data.position(); 118 data.mark(); 119 checksums.mark(); 120 try { 121 // allocate an small buffer for reducing young GC (HBASE-21917), and copy 256 bytes from 122 // ByteBuff to update the checksum each time. if we upgrade to an future JDK and hadoop 123 // version which support DataCheckSum#update(ByteBuffer), we won't need to update the checksum 124 // multiple times then. 125 byte[] buf = new byte[CHECKSUM_BUF_SIZE]; 126 byte[] sum = new byte[checksumTypeSize]; 127 while (data.remaining() > 0) { 128 int n = Math.min(data.remaining(), bytesPerChecksum); 129 checksums.get(sum); 130 dataChecksum.reset(); 131 for (int remain = n, len; remain > 0; remain -= len) { 132 // Copy 256 bytes from ByteBuff to update the checksum each time, if the remaining 133 // bytes is less than 256, then just update the remaining bytes. 134 len = Math.min(CHECKSUM_BUF_SIZE, remain); 135 data.get(buf, 0, len); 136 dataChecksum.update(buf, 0, len); 137 } 138 int calculated = (int) dataChecksum.getValue(); 139 int stored = (sum[0] << 24 & 0xff000000) | (sum[1] << 16 & 0xff0000) 140 | (sum[2] << 8 & 0xff00) | (sum[3] & 0xff); 141 if (calculated != stored) { 142 if (LOG.isTraceEnabled()) { 143 long errPos = data.position() - startDataPos - n; 144 LOG.trace("Checksum error: {} at {} expected: {} got: {}", pathName, errPos, stored, 145 calculated); 146 } 147 return false; 148 } 149 } 150 } finally { 151 data.reset(); 152 checksums.reset(); 153 } 154 return true; 155 } 156 157 /** 158 * Validates that the data in the specified HFileBlock matches the checksum. Generates the 159 * checksums for the data and then validate that it matches those stored in the end of the data. 160 * @param buf Contains the data in following order: HFileBlock header, data, checksums. 161 * @param pathName Path of the HFile to which the {@code data} belongs. Only used for logging. 162 * @param offset offset of the data being validated. Only used for logging. 163 * @param hdrSize Size of the block header in {@code data}. Only used for logging. 164 * @return True if checksum matches, else false. 165 */ 166 static boolean validateChecksum(ByteBuff buf, String pathName, long offset, int hdrSize) { 167 ChecksumType ctype = ChecksumType.codeToType(buf.get(HFileBlock.Header.CHECKSUM_TYPE_INDEX)); 168 if (ctype == ChecksumType.NULL) { 169 return true;// No checksum validations needed for this block. 170 } 171 172 // read in the stored value of the checksum size from the header. 173 int bytesPerChecksum = buf.getInt(HFileBlock.Header.BYTES_PER_CHECKSUM_INDEX); 174 DataChecksum dataChecksum = 175 DataChecksum.newDataChecksum(ctype.getDataChecksumType(), bytesPerChecksum); 176 assert dataChecksum != null; 177 int onDiskDataSizeWithHeader = 178 buf.getInt(HFileBlock.Header.ON_DISK_DATA_SIZE_WITH_HEADER_INDEX); 179 LOG.trace( 180 "dataLength={}, sizeWithHeader={}, checksumType={}, file={}, " 181 + "offset={}, headerSize={}, bytesPerChecksum={}", 182 buf.capacity(), onDiskDataSizeWithHeader, ctype.getName(), pathName, offset, hdrSize, 183 bytesPerChecksum); 184 ByteBuff data = buf.duplicate().position(0).limit(onDiskDataSizeWithHeader); 185 ByteBuff checksums = buf.duplicate().position(onDiskDataSizeWithHeader).limit(buf.limit()); 186 return verifyChunkedSums(dataChecksum, data, checksums, pathName); 187 } 188 189 /** 190 * Returns the number of bytes needed to store the checksums for a specified data size 191 * @param datasize number of bytes of data 192 * @param bytesPerChecksum number of bytes in a checksum chunk 193 * @return The number of bytes needed to store the checksum values 194 */ 195 static long numBytes(long datasize, int bytesPerChecksum) { 196 return numChunks(datasize, bytesPerChecksum) * HFileBlock.CHECKSUM_SIZE; 197 } 198 199 /** 200 * Returns the number of checksum chunks needed to store the checksums for a specified data size 201 * @param datasize number of bytes of data 202 * @param bytesPerChecksum number of bytes in a checksum chunk 203 * @return The number of checksum chunks 204 */ 205 static long numChunks(long datasize, int bytesPerChecksum) { 206 long numChunks = datasize / bytesPerChecksum; 207 if (datasize % bytesPerChecksum != 0) { 208 numChunks++; 209 } 210 return numChunks; 211 } 212 213 /** 214 * Mechanism to throw an exception in case of hbase checksum failure. This is used by unit tests 215 * only. 216 * @param value Setting this to true will cause hbase checksum verification failures to generate 217 * exceptions. 218 */ 219 public static void generateExceptionForChecksumFailureForTest(boolean value) { 220 generateExceptions = value; 221 } 222}