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.BufferedInputStream;
23  import java.io.Closeable;
24  import java.io.DataInputStream;
25  import java.io.DataOutputStream;
26  import java.io.IOException;
27  import java.io.InputStream;
28  import java.io.OutputStream;
29  import java.nio.ByteBuffer;
30  import java.util.ArrayList;
31  import java.util.Arrays;
32  import java.util.List;
33  import java.util.Map;
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 
1056         // We use a buffer of DEFAULT_BLOCKSIZE size.  This might be extreme.
1057         // Could maybe do with less. Study and figure it: TODO
1058         InputStream is = this.compressAlgo.createDecompressionStream(
1059             new BufferedInputStream(
1060           new BoundedRangeFileInputStream(this.istream, offset, compressedSize,
1061             pread),
1062                 Math.min(DEFAULT_BLOCKSIZE, compressedSize)),
1063           decompressor, 0);
1064         buf = ByteBuffer.allocate(decompressedSize);
1065         IOUtils.readFully(is, buf.array(), 0, buf.capacity());
1066         is.close();
1067       } finally {
1068         if (null != decompressor) {
1069           this.compressAlgo.returnDecompressor(decompressor);
1070         }
1071       }
1072       return buf;
1073     }
1074 
1075     /**
1076      * @return First key in the file.  May be null if file has no entries.
1077      * Note that this is not the first rowkey, but rather the byte form of
1078      * the first KeyValue.
1079      */
1080     public byte [] getFirstKey() {
1081       if (blockIndex == null) {
1082         throw new RuntimeException("Block index not loaded");
1083       }
1084       return this.blockIndex.isEmpty()? null: this.blockIndex.blockKeys[0];
1085     }
1086 
1087     /**
1088      * @return the first row key, or null if the file is empty.
1089      * TODO move this to StoreFile after Ryan's patch goes in
1090      * to eliminate KeyValue here
1091      */
1092     public byte[] getFirstRowKey() {
1093       byte[] firstKey = getFirstKey();
1094       if (firstKey == null) return null;
1095       return KeyValue.createKeyValueFromKey(firstKey).getRow();
1096     }
1097 
1098     /**
1099      * @return number of KV entries in this HFile
1100      */
1101     public int getEntries() {
1102       if (!this.isFileInfoLoaded()) {
1103         throw new RuntimeException("File info not loaded");
1104       }
1105       return this.trailer.entryCount;
1106     }
1107 
1108     /**
1109      * @return Last key in the file.  May be null if file has no entries.
1110      * Note that this is not the last rowkey, but rather the byte form of
1111      * the last KeyValue.
1112      */
1113     public byte [] getLastKey() {
1114       if (!isFileInfoLoaded()) {
1115         throw new RuntimeException("Load file info first");
1116       }
1117       return this.blockIndex.isEmpty()? null: this.lastkey;
1118     }
1119 
1120     /**
1121      * @return the last row key, or null if the file is empty.
1122      * TODO move this to StoreFile after Ryan's patch goes in
1123      * to eliminate KeyValue here
1124      */
1125     public byte[] getLastRowKey() {
1126       byte[] lastKey = getLastKey();
1127       if (lastKey == null) return null;
1128       return KeyValue.createKeyValueFromKey(lastKey).getRow();
1129     }
1130 
1131     /**
1132      * @return number of K entries in this HFile's filter.  Returns KV count if no filter.
1133      */
1134     public int getFilterEntries() {
1135       return getEntries();
1136     }
1137 
1138     /**
1139      * @return Comparator.
1140      */
1141     public RawComparator<byte []> getComparator() {
1142       return this.comparator;
1143     }
1144 
1145     /**
1146      * @return index size
1147      */
1148     public long indexSize() {
1149       return (this.blockIndex != null? this.blockIndex.heapSize(): 0) +
1150         ((this.metaIndex != null)? this.metaIndex.heapSize(): 0);
1151     }
1152 
1153     /**
1154      * @return Midkey for this file.  We work with block boundaries only so
1155      * returned midkey is an approximation only.
1156      * @throws IOException
1157      */
1158     public byte [] midkey() throws IOException {
1159       if (!isFileInfoLoaded() || this.blockIndex.isEmpty()) {
1160         return null;
1161       }
1162       return this.blockIndex.midkey();
1163     }
1164 
1165     public void close() throws IOException {
1166       if (this.closeIStream && this.istream != null) {
1167         this.istream.close();
1168         this.istream = null;
1169       }
1170     }
1171 
1172     public String getName() {
1173       return name;
1174     }
1175 
1176     /*
1177      * Implementation of {@link HFileScanner} interface.
1178      */
1179     protected static class Scanner implements HFileScanner {
1180       private final Reader reader;
1181       private ByteBuffer block;
1182       private int currBlock;
1183 
1184       private final boolean cacheBlocks;
1185       private final boolean pread;
1186 
1187       private int currKeyLen = 0;
1188       private int currValueLen = 0;
1189 
1190       public int blockFetches = 0;
1191 
1192       public Scanner(Reader r, boolean cacheBlocks, final boolean pread) {
1193         this.reader = r;
1194         this.cacheBlocks = cacheBlocks;
1195         this.pread = pread;
1196       }
1197 
1198       public KeyValue getKeyValue() {
1199         if(this.block == null) {
1200           return null;
1201         }
1202         return new KeyValue(this.block.array(),
1203             this.block.arrayOffset() + this.block.position() - 8,
1204             this.currKeyLen+this.currValueLen+8);
1205       }
1206 
1207       public ByteBuffer getKey() {
1208         if (this.block == null || this.currKeyLen == 0) {
1209           throw new RuntimeException("you need to seekTo() before calling getKey()");
1210         }
1211         ByteBuffer keyBuff = this.block.slice();
1212         keyBuff.limit(this.currKeyLen);
1213         keyBuff.rewind();
1214         // Do keyBuff.asReadOnly()?
1215         return keyBuff;
1216       }
1217 
1218       public ByteBuffer getValue() {
1219         if (block == null || currKeyLen == 0) {
1220           throw new RuntimeException("you need to seekTo() before calling getValue()");
1221         }
1222         // TODO: Could this be done with one ByteBuffer rather than create two?
1223         ByteBuffer valueBuff = this.block.slice();
1224         valueBuff.position(this.currKeyLen);
1225         valueBuff = valueBuff.slice();
1226         valueBuff.limit(currValueLen);
1227         valueBuff.rewind();
1228         return valueBuff;
1229       }
1230 
1231       public boolean next() throws IOException {
1232         // LOG.deug("rem:" + block.remaining() + " p:" + block.position() +
1233         // " kl: " + currKeyLen + " kv: " + currValueLen);
1234         if (block == null) {
1235           throw new IOException("Next called on non-seeked scanner");
1236         }
1237         block.position(block.position() + currKeyLen + currValueLen);
1238         if (block.remaining() <= 0) {
1239           // LOG.debug("Fetch next block");
1240           currBlock++;
1241           if (currBlock >= reader.blockIndex.count) {
1242             // damn we are at the end
1243             currBlock = 0;
1244             block = null;
1245             return false;
1246           }
1247           block = reader.readBlock(this.currBlock, this.cacheBlocks, this.pread);
1248           currKeyLen = Bytes.toInt(block.array(), block.arrayOffset()+block.position(), 4);
1249           currValueLen = Bytes.toInt(block.array(), block.arrayOffset()+block.position()+4, 4);
1250           block.position(block.position()+8);
1251           blockFetches++;
1252           return true;
1253         }
1254         // LOG.debug("rem:" + block.remaining() + " p:" + block.position() +
1255         // " kl: " + currKeyLen + " kv: " + currValueLen);
1256         currKeyLen = Bytes.toInt(block.array(), block.arrayOffset()+block.position(), 4);
1257         currValueLen = Bytes.toInt(block.array(), block.arrayOffset()+block.position()+4, 4);
1258         block.position(block.position()+8);
1259         return true;
1260       }
1261 
1262       public int seekTo(byte [] key) throws IOException {
1263         return seekTo(key, 0, key.length);
1264       }
1265 
1266       public int seekTo(byte[] key, int offset, int length) throws IOException {
1267         int b = reader.blockContainingKey(key, offset, length);
1268         if (b < 0) return -1; // falls before the beginning of the file! :-(
1269         // Avoid re-reading the same block (that'd be dumb).
1270         loadBlock(b, true);
1271         return blockSeek(key, offset, length, false);
1272       }
1273 
1274       public int reseekTo(byte [] key) throws IOException {
1275         return reseekTo(key, 0, key.length);
1276       }
1277 
1278       public int reseekTo(byte[] key, int offset, int length)
1279         throws IOException {
1280 
1281         if (this.block != null && this.currKeyLen != 0) {
1282           ByteBuffer bb = getKey();
1283           int compared = this.reader.comparator.compare(key, offset, length,
1284               bb.array(), bb.arrayOffset(), bb.limit());
1285           if (compared < 1) {
1286             //If the required key is less than or equal to current key, then
1287             //don't do anything.
1288             return compared;
1289           }
1290         }
1291 
1292         int b = reader.blockContainingKey(key, offset, length);
1293         if (b < 0) {
1294           return -1;
1295         }
1296         loadBlock(b, false);
1297         return blockSeek(key, offset, length, false);
1298       }
1299 
1300       /**
1301        * Within a loaded block, seek looking for the first key
1302        * that is smaller than (or equal to?) the key we are interested in.
1303        *
1304        * A note on the seekBefore - if you have seekBefore = true, AND the
1305        * first key in the block = key, then you'll get thrown exceptions.
1306        * @param key to find
1307        * @param seekBefore find the key before the exact match.
1308        * @return
1309        */
1310       private int blockSeek(byte[] key, int offset, int length, boolean seekBefore) {
1311         int klen, vlen;
1312         int lastLen = 0;
1313         do {
1314           klen = block.getInt();
1315           vlen = block.getInt();
1316           int comp = this.reader.comparator.compare(key, offset, length,
1317             block.array(), block.arrayOffset() + block.position(), klen);
1318           if (comp == 0) {
1319             if (seekBefore) {
1320               block.position(block.position() - lastLen - 16);
1321               currKeyLen = block.getInt();
1322               currValueLen = block.getInt();
1323               return 1; // non exact match.
1324             }
1325             currKeyLen = klen;
1326             currValueLen = vlen;
1327             return 0; // indicate exact match
1328           }
1329           if (comp < 0) {
1330             // go back one key:
1331             block.position(block.position() - lastLen - 16);
1332             currKeyLen = block.getInt();
1333             currValueLen = block.getInt();
1334             return 1;
1335           }
1336           block.position(block.position() + klen + vlen);
1337           lastLen = klen + vlen ;
1338         } while(block.remaining() > 0);
1339         // ok we are at the end, so go back a littleeeeee....
1340         // The 8 in the below is intentionally different to the 16s in the above
1341         // Do the math you you'll figure it.
1342         block.position(block.position() - lastLen - 8);
1343         currKeyLen = block.getInt();
1344         currValueLen = block.getInt();
1345         return 1; // didn't exactly find it.
1346       }
1347 
1348       public boolean seekBefore(byte [] key) throws IOException {
1349         return seekBefore(key, 0, key.length);
1350       }
1351 
1352       public boolean seekBefore(byte[] key, int offset, int length)
1353       throws IOException {
1354         int b = reader.blockContainingKey(key, offset, length);
1355         if (b < 0)
1356           return false; // key is before the start of the file.
1357 
1358         // Question: does this block begin with 'key'?
1359         if (this.reader.comparator.compare(reader.blockIndex.blockKeys[b],
1360             0, reader.blockIndex.blockKeys[b].length,
1361             key, offset, length) == 0) {
1362           // Ok the key we're interested in is the first of the block, so go back one.
1363           if (b == 0) {
1364             // we have a 'problem', the key we want is the first of the file.
1365             return false;
1366           }
1367           b--;
1368           // TODO shortcut: seek forward in this block to the last key of the block.
1369         }
1370         loadBlock(b, true);
1371         blockSeek(key, offset, length, true);
1372         return true;
1373       }
1374 
1375       public String getKeyString() {
1376         return Bytes.toStringBinary(block.array(), block.arrayOffset() +
1377           block.position(), currKeyLen);
1378       }
1379 
1380       public String getValueString() {
1381         return Bytes.toString(block.array(), block.arrayOffset() +
1382           block.position() + currKeyLen, currValueLen);
1383       }
1384 
1385       public Reader getReader() {
1386         return this.reader;
1387       }
1388 
1389       public boolean isSeeked(){
1390         return this.block != null;
1391       }
1392 
1393       public boolean seekTo() throws IOException {
1394         if (this.reader.blockIndex.isEmpty()) {
1395           return false;
1396         }
1397         if (block != null && currBlock == 0) {
1398           block.rewind();
1399           currKeyLen = block.getInt();
1400           currValueLen = block.getInt();
1401           return true;
1402         }
1403         currBlock = 0;
1404         block = reader.readBlock(this.currBlock, this.cacheBlocks, this.pread);
1405         currKeyLen = block.getInt();
1406         currValueLen = block.getInt();
1407         blockFetches++;
1408         return true;
1409       }
1410 
1411       private void loadBlock(int bloc, boolean rewind) throws IOException {
1412         if (block == null) {
1413           block = reader.readBlock(bloc, this.cacheBlocks, this.pread);
1414           currBlock = bloc;
1415           blockFetches++;
1416         } else {
1417           if (bloc != currBlock) {
1418             block = reader.readBlock(bloc, this.cacheBlocks, this.pread);
1419             currBlock = bloc;
1420             blockFetches++;
1421           } else {
1422             // we are already in the same block, just rewind to seek again.
1423             if (rewind) {
1424               block.rewind();
1425             }
1426             else {
1427               //Go back by (size of rowlength + size of valuelength) = 8 bytes
1428               block.position(block.position()-8);
1429             }
1430           }
1431         }
1432       }
1433 
1434       @Override
1435       public String toString() {
1436         return "HFileScanner for reader " + String.valueOf(reader);
1437       }
1438     }
1439 
1440     public String getTrailerInfo() {
1441       return trailer.toString();
1442     }
1443   }
1444 
1445   /*
1446    * The RFile has a fixed trailer which contains offsets to other variable
1447    * parts of the file.  Also includes basic metadata on this file.
1448    */
1449   private static class FixedFileTrailer {
1450     // Offset to the fileinfo data, a small block of vitals..
1451     long fileinfoOffset;
1452     // Offset to the data block index.
1453     long dataIndexOffset;
1454     // How many index counts are there (aka: block count)
1455     int dataIndexCount;
1456     // Offset to the meta block index.
1457     long metaIndexOffset;
1458     // How many meta block index entries (aka: meta block count)
1459     int metaIndexCount;
1460     long totalUncompressedBytes;
1461     int entryCount;
1462     int compressionCodec;
1463     int version = 1;
1464 
1465     FixedFileTrailer() {
1466       super();
1467     }
1468 
1469     static int trailerSize() {
1470       // Keep this up to date...
1471       return
1472       ( Bytes.SIZEOF_INT * 5 ) +
1473       ( Bytes.SIZEOF_LONG * 4 ) +
1474       TRAILERBLOCKMAGIC.length;
1475     }
1476 
1477     void serialize(DataOutputStream outputStream) throws IOException {
1478       outputStream.write(TRAILERBLOCKMAGIC);
1479       outputStream.writeLong(fileinfoOffset);
1480       outputStream.writeLong(dataIndexOffset);
1481       outputStream.writeInt(dataIndexCount);
1482       outputStream.writeLong(metaIndexOffset);
1483       outputStream.writeInt(metaIndexCount);
1484       outputStream.writeLong(totalUncompressedBytes);
1485       outputStream.writeInt(entryCount);
1486       outputStream.writeInt(compressionCodec);
1487       outputStream.writeInt(version);
1488     }
1489 
1490     void deserialize(DataInputStream inputStream) throws IOException {
1491       byte [] header = new byte[TRAILERBLOCKMAGIC.length];
1492       inputStream.readFully(header);
1493       if ( !Arrays.equals(header, TRAILERBLOCKMAGIC)) {
1494         throw new IOException("Trailer 'header' is wrong; does the trailer " +
1495           "size match content?");
1496       }
1497       fileinfoOffset         = inputStream.readLong();
1498       dataIndexOffset        = inputStream.readLong();
1499       dataIndexCount         = inputStream.readInt();
1500 
1501       metaIndexOffset        = inputStream.readLong();
1502       metaIndexCount         = inputStream.readInt();
1503 
1504       totalUncompressedBytes = inputStream.readLong();
1505       entryCount             = inputStream.readInt();
1506       compressionCodec       = inputStream.readInt();
1507       version                = inputStream.readInt();
1508 
1509       if (version != 1) {
1510         throw new IOException("Wrong version: " + version);
1511       }
1512     }
1513 
1514     @Override
1515     public String toString() {
1516       return "fileinfoOffset=" + fileinfoOffset +
1517       ", dataIndexOffset=" + dataIndexOffset +
1518       ", dataIndexCount=" + dataIndexCount +
1519       ", metaIndexOffset=" + metaIndexOffset +
1520       ", metaIndexCount=" + metaIndexCount +
1521       ", totalBytes=" + totalUncompressedBytes +
1522       ", entryCount=" + entryCount +
1523       ", version=" + version;
1524     }
1525   }
1526 
1527   /*
1528    * The block index for a RFile.
1529    * Used reading.
1530    */
1531   static class BlockIndex implements HeapSize {
1532     // How many actual items are there? The next insert location too.
1533     int count = 0;
1534     byte [][] blockKeys;
1535     long [] blockOffsets;
1536     int [] blockDataSizes;
1537     int size = 0;
1538 
1539     /* Needed doing lookup on blocks.
1540      */
1541     final RawComparator<byte []> comparator;
1542 
1543     /*
1544      * Shutdown default constructor
1545      */
1546     @SuppressWarnings("unused")
1547     private BlockIndex() {
1548       this(null);
1549     }
1550 
1551 
1552     /**
1553      * @param c comparator used to compare keys.
1554      */
1555     BlockIndex(final RawComparator<byte []>c) {
1556       this.comparator = c;
1557       // Guess that cost of three arrays + this object is 4 * 8 bytes.
1558       this.size += (4 * 8);
1559     }
1560 
1561     /**
1562      * @return True if block index is empty.
1563      */
1564     boolean isEmpty() {
1565       return this.blockKeys.length <= 0;
1566     }
1567 
1568     /**
1569      * Adds a new entry in the block index.
1570      *
1571      * @param key Last key in the block
1572      * @param offset file offset where the block is stored
1573      * @param dataSize the uncompressed data size
1574      */
1575     void add(final byte[] key, final long offset, final int dataSize) {
1576       blockOffsets[count] = offset;
1577       blockKeys[count] = key;
1578       blockDataSizes[count] = dataSize;
1579       count++;
1580       this.size += (Bytes.SIZEOF_INT * 2 + key.length);
1581     }
1582 
1583     /**
1584      * @param key Key to find
1585      * @return Offset of block containing <code>key</code> or -1 if this file
1586      * does not contain the request.
1587      */
1588     int blockContainingKey(final byte[] key, int offset, int length) {
1589       int pos = Bytes.binarySearch(blockKeys, key, offset, length, this.comparator);
1590       if (pos < 0) {
1591         pos ++;
1592         pos *= -1;
1593         if (pos == 0) {
1594           // falls before the beginning of the file.
1595           return -1;
1596         }
1597         // When switched to "first key in block" index, binarySearch now returns
1598         // the block with a firstKey < key.  This means the value we want is potentially
1599         // in the next block.
1600         pos --; // in previous block.
1601 
1602         return pos;
1603       }
1604       // wow, a perfect hit, how unlikely?
1605       return pos;
1606     }
1607 
1608     /*
1609      * @return File midkey.  Inexact.  Operates on block boundaries.  Does
1610      * not go into blocks.
1611      */
1612     byte [] midkey() throws IOException {
1613       int pos = ((this.count - 1)/2);              // middle of the index
1614       if (pos < 0) {
1615         throw new IOException("HFile empty");
1616       }
1617       return this.blockKeys[pos];
1618     }
1619 
1620     /*
1621      * Write out index. Whatever we write here must jibe with what
1622      * BlockIndex#readIndex is expecting.  Make sure the two ends of the
1623      * index serialization match.
1624      * @param o
1625      * @param keys
1626      * @param offsets
1627      * @param sizes
1628      * @param c
1629      * @return Position at which we entered the index.
1630      * @throws IOException
1631      */
1632     static long writeIndex(final FSDataOutputStream o,
1633       final List<byte []> keys, final List<Long> offsets,
1634       final List<Integer> sizes)
1635     throws IOException {
1636       long pos = o.getPos();
1637       // Don't write an index if nothing in the index.
1638       if (keys.size() > 0) {
1639         o.write(INDEXBLOCKMAGIC);
1640         // Write the index.
1641         for (int i = 0; i < keys.size(); ++i) {
1642           o.writeLong(offsets.get(i).longValue());
1643           o.writeInt(sizes.get(i).intValue());
1644           byte [] key = keys.get(i);
1645           Bytes.writeByteArray(o, key);
1646         }
1647       }
1648       return pos;
1649     }
1650 
1651     /*
1652      * Read in the index that is at <code>indexOffset</code>
1653      * Must match what was written by writeIndex in the Writer.close.
1654      * @param in
1655      * @param indexOffset
1656      * @throws IOException
1657      */
1658     static BlockIndex readIndex(final RawComparator<byte []> c,
1659         final FSDataInputStream in, final long indexOffset, final int indexSize)
1660     throws IOException {
1661       BlockIndex bi = new BlockIndex(c);
1662       bi.blockOffsets = new long[indexSize];
1663       bi.blockKeys = new byte[indexSize][];
1664       bi.blockDataSizes = new int[indexSize];
1665       // If index size is zero, no index was written.
1666       if (indexSize > 0) {
1667         in.seek(indexOffset);
1668         byte [] magic = new byte[INDEXBLOCKMAGIC.length];
1669         IOUtils.readFully(in, magic, 0, magic.length);
1670         if (!Arrays.equals(magic, INDEXBLOCKMAGIC)) {
1671           throw new IOException("Index block magic is wrong: " +
1672             Arrays.toString(magic));
1673         }
1674         for (int i = 0; i < indexSize; ++i ) {
1675           long offset   = in.readLong();
1676           int dataSize  = in.readInt();
1677           byte [] key = Bytes.readByteArray(in);
1678           bi.add(key, offset, dataSize);
1679         }
1680       }
1681       return bi;
1682     }
1683 
1684     @Override
1685     public String toString() {
1686       StringBuilder sb = new StringBuilder();
1687       sb.append("size=" + count);
1688       for (int i = 0; i < count ; i++) {
1689         sb.append(", ");
1690         sb.append("key=").append(Bytes.toStringBinary(blockKeys[i])).
1691           append(", offset=").append(blockOffsets[i]).
1692           append(", dataSize=" + blockDataSizes[i]);
1693       }
1694       return sb.toString();
1695     }
1696 
1697     public long heapSize() {
1698       long heapsize = ClassSize.align(ClassSize.OBJECT +
1699           2 * Bytes.SIZEOF_INT + (3 + 1) * ClassSize.REFERENCE);
1700       //Calculating the size of blockKeys
1701       if(blockKeys != null) {
1702         //Adding array + references overhead
1703         heapsize += ClassSize.align(ClassSize.ARRAY +
1704             blockKeys.length * ClassSize.REFERENCE);
1705         //Adding bytes
1706         for(byte [] bs : blockKeys) {
1707           heapsize += ClassSize.align(ClassSize.ARRAY + bs.length);
1708         }
1709       }
1710       if(blockOffsets != null) {
1711         heapsize += ClassSize.align(ClassSize.ARRAY +
1712             blockOffsets.length * Bytes.SIZEOF_LONG);
1713       }
1714       if(blockDataSizes != null) {
1715         heapsize += ClassSize.align(ClassSize.ARRAY +
1716             blockDataSizes.length * Bytes.SIZEOF_INT);
1717       }
1718 
1719       return ClassSize.align(heapsize);
1720     }
1721 
1722   }
1723 
1724   /*
1725    * Metadata for this file.  Conjured by the writer.  Read in by the reader.
1726    */
1727   static class FileInfo extends HbaseMapWritable<byte [], byte []> {
1728     static final String RESERVED_PREFIX = "hfile.";
1729     static final byte[] RESERVED_PREFIX_BYTES = Bytes.toBytes(RESERVED_PREFIX);
1730     static final byte [] LASTKEY = Bytes.toBytes(RESERVED_PREFIX + "LASTKEY");
1731     static final byte [] AVG_KEY_LEN =
1732       Bytes.toBytes(RESERVED_PREFIX + "AVG_KEY_LEN");
1733     static final byte [] AVG_VALUE_LEN =
1734       Bytes.toBytes(RESERVED_PREFIX + "AVG_VALUE_LEN");
1735     static final byte [] COMPARATOR =
1736       Bytes.toBytes(RESERVED_PREFIX + "COMPARATOR");
1737 
1738     /*
1739      * Constructor.
1740      */
1741     FileInfo() {
1742       super();
1743     }
1744   }
1745 
1746   /**
1747    * Return true if the given file info key is reserved for internal
1748    * use by HFile.
1749    */
1750   public static boolean isReservedFileInfoKey(byte[] key) {
1751     return Bytes.startsWith(key, FileInfo.RESERVED_PREFIX_BYTES);
1752   }
1753 
1754 
1755   /**
1756    * Get names of supported compression algorithms. The names are acceptable by
1757    * HFile.Writer.
1758    *
1759    * @return Array of strings, each represents a supported compression
1760    *         algorithm. Currently, the following compression algorithms are
1761    *         supported.
1762    *         <ul>
1763    *         <li>"none" - No compression.
1764    *         <li>"gz" - GZIP compression.
1765    *         </ul>
1766    */
1767   public static String[] getSupportedCompressionAlgorithms() {
1768     return Compression.getSupportedAlgorithms();
1769   }
1770 
1771   // Utility methods.
1772   /*
1773    * @param l Long to convert to an int.
1774    * @return <code>l</code> cast as an int.
1775    */
1776   static int longToInt(final long l) {
1777     // Expecting the size() of a block not exceeding 4GB. Assuming the
1778     // size() will wrap to negative integer if it exceeds 2GB (From tfile).
1779     return (int)(l & 0x00000000ffffffffL);
1780   }
1781 
1782   /**
1783    * Returns all files belonging to the given region directory. Could return an
1784    * empty list.
1785    *
1786    * @param fs  The file system reference.
1787    * @param regionDir  The region directory to scan.
1788    * @return The list of files found.
1789    * @throws IOException When scanning the files fails.
1790    */
1791   static List<Path> getStoreFiles(FileSystem fs, Path regionDir)
1792   throws IOException {
1793     List<Path> res = new ArrayList<Path>();
1794     PathFilter dirFilter = new FSUtils.DirFilter(fs);
1795     FileStatus[] familyDirs = fs.listStatus(regionDir, dirFilter);
1796     for(FileStatus dir : familyDirs) {
1797       FileStatus[] files = fs.listStatus(dir.getPath());
1798       for (FileStatus file : files) {
1799         if (!file.isDir()) {
1800           res.add(file.getPath());
1801         }
1802       }
1803     }
1804     return res;
1805   }
1806 
1807   public static void main(String []args) throws IOException {
1808     try {
1809       // create options
1810       Options options = new Options();
1811       options.addOption("v", "verbose", false, "Verbose output; emits file and meta data delimiters");
1812       options.addOption("p", "printkv", false, "Print key/value pairs");
1813       options.addOption("m", "printmeta", false, "Print meta data of file");
1814       options.addOption("k", "checkrow", false,
1815         "Enable row order check; looks for out-of-order keys");
1816       options.addOption("a", "checkfamily", false, "Enable family check");
1817       options.addOption("f", "file", true,
1818         "File to scan. Pass full-path; e.g. hdfs://a:9000/hbase/.META./12/34");
1819       options.addOption("r", "region", true,
1820         "Region to scan. Pass region name; e.g. '.META.,,1'");
1821       if (args.length == 0) {
1822         HelpFormatter formatter = new HelpFormatter();
1823         formatter.printHelp("HFile ", options, true);
1824         System.exit(-1);
1825       }
1826       CommandLineParser parser = new PosixParser();
1827       CommandLine cmd = parser.parse(options, args);
1828       boolean verbose = cmd.hasOption("v");
1829       boolean printKeyValue = cmd.hasOption("p");
1830       boolean printMeta = cmd.hasOption("m");
1831       boolean checkRow = cmd.hasOption("k");
1832       boolean checkFamily = cmd.hasOption("a");
1833       // get configuration, file system and get list of files
1834       Configuration conf = HBaseConfiguration.create();
1835       conf.set("fs.defaultFS",
1836         conf.get(org.apache.hadoop.hbase.HConstants.HBASE_DIR));
1837       FileSystem fs = FileSystem.get(conf);
1838       ArrayList<Path> files = new ArrayList<Path>();
1839       if (cmd.hasOption("f")) {
1840         files.add(new Path(cmd.getOptionValue("f")));
1841       }
1842       if (cmd.hasOption("r")) {
1843         String regionName = cmd.getOptionValue("r");
1844         byte[] rn = Bytes.toBytes(regionName);
1845         byte[][] hri = HRegionInfo.parseRegionName(rn);
1846         Path rootDir = FSUtils.getRootDir(conf);
1847         Path tableDir = new Path(rootDir, Bytes.toString(hri[0]));
1848         String enc = HRegionInfo.encodeRegionName(rn);
1849         Path regionDir = new Path(tableDir, enc);
1850         if (verbose) System.out.println("region dir -> " + regionDir);
1851         List<Path> regionFiles = getStoreFiles(fs, regionDir);
1852         if (verbose) System.out.println("Number of region files found -> " +
1853           regionFiles.size());
1854         if (verbose) {
1855           int i = 1;
1856           for (Path p : regionFiles) {
1857             if (verbose) System.out.println("Found file[" + i++ + "] -> " + p);
1858           }
1859         }
1860         files.addAll(regionFiles);
1861       }
1862       // iterate over all files found
1863       for (Path file : files) {
1864         if (verbose) System.out.println("Scanning -> " + file);
1865         if (!fs.exists(file)) {
1866           System.err.println("ERROR, file doesnt exist: " + file);
1867           continue;
1868         }
1869         // create reader and load file info
1870         HFile.Reader reader = new HFile.Reader(fs, file, null, false);
1871         Map<byte[],byte[]> fileInfo = reader.loadFileInfo();
1872         // scan over file and read key/value's and check if requested
1873         HFileScanner scanner = reader.getScanner(false, false);
1874         scanner.seekTo();
1875         KeyValue pkv = null;
1876         int count = 0;
1877         do {
1878           KeyValue kv = scanner.getKeyValue();
1879           // dump key value
1880           if (printKeyValue) {
1881             System.out.println("K: " + kv +
1882               " V: " + Bytes.toStringBinary(kv.getValue()));
1883           }
1884           // check if rows are in order
1885           if (checkRow && pkv != null) {
1886             if (Bytes.compareTo(pkv.getRow(), kv.getRow()) > 0) {
1887               System.err.println("WARNING, previous row is greater then" +
1888                 " current row\n\tfilename -> " + file +
1889                 "\n\tprevious -> " + Bytes.toStringBinary(pkv.getKey()) +
1890                 "\n\tcurrent  -> " + Bytes.toStringBinary(kv.getKey()));
1891             }
1892           }
1893           // check if families are consistent
1894           if (checkFamily) {
1895             String fam = Bytes.toString(kv.getFamily());
1896             if (!file.toString().contains(fam)) {
1897               System.err.println("WARNING, filename does not match kv family," +
1898                 "\n\tfilename -> " + file +
1899                 "\n\tkeyvalue -> " + Bytes.toStringBinary(kv.getKey()));
1900             }
1901             if (pkv != null && Bytes.compareTo(pkv.getFamily(), kv.getFamily()) != 0) {
1902               System.err.println("WARNING, previous kv has different family" +
1903                 " compared to current key\n\tfilename -> " + file +
1904                 "\n\tprevious -> " +  Bytes.toStringBinary(pkv.getKey()) +
1905                 "\n\tcurrent  -> " + Bytes.toStringBinary(kv.getKey()));
1906             }
1907           }
1908           pkv = kv;
1909           count++;
1910         } while (scanner.next());
1911         if (verbose || printKeyValue) {
1912           System.out.println("Scanned kv count -> " + count);
1913         }
1914         // print meta data
1915         if (printMeta) {
1916           System.out.println("Block index size as per heapsize: " + reader.indexSize());
1917           System.out.println(reader.toString());
1918           System.out.println(reader.getTrailerInfo());
1919           System.out.println("Fileinfo:");
1920           for (Map.Entry<byte[], byte[]> e : fileInfo.entrySet()) {
1921             System.out.print(Bytes.toString(e.getKey()) + " = " );
1922             if (Bytes.compareTo(e.getKey(), Bytes.toBytes("MAX_SEQ_ID_KEY"))==0) {
1923               long seqid = Bytes.toLong(e.getValue());
1924               System.out.println(seqid);
1925             } else {
1926               System.out.println(Bytes.toStringBinary(e.getValue()));
1927             }
1928           }
1929         }
1930         reader.close();
1931       }
1932     } catch (Exception e) {
1933       e.printStackTrace();
1934     }
1935   }
1936 }