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