1   /*
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.io.hfile;
20  
21  import java.io.ByteArrayOutputStream;
22  import java.io.DataInput;
23  import java.io.DataInputStream;
24  import java.io.DataOutput;
25  import java.io.DataOutputStream;
26  import java.io.IOException;
27  import java.nio.ByteBuffer;
28  import java.util.ArrayList;
29  import java.util.Arrays;
30  import java.util.Collections;
31  import java.util.List;
32  import java.util.concurrent.atomic.AtomicReference;
33  
34  import org.apache.commons.logging.Log;
35  import org.apache.commons.logging.LogFactory;
36  import org.apache.hadoop.classification.InterfaceAudience;
37  import org.apache.hadoop.conf.Configuration;
38  import org.apache.hadoop.fs.FSDataOutputStream;
39  import org.apache.hadoop.hbase.HConstants;
40  import org.apache.hadoop.hbase.KeyValue;
41  import org.apache.hadoop.hbase.io.HeapSize;
42  import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
43  import org.apache.hadoop.hbase.io.hfile.HFile.CachingBlockReader;
44  import org.apache.hadoop.hbase.util.Bytes;
45  import org.apache.hadoop.hbase.util.ClassSize;
46  import org.apache.hadoop.hbase.util.CompoundBloomFilterWriter;
47  import org.apache.hadoop.io.RawComparator;
48  import org.apache.hadoop.io.WritableUtils;
49  import org.apache.hadoop.util.StringUtils;
50  
51  /**
52   * Provides functionality to write ({@link BlockIndexWriter}) and read
53   * ({@link BlockIndexReader}) single-level and multi-level block indexes.
54   *
55   * Examples of how to use the block index writer can be found in
56   * {@link CompoundBloomFilterWriter} and {@link HFileWriterV2}. Examples of how
57   * to use the reader can be found in {@link HFileReaderV2} and
58   * TestHFileBlockIndex.
59   */
60  @InterfaceAudience.Private
61  public class HFileBlockIndex {
62  
63    private static final Log LOG = LogFactory.getLog(HFileBlockIndex.class);
64  
65    static final int DEFAULT_MAX_CHUNK_SIZE = 128 * 1024;
66  
67    /**
68     * The maximum size guideline for index blocks (both leaf, intermediate, and
69     * root). If not specified, <code>DEFAULT_MAX_CHUNK_SIZE</code> is used.
70     */
71    public static final String MAX_CHUNK_SIZE_KEY = "hfile.index.block.max.size";
72  
73    /**
74     * The number of bytes stored in each "secondary index" entry in addition to
75     * key bytes in the non-root index block format. The first long is the file
76     * offset of the deeper-level block the entry points to, and the int that
77     * follows is that block's on-disk size without including header.
78     */
79    static final int SECONDARY_INDEX_ENTRY_OVERHEAD = Bytes.SIZEOF_INT
80        + Bytes.SIZEOF_LONG;
81  
82    /**
83     * Error message when trying to use inline block API in single-level mode.
84     */
85    private static final String INLINE_BLOCKS_NOT_ALLOWED =
86        "Inline blocks are not allowed in the single-level-only mode";
87  
88    /**
89     * The size of a meta-data record used for finding the mid-key in a
90     * multi-level index. Consists of the middle leaf-level index block offset
91     * (long), its on-disk size without header included (int), and the mid-key
92     * entry's zero-based index in that leaf index block.
93     */
94    private static final int MID_KEY_METADATA_SIZE = Bytes.SIZEOF_LONG +
95        2 * Bytes.SIZEOF_INT;
96  
97    /**
98     * The reader will always hold the root level index in the memory. Index
99     * blocks at all other levels will be cached in the LRU cache in practice,
100    * although this API does not enforce that.
101    *
102    * All non-root (leaf and intermediate) index blocks contain what we call a
103    * "secondary index": an array of offsets to the entries within the block.
104    * This allows us to do binary search for the entry corresponding to the
105    * given key without having to deserialize the block.
106    */
107   public static class BlockIndexReader implements HeapSize {
108     /** Needed doing lookup on blocks. */
109     private final RawComparator<byte[]> comparator;
110 
111     // Root-level data.
112     private byte[][] blockKeys;
113     private long[] blockOffsets;
114     private int[] blockDataSizes;
115     private int rootByteSize = 0;
116     private int rootCount = 0;
117 
118     // Mid-key metadata.
119     private long midLeafBlockOffset = -1;
120     private int midLeafBlockOnDiskSize = -1;
121     private int midKeyEntry = -1;
122 
123     /** Pre-computed mid-key */
124     private AtomicReference<byte[]> midKey = new AtomicReference<byte[]>();
125 
126     /**
127      * The number of levels in the block index tree. One if there is only root
128      * level, two for root and leaf levels, etc.
129      */
130     private int searchTreeLevel;
131 
132     /** A way to read {@link HFile} blocks at a given offset */
133     private CachingBlockReader cachingBlockReader;
134 
135     public BlockIndexReader(final RawComparator<byte[]> c, final int treeLevel,
136         final CachingBlockReader cachingBlockReader) {
137       this(c, treeLevel);
138       this.cachingBlockReader = cachingBlockReader;
139     }
140 
141     public BlockIndexReader(final RawComparator<byte[]> c, final int treeLevel)
142     {
143       comparator = c;
144       searchTreeLevel = treeLevel;
145     }
146 
147     /**
148      * @return true if the block index is empty.
149      */
150     public boolean isEmpty() {
151       return blockKeys.length == 0;
152     }
153 
154     /**
155      * Verifies that the block index is non-empty and throws an
156      * {@link IllegalStateException} otherwise.
157      */
158     public void ensureNonEmpty() {
159       if (blockKeys.length == 0) {
160         throw new IllegalStateException("Block index is empty or not loaded");
161       }
162     }
163 
164     /**
165      * Return the data block which contains this key. This function will only
166      * be called when the HFile version is larger than 1.
167      *
168      * @param key the key we are looking for
169      * @param keyOffset the offset of the key in its byte array
170      * @param keyLength the length of the key
171      * @param currentBlock the current block, to avoid re-reading the same
172      *          block
173      * @return reader a basic way to load blocks
174      * @throws IOException
175      */
176     public HFileBlock seekToDataBlock(final byte[] key, int keyOffset,
177         int keyLength, HFileBlock currentBlock, boolean cacheBlocks,
178         boolean pread, boolean isCompaction)
179         throws IOException {
180       BlockWithScanInfo blockWithScanInfo = loadDataBlockWithScanInfo(key, keyOffset, keyLength,
181           currentBlock, cacheBlocks, pread, isCompaction);
182       if (blockWithScanInfo == null) {
183         return null;
184       } else {
185         return blockWithScanInfo.getHFileBlock();
186       }
187     }
188 
189     /**
190      * Return the BlockWithScanInfo which contains the DataBlock with other scan info
191      * such as nextIndexedKey.
192      * This function will only be called when the HFile version is larger than 1.
193      *
194      * @param key the key we are looking for
195      * @param keyOffset the offset of the key in its byte array
196      * @param keyLength the length of the key
197      * @param currentBlock the current block, to avoid re-reading the same
198      *          block
199      * @param cacheBlocks
200      * @param pread
201      * @param isCompaction
202      * @return the BlockWithScanInfo which contains the DataBlock with other scan info
203      *         such as nextIndexedKey.
204      * @throws IOException
205      */
206     public BlockWithScanInfo loadDataBlockWithScanInfo(final byte[] key, int keyOffset,
207         int keyLength, HFileBlock currentBlock, boolean cacheBlocks,
208         boolean pread, boolean isCompaction)
209         throws IOException {
210       int rootLevelIndex = rootBlockContainingKey(key, keyOffset, keyLength);
211       if (rootLevelIndex < 0 || rootLevelIndex >= blockOffsets.length) {
212         return null;
213       }
214 
215       // the next indexed key
216       byte[] nextIndexedKey = null;
217 
218       // Read the next-level (intermediate or leaf) index block.
219       long currentOffset = blockOffsets[rootLevelIndex];
220       int currentOnDiskSize = blockDataSizes[rootLevelIndex];
221 
222       if (rootLevelIndex < blockKeys.length - 1) {
223         nextIndexedKey = blockKeys[rootLevelIndex + 1];
224       } else {
225         nextIndexedKey = HConstants.NO_NEXT_INDEXED_KEY;
226       }
227 
228       int lookupLevel = 1; // How many levels deep we are in our lookup.
229       int index = -1;
230 
231       HFileBlock block;
232       while (true) {
233 
234         if (currentBlock != null && currentBlock.getOffset() == currentOffset)
235         {
236           // Avoid reading the same block again, even with caching turned off.
237           // This is crucial for compaction-type workload which might have
238           // caching turned off. This is like a one-block cache inside the
239           // scanner.
240           block = currentBlock;
241         } else {
242           // Call HFile's caching block reader API. We always cache index
243           // blocks, otherwise we might get terrible performance.
244           boolean shouldCache = cacheBlocks || (lookupLevel < searchTreeLevel);
245           BlockType expectedBlockType;
246           if (lookupLevel < searchTreeLevel - 1) {
247             expectedBlockType = BlockType.INTERMEDIATE_INDEX;
248           } else if (lookupLevel == searchTreeLevel - 1) {
249             expectedBlockType = BlockType.LEAF_INDEX;
250           } else {
251             // this also accounts for ENCODED_DATA
252             expectedBlockType = BlockType.DATA;
253           }
254           block = cachingBlockReader.readBlock(currentOffset,
255               currentOnDiskSize, shouldCache, pread, isCompaction,
256               expectedBlockType);
257         }
258 
259         if (block == null) {
260           throw new IOException("Failed to read block at offset " +
261               currentOffset + ", onDiskSize=" + currentOnDiskSize);
262         }
263 
264         // Found a data block, break the loop and check our level in the tree.
265         if (block.getBlockType().equals(BlockType.DATA) ||
266             block.getBlockType().equals(BlockType.ENCODED_DATA)) {
267           break;
268         }
269 
270         // Not a data block. This must be a leaf-level or intermediate-level
271         // index block. We don't allow going deeper than searchTreeLevel.
272         if (++lookupLevel > searchTreeLevel) {
273           throw new IOException("Search Tree Level overflow: lookupLevel="+
274               lookupLevel + ", searchTreeLevel=" + searchTreeLevel);
275         }
276 
277         // Locate the entry corresponding to the given key in the non-root
278         // (leaf or intermediate-level) index block.
279         ByteBuffer buffer = block.getBufferWithoutHeader();
280         index = locateNonRootIndexEntry(buffer, key, keyOffset, keyLength, comparator);
281         if (index == -1) {
282           throw new IOException("The key "
283               + Bytes.toStringBinary(key, keyOffset, keyLength)
284               + " is before the" + " first key of the non-root index block "
285               + block);
286         }
287 
288         currentOffset = buffer.getLong();
289         currentOnDiskSize = buffer.getInt();
290 
291         // Only update next indexed key if there is a next indexed key in the current level
292         byte[] tmpNextIndexedKey = getNonRootIndexedKey(buffer, index + 1);
293         if (tmpNextIndexedKey != null) {
294           nextIndexedKey = tmpNextIndexedKey;
295         }
296       }
297 
298       if (lookupLevel != searchTreeLevel) {
299         throw new IOException("Reached a data block at level " + lookupLevel +
300             " but the number of levels is " + searchTreeLevel);
301       }
302 
303       // set the next indexed key for the current block.
304       BlockWithScanInfo blockWithScanInfo = new BlockWithScanInfo(block, nextIndexedKey);
305       return blockWithScanInfo;
306     }
307 
308     /**
309      * An approximation to the {@link HFile}'s mid-key. Operates on block
310      * boundaries, and does not go inside blocks. In other words, returns the
311      * first key of the middle block of the file.
312      *
313      * @return the first key of the middle block
314      */
315     public byte[] midkey() throws IOException {
316       if (rootCount == 0)
317         throw new IOException("HFile empty");
318 
319       byte[] targetMidKey = this.midKey.get();
320       if (targetMidKey != null) {
321         return targetMidKey;
322       }
323 
324       if (midLeafBlockOffset >= 0) {
325         if (cachingBlockReader == null) {
326           throw new IOException("Have to read the middle leaf block but " +
327               "no block reader available");
328         }
329 
330         // Caching, using pread, assuming this is not a compaction.
331         HFileBlock midLeafBlock = cachingBlockReader.readBlock(
332             midLeafBlockOffset, midLeafBlockOnDiskSize, true, true, false,
333             BlockType.LEAF_INDEX);
334 
335         ByteBuffer b = midLeafBlock.getBufferWithoutHeader();
336         int numDataBlocks = b.getInt();
337         int keyRelOffset = b.getInt(Bytes.SIZEOF_INT * (midKeyEntry + 1));
338         int keyLen = b.getInt(Bytes.SIZEOF_INT * (midKeyEntry + 2)) -
339             keyRelOffset;
340         int keyOffset = b.arrayOffset() +
341             Bytes.SIZEOF_INT * (numDataBlocks + 2) + keyRelOffset +
342             SECONDARY_INDEX_ENTRY_OVERHEAD;
343         targetMidKey = Arrays.copyOfRange(b.array(), keyOffset, keyOffset + keyLen);
344       } else {
345         // The middle of the root-level index.
346         targetMidKey = blockKeys[rootCount / 2];
347       }
348 
349       this.midKey.set(targetMidKey);
350       return targetMidKey;
351     }
352 
353     /**
354      * @param i from 0 to {@link #getRootBlockCount() - 1}
355      */
356     public byte[] getRootBlockKey(int i) {
357       return blockKeys[i];
358     }
359 
360     /**
361      * @param i from 0 to {@link #getRootBlockCount() - 1}
362      */
363     public long getRootBlockOffset(int i) {
364       return blockOffsets[i];
365     }
366 
367     /**
368      * @param i zero-based index of a root-level block
369      * @return the on-disk size of the root-level block for version 2, or the
370      *         uncompressed size for version 1
371      */
372     public int getRootBlockDataSize(int i) {
373       return blockDataSizes[i];
374     }
375 
376     /**
377      * @return the number of root-level blocks in this block index
378      */
379     public int getRootBlockCount() {
380       return rootCount;
381     }
382 
383     /**
384      * Finds the root-level index block containing the given key.
385      *
386      * @param key
387      *          Key to find
388      * @return Offset of block containing <code>key</code> (between 0 and the
389      *         number of blocks - 1) or -1 if this file does not contain the
390      *         request.
391      */
392     public int rootBlockContainingKey(final byte[] key, int offset,
393         int length) {
394       int pos = Bytes.binarySearch(blockKeys, key, offset, length,
395           comparator);
396       // pos is between -(blockKeys.length + 1) to blockKeys.length - 1, see
397       // binarySearch's javadoc.
398 
399       if (pos >= 0) {
400         // This means this is an exact match with an element of blockKeys.
401         assert pos < blockKeys.length;
402         return pos;
403       }
404 
405       // Otherwise, pos = -(i + 1), where blockKeys[i - 1] < key < blockKeys[i],
406       // and i is in [0, blockKeys.length]. We are returning j = i - 1 such that
407       // blockKeys[j] <= key < blockKeys[j + 1]. In particular, j = -1 if
408       // key < blockKeys[0], meaning the file does not contain the given key.
409 
410       int i = -pos - 1;
411       assert 0 <= i && i <= blockKeys.length;
412       return i - 1;
413     }
414 
415     /**
416      * Adds a new entry in the root block index. Only used when reading.
417      *
418      * @param key Last key in the block
419      * @param offset file offset where the block is stored
420      * @param dataSize the uncompressed data size
421      */
422     private void add(final byte[] key, final long offset, final int dataSize) {
423       blockOffsets[rootCount] = offset;
424       blockKeys[rootCount] = key;
425       blockDataSizes[rootCount] = dataSize;
426 
427       rootCount++;
428       rootByteSize += SECONDARY_INDEX_ENTRY_OVERHEAD + key.length;
429     }
430 
431     /**
432      * The indexed key at the ith position in the nonRootIndex. The position starts at 0.
433      * @param nonRootIndex
434      * @param i the ith position
435      * @return The indexed key at the ith position in the nonRootIndex.
436      */
437     private byte[] getNonRootIndexedKey(ByteBuffer nonRootIndex, int i) {
438       int numEntries = nonRootIndex.getInt(0);
439       if (i < 0 || i >= numEntries) {
440         return null;
441       }
442 
443       // Entries start after the number of entries and the secondary index.
444       // The secondary index takes numEntries + 1 ints.
445       int entriesOffset = Bytes.SIZEOF_INT * (numEntries + 2);
446       // Targetkey's offset relative to the end of secondary index
447       int targetKeyRelOffset = nonRootIndex.getInt(
448           Bytes.SIZEOF_INT * (i + 1));
449 
450       // The offset of the target key in the blockIndex buffer
451       int targetKeyOffset = entriesOffset     // Skip secondary index
452           + targetKeyRelOffset               // Skip all entries until mid
453           + SECONDARY_INDEX_ENTRY_OVERHEAD;  // Skip offset and on-disk-size
454 
455       // We subtract the two consecutive secondary index elements, which
456       // gives us the size of the whole (offset, onDiskSize, key) tuple. We
457       // then need to subtract the overhead of offset and onDiskSize.
458       int targetKeyLength = nonRootIndex.getInt(Bytes.SIZEOF_INT * (i + 2)) -
459         targetKeyRelOffset - SECONDARY_INDEX_ENTRY_OVERHEAD;
460 
461       int from = nonRootIndex.arrayOffset() + targetKeyOffset;
462       int to = from + targetKeyLength;
463       return Arrays.copyOfRange(nonRootIndex.array(), from, to);
464     }
465 
466     /**
467      * Performs a binary search over a non-root level index block. Utilizes the
468      * secondary index, which records the offsets of (offset, onDiskSize,
469      * firstKey) tuples of all entries.
470      *
471      * @param key the key we are searching for offsets to individual entries in
472      *          the blockIndex buffer
473      * @param keyOffset the offset of the key in its byte array
474      * @param keyLength the length of the key
475      * @param nonRootIndex the non-root index block buffer, starting with the
476      *          secondary index. The position is ignored.
477      * @return the index i in [0, numEntries - 1] such that keys[i] <= key <
478      *         keys[i + 1], if keys is the array of all keys being searched, or
479      *         -1 otherwise
480      * @throws IOException
481      */
482     static int binarySearchNonRootIndex(byte[] key, int keyOffset,
483         int keyLength, ByteBuffer nonRootIndex,
484         RawComparator<byte[]> comparator) {
485 
486       int numEntries = nonRootIndex.getInt(0);
487       int low = 0;
488       int high = numEntries - 1;
489       int mid = 0;
490 
491       // Entries start after the number of entries and the secondary index.
492       // The secondary index takes numEntries + 1 ints.
493       int entriesOffset = Bytes.SIZEOF_INT * (numEntries + 2);
494 
495       // If we imagine that keys[-1] = -Infinity and
496       // keys[numEntries] = Infinity, then we are maintaining an invariant that
497       // keys[low - 1] < key < keys[high + 1] while narrowing down the range.
498 
499       while (low <= high) {
500         mid = (low + high) >>> 1;
501 
502         // Midkey's offset relative to the end of secondary index
503         int midKeyRelOffset = nonRootIndex.getInt(
504             Bytes.SIZEOF_INT * (mid + 1));
505 
506         // The offset of the middle key in the blockIndex buffer
507         int midKeyOffset = entriesOffset       // Skip secondary index
508             + midKeyRelOffset                  // Skip all entries until mid
509             + SECONDARY_INDEX_ENTRY_OVERHEAD;  // Skip offset and on-disk-size
510 
511         // We subtract the two consecutive secondary index elements, which
512         // gives us the size of the whole (offset, onDiskSize, key) tuple. We
513         // then need to subtract the overhead of offset and onDiskSize.
514         int midLength = nonRootIndex.getInt(Bytes.SIZEOF_INT * (mid + 2)) -
515             midKeyRelOffset - SECONDARY_INDEX_ENTRY_OVERHEAD;
516 
517         // we have to compare in this order, because the comparator order
518         // has special logic when the 'left side' is a special key.
519         int cmp = comparator.compare(key, keyOffset, keyLength,
520             nonRootIndex.array(), nonRootIndex.arrayOffset() + midKeyOffset,
521             midLength);
522 
523         // key lives above the midpoint
524         if (cmp > 0)
525           low = mid + 1; // Maintain the invariant that keys[low - 1] < key
526         // key lives below the midpoint
527         else if (cmp < 0)
528           high = mid - 1; // Maintain the invariant that key < keys[high + 1]
529         else
530           return mid; // exact match
531       }
532 
533       // As per our invariant, keys[low - 1] < key < keys[high + 1], meaning
534       // that low - 1 < high + 1 and (low - high) <= 1. As per the loop break
535       // condition, low >= high + 1. Therefore, low = high + 1.
536 
537       if (low != high + 1) {
538         throw new IllegalStateException("Binary search broken: low=" + low
539             + " " + "instead of " + (high + 1));
540       }
541 
542       // OK, our invariant says that keys[low - 1] < key < keys[low]. We need to
543       // return i such that keys[i] <= key < keys[i + 1]. Therefore i = low - 1.
544       int i = low - 1;
545 
546       // Some extra validation on the result.
547       if (i < -1 || i >= numEntries) {
548         throw new IllegalStateException("Binary search broken: result is " +
549             i + " but expected to be between -1 and (numEntries - 1) = " +
550             (numEntries - 1));
551       }
552 
553       return i;
554     }
555 
556     /**
557      * Search for one key using the secondary index in a non-root block. In case
558      * of success, positions the provided buffer at the entry of interest, where
559      * the file offset and the on-disk-size can be read.
560      *
561      * @param nonRootBlock a non-root block without header. Initial position
562      *          does not matter.
563      * @param key the byte array containing the key
564      * @param keyOffset the offset of the key in its byte array
565      * @param keyLength the length of the key
566      * @return the index position where the given key was found,
567      *         otherwise return -1 in the case the given key is before the first key.
568      *
569      */
570     static int locateNonRootIndexEntry(ByteBuffer nonRootBlock, byte[] key,
571         int keyOffset, int keyLength, RawComparator<byte[]> comparator) {
572       int entryIndex = binarySearchNonRootIndex(key, keyOffset, keyLength,
573           nonRootBlock, comparator);
574 
575       if (entryIndex != -1) {
576         int numEntries = nonRootBlock.getInt(0);
577 
578         // The end of secondary index and the beginning of entries themselves.
579         int entriesOffset = Bytes.SIZEOF_INT * (numEntries + 2);
580 
581         // The offset of the entry we are interested in relative to the end of
582         // the secondary index.
583         int entryRelOffset = nonRootBlock.getInt(Bytes.SIZEOF_INT
584             * (1 + entryIndex));
585 
586         nonRootBlock.position(entriesOffset + entryRelOffset);
587       }
588 
589       return entryIndex;
590     }
591 
592     /**
593      * Read in the root-level index from the given input stream. Must match
594      * what was written into the root level by
595      * {@link BlockIndexWriter#writeIndexBlocks(FSDataOutputStream)} at the
596      * offset that function returned.
597      *
598      * @param in the buffered input stream or wrapped byte input stream
599      * @param numEntries the number of root-level index entries
600      * @throws IOException
601      */
602     public void readRootIndex(DataInput in, final int numEntries)
603         throws IOException {
604       blockOffsets = new long[numEntries];
605       blockKeys = new byte[numEntries][];
606       blockDataSizes = new int[numEntries];
607 
608       // If index size is zero, no index was written.
609       if (numEntries > 0) {
610         for (int i = 0; i < numEntries; ++i) {
611           long offset = in.readLong();
612           int dataSize = in.readInt();
613           byte[] key = Bytes.readByteArray(in);
614           add(key, offset, dataSize);
615         }
616       }
617     }
618     
619     /**
620      * Read in the root-level index from the given input stream. Must match
621      * what was written into the root level by
622      * {@link BlockIndexWriter#writeIndexBlocks(FSDataOutputStream)} at the
623      * offset that function returned.
624      *
625      * @param blk the HFile block
626      * @param numEntries the number of root-level index entries
627      * @return the buffered input stream or wrapped byte input stream
628      * @throws IOException
629      */
630     public DataInputStream readRootIndex(HFileBlock blk, final int numEntries) throws IOException {
631       DataInputStream in = blk.getByteStream();
632       readRootIndex(in, numEntries);
633       return in;
634     }
635 
636     /**
637      * Read the root-level metadata of a multi-level block index. Based on
638      * {@link #readRootIndex(DataInput, int)}, but also reads metadata
639      * necessary to compute the mid-key in a multi-level index.
640      *
641      * @param blk the HFile block
642      * @param numEntries the number of root-level index entries
643      * @throws IOException
644      */
645     public void readMultiLevelIndexRoot(HFileBlock blk,
646         final int numEntries) throws IOException {
647       DataInputStream in = readRootIndex(blk, numEntries);
648       // after reading the root index the checksum bytes have to
649       // be subtracted to know if the mid key exists.
650       int checkSumBytes = blk.totalChecksumBytes();
651       if ((in.available() - checkSumBytes) < MID_KEY_METADATA_SIZE) {
652         // No mid-key metadata available.
653         return;
654       }
655       midLeafBlockOffset = in.readLong();
656       midLeafBlockOnDiskSize = in.readInt();
657       midKeyEntry = in.readInt();
658     }
659 
660     @Override
661     public String toString() {
662       StringBuilder sb = new StringBuilder();
663       sb.append("size=" + rootCount).append("\n");
664       for (int i = 0; i < rootCount; i++) {
665         sb.append("key=").append(KeyValue.keyToString(blockKeys[i]))
666             .append("\n  offset=").append(blockOffsets[i])
667             .append(", dataSize=" + blockDataSizes[i]).append("\n");
668       }
669       return sb.toString();
670     }
671 
672     @Override
673     public long heapSize() {
674       long heapSize = ClassSize.align(6 * ClassSize.REFERENCE +
675           3 * Bytes.SIZEOF_INT + ClassSize.OBJECT);
676 
677       // Mid-key metadata.
678       heapSize += MID_KEY_METADATA_SIZE;
679 
680       // Calculating the size of blockKeys
681       if (blockKeys != null) {
682         // Adding array + references overhead
683         heapSize += ClassSize.align(ClassSize.ARRAY + blockKeys.length
684             * ClassSize.REFERENCE);
685 
686         // Adding bytes
687         for (byte[] key : blockKeys) {
688           heapSize += ClassSize.align(ClassSize.ARRAY + key.length);
689         }
690       }
691 
692       if (blockOffsets != null) {
693         heapSize += ClassSize.align(ClassSize.ARRAY + blockOffsets.length
694             * Bytes.SIZEOF_LONG);
695       }
696 
697       if (blockDataSizes != null) {
698         heapSize += ClassSize.align(ClassSize.ARRAY + blockDataSizes.length
699             * Bytes.SIZEOF_INT);
700       }
701 
702       return ClassSize.align(heapSize);
703     }
704 
705   }
706 
707   /**
708    * Writes the block index into the output stream. Generate the tree from
709    * bottom up. The leaf level is written to disk as a sequence of inline
710    * blocks, if it is larger than a certain number of bytes. If the leaf level
711    * is not large enough, we write all entries to the root level instead.
712    *
713    * After all leaf blocks have been written, we end up with an index
714    * referencing the resulting leaf index blocks. If that index is larger than
715    * the allowed root index size, the writer will break it up into
716    * reasonable-size intermediate-level index block chunks write those chunks
717    * out, and create another index referencing those chunks. This will be
718    * repeated until the remaining index is small enough to become the root
719    * index. However, in most practical cases we will only have leaf-level
720    * blocks and the root index, or just the root index.
721    */
722   public static class BlockIndexWriter implements InlineBlockWriter {
723     /**
724      * While the index is being written, this represents the current block
725      * index referencing all leaf blocks, with one exception. If the file is
726      * being closed and there are not enough blocks to complete even a single
727      * leaf block, no leaf blocks get written and this contains the entire
728      * block index. After all levels of the index were written by
729      * {@link #writeIndexBlocks(FSDataOutputStream)}, this contains the final
730      * root-level index.
731      */
732     private BlockIndexChunk rootChunk = new BlockIndexChunk();
733 
734     /**
735      * Current leaf-level chunk. New entries referencing data blocks get added
736      * to this chunk until it grows large enough to be written to disk.
737      */
738     private BlockIndexChunk curInlineChunk = new BlockIndexChunk();
739 
740     /**
741      * The number of block index levels. This is one if there is only root
742      * level (even empty), two if there a leaf level and root level, and is
743      * higher if there are intermediate levels. This is only final after
744      * {@link #writeIndexBlocks(FSDataOutputStream)} has been called. The
745      * initial value accounts for the root level, and will be increased to two
746      * as soon as we find out there is a leaf-level in
747      * {@link #blockWritten(long, int)}.
748      */
749     private int numLevels = 1;
750 
751     private HFileBlock.Writer blockWriter;
752     private byte[] firstKey = null;
753 
754     /**
755      * The total number of leaf-level entries, i.e. entries referenced by
756      * leaf-level blocks. For the data block index this is equal to the number
757      * of data blocks.
758      */
759     private long totalNumEntries;
760 
761     /** Total compressed size of all index blocks. */
762     private long totalBlockOnDiskSize;
763 
764     /** Total uncompressed size of all index blocks. */
765     private long totalBlockUncompressedSize;
766 
767     /** The maximum size guideline of all multi-level index blocks. */
768     private int maxChunkSize;
769 
770     /** Whether we require this block index to always be single-level. */
771     private boolean singleLevelOnly;
772 
773     /** Block cache, or null if cache-on-write is disabled */
774     private BlockCache blockCache;
775 
776     /** Name to use for computing cache keys */
777     private String nameForCaching;
778 
779     /** Creates a single-level block index writer */
780     public BlockIndexWriter() {
781       this(null, null, null);
782       singleLevelOnly = true;
783     }
784 
785     /**
786      * Creates a multi-level block index writer.
787      *
788      * @param blockWriter the block writer to use to write index blocks
789      * @param blockCache if this is not null, index blocks will be cached
790      *    on write into this block cache.
791      */
792     public BlockIndexWriter(HFileBlock.Writer blockWriter,
793         BlockCache blockCache, String nameForCaching) {
794       if ((blockCache == null) != (nameForCaching == null)) {
795         throw new IllegalArgumentException("Block cache and file name for " +
796             "caching must be both specified or both null");
797       }
798 
799       this.blockWriter = blockWriter;
800       this.blockCache = blockCache;
801       this.nameForCaching = nameForCaching;
802       this.maxChunkSize = HFileBlockIndex.DEFAULT_MAX_CHUNK_SIZE;
803     }
804 
805     public void setMaxChunkSize(int maxChunkSize) {
806       if (maxChunkSize <= 0) {
807         throw new IllegalArgumentException("Invald maximum index block size");
808       }
809       this.maxChunkSize = maxChunkSize;
810     }
811 
812     /**
813      * Writes the root level and intermediate levels of the block index into
814      * the output stream, generating the tree from bottom up. Assumes that the
815      * leaf level has been inline-written to the disk if there is enough data
816      * for more than one leaf block. We iterate by breaking the current level
817      * of the block index, starting with the index of all leaf-level blocks,
818      * into chunks small enough to be written to disk, and generate its parent
819      * level, until we end up with a level small enough to become the root
820      * level.
821      *
822      * If the leaf level is not large enough, there is no inline block index
823      * anymore, so we only write that level of block index to disk as the root
824      * level.
825      *
826      * @param out FSDataOutputStream
827      * @return position at which we entered the root-level index.
828      * @throws IOException
829      */
830     public long writeIndexBlocks(FSDataOutputStream out) throws IOException {
831       if (curInlineChunk != null && curInlineChunk.getNumEntries() != 0) {
832         throw new IOException("Trying to write a multi-level block index, " +
833             "but are " + curInlineChunk.getNumEntries() + " entries in the " +
834             "last inline chunk.");
835       }
836 
837       // We need to get mid-key metadata before we create intermediate
838       // indexes and overwrite the root chunk.
839       byte[] midKeyMetadata = numLevels > 1 ? rootChunk.getMidKeyMetadata()
840           : null;
841 
842       if (curInlineChunk != null) {
843         while (rootChunk.getRootSize() > maxChunkSize) {
844           rootChunk = writeIntermediateLevel(out, rootChunk);
845           numLevels += 1;
846         }
847       }
848 
849       // write the root level
850       long rootLevelIndexPos = out.getPos();
851 
852       {
853         DataOutput blockStream =
854             blockWriter.startWriting(BlockType.ROOT_INDEX);
855         rootChunk.writeRoot(blockStream);
856         if (midKeyMetadata != null)
857           blockStream.write(midKeyMetadata);
858         blockWriter.writeHeaderAndData(out);
859       }
860 
861       // Add root index block size
862       totalBlockOnDiskSize += blockWriter.getOnDiskSizeWithoutHeader();
863       totalBlockUncompressedSize +=
864           blockWriter.getUncompressedSizeWithoutHeader();
865 
866       if (LOG.isTraceEnabled()) {
867         LOG.trace("Wrote a " + numLevels + "-level index with root level at pos "
868           + rootLevelIndexPos + ", " + rootChunk.getNumEntries()
869           + " root-level entries, " + totalNumEntries + " total entries, "
870           + StringUtils.humanReadableInt(this.totalBlockOnDiskSize) +
871           " on-disk size, "
872           + StringUtils.humanReadableInt(totalBlockUncompressedSize) +
873           " total uncompressed size.");
874       }
875       return rootLevelIndexPos;
876     }
877 
878     /**
879      * Writes the block index data as a single level only. Does not do any
880      * block framing.
881      *
882      * @param out the buffered output stream to write the index to. Typically a
883      *          stream writing into an {@link HFile} block.
884      * @param description a short description of the index being written. Used
885      *          in a log message.
886      * @throws IOException
887      */
888     public void writeSingleLevelIndex(DataOutput out, String description)
889         throws IOException {
890       expectNumLevels(1);
891 
892       if (!singleLevelOnly)
893         throw new IOException("Single-level mode is turned off");
894 
895       if (rootChunk.getNumEntries() > 0)
896         throw new IOException("Root-level entries already added in " +
897             "single-level mode");
898 
899       rootChunk = curInlineChunk;
900       curInlineChunk = new BlockIndexChunk();
901 
902       if (LOG.isTraceEnabled()) {
903         LOG.trace("Wrote a single-level " + description + " index with "
904           + rootChunk.getNumEntries() + " entries, " + rootChunk.getRootSize()
905           + " bytes");
906       }
907       rootChunk.writeRoot(out);
908     }
909 
910     /**
911      * Split the current level of the block index into intermediate index
912      * blocks of permitted size and write those blocks to disk. Return the next
913      * level of the block index referencing those intermediate-level blocks.
914      *
915      * @param out
916      * @param currentLevel the current level of the block index, such as the a
917      *          chunk referencing all leaf-level index blocks
918      * @return the parent level block index, which becomes the root index after
919      *         a few (usually zero) iterations
920      * @throws IOException
921      */
922     private BlockIndexChunk writeIntermediateLevel(FSDataOutputStream out,
923         BlockIndexChunk currentLevel) throws IOException {
924       // Entries referencing intermediate-level blocks we are about to create.
925       BlockIndexChunk parent = new BlockIndexChunk();
926 
927       // The current intermediate-level block index chunk.
928       BlockIndexChunk curChunk = new BlockIndexChunk();
929 
930       for (int i = 0; i < currentLevel.getNumEntries(); ++i) {
931         curChunk.add(currentLevel.getBlockKey(i),
932             currentLevel.getBlockOffset(i), currentLevel.getOnDiskDataSize(i));
933 
934         if (curChunk.getRootSize() >= maxChunkSize)
935           writeIntermediateBlock(out, parent, curChunk);
936       }
937 
938       if (curChunk.getNumEntries() > 0) {
939         writeIntermediateBlock(out, parent, curChunk);
940       }
941 
942       return parent;
943     }
944 
945     private void writeIntermediateBlock(FSDataOutputStream out,
946         BlockIndexChunk parent, BlockIndexChunk curChunk) throws IOException {
947       long beginOffset = out.getPos();
948       DataOutputStream dos = blockWriter.startWriting(
949           BlockType.INTERMEDIATE_INDEX);
950       curChunk.writeNonRoot(dos);
951       byte[] curFirstKey = curChunk.getBlockKey(0);
952       blockWriter.writeHeaderAndData(out);
953 
954       if (blockCache != null) {
955         HFileBlock blockForCaching = blockWriter.getBlockForCaching();
956         blockCache.cacheBlock(new BlockCacheKey(nameForCaching,
957             beginOffset, DataBlockEncoding.NONE, 
958             blockForCaching.getBlockType()), blockForCaching);
959       }
960 
961       // Add intermediate index block size
962       totalBlockOnDiskSize += blockWriter.getOnDiskSizeWithoutHeader();
963       totalBlockUncompressedSize +=
964           blockWriter.getUncompressedSizeWithoutHeader();
965 
966       // OFFSET is the beginning offset the chunk of block index entries.
967       // SIZE is the total byte size of the chunk of block index entries
968       // + the secondary index size
969       // FIRST_KEY is the first key in the chunk of block index
970       // entries.
971       parent.add(curFirstKey, beginOffset,
972           blockWriter.getOnDiskSizeWithHeader());
973 
974       // clear current block index chunk
975       curChunk.clear();
976       curFirstKey = null;
977     }
978 
979     /**
980      * @return how many block index entries there are in the root level
981      */
982     public final int getNumRootEntries() {
983       return rootChunk.getNumEntries();
984     }
985 
986     /**
987      * @return the number of levels in this block index.
988      */
989     public int getNumLevels() {
990       return numLevels;
991     }
992 
993     private void expectNumLevels(int expectedNumLevels) {
994       if (numLevels != expectedNumLevels) {
995         throw new IllegalStateException("Number of block index levels is "
996             + numLevels + "but is expected to be " + expectedNumLevels);
997       }
998     }
999 
1000     /**
1001      * Whether there is an inline block ready to be written. In general, we
1002      * write an leaf-level index block as an inline block as soon as its size
1003      * as serialized in the non-root format reaches a certain threshold.
1004      */
1005     @Override
1006     public boolean shouldWriteBlock(boolean closing) {
1007       if (singleLevelOnly) {
1008         throw new UnsupportedOperationException(INLINE_BLOCKS_NOT_ALLOWED);
1009       }
1010 
1011       if (curInlineChunk == null) {
1012         throw new IllegalStateException("curInlineChunk is null; has shouldWriteBlock been " +
1013             "called with closing=true and then called again?");
1014       }
1015 
1016       if (curInlineChunk.getNumEntries() == 0) {
1017         return false;
1018       }
1019 
1020       // We do have some entries in the current inline chunk.
1021       if (closing) {
1022         if (rootChunk.getNumEntries() == 0) {
1023           // We did not add any leaf-level blocks yet. Instead of creating a
1024           // leaf level with one block, move these entries to the root level.
1025 
1026           expectNumLevels(1);
1027           rootChunk = curInlineChunk;
1028           curInlineChunk = null;  // Disallow adding any more index entries.
1029           return false;
1030         }
1031 
1032         return true;
1033       } else {
1034         return curInlineChunk.getNonRootSize() >= maxChunkSize;
1035       }
1036     }
1037 
1038     /**
1039      * Write out the current inline index block. Inline blocks are non-root
1040      * blocks, so the non-root index format is used.
1041      *
1042      * @param out
1043      */
1044     @Override
1045     public void writeInlineBlock(DataOutput out) throws IOException {
1046       if (singleLevelOnly)
1047         throw new UnsupportedOperationException(INLINE_BLOCKS_NOT_ALLOWED);
1048 
1049       // Write the inline block index to the output stream in the non-root
1050       // index block format.
1051       curInlineChunk.writeNonRoot(out);
1052 
1053       // Save the first key of the inline block so that we can add it to the
1054       // parent-level index.
1055       firstKey = curInlineChunk.getBlockKey(0);
1056 
1057       // Start a new inline index block
1058       curInlineChunk.clear();
1059     }
1060 
1061     /**
1062      * Called after an inline block has been written so that we can add an
1063      * entry referring to that block to the parent-level index.
1064      */
1065     @Override
1066     public void blockWritten(long offset, int onDiskSize, int uncompressedSize)
1067     {
1068       // Add leaf index block size
1069       totalBlockOnDiskSize += onDiskSize;
1070       totalBlockUncompressedSize += uncompressedSize;
1071 
1072       if (singleLevelOnly)
1073         throw new UnsupportedOperationException(INLINE_BLOCKS_NOT_ALLOWED);
1074 
1075       if (firstKey == null) {
1076         throw new IllegalStateException("Trying to add second-level index " +
1077             "entry with offset=" + offset + " and onDiskSize=" + onDiskSize +
1078             "but the first key was not set in writeInlineBlock");
1079       }
1080 
1081       if (rootChunk.getNumEntries() == 0) {
1082         // We are writing the first leaf block, so increase index level.
1083         expectNumLevels(1);
1084         numLevels = 2;
1085       }
1086 
1087       // Add another entry to the second-level index. Include the number of
1088       // entries in all previous leaf-level chunks for mid-key calculation.
1089       rootChunk.add(firstKey, offset, onDiskSize, totalNumEntries);
1090       firstKey = null;
1091     }
1092 
1093     @Override
1094     public BlockType getInlineBlockType() {
1095       return BlockType.LEAF_INDEX;
1096     }
1097 
1098     /**
1099      * Add one index entry to the current leaf-level block. When the leaf-level
1100      * block gets large enough, it will be flushed to disk as an inline block.
1101      *
1102      * @param firstKey the first key of the data block
1103      * @param blockOffset the offset of the data block
1104      * @param blockDataSize the on-disk size of the data block ({@link HFile}
1105      *          format version 2), or the uncompressed size of the data block (
1106      *          {@link HFile} format version 1).
1107      */
1108     public void addEntry(byte[] firstKey, long blockOffset, int blockDataSize)
1109     {
1110       curInlineChunk.add(firstKey, blockOffset, blockDataSize);
1111       ++totalNumEntries;
1112     }
1113 
1114     /**
1115      * @throws IOException if we happened to write a multi-level index.
1116      */
1117     public void ensureSingleLevel() throws IOException {
1118       if (numLevels > 1) {
1119         throw new IOException ("Wrote a " + numLevels + "-level index with " +
1120             rootChunk.getNumEntries() + " root-level entries, but " +
1121             "this is expected to be a single-level block index.");
1122       }
1123     }
1124 
1125     /**
1126      * @return true if we are using cache-on-write. This is configured by the
1127      *         caller of the constructor by either passing a valid block cache
1128      *         or null.
1129      */
1130     @Override
1131     public boolean getCacheOnWrite() {
1132       return blockCache != null;
1133     }
1134 
1135     /**
1136      * The total uncompressed size of the root index block, intermediate-level
1137      * index blocks, and leaf-level index blocks.
1138      *
1139      * @return the total uncompressed size of all index blocks
1140      */
1141     public long getTotalUncompressedSize() {
1142       return totalBlockUncompressedSize;
1143     }
1144 
1145   }
1146 
1147   /**
1148    * A single chunk of the block index in the process of writing. The data in
1149    * this chunk can become a leaf-level, intermediate-level, or root index
1150    * block.
1151    */
1152   static class BlockIndexChunk {
1153 
1154     /** First keys of the key range corresponding to each index entry. */
1155     private final List<byte[]> blockKeys = new ArrayList<byte[]>();
1156 
1157     /** Block offset in backing stream. */
1158     private final List<Long> blockOffsets = new ArrayList<Long>();
1159 
1160     /** On-disk data sizes of lower-level data or index blocks. */
1161     private final List<Integer> onDiskDataSizes = new ArrayList<Integer>();
1162 
1163     /**
1164      * The cumulative number of sub-entries, i.e. entries on deeper-level block
1165      * index entries. numSubEntriesAt[i] is the number of sub-entries in the
1166      * blocks corresponding to this chunk's entries #0 through #i inclusively.
1167      */
1168     private final List<Long> numSubEntriesAt = new ArrayList<Long>();
1169 
1170     /**
1171      * The offset of the next entry to be added, relative to the end of the
1172      * "secondary index" in the "non-root" format representation of this index
1173      * chunk. This is the next value to be added to the secondary index.
1174      */
1175     private int curTotalNonRootEntrySize = 0;
1176 
1177     /**
1178      * The accumulated size of this chunk if stored in the root index format.
1179      */
1180     private int curTotalRootSize = 0;
1181 
1182     /**
1183      * The "secondary index" used for binary search over variable-length
1184      * records in a "non-root" format block. These offsets are relative to the
1185      * end of this secondary index.
1186      */
1187     private final List<Integer> secondaryIndexOffsetMarks =
1188         new ArrayList<Integer>();
1189 
1190     /**
1191      * Adds a new entry to this block index chunk.
1192      *
1193      * @param firstKey the first key in the block pointed to by this entry
1194      * @param blockOffset the offset of the next-level block pointed to by this
1195      *          entry
1196      * @param onDiskDataSize the on-disk data of the block pointed to by this
1197      *          entry, including header size
1198      * @param curTotalNumSubEntries if this chunk is the root index chunk under
1199      *          construction, this specifies the current total number of
1200      *          sub-entries in all leaf-level chunks, including the one
1201      *          corresponding to the second-level entry being added.
1202      */
1203     void add(byte[] firstKey, long blockOffset, int onDiskDataSize,
1204         long curTotalNumSubEntries) {
1205       // Record the offset for the secondary index
1206       secondaryIndexOffsetMarks.add(curTotalNonRootEntrySize);
1207       curTotalNonRootEntrySize += SECONDARY_INDEX_ENTRY_OVERHEAD
1208           + firstKey.length;
1209 
1210       curTotalRootSize += Bytes.SIZEOF_LONG + Bytes.SIZEOF_INT
1211           + WritableUtils.getVIntSize(firstKey.length) + firstKey.length;
1212 
1213       blockKeys.add(firstKey);
1214       blockOffsets.add(blockOffset);
1215       onDiskDataSizes.add(onDiskDataSize);
1216 
1217       if (curTotalNumSubEntries != -1) {
1218         numSubEntriesAt.add(curTotalNumSubEntries);
1219 
1220         // Make sure the parallel arrays are in sync.
1221         if (numSubEntriesAt.size() != blockKeys.size()) {
1222           throw new IllegalStateException("Only have key/value count " +
1223               "stats for " + numSubEntriesAt.size() + " block index " +
1224               "entries out of " + blockKeys.size());
1225         }
1226       }
1227     }
1228 
1229     /**
1230      * The same as {@link #add(byte[], long, int, long)} but does not take the
1231      * key/value into account. Used for single-level indexes.
1232      *
1233      * @see {@link #add(byte[], long, int, long)}
1234      */
1235     public void add(byte[] firstKey, long blockOffset, int onDiskDataSize) {
1236       add(firstKey, blockOffset, onDiskDataSize, -1);
1237     }
1238 
1239     public void clear() {
1240       blockKeys.clear();
1241       blockOffsets.clear();
1242       onDiskDataSizes.clear();
1243       secondaryIndexOffsetMarks.clear();
1244       numSubEntriesAt.clear();
1245       curTotalNonRootEntrySize = 0;
1246       curTotalRootSize = 0;
1247     }
1248 
1249     /**
1250      * Finds the entry corresponding to the deeper-level index block containing
1251      * the given deeper-level entry (a "sub-entry"), assuming a global 0-based
1252      * ordering of sub-entries.
1253      *
1254      * <p>
1255      * <i> Implementation note. </i> We are looking for i such that
1256      * numSubEntriesAt[i - 1] <= k < numSubEntriesAt[i], because a deeper-level
1257      * block #i (0-based) contains sub-entries # numSubEntriesAt[i - 1]'th
1258      * through numSubEntriesAt[i] - 1, assuming a global 0-based ordering of
1259      * sub-entries. i is by definition the insertion point of k in
1260      * numSubEntriesAt.
1261      *
1262      * @param k sub-entry index, from 0 to the total number sub-entries - 1
1263      * @return the 0-based index of the entry corresponding to the given
1264      *         sub-entry
1265      */
1266     public int getEntryBySubEntry(long k) {
1267       // We define mid-key as the key corresponding to k'th sub-entry
1268       // (0-based).
1269 
1270       int i = Collections.binarySearch(numSubEntriesAt, k);
1271 
1272       // Exact match: cumulativeWeight[i] = k. This means chunks #0 through
1273       // #i contain exactly k sub-entries, and the sub-entry #k (0-based)
1274       // is in the (i + 1)'th chunk.
1275       if (i >= 0)
1276         return i + 1;
1277 
1278       // Inexact match. Return the insertion point.
1279       return -i - 1;
1280     }
1281 
1282     /**
1283      * Used when writing the root block index of a multi-level block index.
1284      * Serializes additional information allowing to efficiently identify the
1285      * mid-key.
1286      *
1287      * @return a few serialized fields for finding the mid-key
1288      * @throws IOException if could not create metadata for computing mid-key
1289      */
1290     public byte[] getMidKeyMetadata() throws IOException {
1291       ByteArrayOutputStream baos = new ByteArrayOutputStream(
1292           MID_KEY_METADATA_SIZE);
1293       DataOutputStream baosDos = new DataOutputStream(baos);
1294       long totalNumSubEntries = numSubEntriesAt.get(blockKeys.size() - 1);
1295       if (totalNumSubEntries == 0) {
1296         throw new IOException("No leaf-level entries, mid-key unavailable");
1297       }
1298       long midKeySubEntry = (totalNumSubEntries - 1) / 2;
1299       int midKeyEntry = getEntryBySubEntry(midKeySubEntry);
1300 
1301       baosDos.writeLong(blockOffsets.get(midKeyEntry));
1302       baosDos.writeInt(onDiskDataSizes.get(midKeyEntry));
1303 
1304       long numSubEntriesBefore = midKeyEntry > 0
1305           ? numSubEntriesAt.get(midKeyEntry - 1) : 0;
1306       long subEntryWithinEntry = midKeySubEntry - numSubEntriesBefore;
1307       if (subEntryWithinEntry < 0 || subEntryWithinEntry > Integer.MAX_VALUE)
1308       {
1309         throw new IOException("Could not identify mid-key index within the "
1310             + "leaf-level block containing mid-key: out of range ("
1311             + subEntryWithinEntry + ", numSubEntriesBefore="
1312             + numSubEntriesBefore + ", midKeySubEntry=" + midKeySubEntry
1313             + ")");
1314       }
1315 
1316       baosDos.writeInt((int) subEntryWithinEntry);
1317 
1318       if (baosDos.size() != MID_KEY_METADATA_SIZE) {
1319         throw new IOException("Could not write mid-key metadata: size=" +
1320             baosDos.size() + ", correct size: " + MID_KEY_METADATA_SIZE);
1321       }
1322 
1323       // Close just to be good citizens, although this has no effect.
1324       baos.close();
1325 
1326       return baos.toByteArray();
1327     }
1328 
1329     /**
1330      * Writes the block index chunk in the non-root index block format. This
1331      * format contains the number of entries, an index of integer offsets
1332      * for quick binary search on variable-length records, and tuples of
1333      * block offset, on-disk block size, and the first key for each entry.
1334      *
1335      * @param out
1336      * @throws IOException
1337      */
1338     void writeNonRoot(DataOutput out) throws IOException {
1339       // The number of entries in the block.
1340       out.writeInt(blockKeys.size());
1341 
1342       if (secondaryIndexOffsetMarks.size() != blockKeys.size()) {
1343         throw new IOException("Corrupted block index chunk writer: " +
1344             blockKeys.size() + " entries but " +
1345             secondaryIndexOffsetMarks.size() + " secondary index items");
1346       }
1347 
1348       // For each entry, write a "secondary index" of relative offsets to the
1349       // entries from the end of the secondary index. This works, because at
1350       // read time we read the number of entries and know where the secondary
1351       // index ends.
1352       for (int currentSecondaryIndex : secondaryIndexOffsetMarks)
1353         out.writeInt(currentSecondaryIndex);
1354 
1355       // We include one other element in the secondary index to calculate the
1356       // size of each entry more easily by subtracting secondary index elements.
1357       out.writeInt(curTotalNonRootEntrySize);
1358 
1359       for (int i = 0; i < blockKeys.size(); ++i) {
1360         out.writeLong(blockOffsets.get(i));
1361         out.writeInt(onDiskDataSizes.get(i));
1362         out.write(blockKeys.get(i));
1363       }
1364     }
1365 
1366     /**
1367      * @return the size of this chunk if stored in the non-root index block
1368      *         format
1369      */
1370     int getNonRootSize() {
1371       return Bytes.SIZEOF_INT                          // Number of entries
1372           + Bytes.SIZEOF_INT * (blockKeys.size() + 1)  // Secondary index
1373           + curTotalNonRootEntrySize;                  // All entries
1374     }
1375 
1376     /**
1377      * Writes this chunk into the given output stream in the root block index
1378      * format. This format is similar to the {@link HFile} version 1 block
1379      * index format, except that we store on-disk size of the block instead of
1380      * its uncompressed size.
1381      *
1382      * @param out the data output stream to write the block index to. Typically
1383      *          a stream writing into an {@link HFile} block.
1384      * @throws IOException
1385      */
1386     void writeRoot(DataOutput out) throws IOException {
1387       for (int i = 0; i < blockKeys.size(); ++i) {
1388         out.writeLong(blockOffsets.get(i));
1389         out.writeInt(onDiskDataSizes.get(i));
1390         Bytes.writeByteArray(out, blockKeys.get(i));
1391       }
1392     }
1393 
1394     /**
1395      * @return the size of this chunk if stored in the root index block format
1396      */
1397     int getRootSize() {
1398       return curTotalRootSize;
1399     }
1400 
1401     /**
1402      * @return the number of entries in this block index chunk
1403      */
1404     public int getNumEntries() {
1405       return blockKeys.size();
1406     }
1407 
1408     public byte[] getBlockKey(int i) {
1409       return blockKeys.get(i);
1410     }
1411 
1412     public long getBlockOffset(int i) {
1413       return blockOffsets.get(i);
1414     }
1415 
1416     public int getOnDiskDataSize(int i) {
1417       return onDiskDataSizes.get(i);
1418     }
1419 
1420     public long getCumulativeNumKV(int i) {
1421       if (i < 0)
1422         return 0;
1423       return numSubEntriesAt.get(i);
1424     }
1425 
1426   }
1427 
1428   public static int getMaxChunkSize(Configuration conf) {
1429     return conf.getInt(MAX_CHUNK_SIZE_KEY, DEFAULT_MAX_CHUNK_SIZE);
1430   }
1431 }