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