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.DataInput;
24  import java.io.IOException;
25  import java.nio.ByteBuffer;
26  import java.util.ArrayList;
27  import java.util.Collection;
28  import java.util.List;
29  import java.util.Map;
30  import java.util.concurrent.ArrayBlockingQueue;
31  import java.util.concurrent.BlockingQueue;
32  import java.util.concurrent.atomic.AtomicInteger;
33  import java.util.concurrent.atomic.AtomicLong;
34  
35  import org.apache.commons.logging.Log;
36  import org.apache.commons.logging.LogFactory;
37  import org.apache.hadoop.conf.Configuration;
38  import org.apache.hadoop.fs.FSDataInputStream;
39  import org.apache.hadoop.fs.FSDataOutputStream;
40  import org.apache.hadoop.fs.FileStatus;
41  import org.apache.hadoop.fs.FileSystem;
42  import org.apache.hadoop.fs.Path;
43  import org.apache.hadoop.fs.PathFilter;
44  import org.apache.hadoop.hbase.HColumnDescriptor;
45  import org.apache.hadoop.hbase.HConstants;
46  import org.apache.hadoop.hbase.KeyValue;
47  import org.apache.hadoop.hbase.KeyValue.KeyComparator;
48  import org.apache.hadoop.hbase.fs.HFileSystem;
49  import org.apache.hadoop.hbase.io.HbaseMapWritable;
50  import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
51  import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
52  import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics.SchemaAware;
53  import org.apache.hadoop.hbase.util.BloomFilterWriter;
54  import org.apache.hadoop.hbase.util.Bytes;
55  import org.apache.hadoop.hbase.util.ChecksumType;
56  import org.apache.hadoop.hbase.util.FSUtils;
57  import org.apache.hadoop.io.RawComparator;
58  import org.apache.hadoop.io.Writable;
59  
60  import com.google.common.base.Preconditions;
61  import com.google.common.collect.Lists;
62  
63  /**
64   * File format for hbase.
65   * A file of sorted key/value pairs. Both keys and values are byte arrays.
66   * <p>
67   * The memory footprint of a HFile includes the following (below is taken from the
68   * <a
69   * href=https://issues.apache.org/jira/browse/HADOOP-3315>TFile</a> documentation
70   * but applies also to HFile):
71   * <ul>
72   * <li>Some constant overhead of reading or writing a compressed block.
73   * <ul>
74   * <li>Each compressed block requires one compression/decompression codec for
75   * I/O.
76   * <li>Temporary space to buffer the key.
77   * <li>Temporary space to buffer the value.
78   * </ul>
79   * <li>HFile index, which is proportional to the total number of Data Blocks.
80   * The total amount of memory needed to hold the index can be estimated as
81   * (56+AvgKeySize)*NumBlocks.
82   * </ul>
83   * Suggestions on performance optimization.
84   * <ul>
85   * <li>Minimum block size. We recommend a setting of minimum block size between
86   * 8KB to 1MB for general usage. Larger block size is preferred if files are
87   * primarily for sequential access. However, it would lead to inefficient random
88   * access (because there are more data to decompress). Smaller blocks are good
89   * for random access, but require more memory to hold the block index, and may
90   * be slower to create (because we must flush the compressor stream at the
91   * conclusion of each data block, which leads to an FS I/O flush). Further, due
92   * to the internal caching in Compression codec, the smallest possible block
93   * size would be around 20KB-30KB.
94   * <li>The current implementation does not offer true multi-threading for
95   * reading. The implementation uses FSDataInputStream seek()+read(), which is
96   * shown to be much faster than positioned-read call in single thread mode.
97   * However, it also means that if multiple threads attempt to access the same
98   * HFile (using multiple scanners) simultaneously, the actual I/O is carried out
99   * sequentially even if they access different DFS blocks (Reexamine! pread seems
100  * to be 10% faster than seek+read in my testing -- stack).
101  * <li>Compression codec. Use "none" if the data is not very compressable (by
102  * compressable, I mean a compression ratio at least 2:1). Generally, use "lzo"
103  * as the starting point for experimenting. "gz" overs slightly better
104  * compression ratio over "lzo" but requires 4x CPU to compress and 2x CPU to
105  * decompress, comparing to "lzo".
106  * </ul>
107  *
108  * For more on the background behind HFile, see <a
109  * href=https://issues.apache.org/jira/browse/HBASE-61>HBASE-61</a>.
110  * <p>
111  * File is made of data blocks followed by meta data blocks (if any), a fileinfo
112  * block, data block index, meta data block index, and a fixed size trailer
113  * which records the offsets at which file changes content type.
114  * <pre>&lt;data blocks>&lt;meta blocks>&lt;fileinfo>&lt;data index>&lt;meta index>&lt;trailer></pre>
115  * Each block has a bit of magic at its start.  Block are comprised of
116  * key/values.  In data blocks, they are both byte arrays.  Metadata blocks are
117  * a String key and a byte array value.  An empty file looks like this:
118  * <pre>&lt;fileinfo>&lt;trailer></pre>.  That is, there are not data nor meta
119  * blocks present.
120  * <p>
121  * TODO: Do scanners need to be able to take a start and end row?
122  * TODO: Should BlockIndex know the name of its file?  Should it have a Path
123  * that points at its file say for the case where an index lives apart from
124  * an HFile instance?
125  */
126 public class HFile {
127   static final Log LOG = LogFactory.getLog(HFile.class);
128 
129   /**
130    * Maximum length of key in HFile.
131    */
132   public final static int MAXIMUM_KEY_LENGTH = Integer.MAX_VALUE;
133 
134   /**
135    * Default block size for an HFile.
136    */
137   public final static int DEFAULT_BLOCKSIZE = 64 * 1024;
138 
139   /**
140    * Default compression: none.
141    */
142   public final static Compression.Algorithm DEFAULT_COMPRESSION_ALGORITHM =
143     Compression.Algorithm.NONE;
144 
145   /** Minimum supported HFile format version */
146   public static final int MIN_FORMAT_VERSION = 1;
147 
148   /** Maximum supported HFile format version */
149   public static final int MAX_FORMAT_VERSION = 2;
150 
151   /** Default compression name: none. */
152   public final static String DEFAULT_COMPRESSION =
153     DEFAULT_COMPRESSION_ALGORITHM.getName();
154 
155   /**
156    * We assume that HFile path ends with
157    * ROOT_DIR/TABLE_NAME/REGION_NAME/CF_NAME/HFILE, so it has at least this
158    * many levels of nesting. This is needed for identifying table and CF name
159    * from an HFile path.
160    */
161   public final static int MIN_NUM_HFILE_PATH_LEVELS = 5;
162 
163   /**
164    * The number of bytes per checksum.
165    */
166   public static final int DEFAULT_BYTES_PER_CHECKSUM = 16 * 1024;
167   public static final ChecksumType DEFAULT_CHECKSUM_TYPE = ChecksumType.CRC32;
168 
169   // For measuring latency of "sequential" reads and writes
170   private static final AtomicInteger readOps = new AtomicInteger();
171   private static final AtomicLong readTimeNano = new AtomicLong();
172   private static final AtomicInteger writeOps = new AtomicInteger();
173   private static final AtomicLong writeTimeNano = new AtomicLong();
174 
175   // For measuring latency of pread
176   private static final AtomicInteger preadOps = new AtomicInteger();
177   private static final AtomicLong preadTimeNano = new AtomicLong();
178 
179   // For measuring number of checksum failures
180   static final AtomicLong checksumFailures = new AtomicLong();
181 
182   // For getting more detailed stats on FS latencies
183   // If, for some reason, the metrics subsystem stops polling for latencies,
184   // I don't want data to pile up in a memory leak
185   // so, after LATENCY_BUFFER_SIZE items have been enqueued for processing,
186   // fs latency stats will be dropped (and this behavior will be logged)
187   private static final int LATENCY_BUFFER_SIZE = 5000;
188   private static final BlockingQueue<Long> fsReadLatenciesNanos =
189       new ArrayBlockingQueue<Long>(LATENCY_BUFFER_SIZE);
190   private static final BlockingQueue<Long> fsWriteLatenciesNanos =
191       new ArrayBlockingQueue<Long>(LATENCY_BUFFER_SIZE);
192   private static final BlockingQueue<Long> fsPreadLatenciesNanos =
193       new ArrayBlockingQueue<Long>(LATENCY_BUFFER_SIZE);
194 
195   public static final void offerReadLatency(long latencyNanos, boolean pread) {
196     if (pread) {
197       fsPreadLatenciesNanos.offer(latencyNanos); // might be silently dropped, if the queue is full
198       preadOps.incrementAndGet();
199       preadTimeNano.addAndGet(latencyNanos);
200     } else {
201       fsReadLatenciesNanos.offer(latencyNanos); // might be silently dropped, if the queue is full
202       readTimeNano.addAndGet(latencyNanos);
203       readOps.incrementAndGet();
204     }
205   }
206 
207   public static final void offerWriteLatency(long latencyNanos) {
208     fsWriteLatenciesNanos.offer(latencyNanos); // might be silently dropped, if the queue is full
209 
210     writeTimeNano.addAndGet(latencyNanos);
211     writeOps.incrementAndGet();
212   }
213 
214   public static final Collection<Long> getReadLatenciesNanos() {
215     final List<Long> latencies =
216         Lists.newArrayListWithCapacity(fsReadLatenciesNanos.size());
217     fsReadLatenciesNanos.drainTo(latencies);
218     return latencies;
219   }
220 
221   public static final Collection<Long> getPreadLatenciesNanos() {
222     final List<Long> latencies =
223         Lists.newArrayListWithCapacity(fsPreadLatenciesNanos.size());
224     fsPreadLatenciesNanos.drainTo(latencies);
225     return latencies;
226   }
227 
228   public static final Collection<Long> getWriteLatenciesNanos() {
229     final List<Long> latencies =
230         Lists.newArrayListWithCapacity(fsWriteLatenciesNanos.size());
231     fsWriteLatenciesNanos.drainTo(latencies);
232     return latencies;
233   }
234 
235   // for test purpose
236   public static volatile AtomicLong dataBlockReadCnt = new AtomicLong(0);
237 
238   // number of sequential reads
239   public static final int getReadOps() {
240     return readOps.getAndSet(0);
241   }
242 
243   public static final long getReadTimeMs() {
244     return readTimeNano.getAndSet(0) / 1000000;
245   }
246 
247   // number of positional reads
248   public static final int getPreadOps() {
249     return preadOps.getAndSet(0);
250   }
251 
252   public static final long getPreadTimeMs() {
253     return preadTimeNano.getAndSet(0) / 1000000;
254   }
255 
256   public static final int getWriteOps() {
257     return writeOps.getAndSet(0);
258   }
259 
260   public static final long getWriteTimeMs() {
261     return writeTimeNano.getAndSet(0) / 1000000;
262   }
263 
264   /**
265    * Number of checksum verification failures. It also
266    * clears the counter.
267    */
268   public static final long getChecksumFailuresCount() {
269     return checksumFailures.getAndSet(0);
270   }
271 
272   /** API required to write an {@link HFile} */
273   public interface Writer extends Closeable {
274 
275     /** Add an element to the file info map. */
276     void appendFileInfo(byte[] key, byte[] value) throws IOException;
277 
278     void append(KeyValue kv) throws IOException;
279 
280     void append(byte[] key, byte[] value) throws IOException;
281 
282     /** @return the path to this {@link HFile} */
283     Path getPath();
284 
285     String getColumnFamilyName();
286 
287     void appendMetaBlock(String bloomFilterMetaKey, Writable metaWriter);
288 
289     /**
290      * Adds an inline block writer such as a multi-level block index writer or
291      * a compound Bloom filter writer.
292      */
293     void addInlineBlockWriter(InlineBlockWriter bloomWriter);
294 
295     /**
296      * Store general Bloom filter in the file. This does not deal with Bloom filter
297      * internals but is necessary, since Bloom filters are stored differently
298      * in HFile version 1 and version 2.
299      */
300     void addGeneralBloomFilter(BloomFilterWriter bfw);
301 
302     /**
303      * Store delete family Bloom filter in the file, which is only supported in
304      * HFile V2.
305      */
306     void addDeleteFamilyBloomFilter(BloomFilterWriter bfw) throws IOException;
307   }
308 
309   /**
310    * This variety of ways to construct writers is used throughout the code, and
311    * we want to be able to swap writer implementations.
312    */
313   public static abstract class WriterFactory {
314     protected final Configuration conf;
315     protected final CacheConfig cacheConf;
316     protected FileSystem fs;
317     protected Path path;
318     protected FSDataOutputStream ostream;
319     protected int blockSize = HColumnDescriptor.DEFAULT_BLOCKSIZE;
320     protected Compression.Algorithm compression =
321         HFile.DEFAULT_COMPRESSION_ALGORITHM;
322     protected HFileDataBlockEncoder encoder = NoOpDataBlockEncoder.INSTANCE;
323     protected KeyComparator comparator;
324     protected ChecksumType checksumType = HFile.DEFAULT_CHECKSUM_TYPE;
325     protected int bytesPerChecksum = DEFAULT_BYTES_PER_CHECKSUM;
326 
327     WriterFactory(Configuration conf, CacheConfig cacheConf) {
328       this.conf = conf;
329       this.cacheConf = cacheConf;
330     }
331 
332     public WriterFactory withPath(FileSystem fs, Path path) {
333       Preconditions.checkNotNull(fs);
334       Preconditions.checkNotNull(path);
335       this.fs = fs;
336       this.path = path;
337       return this;
338     }
339 
340     public WriterFactory withOutputStream(FSDataOutputStream ostream) {
341       Preconditions.checkNotNull(ostream);
342       this.ostream = ostream;
343       return this;
344     }
345 
346     public WriterFactory withBlockSize(int blockSize) {
347       this.blockSize = blockSize;
348       return this;
349     }
350 
351     public WriterFactory withCompression(Compression.Algorithm compression) {
352       Preconditions.checkNotNull(compression);
353       this.compression = compression;
354       return this;
355     }
356 
357     public WriterFactory withCompression(String compressAlgo) {
358       Preconditions.checkNotNull(compression);
359       this.compression = AbstractHFileWriter.compressionByName(compressAlgo);
360       return this;
361     }
362 
363     public WriterFactory withDataBlockEncoder(HFileDataBlockEncoder encoder) {
364       Preconditions.checkNotNull(encoder);
365       this.encoder = encoder;
366       return this;
367     }
368 
369     public WriterFactory withComparator(KeyComparator comparator) {
370       Preconditions.checkNotNull(comparator);
371       this.comparator = comparator;
372       return this;
373     }
374 
375     public WriterFactory withChecksumType(ChecksumType checksumType) {
376       Preconditions.checkNotNull(checksumType);
377       this.checksumType = checksumType;
378       return this;
379     }
380 
381     public WriterFactory withBytesPerChecksum(int bytesPerChecksum) {
382       this.bytesPerChecksum = bytesPerChecksum;
383       return this;
384     }
385 
386     public Writer create() throws IOException {
387       if ((path != null ? 1 : 0) + (ostream != null ? 1 : 0) != 1) {
388         throw new AssertionError("Please specify exactly one of " +
389             "filesystem/path or path");
390       }
391       if (path != null) {
392         ostream = AbstractHFileWriter.createOutputStream(conf, fs, path);
393       }
394       return createWriter(fs, path, ostream, blockSize,
395           compression, encoder, comparator, checksumType, bytesPerChecksum);
396     }
397 
398     protected abstract Writer createWriter(FileSystem fs, Path path,
399         FSDataOutputStream ostream, int blockSize,
400         Compression.Algorithm compress,
401         HFileDataBlockEncoder dataBlockEncoder,
402         KeyComparator comparator, ChecksumType checksumType,
403         int bytesPerChecksum) throws IOException;
404   }
405 
406   /** The configuration key for HFile version to use for new files */
407   public static final String FORMAT_VERSION_KEY = "hfile.format.version";
408 
409   public static int getFormatVersion(Configuration conf) {
410     int version = conf.getInt(FORMAT_VERSION_KEY, MAX_FORMAT_VERSION);
411     checkFormatVersion(version);
412     return version;
413   }
414 
415   /**
416    * Returns the factory to be used to create {@link HFile} writers.
417    * Disables block cache access for all writers created through the
418    * returned factory.
419    */
420   public static final WriterFactory getWriterFactoryNoCache(Configuration
421        conf) {
422     Configuration tempConf = new Configuration(conf);
423     tempConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.0f);
424     return HFile.getWriterFactory(conf, new CacheConfig(tempConf));
425   }
426 
427   /**
428    * Returns the factory to be used to create {@link HFile} writers
429    */
430   public static final WriterFactory getWriterFactory(Configuration conf,
431       CacheConfig cacheConf) {
432     SchemaMetrics.configureGlobally(conf);
433     int version = getFormatVersion(conf);
434     switch (version) {
435     case 1:
436       return new HFileWriterV1.WriterFactoryV1(conf, cacheConf);
437     case 2:
438       return new HFileWriterV2.WriterFactoryV2(conf, cacheConf);
439     default:
440       throw new IllegalArgumentException("Cannot create writer for HFile " +
441           "format version " + version);
442     }
443   }
444 
445   /** An abstraction used by the block index */
446   public interface CachingBlockReader {
447     HFileBlock readBlock(long offset, long onDiskBlockSize,
448         boolean cacheBlock, final boolean pread, final boolean isCompaction,
449         BlockType expectedBlockType)
450         throws IOException;
451   }
452 
453   /** An interface used by clients to open and iterate an {@link HFile}. */
454   public interface Reader extends Closeable, CachingBlockReader,
455       SchemaAware {
456     /**
457      * Returns this reader's "name". Usually the last component of the path.
458      * Needs to be constant as the file is being moved to support caching on
459      * write.
460      */
461     String getName();
462 
463     String getColumnFamilyName();
464 
465     RawComparator<byte []> getComparator();
466 
467     HFileScanner getScanner(boolean cacheBlocks,
468        final boolean pread, final boolean isCompaction);
469 
470     ByteBuffer getMetaBlock(String metaBlockName,
471        boolean cacheBlock) throws IOException;
472 
473     Map<byte[], byte[]> loadFileInfo() throws IOException;
474 
475     byte[] getLastKey();
476 
477     byte[] midkey() throws IOException;
478 
479     long length();
480 
481     long getEntries();
482 
483     byte[] getFirstKey();
484 
485     long indexSize();
486 
487     byte[] getFirstRowKey();
488 
489     byte[] getLastRowKey();
490 
491     FixedFileTrailer getTrailer();
492 
493     HFileBlockIndex.BlockIndexReader getDataBlockIndexReader();
494 
495     HFileScanner getScanner(boolean cacheBlocks, boolean pread);
496 
497     Compression.Algorithm getCompressionAlgorithm();
498 
499     /**
500      * Retrieves general Bloom filter metadata as appropriate for each
501      * {@link HFile} version.
502      * Knows nothing about how that metadata is structured.
503      */
504     DataInput getGeneralBloomFilterMetadata() throws IOException;
505 
506     /**
507      * Retrieves delete family Bloom filter metadata as appropriate for each
508      * {@link HFile}  version.
509      * Knows nothing about how that metadata is structured.
510      */
511     DataInput getDeleteBloomFilterMetadata() throws IOException;
512 
513     Path getPath();
514 
515     /** Close method with optional evictOnClose */
516     void close(boolean evictOnClose) throws IOException;
517 
518     DataBlockEncoding getEncodingOnDisk();
519   }
520 
521   /**
522    * Method returns the reader given the specified arguments.
523    * TODO This is a bad abstraction.  See HBASE-6635.
524    *
525    * @param path hfile's path
526    * @param fsdis an open checksummed stream of path's file
527    * @param fsdisNoFsChecksum an open unchecksummed stream of path's file
528    * @param size max size of the trailer.
529    * @param closeIStream boolean for closing file after the getting the reader version.
530    * @param cacheConf Cache configuation values, cannot be null.
531    * @param preferredEncodingInCache
532    * @param hfs
533    * @return an appropriate instance of HFileReader
534    * @throws IOException If file is invalid, will throw CorruptHFileException flavored IOException
535    */
536   private static Reader pickReaderVersion(Path path, FSDataInputStream fsdis,
537       FSDataInputStream fsdisNoFsChecksum,
538       long size, boolean closeIStream, CacheConfig cacheConf,
539       DataBlockEncoding preferredEncodingInCache, HFileSystem hfs)
540       throws IOException {
541     FixedFileTrailer trailer = null;
542     try {
543       trailer = FixedFileTrailer.readFromStream(fsdis, size);
544     } catch (IllegalArgumentException iae) {
545       throw new CorruptHFileException("Problem reading HFile Trailer from file " + path, iae);
546     }
547     switch (trailer.getMajorVersion()) {
548     case 1:
549       return new HFileReaderV1(path, trailer, fsdis, size, closeIStream,
550           cacheConf);
551     case 2:
552       return new HFileReaderV2(path, trailer, fsdis, fsdisNoFsChecksum,
553           size, closeIStream,
554           cacheConf, preferredEncodingInCache, hfs);
555     default:
556       throw new CorruptHFileException("Invalid HFile version " + trailer.getMajorVersion());
557     }
558   }
559 
560   /**
561    * @param fs A file system
562    * @param path Path to HFile
563    * @param cacheConf Cache configuration for hfile's contents
564    * @param preferredEncodingInCache Preferred in-cache data encoding algorithm.
565    * @return A version specific Hfile Reader
566    * @throws IOException If file is invalid, will throw CorruptHFileException flavored IOException
567    */
568   public static Reader createReaderWithEncoding(
569       FileSystem fs, Path path, CacheConfig cacheConf,
570       DataBlockEncoding preferredEncodingInCache) throws IOException {
571     final boolean closeIStream = true;
572     HFileSystem hfs = null;
573     FSDataInputStream fsdis = fs.open(path);
574     FSDataInputStream fsdisNoFsChecksum = fsdis;
575     // If the fs is not an instance of HFileSystem, then create an
576     // instance of HFileSystem that wraps over the specified fs.
577     // In this case, we will not be able to avoid checksumming inside
578     // the filesystem.
579     if (!(fs instanceof HFileSystem)) {
580       hfs = new HFileSystem(fs);
581     } else {
582       hfs = (HFileSystem)fs;
583       // open a stream to read data without checksum verification in
584       // the filesystem
585       if (hfs != null) {
586         fsdisNoFsChecksum = hfs.getNoChecksumFs().open(path);
587       }
588     }
589     return pickReaderVersion(path, fsdis, fsdisNoFsChecksum,
590         fs.getFileStatus(path).getLen(), closeIStream, cacheConf,
591         preferredEncodingInCache, hfs);
592   }
593 
594   /**
595    * @param fs A file system
596    * @param path Path to HFile
597    * @param fsdis an open checksummed stream of path's file
598    * @param fsdisNoFsChecksum an open unchecksummed stream of path's file
599    * @param size max size of the trailer.
600    * @param cacheConf Cache configuration for hfile's contents
601    * @param preferredEncodingInCache Preferred in-cache data encoding algorithm.
602    * @param closeIStream boolean for closing file after the getting the reader version.
603    * @return A version specific Hfile Reader
604    * @throws IOException If file is invalid, will throw CorruptHFileException flavored IOException
605    */
606   public static Reader createReaderWithEncoding(
607       FileSystem fs, Path path, FSDataInputStream fsdis,
608       FSDataInputStream fsdisNoFsChecksum, long size, CacheConfig cacheConf,
609       DataBlockEncoding preferredEncodingInCache, boolean closeIStream)
610       throws IOException {
611     HFileSystem hfs = null;
612 
613     // If the fs is not an instance of HFileSystem, then create an
614     // instance of HFileSystem that wraps over the specified fs.
615     // In this case, we will not be able to avoid checksumming inside
616     // the filesystem.
617     if (!(fs instanceof HFileSystem)) {
618       hfs = new HFileSystem(fs);
619     } else {
620       hfs = (HFileSystem)fs;
621     }
622     return pickReaderVersion(path, fsdis, fsdisNoFsChecksum, size,
623                              closeIStream, cacheConf,
624                              preferredEncodingInCache, hfs);
625   }
626 
627   /**
628    * @param fs filesystem
629    * @param path Path to file to read
630    * @param cacheConf This must not be null.  @see {@link org.apache.hadoop.hbase.io.hfile.CacheConfig#CacheConfig(Configuration)}
631    * @return an active Reader instance
632    * @throws IOException Will throw a CorruptHFileException (DoNotRetryIOException subtype) if hfile is corrupt/invalid.
633    */
634   public static Reader createReader(
635       FileSystem fs, Path path, CacheConfig cacheConf) throws IOException {
636     return createReaderWithEncoding(fs, path, cacheConf,
637         DataBlockEncoding.NONE);
638   }
639 
640   /**
641    * This factory method is used only by unit tests
642    */
643   static Reader createReaderFromStream(Path path,
644       FSDataInputStream fsdis, long size, CacheConfig cacheConf)
645       throws IOException {
646     final boolean closeIStream = false;
647     return pickReaderVersion(path, fsdis, fsdis, size, closeIStream, cacheConf,
648         DataBlockEncoding.NONE, null);
649   }
650 
651   /*
652    * Metadata for this file.  Conjured by the writer.  Read in by the reader.
653    */
654   static class FileInfo extends HbaseMapWritable<byte [], byte []> {
655     static final String RESERVED_PREFIX = "hfile.";
656     static final byte[] RESERVED_PREFIX_BYTES = Bytes.toBytes(RESERVED_PREFIX);
657     static final byte [] LASTKEY = Bytes.toBytes(RESERVED_PREFIX + "LASTKEY");
658     static final byte [] AVG_KEY_LEN =
659       Bytes.toBytes(RESERVED_PREFIX + "AVG_KEY_LEN");
660     static final byte [] AVG_VALUE_LEN =
661       Bytes.toBytes(RESERVED_PREFIX + "AVG_VALUE_LEN");
662     static final byte [] COMPARATOR =
663       Bytes.toBytes(RESERVED_PREFIX + "COMPARATOR");
664 
665     /**
666      * Append the given key/value pair to the file info, optionally checking the
667      * key prefix.
668      *
669      * @param k key to add
670      * @param v value to add
671      * @param checkPrefix whether to check that the provided key does not start
672      *          with the reserved prefix
673      * @return this file info object
674      * @throws IOException if the key or value is invalid
675      */
676     public FileInfo append(final byte[] k, final byte[] v,
677         final boolean checkPrefix) throws IOException {
678       if (k == null || v == null) {
679         throw new NullPointerException("Key nor value may be null");
680       }
681       if (checkPrefix && isReservedFileInfoKey(k)) {
682         throw new IOException("Keys with a " + FileInfo.RESERVED_PREFIX
683             + " are reserved");
684       }
685       put(k, v);
686       return this;
687     }
688 
689   }
690 
691   /** Return true if the given file info key is reserved for internal use. */
692   public static boolean isReservedFileInfoKey(byte[] key) {
693     return Bytes.startsWith(key, FileInfo.RESERVED_PREFIX_BYTES);
694   }
695 
696   /**
697    * Get names of supported compression algorithms. The names are acceptable by
698    * HFile.Writer.
699    *
700    * @return Array of strings, each represents a supported compression
701    *         algorithm. Currently, the following compression algorithms are
702    *         supported.
703    *         <ul>
704    *         <li>"none" - No compression.
705    *         <li>"gz" - GZIP compression.
706    *         </ul>
707    */
708   public static String[] getSupportedCompressionAlgorithms() {
709     return Compression.getSupportedAlgorithms();
710   }
711 
712   // Utility methods.
713   /*
714    * @param l Long to convert to an int.
715    * @return <code>l</code> cast as an int.
716    */
717   static int longToInt(final long l) {
718     // Expecting the size() of a block not exceeding 4GB. Assuming the
719     // size() will wrap to negative integer if it exceeds 2GB (From tfile).
720     return (int)(l & 0x00000000ffffffffL);
721   }
722 
723   /**
724    * Returns all files belonging to the given region directory. Could return an
725    * empty list.
726    *
727    * @param fs  The file system reference.
728    * @param regionDir  The region directory to scan.
729    * @return The list of files found.
730    * @throws IOException When scanning the files fails.
731    */
732   static List<Path> getStoreFiles(FileSystem fs, Path regionDir)
733       throws IOException {
734     List<Path> res = new ArrayList<Path>();
735     PathFilter dirFilter = new FSUtils.DirFilter(fs);
736     FileStatus[] familyDirs = fs.listStatus(regionDir, dirFilter);
737     for(FileStatus dir : familyDirs) {
738       FileStatus[] files = fs.listStatus(dir.getPath());
739       for (FileStatus file : files) {
740         if (!file.isDir()) {
741           res.add(file.getPath());
742         }
743       }
744     }
745     return res;
746   }
747 
748   public static void main(String[] args) throws IOException {
749     HFilePrettyPrinter prettyPrinter = new HFilePrettyPrinter();
750     System.exit(prettyPrinter.run(args));
751   }
752 
753   /**
754    * Checks the given {@link HFile} format version, and throws an exception if
755    * invalid. Note that if the version number comes from an input file and has
756    * not been verified, the caller needs to re-throw an {@link IOException} to
757    * indicate that this is not a software error, but corrupted input.
758    *
759    * @param version an HFile version
760    * @throws IllegalArgumentException if the version is invalid
761    */
762   public static void checkFormatVersion(int version)
763       throws IllegalArgumentException {
764     if (version < MIN_FORMAT_VERSION || version > MAX_FORMAT_VERSION) {
765       throw new IllegalArgumentException("Invalid HFile version: " + version
766           + " (expected to be " + "between " + MIN_FORMAT_VERSION + " and "
767           + MAX_FORMAT_VERSION + ")");
768     }
769   }
770 
771 }