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