View Javadoc

1   /*
2    * Copyright 2011 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.io.hfile;
21  
22  import java.io.BufferedInputStream;
23  import java.io.ByteArrayInputStream;
24  import java.io.ByteArrayOutputStream;
25  import java.io.DataInputStream;
26  import java.io.DataOutput;
27  import java.io.DataOutputStream;
28  import java.io.IOException;
29  import java.io.InputStream;
30  import java.io.OutputStream;
31  import java.nio.ByteBuffer;
32  import org.apache.hadoop.fs.FSDataInputStream;
33  import org.apache.hadoop.fs.FSDataOutputStream;
34  
35  import org.apache.hadoop.hbase.io.DoubleOutputStream;
36  import org.apache.hadoop.hbase.io.hfile.Compression.Algorithm;
37  import org.apache.hadoop.hbase.util.Bytes;
38  import org.apache.hadoop.hbase.util.ClassSize;
39  import org.apache.hadoop.hbase.util.CompoundBloomFilter;
40  import org.apache.hadoop.hbase.util.Writables;
41  import org.apache.hadoop.io.IOUtils;
42  import org.apache.hadoop.io.Writable;
43  import org.apache.hadoop.io.compress.Compressor;
44  import org.apache.hadoop.io.compress.Decompressor;
45  
46  import com.google.common.base.Preconditions;
47  
48  import static org.apache.hadoop.hbase.io.hfile.BlockType.MAGIC_LENGTH;
49  import static org.apache.hadoop.hbase.io.hfile.Compression.Algorithm.NONE;
50  
51  /**
52   * Reading {@link HFile} version 1 and 2 blocks, and writing version 2 blocks.
53   * <ul>
54   * <li>In version 1 all blocks are always compressed or uncompressed, as
55   * specified by the {@link HFile}'s compression algorithm, with a type-specific
56   * magic record stored in the beginning of the compressed data (i.e. one needs
57   * to uncompress the compressed block to determine the block type). There is
58   * only a single compression algorithm setting for all blocks. Offset and size
59   * information from the block index are required to read a block.
60   * <li>In version 2 a block is structured as follows:
61   * <ul>
62   * <li>Magic record identifying the block type (8 bytes)
63   * <li>Compressed block size, header not included (4 bytes)
64   * <li>Uncompressed block size, header not included (4 bytes)
65   * <li>The offset of the previous block of the same type (8 bytes). This is
66   * used to be able to navigate to the previous block without going to the block
67   * index.
68   * <li>Compressed data (or uncompressed data if compression is disabled). The
69   * compression algorithm is the same for all the blocks in the {@link HFile},
70   * similarly to what was done in version 1.
71   * </ul>
72   * </ul>
73   * The version 2 block representation in the block cache is the same as above,
74   * except that the data section is always uncompressed in the cache.
75   */
76  public class HFileBlock implements Cacheable {
77  
78    /** The size of a version 2 {@link HFile} block header */
79    public static final int HEADER_SIZE = MAGIC_LENGTH + 2 * Bytes.SIZEOF_INT
80        + Bytes.SIZEOF_LONG;
81  
82    /** Just an array of bytes of the right size. */
83    public static final byte[] DUMMY_HEADER = new byte[HEADER_SIZE];
84  
85    public static final int BYTE_BUFFER_HEAP_SIZE = (int) ClassSize.estimateBase(
86        ByteBuffer.wrap(new byte[0], 0, 0).getClass(), false);
87  
88    static final int EXTRA_SERIALIZATION_SPACE = Bytes.SIZEOF_LONG + Bytes.SIZEOF_INT;
89  
90  
91    private static final CacheableDeserializer<Cacheable> blockDeserializer =
92    new CacheableDeserializer<Cacheable>() {
93      public HFileBlock deserialize(ByteBuffer buf) throws IOException{
94        ByteBuffer newByteBuffer = ByteBuffer.allocate(buf.limit()
95            - HFileBlock.EXTRA_SERIALIZATION_SPACE);
96        buf.limit(buf.limit()
97            - HFileBlock.EXTRA_SERIALIZATION_SPACE).rewind();
98        newByteBuffer.put(buf);
99        HFileBlock ourBuffer = new HFileBlock(newByteBuffer);
100 
101       buf.position(buf.limit());
102       buf.limit(buf.limit() + HFileBlock.EXTRA_SERIALIZATION_SPACE);
103       ourBuffer.offset = buf.getLong();
104       ourBuffer.nextBlockOnDiskSizeWithHeader = buf.getInt();
105       return ourBuffer;
106     }
107   };
108   private BlockType blockType;
109   private final int onDiskSizeWithoutHeader;
110   private final int uncompressedSizeWithoutHeader;
111   private final long prevBlockOffset;
112   private ByteBuffer buf;
113 
114   /**
115    * The offset of this block in the file. Populated by the reader for
116    * convenience of access. This offset is not part of the block header.
117    */
118   private long offset = -1;
119 
120   /**
121    * The on-disk size of the next block, including the header, obtained by
122    * peeking into the first {@link HEADER_SIZE} bytes of the next block's
123    * header, or -1 if unknown.
124    */
125   private int nextBlockOnDiskSizeWithHeader = -1;
126 
127   /**
128    * Creates a new {@link HFile} block from the given fields. This constructor
129    * is mostly used when the block data has already been read and uncompressed,
130    * and is sitting in a byte buffer.
131    *
132    * @param blockType the type of this block, see {@link BlockType}
133    * @param onDiskSizeWithoutHeader compressed size of the block if compression
134    *          is used, otherwise uncompressed size, header size not included
135    * @param uncompressedSizeWithoutHeader uncompressed size of the block,
136    *          header size not included. Equals onDiskSizeWithoutHeader if
137    *          compression is disabled.
138    * @param prevBlockOffset the offset of the previous block in the
139    *          {@link HFile}
140    * @param buf block header ({@link #HEADER_SIZE} bytes) followed by
141    *          uncompressed data. This
142    * @param fillHeader true to fill in the first {@link #HEADER_SIZE} bytes of
143    *          the buffer based on the header fields provided
144    * @param offset the file offset the block was read from
145    */
146   public HFileBlock(BlockType blockType, int onDiskSizeWithoutHeader,
147       int uncompressedSizeWithoutHeader, long prevBlockOffset, ByteBuffer buf,
148       boolean fillHeader, long offset) {
149     this.blockType = blockType;
150     this.onDiskSizeWithoutHeader = onDiskSizeWithoutHeader;
151     this.uncompressedSizeWithoutHeader = uncompressedSizeWithoutHeader;
152     this.prevBlockOffset = prevBlockOffset;
153     this.buf = buf;
154     if (fillHeader)
155       overwriteHeader();
156     this.offset = offset;
157   }
158 
159   /**
160    * Creates a block from an existing buffer starting with a header. Rewinds
161    * and takes ownership of the buffer. By definition of rewind, ignores the
162    * buffer position, but if you slice the buffer beforehand, it will rewind
163    * to that point.
164    */
165   private HFileBlock(ByteBuffer b) throws IOException {
166     b.rewind();
167     blockType = BlockType.read(b);
168     onDiskSizeWithoutHeader = b.getInt();
169     uncompressedSizeWithoutHeader = b.getInt();
170     prevBlockOffset = b.getLong();
171     buf = b;
172     buf.rewind();
173   }
174 
175   public BlockType getBlockType() {
176     return blockType;
177   }
178 
179   /**
180    * @return the on-disk size of the block with header size included
181    */
182   public int getOnDiskSizeWithHeader() {
183     return onDiskSizeWithoutHeader + HEADER_SIZE;
184   }
185 
186   /**
187    * Returns the size of the compressed part of the block in case compression
188    * is used, or the uncompressed size of the data part otherwise. Header size
189    * is not included.
190    *
191    * @return the on-disk size of the data part of the block, header not
192    *         included
193    */
194   public int getOnDiskSizeWithoutHeader() {
195     return onDiskSizeWithoutHeader;
196   }
197 
198   /**
199    * @return the uncompressed size of the data part of the block, header not
200    *         included
201    */
202   public int getUncompressedSizeWithoutHeader() {
203     return uncompressedSizeWithoutHeader;
204   }
205 
206   /**
207    * @return the offset of the previous block of the same type in the file, or
208    *         -1 if unknown
209    */
210   public long getPrevBlockOffset() {
211     return prevBlockOffset;
212   }
213 
214   /**
215    * Writes header fields into the first {@link HEADER_SIZE} bytes of the
216    * buffer. Resets the buffer position to the end of header as side effect.
217    */
218   private void overwriteHeader() {
219     buf.rewind();
220     blockType.write(buf);
221     buf.putInt(onDiskSizeWithoutHeader);
222     buf.putInt(uncompressedSizeWithoutHeader);
223     buf.putLong(prevBlockOffset);
224   }
225 
226   /**
227    * Returns a buffer that does not include the header. The array offset points
228    * to the start of the block data right after the header. The underlying data
229    * array is not copied.
230    *
231    * @return the buffer with header skipped
232    */
233   public ByteBuffer getBufferWithoutHeader() {
234     return ByteBuffer.wrap(buf.array(), buf.arrayOffset() + HEADER_SIZE,
235         buf.limit() - HEADER_SIZE).slice();
236   }
237 
238   /**
239    * Returns the buffer this block stores internally. The clients must not
240    * modify the buffer object. This method has to be public because it is
241    * used in {@link CompoundBloomFilter} to avoid object creation on every
242    * Bloom filter lookup, but has to be used with caution.
243    *
244    * @return the buffer of this block for read-only operations
245    */
246   public ByteBuffer getBufferReadOnly() {
247     return buf;
248   }
249 
250   /**
251    * Returns a byte buffer of this block, including header data, positioned at
252    * the beginning of header. The underlying data array is not copied.
253    *
254    * @return the byte buffer with header included
255    */
256   public ByteBuffer getBufferWithHeader() {
257     ByteBuffer dupBuf = buf.duplicate();
258     dupBuf.rewind();
259     return dupBuf;
260   }
261 
262   /**
263    * Deserializes fields of the given writable using the data portion of this
264    * block. Does not check that all the block data has been read.
265    */
266   public void readInto(Writable w) throws IOException {
267     Preconditions.checkNotNull(w);
268 
269     if (Writables.getWritable(buf.array(), buf.arrayOffset() + HEADER_SIZE,
270         buf.limit() - HEADER_SIZE, w) == null) {
271       throw new IOException("Failed to deserialize block " + this + " into a "
272           + w.getClass().getSimpleName());
273     }
274   }
275 
276   private void sanityCheckAssertion(long valueFromBuf, long valueFromField,
277       String fieldName) throws IOException {
278     if (valueFromBuf != valueFromField) {
279       throw new AssertionError(fieldName + " in the buffer (" + valueFromBuf
280           + ") is different from that in the field (" + valueFromField + ")");
281     }
282   }
283 
284   /**
285    * Checks if the block is internally consistent, i.e. the first
286    * {@link #HEADER_SIZE} bytes of the buffer contain a valid header consistent
287    * with the fields. This function is primary for testing and debugging, and
288    * is not thread-safe, because it alters the internal buffer pointer.
289    */
290   void sanityCheck() throws IOException {
291     buf.rewind();
292 
293     {
294       BlockType blockTypeFromBuf = BlockType.read(buf);
295       if (blockTypeFromBuf != blockType) {
296         throw new IOException("Block type stored in the buffer: " +
297             blockTypeFromBuf + ", block type field: " + blockType);
298       }
299     }
300 
301     sanityCheckAssertion(buf.getInt(), onDiskSizeWithoutHeader,
302         "onDiskSizeWithoutHeader");
303 
304     sanityCheckAssertion(buf.getInt(), uncompressedSizeWithoutHeader,
305         "uncompressedSizeWithoutHeader");
306 
307     sanityCheckAssertion(buf.getLong(), prevBlockOffset, "prevBlocKOffset");
308 
309     int expectedBufLimit = uncompressedSizeWithoutHeader + HEADER_SIZE;
310     if (buf.limit() != expectedBufLimit) {
311       throw new AssertionError("Expected buffer limit " + expectedBufLimit
312           + ", got " + buf.limit());
313     }
314 
315     // We might optionally allocate HEADER_SIZE more bytes to read the next
316     // block's, header, so there are two sensible values for buffer capacity.
317     if (buf.capacity() != uncompressedSizeWithoutHeader + HEADER_SIZE &&
318         buf.capacity() != uncompressedSizeWithoutHeader + 2 * HEADER_SIZE) {
319       throw new AssertionError("Invalid buffer capacity: " + buf.capacity() +
320           ", expected " + (uncompressedSizeWithoutHeader + HEADER_SIZE) +
321           " or " + (uncompressedSizeWithoutHeader + 2 * HEADER_SIZE));
322     }
323   }
324 
325   @Override
326   public String toString() {
327     return "blockType="
328         + blockType
329         + ", onDiskSizeWithoutHeader="
330         + onDiskSizeWithoutHeader
331         + ", uncompressedSizeWithoutHeader="
332         + uncompressedSizeWithoutHeader
333         + ", prevBlockOffset="
334         + prevBlockOffset
335         + ", dataBeginsWith="
336         + Bytes.toStringBinary(buf.array(), buf.arrayOffset() + HEADER_SIZE,
337             Math.min(32, buf.limit() - buf.arrayOffset() - HEADER_SIZE))
338         + ", fileOffset=" + offset;
339   }
340 
341   private void validateOnDiskSizeWithoutHeader(
342       int expectedOnDiskSizeWithoutHeader) throws IOException {
343     if (onDiskSizeWithoutHeader != expectedOnDiskSizeWithoutHeader) {
344       String blockInfoMsg =
345         "Block offset: " + offset + ", data starts with: "
346           + Bytes.toStringBinary(buf.array(), buf.arrayOffset(),
347               buf.arrayOffset() + Math.min(32, buf.limit()));
348       throw new IOException("On-disk size without header provided is "
349           + expectedOnDiskSizeWithoutHeader + ", but block "
350           + "header contains " + onDiskSizeWithoutHeader + ". " +
351           blockInfoMsg);
352     }
353   }
354 
355   /**
356    * Always allocates a new buffer of the correct size. Copies header bytes
357    * from the existing buffer. Does not change header fields.
358    *
359    * @param extraBytes whether to reserve room in the buffer to read the next
360    *          block's header
361    */
362   private void allocateBuffer(boolean extraBytes) {
363     int capacityNeeded = HEADER_SIZE + uncompressedSizeWithoutHeader +
364         (extraBytes ? HEADER_SIZE : 0);
365 
366     ByteBuffer newBuf = ByteBuffer.allocate(capacityNeeded);
367 
368     // Copy header bytes.
369     System.arraycopy(buf.array(), buf.arrayOffset(), newBuf.array(),
370         newBuf.arrayOffset(), HEADER_SIZE);
371 
372     buf = newBuf;
373     buf.limit(HEADER_SIZE + uncompressedSizeWithoutHeader);
374   }
375 
376   /** An additional sanity-check in case no compression is being used. */
377   public void assumeUncompressed() throws IOException {
378     if (onDiskSizeWithoutHeader != uncompressedSizeWithoutHeader) {
379       throw new IOException("Using no compression but "
380           + "onDiskSizeWithoutHeader=" + onDiskSizeWithoutHeader + ", "
381           + "uncompressedSizeWithoutHeader=" + uncompressedSizeWithoutHeader);
382     }
383   }
384 
385   /**
386    * @param expectedType the expected type of this block
387    * @throws IOException if this block's type is different than expected
388    */
389   public void expectType(BlockType expectedType) throws IOException {
390     if (blockType != expectedType) {
391       throw new IOException("Invalid block type: expected=" + expectedType
392           + ", actual=" + blockType);
393     }
394   }
395 
396   /** @return the offset of this block in the file it was read from */
397   public long getOffset() {
398     if (offset < 0) {
399       throw new IllegalStateException(
400           "HFile block offset not initialized properly");
401     }
402     return offset;
403   }
404 
405   /**
406    * @return a byte stream reading the data section of this block
407    */
408   public DataInputStream getByteStream() {
409     return new DataInputStream(new ByteArrayInputStream(buf.array(),
410         buf.arrayOffset() + HEADER_SIZE, buf.limit() - HEADER_SIZE));
411   }
412 
413   @Override
414   public long heapSize() {
415     // This object, block type and byte buffer reference, on-disk and
416     // uncompressed size, next block's on-disk size, offset and previous
417     // offset, byte buffer object, and its byte array. Might also need to add
418     // some fields inside the byte buffer.
419 
420     // We only add one BYTE_BUFFER_HEAP_SIZE because at any given moment, one of
421     // the bytebuffers will be null. But we do account for both references.
422 
423     // If we are on heap, then we add the capacity of buf.
424     if (buf != null) {
425       return ClassSize.align(ClassSize.OBJECT + 2 * ClassSize.REFERENCE + 3
426           * Bytes.SIZEOF_INT + 2 * Bytes.SIZEOF_LONG + BYTE_BUFFER_HEAP_SIZE)
427           + ClassSize.align(buf.capacity());
428     } else {
429 
430       return ClassSize.align(ClassSize.OBJECT + 2 * ClassSize.REFERENCE + 3
431           * Bytes.SIZEOF_INT + 2 * Bytes.SIZEOF_LONG + BYTE_BUFFER_HEAP_SIZE);
432     }
433   }
434 
435   /**
436    * Read from an input stream. Analogous to
437    * {@link IOUtils#readFully(InputStream, byte[], int, int)}, but specifies a
438    * number of "extra" bytes that would be desirable but not absolutely
439    * necessary to read.
440    *
441    * @param in the input stream to read from
442    * @param buf the buffer to read into
443    * @param bufOffset the destination offset in the buffer
444    * @param necessaryLen the number of bytes that are absolutely necessary to
445    *          read
446    * @param extraLen the number of extra bytes that would be nice to read
447    * @return true if succeeded reading the extra bytes
448    * @throws IOException if failed to read the necessary bytes
449    */
450   public static boolean readWithExtra(InputStream in, byte buf[],
451       int bufOffset, int necessaryLen, int extraLen) throws IOException {
452     int bytesRemaining = necessaryLen + extraLen;
453     while (bytesRemaining > 0) {
454       int ret = in.read(buf, bufOffset, bytesRemaining);
455       if (ret == -1 && bytesRemaining <= extraLen) {
456         // We could not read the "extra data", but that is OK.
457         break;
458       }
459 
460       if (ret < 0) {
461         throw new IOException("Premature EOF from inputStream (read "
462             + "returned " + ret + ", was trying to read " + necessaryLen
463             + " necessary bytes and " + extraLen + " extra bytes, "
464             + "successfully read "
465             + (necessaryLen + extraLen - bytesRemaining));
466       }
467       bufOffset += ret;
468       bytesRemaining -= ret;
469     }
470     return bytesRemaining <= 0;
471   }
472 
473   /**
474    * @return the on-disk size of the next block (including the header size)
475    *         that was read by peeking into the next block's header
476    */
477   public int getNextBlockOnDiskSizeWithHeader() {
478     return nextBlockOnDiskSizeWithHeader;
479   }
480 
481 
482   /**
483    * Unified version 2 {@link HFile} block writer. The intended usage pattern
484    * is as follows:
485    * <ul>
486    * <li>Construct an {@link HFileBlock.Writer}, providing a compression
487    * algorithm
488    * <li>Call {@link Writer#startWriting(BlockType, boolean)} and get a data stream to
489    * write to
490    * <li>Write your data into the stream
491    * <li>Call {@link Writer#writeHeaderAndData(FSDataOutputStream)} as many times as you need to
492    * store the serialized block into an external stream, or call
493    * {@link Writer#getHeaderAndData()} to get it as a byte array.
494    * <li>Repeat to write more blocks
495    * </ul>
496    * <p>
497    */
498   public static class Writer {
499 
500     private enum State {
501       INIT,
502       WRITING,
503       BLOCK_READY
504     };
505 
506     /** Writer state. Used to ensure the correct usage protocol. */
507     private State state = State.INIT;
508 
509     /** Compression algorithm for all blocks this instance writes. */
510     private final Compression.Algorithm compressAlgo;
511 
512     /**
513      * The stream we use to accumulate data in the on-disk format for each
514      * block (i.e. compressed data, or uncompressed if using no compression).
515      * We reset this stream at the end of each block and reuse it. The header
516      * is written as the first {@link #HEADER_SIZE} bytes into this stream.
517      */
518     private ByteArrayOutputStream baosOnDisk;
519 
520     /**
521      * The stream we use to accumulate uncompressed block data for
522      * cache-on-write. Null when cache-on-write is turned off.
523      */
524     private ByteArrayOutputStream baosInMemory;
525 
526     /** Compressor, which is also reused between consecutive blocks. */
527     private Compressor compressor;
528 
529     /** Current block type. Set in {@link #startWriting(BlockType)}. */
530     private BlockType blockType;
531 
532     /**
533      * A stream that we write uncompressed bytes to, which compresses them and
534      * writes them to {@link #baosOnDisk}.
535      */
536     private DataOutputStream userDataStream;
537 
538     /**
539      * Bytes to be written to the file system, including the header. Compressed
540      * if compression is turned on.
541      */
542     private byte[] onDiskBytesWithHeader;
543 
544     /**
545      * The total number of uncompressed bytes written into the current block,
546      * with header size not included. Valid in the READY state.
547      */
548     private int uncompressedSizeWithoutHeader;
549 
550     /**
551      * Only used when we are using cache-on-write. Valid in the READY state.
552      * Contains the header and the uncompressed bytes, so the length is
553      * {@link #uncompressedSizeWithoutHeader} + {@link HFileBlock#HEADER_SIZE}.
554      */
555     private byte[] uncompressedBytesWithHeader;
556 
557     /**
558      * Current block's start offset in the {@link HFile}. Set in
559      * {@link #writeHeaderAndData(FSDataOutputStream)}.
560      */
561     private long startOffset;
562 
563     /**
564      * Offset of previous block by block type. Updated when the next block is
565      * started.
566      */
567     private long[] prevOffsetByType;
568 
569     /**
570      * Whether we are accumulating uncompressed bytes for the purpose of
571      * caching on write.
572      */
573     private boolean cacheOnWrite;
574 
575     /** The offset of the previous block of the same type */
576     private long prevOffset;
577 
578     /**
579      * @param compressionAlgorithm
580      *          compression algorithm to use
581      */
582     public Writer(Compression.Algorithm compressionAlgorithm) {
583       compressAlgo = compressionAlgorithm == null ? NONE
584           : compressionAlgorithm;
585 
586       baosOnDisk = new ByteArrayOutputStream();
587       if (compressAlgo != NONE)
588         compressor = compressionAlgorithm.getCompressor();
589 
590       prevOffsetByType = new long[BlockType.values().length];
591       for (int i = 0; i < prevOffsetByType.length; ++i)
592         prevOffsetByType[i] = -1;
593     }
594 
595     /**
596      * Starts writing into the block. The previous block's data is discarded.
597      *
598      * @return the stream the user can write their data into
599      * @throws IOException
600      */
601     public DataOutputStream startWriting(BlockType newBlockType,
602         boolean cacheOnWrite) throws IOException {
603       if (state == State.BLOCK_READY && startOffset != -1) {
604         // We had a previous block that was written to a stream at a specific
605         // offset. Save that offset as the last offset of a block of that type.
606         prevOffsetByType[blockType.ordinal()] = startOffset;
607       }
608 
609       this.cacheOnWrite = cacheOnWrite;
610 
611       startOffset = -1;
612       blockType = newBlockType;
613 
614       baosOnDisk.reset();
615       baosOnDisk.write(DUMMY_HEADER);
616 
617       state = State.WRITING;
618       if (compressAlgo == NONE) {
619         // We do not need a compression stream or a second uncompressed stream
620         // for cache-on-write.
621         userDataStream = new DataOutputStream(baosOnDisk);
622       } else {
623         OutputStream compressingOutputStream =
624           compressAlgo.createCompressionStream(baosOnDisk, compressor, 0);
625 
626         if (cacheOnWrite) {
627           // We save uncompressed data in a cache-on-write mode.
628           if (baosInMemory == null)
629             baosInMemory = new ByteArrayOutputStream();
630           baosInMemory.reset();
631           baosInMemory.write(DUMMY_HEADER);
632           userDataStream = new DataOutputStream(new DoubleOutputStream(
633               compressingOutputStream, baosInMemory));
634         } else {
635           userDataStream = new DataOutputStream(compressingOutputStream);
636         }
637       }
638 
639       return userDataStream;
640     }
641 
642     /**
643      * Returns the stream for the user to write to. The block writer takes care
644      * of handling compression and buffering for caching on write. Can only be
645      * called in the "writing" state.
646      *
647      * @return the data output stream for the user to write to
648      */
649     DataOutputStream getUserDataStream() {
650       expectState(State.WRITING);
651       return userDataStream;
652     }
653 
654     /**
655      * Transitions the block writer from the "writing" state to the "block
656      * ready" state.  Does nothing if a block is already finished.
657      */
658     private void ensureBlockReady() throws IOException {
659       Preconditions.checkState(state != State.INIT,
660           "Unexpected state: " + state);
661 
662       if (state == State.BLOCK_READY)
663         return;
664 
665       finishBlock();
666       state = State.BLOCK_READY;
667     }
668 
669     /**
670      * An internal method that flushes the compressing stream (if using
671      * compression), serializes the header, and takes care of the separate
672      * uncompressed stream for caching on write, if applicable. Block writer
673      * state transitions must be managed by the caller.
674      */
675     private void finishBlock() throws IOException {
676       userDataStream.flush();
677       uncompressedSizeWithoutHeader = userDataStream.size();
678 
679       onDiskBytesWithHeader = baosOnDisk.toByteArray();
680       prevOffset = prevOffsetByType[blockType.ordinal()];
681       putHeader(onDiskBytesWithHeader, 0);
682 
683       if (cacheOnWrite && compressAlgo != NONE) {
684         uncompressedBytesWithHeader = baosInMemory.toByteArray();
685 
686         if (uncompressedSizeWithoutHeader !=
687             uncompressedBytesWithHeader.length - HEADER_SIZE) {
688           throw new IOException("Uncompressed size mismatch: "
689               + uncompressedSizeWithoutHeader + " vs. "
690               + (uncompressedBytesWithHeader.length - HEADER_SIZE));
691         }
692 
693         // Write the header into the beginning of the uncompressed byte array.
694         putHeader(uncompressedBytesWithHeader, 0);
695       }
696     }
697 
698     /** Put the header into the given byte array at the given offset. */
699     private void putHeader(byte[] dest, int offset) {
700       offset = blockType.put(dest, offset);
701       offset = Bytes.putInt(dest, offset, onDiskBytesWithHeader.length
702           - HEADER_SIZE);
703       offset = Bytes.putInt(dest, offset, uncompressedSizeWithoutHeader);
704       Bytes.putLong(dest, offset, prevOffset);
705     }
706 
707     /**
708      * Similar to {@link #writeHeaderAndData(FSDataOutputStream)}, but records
709      * the offset of this block so that it can be referenced in the next block
710      * of the same type.
711      *
712      * @param out
713      * @throws IOException
714      */
715     public void writeHeaderAndData(FSDataOutputStream out) throws IOException {
716       long offset = out.getPos();
717       if (startOffset != -1 && offset != startOffset) {
718         throw new IOException("A " + blockType + " block written to a "
719             + "stream twice, first at offset " + startOffset + ", then at "
720             + offset);
721       }
722       startOffset = offset;
723 
724       writeHeaderAndData((DataOutputStream) out);
725     }
726 
727     /**
728      * Writes the header and the compressed data of this block (or uncompressed
729      * data when not using compression) into the given stream. Can be called in
730      * the "writing" state or in the "block ready" state. If called in the
731      * "writing" state, transitions the writer to the "block ready" state.
732      *
733      * @param out the output stream to write the
734      * @throws IOException
735      */
736     private void writeHeaderAndData(DataOutputStream out) throws IOException {
737       ensureBlockReady();
738       out.write(onDiskBytesWithHeader);
739     }
740 
741     /**
742      * Returns the header or the compressed data (or uncompressed data when not
743      * using compression) as a byte array. Can be called in the "writing" state
744      * or in the "block ready" state. If called in the "writing" state,
745      * transitions the writer to the "block ready" state.
746      *
747      * @return header and data as they would be stored on disk in a byte array
748      * @throws IOException
749      */
750     public byte[] getHeaderAndData() throws IOException {
751       ensureBlockReady();
752       return onDiskBytesWithHeader;
753     }
754 
755     /**
756      * Releases the compressor this writer uses to compress blocks into the
757      * compressor pool. Needs to be called before the writer is discarded.
758      */
759     public void releaseCompressor() {
760       if (compressor != null) {
761         compressAlgo.returnCompressor(compressor);
762         compressor = null;
763       }
764     }
765 
766     /**
767      * Returns the on-disk size of the data portion of the block. This is the
768      * compressed size if compression is enabled. Can only be called in the
769      * "block ready" state. Header is not compressed, and its size is not
770      * included in the return value.
771      *
772      * @return the on-disk size of the block, not including the header.
773      */
774     public int getOnDiskSizeWithoutHeader() {
775       expectState(State.BLOCK_READY);
776       return onDiskBytesWithHeader.length - HEADER_SIZE;
777     }
778 
779     /**
780      * Returns the on-disk size of the block. Can only be called in the
781      * "block ready" state.
782      *
783      * @return the on-disk size of the block ready to be written, including the
784      *         header size
785      */
786     public int getOnDiskSizeWithHeader() {
787       expectState(State.BLOCK_READY);
788       return onDiskBytesWithHeader.length;
789     }
790 
791     /**
792      * The uncompressed size of the block data. Does not include header size.
793      */
794     public int getUncompressedSizeWithoutHeader() {
795       expectState(State.BLOCK_READY);
796       return uncompressedSizeWithoutHeader;
797     }
798 
799     /**
800      * The uncompressed size of the block data, including header size.
801      */
802     public int getUncompressedSizeWithHeader() {
803       expectState(State.BLOCK_READY);
804       return uncompressedSizeWithoutHeader + HEADER_SIZE;
805     }
806 
807     /** @return true if a block is being written  */
808     public boolean isWriting() {
809       return state == State.WRITING;
810     }
811 
812     /**
813      * Returns the number of bytes written into the current block so far, or
814      * zero if not writing the block at the moment. Note that this will return
815      * zero in the "block ready" state as well.
816      *
817      * @return the number of bytes written
818      */
819     public int blockSizeWritten() {
820       if (state != State.WRITING)
821         return 0;
822       return userDataStream.size();
823     }
824 
825     /**
826      * Returns the header followed by the uncompressed data, even if using
827      * compression. This is needed for storing uncompressed blocks in the block
828      * cache. Can be called in the "writing" state or the "block ready" state.
829      *
830      * @return uncompressed block bytes for caching on write
831      */
832     private byte[] getUncompressedDataWithHeader() {
833       expectState(State.BLOCK_READY);
834 
835       if (compressAlgo == NONE)
836         return onDiskBytesWithHeader;
837 
838       if (!cacheOnWrite)
839         throw new IllegalStateException("Cache-on-write is turned off");
840 
841       if (uncompressedBytesWithHeader == null)
842         throw new NullPointerException();
843 
844       return uncompressedBytesWithHeader;
845     }
846 
847     private void expectState(State expectedState) {
848       if (state != expectedState) {
849         throw new IllegalStateException("Expected state: " + expectedState +
850             ", actual state: " + state);
851       }
852     }
853 
854     /**
855      * Similar to {@link #getUncompressedBufferWithHeader()} but returns a byte
856      * buffer.
857      *
858      * @return uncompressed block for caching on write in the form of a buffer
859      */
860     public ByteBuffer getUncompressedBufferWithHeader() {
861       byte[] b = getUncompressedDataWithHeader();
862       return ByteBuffer.wrap(b, 0, b.length);
863     }
864 
865     /**
866      * Takes the given {@link BlockWritable} instance, creates a new block of
867      * its appropriate type, writes the writable into this block, and flushes
868      * the block into the output stream. The writer is instructed not to buffer
869      * uncompressed bytes for cache-on-write.
870      *
871      * @param bw the block-writable object to write as a block
872      * @param out the file system output stream
873      * @throws IOException
874      */
875     public void writeBlock(BlockWritable bw, FSDataOutputStream out)
876         throws IOException {
877       bw.writeToBlock(startWriting(bw.getBlockType(), false));
878       writeHeaderAndData(out);
879     }
880 
881     public HFileBlock getBlockForCaching() {
882       return new HFileBlock(blockType, onDiskBytesWithHeader.length
883           - HEADER_SIZE, uncompressedSizeWithoutHeader, prevOffset,
884           getUncompressedBufferWithHeader(), false, startOffset);
885     }
886 
887   }
888 
889   /** Something that can be written into a block. */
890   public interface BlockWritable {
891 
892     /** The type of block this data should use. */
893     BlockType getBlockType();
894 
895     /**
896      * Writes the block to the provided stream. Must not write any magic
897      * records.
898      *
899      * @param out a stream to write uncompressed data into
900      */
901     void writeToBlock(DataOutput out) throws IOException;
902   }
903 
904   // Block readers and writers
905 
906   /** An interface allowing to iterate {@link HFileBlock}s. */
907   public interface BlockIterator {
908 
909     /**
910      * Get the next block, or null if there are no more blocks to iterate.
911      */
912     HFileBlock nextBlock() throws IOException;
913 
914     /**
915      * Similar to {@link #nextBlock()} but checks block type, throws an
916      * exception if incorrect, and returns the data portion of the block as
917      * an input stream.
918      */
919     DataInputStream nextBlockAsStream(BlockType blockType) throws IOException;
920   }
921 
922   /** A full-fledged reader with iteration ability. */
923   public interface FSReader {
924 
925     /**
926      * Reads the block at the given offset in the file with the given on-disk
927      * size and uncompressed size.
928      *
929      * @param offset
930      * @param onDiskSize the on-disk size of the entire block, including all
931      *          applicable headers, or -1 if unknown
932      * @param uncompressedSize the uncompressed size of the compressed part of
933      *          the block, or -1 if unknown
934      * @return the newly read block
935      */
936     HFileBlock readBlockData(long offset, long onDiskSize,
937         int uncompressedSize, boolean pread) throws IOException;
938 
939     /**
940      * Creates a block iterator over the given portion of the {@link HFile}.
941      * The iterator returns blocks starting with offset such that offset <=
942      * startOffset < endOffset.
943      *
944      * @param startOffset the offset of the block to start iteration with
945      * @param endOffset the offset to end iteration at (exclusive)
946      * @return an iterator of blocks between the two given offsets
947      */
948     BlockIterator blockRange(long startOffset, long endOffset);
949   }
950 
951   /**
952    * A common implementation of some methods of {@link FSReader} and some
953    * tools for implementing HFile format version-specific block readers.
954    */
955   public abstract static class AbstractFSReader implements FSReader {
956 
957     /** The file system stream of the underlying {@link HFile} */
958     protected FSDataInputStream istream;
959 
960     /** Compression algorithm used by the {@link HFile} */
961     protected Compression.Algorithm compressAlgo;
962 
963     /** The size of the file we are reading from, or -1 if unknown. */
964     protected long fileSize;
965 
966     /** The default buffer size for our buffered streams */
967     public static final int DEFAULT_BUFFER_SIZE = 1 << 20;
968 
969     public AbstractFSReader(FSDataInputStream istream, Algorithm compressAlgo,
970         long fileSize) {
971       this.istream = istream;
972       this.compressAlgo = compressAlgo;
973       this.fileSize = fileSize;
974     }
975 
976     @Override
977     public BlockIterator blockRange(final long startOffset,
978         final long endOffset) {
979       return new BlockIterator() {
980         private long offset = startOffset;
981 
982         @Override
983         public HFileBlock nextBlock() throws IOException {
984           if (offset >= endOffset)
985             return null;
986           HFileBlock b = readBlockData(offset, -1, -1, false);
987           offset += b.getOnDiskSizeWithHeader();
988           return b;
989         }
990 
991         @Override
992         public DataInputStream nextBlockAsStream(BlockType blockType)
993             throws IOException {
994           HFileBlock blk = nextBlock();
995           if (blk.getBlockType() != blockType) {
996             throw new IOException("Expected block of type " + blockType
997                 + " but found " + blk.getBlockType());
998           }
999           return blk.getByteStream();
1000         }
1001       };
1002     }
1003 
1004     /**
1005      * Does a positional read or a seek and read into the given buffer. Returns
1006      * the on-disk size of the next block, or -1 if it could not be determined.
1007      *
1008      * @param dest destination buffer
1009      * @param destOffset offset in the destination buffer
1010      * @param size size of the block to be read
1011      * @param peekIntoNextBlock whether to read the next block's on-disk size
1012      * @param fileOffset position in the stream to read at
1013      * @param pread whether we should do a positional read
1014      * @return the on-disk size of the next block with header size included, or
1015      *         -1 if it could not be determined
1016      * @throws IOException
1017      */
1018     protected int readAtOffset(byte[] dest, int destOffset, int size,
1019         boolean peekIntoNextBlock, long fileOffset, boolean pread)
1020         throws IOException {
1021       if (peekIntoNextBlock &&
1022           destOffset + size + HEADER_SIZE > dest.length) {
1023         // We are asked to read the next block's header as well, but there is
1024         // not enough room in the array.
1025         throw new IOException("Attempted to read " + size + " bytes and " +
1026             HEADER_SIZE + " bytes of next header into a " + dest.length +
1027             "-byte array at offset " + destOffset);
1028       }
1029 
1030       if (pread) {
1031         // Positional read. Better for random reads.
1032         int extraSize = peekIntoNextBlock ? HEADER_SIZE : 0;
1033 
1034         int ret = istream.read(fileOffset, dest, destOffset, size + extraSize);
1035         if (ret < size) {
1036           throw new IOException("Positional read of " + size + " bytes " +
1037               "failed at offset " + fileOffset + " (returned " + ret + ")");
1038         }
1039 
1040         if (ret == size || ret < size + extraSize) {
1041           // Could not read the next block's header, or did not try.
1042           return -1;
1043         }
1044       } else {
1045         // Seek + read. Better for scanning.
1046         synchronized (istream) {
1047           istream.seek(fileOffset);
1048 
1049           long realOffset = istream.getPos();
1050           if (realOffset != fileOffset) {
1051             throw new IOException("Tried to seek to " + fileOffset + " to "
1052                 + "read " + size + " bytes, but pos=" + realOffset
1053                 + " after seek");
1054           }
1055 
1056           if (!peekIntoNextBlock) {
1057             IOUtils.readFully(istream, dest, destOffset, size);
1058             return -1;
1059           }
1060 
1061           // Try to read the next block header.
1062           if (!readWithExtra(istream, dest, destOffset, size, HEADER_SIZE))
1063             return -1;
1064         }
1065       }
1066 
1067       assert peekIntoNextBlock;
1068       return Bytes.toInt(dest, destOffset + size + BlockType.MAGIC_LENGTH) +
1069           HEADER_SIZE;
1070     }
1071 
1072     /**
1073      * Decompresses data from the given stream using the configured compression
1074      * algorithm.
1075      * @param dest
1076      * @param destOffset
1077      * @param bufferedBoundedStream
1078      *          a stream to read compressed data from, bounded to the exact
1079      *          amount of compressed data
1080      * @param compressedSize
1081      *          compressed data size, header not included
1082      * @param uncompressedSize
1083      *          uncompressed data size, header not included
1084      * @throws IOException
1085      */
1086     protected void decompress(byte[] dest, int destOffset,
1087         InputStream bufferedBoundedStream, int compressedSize,
1088         int uncompressedSize) throws IOException {
1089       Decompressor decompressor = null;
1090       try {
1091         decompressor = compressAlgo.getDecompressor();
1092         InputStream is = compressAlgo.createDecompressionStream(
1093             bufferedBoundedStream, decompressor, 0);
1094 
1095         IOUtils.readFully(is, dest, destOffset, uncompressedSize);
1096         is.close();
1097       } finally {
1098         if (decompressor != null) {
1099           compressAlgo.returnDecompressor(decompressor);
1100         }
1101       }
1102     }
1103 
1104     /**
1105      * Creates a buffered stream reading a certain slice of the file system
1106      * input stream. We need this because the decompression we use seems to
1107      * expect the input stream to be bounded.
1108      *
1109      * @param offset the starting file offset the bounded stream reads from
1110      * @param size the size of the segment of the file the stream should read
1111      * @param pread whether to use position reads
1112      * @return a stream restricted to the given portion of the file
1113      */
1114     protected InputStream createBufferedBoundedStream(long offset,
1115         int size, boolean pread) {
1116       return new BufferedInputStream(new BoundedRangeFileInputStream(istream,
1117           offset, size, pread), Math.min(DEFAULT_BUFFER_SIZE, size));
1118     }
1119 
1120   }
1121 
1122   /**
1123    * Reads version 1 blocks from the file system. In version 1 blocks,
1124    * everything is compressed, including the magic record, if compression is
1125    * enabled. Everything might be uncompressed if no compression is used. This
1126    * reader returns blocks represented in the uniform version 2 format in
1127    * memory.
1128    */
1129   public static class FSReaderV1 extends AbstractFSReader {
1130 
1131     /** Header size difference between version 1 and 2 */
1132     private static final int HEADER_DELTA = HEADER_SIZE - MAGIC_LENGTH;
1133 
1134     public FSReaderV1(FSDataInputStream istream, Algorithm compressAlgo,
1135         long fileSize) {
1136       super(istream, compressAlgo, fileSize);
1137     }
1138 
1139     /**
1140      * Read a version 1 block. There is no uncompressed header, and the block
1141      * type (the magic record) is part of the compressed data. This
1142      * implementation assumes that the bounded range file input stream is
1143      * needed to stop the decompressor reading into next block, because the
1144      * decompressor just grabs a bunch of data without regard to whether it is
1145      * coming to end of the compressed section.
1146      *
1147      * The block returned is still a version 2 block, and in particular, its
1148      * first {@link #HEADER_SIZE} bytes contain a valid version 2 header.
1149      *
1150      * @param offset the offset of the block to read in the file
1151      * @param onDiskSizeWithMagic the on-disk size of the version 1 block,
1152      *          including the magic record, which is the part of compressed
1153      *          data if using compression
1154      * @param uncompressedSizeWithMagic uncompressed size of the version 1
1155      *          block, including the magic record
1156      */
1157     @Override
1158     public HFileBlock readBlockData(long offset, long onDiskSizeWithMagic,
1159         int uncompressedSizeWithMagic, boolean pread) throws IOException {
1160       if (uncompressedSizeWithMagic <= 0) {
1161         throw new IOException("Invalid uncompressedSize="
1162             + uncompressedSizeWithMagic + " for a version 1 block");
1163       }
1164 
1165       if (onDiskSizeWithMagic <= 0 || onDiskSizeWithMagic >= Integer.MAX_VALUE)
1166       {
1167         throw new IOException("Invalid onDiskSize=" + onDiskSizeWithMagic
1168             + " (maximum allowed: " + Integer.MAX_VALUE + ")");
1169       }
1170 
1171       int onDiskSize = (int) onDiskSizeWithMagic;
1172 
1173       if (uncompressedSizeWithMagic < MAGIC_LENGTH) {
1174         throw new IOException("Uncompressed size for a version 1 block is "
1175             + uncompressedSizeWithMagic + " but must be at least "
1176             + MAGIC_LENGTH);
1177       }
1178 
1179       // The existing size already includes magic size, and we are inserting
1180       // a version 2 header.
1181       ByteBuffer buf = ByteBuffer.allocate(uncompressedSizeWithMagic
1182           + HEADER_DELTA);
1183 
1184       int onDiskSizeWithoutHeader;
1185       if (compressAlgo == Compression.Algorithm.NONE) {
1186         // A special case when there is no compression.
1187         if (onDiskSize != uncompressedSizeWithMagic) {
1188           throw new IOException("onDiskSize=" + onDiskSize
1189               + " and uncompressedSize=" + uncompressedSizeWithMagic
1190               + " must be equal for version 1 with no compression");
1191         }
1192 
1193         // The first MAGIC_LENGTH bytes of what this will read will be
1194         // overwritten.
1195         readAtOffset(buf.array(), buf.arrayOffset() + HEADER_DELTA,
1196             onDiskSize, false, offset, pread);
1197 
1198         onDiskSizeWithoutHeader = uncompressedSizeWithMagic - MAGIC_LENGTH;
1199       } else {
1200         InputStream bufferedBoundedStream = createBufferedBoundedStream(
1201             offset, onDiskSize, pread);
1202         decompress(buf.array(), buf.arrayOffset() + HEADER_DELTA,
1203             bufferedBoundedStream, onDiskSize, uncompressedSizeWithMagic);
1204 
1205         // We don't really have a good way to exclude the "magic record" size
1206         // from the compressed block's size, since it is compressed as well.
1207         onDiskSizeWithoutHeader = onDiskSize;
1208       }
1209 
1210       BlockType newBlockType = BlockType.parse(buf.array(), buf.arrayOffset()
1211           + HEADER_DELTA, MAGIC_LENGTH);
1212 
1213       // We set the uncompressed size of the new HFile block we are creating
1214       // to the size of the data portion of the block without the magic record,
1215       // since the magic record gets moved to the header.
1216       HFileBlock b = new HFileBlock(newBlockType, onDiskSizeWithoutHeader,
1217           uncompressedSizeWithMagic - MAGIC_LENGTH, -1L, buf, true, offset);
1218       return b;
1219     }
1220   }
1221 
1222   /**
1223    * We always prefetch the header of the next block, so that we know its
1224    * on-disk size in advance and can read it in one operation.
1225    */
1226   private static class PrefetchedHeader {
1227     long offset = -1;
1228     byte[] header = new byte[HEADER_SIZE];
1229     ByteBuffer buf = ByteBuffer.wrap(header, 0, HEADER_SIZE);
1230   }
1231 
1232   /** Reads version 2 blocks from the filesystem. */
1233   public static class FSReaderV2 extends AbstractFSReader {
1234 
1235     private ThreadLocal<PrefetchedHeader> prefetchedHeaderForThread =
1236         new ThreadLocal<PrefetchedHeader>() {
1237           @Override
1238           public PrefetchedHeader initialValue() {
1239             return new PrefetchedHeader();
1240           }
1241         };
1242 
1243     public FSReaderV2(FSDataInputStream istream, Algorithm compressAlgo,
1244         long fileSize) {
1245       super(istream, compressAlgo, fileSize);
1246     }
1247 
1248     /**
1249      * Reads a version 2 block. Tries to do as little memory allocation as
1250      * possible, using the provided on-disk size.
1251      *
1252      * @param offset the offset in the stream to read at
1253      * @param onDiskSizeWithHeaderL the on-disk size of the block, including
1254      *          the header, or -1 if unknown
1255      * @param uncompressedSize the uncompressed size of the the block. Always
1256      *          expected to be -1. This parameter is only used in version 1.
1257      * @param pread whether to use a positional read
1258      */
1259     @Override
1260     public HFileBlock readBlockData(long offset, long onDiskSizeWithHeaderL,
1261         int uncompressedSize, boolean pread) throws IOException {
1262       if (offset < 0) {
1263         throw new IOException("Invalid offset=" + offset + " trying to read "
1264             + "block (onDiskSize=" + onDiskSizeWithHeaderL
1265             + ", uncompressedSize=" + uncompressedSize + ")");
1266       }
1267       if (uncompressedSize != -1) {
1268         throw new IOException("Version 2 block reader API does not need " +
1269             "the uncompressed size parameter");
1270       }
1271 
1272       if ((onDiskSizeWithHeaderL < HEADER_SIZE && onDiskSizeWithHeaderL != -1)
1273           || onDiskSizeWithHeaderL >= Integer.MAX_VALUE) {
1274         throw new IOException("Invalid onDisksize=" + onDiskSizeWithHeaderL
1275             + ": expected to be at least " + HEADER_SIZE
1276             + " and at most " + Integer.MAX_VALUE + ", or -1 (offset="
1277             + offset + ", uncompressedSize=" + uncompressedSize + ")");
1278       }
1279 
1280       int onDiskSizeWithHeader = (int) onDiskSizeWithHeaderL;
1281 
1282       HFileBlock b;
1283       if (onDiskSizeWithHeader > 0) {
1284         // We know the total on-disk size but not the uncompressed size. Read
1285         // the entire block into memory, then parse the header and decompress
1286         // from memory if using compression. This code path is used when
1287         // doing a random read operation relying on the block index, as well as
1288         // when the client knows the on-disk size from peeking into the next
1289         // block's header (e.g. this block's header) when reading the previous
1290         // block. This is the faster and more preferable case.
1291 
1292         int onDiskSizeWithoutHeader = onDiskSizeWithHeader - HEADER_SIZE;
1293         assert onDiskSizeWithoutHeader >= 0;
1294 
1295         // See if we can avoid reading the header. This is desirable, because
1296         // we will not incur a seek operation to seek back if we have already
1297         // read this block's header as part of the previous read's look-ahead.
1298         PrefetchedHeader prefetchedHeader = prefetchedHeaderForThread.get();
1299         byte[] header = prefetchedHeader.offset == offset
1300             ? prefetchedHeader.header : null;
1301 
1302         // Size that we have to skip in case we have already read the header.
1303         int preReadHeaderSize = header == null ? 0 : HEADER_SIZE;
1304 
1305         if (compressAlgo == Compression.Algorithm.NONE) {
1306           // Just read the whole thing. Allocate enough space to read the
1307           // next block's header too.
1308 
1309           ByteBuffer headerAndData = ByteBuffer.allocate(onDiskSizeWithHeader
1310               + HEADER_SIZE);
1311           headerAndData.limit(onDiskSizeWithHeader);
1312 
1313           if (header != null) {
1314             System.arraycopy(header, 0, headerAndData.array(), 0,
1315                 HEADER_SIZE);
1316           }
1317 
1318           int nextBlockOnDiskSizeWithHeader = readAtOffset(
1319               headerAndData.array(), headerAndData.arrayOffset()
1320                   + preReadHeaderSize, onDiskSizeWithHeader
1321                   - preReadHeaderSize, true, offset + preReadHeaderSize,
1322                   pread);
1323 
1324           b = new HFileBlock(headerAndData);
1325           b.assumeUncompressed();
1326           b.validateOnDiskSizeWithoutHeader(onDiskSizeWithoutHeader);
1327           b.nextBlockOnDiskSizeWithHeader = nextBlockOnDiskSizeWithHeader;
1328 
1329           if (b.nextBlockOnDiskSizeWithHeader > 0)
1330             setNextBlockHeader(offset, b);
1331         } else {
1332           // Allocate enough space to fit the next block's header too.
1333           byte[] onDiskBlock = new byte[onDiskSizeWithHeader + HEADER_SIZE];
1334 
1335           int nextBlockOnDiskSize = readAtOffset(onDiskBlock,
1336               preReadHeaderSize, onDiskSizeWithHeader - preReadHeaderSize,
1337               true, offset + preReadHeaderSize, pread);
1338 
1339           if (header == null)
1340             header = onDiskBlock;
1341 
1342           try {
1343             b = new HFileBlock(ByteBuffer.wrap(header, 0, HEADER_SIZE));
1344           } catch (IOException ex) {
1345             // Seen in load testing. Provide comprehensive debug info.
1346             throw new IOException("Failed to read compressed block at "
1347                 + offset + ", onDiskSizeWithoutHeader=" + onDiskSizeWithHeader
1348                 + ", preReadHeaderSize=" + preReadHeaderSize
1349                 + ", header.length=" + header.length + ", header bytes: "
1350                 + Bytes.toStringBinary(header, 0, HEADER_SIZE), ex);
1351           }
1352           b.validateOnDiskSizeWithoutHeader(onDiskSizeWithoutHeader);
1353           b.nextBlockOnDiskSizeWithHeader = nextBlockOnDiskSize;
1354 
1355           DataInputStream dis = new DataInputStream(new ByteArrayInputStream(
1356               onDiskBlock, HEADER_SIZE, onDiskSizeWithoutHeader));
1357 
1358           // This will allocate a new buffer but keep header bytes.
1359           b.allocateBuffer(b.nextBlockOnDiskSizeWithHeader > 0);
1360 
1361           decompress(b.buf.array(), b.buf.arrayOffset() + HEADER_SIZE, dis,
1362               onDiskSizeWithoutHeader, b.uncompressedSizeWithoutHeader);
1363 
1364           // Copy next block's header bytes into the new block if we have them.
1365           if (nextBlockOnDiskSize > 0) {
1366             System.arraycopy(onDiskBlock, onDiskSizeWithHeader, b.buf.array(),
1367                 b.buf.arrayOffset() + HEADER_SIZE
1368                     + b.uncompressedSizeWithoutHeader, HEADER_SIZE);
1369 
1370             setNextBlockHeader(offset, b);
1371           }
1372         }
1373 
1374       } else {
1375         // We don't know the on-disk size. Read the header first, determine the
1376         // on-disk size from it, and read the remaining data, thereby incurring
1377         // two read operations. This might happen when we are doing the first
1378         // read in a series of reads or a random read, and we don't have access
1379         // to the block index. This is costly and should happen very rarely.
1380 
1381         // Check if we have read this block's header as part of reading the
1382         // previous block. If so, don't read the header again.
1383         PrefetchedHeader prefetchedHeader = prefetchedHeaderForThread.get();
1384         ByteBuffer headerBuf = prefetchedHeader.offset == offset ?
1385             prefetchedHeader.buf : null;
1386 
1387         if (headerBuf == null) {
1388           // Unfortunately, we still have to do a separate read operation to
1389           // read the header.
1390           headerBuf = ByteBuffer.allocate(HEADER_SIZE);;
1391           readAtOffset(headerBuf.array(), headerBuf.arrayOffset(), HEADER_SIZE,
1392               false, offset, pread);
1393         }
1394 
1395         b = new HFileBlock(headerBuf);
1396 
1397         // This will also allocate enough room for the next block's header.
1398         b.allocateBuffer(true);
1399 
1400         if (compressAlgo == Compression.Algorithm.NONE) {
1401 
1402           // Avoid creating bounded streams and using a "codec" that does
1403           // nothing.
1404           b.assumeUncompressed();
1405           b.nextBlockOnDiskSizeWithHeader = readAtOffset(b.buf.array(),
1406               b.buf.arrayOffset() + HEADER_SIZE,
1407               b.uncompressedSizeWithoutHeader, true, offset + HEADER_SIZE,
1408               pread);
1409 
1410           if (b.nextBlockOnDiskSizeWithHeader > 0) {
1411             setNextBlockHeader(offset, b);
1412           }
1413         } else {
1414           // Allocate enough space for the block's header and compressed data.
1415           byte[] compressedBytes = new byte[b.getOnDiskSizeWithHeader()
1416               + HEADER_SIZE];
1417 
1418           b.nextBlockOnDiskSizeWithHeader = readAtOffset(compressedBytes,
1419               HEADER_SIZE, b.onDiskSizeWithoutHeader, true, offset
1420                   + HEADER_SIZE, pread);
1421           DataInputStream dis = new DataInputStream(new ByteArrayInputStream(
1422               compressedBytes, HEADER_SIZE, b.onDiskSizeWithoutHeader));
1423 
1424           decompress(b.buf.array(), b.buf.arrayOffset() + HEADER_SIZE, dis,
1425               b.onDiskSizeWithoutHeader, b.uncompressedSizeWithoutHeader);
1426 
1427           if (b.nextBlockOnDiskSizeWithHeader > 0) {
1428             // Copy the next block's header into the new block.
1429             int nextHeaderOffset = b.buf.arrayOffset() + HEADER_SIZE
1430                 + b.uncompressedSizeWithoutHeader;
1431             System.arraycopy(compressedBytes,
1432                 compressedBytes.length - HEADER_SIZE,
1433                 b.buf.array(),
1434                 nextHeaderOffset,
1435                 HEADER_SIZE);
1436 
1437             setNextBlockHeader(offset, b);
1438           }
1439         }
1440       }
1441 
1442       b.offset = offset;
1443       return b;
1444     }
1445 
1446     private void setNextBlockHeader(long offset, HFileBlock b) {
1447       PrefetchedHeader prefetchedHeader = prefetchedHeaderForThread.get();
1448       prefetchedHeader.offset = offset + b.getOnDiskSizeWithHeader();
1449       int nextHeaderOffset = b.buf.arrayOffset() + HEADER_SIZE
1450           + b.uncompressedSizeWithoutHeader;
1451       System.arraycopy(b.buf.array(), nextHeaderOffset,
1452           prefetchedHeader.header, 0, HEADER_SIZE);
1453     }
1454 
1455   }
1456 
1457   @Override
1458   public int getSerializedLength() {
1459     if (buf != null) {
1460       return this.buf.limit() + HFileBlock.EXTRA_SERIALIZATION_SPACE;
1461     }
1462     return 0;
1463   }
1464 
1465   @Override
1466   public void serialize(ByteBuffer destination) {
1467     destination.put(this.buf.duplicate());
1468     destination.putLong(this.offset);
1469     destination.putInt(this.nextBlockOnDiskSizeWithHeader);
1470     destination.rewind();
1471   }
1472 
1473   @Override
1474   public CacheableDeserializer<Cacheable> getDeserializer() {
1475     return HFileBlock.blockDeserializer;
1476   }
1477 
1478   @Override
1479   public boolean equals(Object comparison) {
1480     if (this == comparison) {
1481       return true;
1482     }
1483     if (comparison == null) {
1484       return false;
1485     }
1486     if (comparison.getClass() != this.getClass()) {
1487       return false;
1488     }
1489 
1490     HFileBlock castedComparison = (HFileBlock) comparison;
1491 
1492     if (castedComparison.blockType != this.blockType) {
1493       return false;
1494     }
1495     if (castedComparison.nextBlockOnDiskSizeWithHeader != this.nextBlockOnDiskSizeWithHeader) {
1496       return false;
1497     }
1498     if (castedComparison.offset != this.offset) {
1499       return false;
1500     }
1501     if (castedComparison.onDiskSizeWithoutHeader != this.onDiskSizeWithoutHeader) {
1502       return false;
1503     }
1504     if (castedComparison.prevBlockOffset != this.prevBlockOffset) {
1505       return false;
1506     }
1507     if (castedComparison.uncompressedSizeWithoutHeader != this.uncompressedSizeWithoutHeader) {
1508       return false;
1509     }
1510     if (this.buf.compareTo(castedComparison.buf) != 0) {
1511       return false;
1512     }
1513     if (this.buf.position() != castedComparison.buf.position()){
1514       return false;
1515     }
1516     if (this.buf.limit() != castedComparison.buf.limit()){
1517       return false;
1518     }
1519     return true;
1520   }
1521 
1522 
1523 }