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}