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