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