View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.io.hfile;
20  
21  import java.io.ByteArrayInputStream;
22  import java.io.Closeable;
23  import java.io.DataInput;
24  import java.io.DataInputStream;
25  import java.io.DataOutputStream;
26  import java.io.IOException;
27  import java.io.SequenceInputStream;
28  import java.net.InetSocketAddress;
29  import java.nio.ByteBuffer;
30  import java.util.ArrayList;
31  import java.util.Collection;
32  import java.util.Comparator;
33  import java.util.List;
34  import java.util.Map;
35  import java.util.Set;
36  import java.util.SortedMap;
37  import java.util.TreeMap;
38  
39  import org.apache.commons.logging.Log;
40  import org.apache.commons.logging.LogFactory;
41  import org.apache.hadoop.hbase.classification.InterfaceAudience;
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.Cell;
50  import org.apache.hadoop.hbase.HConstants;
51  import org.apache.hadoop.hbase.KeyValue;
52  import org.apache.hadoop.hbase.KeyValue.KVComparator;
53  import org.apache.hadoop.hbase.fs.HFileSystem;
54  import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
55  import org.apache.hadoop.hbase.io.compress.Compression;
56  import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
57  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
58  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
59  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
60  import org.apache.hadoop.hbase.protobuf.generated.HFileProtos;
61  import org.apache.hadoop.hbase.util.BloomFilterWriter;
62  import org.apache.hadoop.hbase.util.ByteStringer;
63  import org.apache.hadoop.hbase.util.Bytes;
64  import org.apache.hadoop.hbase.util.ChecksumType;
65  import org.apache.hadoop.hbase.util.Counter;
66  import org.apache.hadoop.hbase.util.FSUtils;
67  import org.apache.hadoop.io.Writable;
68  
69  import com.google.common.base.Preconditions;
70  
71  /**
72   * File format for hbase.
73   * A file of sorted key/value pairs. Both keys and values are byte arrays.
74   * <p>
75   * The memory footprint of a HFile includes the following (below is taken from the
76   * <a
77   * href=https://issues.apache.org/jira/browse/HADOOP-3315>TFile</a> documentation
78   * but applies also to HFile):
79   * <ul>
80   * <li>Some constant overhead of reading or writing a compressed block.
81   * <ul>
82   * <li>Each compressed block requires one compression/decompression codec for
83   * I/O.
84   * <li>Temporary space to buffer the key.
85   * <li>Temporary space to buffer the value.
86   * </ul>
87   * <li>HFile index, which is proportional to the total number of Data Blocks.
88   * The total amount of memory needed to hold the index can be estimated as
89   * (56+AvgKeySize)*NumBlocks.
90   * </ul>
91   * Suggestions on performance optimization.
92   * <ul>
93   * <li>Minimum block size. We recommend a setting of minimum block size between
94   * 8KB to 1MB for general usage. Larger block size is preferred if files are
95   * primarily for sequential access. However, it would lead to inefficient random
96   * access (because there are more data to decompress). Smaller blocks are good
97   * for random access, but require more memory to hold the block index, and may
98   * be slower to create (because we must flush the compressor stream at the
99   * conclusion of each data block, which leads to an FS I/O flush). Further, due
100  * to the internal caching in Compression codec, the smallest possible block
101  * size would be around 20KB-30KB.
102  * <li>The current implementation does not offer true multi-threading for
103  * reading. The implementation uses FSDataInputStream seek()+read(), which is
104  * shown to be much faster than positioned-read call in single thread mode.
105  * However, it also means that if multiple threads attempt to access the same
106  * HFile (using multiple scanners) simultaneously, the actual I/O is carried out
107  * sequentially even if they access different DFS blocks (Reexamine! pread seems
108  * to be 10% faster than seek+read in my testing -- stack).
109  * <li>Compression codec. Use "none" if the data is not very compressable (by
110  * compressable, I mean a compression ratio at least 2:1). Generally, use "lzo"
111  * as the starting point for experimenting. "gz" overs slightly better
112  * compression ratio over "lzo" but requires 4x CPU to compress and 2x CPU to
113  * decompress, comparing to "lzo".
114  * </ul>
115  *
116  * For more on the background behind HFile, see <a
117  * href=https://issues.apache.org/jira/browse/HBASE-61>HBASE-61</a>.
118  * <p>
119  * File is made of data blocks followed by meta data blocks (if any), a fileinfo
120  * block, data block index, meta data block index, and a fixed size trailer
121  * which records the offsets at which file changes content type.
122  * <pre>&lt;data blocks>&lt;meta blocks>&lt;fileinfo>&lt;data index>&lt;meta index>&lt;trailer></pre>
123  * Each block has a bit of magic at its start.  Block are comprised of
124  * key/values.  In data blocks, they are both byte arrays.  Metadata blocks are
125  * a String key and a byte array value.  An empty file looks like this:
126  * <pre>&lt;fileinfo>&lt;trailer></pre>.  That is, there are not data nor meta
127  * blocks present.
128  * <p>
129  * TODO: Do scanners need to be able to take a start and end row?
130  * TODO: Should BlockIndex know the name of its file?  Should it have a Path
131  * that points at its file say for the case where an index lives apart from
132  * an HFile instance?
133  */
134 @InterfaceAudience.Private
135 public class HFile {
136   static final Log LOG = LogFactory.getLog(HFile.class);
137 
138   /**
139    * Maximum length of key in HFile.
140    */
141   public final static int MAXIMUM_KEY_LENGTH = Integer.MAX_VALUE;
142 
143   /**
144    * Default compression: none.
145    */
146   public final static Compression.Algorithm DEFAULT_COMPRESSION_ALGORITHM =
147     Compression.Algorithm.NONE;
148 
149   /** Minimum supported HFile format version */
150   public static final int MIN_FORMAT_VERSION = 2;
151 
152   /** Maximum supported HFile format version
153    */
154   public static final int MAX_FORMAT_VERSION = 3;
155 
156   /**
157    * Minimum HFile format version with support for persisting cell tags
158    */
159   public static final int MIN_FORMAT_VERSION_WITH_TAGS = 3;
160 
161   /** Default compression name: none. */
162   public final static String DEFAULT_COMPRESSION =
163     DEFAULT_COMPRESSION_ALGORITHM.getName();
164 
165   /** Meta data block name for bloom filter bits. */
166   public static final String BLOOM_FILTER_DATA_KEY = "BLOOM_FILTER_DATA";
167 
168   /**
169    * We assume that HFile path ends with
170    * ROOT_DIR/TABLE_NAME/REGION_NAME/CF_NAME/HFILE, so it has at least this
171    * many levels of nesting. This is needed for identifying table and CF name
172    * from an HFile path.
173    */
174   public final static int MIN_NUM_HFILE_PATH_LEVELS = 5;
175 
176   /**
177    * The number of bytes per checksum.
178    */
179   public static final int DEFAULT_BYTES_PER_CHECKSUM = 16 * 1024;
180   // TODO: This define is done in three places.  Fix.
181   public static final ChecksumType DEFAULT_CHECKSUM_TYPE = ChecksumType.CRC32;
182 
183   // For measuring number of checksum failures
184   static final Counter checksumFailures = new Counter();
185 
186   // for test purpose
187   public static final Counter dataBlockReadCnt = new Counter();
188 
189   /**
190    * Number of checksum verification failures. It also
191    * clears the counter.
192    */
193   public static final long getChecksumFailuresCount() {
194     long count = checksumFailures.get();
195     checksumFailures.set(0);
196     return count;
197   }
198 
199   /** API required to write an {@link HFile} */
200   public interface Writer extends Closeable {
201 
202     /** Add an element to the file info map. */
203     void appendFileInfo(byte[] key, byte[] value) throws IOException;
204 
205     void append(Cell cell) throws IOException;
206 
207     /** @return the path to this {@link HFile} */
208     Path getPath();
209 
210     /**
211      * Adds an inline block writer such as a multi-level block index writer or
212      * a compound Bloom filter writer.
213      */
214     void addInlineBlockWriter(InlineBlockWriter bloomWriter);
215 
216     // The below three methods take Writables.  We'd like to undo Writables but undoing the below would be pretty
217     // painful.  Could take a byte [] or a Message but we want to be backward compatible around hfiles so would need
218     // to map between Message and Writable or byte [] and current Writable serialization.  This would be a bit of work
219     // to little gain.  Thats my thinking at moment.  St.Ack 20121129
220 
221     void appendMetaBlock(String bloomFilterMetaKey, Writable metaWriter);
222 
223     /**
224      * Store general Bloom filter in the file. This does not deal with Bloom filter
225      * internals but is necessary, since Bloom filters are stored differently
226      * in HFile version 1 and version 2.
227      */
228     void addGeneralBloomFilter(BloomFilterWriter bfw);
229 
230     /**
231      * Store delete family Bloom filter in the file, which is only supported in
232      * HFile V2.
233      */
234     void addDeleteFamilyBloomFilter(BloomFilterWriter bfw) throws IOException;
235 
236     /**
237      * Return the file context for the HFile this writer belongs to
238      */
239     HFileContext getFileContext();
240   }
241 
242   /**
243    * This variety of ways to construct writers is used throughout the code, and
244    * we want to be able to swap writer implementations.
245    */
246   public static abstract class WriterFactory {
247     protected final Configuration conf;
248     protected final CacheConfig cacheConf;
249     protected FileSystem fs;
250     protected Path path;
251     protected FSDataOutputStream ostream;
252     protected KVComparator comparator = KeyValue.COMPARATOR;
253     protected InetSocketAddress[] favoredNodes;
254     private HFileContext fileContext;
255 
256     WriterFactory(Configuration conf, CacheConfig cacheConf) {
257       this.conf = conf;
258       this.cacheConf = cacheConf;
259     }
260 
261     public WriterFactory withPath(FileSystem fs, Path path) {
262       Preconditions.checkNotNull(fs);
263       Preconditions.checkNotNull(path);
264       this.fs = fs;
265       this.path = path;
266       return this;
267     }
268 
269     public WriterFactory withOutputStream(FSDataOutputStream ostream) {
270       Preconditions.checkNotNull(ostream);
271       this.ostream = ostream;
272       return this;
273     }
274 
275     public WriterFactory withComparator(KVComparator comparator) {
276       Preconditions.checkNotNull(comparator);
277       this.comparator = comparator;
278       return this;
279     }
280 
281     public WriterFactory withFavoredNodes(InetSocketAddress[] favoredNodes) {
282       // Deliberately not checking for null here.
283       this.favoredNodes = favoredNodes;
284       return this;
285     }
286 
287     public WriterFactory withFileContext(HFileContext fileContext) {
288       this.fileContext = fileContext;
289       return this;
290     }
291 
292     public Writer create() throws IOException {
293       if ((path != null ? 1 : 0) + (ostream != null ? 1 : 0) != 1) {
294         throw new AssertionError("Please specify exactly one of " +
295             "filesystem/path or path");
296       }
297       if (path != null) {
298         ostream = AbstractHFileWriter.createOutputStream(conf, fs, path, favoredNodes);
299       }
300       return createWriter(fs, path, ostream,
301                    comparator, fileContext);
302     }
303 
304     protected abstract Writer createWriter(FileSystem fs, Path path, FSDataOutputStream ostream,
305         KVComparator comparator, HFileContext fileContext) throws IOException;
306   }
307 
308   /** The configuration key for HFile version to use for new files */
309   public static final String FORMAT_VERSION_KEY = "hfile.format.version";
310 
311   public static int getFormatVersion(Configuration conf) {
312     int version = conf.getInt(FORMAT_VERSION_KEY, MAX_FORMAT_VERSION);
313     checkFormatVersion(version);
314     return version;
315   }
316 
317   /**
318    * Returns the factory to be used to create {@link HFile} writers.
319    * Disables block cache access for all writers created through the
320    * returned factory.
321    */
322   public static final WriterFactory getWriterFactoryNoCache(Configuration
323        conf) {
324     Configuration tempConf = new Configuration(conf);
325     tempConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.0f);
326     return HFile.getWriterFactory(conf, new CacheConfig(tempConf));
327   }
328 
329   /**
330    * Returns the factory to be used to create {@link HFile} writers
331    */
332   public static final WriterFactory getWriterFactory(Configuration conf,
333       CacheConfig cacheConf) {
334     int version = getFormatVersion(conf);
335     switch (version) {
336     case 2:
337       return new HFileWriterV2.WriterFactoryV2(conf, cacheConf);
338     case 3:
339       return new HFileWriterV3.WriterFactoryV3(conf, cacheConf);
340     default:
341       throw new IllegalArgumentException("Cannot create writer for HFile " +
342           "format version " + version);
343     }
344   }
345 
346   /**
347    * An abstraction used by the block index.
348    * Implementations will check cache for any asked-for block and return cached block if found.
349    * Otherwise, after reading from fs, will try and put block into cache before returning.
350    */
351   public interface CachingBlockReader {
352     /**
353      * Read in a file block.
354      * @param offset offset to read.
355      * @param onDiskBlockSize size of the block
356      * @param cacheBlock
357      * @param pread
358      * @param isCompaction is this block being read as part of a compaction
359      * @param expectedBlockType the block type we are expecting to read with this read operation,
360      *  or null to read whatever block type is available and avoid checking (that might reduce
361      *  caching efficiency of encoded data blocks)
362      * @param expectedDataBlockEncoding the data block encoding the caller is expecting data blocks
363      *  to be in, or null to not perform this check and return the block irrespective of the
364      *  encoding. This check only applies to data blocks and can be set to null when the caller is
365      *  expecting to read a non-data block and has set expectedBlockType accordingly.
366      * @return Block wrapped in a ByteBuffer.
367      * @throws IOException
368      */
369     HFileBlock readBlock(long offset, long onDiskBlockSize,
370         boolean cacheBlock, final boolean pread, final boolean isCompaction,
371         final boolean updateCacheMetrics, BlockType expectedBlockType,
372         DataBlockEncoding expectedDataBlockEncoding)
373         throws IOException;
374   }
375 
376   /** An interface used by clients to open and iterate an {@link HFile}. */
377   public interface Reader extends Closeable, CachingBlockReader {
378     /**
379      * Returns this reader's "name". Usually the last component of the path.
380      * Needs to be constant as the file is being moved to support caching on
381      * write.
382      */
383     String getName();
384 
385     KVComparator getComparator();
386 
387     HFileScanner getScanner(boolean cacheBlocks, final boolean pread, final boolean isCompaction);
388 
389     ByteBuffer getMetaBlock(String metaBlockName, boolean cacheBlock) throws IOException;
390 
391     Map<byte[], byte[]> loadFileInfo() throws IOException;
392 
393     byte[] getLastKey();
394 
395     byte[] midkey() throws IOException;
396 
397     long length();
398 
399     long getEntries();
400 
401     byte[] getFirstKey();
402 
403     long indexSize();
404 
405     byte[] getFirstRowKey();
406 
407     byte[] getLastRowKey();
408 
409     FixedFileTrailer getTrailer();
410 
411     HFileBlockIndex.BlockIndexReader getDataBlockIndexReader();
412 
413     HFileScanner getScanner(boolean cacheBlocks, boolean pread);
414 
415     Compression.Algorithm getCompressionAlgorithm();
416 
417     /**
418      * Retrieves general Bloom filter metadata as appropriate for each
419      * {@link HFile} version.
420      * Knows nothing about how that metadata is structured.
421      */
422     DataInput getGeneralBloomFilterMetadata() throws IOException;
423 
424     /**
425      * Retrieves delete family Bloom filter metadata as appropriate for each
426      * {@link HFile}  version.
427      * Knows nothing about how that metadata is structured.
428      */
429     DataInput getDeleteBloomFilterMetadata() throws IOException;
430 
431     Path getPath();
432 
433     /** Close method with optional evictOnClose */
434     void close(boolean evictOnClose) throws IOException;
435 
436     DataBlockEncoding getDataBlockEncoding();
437 
438     boolean hasMVCCInfo();
439 
440     /**
441      * Return the file context of the HFile this reader belongs to
442      */
443     HFileContext getFileContext();
444     
445     boolean isPrimaryReplicaReader();
446 
447     void setPrimaryReplicaReader(boolean isPrimaryReplicaReader);
448   }
449 
450   /**
451    * Method returns the reader given the specified arguments.
452    * TODO This is a bad abstraction.  See HBASE-6635.
453    *
454    * @param path hfile's path
455    * @param fsdis stream of path's file
456    * @param size max size of the trailer.
457    * @param cacheConf Cache configuation values, cannot be null.
458    * @param hfs
459    * @return an appropriate instance of HFileReader
460    * @throws IOException If file is invalid, will throw CorruptHFileException flavored IOException
461    */
462   private static Reader pickReaderVersion(Path path, FSDataInputStreamWrapper fsdis,
463       long size, CacheConfig cacheConf, HFileSystem hfs, Configuration conf) throws IOException {
464     FixedFileTrailer trailer = null;
465     try {
466       boolean isHBaseChecksum = fsdis.shouldUseHBaseChecksum();
467       assert !isHBaseChecksum; // Initially we must read with FS checksum.
468       trailer = FixedFileTrailer.readFromStream(fsdis.getStream(isHBaseChecksum), size);
469       switch (trailer.getMajorVersion()) {
470       case 2:
471         return new HFileReaderV2(path, trailer, fsdis, size, cacheConf, hfs, conf);
472       case 3 :
473         return new HFileReaderV3(path, trailer, fsdis, size, cacheConf, hfs, conf);
474       default:
475         throw new IllegalArgumentException("Invalid HFile version " + trailer.getMajorVersion());
476       }
477     } catch (Throwable t) {
478       try {
479         fsdis.close();
480       } catch (Throwable t2) {
481         LOG.warn("Error closing fsdis FSDataInputStreamWrapper", t2);
482       }
483       throw new CorruptHFileException("Problem reading HFile Trailer from file " + path, t);
484     }
485   }
486 
487   /**
488    * @param fs A file system
489    * @param path Path to HFile
490    * @param fsdis a stream of path's file
491    * @param size max size of the trailer.
492    * @param cacheConf Cache configuration for hfile's contents
493    * @param conf Configuration
494    * @return A version specific Hfile Reader
495    * @throws IOException If file is invalid, will throw CorruptHFileException flavored IOException
496    */
497   public static Reader createReader(FileSystem fs, Path path,
498       FSDataInputStreamWrapper fsdis, long size, CacheConfig cacheConf, Configuration conf)
499       throws IOException {
500     HFileSystem hfs = null;
501 
502     // If the fs is not an instance of HFileSystem, then create an
503     // instance of HFileSystem that wraps over the specified fs.
504     // In this case, we will not be able to avoid checksumming inside
505     // the filesystem.
506     if (!(fs instanceof HFileSystem)) {
507       hfs = new HFileSystem(fs);
508     } else {
509       hfs = (HFileSystem)fs;
510     }
511     return pickReaderVersion(path, fsdis, size, cacheConf, hfs, conf);
512   }
513 
514   /**
515    *
516    * @param fs filesystem
517    * @param path Path to file to read
518    * @param cacheConf This must not be null.  @see {@link org.apache.hadoop.hbase.io.hfile.CacheConfig#CacheConfig(Configuration)}
519    * @return an active Reader instance
520    * @throws IOException Will throw a CorruptHFileException (DoNotRetryIOException subtype) if hfile is corrupt/invalid.
521    */
522   public static Reader createReader(
523       FileSystem fs, Path path, CacheConfig cacheConf, Configuration conf) throws IOException {
524     Preconditions.checkNotNull(cacheConf, "Cannot create Reader with null CacheConf");
525     FSDataInputStreamWrapper stream = new FSDataInputStreamWrapper(fs, path);
526     return pickReaderVersion(path, stream, fs.getFileStatus(path).getLen(),
527       cacheConf, stream.getHfs(), conf);
528   }
529 
530   /**
531    * This factory method is used only by unit tests
532    */
533   static Reader createReaderFromStream(Path path,
534       FSDataInputStream fsdis, long size, CacheConfig cacheConf, Configuration conf)
535       throws IOException {
536     FSDataInputStreamWrapper wrapper = new FSDataInputStreamWrapper(fsdis);
537     return pickReaderVersion(path, wrapper, size, cacheConf, null, conf);
538   }
539 
540   /**
541    * Returns true if the specified file has a valid HFile Trailer.
542    * @param fs filesystem
543    * @param path Path to file to verify
544    * @return true if the file has a valid HFile Trailer, otherwise false
545    * @throws IOException if failed to read from the underlying stream
546    */
547   public static boolean isHFileFormat(final FileSystem fs, final Path path) throws IOException {
548     return isHFileFormat(fs, fs.getFileStatus(path));
549   }
550 
551   /**
552    * Returns true if the specified file has a valid HFile Trailer.
553    * @param fs filesystem
554    * @param fileStatus the file to verify
555    * @return true if the file has a valid HFile Trailer, otherwise false
556    * @throws IOException if failed to read from the underlying stream
557    */
558   public static boolean isHFileFormat(final FileSystem fs, final FileStatus fileStatus)
559       throws IOException {
560     final Path path = fileStatus.getPath();
561     final long size = fileStatus.getLen();
562     FSDataInputStreamWrapper fsdis = new FSDataInputStreamWrapper(fs, path);
563     try {
564       boolean isHBaseChecksum = fsdis.shouldUseHBaseChecksum();
565       assert !isHBaseChecksum; // Initially we must read with FS checksum.
566       FixedFileTrailer.readFromStream(fsdis.getStream(isHBaseChecksum), size);
567       return true;
568     } catch (IllegalArgumentException e) {
569       return false;
570     } catch (IOException e) {
571       throw e;
572     } finally {
573       try {
574         fsdis.close();
575       } catch (Throwable t) {
576         LOG.warn("Error closing fsdis FSDataInputStreamWrapper: " + path, t);
577       }
578     }
579   }
580 
581   /**
582    * Metadata for this file. Conjured by the writer. Read in by the reader.
583    */
584   public static class FileInfo implements SortedMap<byte[], byte[]> {
585     static final String RESERVED_PREFIX = "hfile.";
586     static final byte[] RESERVED_PREFIX_BYTES = Bytes.toBytes(RESERVED_PREFIX);
587     static final byte [] LASTKEY = Bytes.toBytes(RESERVED_PREFIX + "LASTKEY");
588     static final byte [] AVG_KEY_LEN = Bytes.toBytes(RESERVED_PREFIX + "AVG_KEY_LEN");
589     static final byte [] AVG_VALUE_LEN = Bytes.toBytes(RESERVED_PREFIX + "AVG_VALUE_LEN");
590     static final byte [] CREATE_TIME_TS = Bytes.toBytes(RESERVED_PREFIX + "CREATE_TIME_TS");
591     static final byte [] COMPARATOR = Bytes.toBytes(RESERVED_PREFIX + "COMPARATOR");
592     static final byte [] TAGS_COMPRESSED = Bytes.toBytes(RESERVED_PREFIX + "TAGS_COMPRESSED");
593     public static final byte [] MAX_TAGS_LEN = Bytes.toBytes(RESERVED_PREFIX + "MAX_TAGS_LEN");
594     private final SortedMap<byte [], byte []> map = new TreeMap<byte [], byte []>(Bytes.BYTES_COMPARATOR);
595 
596     public FileInfo() {
597       super();
598     }
599 
600     /**
601      * Append the given key/value pair to the file info, optionally checking the
602      * key prefix.
603      *
604      * @param k key to add
605      * @param v value to add
606      * @param checkPrefix whether to check that the provided key does not start
607      *          with the reserved prefix
608      * @return this file info object
609      * @throws IOException if the key or value is invalid
610      */
611     public FileInfo append(final byte[] k, final byte[] v,
612         final boolean checkPrefix) throws IOException {
613       if (k == null || v == null) {
614         throw new NullPointerException("Key nor value may be null");
615       }
616       if (checkPrefix && isReservedFileInfoKey(k)) {
617         throw new IOException("Keys with a " + FileInfo.RESERVED_PREFIX
618             + " are reserved");
619       }
620       put(k, v);
621       return this;
622     }
623 
624     public void clear() {
625       this.map.clear();
626     }
627 
628     public Comparator<? super byte[]> comparator() {
629       return map.comparator();
630     }
631 
632     public boolean containsKey(Object key) {
633       return map.containsKey(key);
634     }
635 
636     public boolean containsValue(Object value) {
637       return map.containsValue(value);
638     }
639 
640     public Set<java.util.Map.Entry<byte[], byte[]>> entrySet() {
641       return map.entrySet();
642     }
643 
644     public boolean equals(Object o) {
645       return map.equals(o);
646     }
647 
648     public byte[] firstKey() {
649       return map.firstKey();
650     }
651 
652     public byte[] get(Object key) {
653       return map.get(key);
654     }
655 
656     public int hashCode() {
657       return map.hashCode();
658     }
659 
660     public SortedMap<byte[], byte[]> headMap(byte[] toKey) {
661       return this.map.headMap(toKey);
662     }
663 
664     public boolean isEmpty() {
665       return map.isEmpty();
666     }
667 
668     public Set<byte[]> keySet() {
669       return map.keySet();
670     }
671 
672     public byte[] lastKey() {
673       return map.lastKey();
674     }
675 
676     public byte[] put(byte[] key, byte[] value) {
677       return this.map.put(key, value);
678     }
679 
680     public void putAll(Map<? extends byte[], ? extends byte[]> m) {
681       this.map.putAll(m);
682     }
683 
684     public byte[] remove(Object key) {
685       return this.map.remove(key);
686     }
687 
688     public int size() {
689       return map.size();
690     }
691 
692     public SortedMap<byte[], byte[]> subMap(byte[] fromKey, byte[] toKey) {
693       return this.map.subMap(fromKey, toKey);
694     }
695 
696     public SortedMap<byte[], byte[]> tailMap(byte[] fromKey) {
697       return this.map.tailMap(fromKey);
698     }
699 
700     public Collection<byte[]> values() {
701       return map.values();
702     }
703 
704     /**
705      * Write out this instance on the passed in <code>out</code> stream.
706      * We write it as a protobuf.
707      * @param out
708      * @throws IOException
709      * @see #read(DataInputStream)
710      */
711     void write(final DataOutputStream out) throws IOException {
712       HFileProtos.FileInfoProto.Builder builder = HFileProtos.FileInfoProto.newBuilder();
713       for (Map.Entry<byte [], byte[]> e: this.map.entrySet()) {
714         HBaseProtos.BytesBytesPair.Builder bbpBuilder = HBaseProtos.BytesBytesPair.newBuilder();
715         bbpBuilder.setFirst(ByteStringer.wrap(e.getKey()));
716         bbpBuilder.setSecond(ByteStringer.wrap(e.getValue()));
717         builder.addMapEntry(bbpBuilder.build());
718       }
719       out.write(ProtobufUtil.PB_MAGIC);
720       builder.build().writeDelimitedTo(out);
721     }
722 
723     /**
724      * Populate this instance with what we find on the passed in <code>in</code> stream.
725      * Can deserialize protobuf of old Writables format.
726      * @param in
727      * @throws IOException
728      * @see #write(DataOutputStream)
729      */
730     void read(final DataInputStream in) throws IOException {
731       // This code is tested over in TestHFileReaderV1 where we read an old hfile w/ this new code.
732       int pblen = ProtobufUtil.lengthOfPBMagic();
733       byte [] pbuf = new byte[pblen];
734       if (in.markSupported()) in.mark(pblen);
735       int read = in.read(pbuf);
736       if (read != pblen) throw new IOException("read=" + read + ", wanted=" + pblen);
737       if (ProtobufUtil.isPBMagicPrefix(pbuf)) {
738         parsePB(HFileProtos.FileInfoProto.parseDelimitedFrom(in));
739       } else {
740         if (in.markSupported()) {
741           in.reset();
742           parseWritable(in);
743         } else {
744           // We cannot use BufferedInputStream, it consumes more than we read from the underlying IS
745           ByteArrayInputStream bais = new ByteArrayInputStream(pbuf);
746           SequenceInputStream sis = new SequenceInputStream(bais, in); // Concatenate input streams
747           // TODO: Am I leaking anything here wrapping the passed in stream?  We are not calling close on the wrapped
748           // streams but they should be let go after we leave this context?  I see that we keep a reference to the
749           // passed in inputstream but since we no longer have a reference to this after we leave, we should be ok.
750           parseWritable(new DataInputStream(sis));
751         }
752       }
753     }
754 
755     /** Now parse the old Writable format.  It was a list of Map entries.  Each map entry was a key and a value of
756      * a byte [].  The old map format had a byte before each entry that held a code which was short for the key or
757      * value type.  We know it was a byte [] so in below we just read and dump it.
758      * @throws IOException
759      */
760     void parseWritable(final DataInputStream in) throws IOException {
761       // First clear the map.  Otherwise we will just accumulate entries every time this method is called.
762       this.map.clear();
763       // Read the number of entries in the map
764       int entries = in.readInt();
765       // Then read each key/value pair
766       for (int i = 0; i < entries; i++) {
767         byte [] key = Bytes.readByteArray(in);
768         // We used to read a byte that encoded the class type.  Read and ignore it because it is always byte [] in hfile
769         in.readByte();
770         byte [] value = Bytes.readByteArray(in);
771         this.map.put(key, value);
772       }
773     }
774 
775     /**
776      * Fill our map with content of the pb we read off disk
777      * @param fip protobuf message to read
778      */
779     void parsePB(final HFileProtos.FileInfoProto fip) {
780       this.map.clear();
781       for (BytesBytesPair pair: fip.getMapEntryList()) {
782         this.map.put(pair.getFirst().toByteArray(), pair.getSecond().toByteArray());
783       }
784     }
785   }
786 
787   /** Return true if the given file info key is reserved for internal use. */
788   public static boolean isReservedFileInfoKey(byte[] key) {
789     return Bytes.startsWith(key, FileInfo.RESERVED_PREFIX_BYTES);
790   }
791 
792   /**
793    * Get names of supported compression algorithms. The names are acceptable by
794    * HFile.Writer.
795    *
796    * @return Array of strings, each represents a supported compression
797    *         algorithm. Currently, the following compression algorithms are
798    *         supported.
799    *         <ul>
800    *         <li>"none" - No compression.
801    *         <li>"gz" - GZIP compression.
802    *         </ul>
803    */
804   public static String[] getSupportedCompressionAlgorithms() {
805     return Compression.getSupportedAlgorithms();
806   }
807 
808   // Utility methods.
809   /*
810    * @param l Long to convert to an int.
811    * @return <code>l</code> cast as an int.
812    */
813   static int longToInt(final long l) {
814     // Expecting the size() of a block not exceeding 4GB. Assuming the
815     // size() will wrap to negative integer if it exceeds 2GB (From tfile).
816     return (int)(l & 0x00000000ffffffffL);
817   }
818 
819   /**
820    * Returns all HFiles belonging to the given region directory. Could return an
821    * empty list.
822    *
823    * @param fs  The file system reference.
824    * @param regionDir  The region directory to scan.
825    * @return The list of files found.
826    * @throws IOException When scanning the files fails.
827    */
828   static List<Path> getStoreFiles(FileSystem fs, Path regionDir)
829       throws IOException {
830     List<Path> regionHFiles = new ArrayList<Path>();
831     PathFilter dirFilter = new FSUtils.DirFilter(fs);
832     FileStatus[] familyDirs = fs.listStatus(regionDir, dirFilter);
833     for(FileStatus dir : familyDirs) {
834       FileStatus[] files = fs.listStatus(dir.getPath());
835       for (FileStatus file : files) {
836         if (!file.isDirectory() &&
837             (!file.getPath().toString().contains(HConstants.HREGION_OLDLOGDIR_NAME)) &&
838             (!file.getPath().toString().contains(HConstants.RECOVERED_EDITS_DIR))) {
839           regionHFiles.add(file.getPath());
840         }
841       }
842     }
843     return regionHFiles;
844   }
845 
846   /**
847    * Checks the given {@link HFile} format version, and throws an exception if
848    * invalid. Note that if the version number comes from an input file and has
849    * not been verified, the caller needs to re-throw an {@link IOException} to
850    * indicate that this is not a software error, but corrupted input.
851    *
852    * @param version an HFile version
853    * @throws IllegalArgumentException if the version is invalid
854    */
855   public static void checkFormatVersion(int version)
856       throws IllegalArgumentException {
857     if (version < MIN_FORMAT_VERSION || version > MAX_FORMAT_VERSION) {
858       throw new IllegalArgumentException("Invalid HFile version: " + version
859           + " (expected to be " + "between " + MIN_FORMAT_VERSION + " and "
860           + MAX_FORMAT_VERSION + ")");
861     }
862   }
863 
864   public static void main(String[] args) throws Exception {
865     // delegate to preserve old behavior
866     HFilePrettyPrinter.main(args);
867   }
868 }