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  
28  import org.apache.commons.logging.Log;
29  import org.apache.commons.logging.LogFactory;
30  import org.apache.hadoop.fs.FSDataInputStream;
31  import org.apache.hadoop.fs.Path;
32  import org.apache.hadoop.hbase.KeyValue;
33  import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
34  import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
35  import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
36  import org.apache.hadoop.hbase.util.Bytes;
37  import org.apache.hadoop.io.IOUtils;
38  import org.apache.hadoop.io.RawComparator;
39  
40  import com.google.common.base.Preconditions;
41  
42  /**
43   * {@link HFile} reader for version 1.
44   */
45  public class HFileReaderV1 extends AbstractHFileReader {
46    private static final Log LOG = LogFactory.getLog(HFileReaderV1.class);
47  
48    private volatile boolean fileInfoLoaded = false;
49  
50    /**
51     * Opens a HFile.  You must load the index before you can
52     * use it by calling {@link #loadFileInfo()}.
53     *
54     * @param fsdis input stream.  Caller is responsible for closing the passed
55     * stream.
56     * @param size Length of the stream.
57     * @param cacheConf cache references and configuration
58     */
59    public HFileReaderV1(Path path, FixedFileTrailer trailer,
60        final FSDataInputStream fsdis, final long size,
61        final boolean closeIStream,
62        final CacheConfig cacheConf) {
63      super(path, trailer, fsdis, size, closeIStream, cacheConf);
64  
65      trailer.expectVersion(1);
66      fsBlockReader = new HFileBlock.FSReaderV1(fsdis, compressAlgo, fileSize);
67    }
68  
69    private byte[] readAllIndex(final FSDataInputStream in,
70        final long indexOffset, final int indexSize) throws IOException {
71      byte[] allIndex = new byte[indexSize];
72      in.seek(indexOffset);
73      IOUtils.readFully(in, allIndex, 0, allIndex.length);
74  
75      return allIndex;
76    }
77  
78    /**
79     * Read in the index and file info.
80     *
81     * @return A map of fileinfo data.
82     * @see Writer#appendFileInfo(byte[], byte[])
83     * @throws IOException
84     */
85    @Override
86    public FileInfo loadFileInfo() throws IOException {
87      if (fileInfoLoaded)
88        return fileInfo;
89  
90      // Read in the fileinfo and get what we need from it.
91      istream.seek(trailer.getFileInfoOffset());
92      fileInfo = new FileInfo();
93      fileInfo.readFields(istream);
94      lastKey = fileInfo.get(FileInfo.LASTKEY);
95      avgKeyLen = Bytes.toInt(fileInfo.get(FileInfo.AVG_KEY_LEN));
96      avgValueLen = Bytes.toInt(fileInfo.get(FileInfo.AVG_VALUE_LEN));
97  
98      // Comparator is stored in the file info in version 1.
99      String clazzName = Bytes.toString(fileInfo.get(FileInfo.COMPARATOR));
100     comparator = getComparator(clazzName);
101 
102     dataBlockIndexReader =
103         new HFileBlockIndex.BlockIndexReader(comparator, 1);
104     metaBlockIndexReader =
105         new HFileBlockIndex.BlockIndexReader(Bytes.BYTES_RAWCOMPARATOR, 1);
106 
107     int sizeToLoadOnOpen = (int) (fileSize - trailer.getLoadOnOpenDataOffset() -
108         trailer.getTrailerSize());
109     byte[] dataAndMetaIndex = readAllIndex(istream,
110         trailer.getLoadOnOpenDataOffset(), sizeToLoadOnOpen);
111 
112     ByteArrayInputStream bis = new ByteArrayInputStream(dataAndMetaIndex);
113     DataInputStream dis = new DataInputStream(bis);
114 
115     // Read in the data index.
116     if (trailer.getDataIndexCount() > 0)
117       BlockType.INDEX_V1.readAndCheck(dis);
118     dataBlockIndexReader.readRootIndex(dis, trailer.getDataIndexCount());
119 
120     // Read in the metadata index.
121     if (trailer.getMetaIndexCount() > 0)
122       BlockType.INDEX_V1.readAndCheck(dis);
123     metaBlockIndexReader.readRootIndex(dis, trailer.getMetaIndexCount());
124 
125     fileInfoLoaded = true;
126     return fileInfo;
127   }
128 
129   /**
130    * Creates comparator from the given class name.
131    *
132    * @param clazzName the comparator class name read from the trailer
133    * @return an instance of the comparator to use
134    * @throws IOException in case comparator class name is invalid
135    */
136   @SuppressWarnings("unchecked")
137   private RawComparator<byte[]> getComparator(final String clazzName)
138   throws IOException {
139     if (clazzName == null || clazzName.length() == 0) {
140       return null;
141     }
142     try {
143       return (RawComparator<byte[]>)Class.forName(clazzName).newInstance();
144     } catch (InstantiationException e) {
145       throw new IOException(e);
146     } catch (IllegalAccessException e) {
147       throw new IOException(e);
148     } catch (ClassNotFoundException e) {
149       throw new IOException(e);
150     }
151   }
152 
153   /**
154    * Create a Scanner on this file. No seeks or reads are done on creation. Call
155    * {@link HFileScanner#seekTo(byte[])} to position an start the read. There is
156    * nothing to clean up in a Scanner. Letting go of your references to the
157    * scanner is sufficient.
158    *
159    * @param cacheBlocks True if we should cache blocks read in by this scanner.
160    * @param pread Use positional read rather than seek+read if true (pread is
161    *          better for random reads, seek+read is better scanning).
162    * @param isCompaction is scanner being used for a compaction?
163    * @return Scanner on this file.
164    */
165   @Override
166   public HFileScanner getScanner(boolean cacheBlocks, final boolean pread,
167                                 final boolean isCompaction) {
168     return new ScannerV1(this, cacheBlocks, pread, isCompaction);
169   }
170 
171   /**
172    * @param key Key to search.
173    * @return Block number of the block containing the key or -1 if not in this
174    * file.
175    */
176   protected int blockContainingKey(final byte[] key, int offset, int length) {
177     Preconditions.checkState(!dataBlockIndexReader.isEmpty(),
178         "Block index not loaded");
179     return dataBlockIndexReader.rootBlockContainingKey(key, offset, length);
180   }
181 
182   /**
183    * @param metaBlockName
184    * @param cacheBlock Add block to cache, if found
185    * @return Block wrapped in a ByteBuffer
186    * @throws IOException
187    */
188   @Override
189   public ByteBuffer getMetaBlock(String metaBlockName, boolean cacheBlock)
190       throws IOException {
191     if (trailer.getMetaIndexCount() == 0) {
192       return null; // there are no meta blocks
193     }
194     if (metaBlockIndexReader == null) {
195       throw new IOException("Meta index not loaded");
196     }
197 
198     byte[] nameBytes = Bytes.toBytes(metaBlockName);
199     int block = metaBlockIndexReader.rootBlockContainingKey(nameBytes, 0,
200         nameBytes.length);
201     if (block == -1)
202       return null;
203     long offset = metaBlockIndexReader.getRootBlockOffset(block);
204     long nextOffset;
205     if (block == metaBlockIndexReader.getRootBlockCount() - 1) {
206       nextOffset = trailer.getFileInfoOffset();
207     } else {
208       nextOffset = metaBlockIndexReader.getRootBlockOffset(block + 1);
209     }
210 
211     long startTimeNs = System.nanoTime();
212 
213     String cacheKey = HFile.getBlockCacheKey(name, offset);
214 
215     // Per meta key from any given file, synchronize reads for said block
216     synchronized (metaBlockIndexReader.getRootBlockKey(block)) {
217       metaLoads.incrementAndGet();
218       // Check cache for block.  If found return.
219       if (cacheConf.isBlockCacheEnabled()) {
220         HFileBlock cachedBlock =
221           (HFileBlock) cacheConf.getBlockCache().getBlock(cacheKey,
222               cacheConf.shouldCacheDataOnRead());
223         if (cachedBlock != null) {
224           cacheHits.incrementAndGet();
225           return cachedBlock.getBufferWithoutHeader();
226         }
227         // Cache Miss, please load.
228       }
229 
230       HFileBlock hfileBlock = fsBlockReader.readBlockData(offset,
231           nextOffset - offset, metaBlockIndexReader.getRootBlockDataSize(block),
232           true);
233       hfileBlock.expectType(BlockType.META);
234 
235       HFile.readTimeNano.addAndGet(System.nanoTime() - startTimeNs);
236       HFile.readOps.incrementAndGet();
237 
238       // Cache the block
239       if (cacheConf.shouldCacheDataOnRead() && cacheBlock) {
240         cacheConf.getBlockCache().cacheBlock(cacheKey, hfileBlock,
241             cacheConf.isInMemory());
242       }
243 
244       return hfileBlock.getBufferWithoutHeader();
245     }
246   }
247 
248   /**
249    * Read in a file block.
250    * @param block Index of block to read.
251    * @param pread Use positional read instead of seek+read (positional is
252    * better doing random reads whereas seek+read is better scanning).
253    * @param isCompaction is this block being read as part of a compaction
254    * @return Block wrapped in a ByteBuffer.
255    * @throws IOException
256    */
257   ByteBuffer readBlockBuffer(int block, boolean cacheBlock,
258       final boolean pread, final boolean isCompaction) throws IOException {
259     if (dataBlockIndexReader == null) {
260       throw new IOException("Block index not loaded");
261     }
262     if (block < 0 || block >= dataBlockIndexReader.getRootBlockCount()) {
263       throw new IOException("Requested block is out of range: " + block +
264         ", max: " + dataBlockIndexReader.getRootBlockCount());
265     }
266 
267     long offset = dataBlockIndexReader.getRootBlockOffset(block);
268     String cacheKey = HFile.getBlockCacheKey(name, offset);
269 
270     // For any given block from any given file, synchronize reads for said
271     // block.
272     // Without a cache, this synchronizing is needless overhead, but really
273     // the other choice is to duplicate work (which the cache would prevent you
274     // from doing).
275     synchronized (dataBlockIndexReader.getRootBlockKey(block)) {
276       blockLoads.incrementAndGet();
277 
278       // Check cache for block.  If found return.
279       if (cacheConf.isBlockCacheEnabled()) {
280         HFileBlock cachedBlock =
281           (HFileBlock) cacheConf.getBlockCache().getBlock(cacheKey,
282               cacheConf.shouldCacheDataOnRead());
283         if (cachedBlock != null) {
284           cacheHits.incrementAndGet();
285           return cachedBlock.getBufferWithoutHeader();
286         }
287         // Carry on, please load.
288       }
289 
290       // Load block from filesystem.
291       long startTimeNs = System.nanoTime();
292       long nextOffset;
293 
294       if (block == dataBlockIndexReader.getRootBlockCount() - 1) {
295         // last block!  The end of data block is first meta block if there is
296         // one or if there isn't, the fileinfo offset.
297         nextOffset = (metaBlockIndexReader.getRootBlockCount() == 0) ?
298             this.trailer.getFileInfoOffset() :
299             metaBlockIndexReader.getRootBlockOffset(0);
300       } else {
301         nextOffset = dataBlockIndexReader.getRootBlockOffset(block + 1);
302       }
303 
304       HFileBlock hfileBlock = fsBlockReader.readBlockData(offset, nextOffset
305           - offset, dataBlockIndexReader.getRootBlockDataSize(block), pread);
306       hfileBlock.expectType(BlockType.DATA);
307       ByteBuffer buf = hfileBlock.getBufferWithoutHeader();
308 
309       HFile.readTimeNano.addAndGet(System.nanoTime() - startTimeNs);
310       HFile.readOps.incrementAndGet();
311 
312       // Cache the block
313       if (cacheConf.shouldCacheDataOnRead() && cacheBlock) {
314         cacheConf.getBlockCache().cacheBlock(cacheKey, hfileBlock,
315             cacheConf.isInMemory());
316       }
317 
318       return buf;
319     }
320   }
321 
322   /**
323    * @return Last key in the file.  May be null if file has no entries.
324    * Note that this is not the last rowkey, but rather the byte form of
325    * the last KeyValue.
326    */
327   public byte[] getLastKey() {
328     if (!fileInfoLoaded) {
329       throw new RuntimeException("Load file info first");
330     }
331     return dataBlockIndexReader.isEmpty() ? null : lastKey;
332   }
333 
334   /**
335    * @return Midkey for this file. We work with block boundaries only so
336    *         returned midkey is an approximation only.
337    *
338    * @throws IOException
339    */
340   @Override
341   public byte[] midkey() throws IOException {
342     Preconditions.checkState(isFileInfoLoaded(), "File info is not loaded");
343     Preconditions.checkState(!dataBlockIndexReader.isEmpty(),
344         "Data block index is not loaded or is empty");
345     return dataBlockIndexReader.midkey();
346   }
347 
348   @Override
349   public void close() throws IOException {
350     close(cacheConf.shouldEvictOnClose());
351   }
352 
353   @Override
354   public void close(boolean evictOnClose) throws IOException {
355     if (evictOnClose) {
356       int numEvicted = 0;
357       for (int i = 0; i < dataBlockIndexReader.getRootBlockCount(); i++) {
358         if (cacheConf.getBlockCache().evictBlock(HFile.getBlockCacheKey(name,
359             dataBlockIndexReader.getRootBlockOffset(i))))
360           numEvicted++;
361       }
362       LOG.debug("On close of file " + name + " evicted " + numEvicted
363           + " block(s) of " + dataBlockIndexReader.getRootBlockCount()
364           + " total blocks");
365     }
366     if (this.closeIStream && this.istream != null) {
367       this.istream.close();
368       this.istream = null;
369     }
370   }
371 
372   /**
373    * Implementation of {@link HFileScanner} interface.
374    */
375   protected static class ScannerV1 extends AbstractHFileReader.Scanner {
376     private final HFileReaderV1 reader;
377     private int currBlock;
378 
379     public ScannerV1(HFileReaderV1 reader, boolean cacheBlocks,
380         final boolean pread, final boolean isCompaction) {
381       super(cacheBlocks, pread, isCompaction);
382       this.reader = reader;
383     }
384 
385     @Override
386     public KeyValue getKeyValue() {
387       if (blockBuffer == null) {
388         return null;
389       }
390       return new KeyValue(blockBuffer.array(), blockBuffer.arrayOffset()
391           + blockBuffer.position() - 8);
392     }
393 
394     @Override
395     public ByteBuffer getKey() {
396       Preconditions.checkState(blockBuffer != null && currKeyLen > 0,
397           "you need to seekTo() before calling getKey()");
398 
399       ByteBuffer keyBuff = blockBuffer.slice();
400       keyBuff.limit(currKeyLen);
401       keyBuff.rewind();
402       // Do keyBuff.asReadOnly()?
403       return keyBuff;
404     }
405 
406     @Override
407     public ByteBuffer getValue() {
408       if (blockBuffer == null || currKeyLen == 0) {
409         throw new RuntimeException(
410             "you need to seekTo() before calling getValue()");
411       }
412 
413       // TODO: Could this be done with one ByteBuffer rather than create two?
414       ByteBuffer valueBuff = blockBuffer.slice();
415       valueBuff.position(currKeyLen);
416       valueBuff = valueBuff.slice();
417       valueBuff.limit(currValueLen);
418       valueBuff.rewind();
419       return valueBuff;
420     }
421 
422     @Override
423     public boolean next() throws IOException {
424       if (blockBuffer == null) {
425         throw new IOException("Next called on non-seeked scanner");
426       }
427 
428       try {
429         blockBuffer.position(blockBuffer.position() + currKeyLen
430             + currValueLen);
431       } catch (IllegalArgumentException e) {
432         LOG.error("Current pos = " + blockBuffer.position() +
433                   "; currKeyLen = " + currKeyLen +
434                   "; currValLen = " + currValueLen +
435                   "; block limit = " + blockBuffer.limit() +
436                   "; HFile name = " + reader.getName() +
437                   "; currBlock id = " + currBlock, e);
438         throw e;
439       }
440       if (blockBuffer.remaining() <= 0) {
441         // LOG.debug("Fetch next block");
442         currBlock++;
443         if (currBlock >= reader.getDataBlockIndexReader().getRootBlockCount()) {
444           // damn we are at the end
445           currBlock = 0;
446           blockBuffer = null;
447           return false;
448         }
449         blockBuffer = reader.readBlockBuffer(currBlock, cacheBlocks, pread,
450             isCompaction);
451         currKeyLen = blockBuffer.getInt();
452         currValueLen = blockBuffer.getInt();
453         blockFetches++;
454         return true;
455       }
456 
457       currKeyLen = blockBuffer.getInt();
458       currValueLen = blockBuffer.getInt();
459       return true;
460     }
461 
462     @Override
463     public int seekTo(byte[] key) throws IOException {
464       return seekTo(key, 0, key.length);
465     }
466 
467     @Override
468     public int seekTo(byte[] key, int offset, int length) throws IOException {
469       int b = reader.blockContainingKey(key, offset, length);
470       if (b < 0) return -1; // falls before the beginning of the file! :-(
471       // Avoid re-reading the same block (that'd be dumb).
472       loadBlock(b, true);
473       return blockSeek(key, offset, length, false);
474     }
475 
476     @Override
477     public int reseekTo(byte[] key) throws IOException {
478       return reseekTo(key, 0, key.length);
479     }
480 
481     @Override
482     public int reseekTo(byte[] key, int offset, int length)
483         throws IOException {
484       if (blockBuffer != null && currKeyLen != 0) {
485         ByteBuffer bb = getKey();
486         int compared = reader.getComparator().compare(key, offset,
487             length, bb.array(), bb.arrayOffset(), bb.limit());
488         if (compared <= 0) {
489           // If the required key is less than or equal to current key, then
490           // don't do anything.
491           return compared;
492         }
493       }
494 
495       int b = reader.blockContainingKey(key, offset, length);
496       if (b < 0) {
497         return -1;
498       }
499       loadBlock(b, false);
500       return blockSeek(key, offset, length, false);
501     }
502 
503     /**
504      * Within a loaded block, seek looking for the first key
505      * that is smaller than (or equal to?) the key we are interested in.
506      *
507      * A note on the seekBefore - if you have seekBefore = true, AND the
508      * first key in the block = key, then you'll get thrown exceptions.
509      * @param key to find
510      * @param seekBefore find the key before the exact match.
511      * @return
512      */
513     private int blockSeek(byte[] key, int offset, int length,
514         boolean seekBefore) {
515       int klen, vlen;
516       int lastLen = 0;
517       do {
518         klen = blockBuffer.getInt();
519         vlen = blockBuffer.getInt();
520         int comp = reader.getComparator().compare(key, offset, length,
521             blockBuffer.array(),
522             blockBuffer.arrayOffset() + blockBuffer.position(), klen);
523         if (comp == 0) {
524           if (seekBefore) {
525             blockBuffer.position(blockBuffer.position() - lastLen - 16);
526             currKeyLen = blockBuffer.getInt();
527             currValueLen = blockBuffer.getInt();
528             return 1; // non exact match.
529           }
530           currKeyLen = klen;
531           currValueLen = vlen;
532           return 0; // indicate exact match
533         }
534         if (comp < 0) {
535           // go back one key:
536           blockBuffer.position(blockBuffer.position() - lastLen - 16);
537           currKeyLen = blockBuffer.getInt();
538           currValueLen = blockBuffer.getInt();
539           return 1;
540         }
541         blockBuffer.position(blockBuffer.position() + klen + vlen);
542         lastLen = klen + vlen;
543       } while (blockBuffer.remaining() > 0);
544 
545       // ok we are at the end, so go back a littleeeeee....
546       // The 8 in the below is intentionally different to the 16s in the above
547       // Do the math you you'll figure it.
548       blockBuffer.position(blockBuffer.position() - lastLen - 8);
549       currKeyLen = blockBuffer.getInt();
550       currValueLen = blockBuffer.getInt();
551       return 1; // didn't exactly find it.
552     }
553 
554     @Override
555     public boolean seekBefore(byte[] key) throws IOException {
556       return seekBefore(key, 0, key.length);
557     }
558 
559     @Override
560     public boolean seekBefore(byte[] key, int offset, int length)
561     throws IOException {
562       int b = reader.blockContainingKey(key, offset, length);
563       if (b < 0)
564         return false; // key is before the start of the file.
565 
566       // Question: does this block begin with 'key'?
567       byte[] firstkKey = reader.getDataBlockIndexReader().getRootBlockKey(b);
568       if (reader.getComparator().compare(firstkKey, 0, firstkKey.length,
569           key, offset, length) == 0) {
570         // Ok the key we're interested in is the first of the block, so go back
571         // by one.
572         if (b == 0) {
573           // we have a 'problem', the key we want is the first of the file.
574           return false;
575         }
576         b--;
577         // TODO shortcut: seek forward in this block to the last key of the
578         // block.
579       }
580       loadBlock(b, true);
581       blockSeek(key, offset, length, true);
582       return true;
583     }
584 
585     @Override
586     public String getKeyString() {
587       return Bytes.toStringBinary(blockBuffer.array(),
588           blockBuffer.arrayOffset() + blockBuffer.position(), currKeyLen);
589     }
590 
591     @Override
592     public String getValueString() {
593       return Bytes.toString(blockBuffer.array(), blockBuffer.arrayOffset() +
594         blockBuffer.position() + currKeyLen, currValueLen);
595     }
596 
597     @Override
598     public Reader getReader() {
599       return reader;
600     }
601 
602     @Override
603     public boolean seekTo() throws IOException {
604       if (reader.getDataBlockIndexReader().isEmpty()) {
605         return false;
606       }
607       if (blockBuffer != null && currBlock == 0) {
608         blockBuffer.rewind();
609         currKeyLen = blockBuffer.getInt();
610         currValueLen = blockBuffer.getInt();
611         return true;
612       }
613       currBlock = 0;
614       blockBuffer = reader.readBlockBuffer(currBlock, cacheBlocks, pread,
615           isCompaction);
616       currKeyLen = blockBuffer.getInt();
617       currValueLen = blockBuffer.getInt();
618       blockFetches++;
619       return true;
620     }
621 
622     private void loadBlock(int bloc, boolean rewind) throws IOException {
623       if (blockBuffer == null) {
624         blockBuffer = reader.readBlockBuffer(bloc, cacheBlocks, pread,
625             isCompaction);
626         currBlock = bloc;
627         blockFetches++;
628       } else {
629         if (bloc != currBlock) {
630           blockBuffer = reader.readBlockBuffer(bloc, cacheBlocks, pread,
631               isCompaction);
632           currBlock = bloc;
633           blockFetches++;
634         } else {
635           // we are already in the same block, just rewind to seek again.
636           if (rewind) {
637             blockBuffer.rewind();
638           }
639           else {
640             // Go back by (size of rowlength + size of valuelength) = 8 bytes
641             blockBuffer.position(blockBuffer.position()-8);
642           }
643         }
644       }
645     }
646 
647   }
648 
649   @Override
650   public HFileBlock readBlock(long offset, long onDiskBlockSize,
651       boolean cacheBlock, boolean pread, boolean isCompaction) {
652     throw new UnsupportedOperationException();
653   }
654 
655   @Override
656   public DataInput getBloomFilterMetadata() throws IOException {
657     ByteBuffer buf = getMetaBlock(HFileWriterV1.BLOOM_FILTER_META_KEY, false);
658     if (buf == null)
659       return null;
660     ByteArrayInputStream bais = new ByteArrayInputStream(buf.array(),
661         buf.arrayOffset(), buf.limit());
662     return new DataInputStream(bais);
663   }
664 
665   @Override
666   public boolean isFileInfoLoaded() {
667     return fileInfoLoaded;
668   }
669 
670 }