View Javadoc

1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.io.hfile;
19  
20  import java.io.ByteArrayInputStream;
21  import java.io.ByteArrayOutputStream;
22  import java.io.DataInputStream;
23  import java.io.DataOutput;
24  import java.io.DataOutputStream;
25  import java.io.IOException;
26  import java.io.InputStream;
27  import java.nio.ByteBuffer;
28  import java.util.concurrent.locks.Lock;
29  import java.util.concurrent.locks.ReentrantLock;
30  
31  import org.apache.hadoop.hbase.classification.InterfaceAudience;
32  import org.apache.hadoop.fs.FSDataInputStream;
33  import org.apache.hadoop.fs.FSDataOutputStream;
34  import org.apache.hadoop.fs.Path;
35  import org.apache.hadoop.hbase.HConstants;
36  import org.apache.hadoop.hbase.fs.HFileSystem;
37  import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
38  import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
39  import org.apache.hadoop.hbase.io.encoding.HFileBlockDecodingContext;
40  import org.apache.hadoop.hbase.io.encoding.HFileBlockDefaultDecodingContext;
41  import org.apache.hadoop.hbase.io.encoding.HFileBlockDefaultEncodingContext;
42  import org.apache.hadoop.hbase.io.encoding.HFileBlockEncodingContext;
43  import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
44  import org.apache.hadoop.hbase.util.Bytes;
45  import org.apache.hadoop.hbase.util.ChecksumType;
46  import org.apache.hadoop.hbase.util.ClassSize;
47  import org.apache.hadoop.hbase.util.CompoundBloomFilter;
48  import org.apache.hadoop.io.IOUtils;
49  
50  import com.google.common.annotations.VisibleForTesting;
51  import com.google.common.base.Preconditions;
52  
53  /**
54   * Reading {@link HFile} version 1 and 2 blocks, and writing version 2 blocks.
55   * <ul>
56   * <li>In version 1 all blocks are always compressed or uncompressed, as
57   * specified by the {@link HFile}'s compression algorithm, with a type-specific
58   * magic record stored in the beginning of the compressed data (i.e. one needs
59   * to uncompress the compressed block to determine the block type). There is
60   * only a single compression algorithm setting for all blocks. Offset and size
61   * information from the block index are required to read a block.
62   * <li>In version 2 a block is structured as follows:
63   * <ul>
64   * <li>header (see {@link Writer#finishBlock()})
65   * <ul>
66   * <li>Magic record identifying the block type (8 bytes)
67   * <li>Compressed block size, excluding header, including checksum (4 bytes)
68   * <li>Uncompressed block size, excluding header, excluding checksum (4 bytes)
69   * <li>The offset of the previous block of the same type (8 bytes). This is
70   * used to be able to navigate to the previous block without going to the block
71   * <li>For minorVersions >=1, the ordinal describing checksum type (1 byte)
72   * <li>For minorVersions >=1, the number of data bytes/checksum chunk (4 bytes)
73   * <li>For minorVersions >=1, the size of data on disk, including header,
74   * excluding checksums (4 bytes)
75   * </ul>
76   * </li>
77   * <li>Raw/Compressed/Encrypted/Encoded data. The compression algorithm is the
78   * same for all the blocks in the {@link HFile}, similarly to what was done in
79   * version 1.
80   * <li>For minorVersions >=1, a series of 4 byte checksums, one each for
81   * the number of bytes specified by bytesPerChecksum.
82   * </ul>
83   * </ul>
84   */
85  @InterfaceAudience.Private
86  public class HFileBlock implements Cacheable {
87  
88    /**
89     * On a checksum failure on a Reader, these many suceeding read
90     * requests switch back to using hdfs checksums before auto-reenabling
91     * hbase checksum verification.
92     */
93    static final int CHECKSUM_VERIFICATION_NUM_IO_THRESHOLD = 3;
94  
95    public static final boolean FILL_HEADER = true;
96    public static final boolean DONT_FILL_HEADER = false;
97  
98    /**
99     * The size of block header when blockType is {@link BlockType#ENCODED_DATA}.
100    * This extends normal header by adding the id of encoder.
101    */
102   public static final int ENCODED_HEADER_SIZE = HConstants.HFILEBLOCK_HEADER_SIZE
103       + DataBlockEncoding.ID_SIZE;
104 
105   static final byte[] DUMMY_HEADER_NO_CHECKSUM =
106      new byte[HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM];
107 
108   public static final int BYTE_BUFFER_HEAP_SIZE = (int) ClassSize.estimateBase(
109       ByteBuffer.wrap(new byte[0], 0, 0).getClass(), false);
110 
111   // meta.usesHBaseChecksum+offset+nextBlockOnDiskSizeWithHeader
112   public static final int EXTRA_SERIALIZATION_SPACE = Bytes.SIZEOF_BYTE + Bytes.SIZEOF_INT
113       + Bytes.SIZEOF_LONG;
114 
115   /**
116    * Each checksum value is an integer that can be stored in 4 bytes.
117    */
118   static final int CHECKSUM_SIZE = Bytes.SIZEOF_INT;
119 
120   private static final CacheableDeserializer<Cacheable> blockDeserializer =
121       new CacheableDeserializer<Cacheable>() {
122         public HFileBlock deserialize(ByteBuffer buf, boolean reuse) throws IOException{
123           buf.limit(buf.limit() - HFileBlock.EXTRA_SERIALIZATION_SPACE).rewind();
124           ByteBuffer newByteBuffer;
125           if (reuse) {
126             newByteBuffer = buf.slice();
127           } else {
128            newByteBuffer = ByteBuffer.allocate(buf.limit());
129            newByteBuffer.put(buf);
130           }
131           buf.position(buf.limit());
132           buf.limit(buf.limit() + HFileBlock.EXTRA_SERIALIZATION_SPACE);
133           boolean usesChecksum = buf.get() == (byte)1;
134           HFileBlock ourBuffer = new HFileBlock(newByteBuffer, usesChecksum);
135           ourBuffer.offset = buf.getLong();
136           ourBuffer.nextBlockOnDiskSizeWithHeader = buf.getInt();
137           if (ourBuffer.hasNextBlockHeader()) {
138             ourBuffer.buf.limit(ourBuffer.buf.limit() - ourBuffer.headerSize());
139           }
140           return ourBuffer;
141         }
142         
143         @Override
144         public int getDeserialiserIdentifier() {
145           return deserializerIdentifier;
146         }
147 
148         @Override
149         public HFileBlock deserialize(ByteBuffer b) throws IOException {
150           return deserialize(b, false);
151         }
152       };
153   private static final int deserializerIdentifier;
154   static {
155     deserializerIdentifier = CacheableDeserializerIdManager
156         .registerDeserializer(blockDeserializer);
157   }
158 
159   /** Type of block. Header field 0. */
160   private BlockType blockType;
161 
162   /** Size on disk excluding header, including checksum. Header field 1. */
163   private int onDiskSizeWithoutHeader;
164 
165   /** Size of pure data. Does not include header or checksums. Header field 2. */
166   private final int uncompressedSizeWithoutHeader;
167 
168   /** The offset of the previous block on disk. Header field 3. */
169   private final long prevBlockOffset;
170 
171   /**
172    * Size on disk of header + data. Excludes checksum. Header field 6,
173    * OR calculated from {@link #onDiskSizeWithoutHeader} when using HDFS checksum.
174    */
175   private final int onDiskDataSizeWithHeader;
176 
177   /** The in-memory representation of the hfile block */
178   private ByteBuffer buf;
179 
180   /** Meta data that holds meta information on the hfileblock */
181   private HFileContext fileContext;
182 
183   /**
184    * The offset of this block in the file. Populated by the reader for
185    * convenience of access. This offset is not part of the block header.
186    */
187   private long offset = -1;
188 
189   /**
190    * The on-disk size of the next block, including the header, obtained by
191    * peeking into the first {@link HConstants#HFILEBLOCK_HEADER_SIZE} bytes of the next block's
192    * header, or -1 if unknown.
193    */
194   private int nextBlockOnDiskSizeWithHeader = -1;
195 
196   /**
197    * Creates a new {@link HFile} block from the given fields. This constructor
198    * is mostly used when the block data has already been read and uncompressed,
199    * and is sitting in a byte buffer. 
200    *
201    * @param blockType the type of this block, see {@link BlockType}
202    * @param onDiskSizeWithoutHeader see {@link #onDiskSizeWithoutHeader}
203    * @param uncompressedSizeWithoutHeader see {@link #uncompressedSizeWithoutHeader}
204    * @param prevBlockOffset see {@link #prevBlockOffset}
205    * @param buf block header ({@link HConstants#HFILEBLOCK_HEADER_SIZE} bytes) followed by
206    *          uncompressed data. This
207    * @param fillHeader when true, parse {@code buf} and override the first 4 header fields.
208    * @param offset the file offset the block was read from
209    * @param onDiskDataSizeWithHeader see {@link #onDiskDataSizeWithHeader}
210    * @param fileContext HFile meta data
211    */
212   HFileBlock(BlockType blockType, int onDiskSizeWithoutHeader, int uncompressedSizeWithoutHeader,
213       long prevBlockOffset, ByteBuffer buf, boolean fillHeader, long offset,
214       int onDiskDataSizeWithHeader, HFileContext fileContext) {
215     this.blockType = blockType;
216     this.onDiskSizeWithoutHeader = onDiskSizeWithoutHeader;
217     this.uncompressedSizeWithoutHeader = uncompressedSizeWithoutHeader;
218     this.prevBlockOffset = prevBlockOffset;
219     this.buf = buf;
220     this.offset = offset;
221     this.onDiskDataSizeWithHeader = onDiskDataSizeWithHeader;
222     this.fileContext = fileContext;
223     if (fillHeader)
224       overwriteHeader();
225     this.buf.rewind();
226   }
227 
228   /**
229    * Copy constructor. Creates a shallow copy of {@code that}'s buffer.
230    */
231   HFileBlock(HFileBlock that) {
232     this.blockType = that.blockType;
233     this.onDiskSizeWithoutHeader = that.onDiskSizeWithoutHeader;
234     this.uncompressedSizeWithoutHeader = that.uncompressedSizeWithoutHeader;
235     this.prevBlockOffset = that.prevBlockOffset;
236     this.buf = that.buf.duplicate();
237     this.offset = that.offset;
238     this.onDiskDataSizeWithHeader = that.onDiskDataSizeWithHeader;
239     this.fileContext = that.fileContext;
240     this.nextBlockOnDiskSizeWithHeader = that.nextBlockOnDiskSizeWithHeader;
241   }
242 
243   /**
244    * Creates a block from an existing buffer starting with a header. Rewinds
245    * and takes ownership of the buffer. By definition of rewind, ignores the
246    * buffer position, but if you slice the buffer beforehand, it will rewind
247    * to that point. The reason this has a minorNumber and not a majorNumber is
248    * because majorNumbers indicate the format of a HFile whereas minorNumbers 
249    * indicate the format inside a HFileBlock.
250    */
251   HFileBlock(ByteBuffer b, boolean usesHBaseChecksum) throws IOException {
252     b.rewind();
253     blockType = BlockType.read(b);
254     onDiskSizeWithoutHeader = b.getInt();
255     uncompressedSizeWithoutHeader = b.getInt();
256     prevBlockOffset = b.getLong();
257     HFileContextBuilder contextBuilder = new HFileContextBuilder();
258     contextBuilder.withHBaseCheckSum(usesHBaseChecksum);
259     if (usesHBaseChecksum) {
260       contextBuilder.withChecksumType(ChecksumType.codeToType(b.get()));
261       contextBuilder.withBytesPerCheckSum(b.getInt());
262       this.onDiskDataSizeWithHeader = b.getInt();
263     } else {
264       contextBuilder.withChecksumType(ChecksumType.NULL);
265       contextBuilder.withBytesPerCheckSum(0);
266       this.onDiskDataSizeWithHeader = onDiskSizeWithoutHeader +
267                                        HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM;
268     }
269     this.fileContext = contextBuilder.build();
270     buf = b;
271     buf.rewind();
272   }
273 
274   public BlockType getBlockType() {
275     return blockType;
276   }
277 
278   /** @return get data block encoding id that was used to encode this block */
279   public short getDataBlockEncodingId() {
280     if (blockType != BlockType.ENCODED_DATA) {
281       throw new IllegalArgumentException("Querying encoder ID of a block " +
282           "of type other than " + BlockType.ENCODED_DATA + ": " + blockType);
283     }
284     return buf.getShort(headerSize());
285   }
286 
287   /**
288    * @return the on-disk size of header + data part + checksum.
289    */
290   public int getOnDiskSizeWithHeader() {
291     return onDiskSizeWithoutHeader + headerSize();
292   }
293 
294   /**
295    * @return the on-disk size of the data part + checksum (header excluded).
296    */
297   public int getOnDiskSizeWithoutHeader() {
298     return onDiskSizeWithoutHeader;
299   }
300 
301   /**
302    * @return the uncompressed size of data part (header and checksum excluded).
303    */
304    public int getUncompressedSizeWithoutHeader() {
305     return uncompressedSizeWithoutHeader;
306   }
307 
308   /**
309    * @return the offset of the previous block of the same type in the file, or
310    *         -1 if unknown
311    */
312   public long getPrevBlockOffset() {
313     return prevBlockOffset;
314   }
315 
316   /**
317    * Rewinds {@code buf} and writes first 4 header fields. {@code buf} position
318    * is modified as side-effect.
319    */
320   private void overwriteHeader() {
321     buf.rewind();
322     blockType.write(buf);
323     buf.putInt(onDiskSizeWithoutHeader);
324     buf.putInt(uncompressedSizeWithoutHeader);
325     buf.putLong(prevBlockOffset);
326     if (this.fileContext.isUseHBaseChecksum()) {
327       buf.put(fileContext.getChecksumType().getCode());
328       buf.putInt(fileContext.getBytesPerChecksum());
329       buf.putInt(onDiskDataSizeWithHeader);
330     }
331   }
332 
333   /**
334    * Returns a buffer that does not include the header or checksum.
335    *
336    * @return the buffer with header skipped and checksum omitted.
337    */
338   public ByteBuffer getBufferWithoutHeader() {
339     return ByteBuffer.wrap(buf.array(), buf.arrayOffset() + headerSize(),
340         buf.limit() - headerSize() - totalChecksumBytes()).slice();
341   }
342 
343   /**
344    * Returns the buffer this block stores internally. The clients must not
345    * modify the buffer object. This method has to be public because it is
346    * used in {@link CompoundBloomFilter} to avoid object creation on every
347    * Bloom filter lookup, but has to be used with caution. Checksum data
348    * is not included in the returned buffer but header data is.
349    *
350    * @return the buffer of this block for read-only operations
351    */
352   public ByteBuffer getBufferReadOnly() {
353     return ByteBuffer.wrap(buf.array(), buf.arrayOffset(),
354         buf.limit() - totalChecksumBytes()).slice();
355   }
356 
357   /**
358    * Returns the buffer of this block, including header data. The clients must
359    * not modify the buffer object. This method has to be public because it is
360    * used in {@link BucketCache} to avoid buffer copy.
361    * 
362    * @return the buffer with header and checksum included for read-only operations
363    */
364   public ByteBuffer getBufferReadOnlyWithHeader() {
365     return ByteBuffer.wrap(buf.array(), buf.arrayOffset(), buf.limit()).slice();
366   }
367 
368   /**
369    * Returns a byte buffer of this block, including header data and checksum, positioned at
370    * the beginning of header. The underlying data array is not copied.
371    *
372    * @return the byte buffer with header and checksum included
373    */
374   ByteBuffer getBufferWithHeader() {
375     ByteBuffer dupBuf = buf.duplicate();
376     dupBuf.rewind();
377     return dupBuf;
378   }
379 
380   private void sanityCheckAssertion(long valueFromBuf, long valueFromField,
381       String fieldName) throws IOException {
382     if (valueFromBuf != valueFromField) {
383       throw new AssertionError(fieldName + " in the buffer (" + valueFromBuf
384           + ") is different from that in the field (" + valueFromField + ")");
385     }
386   }
387 
388   private void sanityCheckAssertion(BlockType valueFromBuf, BlockType valueFromField)
389       throws IOException {
390     if (valueFromBuf != valueFromField) {
391       throw new IOException("Block type stored in the buffer: " +
392         valueFromBuf + ", block type field: " + valueFromField);
393     }
394   }
395 
396   /**
397    * Checks if the block is internally consistent, i.e. the first
398    * {@link HConstants#HFILEBLOCK_HEADER_SIZE} bytes of the buffer contain a
399    * valid header consistent with the fields. Assumes a packed block structure.
400    * This function is primary for testing and debugging, and is not
401    * thread-safe, because it alters the internal buffer pointer.
402    */
403   void sanityCheck() throws IOException {
404     buf.rewind();
405 
406     sanityCheckAssertion(BlockType.read(buf), blockType);
407 
408     sanityCheckAssertion(buf.getInt(), onDiskSizeWithoutHeader,
409         "onDiskSizeWithoutHeader");
410 
411     sanityCheckAssertion(buf.getInt(), uncompressedSizeWithoutHeader,
412         "uncompressedSizeWithoutHeader");
413 
414     sanityCheckAssertion(buf.getLong(), prevBlockOffset, "prevBlocKOffset");
415     if (this.fileContext.isUseHBaseChecksum()) {
416       sanityCheckAssertion(buf.get(), this.fileContext.getChecksumType().getCode(), "checksumType");
417       sanityCheckAssertion(buf.getInt(), this.fileContext.getBytesPerChecksum(), "bytesPerChecksum");
418       sanityCheckAssertion(buf.getInt(), onDiskDataSizeWithHeader, "onDiskDataSizeWithHeader");
419     }
420 
421     int cksumBytes = totalChecksumBytes();
422     int expectedBufLimit = onDiskDataSizeWithHeader + cksumBytes;
423     if (buf.limit() != expectedBufLimit) {
424       throw new AssertionError("Expected buffer limit " + expectedBufLimit
425           + ", got " + buf.limit());
426     }
427 
428     // We might optionally allocate HFILEBLOCK_HEADER_SIZE more bytes to read the next
429     // block's header, so there are two sensible values for buffer capacity.
430     int hdrSize = headerSize();
431     if (buf.capacity() != expectedBufLimit &&
432         buf.capacity() != expectedBufLimit + hdrSize) {
433       throw new AssertionError("Invalid buffer capacity: " + buf.capacity() +
434           ", expected " + expectedBufLimit + " or " + (expectedBufLimit + hdrSize));
435     }
436   }
437 
438   @Override
439   public String toString() {
440     StringBuilder sb = new StringBuilder()
441       .append("HFileBlock [")
442       .append(" fileOffset=").append(offset)
443       .append(" headerSize()=").append(headerSize())
444       .append(" blockType=").append(blockType)
445       .append(" onDiskSizeWithoutHeader=").append(onDiskSizeWithoutHeader)
446       .append(" uncompressedSizeWithoutHeader=").append(uncompressedSizeWithoutHeader)
447       .append(" prevBlockOffset=").append(prevBlockOffset)
448       .append(" isUseHBaseChecksum()=").append(fileContext.isUseHBaseChecksum());
449     if (fileContext.isUseHBaseChecksum()) {
450       sb.append(" checksumType=").append(ChecksumType.codeToType(this.buf.get(24)))
451         .append(" bytesPerChecksum=").append(this.buf.getInt(24 + 1))
452         .append(" onDiskDataSizeWithHeader=").append(onDiskDataSizeWithHeader);
453     } else {
454       sb.append(" onDiskDataSizeWithHeader=").append(onDiskDataSizeWithHeader)
455         .append("(").append(onDiskSizeWithoutHeader)
456         .append("+").append(HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM).append(")");
457     }
458     sb.append(" getOnDiskSizeWithHeader()=").append(getOnDiskSizeWithHeader())
459       .append(" totalChecksumBytes()=").append(totalChecksumBytes())
460       .append(" isUnpacked()=").append(isUnpacked())
461       .append(" buf=[ ")
462         .append(buf)
463         .append(", array().length=").append(buf.array().length)
464         .append(", arrayOffset()=").append(buf.arrayOffset())
465       .append(" ]")
466       .append(" dataBeginsWith=")
467       .append(Bytes.toStringBinary(buf.array(), buf.arrayOffset() + headerSize(),
468         Math.min(32, buf.limit() - buf.arrayOffset() - headerSize())))
469       .append(" fileContext=").append(fileContext)
470       .append(" ]");
471     return sb.toString();
472   }
473 
474   /**
475    * Called after reading a block with provided onDiskSizeWithHeader.
476    */
477   private void validateOnDiskSizeWithoutHeader(
478       int expectedOnDiskSizeWithoutHeader) throws IOException {
479     if (onDiskSizeWithoutHeader != expectedOnDiskSizeWithoutHeader) {
480       String blockInfoMsg =
481         "Block offset: " + offset + ", data starts with: "
482           + Bytes.toStringBinary(buf.array(), buf.arrayOffset(),
483               buf.arrayOffset() + Math.min(32, buf.limit()));
484       throw new IOException("On-disk size without header provided is "
485           + expectedOnDiskSizeWithoutHeader + ", but block "
486           + "header contains " + onDiskSizeWithoutHeader + ". " +
487           blockInfoMsg);
488     }
489   }
490 
491   /**
492    * Retrieves the decompressed/decrypted view of this block. An encoded block remains in its
493    * encoded structure. Internal structures are shared between instances where applicable.
494    */
495   HFileBlock unpack(HFileContext fileContext, FSReader reader) throws IOException {
496     if (!fileContext.isCompressedOrEncrypted()) {
497       // TODO: cannot use our own fileContext here because HFileBlock(ByteBuffer, boolean),
498       // which is used for block serialization to L2 cache, does not preserve encoding and
499       // encryption details.
500       return this;
501     }
502 
503     HFileBlock unpacked = new HFileBlock(this);
504     unpacked.allocateBuffer(); // allocates space for the decompressed block
505 
506     HFileBlockDecodingContext ctx = blockType == BlockType.ENCODED_DATA ?
507       reader.getBlockDecodingContext() : reader.getDefaultBlockDecodingContext();
508     ctx.prepareDecoding(unpacked.getOnDiskSizeWithoutHeader(),
509       unpacked.getUncompressedSizeWithoutHeader(), unpacked.getBufferWithoutHeader(),
510       this.getBufferReadOnlyWithHeader().array(), this.headerSize());
511 
512     // Preserve the next block's header bytes in the new block if we have them.
513     if (unpacked.hasNextBlockHeader()) {
514       System.arraycopy(this.buf.array(), this.buf.arrayOffset() + this.onDiskDataSizeWithHeader,
515         unpacked.buf.array(), unpacked.buf.arrayOffset() + unpacked.headerSize() +
516           unpacked.uncompressedSizeWithoutHeader + unpacked.totalChecksumBytes(),
517         unpacked.headerSize());
518     }
519     return unpacked;
520   }
521 
522   /**
523    * Return true when this buffer includes next block's header.
524    */
525   private boolean hasNextBlockHeader() {
526     return nextBlockOnDiskSizeWithHeader > 0;
527   }
528 
529   /**
530    * Always allocates a new buffer of the correct size. Copies header bytes
531    * from the existing buffer. Does not change header fields. 
532    * Reserve room to keep checksum bytes too.
533    */
534   private void allocateBuffer() {
535     int cksumBytes = totalChecksumBytes();
536     int headerSize = headerSize();
537     int capacityNeeded = headerSize + uncompressedSizeWithoutHeader +
538         cksumBytes + (hasNextBlockHeader() ? headerSize : 0);
539 
540     ByteBuffer newBuf = ByteBuffer.allocate(capacityNeeded);
541 
542     // Copy header bytes.
543     System.arraycopy(buf.array(), buf.arrayOffset(), newBuf.array(),
544         newBuf.arrayOffset(), headerSize);
545 
546     buf = newBuf;
547     // set limit to exclude next block's header
548     buf.limit(headerSize + uncompressedSizeWithoutHeader + cksumBytes);
549   }
550 
551   /**
552    * Return true when this block's buffer has been unpacked, false otherwise. Note this is a
553    * calculated heuristic, not tracked attribute of the block.
554    */
555   public boolean isUnpacked() {
556     final int cksumBytes = totalChecksumBytes();
557     final int headerSize = headerSize();
558     final int expectedCapacity = headerSize + uncompressedSizeWithoutHeader + cksumBytes;
559     final int bufCapacity = buf.capacity();
560     return bufCapacity == expectedCapacity || bufCapacity == expectedCapacity + headerSize;
561   }
562 
563   /** An additional sanity-check in case no compression or encryption is being used. */
564   public void assumeUncompressed() throws IOException {
565     if (onDiskSizeWithoutHeader != uncompressedSizeWithoutHeader +
566         totalChecksumBytes()) {
567       throw new IOException("Using no compression but "
568           + "onDiskSizeWithoutHeader=" + onDiskSizeWithoutHeader + ", "
569           + "uncompressedSizeWithoutHeader=" + uncompressedSizeWithoutHeader
570           + ", numChecksumbytes=" + totalChecksumBytes());
571     }
572   }
573 
574   /**
575    * @param expectedType the expected type of this block
576    * @throws IOException if this block's type is different than expected
577    */
578   public void expectType(BlockType expectedType) throws IOException {
579     if (blockType != expectedType) {
580       throw new IOException("Invalid block type: expected=" + expectedType
581           + ", actual=" + blockType);
582     }
583   }
584 
585   /** @return the offset of this block in the file it was read from */
586   public long getOffset() {
587     if (offset < 0) {
588       throw new IllegalStateException(
589           "HFile block offset not initialized properly");
590     }
591     return offset;
592   }
593 
594   /**
595    * @return a byte stream reading the data + checksum of this block
596    */
597   public DataInputStream getByteStream() {
598     return new DataInputStream(new ByteArrayInputStream(buf.array(),
599         buf.arrayOffset() + headerSize(), buf.limit() - headerSize()));
600   }
601 
602   @Override
603   public long heapSize() {
604     long size = ClassSize.align(
605         ClassSize.OBJECT +
606         // Block type, byte buffer and meta references
607         3 * ClassSize.REFERENCE +
608         // On-disk size, uncompressed size, and next block's on-disk size
609         // bytePerChecksum and onDiskDataSize
610         4 * Bytes.SIZEOF_INT +
611         // This and previous block offset
612         2 * Bytes.SIZEOF_LONG +
613         // Heap size of the meta object. meta will be always not null.
614         fileContext.heapSize()
615     );
616 
617     if (buf != null) {
618       // Deep overhead of the byte buffer. Needs to be aligned separately.
619       size += ClassSize.align(buf.capacity() + BYTE_BUFFER_HEAP_SIZE);
620     }
621 
622     return ClassSize.align(size);
623   }
624 
625   /**
626    * Read from an input stream. Analogous to
627    * {@link IOUtils#readFully(InputStream, byte[], int, int)}, but specifies a
628    * number of "extra" bytes that would be desirable but not absolutely
629    * necessary to read.
630    *
631    * @param in the input stream to read from
632    * @param buf the buffer to read into
633    * @param bufOffset the destination offset in the buffer
634    * @param necessaryLen the number of bytes that are absolutely necessary to
635    *          read
636    * @param extraLen the number of extra bytes that would be nice to read
637    * @return true if succeeded reading the extra bytes
638    * @throws IOException if failed to read the necessary bytes
639    */
640   public static boolean readWithExtra(InputStream in, byte buf[],
641       int bufOffset, int necessaryLen, int extraLen) throws IOException {
642     int bytesRemaining = necessaryLen + extraLen;
643     while (bytesRemaining > 0) {
644       int ret = in.read(buf, bufOffset, bytesRemaining);
645       if (ret == -1 && bytesRemaining <= extraLen) {
646         // We could not read the "extra data", but that is OK.
647         break;
648       }
649 
650       if (ret < 0) {
651         throw new IOException("Premature EOF from inputStream (read "
652             + "returned " + ret + ", was trying to read " + necessaryLen
653             + " necessary bytes and " + extraLen + " extra bytes, "
654             + "successfully read "
655             + (necessaryLen + extraLen - bytesRemaining));
656       }
657       bufOffset += ret;
658       bytesRemaining -= ret;
659     }
660     return bytesRemaining <= 0;
661   }
662 
663   /**
664    * Read from an input stream. Analogous to
665    * {@link IOUtils#readFully(InputStream, byte[], int, int)}, but uses
666    * positional read and specifies a number of "extra" bytes that would be
667    * desirable but not absolutely necessary to read.
668    *
669    * @param in the input stream to read from
670    * @param position the position within the stream from which to start reading
671    * @param buf the buffer to read into
672    * @param bufOffset the destination offset in the buffer
673    * @param necessaryLen the number of bytes that are absolutely necessary to
674    *     read
675    * @param extraLen the number of extra bytes that would be nice to read
676    * @return true if and only if extraLen is > 0 and reading those extra bytes
677    *     was successful
678    * @throws IOException if failed to read the necessary bytes
679    */
680   @VisibleForTesting
681   static boolean positionalReadWithExtra(FSDataInputStream in,
682       long position, byte[] buf, int bufOffset, int necessaryLen, int extraLen)
683       throws IOException {
684     int bytesRemaining = necessaryLen + extraLen;
685     int bytesRead = 0;
686     while (bytesRead < necessaryLen) {
687       int ret = in.read(position, buf, bufOffset, bytesRemaining);
688       if (ret < 0) {
689         throw new IOException("Premature EOF from inputStream (positional read "
690             + "returned " + ret + ", was trying to read " + necessaryLen
691             + " necessary bytes and " + extraLen + " extra bytes, "
692             + "successfully read " + bytesRead);
693       }
694       position += ret;
695       bufOffset += ret;
696       bytesRemaining -= ret;
697       bytesRead += ret;
698     }
699     return bytesRead != necessaryLen && bytesRemaining <= 0;
700   }
701 
702   /**
703    * @return the on-disk size of the next block (including the header size)
704    *         that was read by peeking into the next block's header
705    */
706   public int getNextBlockOnDiskSizeWithHeader() {
707     return nextBlockOnDiskSizeWithHeader;
708   }
709 
710   /**
711    * Unified version 2 {@link HFile} block writer. The intended usage pattern
712    * is as follows:
713    * <ol>
714    * <li>Construct an {@link HFileBlock.Writer}, providing a compression algorithm.
715    * <li>Call {@link Writer#startWriting} and get a data stream to write to.
716    * <li>Write your data into the stream.
717    * <li>Call {@link Writer#writeHeaderAndData(FSDataOutputStream)} as many times as you need to.
718    * store the serialized block into an external stream.
719    * <li>Repeat to write more blocks.
720    * </ol>
721    * <p>
722    */
723   public static class Writer {
724 
725     private enum State {
726       INIT,
727       WRITING,
728       BLOCK_READY
729     };
730 
731     /** Writer state. Used to ensure the correct usage protocol. */
732     private State state = State.INIT;
733 
734     /** Data block encoder used for data blocks */
735     private final HFileDataBlockEncoder dataBlockEncoder;
736 
737     private HFileBlockEncodingContext dataBlockEncodingCtx;
738 
739     /** block encoding context for non-data blocks */
740     private HFileBlockDefaultEncodingContext defaultBlockEncodingCtx;
741 
742     /**
743      * The stream we use to accumulate data in uncompressed format for each
744      * block. We reset this stream at the end of each block and reuse it. The
745      * header is written as the first {@link HConstants#HFILEBLOCK_HEADER_SIZE} bytes into this
746      * stream.
747      */
748     private ByteArrayOutputStream baosInMemory;
749 
750     /**
751      * Current block type. Set in {@link #startWriting(BlockType)}. Could be
752      * changed in {@link #finishBlock()} from {@link BlockType#DATA}
753      * to {@link BlockType#ENCODED_DATA}.
754      */
755     private BlockType blockType;
756 
757     /**
758      * A stream that we write uncompressed bytes to, which compresses them and
759      * writes them to {@link #baosInMemory}.
760      */
761     private DataOutputStream userDataStream;
762 
763     /**
764      * Bytes to be written to the file system, including the header. Compressed
765      * if compression is turned on. It also includes the checksum data that
766      * immediately follows the block data. (header + data + checksums)
767      */
768     private byte[] onDiskBytesWithHeader;
769 
770     /**
771      * The size of the checksum data on disk. It is used only if data is
772      * not compressed. If data is compressed, then the checksums are already
773      * part of onDiskBytesWithHeader. If data is uncompressed, then this
774      * variable stores the checksum data for this block.
775      */
776     private byte[] onDiskChecksum;
777 
778     /**
779      * Valid in the READY state. Contains the header and the uncompressed (but
780      * potentially encoded, if this is a data block) bytes, so the length is
781      * {@link #uncompressedSizeWithoutHeader} + {@link org.apache.hadoop.hbase.HConstants#HFILEBLOCK_HEADER_SIZE}.
782      * Does not store checksums.
783      */
784     private byte[] uncompressedBytesWithHeader;
785 
786     /**
787      * Current block's start offset in the {@link HFile}. Set in
788      * {@link #writeHeaderAndData(FSDataOutputStream)}.
789      */
790     private long startOffset;
791 
792     /**
793      * Offset of previous block by block type. Updated when the next block is
794      * started.
795      */
796     private long[] prevOffsetByType;
797 
798     /** The offset of the previous block of the same type */
799     private long prevOffset;
800     /** Meta data that holds information about the hfileblock**/
801     private HFileContext fileContext;
802 
803     /**
804      * @param dataBlockEncoder data block encoding algorithm to use
805      */
806     public Writer(HFileDataBlockEncoder dataBlockEncoder, HFileContext fileContext) {
807       this.dataBlockEncoder = dataBlockEncoder != null
808           ? dataBlockEncoder : NoOpDataBlockEncoder.INSTANCE;
809       defaultBlockEncodingCtx = new HFileBlockDefaultEncodingContext(null,
810           HConstants.HFILEBLOCK_DUMMY_HEADER, fileContext);
811       dataBlockEncodingCtx = this.dataBlockEncoder
812           .newDataBlockEncodingContext(HConstants.HFILEBLOCK_DUMMY_HEADER, fileContext);
813 
814       if (fileContext.getBytesPerChecksum() < HConstants.HFILEBLOCK_HEADER_SIZE) {
815         throw new RuntimeException("Unsupported value of bytesPerChecksum. " +
816             " Minimum is " + HConstants.HFILEBLOCK_HEADER_SIZE + " but the configured value is " +
817             fileContext.getBytesPerChecksum());
818       }
819 
820       baosInMemory = new ByteArrayOutputStream();
821       
822       prevOffsetByType = new long[BlockType.values().length];
823       for (int i = 0; i < prevOffsetByType.length; ++i)
824         prevOffsetByType[i] = -1;
825 
826       this.fileContext = fileContext;
827     }
828 
829     /**
830      * Starts writing into the block. The previous block's data is discarded.
831      *
832      * @return the stream the user can write their data into
833      * @throws IOException
834      */
835     public DataOutputStream startWriting(BlockType newBlockType)
836         throws IOException {
837       if (state == State.BLOCK_READY && startOffset != -1) {
838         // We had a previous block that was written to a stream at a specific
839         // offset. Save that offset as the last offset of a block of that type.
840         prevOffsetByType[blockType.getId()] = startOffset;
841       }
842 
843       startOffset = -1;
844       blockType = newBlockType;
845 
846       baosInMemory.reset();
847       baosInMemory.write(HConstants.HFILEBLOCK_DUMMY_HEADER);
848 
849       state = State.WRITING;
850 
851       // We will compress it later in finishBlock()
852       userDataStream = new DataOutputStream(baosInMemory);
853       return userDataStream;
854     }
855 
856     /**
857      * Returns the stream for the user to write to. The block writer takes care
858      * of handling compression and buffering for caching on write. Can only be
859      * called in the "writing" state.
860      *
861      * @return the data output stream for the user to write to
862      */
863     DataOutputStream getUserDataStream() {
864       expectState(State.WRITING);
865       return userDataStream;
866     }
867 
868     /**
869      * Transitions the block writer from the "writing" state to the "block
870      * ready" state.  Does nothing if a block is already finished.
871      */
872     private void ensureBlockReady() throws IOException {
873       Preconditions.checkState(state != State.INIT,
874           "Unexpected state: " + state);
875 
876       if (state == State.BLOCK_READY)
877         return;
878 
879       // This will set state to BLOCK_READY.
880       finishBlock();
881     }
882 
883     /**
884      * An internal method that flushes the compressing stream (if using
885      * compression), serializes the header, and takes care of the separate
886      * uncompressed stream for caching on write, if applicable. Sets block
887      * write state to "block ready".
888      */
889     private void finishBlock() throws IOException {
890       userDataStream.flush();
891       // This does an array copy, so it is safe to cache this byte array.
892       uncompressedBytesWithHeader = baosInMemory.toByteArray();
893       prevOffset = prevOffsetByType[blockType.getId()];
894 
895       // We need to set state before we can package the block up for
896       // cache-on-write. In a way, the block is ready, but not yet encoded or
897       // compressed.
898       state = State.BLOCK_READY;
899       if (blockType == BlockType.DATA) {
900         encodeDataBlockForDisk();
901       } else {
902         defaultBlockEncodingCtx.compressAfterEncodingWithBlockType(
903             uncompressedBytesWithHeader, blockType);
904         onDiskBytesWithHeader =
905           defaultBlockEncodingCtx.getOnDiskBytesWithHeader();
906       }
907 
908       int numBytes = (int) ChecksumUtil.numBytes(
909           onDiskBytesWithHeader.length,
910           fileContext.getBytesPerChecksum());
911 
912       // put the header for on disk bytes
913       putHeader(onDiskBytesWithHeader, 0,
914           onDiskBytesWithHeader.length + numBytes,
915           uncompressedBytesWithHeader.length, onDiskBytesWithHeader.length);
916       // set the header for the uncompressed bytes (for cache-on-write)
917       putHeader(uncompressedBytesWithHeader, 0,
918           onDiskBytesWithHeader.length + numBytes,
919           uncompressedBytesWithHeader.length, onDiskBytesWithHeader.length);
920 
921       onDiskChecksum = new byte[numBytes];
922       ChecksumUtil.generateChecksums(
923           onDiskBytesWithHeader, 0, onDiskBytesWithHeader.length,
924           onDiskChecksum, 0, fileContext.getChecksumType(), fileContext.getBytesPerChecksum());
925     }
926 
927     /**
928      * Encodes this block if it is a data block and encoding is turned on in
929      * {@link #dataBlockEncoder}.
930      */
931     private void encodeDataBlockForDisk() throws IOException {
932       // do data block encoding, if data block encoder is set
933       ByteBuffer rawKeyValues =
934           ByteBuffer.wrap(uncompressedBytesWithHeader, HConstants.HFILEBLOCK_HEADER_SIZE,
935               uncompressedBytesWithHeader.length - HConstants.HFILEBLOCK_HEADER_SIZE).slice();
936 
937       // do the encoding
938       dataBlockEncoder.beforeWriteToDisk(rawKeyValues, dataBlockEncodingCtx, blockType);
939 
940       uncompressedBytesWithHeader =
941           dataBlockEncodingCtx.getUncompressedBytesWithHeader();
942       onDiskBytesWithHeader =
943           dataBlockEncodingCtx.getOnDiskBytesWithHeader();
944       blockType = dataBlockEncodingCtx.getBlockType();
945     }
946 
947     /**
948      * Put the header into the given byte array at the given offset.
949      * @param onDiskSize size of the block on disk header + data + checksum
950      * @param uncompressedSize size of the block after decompression (but
951      *          before optional data block decoding) including header
952      * @param onDiskDataSize size of the block on disk with header
953      *        and data but not including the checksums
954      */
955     private void putHeader(byte[] dest, int offset, int onDiskSize,
956         int uncompressedSize, int onDiskDataSize) {
957       offset = blockType.put(dest, offset);
958       offset = Bytes.putInt(dest, offset, onDiskSize - HConstants.HFILEBLOCK_HEADER_SIZE);
959       offset = Bytes.putInt(dest, offset, uncompressedSize - HConstants.HFILEBLOCK_HEADER_SIZE);
960       offset = Bytes.putLong(dest, offset, prevOffset);
961       offset = Bytes.putByte(dest, offset, fileContext.getChecksumType().getCode());
962       offset = Bytes.putInt(dest, offset, fileContext.getBytesPerChecksum());
963       Bytes.putInt(dest, offset, onDiskDataSize);
964     }
965 
966     /**
967      * Similar to {@link #writeHeaderAndData(FSDataOutputStream)}, but records
968      * the offset of this block so that it can be referenced in the next block
969      * of the same type.
970      *
971      * @param out
972      * @throws IOException
973      */
974     public void writeHeaderAndData(FSDataOutputStream out) throws IOException {
975       long offset = out.getPos();
976       if (startOffset != -1 && offset != startOffset) {
977         throw new IOException("A " + blockType + " block written to a "
978             + "stream twice, first at offset " + startOffset + ", then at "
979             + offset);
980       }
981       startOffset = offset;
982 
983       finishBlockAndWriteHeaderAndData((DataOutputStream) out);
984     }
985 
986     /**
987      * Writes the header and the compressed data of this block (or uncompressed
988      * data when not using compression) into the given stream. Can be called in
989      * the "writing" state or in the "block ready" state. If called in the
990      * "writing" state, transitions the writer to the "block ready" state.
991      *
992      * @param out the output stream to write the
993      * @throws IOException
994      */
995     private void finishBlockAndWriteHeaderAndData(DataOutputStream out)
996       throws IOException {
997       ensureBlockReady();
998       out.write(onDiskBytesWithHeader);
999       out.write(onDiskChecksum);
1000     }
1001 
1002     /**
1003      * Returns the header or the compressed data (or uncompressed data when not
1004      * using compression) as a byte array. Can be called in the "writing" state
1005      * or in the "block ready" state. If called in the "writing" state,
1006      * transitions the writer to the "block ready" state. This returns
1007      * the header + data + checksums stored on disk.
1008      *
1009      * @return header and data as they would be stored on disk in a byte array
1010      * @throws IOException
1011      */
1012     byte[] getHeaderAndDataForTest() throws IOException {
1013       ensureBlockReady();
1014       // This is not very optimal, because we are doing an extra copy.
1015       // But this method is used only by unit tests.
1016       byte[] output =
1017           new byte[onDiskBytesWithHeader.length
1018               + onDiskChecksum.length];
1019       System.arraycopy(onDiskBytesWithHeader, 0, output, 0,
1020           onDiskBytesWithHeader.length);
1021       System.arraycopy(onDiskChecksum, 0, output,
1022           onDiskBytesWithHeader.length, onDiskChecksum.length);
1023       return output;
1024     }
1025 
1026     /**
1027      * Releases resources used by this writer.
1028      */
1029     public void release() {
1030       if (dataBlockEncodingCtx != null) {
1031         dataBlockEncodingCtx.close();
1032         dataBlockEncodingCtx = null;
1033       }
1034       if (defaultBlockEncodingCtx != null) {
1035         defaultBlockEncodingCtx.close();
1036         defaultBlockEncodingCtx = null;
1037       }
1038     }
1039 
1040     /**
1041      * Returns the on-disk size of the data portion of the block. This is the
1042      * compressed size if compression is enabled. Can only be called in the
1043      * "block ready" state. Header is not compressed, and its size is not
1044      * included in the return value.
1045      *
1046      * @return the on-disk size of the block, not including the header.
1047      */
1048     int getOnDiskSizeWithoutHeader() {
1049       expectState(State.BLOCK_READY);
1050       return onDiskBytesWithHeader.length + onDiskChecksum.length - HConstants.HFILEBLOCK_HEADER_SIZE;
1051     }
1052 
1053     /**
1054      * Returns the on-disk size of the block. Can only be called in the
1055      * "block ready" state.
1056      *
1057      * @return the on-disk size of the block ready to be written, including the
1058      *         header size, the data and the checksum data.
1059      */
1060     int getOnDiskSizeWithHeader() {
1061       expectState(State.BLOCK_READY);
1062       return onDiskBytesWithHeader.length + onDiskChecksum.length;
1063     }
1064 
1065     /**
1066      * The uncompressed size of the block data. Does not include header size.
1067      */
1068     int getUncompressedSizeWithoutHeader() {
1069       expectState(State.BLOCK_READY);
1070       return uncompressedBytesWithHeader.length - HConstants.HFILEBLOCK_HEADER_SIZE;
1071     }
1072 
1073     /**
1074      * The uncompressed size of the block data, including header size.
1075      */
1076     int getUncompressedSizeWithHeader() {
1077       expectState(State.BLOCK_READY);
1078       return uncompressedBytesWithHeader.length;
1079     }
1080 
1081     /** @return true if a block is being written  */
1082     public boolean isWriting() {
1083       return state == State.WRITING;
1084     }
1085 
1086     /**
1087      * Returns the number of bytes written into the current block so far, or
1088      * zero if not writing the block at the moment. Note that this will return
1089      * zero in the "block ready" state as well.
1090      *
1091      * @return the number of bytes written
1092      */
1093     public int blockSizeWritten() {
1094       if (state != State.WRITING)
1095         return 0;
1096       return userDataStream.size();
1097     }
1098 
1099     /**
1100      * Returns the header followed by the uncompressed data, even if using
1101      * compression. This is needed for storing uncompressed blocks in the block
1102      * cache. Can be called in the "writing" state or the "block ready" state.
1103      * Returns only the header and data, does not include checksum data.
1104      *
1105      * @return uncompressed block bytes for caching on write
1106      */
1107     ByteBuffer getUncompressedBufferWithHeader() {
1108       expectState(State.BLOCK_READY);
1109       return ByteBuffer.wrap(uncompressedBytesWithHeader);
1110     }
1111 
1112     /**
1113      * Returns the header followed by the on-disk (compressed/encoded/encrypted) data. This is
1114      * needed for storing packed blocks in the block cache. Expects calling semantics identical to
1115      * {@link #getUncompressedBufferWithHeader()}. Returns only the header and data,
1116      * Does not include checksum data.
1117      *
1118      * @return packed block bytes for caching on write
1119      */
1120     ByteBuffer getOnDiskBufferWithHeader() {
1121       expectState(State.BLOCK_READY);
1122       return ByteBuffer.wrap(onDiskBytesWithHeader);
1123     }
1124 
1125     private void expectState(State expectedState) {
1126       if (state != expectedState) {
1127         throw new IllegalStateException("Expected state: " + expectedState +
1128             ", actual state: " + state);
1129       }
1130     }
1131 
1132     /**
1133      * Takes the given {@link BlockWritable} instance, creates a new block of
1134      * its appropriate type, writes the writable into this block, and flushes
1135      * the block into the output stream. The writer is instructed not to buffer
1136      * uncompressed bytes for cache-on-write.
1137      *
1138      * @param bw the block-writable object to write as a block
1139      * @param out the file system output stream
1140      * @throws IOException
1141      */
1142     public void writeBlock(BlockWritable bw, FSDataOutputStream out)
1143         throws IOException {
1144       bw.writeToBlock(startWriting(bw.getBlockType()));
1145       writeHeaderAndData(out);
1146     }
1147 
1148     /**
1149      * Creates a new HFileBlock. Checksums have already been validated, so
1150      * the byte buffer passed into the constructor of this newly created
1151      * block does not have checksum data even though the header minor 
1152      * version is MINOR_VERSION_WITH_CHECKSUM. This is indicated by setting a
1153      * 0 value in bytesPerChecksum.
1154      */
1155     public HFileBlock getBlockForCaching(CacheConfig cacheConf) {
1156       HFileContext newContext = new HFileContextBuilder()
1157                                 .withBlockSize(fileContext.getBlocksize())
1158                                 .withBytesPerCheckSum(0)
1159                                 .withChecksumType(ChecksumType.NULL) // no checksums in cached data
1160                                 .withCompression(fileContext.getCompression())
1161                                 .withDataBlockEncoding(fileContext.getDataBlockEncoding())
1162                                 .withHBaseCheckSum(fileContext.isUseHBaseChecksum())
1163                                 .withCompressTags(fileContext.isCompressTags())
1164                                 .withIncludesMvcc(fileContext.isIncludesMvcc())
1165                                 .withIncludesTags(fileContext.isIncludesTags())
1166                                 .build();
1167       return new HFileBlock(blockType, getOnDiskSizeWithoutHeader(),
1168           getUncompressedSizeWithoutHeader(), prevOffset,
1169           cacheConf.shouldCacheCompressed(blockType.getCategory()) ?
1170             getOnDiskBufferWithHeader() :
1171             getUncompressedBufferWithHeader(),
1172           FILL_HEADER, startOffset,
1173           onDiskBytesWithHeader.length + onDiskChecksum.length, newContext);
1174     }
1175   }
1176 
1177   /** Something that can be written into a block. */
1178   public interface BlockWritable {
1179 
1180     /** The type of block this data should use. */
1181     BlockType getBlockType();
1182 
1183     /**
1184      * Writes the block to the provided stream. Must not write any magic
1185      * records.
1186      *
1187      * @param out a stream to write uncompressed data into
1188      */
1189     void writeToBlock(DataOutput out) throws IOException;
1190   }
1191 
1192   // Block readers and writers
1193 
1194   /** An interface allowing to iterate {@link HFileBlock}s. */
1195   public interface BlockIterator {
1196 
1197     /**
1198      * Get the next block, or null if there are no more blocks to iterate.
1199      */
1200     HFileBlock nextBlock() throws IOException;
1201 
1202     /**
1203      * Similar to {@link #nextBlock()} but checks block type, throws an
1204      * exception if incorrect, and returns the HFile block
1205      */
1206     HFileBlock nextBlockWithBlockType(BlockType blockType) throws IOException;
1207   }
1208 
1209   /** A full-fledged reader with iteration ability. */
1210   public interface FSReader {
1211 
1212     /**
1213      * Reads the block at the given offset in the file with the given on-disk
1214      * size and uncompressed size.
1215      *
1216      * @param offset
1217      * @param onDiskSize the on-disk size of the entire block, including all
1218      *          applicable headers, or -1 if unknown
1219      * @param uncompressedSize the uncompressed size of the compressed part of
1220      *          the block, or -1 if unknown
1221      * @return the newly read block
1222      */
1223     HFileBlock readBlockData(long offset, long onDiskSize,
1224         int uncompressedSize, boolean pread) throws IOException;
1225 
1226     /**
1227      * Creates a block iterator over the given portion of the {@link HFile}.
1228      * The iterator returns blocks starting with offset such that offset <=
1229      * startOffset < endOffset. Returned blocks are always unpacked.
1230      *
1231      * @param startOffset the offset of the block to start iteration with
1232      * @param endOffset the offset to end iteration at (exclusive)
1233      * @return an iterator of blocks between the two given offsets
1234      */
1235     BlockIterator blockRange(long startOffset, long endOffset);
1236 
1237     /** Closes the backing streams */
1238     void closeStreams() throws IOException;
1239 
1240     /** Get a decoder for {@link BlockType#ENCODED_DATA} blocks from this file. */
1241     HFileBlockDecodingContext getBlockDecodingContext();
1242 
1243     /** Get the default decoder for blocks from this file. */
1244     HFileBlockDecodingContext getDefaultBlockDecodingContext();
1245   }
1246 
1247   /**
1248    * A common implementation of some methods of {@link FSReader} and some
1249    * tools for implementing HFile format version-specific block readers.
1250    */
1251   private abstract static class AbstractFSReader implements FSReader {
1252     /** Compression algorithm used by the {@link HFile} */
1253 
1254     /** The size of the file we are reading from, or -1 if unknown. */
1255     protected long fileSize;
1256 
1257     /** The size of the header */
1258     protected final int hdrSize;
1259 
1260     /** The filesystem used to access data */
1261     protected HFileSystem hfs;
1262 
1263     /** The path (if any) where this data is coming from */
1264     protected Path path;
1265 
1266     private final Lock streamLock = new ReentrantLock();
1267 
1268     /** The default buffer size for our buffered streams */
1269     public static final int DEFAULT_BUFFER_SIZE = 1 << 20;
1270 
1271     protected HFileContext fileContext;
1272 
1273     public AbstractFSReader(long fileSize, HFileSystem hfs, Path path, HFileContext fileContext)
1274         throws IOException {
1275       this.fileSize = fileSize;
1276       this.hfs = hfs;
1277       this.path = path;
1278       this.fileContext = fileContext;
1279       this.hdrSize = headerSize(fileContext.isUseHBaseChecksum());
1280     }
1281 
1282     @Override
1283     public BlockIterator blockRange(final long startOffset,
1284         final long endOffset) {
1285       final FSReader owner = this; // handle for inner class
1286       return new BlockIterator() {
1287         private long offset = startOffset;
1288 
1289         @Override
1290         public HFileBlock nextBlock() throws IOException {
1291           if (offset >= endOffset)
1292             return null;
1293           HFileBlock b = readBlockData(offset, -1, -1, false);
1294           offset += b.getOnDiskSizeWithHeader();
1295           return b.unpack(fileContext, owner);
1296         }
1297 
1298         @Override
1299         public HFileBlock nextBlockWithBlockType(BlockType blockType)
1300             throws IOException {
1301           HFileBlock blk = nextBlock();
1302           if (blk.getBlockType() != blockType) {
1303             throw new IOException("Expected block of type " + blockType
1304                 + " but found " + blk.getBlockType());
1305           }
1306           return blk;
1307         }
1308       };
1309     }
1310 
1311     /**
1312      * Does a positional read or a seek and read into the given buffer. Returns
1313      * the on-disk size of the next block, or -1 if it could not be determined.
1314      *
1315      * @param dest destination buffer
1316      * @param destOffset offset in the destination buffer
1317      * @param size size of the block to be read
1318      * @param peekIntoNextBlock whether to read the next block's on-disk size
1319      * @param fileOffset position in the stream to read at
1320      * @param pread whether we should do a positional read
1321      * @param istream The input source of data
1322      * @return the on-disk size of the next block with header size included, or
1323      *         -1 if it could not be determined
1324      * @throws IOException
1325      */
1326     protected int readAtOffset(FSDataInputStream istream,
1327         byte[] dest, int destOffset, int size,
1328         boolean peekIntoNextBlock, long fileOffset, boolean pread)
1329         throws IOException {
1330       if (peekIntoNextBlock &&
1331           destOffset + size + hdrSize > dest.length) {
1332         // We are asked to read the next block's header as well, but there is
1333         // not enough room in the array.
1334         throw new IOException("Attempted to read " + size + " bytes and " +
1335             hdrSize + " bytes of next header into a " + dest.length +
1336             "-byte array at offset " + destOffset);
1337       }
1338 
1339       if (!pread && streamLock.tryLock()) {
1340         // Seek + read. Better for scanning.
1341         try {
1342           istream.seek(fileOffset);
1343 
1344           long realOffset = istream.getPos();
1345           if (realOffset != fileOffset) {
1346             throw new IOException("Tried to seek to " + fileOffset + " to "
1347                 + "read " + size + " bytes, but pos=" + realOffset
1348                 + " after seek");
1349           }
1350 
1351           if (!peekIntoNextBlock) {
1352             IOUtils.readFully(istream, dest, destOffset, size);
1353             return -1;
1354           }
1355 
1356           // Try to read the next block header.
1357           if (!readWithExtra(istream, dest, destOffset, size, hdrSize))
1358             return -1;
1359         } finally {
1360           streamLock.unlock();
1361         }
1362       } else {
1363         // Positional read. Better for random reads; or when the streamLock is already locked.
1364         int extraSize = peekIntoNextBlock ? hdrSize : 0;
1365         if (!positionalReadWithExtra(istream, fileOffset, dest, destOffset,
1366             size, extraSize)) {
1367           return -1;
1368         }
1369       }
1370 
1371       assert peekIntoNextBlock;
1372       return Bytes.toInt(dest, destOffset + size + BlockType.MAGIC_LENGTH) + hdrSize;
1373     }
1374 
1375   }
1376 
1377   /**
1378    * We always prefetch the header of the next block, so that we know its
1379    * on-disk size in advance and can read it in one operation.
1380    */
1381   private static class PrefetchedHeader {
1382     long offset = -1;
1383     byte[] header = new byte[HConstants.HFILEBLOCK_HEADER_SIZE];
1384     ByteBuffer buf = ByteBuffer.wrap(header, 0, HConstants.HFILEBLOCK_HEADER_SIZE);
1385   }
1386 
1387   /** Reads version 2 blocks from the filesystem. */
1388   static class FSReaderV2 extends AbstractFSReader {
1389     /** The file system stream of the underlying {@link HFile} that 
1390      * does or doesn't do checksum validations in the filesystem */
1391     protected FSDataInputStreamWrapper streamWrapper;
1392 
1393     private HFileBlockDecodingContext encodedBlockDecodingCtx;
1394 
1395     /** Default context used when BlockType != {@link BlockType#ENCODED_DATA}. */
1396     private final HFileBlockDefaultDecodingContext defaultDecodingCtx;
1397 
1398     private ThreadLocal<PrefetchedHeader> prefetchedHeaderForThread =
1399         new ThreadLocal<PrefetchedHeader>() {
1400           @Override
1401           public PrefetchedHeader initialValue() {
1402             return new PrefetchedHeader();
1403           }
1404         };
1405 
1406     public FSReaderV2(FSDataInputStreamWrapper stream, long fileSize, HFileSystem hfs, Path path,
1407         HFileContext fileContext) throws IOException {
1408       super(fileSize, hfs, path, fileContext);
1409       this.streamWrapper = stream;
1410       // Older versions of HBase didn't support checksum.
1411       this.streamWrapper.prepareForBlockReader(!fileContext.isUseHBaseChecksum());
1412       defaultDecodingCtx = new HFileBlockDefaultDecodingContext(fileContext);
1413       encodedBlockDecodingCtx = defaultDecodingCtx;
1414     }
1415 
1416     /**
1417      * A constructor that reads files with the latest minor version.
1418      * This is used by unit tests only.
1419      */
1420     FSReaderV2(FSDataInputStream istream, long fileSize, HFileContext fileContext) throws IOException {
1421       this(new FSDataInputStreamWrapper(istream), fileSize, null, null, fileContext);
1422     }
1423 
1424     /**
1425      * Reads a version 2 block. Tries to do as little memory allocation as
1426      * possible, using the provided on-disk size.
1427      *
1428      * @param offset the offset in the stream to read at
1429      * @param onDiskSizeWithHeaderL the on-disk size of the block, including
1430      *          the header, or -1 if unknown
1431      * @param uncompressedSize the uncompressed size of the the block. Always
1432      *          expected to be -1. This parameter is only used in version 1.
1433      * @param pread whether to use a positional read
1434      */
1435     @Override
1436     public HFileBlock readBlockData(long offset, long onDiskSizeWithHeaderL,
1437         int uncompressedSize, boolean pread) throws IOException {
1438 
1439       // get a copy of the current state of whether to validate
1440       // hbase checksums or not for this read call. This is not 
1441       // thread-safe but the one constaint is that if we decide 
1442       // to skip hbase checksum verification then we are 
1443       // guaranteed to use hdfs checksum verification.
1444       boolean doVerificationThruHBaseChecksum = streamWrapper.shouldUseHBaseChecksum();
1445       FSDataInputStream is = streamWrapper.getStream(doVerificationThruHBaseChecksum);
1446 
1447       HFileBlock blk = readBlockDataInternal(is, offset, 
1448                          onDiskSizeWithHeaderL, 
1449                          uncompressedSize, pread,
1450                          doVerificationThruHBaseChecksum);
1451       if (blk == null) {
1452         HFile.LOG.warn("HBase checksum verification failed for file " +
1453                        path + " at offset " +
1454                        offset + " filesize " + fileSize +
1455                        ". Retrying read with HDFS checksums turned on...");
1456 
1457         if (!doVerificationThruHBaseChecksum) {
1458           String msg = "HBase checksum verification failed for file " +
1459                        path + " at offset " +
1460                        offset + " filesize " + fileSize + 
1461                        " but this cannot happen because doVerify is " +
1462                        doVerificationThruHBaseChecksum;
1463           HFile.LOG.warn(msg);
1464           throw new IOException(msg); // cannot happen case here
1465         }
1466         HFile.checksumFailures.incrementAndGet(); // update metrics
1467 
1468         // If we have a checksum failure, we fall back into a mode where
1469         // the next few reads use HDFS level checksums. We aim to make the
1470         // next CHECKSUM_VERIFICATION_NUM_IO_THRESHOLD reads avoid
1471         // hbase checksum verification, but since this value is set without
1472         // holding any locks, it can so happen that we might actually do
1473         // a few more than precisely this number.
1474         is = this.streamWrapper.fallbackToFsChecksum(CHECKSUM_VERIFICATION_NUM_IO_THRESHOLD);
1475         doVerificationThruHBaseChecksum = false;
1476         blk = readBlockDataInternal(is, offset, onDiskSizeWithHeaderL,
1477                                     uncompressedSize, pread,
1478                                     doVerificationThruHBaseChecksum);
1479         if (blk != null) {
1480           HFile.LOG.warn("HDFS checksum verification suceeded for file " +
1481                          path + " at offset " +
1482                          offset + " filesize " + fileSize);
1483         }
1484       } 
1485       if (blk == null && !doVerificationThruHBaseChecksum) {
1486         String msg = "readBlockData failed, possibly due to " +
1487                      "checksum verification failed for file " + path +
1488                      " at offset " + offset + " filesize " + fileSize;
1489         HFile.LOG.warn(msg);
1490         throw new IOException(msg);
1491       }
1492 
1493       // If there is a checksum mismatch earlier, then retry with 
1494       // HBase checksums switched off and use HDFS checksum verification.
1495       // This triggers HDFS to detect and fix corrupt replicas. The
1496       // next checksumOffCount read requests will use HDFS checksums.
1497       // The decrementing of this.checksumOffCount is not thread-safe,
1498       // but it is harmless because eventually checksumOffCount will be
1499       // a negative number.
1500       streamWrapper.checksumOk();
1501       return blk;
1502     }
1503 
1504     /**
1505      * Reads a version 2 block. 
1506      *
1507      * @param offset the offset in the stream to read at
1508      * @param onDiskSizeWithHeaderL the on-disk size of the block, including
1509      *          the header, or -1 if unknown
1510      * @param uncompressedSize the uncompressed size of the the block. Always
1511      *          expected to be -1. This parameter is only used in version 1.
1512      * @param pread whether to use a positional read
1513      * @param verifyChecksum Whether to use HBase checksums. 
1514      *        If HBase checksum is switched off, then use HDFS checksum.
1515      * @return the HFileBlock or null if there is a HBase checksum mismatch
1516      */
1517     private HFileBlock readBlockDataInternal(FSDataInputStream is, long offset, 
1518         long onDiskSizeWithHeaderL, int uncompressedSize, boolean pread,
1519         boolean verifyChecksum) throws IOException {
1520       if (offset < 0) {
1521         throw new IOException("Invalid offset=" + offset + " trying to read "
1522             + "block (onDiskSize=" + onDiskSizeWithHeaderL
1523             + ", uncompressedSize=" + uncompressedSize + ")");
1524       }
1525       if (uncompressedSize != -1) {
1526         throw new IOException("Version 2 block reader API does not need " +
1527             "the uncompressed size parameter");
1528       }
1529 
1530       if ((onDiskSizeWithHeaderL < hdrSize && onDiskSizeWithHeaderL != -1)
1531           || onDiskSizeWithHeaderL >= Integer.MAX_VALUE) {
1532         throw new IOException("Invalid onDisksize=" + onDiskSizeWithHeaderL
1533             + ": expected to be at least " + hdrSize
1534             + " and at most " + Integer.MAX_VALUE + ", or -1 (offset="
1535             + offset + ", uncompressedSize=" + uncompressedSize + ")");
1536       }
1537 
1538       int onDiskSizeWithHeader = (int) onDiskSizeWithHeaderL;
1539       // See if we can avoid reading the header. This is desirable, because
1540       // we will not incur a backward seek operation if we have already
1541       // read this block's header as part of the previous read's look-ahead.
1542       // And we also want to skip reading the header again if it has already
1543       // been read.
1544       PrefetchedHeader prefetchedHeader = prefetchedHeaderForThread.get();
1545       ByteBuffer headerBuf = prefetchedHeader.offset == offset ?
1546           prefetchedHeader.buf : null;
1547 
1548       int nextBlockOnDiskSize = 0;
1549       // Allocate enough space to fit the next block's header too.
1550       byte[] onDiskBlock = null;
1551 
1552       HFileBlock b = null;
1553       if (onDiskSizeWithHeader > 0) {
1554         // We know the total on-disk size. Read the entire block into memory,
1555         // then parse the header. This code path is used when
1556         // doing a random read operation relying on the block index, as well as
1557         // when the client knows the on-disk size from peeking into the next
1558         // block's header (e.g. this block's header) when reading the previous
1559         // block. This is the faster and more preferable case.
1560 
1561         // Size that we have to skip in case we have already read the header.
1562         int preReadHeaderSize = headerBuf == null ? 0 : hdrSize;
1563         onDiskBlock = new byte[onDiskSizeWithHeader + hdrSize]; // room for this block plus the
1564                                                                 // next block's header
1565         nextBlockOnDiskSize = readAtOffset(is, onDiskBlock,
1566             preReadHeaderSize, onDiskSizeWithHeader - preReadHeaderSize,
1567             true, offset + preReadHeaderSize, pread);
1568         if (headerBuf != null) {
1569           // the header has been read when reading the previous block, copy
1570           // to this block's header
1571           System.arraycopy(headerBuf.array(),
1572               headerBuf.arrayOffset(), onDiskBlock, 0, hdrSize);
1573         } else {
1574           headerBuf = ByteBuffer.wrap(onDiskBlock, 0, hdrSize);
1575         }
1576         // We know the total on-disk size but not the uncompressed size. Read
1577         // the entire block into memory, then parse the header. Here we have
1578         // already read the block's header
1579         try {
1580           b = new HFileBlock(headerBuf, fileContext.isUseHBaseChecksum());
1581         } catch (IOException ex) {
1582           // Seen in load testing. Provide comprehensive debug info.
1583           throw new IOException("Failed to read compressed block at "
1584               + offset
1585               + ", onDiskSizeWithoutHeader="
1586               + onDiskSizeWithHeader
1587               + ", preReadHeaderSize="
1588               + hdrSize
1589               + ", header.length="
1590               + prefetchedHeader.header.length
1591               + ", header bytes: "
1592               + Bytes.toStringBinary(prefetchedHeader.header, 0,
1593                   hdrSize), ex);
1594         }
1595         // if the caller specifies a onDiskSizeWithHeader, validate it.
1596         int onDiskSizeWithoutHeader = onDiskSizeWithHeader - hdrSize;
1597         assert onDiskSizeWithoutHeader >= 0;
1598         b.validateOnDiskSizeWithoutHeader(onDiskSizeWithoutHeader);
1599       } else {
1600         // Check headerBuf to see if we have read this block's header as part of
1601         // reading the previous block. This is an optimization of peeking into
1602         // the next block's header (e.g.this block's header) when reading the
1603         // previous block. This is the faster and more preferable case. If the
1604         // header is already there, don't read the header again.
1605 
1606         // Unfortunately, we still have to do a separate read operation to
1607         // read the header.
1608         if (headerBuf == null) {
1609           // From the header, determine the on-disk size of the given hfile
1610           // block, and read the remaining data, thereby incurring two read
1611           // operations. This might happen when we are doing the first read
1612           // in a series of reads or a random read, and we don't have access
1613           // to the block index. This is costly and should happen very rarely.
1614           headerBuf = ByteBuffer.allocate(hdrSize);
1615           readAtOffset(is, headerBuf.array(), headerBuf.arrayOffset(),
1616               hdrSize, false, offset, pread);
1617         }
1618         b = new HFileBlock(headerBuf, fileContext.isUseHBaseChecksum());
1619         onDiskBlock = new byte[b.getOnDiskSizeWithHeader() + hdrSize];
1620         System.arraycopy(headerBuf.array(), headerBuf.arrayOffset(), onDiskBlock, 0, hdrSize);
1621         nextBlockOnDiskSize =
1622           readAtOffset(is, onDiskBlock, hdrSize, b.getOnDiskSizeWithHeader()
1623               - hdrSize, true, offset + hdrSize, pread);
1624         onDiskSizeWithHeader = b.onDiskSizeWithoutHeader + hdrSize;
1625       }
1626 
1627       if (!fileContext.isCompressedOrEncrypted()) {
1628         b.assumeUncompressed();
1629       }
1630 
1631       if (verifyChecksum && !validateBlockChecksum(b, offset, onDiskBlock, hdrSize)) {
1632         return null;             // checksum mismatch
1633       }
1634 
1635       // The onDiskBlock will become the headerAndDataBuffer for this block.
1636       // If nextBlockOnDiskSizeWithHeader is not zero, the onDiskBlock already
1637       // contains the header of next block, so no need to set next
1638       // block's header in it.
1639       b = new HFileBlock(ByteBuffer.wrap(onDiskBlock, 0, onDiskSizeWithHeader),
1640         this.fileContext.isUseHBaseChecksum());
1641 
1642       b.nextBlockOnDiskSizeWithHeader = nextBlockOnDiskSize;
1643 
1644       // Set prefetched header
1645       if (b.hasNextBlockHeader()) {
1646         prefetchedHeader.offset = offset + b.getOnDiskSizeWithHeader();
1647         System.arraycopy(onDiskBlock, onDiskSizeWithHeader,
1648             prefetchedHeader.header, 0, hdrSize);
1649       }
1650 
1651       b.offset = offset;
1652       b.fileContext.setIncludesTags(this.fileContext.isIncludesTags());
1653       b.fileContext.setIncludesMvcc(this.fileContext.isIncludesMvcc());
1654       return b;
1655     }
1656 
1657     void setIncludesMemstoreTS(boolean includesMemstoreTS) {
1658       this.fileContext.setIncludesMvcc(includesMemstoreTS);
1659     }
1660 
1661     void setDataBlockEncoder(HFileDataBlockEncoder encoder) {
1662       encodedBlockDecodingCtx = encoder.newDataBlockDecodingContext(this.fileContext);
1663     }
1664 
1665     @Override
1666     public HFileBlockDecodingContext getBlockDecodingContext() {
1667       return this.encodedBlockDecodingCtx;
1668     }
1669 
1670     @Override
1671     public HFileBlockDecodingContext getDefaultBlockDecodingContext() {
1672       return this.defaultDecodingCtx;
1673     }
1674 
1675     /**
1676      * Generates the checksum for the header as well as the data and
1677      * then validates that it matches the value stored in the header.
1678      * If there is a checksum mismatch, then return false. Otherwise
1679      * return true.
1680      */
1681     protected boolean validateBlockChecksum(HFileBlock block, long offset, byte[] data,
1682         int hdrSize)
1683     throws IOException {
1684       return ChecksumUtil.validateBlockChecksum(path, offset, block, data, hdrSize);
1685     }
1686 
1687     @Override
1688     public void closeStreams() throws IOException {
1689       streamWrapper.close();
1690     }
1691 
1692     @Override
1693     public String toString() {
1694       return "FSReaderV2 [ hfs=" + hfs + " path=" + path + " fileContext=" + fileContext + " ]";
1695     }
1696   }
1697 
1698   @Override
1699   public int getSerializedLength() {
1700     if (buf != null) {
1701       // include extra bytes for the next header when it's available.
1702       int extraSpace = hasNextBlockHeader() ? headerSize() : 0;
1703       return this.buf.limit() + extraSpace + HFileBlock.EXTRA_SERIALIZATION_SPACE;
1704     }
1705     return 0;
1706   }
1707 
1708   @Override
1709   public void serialize(ByteBuffer destination) {
1710     // assumes HeapByteBuffer
1711     destination.put(this.buf.array(), this.buf.arrayOffset(),
1712       getSerializedLength() - EXTRA_SERIALIZATION_SPACE);
1713     serializeExtraInfo(destination);
1714   }
1715 
1716   public void serializeExtraInfo(ByteBuffer destination) {
1717     destination.put(this.fileContext.isUseHBaseChecksum() ? (byte) 1 : (byte) 0);
1718     destination.putLong(this.offset);
1719     destination.putInt(this.nextBlockOnDiskSizeWithHeader);
1720     destination.rewind();
1721   }
1722 
1723   @Override
1724   public CacheableDeserializer<Cacheable> getDeserializer() {
1725     return HFileBlock.blockDeserializer;
1726   }
1727 
1728   @Override
1729   public boolean equals(Object comparison) {
1730     if (this == comparison) {
1731       return true;
1732     }
1733     if (comparison == null) {
1734       return false;
1735     }
1736     if (comparison.getClass() != this.getClass()) {
1737       return false;
1738     }
1739 
1740     HFileBlock castedComparison = (HFileBlock) comparison;
1741 
1742     if (castedComparison.blockType != this.blockType) {
1743       return false;
1744     }
1745     if (castedComparison.nextBlockOnDiskSizeWithHeader != this.nextBlockOnDiskSizeWithHeader) {
1746       return false;
1747     }
1748     if (castedComparison.offset != this.offset) {
1749       return false;
1750     }
1751     if (castedComparison.onDiskSizeWithoutHeader != this.onDiskSizeWithoutHeader) {
1752       return false;
1753     }
1754     if (castedComparison.prevBlockOffset != this.prevBlockOffset) {
1755       return false;
1756     }
1757     if (castedComparison.uncompressedSizeWithoutHeader != this.uncompressedSizeWithoutHeader) {
1758       return false;
1759     }
1760     if (Bytes.compareTo(this.buf.array(), this.buf.arrayOffset(), this.buf.limit(),
1761       castedComparison.buf.array(), castedComparison.buf.arrayOffset(),
1762       castedComparison.buf.limit()) != 0) {
1763       return false;
1764     }
1765     return true;
1766   }
1767 
1768   public DataBlockEncoding getDataBlockEncoding() {
1769     if (blockType == BlockType.ENCODED_DATA) {
1770       return DataBlockEncoding.getEncodingById(getDataBlockEncodingId());
1771     }
1772     return DataBlockEncoding.NONE;
1773   }
1774 
1775   byte getChecksumType() {
1776     return this.fileContext.getChecksumType().getCode();
1777   }
1778 
1779   int getBytesPerChecksum() {
1780     return this.fileContext.getBytesPerChecksum();
1781   }
1782 
1783   /** @return the size of data on disk + header. Excludes checksum. */
1784   int getOnDiskDataSizeWithHeader() {
1785     return this.onDiskDataSizeWithHeader;
1786   }
1787 
1788   /** 
1789    * Calcuate the number of bytes required to store all the checksums
1790    * for this block. Each checksum value is a 4 byte integer.
1791    */
1792   int totalChecksumBytes() {
1793     // If the hfile block has minorVersion 0, then there are no checksum
1794     // data to validate. Similarly, a zero value in this.bytesPerChecksum
1795     // indicates that cached blocks do not have checksum data because
1796     // checksums were already validated when the block was read from disk.
1797     if (!fileContext.isUseHBaseChecksum() || this.fileContext.getBytesPerChecksum() == 0) {
1798       return 0;
1799     }
1800     return (int)ChecksumUtil.numBytes(onDiskDataSizeWithHeader, this.fileContext.getBytesPerChecksum());
1801   }
1802 
1803   /**
1804    * Returns the size of this block header.
1805    */
1806   public int headerSize() {
1807     return headerSize(this.fileContext.isUseHBaseChecksum());
1808   }
1809 
1810   /**
1811    * Maps a minor version to the size of the header.
1812    */
1813   public static int headerSize(boolean usesHBaseChecksum) {
1814     if (usesHBaseChecksum) {
1815       return HConstants.HFILEBLOCK_HEADER_SIZE;
1816     }
1817     return HConstants.HFILEBLOCK_HEADER_SIZE_NO_CHECKSUM;
1818   }
1819 
1820   /**
1821    * Return the appropriate DUMMY_HEADER for the minor version
1822    */
1823   public byte[] getDummyHeaderForVersion() {
1824     return getDummyHeaderForVersion(this.fileContext.isUseHBaseChecksum());
1825   }
1826 
1827   /**
1828    * Return the appropriate DUMMY_HEADER for the minor version
1829    */
1830   static private byte[] getDummyHeaderForVersion(boolean usesHBaseChecksum) {
1831     if (usesHBaseChecksum) {
1832       return HConstants.HFILEBLOCK_DUMMY_HEADER;
1833     }
1834     return DUMMY_HEADER_NO_CHECKSUM;
1835   }
1836 
1837   /**
1838    * @return the HFileContext used to create this HFileBlock. Not necessary the
1839    * fileContext for the file from which this block's data was originally read.
1840    */
1841   public HFileContext getHFileContext() {
1842     return this.fileContext;
1843   }
1844 
1845   /**
1846    * Convert the contents of the block header into a human readable string.
1847    * This is mostly helpful for debugging. This assumes that the block
1848    * has minor version > 0.
1849    */
1850   static String toStringHeader(ByteBuffer buf) throws IOException {
1851     int offset = buf.arrayOffset();
1852     byte[] b = buf.array();
1853     long magic = Bytes.toLong(b, offset);
1854     BlockType bt = BlockType.read(buf);
1855     offset += Bytes.SIZEOF_LONG;
1856     int compressedBlockSizeNoHeader = Bytes.toInt(b, offset);
1857     offset += Bytes.SIZEOF_INT;
1858     int uncompressedBlockSizeNoHeader = Bytes.toInt(b, offset);
1859     offset += Bytes.SIZEOF_INT;
1860     long prevBlockOffset = Bytes.toLong(b, offset); 
1861     offset += Bytes.SIZEOF_LONG;
1862     byte cksumtype = b[offset];
1863     offset += Bytes.SIZEOF_BYTE;
1864     long bytesPerChecksum = Bytes.toInt(b, offset); 
1865     offset += Bytes.SIZEOF_INT;
1866     long onDiskDataSizeWithHeader = Bytes.toInt(b, offset); 
1867     offset += Bytes.SIZEOF_INT;
1868     return " Header dump: magic: " + magic +
1869                    " blockType " + bt +
1870                    " compressedBlockSizeNoHeader " + 
1871                    compressedBlockSizeNoHeader +
1872                    " uncompressedBlockSizeNoHeader " + 
1873                    uncompressedBlockSizeNoHeader +
1874                    " prevBlockOffset " + prevBlockOffset +
1875                    " checksumType " + ChecksumType.codeToType(cksumtype) +
1876                    " bytesPerChecksum " + bytesPerChecksum +
1877                    " onDiskDataSizeWithHeader " + onDiskDataSizeWithHeader;
1878   }
1879 }