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     protected boolean includeMVCCReadpoint = true;
327 
328     WriterFactory(Configuration conf, CacheConfig cacheConf) {
329       this.conf = conf;
330       this.cacheConf = cacheConf;
331     }
332 
333     public WriterFactory withPath(FileSystem fs, Path path) {
334       Preconditions.checkNotNull(fs);
335       Preconditions.checkNotNull(path);
336       this.fs = fs;
337       this.path = path;
338       return this;
339     }
340 
341     public WriterFactory withOutputStream(FSDataOutputStream ostream) {
342       Preconditions.checkNotNull(ostream);
343       this.ostream = ostream;
344       return this;
345     }
346 
347     public WriterFactory withBlockSize(int blockSize) {
348       this.blockSize = blockSize;
349       return this;
350     }
351 
352     public WriterFactory withCompression(Compression.Algorithm compression) {
353       Preconditions.checkNotNull(compression);
354       this.compression = compression;
355       return this;
356     }
357 
358     public WriterFactory withCompression(String compressAlgo) {
359       Preconditions.checkNotNull(compression);
360       this.compression = AbstractHFileWriter.compressionByName(compressAlgo);
361       return this;
362     }
363 
364     public WriterFactory withDataBlockEncoder(HFileDataBlockEncoder encoder) {
365       Preconditions.checkNotNull(encoder);
366       this.encoder = encoder;
367       return this;
368     }
369 
370     public WriterFactory withComparator(KeyComparator comparator) {
371       Preconditions.checkNotNull(comparator);
372       this.comparator = comparator;
373       return this;
374     }
375 
376     public WriterFactory withChecksumType(ChecksumType checksumType) {
377       Preconditions.checkNotNull(checksumType);
378       this.checksumType = checksumType;
379       return this;
380     }
381 
382     public WriterFactory withBytesPerChecksum(int bytesPerChecksum) {
383       this.bytesPerChecksum = bytesPerChecksum;
384       return this;
385     }
386 
387     public WriterFactory includeMVCCReadpoint(boolean includeMVCCReadpoint) {
388       this.includeMVCCReadpoint = includeMVCCReadpoint;
389       return this;
390     }
391 
392     public Writer create() throws IOException {
393       if ((path != null ? 1 : 0) + (ostream != null ? 1 : 0) != 1) {
394         throw new AssertionError("Please specify exactly one of " +
395             "filesystem/path or path");
396       }
397       if (path != null) {
398         ostream = AbstractHFileWriter.createOutputStream(conf, fs, path);
399       }
400       return createWriter(fs, path, ostream, blockSize, compression, encoder, comparator,
401           checksumType, bytesPerChecksum, includeMVCCReadpoint);
402     }
403 
404     protected abstract Writer createWriter(FileSystem fs, Path path,
405         FSDataOutputStream ostream, int blockSize,
406         Compression.Algorithm compress,
407         HFileDataBlockEncoder dataBlockEncoder,
408         KeyComparator comparator, ChecksumType checksumType,
409         int bytesPerChecksum, boolean includeMVCCReadpoint) throws IOException;
410   }
411 
412   /** The configuration key for HFile version to use for new files */
413   public static final String FORMAT_VERSION_KEY = "hfile.format.version";
414 
415   public static int getFormatVersion(Configuration conf) {
416     int version = conf.getInt(FORMAT_VERSION_KEY, MAX_FORMAT_VERSION);
417     checkFormatVersion(version);
418     return version;
419   }
420 
421   /**
422    * Returns the factory to be used to create {@link HFile} writers.
423    * Disables block cache access for all writers created through the
424    * returned factory.
425    */
426   public static final WriterFactory getWriterFactoryNoCache(Configuration
427        conf) {
428     Configuration tempConf = new Configuration(conf);
429     tempConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.0f);
430     return HFile.getWriterFactory(conf, new CacheConfig(tempConf));
431   }
432 
433   /**
434    * Returns the factory to be used to create {@link HFile} writers
435    */
436   public static final WriterFactory getWriterFactory(Configuration conf,
437       CacheConfig cacheConf) {
438     SchemaMetrics.configureGlobally(conf);
439     int version = getFormatVersion(conf);
440     switch (version) {
441     case 1:
442       return new HFileWriterV1.WriterFactoryV1(conf, cacheConf);
443     case 2:
444       return new HFileWriterV2.WriterFactoryV2(conf, cacheConf);
445     default:
446       throw new IllegalArgumentException("Cannot create writer for HFile " +
447           "format version " + version);
448     }
449   }
450 
451   /** An abstraction used by the block index */
452   public interface CachingBlockReader {
453     HFileBlock readBlock(long offset, long onDiskBlockSize,
454         boolean cacheBlock, final boolean pread, final boolean isCompaction,
455         BlockType expectedBlockType)
456         throws IOException;
457   }
458 
459   /** An interface used by clients to open and iterate an {@link HFile}. */
460   public interface Reader extends Closeable, CachingBlockReader,
461       SchemaAware {
462     /**
463      * Returns this reader's "name". Usually the last component of the path.
464      * Needs to be constant as the file is being moved to support caching on
465      * write.
466      */
467     String getName();
468 
469     String getColumnFamilyName();
470 
471     RawComparator<byte []> getComparator();
472 
473     HFileScanner getScanner(boolean cacheBlocks,
474        final boolean pread, final boolean isCompaction);
475 
476     ByteBuffer getMetaBlock(String metaBlockName,
477        boolean cacheBlock) throws IOException;
478 
479     Map<byte[], byte[]> loadFileInfo() throws IOException;
480 
481     byte[] getLastKey();
482 
483     byte[] midkey() throws IOException;
484 
485     long length();
486 
487     long getEntries();
488 
489     byte[] getFirstKey();
490 
491     long indexSize();
492 
493     byte[] getFirstRowKey();
494 
495     byte[] getLastRowKey();
496 
497     FixedFileTrailer getTrailer();
498 
499     HFileBlockIndex.BlockIndexReader getDataBlockIndexReader();
500 
501     HFileScanner getScanner(boolean cacheBlocks, boolean pread);
502 
503     Compression.Algorithm getCompressionAlgorithm();
504 
505     /**
506      * Retrieves general Bloom filter metadata as appropriate for each
507      * {@link HFile} version.
508      * Knows nothing about how that metadata is structured.
509      */
510     DataInput getGeneralBloomFilterMetadata() throws IOException;
511 
512     /**
513      * Retrieves delete family Bloom filter metadata as appropriate for each
514      * {@link HFile}  version.
515      * Knows nothing about how that metadata is structured.
516      */
517     DataInput getDeleteBloomFilterMetadata() throws IOException;
518 
519     Path getPath();
520 
521     /** Close method with optional evictOnClose */
522     void close(boolean evictOnClose) throws IOException;
523 
524     DataBlockEncoding getEncodingOnDisk();
525   }
526 
527   /**
528    * Method returns the reader given the specified arguments.
529    * TODO This is a bad abstraction.  See HBASE-6635.
530    *
531    * @param path hfile's path
532    * @param fsdis an open checksummed stream of path's file
533    * @param fsdisNoFsChecksum an open unchecksummed stream of path's file
534    * @param size max size of the trailer.
535    * @param closeIStream boolean for closing file after the getting the reader version.
536    * @param cacheConf Cache configuation values, cannot be null.
537    * @param preferredEncodingInCache
538    * @param hfs
539    * @return an appropriate instance of HFileReader
540    * @throws IOException If file is invalid, will throw CorruptHFileException flavored IOException
541    */
542   private static Reader pickReaderVersion(Path path, FSDataInputStream fsdis,
543       FSDataInputStream fsdisNoFsChecksum,
544       long size, boolean closeIStream, CacheConfig cacheConf,
545       DataBlockEncoding preferredEncodingInCache, HFileSystem hfs)
546       throws IOException {
547     FixedFileTrailer trailer = null;
548     try {
549       trailer = FixedFileTrailer.readFromStream(fsdis, size);
550     } catch (IllegalArgumentException iae) {
551       throw new CorruptHFileException("Problem reading HFile Trailer from file " + path, iae);
552     }
553     switch (trailer.getMajorVersion()) {
554     case 1:
555       return new HFileReaderV1(path, trailer, fsdis, size, closeIStream,
556           cacheConf);
557     case 2:
558       return new HFileReaderV2(path, trailer, fsdis, fsdisNoFsChecksum,
559           size, closeIStream,
560           cacheConf, preferredEncodingInCache, hfs);
561     default:
562       throw new CorruptHFileException("Invalid HFile version " + trailer.getMajorVersion());
563     }
564   }
565 
566   /**
567    * @param fs A file system
568    * @param path Path to HFile
569    * @param cacheConf Cache configuration for hfile's contents
570    * @param preferredEncodingInCache Preferred in-cache data encoding algorithm.
571    * @return A version specific Hfile Reader
572    * @throws IOException If file is invalid, will throw CorruptHFileException flavored IOException
573    */
574   public static Reader createReaderWithEncoding(
575       FileSystem fs, Path path, CacheConfig cacheConf,
576       DataBlockEncoding preferredEncodingInCache) throws IOException {
577     final boolean closeIStream = true;
578     HFileSystem hfs = null;
579     FSDataInputStream fsdis = fs.open(path);
580     FSDataInputStream fsdisNoFsChecksum = fsdis;
581     // If the fs is not an instance of HFileSystem, then create an
582     // instance of HFileSystem that wraps over the specified fs.
583     // In this case, we will not be able to avoid checksumming inside
584     // the filesystem.
585     if (!(fs instanceof HFileSystem)) {
586       hfs = new HFileSystem(fs);
587     } else {
588       hfs = (HFileSystem)fs;
589       // open a stream to read data without checksum verification in
590       // the filesystem
591       if (hfs != null) {
592         fsdisNoFsChecksum = hfs.getNoChecksumFs().open(path);
593       }
594     }
595     return pickReaderVersion(path, fsdis, fsdisNoFsChecksum,
596         fs.getFileStatus(path).getLen(), closeIStream, cacheConf,
597         preferredEncodingInCache, hfs);
598   }
599 
600   /**
601    * @param fs A file system
602    * @param path Path to HFile
603    * @param fsdis an open checksummed stream of path's file
604    * @param fsdisNoFsChecksum an open unchecksummed stream of path's file
605    * @param size max size of the trailer.
606    * @param cacheConf Cache configuration for hfile's contents
607    * @param preferredEncodingInCache Preferred in-cache data encoding algorithm.
608    * @param closeIStream boolean for closing file after the getting the reader version.
609    * @return A version specific Hfile Reader
610    * @throws IOException If file is invalid, will throw CorruptHFileException flavored IOException
611    */
612   public static Reader createReaderWithEncoding(
613       FileSystem fs, Path path, FSDataInputStream fsdis,
614       FSDataInputStream fsdisNoFsChecksum, long size, CacheConfig cacheConf,
615       DataBlockEncoding preferredEncodingInCache, boolean closeIStream)
616       throws IOException {
617     HFileSystem hfs = null;
618 
619     // If the fs is not an instance of HFileSystem, then create an
620     // instance of HFileSystem that wraps over the specified fs.
621     // In this case, we will not be able to avoid checksumming inside
622     // the filesystem.
623     if (!(fs instanceof HFileSystem)) {
624       hfs = new HFileSystem(fs);
625     } else {
626       hfs = (HFileSystem)fs;
627     }
628     return pickReaderVersion(path, fsdis, fsdisNoFsChecksum, size,
629                              closeIStream, cacheConf,
630                              preferredEncodingInCache, hfs);
631   }
632 
633   /**
634    * @param fs filesystem
635    * @param path Path to file to read
636    * @param cacheConf This must not be null.  @see {@link org.apache.hadoop.hbase.io.hfile.CacheConfig#CacheConfig(Configuration)}
637    * @return an active Reader instance
638    * @throws IOException Will throw a CorruptHFileException (DoNotRetryIOException subtype) if hfile is corrupt/invalid.
639    */
640   public static Reader createReader(
641       FileSystem fs, Path path, CacheConfig cacheConf) throws IOException {
642     return createReaderWithEncoding(fs, path, cacheConf,
643         DataBlockEncoding.NONE);
644   }
645 
646   /**
647    * This factory method is used only by unit tests
648    */
649   static Reader createReaderFromStream(Path path,
650       FSDataInputStream fsdis, long size, CacheConfig cacheConf)
651       throws IOException {
652     final boolean closeIStream = false;
653     return pickReaderVersion(path, fsdis, fsdis, size, closeIStream, cacheConf,
654         DataBlockEncoding.NONE, null);
655   }
656 
657   /*
658    * Metadata for this file.  Conjured by the writer.  Read in by the reader.
659    */
660   static class FileInfo extends HbaseMapWritable<byte [], byte []> {
661     static final String RESERVED_PREFIX = "hfile.";
662     static final byte[] RESERVED_PREFIX_BYTES = Bytes.toBytes(RESERVED_PREFIX);
663     static final byte [] LASTKEY = Bytes.toBytes(RESERVED_PREFIX + "LASTKEY");
664     static final byte [] AVG_KEY_LEN =
665       Bytes.toBytes(RESERVED_PREFIX + "AVG_KEY_LEN");
666     static final byte [] AVG_VALUE_LEN =
667       Bytes.toBytes(RESERVED_PREFIX + "AVG_VALUE_LEN");
668     static final byte [] COMPARATOR =
669       Bytes.toBytes(RESERVED_PREFIX + "COMPARATOR");
670 
671     /**
672      * Append the given key/value pair to the file info, optionally checking the
673      * key prefix.
674      *
675      * @param k key to add
676      * @param v value to add
677      * @param checkPrefix whether to check that the provided key does not start
678      *          with the reserved prefix
679      * @return this file info object
680      * @throws IOException if the key or value is invalid
681      */
682     public FileInfo append(final byte[] k, final byte[] v,
683         final boolean checkPrefix) throws IOException {
684       if (k == null || v == null) {
685         throw new NullPointerException("Key nor value may be null");
686       }
687       if (checkPrefix && isReservedFileInfoKey(k)) {
688         throw new IOException("Keys with a " + FileInfo.RESERVED_PREFIX
689             + " are reserved");
690       }
691       put(k, v);
692       return this;
693     }
694 
695   }
696 
697   /** Return true if the given file info key is reserved for internal use. */
698   public static boolean isReservedFileInfoKey(byte[] key) {
699     return Bytes.startsWith(key, FileInfo.RESERVED_PREFIX_BYTES);
700   }
701 
702   /**
703    * Get names of supported compression algorithms. The names are acceptable by
704    * HFile.Writer.
705    *
706    * @return Array of strings, each represents a supported compression
707    *         algorithm. Currently, the following compression algorithms are
708    *         supported.
709    *         <ul>
710    *         <li>"none" - No compression.
711    *         <li>"gz" - GZIP compression.
712    *         </ul>
713    */
714   public static String[] getSupportedCompressionAlgorithms() {
715     return Compression.getSupportedAlgorithms();
716   }
717 
718   // Utility methods.
719   /*
720    * @param l Long to convert to an int.
721    * @return <code>l</code> cast as an int.
722    */
723   static int longToInt(final long l) {
724     // Expecting the size() of a block not exceeding 4GB. Assuming the
725     // size() will wrap to negative integer if it exceeds 2GB (From tfile).
726     return (int)(l & 0x00000000ffffffffL);
727   }
728 
729   /**
730    * Returns all files belonging to the given region directory. Could return an
731    * empty list.
732    *
733    * @param fs  The file system reference.
734    * @param regionDir  The region directory to scan.
735    * @return The list of files found.
736    * @throws IOException When scanning the files fails.
737    */
738   static List<Path> getStoreFiles(FileSystem fs, Path regionDir)
739       throws IOException {
740     List<Path> res = new ArrayList<Path>();
741     PathFilter dirFilter = new FSUtils.DirFilter(fs);
742     FileStatus[] familyDirs = fs.listStatus(regionDir, dirFilter);
743     for(FileStatus dir : familyDirs) {
744       FileStatus[] files = fs.listStatus(dir.getPath());
745       for (FileStatus file : files) {
746         if (!file.isDir()) {
747           res.add(file.getPath());
748         }
749       }
750     }
751     return res;
752   }
753 
754   public static void main(String[] args) throws IOException {
755     HFilePrettyPrinter prettyPrinter = new HFilePrettyPrinter();
756     System.exit(prettyPrinter.run(args));
757   }
758 
759   /**
760    * Checks the given {@link HFile} format version, and throws an exception if
761    * invalid. Note that if the version number comes from an input file and has
762    * not been verified, the caller needs to re-throw an {@link IOException} to
763    * indicate that this is not a software error, but corrupted input.
764    *
765    * @param version an HFile version
766    * @throws IllegalArgumentException if the version is invalid
767    */
768   public static void checkFormatVersion(int version)
769       throws IllegalArgumentException {
770     if (version < MIN_FORMAT_VERSION || version > MAX_FORMAT_VERSION) {
771       throw new IllegalArgumentException("Invalid HFile version: " + version
772           + " (expected to be " + "between " + MIN_FORMAT_VERSION + " and "
773           + MAX_FORMAT_VERSION + ")");
774     }
775   }
776 
777 }