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 static org.apache.hadoop.hbase.trace.HBaseSemanticAttributes.BLOCK_CACHE_KEY_KEY;
021
022import io.opentelemetry.api.common.Attributes;
023import io.opentelemetry.api.trace.Span;
024import java.io.DataInput;
025import java.io.IOException;
026import java.nio.ByteBuffer;
027import java.util.ArrayList;
028import java.util.Optional;
029import org.apache.hadoop.conf.Configurable;
030import org.apache.hadoop.conf.Configuration;
031import org.apache.hadoop.fs.Path;
032import org.apache.hadoop.hbase.ByteBufferKeyOnlyKeyValue;
033import org.apache.hadoop.hbase.Cell;
034import org.apache.hadoop.hbase.CellComparator;
035import org.apache.hadoop.hbase.CellUtil;
036import org.apache.hadoop.hbase.HConstants;
037import org.apache.hadoop.hbase.KeyValue;
038import org.apache.hadoop.hbase.PrivateCellUtil;
039import org.apache.hadoop.hbase.SizeCachedByteBufferKeyValue;
040import org.apache.hadoop.hbase.SizeCachedKeyValue;
041import org.apache.hadoop.hbase.SizeCachedNoTagsByteBufferKeyValue;
042import org.apache.hadoop.hbase.SizeCachedNoTagsKeyValue;
043import org.apache.hadoop.hbase.io.compress.Compression;
044import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder;
045import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
046import org.apache.hadoop.hbase.io.encoding.HFileBlockDecodingContext;
047import org.apache.hadoop.hbase.nio.ByteBuff;
048import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
049import org.apache.hadoop.hbase.util.ByteBufferUtils;
050import org.apache.hadoop.hbase.util.Bytes;
051import org.apache.hadoop.hbase.util.IdLock;
052import org.apache.hadoop.hbase.util.ObjectIntPair;
053import org.apache.hadoop.io.WritableUtils;
054import org.apache.yetus.audience.InterfaceAudience;
055import org.slf4j.Logger;
056import org.slf4j.LoggerFactory;
057
058/**
059 * Implementation that can handle all hfile versions of {@link HFile.Reader}.
060 */
061@InterfaceAudience.Private
062@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "URF_UNREAD_PUBLIC_OR_PROTECTED_FIELD")
063public abstract class HFileReaderImpl implements HFile.Reader, Configurable {
064  // This class is HFileReaderV3 + HFileReaderV2 + AbstractHFileReader all squashed together into
065  // one file. Ditto for all the HFileReader.ScannerV? implementations. I was running up against
066  // the MaxInlineLevel limit because too many tiers involved reading from an hfile. Was also hard
067  // to navigate the source code when so many classes participating in read.
068  private static final Logger LOG = LoggerFactory.getLogger(HFileReaderImpl.class);
069
070  /** Data block index reader keeping the root data index in memory */
071  protected HFileBlockIndex.CellBasedKeyBlockIndexReader dataBlockIndexReader;
072
073  /** Meta block index reader -- always single level */
074  protected HFileBlockIndex.ByteArrayKeyBlockIndexReader metaBlockIndexReader;
075
076  protected FixedFileTrailer trailer;
077
078  private final boolean primaryReplicaReader;
079
080  /**
081   * What kind of data block encoding should be used while reading, writing, and handling cache.
082   */
083  protected HFileDataBlockEncoder dataBlockEncoder = NoOpDataBlockEncoder.INSTANCE;
084
085  /** Block cache configuration. */
086  protected final CacheConfig cacheConf;
087
088  protected ReaderContext context;
089
090  protected final HFileInfo fileInfo;
091
092  /** Path of file */
093  protected final Path path;
094
095  /** File name to be used for block names */
096  protected final String name;
097
098  private Configuration conf;
099
100  protected HFileContext hfileContext;
101
102  /** Filesystem-level block reader. */
103  protected HFileBlock.FSReader fsBlockReader;
104
105  /**
106   * A "sparse lock" implementation allowing to lock on a particular block identified by offset. The
107   * purpose of this is to avoid two clients loading the same block, and have all but one client
108   * wait to get the block from the cache.
109   */
110  private IdLock offsetLock = new IdLock();
111
112  /** Minimum minor version supported by this HFile format */
113  static final int MIN_MINOR_VERSION = 0;
114
115  /** Maximum minor version supported by this HFile format */
116  // We went to version 2 when we moved to pb'ing fileinfo and the trailer on
117  // the file. This version can read Writables version 1.
118  static final int MAX_MINOR_VERSION = 3;
119
120  /** Minor versions starting with this number have faked index key */
121  static final int MINOR_VERSION_WITH_FAKED_KEY = 3;
122
123  /**
124   * Opens a HFile.
125   * @param context   Reader context info
126   * @param fileInfo  HFile info
127   * @param cacheConf Cache configuration.
128   * @param conf      Configuration
129   */
130  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "URF_UNREAD_PUBLIC_OR_PROTECTED_FIELD")
131  public HFileReaderImpl(ReaderContext context, HFileInfo fileInfo, CacheConfig cacheConf,
132    Configuration conf) throws IOException {
133    this.cacheConf = cacheConf;
134    this.context = context;
135    this.path = context.getFilePath();
136    this.name = path.getName();
137    this.conf = conf;
138    this.primaryReplicaReader = context.isPrimaryReplicaReader();
139    this.fileInfo = fileInfo;
140    this.trailer = fileInfo.getTrailer();
141    this.hfileContext = fileInfo.getHFileContext();
142    this.fsBlockReader =
143      new HFileBlock.FSReaderImpl(context, hfileContext, cacheConf.getByteBuffAllocator(), conf);
144    this.dataBlockEncoder = HFileDataBlockEncoderImpl.createFromFileInfo(fileInfo);
145    fsBlockReader.setDataBlockEncoder(dataBlockEncoder, conf);
146    dataBlockIndexReader = fileInfo.getDataBlockIndexReader();
147    metaBlockIndexReader = fileInfo.getMetaBlockIndexReader();
148  }
149
150  @SuppressWarnings("serial")
151  public static class BlockIndexNotLoadedException extends IllegalStateException {
152    public BlockIndexNotLoadedException(Path path) {
153      // Add a message in case anyone relies on it as opposed to class name.
154      super(path + " block index not loaded");
155    }
156  }
157
158  private Optional<String> toStringFirstKey() {
159    return getFirstKey().map(CellUtil::getCellKeyAsString);
160  }
161
162  private Optional<String> toStringLastKey() {
163    return getLastKey().map(CellUtil::getCellKeyAsString);
164  }
165
166  @Override
167  public String toString() {
168    return "reader=" + path.toString()
169      + (!isFileInfoLoaded()
170        ? ""
171        : ", compression=" + trailer.getCompressionCodec().getName() + ", cacheConf=" + cacheConf
172          + ", firstKey=" + toStringFirstKey() + ", lastKey=" + toStringLastKey())
173      + ", avgKeyLen=" + fileInfo.getAvgKeyLen() + ", avgValueLen=" + fileInfo.getAvgValueLen()
174      + ", entries=" + trailer.getEntryCount() + ", length=" + context.getFileSize();
175  }
176
177  @Override
178  public long length() {
179    return context.getFileSize();
180  }
181
182  /**
183   * @return the first key in the file. May be null if file has no entries. Note that this is not
184   *         the first row key, but rather the byte form of the first KeyValue.
185   */
186  @Override
187  public Optional<Cell> getFirstKey() {
188    if (dataBlockIndexReader == null) {
189      throw new BlockIndexNotLoadedException(path);
190    }
191    return dataBlockIndexReader.isEmpty()
192      ? Optional.empty()
193      : Optional.of(dataBlockIndexReader.getRootBlockKey(0));
194  }
195
196  /**
197   * TODO left from {@link HFile} version 1: move this to StoreFile after Ryan's patch goes in to
198   * eliminate {@link KeyValue} here.
199   * @return the first row key, or null if the file is empty.
200   */
201  @Override
202  public Optional<byte[]> getFirstRowKey() {
203    // We have to copy the row part to form the row key alone
204    return getFirstKey().map(CellUtil::cloneRow);
205  }
206
207  /**
208   * TODO left from {@link HFile} version 1: move this to StoreFile after Ryan's patch goes in to
209   * eliminate {@link KeyValue} here.
210   * @return the last row key, or null if the file is empty.
211   */
212  @Override
213  public Optional<byte[]> getLastRowKey() {
214    // We have to copy the row part to form the row key alone
215    return getLastKey().map(CellUtil::cloneRow);
216  }
217
218  /** Returns number of KV entries in this HFile */
219  @Override
220  public long getEntries() {
221    return trailer.getEntryCount();
222  }
223
224  /** Returns comparator */
225  @Override
226  public CellComparator getComparator() {
227    return this.hfileContext.getCellComparator();
228  }
229
230  public Compression.Algorithm getCompressionAlgorithm() {
231    return trailer.getCompressionCodec();
232  }
233
234  /**
235   * @return the total heap size of data and meta block indexes in bytes. Does not take into account
236   *         non-root blocks of a multilevel data index.
237   */
238  @Override
239  public long indexSize() {
240    return (dataBlockIndexReader != null ? dataBlockIndexReader.heapSize() : 0)
241      + ((metaBlockIndexReader != null) ? metaBlockIndexReader.heapSize() : 0);
242  }
243
244  @Override
245  public String getName() {
246    return name;
247  }
248
249  @Override
250  public void setDataBlockEncoder(HFileDataBlockEncoder dataBlockEncoder) {
251    this.dataBlockEncoder = dataBlockEncoder;
252    this.fsBlockReader.setDataBlockEncoder(dataBlockEncoder, conf);
253  }
254
255  @Override
256  public void setDataBlockIndexReader(HFileBlockIndex.CellBasedKeyBlockIndexReader reader) {
257    this.dataBlockIndexReader = reader;
258  }
259
260  @Override
261  public HFileBlockIndex.CellBasedKeyBlockIndexReader getDataBlockIndexReader() {
262    return dataBlockIndexReader;
263  }
264
265  @Override
266  public void setMetaBlockIndexReader(HFileBlockIndex.ByteArrayKeyBlockIndexReader reader) {
267    this.metaBlockIndexReader = reader;
268  }
269
270  @Override
271  public HFileBlockIndex.ByteArrayKeyBlockIndexReader getMetaBlockIndexReader() {
272    return metaBlockIndexReader;
273  }
274
275  @Override
276  public FixedFileTrailer getTrailer() {
277    return trailer;
278  }
279
280  @Override
281  public ReaderContext getContext() {
282    return this.context;
283  }
284
285  @Override
286  public HFileInfo getHFileInfo() {
287    return this.fileInfo;
288  }
289
290  @Override
291  public boolean isPrimaryReplicaReader() {
292    return primaryReplicaReader;
293  }
294
295  /**
296   * An exception thrown when an operation requiring a scanner to be seeked is invoked on a scanner
297   * that is not seeked.
298   */
299  @SuppressWarnings("serial")
300  public static class NotSeekedException extends IllegalStateException {
301    public NotSeekedException(Path path) {
302      super(path + " not seeked to a key/value");
303    }
304  }
305
306  protected static class HFileScannerImpl implements HFileScanner {
307    private ByteBuff blockBuffer;
308    protected final boolean cacheBlocks;
309    protected final boolean pread;
310    protected final boolean isCompaction;
311    private int currKeyLen;
312    private int currValueLen;
313    private int currMemstoreTSLen;
314    private long currMemstoreTS;
315    protected final HFile.Reader reader;
316    private int currTagsLen;
317    private short rowLen;
318    // buffer backed keyonlyKV
319    private ByteBufferKeyOnlyKeyValue bufBackedKeyOnlyKv = new ByteBufferKeyOnlyKeyValue();
320    // A pair for reusing in blockSeek() so that we don't garbage lot of objects
321    final ObjectIntPair<ByteBuffer> pair = new ObjectIntPair<>();
322
323    /**
324     * The next indexed key is to keep track of the indexed key of the next data block. If the
325     * nextIndexedKey is HConstants.NO_NEXT_INDEXED_KEY, it means that the current data block is the
326     * last data block. If the nextIndexedKey is null, it means the nextIndexedKey has not been
327     * loaded yet.
328     */
329    protected Cell nextIndexedKey;
330    // Current block being used. NOTICE: DON't release curBlock separately except in shipped() or
331    // close() methods. Because the shipped() or close() will do the release finally, even if any
332    // exception occur the curBlock will be released by the close() method (see
333    // RegionScannerImpl#handleException). Call the releaseIfNotCurBlock() to release the
334    // unreferenced block please.
335    protected HFileBlock curBlock;
336    // Previous blocks that were used in the course of the read
337    protected final ArrayList<HFileBlock> prevBlocks = new ArrayList<>();
338
339    public HFileScannerImpl(final HFile.Reader reader, final boolean cacheBlocks,
340      final boolean pread, final boolean isCompaction) {
341      this.reader = reader;
342      this.cacheBlocks = cacheBlocks;
343      this.pread = pread;
344      this.isCompaction = isCompaction;
345    }
346
347    void updateCurrBlockRef(HFileBlock block) {
348      if (block != null && curBlock != null && block.getOffset() == curBlock.getOffset()) {
349        return;
350      }
351      if (this.curBlock != null && this.curBlock.isSharedMem()) {
352        prevBlocks.add(this.curBlock);
353      }
354      this.curBlock = block;
355    }
356
357    void reset() {
358      // We don't have to keep ref to heap block
359      if (this.curBlock != null && this.curBlock.isSharedMem()) {
360        this.prevBlocks.add(this.curBlock);
361      }
362      this.curBlock = null;
363    }
364
365    private void returnBlocks(boolean returnAll) {
366      this.prevBlocks.forEach(HFileBlock::release);
367      this.prevBlocks.clear();
368      if (returnAll && this.curBlock != null) {
369        this.curBlock.release();
370        this.curBlock = null;
371      }
372    }
373
374    @Override
375    public boolean isSeeked() {
376      return blockBuffer != null;
377    }
378
379    @Override
380    public String toString() {
381      return "HFileScanner for reader " + String.valueOf(getReader());
382    }
383
384    protected void assertSeeked() {
385      if (!isSeeked()) {
386        throw new NotSeekedException(reader.getPath());
387      }
388    }
389
390    @Override
391    public HFile.Reader getReader() {
392      return reader;
393    }
394
395    // From non encoded HFiles, we always read back KeyValue or its descendant.(Note: When HFile
396    // block is in DBB, it will be OffheapKV). So all parts of the Cell is in a contiguous
397    // array/buffer. How many bytes we should wrap to make the KV is what this method returns.
398    private int getKVBufSize() {
399      int kvBufSize = KEY_VALUE_LEN_SIZE + currKeyLen + currValueLen;
400      if (currTagsLen > 0) {
401        kvBufSize += Bytes.SIZEOF_SHORT + currTagsLen;
402      }
403      return kvBufSize;
404    }
405
406    @Override
407    public void close() {
408      if (!pread) {
409        // For seek + pread stream socket should be closed when the scanner is closed. HBASE-9393
410        reader.unbufferStream();
411      }
412      this.returnBlocks(true);
413    }
414
415    // Returns the #bytes in HFile for the current cell. Used to skip these many bytes in current
416    // HFile block's buffer so as to position to the next cell.
417    private int getCurCellSerializedSize() {
418      int curCellSize = KEY_VALUE_LEN_SIZE + currKeyLen + currValueLen + currMemstoreTSLen;
419      if (this.reader.getFileContext().isIncludesTags()) {
420        curCellSize += Bytes.SIZEOF_SHORT + currTagsLen;
421      }
422      return curCellSize;
423    }
424
425    protected void readKeyValueLen() {
426      // This is a hot method. We go out of our way to make this method short so it can be
427      // inlined and is not too big to compile. We also manage position in ByteBuffer ourselves
428      // because it is faster than going via range-checked ByteBuffer methods or going through a
429      // byte buffer array a byte at a time.
430      // Get a long at a time rather than read two individual ints. In micro-benchmarking, even
431      // with the extra bit-fiddling, this is order-of-magnitude faster than getting two ints.
432      // Trying to imitate what was done - need to profile if this is better or
433      // earlier way is better by doing mark and reset?
434      // But ensure that you read long instead of two ints
435      long ll = blockBuffer.getLongAfterPosition(0);
436      // Read top half as an int of key length and bottom int as value length
437      this.currKeyLen = (int) (ll >> Integer.SIZE);
438      this.currValueLen = (int) (Bytes.MASK_FOR_LOWER_INT_IN_LONG ^ ll);
439      checkKeyValueLen();
440      this.rowLen = blockBuffer.getShortAfterPosition(Bytes.SIZEOF_LONG);
441      // Move position past the key and value lengths and then beyond the key and value
442      int p = (Bytes.SIZEOF_LONG + currKeyLen + currValueLen);
443      if (reader.getFileContext().isIncludesTags()) {
444        // Tags length is a short.
445        this.currTagsLen = blockBuffer.getShortAfterPosition(p);
446        checkTagsLen();
447        p += (Bytes.SIZEOF_SHORT + currTagsLen);
448      }
449      readMvccVersion(p);
450    }
451
452    private final void checkTagsLen() {
453      if (checkLen(this.currTagsLen)) {
454        throw new IllegalStateException(
455          "Invalid currTagsLen " + this.currTagsLen + ". Block offset: " + curBlock.getOffset()
456            + ", block length: " + this.blockBuffer.limit() + ", position: "
457            + this.blockBuffer.position() + " (without header)." + " path=" + reader.getPath());
458      }
459    }
460
461    /**
462     * Read mvcc. Does checks to see if we even need to read the mvcc at all.
463     */
464    protected void readMvccVersion(final int offsetFromPos) {
465      // See if we even need to decode mvcc.
466      if (!this.reader.getHFileInfo().shouldIncludeMemStoreTS()) {
467        return;
468      }
469      if (!this.reader.getHFileInfo().isDecodeMemstoreTS()) {
470        currMemstoreTS = 0;
471        currMemstoreTSLen = 1;
472        return;
473      }
474      _readMvccVersion(offsetFromPos);
475    }
476
477    /**
478     * Actually do the mvcc read. Does no checks.
479     */
480    private void _readMvccVersion(int offsetFromPos) {
481      // This is Bytes#bytesToVint inlined so can save a few instructions in this hot method; i.e.
482      // previous if one-byte vint, we'd redo the vint call to find int size.
483      // Also the method is kept small so can be inlined.
484      byte firstByte = blockBuffer.getByteAfterPosition(offsetFromPos);
485      int len = WritableUtils.decodeVIntSize(firstByte);
486      if (len == 1) {
487        this.currMemstoreTS = firstByte;
488      } else {
489        int remaining = len - 1;
490        long i = 0;
491        offsetFromPos++;
492        if (remaining >= Bytes.SIZEOF_INT) {
493          // The int read has to be converted to unsigned long so the & op
494          i = (blockBuffer.getIntAfterPosition(offsetFromPos) & 0x00000000ffffffffL);
495          remaining -= Bytes.SIZEOF_INT;
496          offsetFromPos += Bytes.SIZEOF_INT;
497        }
498        if (remaining >= Bytes.SIZEOF_SHORT) {
499          short s = blockBuffer.getShortAfterPosition(offsetFromPos);
500          i = i << 16;
501          i = i | (s & 0xFFFF);
502          remaining -= Bytes.SIZEOF_SHORT;
503          offsetFromPos += Bytes.SIZEOF_SHORT;
504        }
505        for (int idx = 0; idx < remaining; idx++) {
506          byte b = blockBuffer.getByteAfterPosition(offsetFromPos + idx);
507          i = i << 8;
508          i = i | (b & 0xFF);
509        }
510        currMemstoreTS = (WritableUtils.isNegativeVInt(firstByte) ? ~i : i);
511      }
512      this.currMemstoreTSLen = len;
513    }
514
515    /**
516     * Within a loaded block, seek looking for the last key that is smaller than (or equal to?) the
517     * key we are interested in. A note on the seekBefore: if you have seekBefore = true, AND the
518     * first key in the block = key, then you'll get thrown exceptions. The caller has to check for
519     * that case and load the previous block as appropriate. n * the key to find n * find the key
520     * before the given key in case of exact match.
521     * @return 0 in case of an exact key match, 1 in case of an inexact match, -2 in case of an
522     *         inexact match and furthermore, the input key less than the first key of current
523     *         block(e.g. using a faked index key)
524     */
525    protected int blockSeek(Cell key, boolean seekBefore) {
526      int klen, vlen, tlen = 0;
527      int lastKeyValueSize = -1;
528      int offsetFromPos;
529      do {
530        offsetFromPos = 0;
531        // Better to ensure that we use the BB Utils here
532        long ll = blockBuffer.getLongAfterPosition(offsetFromPos);
533        klen = (int) (ll >> Integer.SIZE);
534        vlen = (int) (Bytes.MASK_FOR_LOWER_INT_IN_LONG ^ ll);
535        if (checkKeyLen(klen) || checkLen(vlen)) {
536          throw new IllegalStateException(
537            "Invalid klen " + klen + " or vlen " + vlen + ". Block offset: " + curBlock.getOffset()
538              + ", block length: " + blockBuffer.limit() + ", position: " + blockBuffer.position()
539              + " (without header)." + " path=" + reader.getPath());
540        }
541        offsetFromPos += Bytes.SIZEOF_LONG;
542        this.rowLen = blockBuffer.getShortAfterPosition(offsetFromPos);
543        blockBuffer.asSubByteBuffer(blockBuffer.position() + offsetFromPos, klen, pair);
544        bufBackedKeyOnlyKv.setKey(pair.getFirst(), pair.getSecond(), klen, rowLen);
545        int comp =
546          PrivateCellUtil.compareKeyIgnoresMvcc(reader.getComparator(), key, bufBackedKeyOnlyKv);
547        offsetFromPos += klen + vlen;
548        if (this.reader.getFileContext().isIncludesTags()) {
549          // Read short as unsigned, high byte first
550          tlen = ((blockBuffer.getByteAfterPosition(offsetFromPos) & 0xff) << 8)
551            ^ (blockBuffer.getByteAfterPosition(offsetFromPos + 1) & 0xff);
552          if (checkLen(tlen)) {
553            throw new IllegalStateException("Invalid tlen " + tlen + ". Block offset: "
554              + curBlock.getOffset() + ", block length: " + blockBuffer.limit() + ", position: "
555              + blockBuffer.position() + " (without header)." + " path=" + reader.getPath());
556          }
557          // add the two bytes read for the tags.
558          offsetFromPos += tlen + (Bytes.SIZEOF_SHORT);
559        }
560        if (this.reader.getHFileInfo().shouldIncludeMemStoreTS()) {
561          // Directly read the mvcc based on current position
562          readMvccVersion(offsetFromPos);
563        }
564        if (comp == 0) {
565          if (seekBefore) {
566            if (lastKeyValueSize < 0) {
567              throw new IllegalStateException("blockSeek with seekBefore "
568                + "at the first key of the block: key=" + CellUtil.getCellKeyAsString(key)
569                + ", blockOffset=" + curBlock.getOffset() + ", onDiskSize="
570                + curBlock.getOnDiskSizeWithHeader() + ", path=" + reader.getPath());
571            }
572            blockBuffer.moveBack(lastKeyValueSize);
573            readKeyValueLen();
574            return 1; // non exact match.
575          }
576          currKeyLen = klen;
577          currValueLen = vlen;
578          currTagsLen = tlen;
579          return 0; // indicate exact match
580        } else if (comp < 0) {
581          if (lastKeyValueSize > 0) {
582            blockBuffer.moveBack(lastKeyValueSize);
583          }
584          readKeyValueLen();
585          if (lastKeyValueSize == -1 && blockBuffer.position() == 0) {
586            return HConstants.INDEX_KEY_MAGIC;
587          }
588          return 1;
589        }
590        // The size of this key/value tuple, including key/value length fields.
591        lastKeyValueSize = klen + vlen + currMemstoreTSLen + KEY_VALUE_LEN_SIZE;
592        // include tag length also if tags included with KV
593        if (reader.getFileContext().isIncludesTags()) {
594          lastKeyValueSize += tlen + Bytes.SIZEOF_SHORT;
595        }
596        blockBuffer.skip(lastKeyValueSize);
597      } while (blockBuffer.hasRemaining());
598
599      // Seek to the last key we successfully read. This will happen if this is
600      // the last key/value pair in the file, in which case the following call
601      // to next() has to return false.
602      blockBuffer.moveBack(lastKeyValueSize);
603      readKeyValueLen();
604      return 1; // didn't exactly find it.
605    }
606
607    @Override
608    public Cell getNextIndexedKey() {
609      return nextIndexedKey;
610    }
611
612    @Override
613    public int seekTo(Cell key) throws IOException {
614      return seekTo(key, true);
615    }
616
617    @Override
618    public int reseekTo(Cell key) throws IOException {
619      int compared;
620      if (isSeeked()) {
621        compared = compareKey(reader.getComparator(), key);
622        if (compared < 1) {
623          // If the required key is less than or equal to current key, then
624          // don't do anything.
625          return compared;
626        } else {
627          // The comparison with no_next_index_key has to be checked
628          if (
629            this.nextIndexedKey != null && (this.nextIndexedKey
630                == KeyValueScanner.NO_NEXT_INDEXED_KEY
631              || PrivateCellUtil.compareKeyIgnoresMvcc(reader.getComparator(), key, nextIndexedKey)
632                  < 0)
633          ) {
634            // The reader shall continue to scan the current data block instead
635            // of querying the
636            // block index as long as it knows the target key is strictly
637            // smaller than
638            // the next indexed key or the current data block is the last data
639            // block.
640            return loadBlockAndSeekToKey(this.curBlock, nextIndexedKey, false, key, false);
641          }
642        }
643      }
644      // Don't rewind on a reseek operation, because reseek implies that we are
645      // always going forward in the file.
646      return seekTo(key, false);
647    }
648
649    /**
650     * An internal API function. Seek to the given key, optionally rewinding to the first key of the
651     * block before doing the seek.
652     * @param key    - a cell representing the key that we need to fetch
653     * @param rewind whether to rewind to the first key of the block before doing the seek. If this
654     *               is false, we are assuming we never go back, otherwise the result is undefined.
655     * @return -1 if the key is earlier than the first key of the file, 0 if we are at the given
656     *         key, 1 if we are past the given key -2 if the key is earlier than the first key of
657     *         the file while using a faked index key
658     */
659    public int seekTo(Cell key, boolean rewind) throws IOException {
660      HFileBlockIndex.BlockIndexReader indexReader = reader.getDataBlockIndexReader();
661      BlockWithScanInfo blockWithScanInfo = indexReader.loadDataBlockWithScanInfo(key, curBlock,
662        cacheBlocks, pread, isCompaction, getEffectiveDataBlockEncoding(), reader);
663      if (blockWithScanInfo == null || blockWithScanInfo.getHFileBlock() == null) {
664        // This happens if the key e.g. falls before the beginning of the file.
665        return -1;
666      }
667      return loadBlockAndSeekToKey(blockWithScanInfo.getHFileBlock(),
668        blockWithScanInfo.getNextIndexedKey(), rewind, key, false);
669    }
670
671    @Override
672    public boolean seekBefore(Cell key) throws IOException {
673      HFileBlock seekToBlock = reader.getDataBlockIndexReader().seekToDataBlock(key, curBlock,
674        cacheBlocks, pread, isCompaction, reader.getEffectiveEncodingInCache(isCompaction), reader);
675      if (seekToBlock == null) {
676        return false;
677      }
678      Cell firstKey = getFirstKeyCellInBlock(seekToBlock);
679      if (PrivateCellUtil.compareKeyIgnoresMvcc(reader.getComparator(), firstKey, key) >= 0) {
680        long previousBlockOffset = seekToBlock.getPrevBlockOffset();
681        // The key we are interested in
682        if (previousBlockOffset == -1) {
683          // we have a 'problem', the key we want is the first of the file.
684          releaseIfNotCurBlock(seekToBlock);
685          return false;
686        }
687
688        // The first key in the current block 'seekToBlock' is greater than the given
689        // seekBefore key. We will go ahead by reading the next block that satisfies the
690        // given key. Return the current block before reading the next one.
691        releaseIfNotCurBlock(seekToBlock);
692        // It is important that we compute and pass onDiskSize to the block
693        // reader so that it does not have to read the header separately to
694        // figure out the size. Currently, we do not have a way to do this
695        // correctly in the general case however.
696        // TODO: See https://issues.apache.org/jira/browse/HBASE-14576
697        int prevBlockSize = -1;
698        seekToBlock = reader.readBlock(previousBlockOffset, prevBlockSize, cacheBlocks, pread,
699          isCompaction, true, BlockType.DATA, getEffectiveDataBlockEncoding());
700        // TODO shortcut: seek forward in this block to the last key of the
701        // block.
702      }
703      loadBlockAndSeekToKey(seekToBlock, firstKey, true, key, true);
704      return true;
705    }
706
707    /**
708     * The curBlock will be released by shipping or close method, so only need to consider releasing
709     * the block, which was read from HFile before and not referenced by curBlock.
710     */
711    protected void releaseIfNotCurBlock(HFileBlock block) {
712      if (curBlock != block) {
713        block.release();
714      }
715    }
716
717    /**
718     * Scans blocks in the "scanned" section of the {@link HFile} until the next data block is
719     * found.
720     * @return the next block, or null if there are no more data blocks
721     */
722    @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NP_NULL_ON_SOME_PATH",
723        justification = "Yeah, unnecessary null check; could do w/ clean up")
724    protected HFileBlock readNextDataBlock() throws IOException {
725      long lastDataBlockOffset = reader.getTrailer().getLastDataBlockOffset();
726      if (curBlock == null) {
727        return null;
728      }
729      HFileBlock block = this.curBlock;
730      do {
731        if (block.getOffset() >= lastDataBlockOffset) {
732          releaseIfNotCurBlock(block);
733          return null;
734        }
735        if (block.getOffset() < 0) {
736          releaseIfNotCurBlock(block);
737          throw new IOException("Invalid block offset=" + block + ", path=" + reader.getPath());
738        }
739        // We are reading the next block without block type validation, because
740        // it might turn out to be a non-data block.
741        block = reader.readBlock(block.getOffset() + block.getOnDiskSizeWithHeader(),
742          block.getNextBlockOnDiskSize(), cacheBlocks, pread, isCompaction, true, null,
743          getEffectiveDataBlockEncoding());
744        if (block != null && !block.getBlockType().isData()) {
745          // Whatever block we read we will be returning it unless
746          // it is a datablock. Just in case the blocks are non data blocks
747          block.release();
748        }
749      } while (!block.getBlockType().isData());
750      return block;
751    }
752
753    public DataBlockEncoding getEffectiveDataBlockEncoding() {
754      return this.reader.getEffectiveEncodingInCache(isCompaction);
755    }
756
757    @Override
758    public Cell getCell() {
759      if (!isSeeked()) {
760        return null;
761      }
762
763      Cell ret;
764      int cellBufSize = getKVBufSize();
765      long seqId = 0L;
766      if (this.reader.getHFileInfo().shouldIncludeMemStoreTS()) {
767        seqId = currMemstoreTS;
768      }
769      if (blockBuffer.hasArray()) {
770        // TODO : reduce the varieties of KV here. Check if based on a boolean
771        // we can handle the 'no tags' case.
772        if (currTagsLen > 0) {
773          ret = new SizeCachedKeyValue(blockBuffer.array(),
774            blockBuffer.arrayOffset() + blockBuffer.position(), cellBufSize, seqId, currKeyLen,
775            rowLen);
776        } else {
777          ret = new SizeCachedNoTagsKeyValue(blockBuffer.array(),
778            blockBuffer.arrayOffset() + blockBuffer.position(), cellBufSize, seqId, currKeyLen,
779            rowLen);
780        }
781      } else {
782        ByteBuffer buf = blockBuffer.asSubByteBuffer(cellBufSize);
783        if (buf.isDirect()) {
784          ret = currTagsLen > 0
785            ? new SizeCachedByteBufferKeyValue(buf, buf.position(), cellBufSize, seqId, currKeyLen,
786              rowLen)
787            : new SizeCachedNoTagsByteBufferKeyValue(buf, buf.position(), cellBufSize, seqId,
788              currKeyLen, rowLen);
789        } else {
790          if (currTagsLen > 0) {
791            ret = new SizeCachedKeyValue(buf.array(), buf.arrayOffset() + buf.position(),
792              cellBufSize, seqId, currKeyLen, rowLen);
793          } else {
794            ret = new SizeCachedNoTagsKeyValue(buf.array(), buf.arrayOffset() + buf.position(),
795              cellBufSize, seqId, currKeyLen, rowLen);
796          }
797        }
798      }
799      return ret;
800    }
801
802    @Override
803    public Cell getKey() {
804      assertSeeked();
805      // Create a new object so that this getKey is cached as firstKey, lastKey
806      ObjectIntPair<ByteBuffer> keyPair = new ObjectIntPair<>();
807      blockBuffer.asSubByteBuffer(blockBuffer.position() + KEY_VALUE_LEN_SIZE, currKeyLen, keyPair);
808      ByteBuffer keyBuf = keyPair.getFirst();
809      if (keyBuf.hasArray()) {
810        return new KeyValue.KeyOnlyKeyValue(keyBuf.array(),
811          keyBuf.arrayOffset() + keyPair.getSecond(), currKeyLen);
812      } else {
813        // Better to do a copy here instead of holding on to this BB so that
814        // we could release the blocks referring to this key. This key is specifically used
815        // in HalfStoreFileReader to get the firstkey and lastkey by creating a new scanner
816        // every time. So holding onto the BB (incase of DBB) is not advised here.
817        byte[] key = new byte[currKeyLen];
818        ByteBufferUtils.copyFromBufferToArray(key, keyBuf, keyPair.getSecond(), 0, currKeyLen);
819        return new KeyValue.KeyOnlyKeyValue(key, 0, currKeyLen);
820      }
821    }
822
823    @Override
824    public ByteBuffer getValue() {
825      assertSeeked();
826      // Okie to create new Pair. Not used in hot path
827      ObjectIntPair<ByteBuffer> valuePair = new ObjectIntPair<>();
828      this.blockBuffer.asSubByteBuffer(blockBuffer.position() + KEY_VALUE_LEN_SIZE + currKeyLen,
829        currValueLen, valuePair);
830      ByteBuffer valBuf = valuePair.getFirst().duplicate();
831      valBuf.position(valuePair.getSecond());
832      valBuf.limit(currValueLen + valuePair.getSecond());
833      return valBuf.slice();
834    }
835
836    protected void setNonSeekedState() {
837      reset();
838      blockBuffer = null;
839      currKeyLen = 0;
840      currValueLen = 0;
841      currMemstoreTS = 0;
842      currMemstoreTSLen = 0;
843      currTagsLen = 0;
844    }
845
846    /**
847     * Set the position on current backing blockBuffer.
848     */
849    private void positionThisBlockBuffer() {
850      try {
851        blockBuffer.skip(getCurCellSerializedSize());
852      } catch (IllegalArgumentException e) {
853        LOG.error("Current pos = " + blockBuffer.position() + "; currKeyLen = " + currKeyLen
854          + "; currValLen = " + currValueLen + "; block limit = " + blockBuffer.limit()
855          + "; currBlock currBlockOffset = " + this.curBlock.getOffset() + "; path="
856          + reader.getPath());
857        throw e;
858      }
859    }
860
861    /**
862     * Set our selves up for the next 'next' invocation, set up next block.
863     * @return True is more to read else false if at the end.
864     */
865    private boolean positionForNextBlock() throws IOException {
866      // Methods are small so they get inlined because they are 'hot'.
867      long lastDataBlockOffset = reader.getTrailer().getLastDataBlockOffset();
868      if (this.curBlock.getOffset() >= lastDataBlockOffset) {
869        setNonSeekedState();
870        return false;
871      }
872      return isNextBlock();
873    }
874
875    private boolean isNextBlock() throws IOException {
876      // Methods are small so they get inlined because they are 'hot'.
877      HFileBlock nextBlock = readNextDataBlock();
878      if (nextBlock == null) {
879        setNonSeekedState();
880        return false;
881      }
882      updateCurrentBlock(nextBlock);
883      return true;
884    }
885
886    private final boolean _next() throws IOException {
887      // Small method so can be inlined. It is a hot one.
888      if (blockBuffer.remaining() <= 0) {
889        return positionForNextBlock();
890      }
891
892      // We are still in the same block.
893      readKeyValueLen();
894      return true;
895    }
896
897    /**
898     * Go to the next key/value in the block section. Loads the next block if necessary. If
899     * successful, {@link #getKey()} and {@link #getValue()} can be called.
900     * @return true if successfully navigated to the next key/value
901     */
902    @Override
903    public boolean next() throws IOException {
904      // This is a hot method so extreme measures taken to ensure it is small and inlineable.
905      // Checked by setting: -XX:+UnlockDiagnosticVMOptions -XX:+PrintInlining -XX:+PrintCompilation
906      assertSeeked();
907      positionThisBlockBuffer();
908      return _next();
909    }
910
911    /**
912     * Positions this scanner at the start of the file.
913     * @return false if empty file; i.e. a call to next would return false and the current key and
914     *         value are undefined.
915     */
916    @Override
917    public boolean seekTo() throws IOException {
918      if (reader == null) {
919        return false;
920      }
921
922      if (reader.getTrailer().getEntryCount() == 0) {
923        // No data blocks.
924        return false;
925      }
926
927      long firstDataBlockOffset = reader.getTrailer().getFirstDataBlockOffset();
928      if (curBlock != null && curBlock.getOffset() == firstDataBlockOffset) {
929        return processFirstDataBlock();
930      }
931
932      readAndUpdateNewBlock(firstDataBlockOffset);
933      return true;
934    }
935
936    protected boolean processFirstDataBlock() throws IOException {
937      blockBuffer.rewind();
938      readKeyValueLen();
939      return true;
940    }
941
942    protected void readAndUpdateNewBlock(long firstDataBlockOffset) throws IOException {
943      HFileBlock newBlock = reader.readBlock(firstDataBlockOffset, -1, cacheBlocks, pread,
944        isCompaction, true, BlockType.DATA, getEffectiveDataBlockEncoding());
945      if (newBlock.getOffset() < 0) {
946        releaseIfNotCurBlock(newBlock);
947        throw new IOException(
948          "Invalid offset=" + newBlock.getOffset() + ", path=" + reader.getPath());
949      }
950      updateCurrentBlock(newBlock);
951    }
952
953    protected int loadBlockAndSeekToKey(HFileBlock seekToBlock, Cell nextIndexedKey, boolean rewind,
954      Cell key, boolean seekBefore) throws IOException {
955      if (this.curBlock == null || this.curBlock.getOffset() != seekToBlock.getOffset()) {
956        updateCurrentBlock(seekToBlock);
957      } else if (rewind) {
958        blockBuffer.rewind();
959      }
960      // Update the nextIndexedKey
961      this.nextIndexedKey = nextIndexedKey;
962      return blockSeek(key, seekBefore);
963    }
964
965    /** Returns True if v &lt;= 0 or v &gt; current block buffer limit. */
966    protected final boolean checkKeyLen(final int v) {
967      return v <= 0 || v > this.blockBuffer.limit();
968    }
969
970    /** Returns True if v &lt; 0 or v &gt; current block buffer limit. */
971    protected final boolean checkLen(final int v) {
972      return v < 0 || v > this.blockBuffer.limit();
973    }
974
975    /**
976     * Check key and value lengths are wholesome.
977     */
978    protected final void checkKeyValueLen() {
979      if (checkKeyLen(this.currKeyLen) || checkLen(this.currValueLen)) {
980        throw new IllegalStateException("Invalid currKeyLen " + this.currKeyLen
981          + " or currValueLen " + this.currValueLen + ". Block offset: " + this.curBlock.getOffset()
982          + ", block length: " + this.blockBuffer.limit() + ", position: "
983          + this.blockBuffer.position() + " (without header)." + ", path=" + reader.getPath());
984      }
985    }
986
987    /**
988     * Updates the current block to be the given {@link HFileBlock}. Seeks to the the first
989     * key/value pair.
990     * @param newBlock the block read by {@link HFileReaderImpl#readBlock}, it's a totally new block
991     *                 with new allocated {@link ByteBuff}, so if no further reference to this
992     *                 block, we should release it carefully.
993     */
994    protected void updateCurrentBlock(HFileBlock newBlock) throws IOException {
995      try {
996        if (newBlock.getBlockType() != BlockType.DATA) {
997          throw new IllegalStateException(
998            "ScannerV2 works only on data blocks, got " + newBlock.getBlockType() + "; "
999              + "HFileName=" + reader.getPath() + ", " + "dataBlockEncoder="
1000              + reader.getDataBlockEncoding() + ", " + "isCompaction=" + isCompaction);
1001        }
1002        updateCurrBlockRef(newBlock);
1003        blockBuffer = newBlock.getBufferWithoutHeader();
1004        readKeyValueLen();
1005      } finally {
1006        releaseIfNotCurBlock(newBlock);
1007      }
1008      // Reset the next indexed key
1009      this.nextIndexedKey = null;
1010    }
1011
1012    protected Cell getFirstKeyCellInBlock(HFileBlock curBlock) {
1013      ByteBuff buffer = curBlock.getBufferWithoutHeader();
1014      // It is safe to manipulate this buffer because we own the buffer object.
1015      buffer.rewind();
1016      int klen = buffer.getInt();
1017      buffer.skip(Bytes.SIZEOF_INT);// Skip value len part
1018      ByteBuffer keyBuff = buffer.asSubByteBuffer(klen);
1019      if (keyBuff.hasArray()) {
1020        return new KeyValue.KeyOnlyKeyValue(keyBuff.array(),
1021          keyBuff.arrayOffset() + keyBuff.position(), klen);
1022      } else {
1023        return new ByteBufferKeyOnlyKeyValue(keyBuff, keyBuff.position(), klen);
1024      }
1025    }
1026
1027    @Override
1028    public String getKeyString() {
1029      return CellUtil.toString(getKey(), false);
1030    }
1031
1032    @Override
1033    public String getValueString() {
1034      return ByteBufferUtils.toStringBinary(getValue());
1035    }
1036
1037    public int compareKey(CellComparator comparator, Cell key) {
1038      blockBuffer.asSubByteBuffer(blockBuffer.position() + KEY_VALUE_LEN_SIZE, currKeyLen, pair);
1039      this.bufBackedKeyOnlyKv.setKey(pair.getFirst(), pair.getSecond(), currKeyLen, rowLen);
1040      return PrivateCellUtil.compareKeyIgnoresMvcc(comparator, key, this.bufBackedKeyOnlyKv);
1041    }
1042
1043    @Override
1044    public void shipped() throws IOException {
1045      this.returnBlocks(false);
1046    }
1047  }
1048
1049  @Override
1050  public Path getPath() {
1051    return path;
1052  }
1053
1054  @Override
1055  public DataBlockEncoding getDataBlockEncoding() {
1056    return dataBlockEncoder.getDataBlockEncoding();
1057  }
1058
1059  @Override
1060  public Configuration getConf() {
1061    return conf;
1062  }
1063
1064  @Override
1065  public void setConf(Configuration conf) {
1066    this.conf = conf;
1067  }
1068
1069  /** Minor versions in HFile starting with this number have hbase checksums */
1070  public static final int MINOR_VERSION_WITH_CHECKSUM = 1;
1071  /** In HFile minor version that does not support checksums */
1072  public static final int MINOR_VERSION_NO_CHECKSUM = 0;
1073
1074  /** HFile minor version that introduced pbuf filetrailer */
1075  public static final int PBUF_TRAILER_MINOR_VERSION = 2;
1076
1077  /**
1078   * The size of a (key length, value length) tuple that prefixes each entry in a data block.
1079   */
1080  public final static int KEY_VALUE_LEN_SIZE = 2 * Bytes.SIZEOF_INT;
1081
1082  /**
1083   * Retrieve block from cache. Validates the retrieved block's type vs {@code expectedBlockType}
1084   * and its encoding vs. {@code expectedDataBlockEncoding}. Unpacks the block as necessary.
1085   */
1086  private HFileBlock getCachedBlock(BlockCacheKey cacheKey, boolean cacheBlock, boolean useLock,
1087    boolean isCompaction, boolean updateCacheMetrics, BlockType expectedBlockType,
1088    DataBlockEncoding expectedDataBlockEncoding) throws IOException {
1089    // Check cache for block. If found return.
1090    BlockCache cache = cacheConf.getBlockCache().orElse(null);
1091    if (cache != null) {
1092      HFileBlock cachedBlock =
1093        (HFileBlock) cache.getBlock(cacheKey, cacheBlock, useLock, updateCacheMetrics);
1094      if (cachedBlock != null) {
1095        if (cacheConf.shouldCacheCompressed(cachedBlock.getBlockType().getCategory())) {
1096          HFileBlock compressedBlock = cachedBlock;
1097          cachedBlock = compressedBlock.unpack(hfileContext, fsBlockReader);
1098          // In case of compressed block after unpacking we can release the compressed block
1099          if (compressedBlock != cachedBlock) {
1100            compressedBlock.release();
1101          }
1102        }
1103        try {
1104          validateBlockType(cachedBlock, expectedBlockType);
1105        } catch (IOException e) {
1106          returnAndEvictBlock(cache, cacheKey, cachedBlock);
1107          throw e;
1108        }
1109
1110        if (expectedDataBlockEncoding == null) {
1111          return cachedBlock;
1112        }
1113        DataBlockEncoding actualDataBlockEncoding = cachedBlock.getDataBlockEncoding();
1114        // Block types other than data blocks always have
1115        // DataBlockEncoding.NONE. To avoid false negative cache misses, only
1116        // perform this check if cached block is a data block.
1117        if (
1118          cachedBlock.getBlockType().isData()
1119            && !actualDataBlockEncoding.equals(expectedDataBlockEncoding)
1120        ) {
1121          // This mismatch may happen if a Scanner, which is used for say a
1122          // compaction, tries to read an encoded block from the block cache.
1123          // The reverse might happen when an EncodedScanner tries to read
1124          // un-encoded blocks which were cached earlier.
1125          //
1126          // Because returning a data block with an implicit BlockType mismatch
1127          // will cause the requesting scanner to throw a disk read should be
1128          // forced here. This will potentially cause a significant number of
1129          // cache misses, so update so we should keep track of this as it might
1130          // justify the work on a CompoundScanner.
1131          if (
1132            !expectedDataBlockEncoding.equals(DataBlockEncoding.NONE)
1133              && !actualDataBlockEncoding.equals(DataBlockEncoding.NONE)
1134          ) {
1135            // If the block is encoded but the encoding does not match the
1136            // expected encoding it is likely the encoding was changed but the
1137            // block was not yet evicted. Evictions on file close happen async
1138            // so blocks with the old encoding still linger in cache for some
1139            // period of time. This event should be rare as it only happens on
1140            // schema definition change.
1141            LOG.info(
1142              "Evicting cached block with key {} because data block encoding mismatch; "
1143                + "expected {}, actual {}, path={}",
1144              cacheKey, actualDataBlockEncoding, expectedDataBlockEncoding, path);
1145            // This is an error scenario. so here we need to release the block.
1146            returnAndEvictBlock(cache, cacheKey, cachedBlock);
1147          }
1148          return null;
1149        }
1150        return cachedBlock;
1151      }
1152    }
1153    return null;
1154  }
1155
1156  private void returnAndEvictBlock(BlockCache cache, BlockCacheKey cacheKey, Cacheable block) {
1157    block.release();
1158    cache.evictBlock(cacheKey);
1159  }
1160
1161  /**
1162   * @param cacheBlock Add block to cache, if found
1163   * @return block wrapped in a ByteBuffer, with header skipped
1164   */
1165  @Override
1166  public HFileBlock getMetaBlock(String metaBlockName, boolean cacheBlock) throws IOException {
1167    if (trailer.getMetaIndexCount() == 0) {
1168      return null; // there are no meta blocks
1169    }
1170    if (metaBlockIndexReader == null) {
1171      throw new IOException(path + " meta index not loaded");
1172    }
1173
1174    byte[] mbname = Bytes.toBytes(metaBlockName);
1175    int block = metaBlockIndexReader.rootBlockContainingKey(mbname, 0, mbname.length);
1176    if (block == -1) {
1177      return null;
1178    }
1179    long blockSize = metaBlockIndexReader.getRootBlockDataSize(block);
1180
1181    // Per meta key from any given file, synchronize reads for said block. This
1182    // is OK to do for meta blocks because the meta block index is always
1183    // single-level.
1184    synchronized (metaBlockIndexReader.getRootBlockKey(block)) {
1185      // Check cache for block. If found return.
1186      long metaBlockOffset = metaBlockIndexReader.getRootBlockOffset(block);
1187      BlockCacheKey cacheKey =
1188        new BlockCacheKey(name, metaBlockOffset, this.isPrimaryReplicaReader(), BlockType.META);
1189
1190      cacheBlock &= cacheConf.shouldCacheBlockOnRead(BlockType.META.getCategory());
1191      HFileBlock cachedBlock =
1192        getCachedBlock(cacheKey, cacheBlock, false, true, true, BlockType.META, null);
1193      if (cachedBlock != null) {
1194        assert cachedBlock.isUnpacked() : "Packed block leak.";
1195        // Return a distinct 'shallow copy' of the block,
1196        // so pos does not get messed by the scanner
1197        return cachedBlock;
1198      }
1199      // Cache Miss, please load.
1200
1201      HFileBlock compressedBlock =
1202        fsBlockReader.readBlockData(metaBlockOffset, blockSize, true, false, true);
1203      HFileBlock uncompressedBlock = compressedBlock.unpack(hfileContext, fsBlockReader);
1204      if (compressedBlock != uncompressedBlock) {
1205        compressedBlock.release();
1206      }
1207
1208      // Cache the block
1209      if (cacheBlock) {
1210        cacheConf.getBlockCache().ifPresent(
1211          cache -> cache.cacheBlock(cacheKey, uncompressedBlock, cacheConf.isInMemory()));
1212      }
1213      return uncompressedBlock;
1214    }
1215  }
1216
1217  /**
1218   * If expected block is data block, we'll allocate the ByteBuff of block from
1219   * {@link org.apache.hadoop.hbase.io.ByteBuffAllocator} and it's usually an off-heap one,
1220   * otherwise it will allocate from heap.
1221   * @see org.apache.hadoop.hbase.io.hfile.HFileBlock.FSReader#readBlockData(long, long, boolean,
1222   *      boolean, boolean)
1223   */
1224  private boolean shouldUseHeap(BlockType expectedBlockType) {
1225    if (!cacheConf.getBlockCache().isPresent()) {
1226      return false;
1227    } else if (!cacheConf.isCombinedBlockCache()) {
1228      // Block to cache in LruBlockCache must be an heap one. So just allocate block memory from
1229      // heap for saving an extra off-heap to heap copying.
1230      return true;
1231    }
1232    return expectedBlockType != null && !expectedBlockType.isData();
1233  }
1234
1235  @Override
1236  public HFileBlock readBlock(long dataBlockOffset, long onDiskBlockSize, final boolean cacheBlock,
1237    boolean pread, final boolean isCompaction, boolean updateCacheMetrics,
1238    BlockType expectedBlockType, DataBlockEncoding expectedDataBlockEncoding) throws IOException {
1239    if (dataBlockIndexReader == null) {
1240      throw new IOException(path + " block index not loaded");
1241    }
1242    long trailerOffset = trailer.getLoadOnOpenDataOffset();
1243    if (dataBlockOffset < 0 || dataBlockOffset >= trailerOffset) {
1244      throw new IOException("Requested block is out of range: " + dataBlockOffset
1245        + ", lastDataBlockOffset: " + trailer.getLastDataBlockOffset()
1246        + ", trailer.getLoadOnOpenDataOffset: " + trailerOffset + ", path=" + path);
1247    }
1248    // For any given block from any given file, synchronize reads for said
1249    // block.
1250    // Without a cache, this synchronizing is needless overhead, but really
1251    // the other choice is to duplicate work (which the cache would prevent you
1252    // from doing).
1253
1254    BlockCacheKey cacheKey =
1255      new BlockCacheKey(name, dataBlockOffset, this.isPrimaryReplicaReader(), expectedBlockType);
1256    Attributes attributes = Attributes.of(BLOCK_CACHE_KEY_KEY, cacheKey.toString());
1257
1258    boolean useLock = false;
1259    IdLock.Entry lockEntry = null;
1260    final Span span = Span.current();
1261    try {
1262      while (true) {
1263        // Check cache for block. If found return.
1264        if (cacheConf.shouldReadBlockFromCache(expectedBlockType)) {
1265          if (useLock) {
1266            lockEntry = offsetLock.getLockEntry(dataBlockOffset);
1267          }
1268          // Try and get the block from the block cache. If the useLock variable is true then this
1269          // is the second time through the loop and it should not be counted as a block cache miss.
1270          HFileBlock cachedBlock = getCachedBlock(cacheKey, cacheBlock, useLock, isCompaction,
1271            updateCacheMetrics, expectedBlockType, expectedDataBlockEncoding);
1272          if (cachedBlock != null) {
1273            if (LOG.isTraceEnabled()) {
1274              LOG.trace("From Cache {}", cachedBlock);
1275            }
1276            span.addEvent("block cache hit", attributes);
1277            assert cachedBlock.isUnpacked() : "Packed block leak.";
1278            if (cachedBlock.getBlockType().isData()) {
1279              if (updateCacheMetrics) {
1280                HFile.DATABLOCK_READ_COUNT.increment();
1281              }
1282              // Validate encoding type for data blocks. We include encoding
1283              // type in the cache key, and we expect it to match on a cache hit.
1284              if (cachedBlock.getDataBlockEncoding() != dataBlockEncoder.getDataBlockEncoding()) {
1285                // Remember to release the block when in exceptional path.
1286                cacheConf.getBlockCache().ifPresent(cache -> {
1287                  returnAndEvictBlock(cache, cacheKey, cachedBlock);
1288                });
1289                throw new IOException("Cached block under key " + cacheKey + " "
1290                  + "has wrong encoding: " + cachedBlock.getDataBlockEncoding() + " (expected: "
1291                  + dataBlockEncoder.getDataBlockEncoding() + "), path=" + path);
1292              }
1293            }
1294            // Cache-hit. Return!
1295            return cachedBlock;
1296          }
1297
1298          if (!useLock && cacheBlock && cacheConf.shouldLockOnCacheMiss(expectedBlockType)) {
1299            // check cache again with lock
1300            useLock = true;
1301            continue;
1302          }
1303          // Carry on, please load.
1304        }
1305
1306        span.addEvent("block cache miss", attributes);
1307        // Load block from filesystem.
1308        HFileBlock hfileBlock = fsBlockReader.readBlockData(dataBlockOffset, onDiskBlockSize, pread,
1309          !isCompaction, shouldUseHeap(expectedBlockType));
1310        validateBlockType(hfileBlock, expectedBlockType);
1311        HFileBlock unpacked = hfileBlock.unpack(hfileContext, fsBlockReader);
1312        BlockType.BlockCategory category = hfileBlock.getBlockType().getCategory();
1313
1314        // Cache the block if necessary
1315        cacheConf.getBlockCache().ifPresent(cache -> {
1316          if (cacheBlock && cacheConf.shouldCacheBlockOnRead(category)) {
1317            cache.cacheBlock(cacheKey,
1318              cacheConf.shouldCacheCompressed(category) ? hfileBlock : unpacked,
1319              cacheConf.isInMemory());
1320          }
1321        });
1322        if (unpacked != hfileBlock) {
1323          // End of life here if hfileBlock is an independent block.
1324          hfileBlock.release();
1325        }
1326        if (updateCacheMetrics && hfileBlock.getBlockType().isData()) {
1327          HFile.DATABLOCK_READ_COUNT.increment();
1328        }
1329
1330        return unpacked;
1331      }
1332    } finally {
1333      if (lockEntry != null) {
1334        offsetLock.releaseLockEntry(lockEntry);
1335      }
1336    }
1337  }
1338
1339  @Override
1340  public boolean hasMVCCInfo() {
1341    return fileInfo.shouldIncludeMemStoreTS() && fileInfo.isDecodeMemstoreTS();
1342  }
1343
1344  /**
1345   * Compares the actual type of a block retrieved from cache or disk with its expected type and
1346   * throws an exception in case of a mismatch. Expected block type of {@link BlockType#DATA} is
1347   * considered to match the actual block type [@link {@link BlockType#ENCODED_DATA} as well.
1348   * @param block             a block retrieved from cache or disk
1349   * @param expectedBlockType the expected block type, or null to skip the check
1350   */
1351  private void validateBlockType(HFileBlock block, BlockType expectedBlockType) throws IOException {
1352    if (expectedBlockType == null) {
1353      return;
1354    }
1355    BlockType actualBlockType = block.getBlockType();
1356    if (expectedBlockType.isData() && actualBlockType.isData()) {
1357      // We consider DATA to match ENCODED_DATA for the purpose of this
1358      // verification.
1359      return;
1360    }
1361    if (actualBlockType != expectedBlockType) {
1362      throw new IOException("Expected block type " + expectedBlockType + ", " + "but got "
1363        + actualBlockType + ": " + block + ", path=" + path);
1364    }
1365  }
1366
1367  /**
1368   * @return Last key as cell in the file. May be null if file has no entries. Note that this is not
1369   *         the last row key, but it is the Cell representation of the last key
1370   */
1371  @Override
1372  public Optional<Cell> getLastKey() {
1373    return dataBlockIndexReader.isEmpty()
1374      ? Optional.empty()
1375      : Optional.of(fileInfo.getLastKeyCell());
1376  }
1377
1378  /**
1379   * @return Midkey for this file. We work with block boundaries only so returned midkey is an
1380   *         approximation only.
1381   */
1382  @Override
1383  public Optional<Cell> midKey() throws IOException {
1384    return Optional.ofNullable(dataBlockIndexReader.midkey(this));
1385  }
1386
1387  @Override
1388  public void close() throws IOException {
1389    close(cacheConf.shouldEvictOnClose());
1390  }
1391
1392  @Override
1393  public DataBlockEncoding getEffectiveEncodingInCache(boolean isCompaction) {
1394    return dataBlockEncoder.getEffectiveEncodingInCache(isCompaction);
1395  }
1396
1397  /** For testing */
1398  @Override
1399  public HFileBlock.FSReader getUncachedBlockReader() {
1400    return fsBlockReader;
1401  }
1402
1403  /**
1404   * Scanner that operates on encoded data blocks.
1405   */
1406  protected static class EncodedScanner extends HFileScannerImpl {
1407    private final HFileBlockDecodingContext decodingCtx;
1408    private final DataBlockEncoder.EncodedSeeker seeker;
1409    private final DataBlockEncoder dataBlockEncoder;
1410
1411    public EncodedScanner(HFile.Reader reader, boolean cacheBlocks, boolean pread,
1412      boolean isCompaction, HFileContext meta, Configuration conf) {
1413      super(reader, cacheBlocks, pread, isCompaction);
1414      DataBlockEncoding encoding = reader.getDataBlockEncoding();
1415      dataBlockEncoder = encoding.getEncoder();
1416      decodingCtx = dataBlockEncoder.newDataBlockDecodingContext(conf, meta);
1417      seeker = dataBlockEncoder.createSeeker(decodingCtx);
1418    }
1419
1420    @Override
1421    public boolean isSeeked() {
1422      return curBlock != null;
1423    }
1424
1425    @Override
1426    public void setNonSeekedState() {
1427      reset();
1428    }
1429
1430    /**
1431     * Updates the current block to be the given {@link HFileBlock}. Seeks to the the first
1432     * key/value pair.
1433     * @param newBlock the block to make current, and read by {@link HFileReaderImpl#readBlock},
1434     *                 it's a totally new block with new allocated {@link ByteBuff}, so if no
1435     *                 further reference to this block, we should release it carefully.
1436     */
1437    @Override
1438    protected void updateCurrentBlock(HFileBlock newBlock) throws CorruptHFileException {
1439      try {
1440        // sanity checks
1441        if (newBlock.getBlockType() != BlockType.ENCODED_DATA) {
1442          throw new IllegalStateException("EncodedScanner works only on encoded data blocks");
1443        }
1444        short dataBlockEncoderId = newBlock.getDataBlockEncodingId();
1445        if (!DataBlockEncoding.isCorrectEncoder(dataBlockEncoder, dataBlockEncoderId)) {
1446          String encoderCls = dataBlockEncoder.getClass().getName();
1447          throw new CorruptHFileException(
1448            "Encoder " + encoderCls + " doesn't support data block encoding "
1449              + DataBlockEncoding.getNameFromId(dataBlockEncoderId) + ",path=" + reader.getPath());
1450        }
1451        updateCurrBlockRef(newBlock);
1452        ByteBuff encodedBuffer = getEncodedBuffer(newBlock);
1453        seeker.setCurrentBuffer(encodedBuffer);
1454      } finally {
1455        releaseIfNotCurBlock(newBlock);
1456      }
1457      // Reset the next indexed key
1458      this.nextIndexedKey = null;
1459    }
1460
1461    private ByteBuff getEncodedBuffer(HFileBlock newBlock) {
1462      ByteBuff origBlock = newBlock.getBufferReadOnly();
1463      int pos = newBlock.headerSize() + DataBlockEncoding.ID_SIZE;
1464      origBlock.position(pos);
1465      origBlock
1466        .limit(pos + newBlock.getUncompressedSizeWithoutHeader() - DataBlockEncoding.ID_SIZE);
1467      return origBlock.slice();
1468    }
1469
1470    @Override
1471    protected boolean processFirstDataBlock() throws IOException {
1472      seeker.rewind();
1473      return true;
1474    }
1475
1476    @Override
1477    public boolean next() throws IOException {
1478      boolean isValid = seeker.next();
1479      if (!isValid) {
1480        HFileBlock newBlock = readNextDataBlock();
1481        isValid = newBlock != null;
1482        if (isValid) {
1483          updateCurrentBlock(newBlock);
1484        } else {
1485          setNonSeekedState();
1486        }
1487      }
1488      return isValid;
1489    }
1490
1491    @Override
1492    public Cell getKey() {
1493      assertValidSeek();
1494      return seeker.getKey();
1495    }
1496
1497    @Override
1498    public ByteBuffer getValue() {
1499      assertValidSeek();
1500      return seeker.getValueShallowCopy();
1501    }
1502
1503    @Override
1504    public Cell getCell() {
1505      if (this.curBlock == null) {
1506        return null;
1507      }
1508      return seeker.getCell();
1509    }
1510
1511    @Override
1512    public String getKeyString() {
1513      return CellUtil.toString(getKey(), false);
1514    }
1515
1516    @Override
1517    public String getValueString() {
1518      ByteBuffer valueBuffer = getValue();
1519      return ByteBufferUtils.toStringBinary(valueBuffer);
1520    }
1521
1522    private void assertValidSeek() {
1523      if (this.curBlock == null) {
1524        throw new NotSeekedException(reader.getPath());
1525      }
1526    }
1527
1528    @Override
1529    protected Cell getFirstKeyCellInBlock(HFileBlock curBlock) {
1530      return dataBlockEncoder.getFirstKeyCellInBlock(getEncodedBuffer(curBlock));
1531    }
1532
1533    @Override
1534    protected int loadBlockAndSeekToKey(HFileBlock seekToBlock, Cell nextIndexedKey, boolean rewind,
1535      Cell key, boolean seekBefore) throws IOException {
1536      if (this.curBlock == null || this.curBlock.getOffset() != seekToBlock.getOffset()) {
1537        updateCurrentBlock(seekToBlock);
1538      } else if (rewind) {
1539        seeker.rewind();
1540      }
1541      this.nextIndexedKey = nextIndexedKey;
1542      return seeker.seekToKeyInBlock(key, seekBefore);
1543    }
1544
1545    @Override
1546    public int compareKey(CellComparator comparator, Cell key) {
1547      return seeker.compareKey(comparator, key);
1548    }
1549  }
1550
1551  /**
1552   * Returns a buffer with the Bloom filter metadata. The caller takes ownership of the buffer.
1553   */
1554  @Override
1555  public DataInput getGeneralBloomFilterMetadata() throws IOException {
1556    return this.getBloomFilterMetadata(BlockType.GENERAL_BLOOM_META);
1557  }
1558
1559  @Override
1560  public DataInput getDeleteBloomFilterMetadata() throws IOException {
1561    return this.getBloomFilterMetadata(BlockType.DELETE_FAMILY_BLOOM_META);
1562  }
1563
1564  private DataInput getBloomFilterMetadata(BlockType blockType) throws IOException {
1565    if (
1566      blockType != BlockType.GENERAL_BLOOM_META && blockType != BlockType.DELETE_FAMILY_BLOOM_META
1567    ) {
1568      throw new RuntimeException(
1569        "Block Type: " + blockType.toString() + " is not supported, path=" + path);
1570    }
1571
1572    for (HFileBlock b : fileInfo.getLoadOnOpenBlocks()) {
1573      if (b.getBlockType() == blockType) {
1574        return b.getByteStream();
1575      }
1576    }
1577    return null;
1578  }
1579
1580  public boolean isFileInfoLoaded() {
1581    return true; // We load file info in constructor in version 2.
1582  }
1583
1584  @Override
1585  public HFileContext getFileContext() {
1586    return hfileContext;
1587  }
1588
1589  /**
1590   * Returns false if block prefetching was requested for this file and has not completed, true
1591   * otherwise
1592   */
1593  @Override
1594  public boolean prefetchComplete() {
1595    return PrefetchExecutor.isCompleted(path);
1596  }
1597
1598  /**
1599   * Create a Scanner on this file. No seeks or reads are done on creation. Call
1600   * {@link HFileScanner#seekTo(Cell)} to position an start the read. There is nothing to clean up
1601   * in a Scanner. Letting go of your references to the scanner is sufficient. NOTE: Do not use this
1602   * overload of getScanner for compactions. See
1603   * {@link #getScanner(Configuration, boolean, boolean, boolean)}
1604   * @param conf        Store configuration.
1605   * @param cacheBlocks True if we should cache blocks read in by this scanner.
1606   * @param pread       Use positional read rather than seek+read if true (pread is better for
1607   *                    random reads, seek+read is better scanning).
1608   * @return Scanner on this file.
1609   */
1610  @Override
1611  public HFileScanner getScanner(Configuration conf, boolean cacheBlocks, final boolean pread) {
1612    return getScanner(conf, cacheBlocks, pread, false);
1613  }
1614
1615  /**
1616   * Create a Scanner on this file. No seeks or reads are done on creation. Call
1617   * {@link HFileScanner#seekTo(Cell)} to position an start the read. There is nothing to clean up
1618   * in a Scanner. Letting go of your references to the scanner is sufficient. n * Store
1619   * configuration. n * True if we should cache blocks read in by this scanner. n * Use positional
1620   * read rather than seek+read if true (pread is better for random reads, seek+read is better
1621   * scanning). n * is scanner being used for a compaction?
1622   * @return Scanner on this file.
1623   */
1624  @Override
1625  public HFileScanner getScanner(Configuration conf, boolean cacheBlocks, final boolean pread,
1626    final boolean isCompaction) {
1627    if (dataBlockEncoder.useEncodedScanner()) {
1628      return new EncodedScanner(this, cacheBlocks, pread, isCompaction, this.hfileContext, conf);
1629    }
1630    return new HFileScannerImpl(this, cacheBlocks, pread, isCompaction);
1631  }
1632
1633  public int getMajorVersion() {
1634    return 3;
1635  }
1636
1637  @Override
1638  public void unbufferStream() {
1639    fsBlockReader.unbufferStream();
1640  }
1641}