View Javadoc

1   /*
2    * Copyright 2011 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.io.hfile;
21  
22  import java.io.ByteArrayInputStream;
23  import java.io.DataInput;
24  import java.io.DataInputStream;
25  import java.io.IOException;
26  import java.nio.ByteBuffer;
27  import java.util.ArrayList;
28  import java.util.List;
29  
30  import org.apache.commons.logging.Log;
31  import org.apache.commons.logging.LogFactory;
32  import org.apache.hadoop.fs.FSDataInputStream;
33  import org.apache.hadoop.fs.Path;
34  import org.apache.hadoop.hbase.KeyValue;
35  import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
36  import org.apache.hadoop.hbase.util.Bytes;
37  import org.apache.hadoop.hbase.util.IdLock;
38  import org.apache.hadoop.io.WritableUtils;
39  
40  /**
41   * {@link HFile} reader for version 2.
42   */
43  public class HFileReaderV2 extends AbstractHFileReader {
44  
45    private static final Log LOG = LogFactory.getLog(HFileReaderV2.class);
46  
47    /**
48     * The size of a (key length, value length) tuple that prefixes each entry in
49     * a data block.
50     */
51    private static int KEY_VALUE_LEN_SIZE = 2 * Bytes.SIZEOF_INT;
52  
53    private boolean includesMemstoreTS = false;
54  
55    private boolean shouldIncludeMemstoreTS() {
56      return includesMemstoreTS;
57    }
58  
59    /**
60     * A "sparse lock" implementation allowing to lock on a particular block
61     * identified by offset. The purpose of this is to avoid two clients loading
62     * the same block, and have all but one client wait to get the block from the
63     * cache.
64     */
65    private IdLock offsetLock = new IdLock();
66  
67    /**
68     * Blocks read from the load-on-open section, excluding data root index, meta
69     * index, and file info.
70     */
71    private List<HFileBlock> loadOnOpenBlocks = new ArrayList<HFileBlock>();
72  
73    /**
74     * Opens a HFile. You must load the index before you can use it by calling
75     * {@link #loadFileInfo()}.
76     *
77     * @param path Path to HFile.
78     * @param trailer File trailer.
79     * @param fsdis input stream. Caller is responsible for closing the passed
80     *          stream.
81     * @param size Length of the stream.
82     * @param closeIStream Whether to close the stream.
83     * @param cacheConf Cache configuration.
84     * @throws IOException
85     */
86    public HFileReaderV2(Path path, FixedFileTrailer trailer,
87        final FSDataInputStream fsdis, final long size,
88        final boolean closeIStream, final CacheConfig cacheConf)
89    throws IOException {
90      super(path, trailer, fsdis, size, closeIStream, cacheConf);
91  
92      trailer.expectVersion(2);
93      fsBlockReader = new HFileBlock.FSReaderV2(fsdis, compressAlgo,
94          fileSize);
95  
96      // Comparator class name is stored in the trailer in version 2.
97      comparator = trailer.createComparator();
98      dataBlockIndexReader = new HFileBlockIndex.BlockIndexReader(comparator,
99          trailer.getNumDataIndexLevels(), this);
100     metaBlockIndexReader = new HFileBlockIndex.BlockIndexReader(
101         Bytes.BYTES_RAWCOMPARATOR, 1);
102 
103     // Parse load-on-open data.
104 
105     HFileBlock.BlockIterator blockIter = fsBlockReader.blockRange(
106         trailer.getLoadOnOpenDataOffset(),
107         fileSize - trailer.getTrailerSize());
108 
109     // Data index. We also read statistics about the block index written after
110     // the root level.
111     dataBlockIndexReader.readMultiLevelIndexRoot(
112         blockIter.nextBlockAsStream(BlockType.ROOT_INDEX),
113         trailer.getDataIndexCount());
114 
115     // Meta index.
116     metaBlockIndexReader.readRootIndex(
117         blockIter.nextBlockAsStream(BlockType.ROOT_INDEX),
118         trailer.getMetaIndexCount());
119 
120     // File info
121     fileInfo = new FileInfo();
122     fileInfo.readFields(blockIter.nextBlockAsStream(BlockType.FILE_INFO));
123     lastKey = fileInfo.get(FileInfo.LASTKEY);
124     avgKeyLen = Bytes.toInt(fileInfo.get(FileInfo.AVG_KEY_LEN));
125     avgValueLen = Bytes.toInt(fileInfo.get(FileInfo.AVG_VALUE_LEN));
126     byte [] keyValueFormatVersion = fileInfo.get(HFileWriterV2.KEY_VALUE_VERSION);
127     includesMemstoreTS = (keyValueFormatVersion != null &&
128         Bytes.toInt(keyValueFormatVersion) == HFileWriterV2.KEY_VALUE_VER_WITH_MEMSTORE);
129 
130     // Store all other load-on-open blocks for further consumption.
131     HFileBlock b;
132     while ((b = blockIter.nextBlock()) != null) {
133       loadOnOpenBlocks.add(b);
134     }
135   }
136 
137   /**
138    * Create a Scanner on this file. No seeks or reads are done on creation. Call
139    * {@link HFileScanner#seekTo(byte[])} to position an start the read. There is
140    * nothing to clean up in a Scanner. Letting go of your references to the
141    * scanner is sufficient.
142    *
143    * @param cacheBlocks True if we should cache blocks read in by this scanner.
144    * @param pread Use positional read rather than seek+read if true (pread is
145    *          better for random reads, seek+read is better scanning).
146    * @param isCompaction is scanner being used for a compaction?
147    * @return Scanner on this file.
148    */
149   @Override
150   public HFileScanner getScanner(boolean cacheBlocks, final boolean pread,
151       final boolean isCompaction) {
152     return new ScannerV2(this, cacheBlocks, pread, isCompaction);
153   }
154 
155   /**
156    * @param metaBlockName
157    * @param cacheBlock Add block to cache, if found
158    * @return block wrapped in a ByteBuffer, with header skipped
159    * @throws IOException
160    */
161   @Override
162   public ByteBuffer getMetaBlock(String metaBlockName, boolean cacheBlock)
163       throws IOException {
164     if (trailer.getMetaIndexCount() == 0) {
165       return null; // there are no meta blocks
166     }
167     if (metaBlockIndexReader == null) {
168       throw new IOException("Meta index not loaded");
169     }
170 
171     byte[] mbname = Bytes.toBytes(metaBlockName);
172     int block = metaBlockIndexReader.rootBlockContainingKey(mbname, 0,
173         mbname.length);
174     if (block == -1)
175       return null;
176     long blockSize = metaBlockIndexReader.getRootBlockDataSize(block);
177     long startTimeNs = System.nanoTime();
178 
179     // Per meta key from any given file, synchronize reads for said block. This
180     // is OK to do for meta blocks because the meta block index is always
181     // single-level.
182     synchronized (metaBlockIndexReader.getRootBlockKey(block)) {
183       metaLoads.incrementAndGet();
184 
185       // Check cache for block. If found return.
186       long metaBlockOffset = metaBlockIndexReader.getRootBlockOffset(block);
187       String cacheKey = HFile.getBlockCacheKey(name, metaBlockOffset);
188 
189       cacheBlock &= cacheConf.shouldCacheDataOnRead();
190       if (cacheConf.isBlockCacheEnabled()) {
191         HFileBlock cachedBlock =
192           (HFileBlock) cacheConf.getBlockCache().getBlock(cacheKey, cacheBlock);
193         if (cachedBlock != null) {
194           // Return a distinct 'shallow copy' of the block,
195           // so pos does not get messed by the scanner
196           cacheHits.incrementAndGet();
197           return cachedBlock.getBufferWithoutHeader();
198         }
199         // Cache Miss, please load.
200       }
201 
202       HFileBlock metaBlock = fsBlockReader.readBlockData(metaBlockOffset,
203           blockSize, -1, true);
204 
205       HFile.readTimeNano.addAndGet(System.nanoTime() - startTimeNs);
206       HFile.readOps.incrementAndGet();
207 
208       // Cache the block
209       if (cacheBlock) {
210         cacheConf.getBlockCache().cacheBlock(cacheKey, metaBlock,
211             cacheConf.isInMemory());
212       }
213 
214       return metaBlock.getBufferWithoutHeader();
215     }
216   }
217 
218   /**
219    * Read in a file block.
220    *
221    * @param dataBlockOffset offset to read.
222    * @param onDiskBlockSize size of the block
223    * @param cacheBlock
224    * @param pread Use positional read instead of seek+read (positional is better
225    *          doing random reads whereas seek+read is better scanning).
226    * @param isCompaction is this block being read as part of a compaction
227    * @return Block wrapped in a ByteBuffer.
228    * @throws IOException
229    */
230   @Override
231   public HFileBlock readBlock(long dataBlockOffset, long onDiskBlockSize,
232       boolean cacheBlock, boolean pread, final boolean isCompaction)
233       throws IOException {
234     if (dataBlockIndexReader == null) {
235       throw new IOException("Block index not loaded");
236     }
237     if (dataBlockOffset < 0
238         || dataBlockOffset >= trailer.getLoadOnOpenDataOffset()) {
239       throw new IOException("Requested block is out of range: "
240           + dataBlockOffset + ", lastDataBlockOffset: "
241           + trailer.getLastDataBlockOffset());
242     }
243     // For any given block from any given file, synchronize reads for said
244     // block.
245     // Without a cache, this synchronizing is needless overhead, but really
246     // the other choice is to duplicate work (which the cache would prevent you
247     // from doing).
248 
249     String cacheKey = HFile.getBlockCacheKey(name, dataBlockOffset);
250     IdLock.Entry lockEntry = offsetLock.getLockEntry(dataBlockOffset);
251     try {
252       blockLoads.incrementAndGet();
253 
254       // Check cache for block. If found return.
255       cacheBlock &= cacheConf.shouldCacheDataOnRead();
256       if (cacheConf.isBlockCacheEnabled()) {
257         HFileBlock cachedBlock =
258           (HFileBlock) cacheConf.getBlockCache().getBlock(cacheKey, cacheBlock);
259         if (cachedBlock != null) {
260           cacheHits.incrementAndGet();
261 
262           return cachedBlock;
263         }
264         // Carry on, please load.
265       }
266 
267       // Load block from filesystem.
268       long startTimeNs = System.nanoTime();
269       HFileBlock dataBlock = fsBlockReader.readBlockData(dataBlockOffset,
270           onDiskBlockSize, -1, pread);
271 
272       HFile.readTimeNano.addAndGet(System.nanoTime() - startTimeNs);
273       HFile.readOps.incrementAndGet();
274 
275       // Cache the block
276       if (cacheBlock) {
277         cacheConf.getBlockCache().cacheBlock(cacheKey, dataBlock,
278             cacheConf.isInMemory());
279       }
280 
281       return dataBlock;
282     } finally {
283       offsetLock.releaseLockEntry(lockEntry);
284     }
285   }
286 
287   /**
288    * @return Last key in the file. May be null if file has no entries. Note that
289    *         this is not the last row key, but rather the byte form of the last
290    *         KeyValue.
291    */
292   @Override
293   public byte[] getLastKey() {
294     return dataBlockIndexReader.isEmpty() ? null : lastKey;
295   }
296 
297   /**
298    * @return Midkey for this file. We work with block boundaries only so
299    *         returned midkey is an approximation only.
300    * @throws IOException
301    */
302   @Override
303   public byte[] midkey() throws IOException {
304     return dataBlockIndexReader.midkey();
305   }
306 
307   @Override
308   public void close() throws IOException {
309     close(cacheConf.shouldEvictOnClose());
310   }
311 
312   public void close(boolean evictOnClose) throws IOException {
313     if (evictOnClose && cacheConf.isBlockCacheEnabled()) {
314       int numEvicted = cacheConf.getBlockCache().evictBlocksByPrefix(name
315           + HFile.CACHE_KEY_SEPARATOR);
316       if (LOG.isTraceEnabled()) {
317         LOG.trace("On close, file=" + name + " evicted=" + numEvicted
318           + " block(s)");
319       }
320     }
321     if (closeIStream && istream != null) {
322       istream.close();
323       istream = null;
324     }
325   }
326 
327   /**
328    * Implementation of {@link HFileScanner} interface.
329    */
330   protected static class ScannerV2 extends AbstractHFileReader.Scanner {
331     private HFileBlock block;
332     private HFileReaderV2 reader;
333 
334     public ScannerV2(HFileReaderV2 r, boolean cacheBlocks,
335         final boolean pread, final boolean isCompaction) {
336       super(cacheBlocks, pread, isCompaction);
337       this.reader = r;
338     }
339 
340     @Override
341     public HFileReaderV2 getReader() {
342       return reader;
343     }
344 
345     @Override
346     public KeyValue getKeyValue() {
347       if (!isSeeked())
348         return null;
349 
350       KeyValue ret = new KeyValue(blockBuffer.array(), blockBuffer.arrayOffset()
351           + blockBuffer.position());
352       if (this.reader.shouldIncludeMemstoreTS()) {
353         ret.setMemstoreTS(currMemstoreTS);
354       }
355       return ret;
356     }
357 
358     @Override
359     public ByteBuffer getKey() {
360       assertSeeked();
361       return ByteBuffer.wrap(
362           blockBuffer.array(),
363           blockBuffer.arrayOffset() + blockBuffer.position()
364               + KEY_VALUE_LEN_SIZE, currKeyLen).slice();
365     }
366 
367     @Override
368     public ByteBuffer getValue() {
369       assertSeeked();
370       return ByteBuffer.wrap(
371           blockBuffer.array(),
372           blockBuffer.arrayOffset() + blockBuffer.position()
373               + KEY_VALUE_LEN_SIZE + currKeyLen, currValueLen).slice();
374     }
375 
376     private void setNonSeekedState() {
377       block = null;
378       blockBuffer = null;
379       currKeyLen = 0;
380       currValueLen = 0;
381       currMemstoreTS = 0;
382       currMemstoreTSLen = 0;
383     }
384 
385     /**
386      * Go to the next key/value in the block section. Loads the next block if
387      * necessary. If successful, {@link #getKey()} and {@link #getValue()} can
388      * be called.
389      *
390      * @return true if successfully navigated to the next key/value
391      */
392     @Override
393     public boolean next() throws IOException {
394       assertSeeked();
395 
396       try {
397         blockBuffer.position(blockBuffer.position() + KEY_VALUE_LEN_SIZE
398             + currKeyLen + currValueLen + currMemstoreTSLen);
399       } catch (IllegalArgumentException e) {
400         LOG.error("Current pos = " + blockBuffer.position()
401             + "; currKeyLen = " + currKeyLen + "; currValLen = "
402             + currValueLen + "; block limit = " + blockBuffer.limit()
403             + "; HFile name = " + reader.getName()
404             + "; currBlock currBlockOffset = " + block.getOffset());
405         throw e;
406       }
407 
408       if (blockBuffer.remaining() <= 0) {
409         long lastDataBlockOffset =
410             reader.getTrailer().getLastDataBlockOffset();
411 
412         if (block.getOffset() >= lastDataBlockOffset) {
413           setNonSeekedState();
414           return false;
415         }
416 
417         // read the next block
418         HFileBlock nextBlock = readNextDataBlock();
419         if (nextBlock == null) {
420           setNonSeekedState();
421           return false;
422         }
423 
424         updateCurrBlock(nextBlock);
425         return true;
426       }
427 
428       // We are still in the same block.
429       readKeyValueLen();
430       return true;
431     }
432 
433     /**
434      * Scans blocks in the "scanned" section of the {@link HFile} until the next
435      * data block is found.
436      *
437      * @return the next block, or null if there are no more data blocks
438      * @throws IOException
439      */
440     private HFileBlock readNextDataBlock() throws IOException {
441       long lastDataBlockOffset = reader.getTrailer().getLastDataBlockOffset();
442       if (block == null)
443         return null;
444 
445       HFileBlock curBlock = block;
446 
447       do {
448         if (curBlock.getOffset() >= lastDataBlockOffset)
449           return null;
450 
451         if (curBlock.getOffset() < 0) {
452           throw new IOException("Invalid block file offset: " + block);
453         }
454         curBlock = reader.readBlock(curBlock.getOffset()
455             + curBlock.getOnDiskSizeWithHeader(),
456             curBlock.getNextBlockOnDiskSizeWithHeader(), cacheBlocks, pread,
457             isCompaction);
458       } while (!curBlock.getBlockType().equals(BlockType.DATA));
459 
460       return curBlock;
461     }
462 
463     /**
464      * Positions this scanner at the start of the file.
465      *
466      * @return false if empty file; i.e. a call to next would return false and
467      *         the current key and value are undefined.
468      * @throws IOException
469      */
470     @Override
471     public boolean seekTo() throws IOException {
472       if (reader == null) {
473         return false;
474       }
475 
476       if (reader.getTrailer().getEntryCount() == 0) {
477         // No data blocks.
478         return false;
479       }
480 
481       long firstDataBlockOffset =
482           reader.getTrailer().getFirstDataBlockOffset();
483       if (block != null && block.getOffset() == firstDataBlockOffset) {
484         blockBuffer.rewind();
485         readKeyValueLen();
486         return true;
487       }
488 
489       block = reader.readBlock(firstDataBlockOffset, -1, cacheBlocks, pread,
490           isCompaction);
491       if (block.getOffset() < 0) {
492         throw new IOException("Invalid block offset: " + block.getOffset());
493       }
494       updateCurrBlock(block);
495       return true;
496     }
497 
498     @Override
499     public int seekTo(byte[] key) throws IOException {
500       return seekTo(key, 0, key.length);
501     }
502 
503     /**
504      * An internal API function. Seek to the given key, optionally rewinding to
505      * the first key of the block before doing the seek.
506      *
507      * @param key key byte array
508      * @param offset key offset in the key byte array
509      * @param length key length
510      * @param rewind whether to rewind to the first key of the block before
511      *        doing the seek. If this is false, we are assuming we never go
512      *        back, otherwise the result is undefined.
513      * @return -1 if the key is earlier than the first key of the file,
514      *         0 if we are at the given key, and 1 if we are past the given key
515      * @throws IOException
516      */
517     private int seekTo(byte[] key, int offset, int length, boolean rewind)
518         throws IOException {
519       HFileBlockIndex.BlockIndexReader indexReader =
520           reader.getDataBlockIndexReader();
521       HFileBlock seekToBlock = indexReader.seekToDataBlock(key, offset, length,
522           block, cacheBlocks, pread, isCompaction);
523 
524       if (seekToBlock == null) {
525         // This happens if the key e.g. falls before the beginning of the file.
526         return -1;
527       }
528       return loadBlockAndSeekToKey(seekToBlock, rewind, key, offset, length,
529           false);
530     }
531 
532     @Override
533     public int seekTo(byte[] key, int offset, int length) throws IOException {
534       // Always rewind to the first key of the block, because the given key
535       // might be before or after the current key.
536       return seekTo(key, offset, length, true);
537     }
538 
539     @Override
540     public int reseekTo(byte[] key) throws IOException {
541       return reseekTo(key, 0, key.length);
542     }
543 
544     @Override
545     public int reseekTo(byte[] key, int offset, int length) throws IOException {
546       if (isSeeked()) {
547         ByteBuffer bb = getKey();
548         int compared = reader.getComparator().compare(key, offset,
549             length, bb.array(), bb.arrayOffset(), bb.limit());
550         if (compared < 1) {
551           // If the required key is less than or equal to current key, then
552           // don't do anything.
553           return compared;
554         }
555       }
556 
557       // Don't rewind on a reseek operation, because reseek implies that we are
558       // always going forward in the file.
559       return seekTo(key, offset, length, false);
560     }
561 
562     private int loadBlockAndSeekToKey(HFileBlock seekToBlock, boolean rewind,
563         byte[] key, int offset, int length, boolean seekBefore)
564         throws IOException {
565       if (block == null || block.getOffset() != seekToBlock.getOffset()) {
566         updateCurrBlock(seekToBlock);
567       } else if (rewind) {
568         blockBuffer.rewind();
569       }
570       return blockSeek(key, offset, length, seekBefore);
571     }
572 
573     /**
574      * Updates the current block to be the given {@link HFileBlock}. Seeks to
575      * the the first key/value pair.
576      *
577      * @param newBlock the block to make current
578      */
579     private void updateCurrBlock(HFileBlock newBlock) {
580       block = newBlock;
581       blockBuffer = block.getBufferWithoutHeader();
582       readKeyValueLen();
583       blockFetches++;
584     }
585 
586     private final void readKeyValueLen() {
587       blockBuffer.mark();
588       currKeyLen = blockBuffer.getInt();
589       currValueLen = blockBuffer.getInt();
590       blockBuffer.reset();
591       if (this.reader.shouldIncludeMemstoreTS()) {
592         try {
593           int memstoreTSOffset = blockBuffer.arrayOffset() + blockBuffer.position()
594                                   + KEY_VALUE_LEN_SIZE + currKeyLen + currValueLen;
595           currMemstoreTS = Bytes.readVLong(blockBuffer.array(), memstoreTSOffset);
596           currMemstoreTSLen = WritableUtils.getVIntSize(currMemstoreTS);
597         } catch (Exception e) {
598           throw new RuntimeException("Error reading memstoreTS. " + e);
599         }
600       }
601 
602       if (currKeyLen < 0 || currValueLen < 0
603           || currKeyLen > blockBuffer.limit()
604           || currValueLen > blockBuffer.limit()) {
605         throw new IllegalStateException("Invalid currKeyLen " + currKeyLen
606             + " or currValueLen " + currValueLen + ". Block offset: "
607             + block.getOffset() + ", block length: " + blockBuffer.limit()
608             + ", position: " + blockBuffer.position() + " (without header).");
609       }
610     }
611 
612     /**
613      * Within a loaded block, seek looking for the first key that is smaller
614      * than (or equal to?) the key we are interested in.
615      *
616      * A note on the seekBefore: if you have seekBefore = true, AND the first
617      * key in the block = key, then you'll get thrown exceptions. The caller has
618      * to check for that case and load the previous block as appropriate.
619      *
620      * @param key the key to find
621      * @param seekBefore find the key before the given key in case of exact
622      *          match.
623      * @return 0 in case of an exact key match, 1 in case of an inexact match
624      */
625     private int blockSeek(byte[] key, int offset, int length,
626         boolean seekBefore) {
627       int klen, vlen;
628       long memstoreTS = 0;
629       int memstoreTSLen = 0;
630       int lastKeyValueSize = -1;
631       do {
632         blockBuffer.mark();
633         klen = blockBuffer.getInt();
634         vlen = blockBuffer.getInt();
635         blockBuffer.reset();
636         if (this.reader.shouldIncludeMemstoreTS()) {
637           try {
638             int memstoreTSOffset = blockBuffer.arrayOffset() + blockBuffer.position()
639                                   + KEY_VALUE_LEN_SIZE + klen + vlen;
640             memstoreTS = Bytes.readVLong(blockBuffer.array(), memstoreTSOffset);
641             memstoreTSLen = WritableUtils.getVIntSize(memstoreTS);
642           } catch (Exception e) {
643             throw new RuntimeException("Error reading memstoreTS. " + e);
644           }
645         }
646 
647         int keyOffset = blockBuffer.arrayOffset() + blockBuffer.position()
648             + KEY_VALUE_LEN_SIZE;
649         int comp = reader.getComparator().compare(key, offset, length,
650             blockBuffer.array(), keyOffset, klen);
651 
652         if (comp == 0) {
653           if (seekBefore) {
654             if (lastKeyValueSize < 0) {
655               throw new IllegalStateException("blockSeek with seekBefore "
656                   + "at the first key of the block: key="
657                   + Bytes.toStringBinary(key) + ", blockOffset="
658                   + block.getOffset() + ", onDiskSize="
659                   + block.getOnDiskSizeWithHeader());
660             }
661             blockBuffer.position(blockBuffer.position() - lastKeyValueSize);
662             readKeyValueLen();
663             return 1; // non exact match.
664           }
665           currKeyLen = klen;
666           currValueLen = vlen;
667           if (this.reader.shouldIncludeMemstoreTS()) {
668             currMemstoreTS = memstoreTS;
669             currMemstoreTSLen = memstoreTSLen;
670           }
671           return 0; // indicate exact match
672         }
673 
674         if (comp < 0) {
675           if (lastKeyValueSize > 0)
676             blockBuffer.position(blockBuffer.position() - lastKeyValueSize);
677           readKeyValueLen();
678           return 1;
679         }
680 
681         // The size of this key/value tuple, including key/value length fields.
682         lastKeyValueSize = klen + vlen + memstoreTSLen + KEY_VALUE_LEN_SIZE;
683         blockBuffer.position(blockBuffer.position() + lastKeyValueSize);
684       } while (blockBuffer.remaining() > 0);
685 
686       // Seek to the last key we successfully read. This will happen if this is
687       // the last key/value pair in the file, in which case the following call
688       // to next() has to return false.
689       blockBuffer.position(blockBuffer.position() - lastKeyValueSize);
690       readKeyValueLen();
691       return 1; // didn't exactly find it.
692     }
693 
694     @Override
695     public boolean seekBefore(byte[] key) throws IOException {
696       return seekBefore(key, 0, key.length);
697     }
698 
699     private ByteBuffer getFirstKeyInBlock(HFileBlock curBlock) {
700       ByteBuffer buffer = curBlock.getBufferWithoutHeader();
701       // It is safe to manipulate this buffer because we own the buffer object.
702       buffer.rewind();
703       int klen = buffer.getInt();
704       buffer.getInt();
705       ByteBuffer keyBuff = buffer.slice();
706       keyBuff.limit(klen);
707       keyBuff.rewind();
708       return keyBuff;
709     }
710 
711     @Override
712     public boolean seekBefore(byte[] key, int offset, int length)
713         throws IOException {
714       HFileBlock seekToBlock =
715           reader.getDataBlockIndexReader().seekToDataBlock(key, offset,
716               length, block, cacheBlocks, pread, isCompaction);
717       if (seekToBlock == null) {
718         return false;
719       }
720       ByteBuffer firstKey = getFirstKeyInBlock(seekToBlock);
721       if (reader.getComparator().compare(firstKey.array(),
722           firstKey.arrayOffset(), firstKey.limit(), key, offset, length) == 0)
723       {
724         long previousBlockOffset = seekToBlock.getPrevBlockOffset();
725         // The key we are interested in
726         if (previousBlockOffset == -1) {
727           // we have a 'problem', the key we want is the first of the file.
728           return false;
729         }
730 
731         // It is important that we compute and pass onDiskSize to the block
732         // reader so that it does not have to read the header separately to
733         // figure out the size.
734         seekToBlock = reader.readBlock(previousBlockOffset,
735             seekToBlock.getOffset() - previousBlockOffset, cacheBlocks,
736             pread, isCompaction);
737 
738         // TODO shortcut: seek forward in this block to the last key of the
739         // block.
740       }
741       loadBlockAndSeekToKey(seekToBlock, true, key, offset, length, true);
742       return true;
743     }
744 
745     @Override
746     public String getKeyString() {
747       return Bytes.toStringBinary(blockBuffer.array(),
748           blockBuffer.arrayOffset() + blockBuffer.position()
749               + KEY_VALUE_LEN_SIZE, currKeyLen);
750     }
751 
752     @Override
753     public String getValueString() {
754       return Bytes.toString(blockBuffer.array(), blockBuffer.arrayOffset()
755           + blockBuffer.position() + KEY_VALUE_LEN_SIZE + currKeyLen,
756           currValueLen);
757     }
758 
759   }
760 
761   /**
762    * Returns a buffer with the Bloom filter metadata. The caller takes
763    * ownership of the buffer.
764    */
765   @Override
766   public DataInput getBloomFilterMetadata() throws IOException {
767     for (HFileBlock b : loadOnOpenBlocks)
768       if (b.getBlockType() == BlockType.BLOOM_META)
769         return b.getByteStream();
770     return null;
771   }
772 
773   @Override
774   public boolean isFileInfoLoaded() {
775     return true; // We load file info in constructor in version 2.
776   }
777 
778 }