View Javadoc

1   /**
2    * Copyright 2009 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.Closeable;
23  import java.io.DataInputStream;
24  import java.io.DataOutputStream;
25  import java.io.IOException;
26  import java.io.InputStream;
27  import java.io.OutputStream;
28  import java.nio.ByteBuffer;
29  import java.util.ArrayList;
30  import java.util.Arrays;
31  import java.util.List;
32  import java.util.Map;
33  import java.util.SortedSet;
34  
35  import org.apache.commons.cli.CommandLine;
36  import org.apache.commons.cli.CommandLineParser;
37  import org.apache.commons.cli.HelpFormatter;
38  import org.apache.commons.cli.Options;
39  import org.apache.commons.cli.PosixParser;
40  import org.apache.commons.logging.Log;
41  import org.apache.commons.logging.LogFactory;
42  import org.apache.hadoop.conf.Configuration;
43  import org.apache.hadoop.fs.FSDataInputStream;
44  import org.apache.hadoop.fs.FSDataOutputStream;
45  import org.apache.hadoop.fs.FileStatus;
46  import org.apache.hadoop.fs.FileSystem;
47  import org.apache.hadoop.fs.Path;
48  import org.apache.hadoop.fs.PathFilter;
49  import org.apache.hadoop.hbase.KeyValue.KeyComparator;
50  import org.apache.hadoop.hbase.HBaseConfiguration;
51  import org.apache.hadoop.hbase.HRegionInfo;
52  import org.apache.hadoop.hbase.KeyValue;
53  import org.apache.hadoop.hbase.io.HbaseMapWritable;
54  import org.apache.hadoop.hbase.io.HeapSize;
55  import org.apache.hadoop.hbase.util.Bytes;
56  import org.apache.hadoop.hbase.util.ClassSize;
57  import org.apache.hadoop.hbase.util.FSUtils;
58  import org.apache.hadoop.io.IOUtils;
59  import org.apache.hadoop.io.RawComparator;
60  import org.apache.hadoop.io.Writable;
61  import org.apache.hadoop.io.compress.Compressor;
62  import org.apache.hadoop.io.compress.Decompressor;
63  
64  /**
65   * File format for hbase.
66   * A file of sorted key/value pairs. Both keys and values are byte arrays.
67   * <p>
68   * The memory footprint of a HFile includes the following (below is taken from the
69   * <a
70   * href=https://issues.apache.org/jira/browse/HADOOP-3315>TFile</a> documentation
71   * but applies also to HFile):
72   * <ul>
73   * <li>Some constant overhead of reading or writing a compressed block.
74   * <ul>
75   * <li>Each compressed block requires one compression/decompression codec for
76   * I/O.
77   * <li>Temporary space to buffer the key.
78   * <li>Temporary space to buffer the value.
79   * </ul>
80   * <li>HFile index, which is proportional to the total number of Data Blocks.
81   * The total amount of memory needed to hold the index can be estimated as
82   * (56+AvgKeySize)*NumBlocks.
83   * </ul>
84   * Suggestions on performance optimization.
85   * <ul>
86   * <li>Minimum block size. We recommend a setting of minimum block size between
87   * 8KB to 1MB for general usage. Larger block size is preferred if files are
88   * primarily for sequential access. However, it would lead to inefficient random
89   * access (because there are more data to decompress). Smaller blocks are good
90   * for random access, but require more memory to hold the block index, and may
91   * be slower to create (because we must flush the compressor stream at the
92   * conclusion of each data block, which leads to an FS I/O flush). Further, due
93   * to the internal caching in Compression codec, the smallest possible block
94   * size would be around 20KB-30KB.
95   * <li>The current implementation does not offer true multi-threading for
96   * reading. The implementation uses FSDataInputStream seek()+read(), which is
97   * shown to be much faster than positioned-read call in single thread mode.
98   * However, it also means that if multiple threads attempt to access the same
99   * HFile (using multiple scanners) simultaneously, the actual I/O is carried out
100  * sequentially even if they access different DFS blocks (Reexamine! pread seems
101  * to be 10% faster than seek+read in my testing -- stack).
102  * <li>Compression codec. Use "none" if the data is not very compressable (by
103  * compressable, I mean a compression ratio at least 2:1). Generally, use "lzo"
104  * as the starting point for experimenting. "gz" overs slightly better
105  * compression ratio over "lzo" but requires 4x CPU to compress and 2x CPU to
106  * decompress, comparing to "lzo".
107  * </ul>
108  *
109  * For more on the background behind HFile, see <a
110  * href=https://issues.apache.org/jira/browse/HBASE-61>HBASE-61</a>.
111  * <p>
112  * File is made of data blocks followed by meta data blocks (if any), a fileinfo
113  * block, data block index, meta data block index, and a fixed size trailer
114  * which records the offsets at which file changes content type.
115  * <pre>&lt;data blocks>&lt;meta blocks>&lt;fileinfo>&lt;data index>&lt;meta index>&lt;trailer></pre>
116  * Each block has a bit of magic at its start.  Block are comprised of
117  * key/values.  In data blocks, they are both byte arrays.  Metadata blocks are
118  * a String key and a byte array value.  An empty file looks like this:
119  * <pre>&lt;fileinfo>&lt;trailer></pre>.  That is, there are not data nor meta
120  * blocks present.
121  * <p>
122  * TODO: Do scanners need to be able to take a start and end row?
123  * TODO: Should BlockIndex know the name of its file?  Should it have a Path
124  * that points at its file say for the case where an index lives apart from
125  * an HFile instance?
126  */
127 public class HFile {
128   static final Log LOG = LogFactory.getLog(HFile.class);
129 
130   /* These values are more or less arbitrary, and they are used as a
131    * form of check to make sure the file isn't completely corrupt.
132    */
133   final static byte [] DATABLOCKMAGIC =
134     {'D', 'A', 'T', 'A', 'B', 'L', 'K', 42 };
135   final static byte [] INDEXBLOCKMAGIC =
136     { 'I', 'D', 'X', 'B', 'L', 'K', 41, 43 };
137   final static byte [] METABLOCKMAGIC =
138     { 'M', 'E', 'T', 'A', 'B', 'L', 'K', 99 };
139   final static byte [] TRAILERBLOCKMAGIC =
140     { 'T', 'R', 'A', 'B', 'L', 'K', 34, 36 };
141 
142   /**
143    * Maximum length of key in HFile.
144    */
145   public final static int MAXIMUM_KEY_LENGTH = Integer.MAX_VALUE;
146 
147   /**
148    * Default blocksize for hfile.
149    */
150   public final static int DEFAULT_BLOCKSIZE = 64 * 1024;
151 
152   /**
153    * Default compression: none.
154    */
155   public final static Compression.Algorithm DEFAULT_COMPRESSION_ALGORITHM =
156     Compression.Algorithm.NONE;
157   /** Default compression name: none. */
158   public final static String DEFAULT_COMPRESSION =
159     DEFAULT_COMPRESSION_ALGORITHM.getName();
160 
161   // For measuring latency of "typical" reads and writes
162   private static volatile long readOps;
163   private static volatile long readTime;
164   private static volatile long writeOps;
165   private static volatile long writeTime;
166 
167   public static final long getReadOps() {
168     long ret = readOps;
169     readOps = 0;
170     return ret;
171   }
172 
173   public static final long getReadTime() {
174     long ret = readTime;
175     readTime = 0;
176     return ret;
177   }
178 
179   public static final long getWriteOps() {
180     long ret = writeOps;
181     writeOps = 0;
182     return ret;
183   }
184 
185   public static final long getWriteTime() {
186     long ret = writeTime;
187     writeTime = 0;
188     return ret;
189   }
190 
191   /**
192    * HFile Writer.
193    */
194   public static class Writer implements Closeable {
195     // FileSystem stream to write on.
196     private FSDataOutputStream outputStream;
197     // True if we opened the <code>outputStream</code> (and so will close it).
198     private boolean closeOutputStream;
199 
200     // Name for this object used when logging or in toString.  Is either
201     // the result of a toString on stream or else toString of passed file Path.
202     protected String name;
203 
204     // Total uncompressed bytes, maybe calculate a compression ratio later.
205     private long totalBytes = 0;
206 
207     // Total # of key/value entries, ie: how many times add() was called.
208     private int entryCount = 0;
209 
210     // Used calculating average key and value lengths.
211     private long keylength = 0;
212     private long valuelength = 0;
213 
214     // Used to ensure we write in order.
215     private final RawComparator<byte []> rawComparator;
216 
217     // A stream made per block written.
218     private DataOutputStream out;
219 
220     // Number of uncompressed bytes per block.  Reinitialized when we start
221     // new block.
222     private int blocksize;
223 
224     // Offset where the current block began.
225     private long blockBegin;
226 
227     // First key in a block (Not first key in file).
228     private byte [] firstKey = null;
229 
230     // Key previously appended.  Becomes the last key in the file.
231     private byte [] lastKeyBuffer = null;
232     private int lastKeyOffset = -1;
233     private int lastKeyLength = -1;
234 
235     // See {@link BlockIndex}. Below four fields are used to write the block
236     // index.
237     ArrayList<byte[]> blockKeys = new ArrayList<byte[]>();
238     // Block offset in backing stream.
239     ArrayList<Long> blockOffsets = new ArrayList<Long>();
240     // Raw (decompressed) data size.
241     ArrayList<Integer> blockDataSizes = new ArrayList<Integer>();
242 
243     // Meta block system.
244     private ArrayList<byte []> metaNames = new ArrayList<byte []>();
245     private ArrayList<Writable> metaData = new ArrayList<Writable>();
246 
247     // Used compression.  Used even if no compression -- 'none'.
248     private final Compression.Algorithm compressAlgo;
249     private Compressor compressor;
250 
251     // Special datastructure to hold fileinfo.
252     private FileInfo fileinfo = new FileInfo();
253 
254     // May be null if we were passed a stream.
255     private Path path = null;
256 
257     /**
258      * Constructor that uses all defaults for compression and block size.
259      * @param fs
260      * @param path
261      * @throws IOException
262      */
263     public Writer(FileSystem fs, Path path)
264     throws IOException {
265       this(fs, path, DEFAULT_BLOCKSIZE, (Compression.Algorithm) null, null);
266     }
267 
268     /**
269      * Constructor that takes a Path.
270      * @param fs
271      * @param path
272      * @param blocksize
273      * @param compress
274      * @param comparator
275      * @throws IOException
276      * @throws IOException
277      */
278     public Writer(FileSystem fs, Path path, int blocksize,
279       String compress, final KeyComparator comparator)
280     throws IOException {
281       this(fs, path, blocksize,
282         compress == null? DEFAULT_COMPRESSION_ALGORITHM:
283           Compression.getCompressionAlgorithmByName(compress),
284         comparator);
285     }
286 
287     /**
288      * Constructor that takes a Path.
289      * @param fs
290      * @param path
291      * @param blocksize
292      * @param compress
293      * @param comparator
294      * @throws IOException
295      */
296     public Writer(FileSystem fs, Path path, int blocksize,
297       Compression.Algorithm compress,
298       final KeyComparator comparator)
299     throws IOException {
300       this(fs.create(path), blocksize, compress, comparator);
301       this.closeOutputStream = true;
302       this.name = path.toString();
303       this.path = path;
304     }
305 
306     /**
307      * Constructor that takes a stream.
308      * @param ostream Stream to use.
309      * @param blocksize
310      * @param compress
311      * @param c RawComparator to use.
312      * @throws IOException
313      */
314     public Writer(final FSDataOutputStream ostream, final int blocksize,
315       final String  compress, final KeyComparator c)
316     throws IOException {
317       this(ostream, blocksize,
318         Compression.getCompressionAlgorithmByName(compress), c);
319     }
320 
321     /**
322      * Constructor that takes a stream.
323      * @param ostream Stream to use.
324      * @param blocksize
325      * @param compress
326      * @param c
327      * @throws IOException
328      */
329     public Writer(final FSDataOutputStream ostream, final int blocksize,
330       final Compression.Algorithm  compress, final KeyComparator c)
331     throws IOException {
332       this.outputStream = ostream;
333       this.closeOutputStream = false;
334       this.blocksize = blocksize;
335       this.rawComparator = c == null? Bytes.BYTES_RAWCOMPARATOR: c;
336       this.name = this.outputStream.toString();
337       this.compressAlgo = compress == null?
338         DEFAULT_COMPRESSION_ALGORITHM: compress;
339     }
340 
341     /*
342      * If at block boundary, opens new block.
343      * @throws IOException
344      */
345     private void checkBlockBoundary() throws IOException {
346       if (this.out != null && this.out.size() < blocksize) return;
347       finishBlock();
348       newBlock();
349     }
350 
351     /*
352      * Do the cleanup if a current block.
353      * @throws IOException
354      */
355     private void finishBlock() throws IOException {
356       if (this.out == null) return;
357       long now = System.currentTimeMillis();
358 
359       int size = releaseCompressingStream(this.out);
360       this.out = null;
361       blockKeys.add(firstKey);
362       blockOffsets.add(Long.valueOf(blockBegin));
363       blockDataSizes.add(Integer.valueOf(size));
364       this.totalBytes += size;
365 
366       writeTime += System.currentTimeMillis() - now;
367       writeOps++;
368     }
369 
370     /*
371      * Ready a new block for writing.
372      * @throws IOException
373      */
374     private void newBlock() throws IOException {
375       // This is where the next block begins.
376       blockBegin = outputStream.getPos();
377       this.out = getCompressingStream();
378       this.out.write(DATABLOCKMAGIC);
379       firstKey = null;
380     }
381 
382     /*
383      * Sets up a compressor and creates a compression stream on top of
384      * this.outputStream.  Get one per block written.
385      * @return A compressing stream; if 'none' compression, returned stream
386      * does not compress.
387      * @throws IOException
388      * @see {@link #releaseCompressingStream(DataOutputStream)}
389      */
390     private DataOutputStream getCompressingStream() throws IOException {
391       this.compressor = compressAlgo.getCompressor();
392       // Get new DOS compression stream.  In tfile, the DOS, is not closed,
393       // just finished, and that seems to be fine over there.  TODO: Check
394       // no memory retention of the DOS.  Should I disable the 'flush' on the
395       // DOS as the BCFile over in tfile does?  It wants to make it so flushes
396       // don't go through to the underlying compressed stream.  Flush on the
397       // compressed downstream should be only when done.  I was going to but
398       // looks like when we call flush in here, its legitimate flush that
399       // should go through to the compressor.
400       OutputStream os =
401         this.compressAlgo.createCompressionStream(this.outputStream,
402         this.compressor, 0);
403       return new DataOutputStream(os);
404     }
405 
406     /*
407      * Let go of block compressor and compressing stream gotten in call
408      * {@link #getCompressingStream}.
409      * @param dos
410      * @return How much was written on this stream since it was taken out.
411      * @see #getCompressingStream()
412      * @throws IOException
413      */
414     private int releaseCompressingStream(final DataOutputStream dos)
415     throws IOException {
416       dos.flush();
417       this.compressAlgo.returnCompressor(this.compressor);
418       this.compressor = null;
419       return dos.size();
420     }
421 
422     /**
423      * Add a meta block to the end of the file. Call before close().
424      * Metadata blocks are expensive.  Fill one with a bunch of serialized data
425      * rather than do a metadata block per metadata instance.  If metadata is
426      * small, consider adding to file info using
427      * {@link #appendFileInfo(byte[], byte[])}
428      * @param metaBlockName name of the block
429      * @param content will call readFields to get data later (DO NOT REUSE)
430      */
431     public void appendMetaBlock(String metaBlockName, Writable content) {
432       byte[] key = Bytes.toBytes(metaBlockName);
433       int i;
434       for (i = 0; i < metaNames.size(); ++i) {
435         // stop when the current key is greater than our own
436         byte[] cur = metaNames.get(i);
437         if (this.rawComparator.compare(cur, 0, cur.length, key, 0, key.length)
438             > 0) {
439           break;
440         }
441       }
442       metaNames.add(i, key);
443       metaData.add(i, content);
444     }
445 
446     /**
447      * Add to the file info.  Added key value can be gotten out of the return
448      * from {@link Reader#loadFileInfo()}.
449      * @param k Key
450      * @param v Value
451      * @throws IOException
452      */
453     public void appendFileInfo(final byte [] k, final byte [] v)
454     throws IOException {
455       appendFileInfo(this.fileinfo, k, v, true);
456     }
457 
458     static FileInfo appendFileInfo(FileInfo fi, final byte [] k, final byte [] v,
459       final boolean checkPrefix)
460     throws IOException {
461       if (k == null || v == null) {
462         throw new NullPointerException("Key nor value may be null");
463       }
464       if (checkPrefix &&
465           Bytes.startsWith(k, FileInfo.RESERVED_PREFIX_BYTES)) {
466         throw new IOException("Keys with a " + FileInfo.RESERVED_PREFIX +
467           " are reserved");
468       }
469       fi.put(k, v);
470       return fi;
471     }
472 
473     /**
474      * @return Path or null if we were passed a stream rather than a Path.
475      */
476     public Path getPath() {
477       return this.path;
478     }
479 
480     @Override
481     public String toString() {
482       return "writer=" + this.name + ", compression=" +
483         this.compressAlgo.getName();
484     }
485 
486     /**
487      * Add key/value to file.
488      * Keys must be added in an order that agrees with the Comparator passed
489      * on construction.
490      * @param kv KeyValue to add.  Cannot be empty nor null.
491      * @throws IOException
492      */
493     public void append(final KeyValue kv)
494     throws IOException {
495       append(kv.getBuffer(), kv.getKeyOffset(), kv.getKeyLength(),
496         kv.getBuffer(), kv.getValueOffset(), kv.getValueLength());
497     }
498 
499     /**
500      * Add key/value to file.
501      * Keys must be added in an order that agrees with the Comparator passed
502      * on construction.
503      * @param key Key to add.  Cannot be empty nor null.
504      * @param value Value to add.  Cannot be empty nor null.
505      * @throws IOException
506      */
507     public void append(final byte [] key, final byte [] value)
508     throws IOException {
509       append(key, 0, key.length, value, 0, value.length);
510     }
511 
512     /**
513      * Add key/value to file.
514      * Keys must be added in an order that agrees with the Comparator passed
515      * on construction.
516      * @param key
517      * @param koffset
518      * @param klength
519      * @param value
520      * @param voffset
521      * @param vlength
522      * @throws IOException
523      */
524     private void append(final byte [] key, final int koffset, final int klength,
525         final byte [] value, final int voffset, final int vlength)
526     throws IOException {
527       boolean dupKey = checkKey(key, koffset, klength);
528       checkValue(value, voffset, vlength);
529       if (!dupKey) {
530         checkBlockBoundary();
531       }
532       // Write length of key and value and then actual key and value bytes.
533       this.out.writeInt(klength);
534       this.keylength += klength;
535       this.out.writeInt(vlength);
536       this.valuelength += vlength;
537       this.out.write(key, koffset, klength);
538       this.out.write(value, voffset, vlength);
539       // Are we the first key in this block?
540       if (this.firstKey == null) {
541         // Copy the key.
542         this.firstKey = new byte [klength];
543         System.arraycopy(key, koffset, this.firstKey, 0, klength);
544       }
545       this.lastKeyBuffer = key;
546       this.lastKeyOffset = koffset;
547       this.lastKeyLength = klength;
548       this.entryCount ++;
549     }
550 
551     /*
552      * @param key Key to check.
553      * @return the flag of duplicate Key or not
554      * @throws IOException
555      */
556     private boolean checkKey(final byte [] key, final int offset, final int length)
557     throws IOException {
558       boolean dupKey = false;
559 
560       if (key == null || length <= 0) {
561         throw new IOException("Key cannot be null or empty");
562       }
563       if (length > MAXIMUM_KEY_LENGTH) {
564         throw new IOException("Key length " + length + " > " +
565           MAXIMUM_KEY_LENGTH);
566       }
567       if (this.lastKeyBuffer != null) {
568         int keyComp = this.rawComparator.compare(this.lastKeyBuffer, this.lastKeyOffset,
569             this.lastKeyLength, key, offset, length);
570         if (keyComp > 0) {
571           throw new IOException("Added a key not lexically larger than" +
572             " previous key=" + Bytes.toStringBinary(key, offset, length) +
573             ", lastkey=" + Bytes.toStringBinary(this.lastKeyBuffer, this.lastKeyOffset,
574                 this.lastKeyLength));
575         } else if (keyComp == 0) {
576           dupKey = true;
577         }
578       }
579       return dupKey;
580     }
581 
582     private void checkValue(final byte [] value, final int offset,
583         final int length) throws IOException {
584       if (value == null) {
585         throw new IOException("Value cannot be null");
586       }
587     }
588 
589     public long getTotalBytes() {
590       return this.totalBytes;
591     }
592 
593     public void close() throws IOException {
594       if (this.outputStream == null) {
595         return;
596       }
597       // Write out the end of the data blocks, then write meta data blocks.
598       // followed by fileinfo, data block index and meta block index.
599 
600       finishBlock();
601 
602       FixedFileTrailer trailer = new FixedFileTrailer();
603 
604       // Write out the metadata blocks if any.
605       ArrayList<Long> metaOffsets = null;
606       ArrayList<Integer> metaDataSizes = null;
607       if (metaNames.size() > 0) {
608         metaOffsets = new ArrayList<Long>(metaNames.size());
609         metaDataSizes = new ArrayList<Integer>(metaNames.size());
610         for (int i = 0 ; i < metaNames.size() ; ++ i ) {
611           // store the beginning offset
612           long curPos = outputStream.getPos();
613           metaOffsets.add(curPos);
614           // write the metadata content
615           DataOutputStream dos = getCompressingStream();
616           dos.write(METABLOCKMAGIC);
617           metaData.get(i).write(dos);
618           int size = releaseCompressingStream(dos);
619           // store the metadata size
620           metaDataSizes.add(size);
621         }
622       }
623 
624       // Write fileinfo.
625       trailer.fileinfoOffset = writeFileInfo(this.outputStream);
626 
627       // Write the data block index.
628       trailer.dataIndexOffset = BlockIndex.writeIndex(this.outputStream,
629         this.blockKeys, this.blockOffsets, this.blockDataSizes);
630 
631       // Meta block index.
632       if (metaNames.size() > 0) {
633         trailer.metaIndexOffset = BlockIndex.writeIndex(this.outputStream,
634           this.metaNames, metaOffsets, metaDataSizes);
635       }
636 
637       // Now finish off the trailer.
638       trailer.dataIndexCount = blockKeys.size();
639       trailer.metaIndexCount = metaNames.size();
640 
641       trailer.totalUncompressedBytes = totalBytes;
642       trailer.entryCount = entryCount;
643 
644       trailer.compressionCodec = this.compressAlgo.ordinal();
645 
646       trailer.serialize(outputStream);
647 
648       if (this.closeOutputStream) {
649         this.outputStream.close();
650         this.outputStream = null;
651       }
652     }
653 
654     /*
655      * Add last bits of metadata to fileinfo and then write it out.
656      * Reader will be expecting to find all below.
657      * @param o Stream to write on.
658      * @return Position at which we started writing.
659      * @throws IOException
660      */
661     private long writeFileInfo(FSDataOutputStream o) throws IOException {
662       if (this.lastKeyBuffer != null) {
663         // Make a copy.  The copy is stuffed into HMapWritable.  Needs a clean
664         // byte buffer.  Won't take a tuple.
665         byte [] b = new byte[this.lastKeyLength];
666         System.arraycopy(this.lastKeyBuffer, this.lastKeyOffset, b, 0,
667           this.lastKeyLength);
668         appendFileInfo(this.fileinfo, FileInfo.LASTKEY, b, false);
669       }
670       int avgKeyLen = this.entryCount == 0? 0:
671         (int)(this.keylength/this.entryCount);
672       appendFileInfo(this.fileinfo, FileInfo.AVG_KEY_LEN,
673         Bytes.toBytes(avgKeyLen), false);
674       int avgValueLen = this.entryCount == 0? 0:
675         (int)(this.valuelength/this.entryCount);
676       appendFileInfo(this.fileinfo, FileInfo.AVG_VALUE_LEN,
677         Bytes.toBytes(avgValueLen), false);
678       appendFileInfo(this.fileinfo, FileInfo.COMPARATOR,
679         Bytes.toBytes(this.rawComparator.getClass().getName()), false);
680       long pos = o.getPos();
681       this.fileinfo.write(o);
682       return pos;
683     }
684   }
685 
686   /**
687    * HFile Reader.
688    */
689   public static class Reader implements Closeable {
690     // Stream to read from.
691     private FSDataInputStream istream;
692     // True if we should close istream when done.  We don't close it if we
693     // didn't open it.
694     private boolean closeIStream;
695 
696     // These are read in when the file info is loaded.
697     HFile.BlockIndex blockIndex;
698     private BlockIndex metaIndex;
699     FixedFileTrailer trailer;
700     private volatile boolean fileInfoLoaded = false;
701 
702     // Filled when we read in the trailer.
703     private Compression.Algorithm compressAlgo;
704 
705     // Last key in the file.  Filled in when we read in the file info
706     private byte [] lastkey = null;
707     // Stats read in when we load file info.
708     private int avgKeyLen = -1;
709     private int avgValueLen = -1;
710 
711     // Used to ensure we seek correctly.
712     RawComparator<byte []> comparator;
713 
714     // Size of this file.
715     private final long fileSize;
716 
717     // Block cache to use.
718     private final BlockCache cache;
719     public int cacheHits = 0;
720     public int blockLoads = 0;
721     public int metaLoads = 0;
722 
723     // Whether file is from in-memory store
724     private boolean inMemory = false;
725 
726     // Name for this object used when logging or in toString.  Is either
727     // the result of a toString on the stream or else is toString of passed
728     // file Path plus metadata key/value pairs.
729     protected String name;
730 
731     /**
732      * Opens a HFile.  You must load the file info before you can
733      * use it by calling {@link #loadFileInfo()}.
734      *
735      * @param fs filesystem to load from
736      * @param path path within said filesystem
737      * @param cache block cache. Pass null if none.
738      * @throws IOException
739      */
740     public Reader(FileSystem fs, Path path, BlockCache cache, boolean inMemory)
741     throws IOException {
742       this(fs.open(path), fs.getFileStatus(path).getLen(), cache, inMemory);
743       this.closeIStream = true;
744       this.name = path.toString();
745     }
746 
747     /**
748      * Opens a HFile.  You must load the index before you can
749      * use it by calling {@link #loadFileInfo()}.
750      *
751      * @param fsdis input stream.  Caller is responsible for closing the passed
752      * stream.
753      * @param size Length of the stream.
754      * @param cache block cache. Pass null if none.
755      * @throws IOException
756      */
757     public Reader(final FSDataInputStream fsdis, final long size,
758         final BlockCache cache, final boolean inMemory) {
759       this.cache = cache;
760       this.fileSize = size;
761       this.istream = fsdis;
762       this.closeIStream = false;
763       this.name = this.istream == null? "": this.istream.toString();
764       this.inMemory = inMemory;
765     }
766 
767     @Override
768     public String toString() {
769       return "reader=" + this.name +
770           (!isFileInfoLoaded()? "":
771             ", compression=" + this.compressAlgo.getName() +
772             ", inMemory=" + this.inMemory +
773             ", firstKey=" + toStringFirstKey() +
774             ", lastKey=" + toStringLastKey()) +
775             ", avgKeyLen=" + this.avgKeyLen +
776             ", avgValueLen=" + this.avgValueLen +
777             ", entries=" + this.trailer.entryCount +
778             ", length=" + this.fileSize;
779     }
780 
781     protected String toStringFirstKey() {
782       return KeyValue.keyToString(getFirstKey());
783     }
784 
785     protected String toStringLastKey() {
786       return KeyValue.keyToString(getLastKey());
787     }
788 
789     public long length() {
790       return this.fileSize;
791     }
792 
793     public boolean inMemory() {
794       return this.inMemory;
795     }
796 
797     /**
798      * Read in the index and file info.
799      * @return A map of fileinfo data.
800      * See {@link Writer#appendFileInfo(byte[], byte[])}.
801      * @throws IOException
802      */
803     public Map<byte [], byte []> loadFileInfo()
804     throws IOException {
805       this.trailer = readTrailer();
806 
807       // Read in the fileinfo and get what we need from it.
808       this.istream.seek(this.trailer.fileinfoOffset);
809       FileInfo fi = new FileInfo();
810       fi.readFields(this.istream);
811       this.lastkey = fi.get(FileInfo.LASTKEY);
812       this.avgKeyLen = Bytes.toInt(fi.get(FileInfo.AVG_KEY_LEN));
813       this.avgValueLen = Bytes.toInt(fi.get(FileInfo.AVG_VALUE_LEN));
814       String clazzName = Bytes.toString(fi.get(FileInfo.COMPARATOR));
815       this.comparator = getComparator(clazzName);
816 
817       // Read in the data index.
818       this.blockIndex = BlockIndex.readIndex(this.comparator, this.istream,
819         this.trailer.dataIndexOffset, this.trailer.dataIndexCount);
820 
821       // Read in the metadata index.
822       if (trailer.metaIndexCount > 0) {
823         this.metaIndex = BlockIndex.readIndex(Bytes.BYTES_RAWCOMPARATOR,
824           this.istream, this.trailer.metaIndexOffset, trailer.metaIndexCount);
825       }
826       this.fileInfoLoaded = true;
827       return fi;
828     }
829 
830     boolean isFileInfoLoaded() {
831       return this.fileInfoLoaded;
832     }
833 
834     @SuppressWarnings("unchecked")
835     private RawComparator<byte []> getComparator(final String clazzName)
836     throws IOException {
837       if (clazzName == null || clazzName.length() == 0) {
838         return null;
839       }
840       try {
841         return (RawComparator<byte []>)Class.forName(clazzName).newInstance();
842       } catch (InstantiationException e) {
843         throw new IOException(e);
844       } catch (IllegalAccessException e) {
845         throw new IOException(e);
846       } catch (ClassNotFoundException e) {
847         throw new IOException(e);
848       }
849     }
850 
851     /* Read the trailer off the input stream.  As side effect, sets the
852      * compression algorithm.
853      * @return Populated FixedFileTrailer.
854      * @throws IOException
855      */
856     private FixedFileTrailer readTrailer() throws IOException {
857       FixedFileTrailer fft = new FixedFileTrailer();
858       long seekPoint = this.fileSize - FixedFileTrailer.trailerSize();
859       this.istream.seek(seekPoint);
860       fft.deserialize(this.istream);
861       // Set up the codec.
862       this.compressAlgo =
863         Compression.Algorithm.values()[fft.compressionCodec];
864       return fft;
865     }
866 
867     /**
868      * Create a Scanner on this file.  No seeks or reads are done on creation.
869      * Call {@link HFileScanner#seekTo(byte[])} to position an start the read.
870      * There is nothing to clean up in a Scanner. Letting go of your references
871      * to the scanner is sufficient.
872      * @param pread Use positional read rather than seek+read if true (pread is
873      * better for random reads, seek+read is better scanning).
874      * @param cacheBlocks True if we should cache blocks read in by this scanner.
875      * @return Scanner on this file.
876      */
877     public HFileScanner getScanner(boolean cacheBlocks, final boolean pread) {
878       return new Scanner(this, cacheBlocks, pread);
879     }
880 
881     /**
882      * @param key Key to search.
883      * @return Block number of the block containing the key or -1 if not in this
884      * file.
885      */
886     protected int blockContainingKey(final byte [] key, int offset, int length) {
887       if (blockIndex == null) {
888         throw new RuntimeException("Block index not loaded");
889       }
890       return blockIndex.blockContainingKey(key, offset, length);
891     }
892     /**
893      * @param metaBlockName
894      * @param cacheBlock Add block to cache, if found
895      * @return Block wrapped in a ByteBuffer
896      * @throws IOException
897      */
898     public ByteBuffer getMetaBlock(String metaBlockName, boolean cacheBlock)
899     throws IOException {
900       if (trailer.metaIndexCount == 0) {
901         return null; // there are no meta blocks
902       }
903       if (metaIndex == null) {
904         throw new IOException("Meta index not loaded");
905       }
906 
907       byte [] mbname = Bytes.toBytes(metaBlockName);
908       int block = metaIndex.blockContainingKey(mbname, 0, mbname.length);
909       if (block == -1)
910         return null;
911       long blockSize;
912       if (block == metaIndex.count - 1) {
913         blockSize = trailer.fileinfoOffset - metaIndex.blockOffsets[block];
914       } else {
915         blockSize = metaIndex.blockOffsets[block+1] - metaIndex.blockOffsets[block];
916       }
917 
918       long now = System.currentTimeMillis();
919 
920       // Per meta key from any given file, synchronize reads for said block
921       synchronized (metaIndex.blockKeys[block]) {
922         metaLoads++;
923         // Check cache for block.  If found return.
924         if (cache != null) {
925           ByteBuffer cachedBuf = cache.getBlock(name + "meta" + block);
926           if (cachedBuf != null) {
927             // Return a distinct 'shallow copy' of the block,
928             // so pos doesnt get messed by the scanner
929             cacheHits++;
930             return cachedBuf.duplicate();
931           }
932           // Cache Miss, please load.
933         }
934 
935         ByteBuffer buf = decompress(metaIndex.blockOffsets[block],
936           longToInt(blockSize), metaIndex.blockDataSizes[block], true);
937         byte [] magic = new byte[METABLOCKMAGIC.length];
938         buf.get(magic, 0, magic.length);
939 
940         if (! Arrays.equals(magic, METABLOCKMAGIC)) {
941           throw new IOException("Meta magic is bad in block " + block);
942         }
943 
944         // Create a new ByteBuffer 'shallow copy' to hide the magic header
945         buf = buf.slice();
946 
947         readTime += System.currentTimeMillis() - now;
948         readOps++;
949 
950         // Cache the block
951         if(cacheBlock && cache != null) {
952           cache.cacheBlock(name + "meta" + block, buf.duplicate(), inMemory);
953         }
954 
955         return buf;
956       }
957     }
958 
959     /**
960      * Read in a file block.
961      * @param block Index of block to read.
962      * @param pread Use positional read instead of seek+read (positional is
963      * better doing random reads whereas seek+read is better scanning).
964      * @return Block wrapped in a ByteBuffer.
965      * @throws IOException
966      */
967     ByteBuffer readBlock(int block, boolean cacheBlock, final boolean pread)
968     throws IOException {
969       if (blockIndex == null) {
970         throw new IOException("Block index not loaded");
971       }
972       if (block < 0 || block >= blockIndex.count) {
973         throw new IOException("Requested block is out of range: " + block +
974           ", max: " + blockIndex.count);
975       }
976       // For any given block from any given file, synchronize reads for said
977       // block.
978       // Without a cache, this synchronizing is needless overhead, but really
979       // the other choice is to duplicate work (which the cache would prevent you from doing).
980       synchronized (blockIndex.blockKeys[block]) {
981         blockLoads++;
982         // Check cache for block.  If found return.
983         if (cache != null) {
984           ByteBuffer cachedBuf = cache.getBlock(name + block);
985           if (cachedBuf != null) {
986             // Return a distinct 'shallow copy' of the block,
987             // so pos doesnt get messed by the scanner
988             cacheHits++;
989             return cachedBuf.duplicate();
990           }
991           // Carry on, please load.
992         }
993 
994         // Load block from filesystem.
995         long now = System.currentTimeMillis();
996         long onDiskBlockSize;
997         if (block == blockIndex.count - 1) {
998           // last block!  The end of data block is first meta block if there is
999           // one or if there isn't, the fileinfo offset.
1000           long offset = this.metaIndex != null?
1001             this.metaIndex.blockOffsets[0]: this.trailer.fileinfoOffset;
1002           onDiskBlockSize = offset - blockIndex.blockOffsets[block];
1003         } else {
1004           onDiskBlockSize = blockIndex.blockOffsets[block+1] -
1005           blockIndex.blockOffsets[block];
1006         }
1007         ByteBuffer buf = decompress(blockIndex.blockOffsets[block],
1008           longToInt(onDiskBlockSize), this.blockIndex.blockDataSizes[block],
1009           pread);
1010 
1011         byte [] magic = new byte[DATABLOCKMAGIC.length];
1012         buf.get(magic, 0, magic.length);
1013         if (!Arrays.equals(magic, DATABLOCKMAGIC)) {
1014           throw new IOException("Data magic is bad in block " + block);
1015         }
1016 
1017         // 'shallow copy' to hide the header
1018         // NOTE: you WILL GET BIT if you call buf.array() but don't start
1019         //       reading at buf.arrayOffset()
1020         buf = buf.slice();
1021 
1022         readTime += System.currentTimeMillis() - now;
1023         readOps++;
1024 
1025         // Cache the block
1026         if(cacheBlock && cache != null) {
1027           cache.cacheBlock(name + block, buf.duplicate(), inMemory);
1028         }
1029 
1030         return buf;
1031       }
1032     }
1033 
1034     /*
1035      * Decompress <code>compressedSize</code> bytes off the backing
1036      * FSDataInputStream.
1037      * @param offset
1038      * @param compressedSize
1039      * @param decompressedSize
1040      *
1041      * @return
1042      * @throws IOException
1043      */
1044     private ByteBuffer decompress(final long offset, final int compressedSize,
1045       final int decompressedSize, final boolean pread)
1046     throws IOException {
1047       Decompressor decompressor = null;
1048       ByteBuffer buf = null;
1049       try {
1050         decompressor = this.compressAlgo.getDecompressor();
1051         // My guess is that the bounded range fis is needed to stop the
1052         // decompressor reading into next block -- IIRC, it just grabs a
1053         // bunch of data w/o regard to whether decompressor is coming to end of a
1054         // decompression.
1055         InputStream is = this.compressAlgo.createDecompressionStream(
1056           new BoundedRangeFileInputStream(this.istream, offset, compressedSize,
1057             pread),
1058           decompressor, 0);
1059         buf = ByteBuffer.allocate(decompressedSize);
1060         IOUtils.readFully(is, buf.array(), 0, buf.capacity());
1061         is.close();
1062       } finally {
1063         if (null != decompressor) {
1064           this.compressAlgo.returnDecompressor(decompressor);
1065         }
1066       }
1067       return buf;
1068     }
1069 
1070     /**
1071      * @return First key in the file.  May be null if file has no entries.
1072      * Note that this is not the first rowkey, but rather the byte form of
1073      * the first KeyValue.
1074      */
1075     public byte [] getFirstKey() {
1076       if (blockIndex == null) {
1077         throw new RuntimeException("Block index not loaded");
1078       }
1079       return this.blockIndex.isEmpty()? null: this.blockIndex.blockKeys[0];
1080     }
1081 
1082     /**
1083      * @return the first row key, or null if the file is empty.
1084      * TODO move this to StoreFile after Ryan's patch goes in
1085      * to eliminate KeyValue here
1086      */
1087     public byte[] getFirstRowKey() {
1088       byte[] firstKey = getFirstKey();
1089       if (firstKey == null) return null;
1090       return KeyValue.createKeyValueFromKey(firstKey).getRow();
1091     }
1092 
1093     /**
1094      * @return number of KV entries in this HFile
1095      */
1096     public int getEntries() {
1097       if (!this.isFileInfoLoaded()) {
1098         throw new RuntimeException("File info not loaded");
1099       }
1100       return this.trailer.entryCount;
1101     }
1102 
1103     /**
1104      * @return Last key in the file.  May be null if file has no entries.
1105      * Note that this is not the last rowkey, but rather the byte form of
1106      * the last KeyValue.
1107      */
1108     public byte [] getLastKey() {
1109       if (!isFileInfoLoaded()) {
1110         throw new RuntimeException("Load file info first");
1111       }
1112       return this.blockIndex.isEmpty()? null: this.lastkey;
1113     }
1114 
1115     /**
1116      * @return the last row key, or null if the file is empty.
1117      * TODO move this to StoreFile after Ryan's patch goes in
1118      * to eliminate KeyValue here
1119      */
1120     public byte[] getLastRowKey() {
1121       byte[] lastKey = getLastKey();
1122       if (lastKey == null) return null;
1123       return KeyValue.createKeyValueFromKey(lastKey).getRow();
1124     }
1125 
1126     /**
1127      * @return number of K entries in this HFile's filter.  Returns KV count if no filter.
1128      */
1129     public int getFilterEntries() {
1130       return getEntries();
1131     }
1132 
1133     /**
1134      * @return Comparator.
1135      */
1136     public RawComparator<byte []> getComparator() {
1137       return this.comparator;
1138     }
1139 
1140     /**
1141      * @return index size
1142      */
1143     public long indexSize() {
1144       return (this.blockIndex != null? this.blockIndex.heapSize(): 0) +
1145         ((this.metaIndex != null)? this.metaIndex.heapSize(): 0);
1146     }
1147 
1148     /**
1149      * @return Midkey for this file.  We work with block boundaries only so
1150      * returned midkey is an approximation only.
1151      * @throws IOException
1152      */
1153     public byte [] midkey() throws IOException {
1154       if (!isFileInfoLoaded() || this.blockIndex.isEmpty()) {
1155         return null;
1156       }
1157       return this.blockIndex.midkey();
1158     }
1159 
1160     public void close() throws IOException {
1161       if (this.closeIStream && this.istream != null) {
1162         this.istream.close();
1163         this.istream = null;
1164       }
1165     }
1166 
1167     public String getName() {
1168       return name;
1169     }
1170 
1171     /*
1172      * Implementation of {@link HFileScanner} interface.
1173      */
1174     protected static class Scanner implements HFileScanner {
1175       private final Reader reader;
1176       private ByteBuffer block;
1177       private int currBlock;
1178 
1179       private final boolean cacheBlocks;
1180       private final boolean pread;
1181 
1182       private int currKeyLen = 0;
1183       private int currValueLen = 0;
1184 
1185       public int blockFetches = 0;
1186 
1187       public Scanner(Reader r, boolean cacheBlocks, final boolean pread) {
1188         this.reader = r;
1189         this.cacheBlocks = cacheBlocks;
1190         this.pread = pread;
1191       }
1192 
1193       public KeyValue getKeyValue() {
1194         if(this.block == null) {
1195           return null;
1196         }
1197         return new KeyValue(this.block.array(),
1198             this.block.arrayOffset() + this.block.position() - 8);
1199       }
1200 
1201       public ByteBuffer getKey() {
1202         if (this.block == null || this.currKeyLen == 0) {
1203           throw new RuntimeException("you need to seekTo() before calling getKey()");
1204         }
1205         ByteBuffer keyBuff = this.block.slice();
1206         keyBuff.limit(this.currKeyLen);
1207         keyBuff.rewind();
1208         // Do keyBuff.asReadOnly()?
1209         return keyBuff;
1210       }
1211 
1212       public ByteBuffer getValue() {
1213         if (block == null || currKeyLen == 0) {
1214           throw new RuntimeException("you need to seekTo() before calling getValue()");
1215         }
1216         // TODO: Could this be done with one ByteBuffer rather than create two?
1217         ByteBuffer valueBuff = this.block.slice();
1218         valueBuff.position(this.currKeyLen);
1219         valueBuff = valueBuff.slice();
1220         valueBuff.limit(currValueLen);
1221         valueBuff.rewind();
1222         return valueBuff;
1223       }
1224 
1225       public boolean next() throws IOException {
1226         // LOG.deug("rem:" + block.remaining() + " p:" + block.position() +
1227         // " kl: " + currKeyLen + " kv: " + currValueLen);
1228         if (block == null) {
1229           throw new IOException("Next called on non-seeked scanner");
1230         }
1231         block.position(block.position() + currKeyLen + currValueLen);
1232         if (block.remaining() <= 0) {
1233           // LOG.debug("Fetch next block");
1234           currBlock++;
1235           if (currBlock >= reader.blockIndex.count) {
1236             // damn we are at the end
1237             currBlock = 0;
1238             block = null;
1239             return false;
1240           }
1241           block = reader.readBlock(this.currBlock, this.cacheBlocks, this.pread);
1242           currKeyLen = block.getInt();
1243           currValueLen = block.getInt();
1244           blockFetches++;
1245           return true;
1246         }
1247         // LOG.debug("rem:" + block.remaining() + " p:" + block.position() +
1248         // " kl: " + currKeyLen + " kv: " + currValueLen);
1249 
1250         currKeyLen = block.getInt();
1251         currValueLen = block.getInt();
1252         return true;
1253       }
1254 
1255       public int seekTo(byte [] key) throws IOException {
1256         return seekTo(key, 0, key.length);
1257       }
1258 
1259 
1260       public int seekTo(byte[] key, int offset, int length) throws IOException {
1261         int b = reader.blockContainingKey(key, offset, length);
1262         if (b < 0) return -1; // falls before the beginning of the file! :-(
1263         // Avoid re-reading the same block (that'd be dumb).
1264         loadBlock(b);
1265 
1266         return blockSeek(key, offset, length, false);
1267       }
1268 
1269       /**
1270        * Within a loaded block, seek looking for the first key
1271        * that is smaller than (or equal to?) the key we are interested in.
1272        *
1273        * A note on the seekBefore - if you have seekBefore = true, AND the
1274        * first key in the block = key, then you'll get thrown exceptions.
1275        * @param key to find
1276        * @param seekBefore find the key before the exact match.
1277        * @return
1278        */
1279       private int blockSeek(byte[] key, int offset, int length, boolean seekBefore) {
1280         int klen, vlen;
1281         int lastLen = 0;
1282         do {
1283           klen = block.getInt();
1284           vlen = block.getInt();
1285           int comp = this.reader.comparator.compare(key, offset, length,
1286             block.array(), block.arrayOffset() + block.position(), klen);
1287           if (comp == 0) {
1288             if (seekBefore) {
1289               block.position(block.position() - lastLen - 16);
1290               currKeyLen = block.getInt();
1291               currValueLen = block.getInt();
1292               return 1; // non exact match.
1293             }
1294             currKeyLen = klen;
1295             currValueLen = vlen;
1296             return 0; // indicate exact match
1297           }
1298           if (comp < 0) {
1299             // go back one key:
1300             block.position(block.position() - lastLen - 16);
1301             currKeyLen = block.getInt();
1302             currValueLen = block.getInt();
1303             return 1;
1304           }
1305           block.position(block.position() + klen + vlen);
1306           lastLen = klen + vlen ;
1307         } while(block.remaining() > 0);
1308         // ok we are at the end, so go back a littleeeeee....
1309         // The 8 in the below is intentionally different to the 16s in the above
1310         // Do the math you you'll figure it.
1311         block.position(block.position() - lastLen - 8);
1312         currKeyLen = block.getInt();
1313         currValueLen = block.getInt();
1314         return 1; // didn't exactly find it.
1315       }
1316 
1317       public boolean seekBefore(byte [] key) throws IOException {
1318         return seekBefore(key, 0, key.length);
1319       }
1320 
1321       public boolean seekBefore(byte[] key, int offset, int length)
1322       throws IOException {
1323         int b = reader.blockContainingKey(key, offset, length);
1324         if (b < 0)
1325           return false; // key is before the start of the file.
1326 
1327         // Question: does this block begin with 'key'?
1328         if (this.reader.comparator.compare(reader.blockIndex.blockKeys[b],
1329             0, reader.blockIndex.blockKeys[b].length,
1330             key, offset, length) == 0) {
1331           // Ok the key we're interested in is the first of the block, so go back one.
1332           if (b == 0) {
1333             // we have a 'problem', the key we want is the first of the file.
1334             return false;
1335           }
1336           b--;
1337           // TODO shortcut: seek forward in this block to the last key of the block.
1338         }
1339         loadBlock(b);
1340         blockSeek(key, offset, length, true);
1341         return true;
1342       }
1343 
1344       public String getKeyString() {
1345         return Bytes.toStringBinary(block.array(), block.arrayOffset() +
1346           block.position(), currKeyLen);
1347       }
1348 
1349       public String getValueString() {
1350         return Bytes.toString(block.array(), block.arrayOffset() +
1351           block.position() + currKeyLen, currValueLen);
1352       }
1353 
1354       public Reader getReader() {
1355         return this.reader;
1356       }
1357 
1358       public boolean isSeeked(){
1359         return this.block != null;
1360       }
1361 
1362       public boolean seekTo() throws IOException {
1363         if (this.reader.blockIndex.isEmpty()) {
1364           return false;
1365         }
1366         if (block != null && currBlock == 0) {
1367           block.rewind();
1368           currKeyLen = block.getInt();
1369           currValueLen = block.getInt();
1370           return true;
1371         }
1372         currBlock = 0;
1373         block = reader.readBlock(this.currBlock, this.cacheBlocks, this.pread);
1374         currKeyLen = block.getInt();
1375         currValueLen = block.getInt();
1376         blockFetches++;
1377         return true;
1378       }
1379 
1380       private void loadBlock(int bloc) throws IOException {
1381         if (block == null) {
1382           block = reader.readBlock(bloc, this.cacheBlocks, this.pread);
1383           currBlock = bloc;
1384           blockFetches++;
1385         } else {
1386           if (bloc != currBlock) {
1387             block = reader.readBlock(bloc, this.cacheBlocks, this.pread);
1388             currBlock = bloc;
1389             blockFetches++;
1390           } else {
1391             // we are already in the same block, just rewind to seek again.
1392             block.rewind();
1393           }
1394         }
1395       }
1396 
1397       @Override
1398       public String toString() {
1399         return "HFileScanner for reader " + String.valueOf(reader);
1400       }
1401     }
1402 
1403     public String getTrailerInfo() {
1404       return trailer.toString();
1405     }
1406   }
1407 
1408   /*
1409    * The RFile has a fixed trailer which contains offsets to other variable
1410    * parts of the file.  Also includes basic metadata on this file.
1411    */
1412   private static class FixedFileTrailer {
1413     // Offset to the fileinfo data, a small block of vitals..
1414     long fileinfoOffset;
1415     // Offset to the data block index.
1416     long dataIndexOffset;
1417     // How many index counts are there (aka: block count)
1418     int dataIndexCount;
1419     // Offset to the meta block index.
1420     long metaIndexOffset;
1421     // How many meta block index entries (aka: meta block count)
1422     int metaIndexCount;
1423     long totalUncompressedBytes;
1424     int entryCount;
1425     int compressionCodec;
1426     int version = 1;
1427 
1428     FixedFileTrailer() {
1429       super();
1430     }
1431 
1432     static int trailerSize() {
1433       // Keep this up to date...
1434       return
1435       ( Bytes.SIZEOF_INT * 5 ) +
1436       ( Bytes.SIZEOF_LONG * 4 ) +
1437       TRAILERBLOCKMAGIC.length;
1438     }
1439 
1440     void serialize(DataOutputStream outputStream) throws IOException {
1441       outputStream.write(TRAILERBLOCKMAGIC);
1442       outputStream.writeLong(fileinfoOffset);
1443       outputStream.writeLong(dataIndexOffset);
1444       outputStream.writeInt(dataIndexCount);
1445       outputStream.writeLong(metaIndexOffset);
1446       outputStream.writeInt(metaIndexCount);
1447       outputStream.writeLong(totalUncompressedBytes);
1448       outputStream.writeInt(entryCount);
1449       outputStream.writeInt(compressionCodec);
1450       outputStream.writeInt(version);
1451     }
1452 
1453     void deserialize(DataInputStream inputStream) throws IOException {
1454       byte [] header = new byte[TRAILERBLOCKMAGIC.length];
1455       inputStream.readFully(header);
1456       if ( !Arrays.equals(header, TRAILERBLOCKMAGIC)) {
1457         throw new IOException("Trailer 'header' is wrong; does the trailer " +
1458           "size match content?");
1459       }
1460       fileinfoOffset         = inputStream.readLong();
1461       dataIndexOffset        = inputStream.readLong();
1462       dataIndexCount         = inputStream.readInt();
1463 
1464       metaIndexOffset        = inputStream.readLong();
1465       metaIndexCount         = inputStream.readInt();
1466 
1467       totalUncompressedBytes = inputStream.readLong();
1468       entryCount             = inputStream.readInt();
1469       compressionCodec       = inputStream.readInt();
1470       version                = inputStream.readInt();
1471 
1472       if (version != 1) {
1473         throw new IOException("Wrong version: " + version);
1474       }
1475     }
1476 
1477     @Override
1478     public String toString() {
1479       return "fileinfoOffset=" + fileinfoOffset +
1480       ", dataIndexOffset=" + dataIndexOffset +
1481       ", dataIndexCount=" + dataIndexCount +
1482       ", metaIndexOffset=" + metaIndexOffset +
1483       ", metaIndexCount=" + metaIndexCount +
1484       ", totalBytes=" + totalUncompressedBytes +
1485       ", entryCount=" + entryCount +
1486       ", version=" + version;
1487     }
1488   }
1489 
1490   /*
1491    * The block index for a RFile.
1492    * Used reading.
1493    */
1494   static class BlockIndex implements HeapSize {
1495     // How many actual items are there? The next insert location too.
1496     int count = 0;
1497     byte [][] blockKeys;
1498     long [] blockOffsets;
1499     int [] blockDataSizes;
1500     int size = 0;
1501 
1502     /* Needed doing lookup on blocks.
1503      */
1504     final RawComparator<byte []> comparator;
1505 
1506     /*
1507      * Shutdown default constructor
1508      */
1509     @SuppressWarnings("unused")
1510     private BlockIndex() {
1511       this(null);
1512     }
1513 
1514 
1515     /**
1516      * @param c comparator used to compare keys.
1517      */
1518     BlockIndex(final RawComparator<byte []>c) {
1519       this.comparator = c;
1520       // Guess that cost of three arrays + this object is 4 * 8 bytes.
1521       this.size += (4 * 8);
1522     }
1523 
1524     /**
1525      * @return True if block index is empty.
1526      */
1527     boolean isEmpty() {
1528       return this.blockKeys.length <= 0;
1529     }
1530 
1531     /**
1532      * Adds a new entry in the block index.
1533      *
1534      * @param key Last key in the block
1535      * @param offset file offset where the block is stored
1536      * @param dataSize the uncompressed data size
1537      */
1538     void add(final byte[] key, final long offset, final int dataSize) {
1539       blockOffsets[count] = offset;
1540       blockKeys[count] = key;
1541       blockDataSizes[count] = dataSize;
1542       count++;
1543       this.size += (Bytes.SIZEOF_INT * 2 + key.length);
1544     }
1545 
1546     /**
1547      * @param key Key to find
1548      * @return Offset of block containing <code>key</code> or -1 if this file
1549      * does not contain the request.
1550      */
1551     int blockContainingKey(final byte[] key, int offset, int length) {
1552       int pos = Bytes.binarySearch(blockKeys, key, offset, length, this.comparator);
1553       if (pos < 0) {
1554         pos ++;
1555         pos *= -1;
1556         if (pos == 0) {
1557           // falls before the beginning of the file.
1558           return -1;
1559         }
1560         // When switched to "first key in block" index, binarySearch now returns
1561         // the block with a firstKey < key.  This means the value we want is potentially
1562         // in the next block.
1563         pos --; // in previous block.
1564 
1565         return pos;
1566       }
1567       // wow, a perfect hit, how unlikely?
1568       return pos;
1569     }
1570 
1571     /*
1572      * @return File midkey.  Inexact.  Operates on block boundaries.  Does
1573      * not go into blocks.
1574      */
1575     byte [] midkey() throws IOException {
1576       int pos = ((this.count - 1)/2);              // middle of the index
1577       if (pos < 0) {
1578         throw new IOException("HFile empty");
1579       }
1580       return this.blockKeys[pos];
1581     }
1582 
1583     /*
1584      * Write out index. Whatever we write here must jibe with what
1585      * BlockIndex#readIndex is expecting.  Make sure the two ends of the
1586      * index serialization match.
1587      * @param o
1588      * @param keys
1589      * @param offsets
1590      * @param sizes
1591      * @param c
1592      * @return Position at which we entered the index.
1593      * @throws IOException
1594      */
1595     static long writeIndex(final FSDataOutputStream o,
1596       final List<byte []> keys, final List<Long> offsets,
1597       final List<Integer> sizes)
1598     throws IOException {
1599       long pos = o.getPos();
1600       // Don't write an index if nothing in the index.
1601       if (keys.size() > 0) {
1602         o.write(INDEXBLOCKMAGIC);
1603         // Write the index.
1604         for (int i = 0; i < keys.size(); ++i) {
1605           o.writeLong(offsets.get(i).longValue());
1606           o.writeInt(sizes.get(i).intValue());
1607           byte [] key = keys.get(i);
1608           Bytes.writeByteArray(o, key);
1609         }
1610       }
1611       return pos;
1612     }
1613 
1614     /*
1615      * Read in the index that is at <code>indexOffset</code>
1616      * Must match what was written by writeIndex in the Writer.close.
1617      * @param in
1618      * @param indexOffset
1619      * @throws IOException
1620      */
1621     static BlockIndex readIndex(final RawComparator<byte []> c,
1622         final FSDataInputStream in, final long indexOffset, final int indexSize)
1623     throws IOException {
1624       BlockIndex bi = new BlockIndex(c);
1625       bi.blockOffsets = new long[indexSize];
1626       bi.blockKeys = new byte[indexSize][];
1627       bi.blockDataSizes = new int[indexSize];
1628       // If index size is zero, no index was written.
1629       if (indexSize > 0) {
1630         in.seek(indexOffset);
1631         byte [] magic = new byte[INDEXBLOCKMAGIC.length];
1632         IOUtils.readFully(in, magic, 0, magic.length);
1633         if (!Arrays.equals(magic, INDEXBLOCKMAGIC)) {
1634           throw new IOException("Index block magic is wrong: " +
1635             Arrays.toString(magic));
1636         }
1637         for (int i = 0; i < indexSize; ++i ) {
1638           long offset   = in.readLong();
1639           int dataSize  = in.readInt();
1640           byte [] key = Bytes.readByteArray(in);
1641           bi.add(key, offset, dataSize);
1642         }
1643       }
1644       return bi;
1645     }
1646 
1647     @Override
1648     public String toString() {
1649       StringBuilder sb = new StringBuilder();
1650       sb.append("size=" + count);
1651       for (int i = 0; i < count ; i++) {
1652         sb.append(", ");
1653         sb.append("key=").append(Bytes.toStringBinary(blockKeys[i])).
1654           append(", offset=").append(blockOffsets[i]).
1655           append(", dataSize=" + blockDataSizes[i]);
1656       }
1657       return sb.toString();
1658     }
1659 
1660     public long heapSize() {
1661       long heapsize = ClassSize.align(ClassSize.OBJECT +
1662           2 * Bytes.SIZEOF_INT + (3 + 1) * ClassSize.REFERENCE);
1663       //Calculating the size of blockKeys
1664       if(blockKeys != null) {
1665         //Adding array + references overhead
1666         heapsize += ClassSize.align(ClassSize.ARRAY +
1667             blockKeys.length * ClassSize.REFERENCE);
1668         //Adding bytes
1669         for(byte [] bs : blockKeys) {
1670           heapsize += ClassSize.align(ClassSize.ARRAY + bs.length);
1671         }
1672       }
1673       if(blockOffsets != null) {
1674         heapsize += ClassSize.align(ClassSize.ARRAY +
1675             blockOffsets.length * Bytes.SIZEOF_LONG);
1676       }
1677       if(blockDataSizes != null) {
1678         heapsize += ClassSize.align(ClassSize.ARRAY +
1679             blockDataSizes.length * Bytes.SIZEOF_INT);
1680       }
1681 
1682       return ClassSize.align(heapsize);
1683     }
1684 
1685   }
1686 
1687   /*
1688    * Metadata for this file.  Conjured by the writer.  Read in by the reader.
1689    */
1690   static class FileInfo extends HbaseMapWritable<byte [], byte []> {
1691     static final String RESERVED_PREFIX = "hfile.";
1692     static final byte[] RESERVED_PREFIX_BYTES = Bytes.toBytes(RESERVED_PREFIX);
1693     static final byte [] LASTKEY = Bytes.toBytes(RESERVED_PREFIX + "LASTKEY");
1694     static final byte [] AVG_KEY_LEN =
1695       Bytes.toBytes(RESERVED_PREFIX + "AVG_KEY_LEN");
1696     static final byte [] AVG_VALUE_LEN =
1697       Bytes.toBytes(RESERVED_PREFIX + "AVG_VALUE_LEN");
1698     static final byte [] COMPARATOR =
1699       Bytes.toBytes(RESERVED_PREFIX + "COMPARATOR");
1700 
1701     /*
1702      * Constructor.
1703      */
1704     FileInfo() {
1705       super();
1706     }
1707   }
1708 
1709   /**
1710    * Return true if the given file info key is reserved for internal
1711    * use by HFile.
1712    */
1713   public static boolean isReservedFileInfoKey(byte[] key) {
1714     return Bytes.startsWith(key, FileInfo.RESERVED_PREFIX_BYTES);
1715   }
1716 
1717 
1718   /**
1719    * Get names of supported compression algorithms. The names are acceptable by
1720    * HFile.Writer.
1721    *
1722    * @return Array of strings, each represents a supported compression
1723    *         algorithm. Currently, the following compression algorithms are
1724    *         supported.
1725    *         <ul>
1726    *         <li>"none" - No compression.
1727    *         <li>"gz" - GZIP compression.
1728    *         </ul>
1729    */
1730   public static String[] getSupportedCompressionAlgorithms() {
1731     return Compression.getSupportedAlgorithms();
1732   }
1733 
1734   // Utility methods.
1735   /*
1736    * @param l Long to convert to an int.
1737    * @return <code>l</code> cast as an int.
1738    */
1739   static int longToInt(final long l) {
1740     // Expecting the size() of a block not exceeding 4GB. Assuming the
1741     // size() will wrap to negative integer if it exceeds 2GB (From tfile).
1742     return (int)(l & 0x00000000ffffffffL);
1743   }
1744 
1745   /**
1746    * Returns all files belonging to the given region directory. Could return an
1747    * empty list.
1748    *
1749    * @param fs  The file system reference.
1750    * @param regionDir  The region directory to scan.
1751    * @return The list of files found.
1752    * @throws IOException When scanning the files fails.
1753    */
1754   static List<Path> getStoreFiles(FileSystem fs, Path regionDir)
1755   throws IOException {
1756     List<Path> res = new ArrayList<Path>();
1757     PathFilter dirFilter = new FSUtils.DirFilter(fs);
1758     FileStatus[] familyDirs = fs.listStatus(regionDir, dirFilter);
1759     for(FileStatus dir : familyDirs) {
1760       FileStatus[] files = fs.listStatus(dir.getPath());
1761       for (FileStatus file : files) {
1762         if (!file.isDir()) {
1763           res.add(file.getPath());
1764         }
1765       }
1766     }
1767     return res;
1768   }
1769 
1770   public static void main(String []args) throws IOException {
1771     try {
1772       // create options
1773       Options options = new Options();
1774       options.addOption("v", "verbose", false, "Verbose output; emits file and meta data delimiters");
1775       options.addOption("p", "printkv", false, "Print key/value pairs");
1776       options.addOption("m", "printmeta", false, "Print meta data of file");
1777       options.addOption("k", "checkrow", false,
1778         "Enable row order check; looks for out-of-order keys");
1779       options.addOption("a", "checkfamily", false, "Enable family check");
1780       options.addOption("f", "file", true,
1781         "File to scan. Pass full-path; e.g. hdfs://a:9000/hbase/.META./12/34");
1782       options.addOption("r", "region", true,
1783         "Region to scan. Pass region name; e.g. '.META.,,1'");
1784       if (args.length == 0) {
1785         HelpFormatter formatter = new HelpFormatter();
1786         formatter.printHelp("HFile ", options, true);
1787         System.exit(-1);
1788       }
1789       CommandLineParser parser = new PosixParser();
1790       CommandLine cmd = parser.parse(options, args);
1791       boolean verbose = cmd.hasOption("v");
1792       boolean printKeyValue = cmd.hasOption("p");
1793       boolean printMeta = cmd.hasOption("m");
1794       boolean checkRow = cmd.hasOption("k");
1795       boolean checkFamily = cmd.hasOption("a");
1796       // get configuration, file system and get list of files
1797       Configuration conf = HBaseConfiguration.create();
1798       conf.set("fs.defaultFS",
1799         conf.get(org.apache.hadoop.hbase.HConstants.HBASE_DIR));
1800       FileSystem fs = FileSystem.get(conf);
1801       ArrayList<Path> files = new ArrayList<Path>();
1802       if (cmd.hasOption("f")) {
1803         files.add(new Path(cmd.getOptionValue("f")));
1804       }
1805       if (cmd.hasOption("r")) {
1806         String regionName = cmd.getOptionValue("r");
1807         byte[] rn = Bytes.toBytes(regionName);
1808         byte[][] hri = HRegionInfo.parseRegionName(rn);
1809         Path rootDir = FSUtils.getRootDir(conf);
1810         Path tableDir = new Path(rootDir, Bytes.toString(hri[0]));
1811         String enc = HRegionInfo.encodeRegionName(rn);
1812         Path regionDir = new Path(tableDir, enc);
1813         if (verbose) System.out.println("region dir -> " + regionDir);
1814         List<Path> regionFiles = getStoreFiles(fs, regionDir);
1815         if (verbose) System.out.println("Number of region files found -> " +
1816           regionFiles.size());
1817         if (verbose) {
1818           int i = 1;
1819           for (Path p : regionFiles) {
1820             if (verbose) System.out.println("Found file[" + i++ + "] -> " + p);
1821           }
1822         }
1823         files.addAll(regionFiles);
1824       }
1825       // iterate over all files found
1826       for (Path file : files) {
1827         if (verbose) System.out.println("Scanning -> " + file);
1828         if (!fs.exists(file)) {
1829           System.err.println("ERROR, file doesnt exist: " + file);
1830           continue;
1831         }
1832         // create reader and load file info
1833         HFile.Reader reader = new HFile.Reader(fs, file, null, false);
1834         Map<byte[],byte[]> fileInfo = reader.loadFileInfo();
1835         // scan over file and read key/value's and check if requested
1836         HFileScanner scanner = reader.getScanner(false, false);
1837         scanner.seekTo();
1838         KeyValue pkv = null;
1839         int count = 0;
1840         do {
1841           KeyValue kv = scanner.getKeyValue();
1842           // dump key value
1843           if (printKeyValue) {
1844             System.out.println("K: " + kv +
1845               " V: " + Bytes.toStringBinary(kv.getValue()));
1846           }
1847           // check if rows are in order
1848           if (checkRow && pkv != null) {
1849             if (Bytes.compareTo(pkv.getRow(), kv.getRow()) > 0) {
1850               System.err.println("WARNING, previous row is greater then" +
1851                 " current row\n\tfilename -> " + file +
1852                 "\n\tprevious -> " + Bytes.toStringBinary(pkv.getKey()) +
1853                 "\n\tcurrent  -> " + Bytes.toStringBinary(kv.getKey()));
1854             }
1855           }
1856           // check if families are consistent
1857           if (checkFamily) {
1858             String fam = Bytes.toString(kv.getFamily());
1859             if (!file.toString().contains(fam)) {
1860               System.err.println("WARNING, filename does not match kv family," +
1861                 "\n\tfilename -> " + file +
1862                 "\n\tkeyvalue -> " + Bytes.toStringBinary(kv.getKey()));
1863             }
1864             if (pkv != null && Bytes.compareTo(pkv.getFamily(), kv.getFamily()) != 0) {
1865               System.err.println("WARNING, previous kv has different family" +
1866                 " compared to current key\n\tfilename -> " + file +
1867                 "\n\tprevious -> " +  Bytes.toStringBinary(pkv.getKey()) +
1868                 "\n\tcurrent  -> " + Bytes.toStringBinary(kv.getKey()));
1869             }
1870           }
1871           pkv = kv;
1872           count++;
1873         } while (scanner.next());
1874         if (verbose || printKeyValue) {
1875           System.out.println("Scanned kv count -> " + count);
1876         }
1877         // print meta data
1878         if (printMeta) {
1879           System.out.println("Block index size as per heapsize: " + reader.indexSize());
1880           System.out.println(reader.toString());
1881           System.out.println(reader.getTrailerInfo());
1882           System.out.println("Fileinfo:");
1883           for (Map.Entry<byte[], byte[]> e : fileInfo.entrySet()) {
1884             System.out.print(Bytes.toString(e.getKey()) + " = " );
1885             if (Bytes.compareTo(e.getKey(), Bytes.toBytes("MAX_SEQ_ID_KEY"))==0) {
1886               long seqid = Bytes.toLong(e.getValue());
1887               System.out.println(seqid);
1888             } else {
1889               System.out.println(Bytes.toStringBinary(e.getValue()));
1890             }
1891           }
1892         }
1893         reader.close();
1894       }
1895     } catch (Exception e) {
1896       e.printStackTrace();
1897     }
1898   }
1899 }