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.List;
28  import java.util.Map;
29  import java.util.concurrent.atomic.AtomicLong;
30  
31  import org.apache.commons.logging.Log;
32  import org.apache.commons.logging.LogFactory;
33  import org.apache.hadoop.conf.Configuration;
34  import org.apache.hadoop.fs.FSDataInputStream;
35  import org.apache.hadoop.fs.FSDataOutputStream;
36  import org.apache.hadoop.fs.FileStatus;
37  import org.apache.hadoop.fs.FileSystem;
38  import org.apache.hadoop.fs.Path;
39  import org.apache.hadoop.fs.PathFilter;
40  import org.apache.hadoop.hbase.KeyValue;
41  import org.apache.hadoop.hbase.KeyValue.KeyComparator;
42  import org.apache.hadoop.hbase.io.HbaseMapWritable;
43  import org.apache.hadoop.hbase.util.BloomFilterWriter;
44  import org.apache.hadoop.hbase.util.Bytes;
45  import org.apache.hadoop.hbase.util.FSUtils;
46  import org.apache.hadoop.io.RawComparator;
47  import org.apache.hadoop.io.Writable;
48  
49  /**
50   * File format for hbase.
51   * A file of sorted key/value pairs. Both keys and values are byte arrays.
52   * <p>
53   * The memory footprint of a HFile includes the following (below is taken from the
54   * <a
55   * href=https://issues.apache.org/jira/browse/HADOOP-3315>TFile</a> documentation
56   * but applies also to HFile):
57   * <ul>
58   * <li>Some constant overhead of reading or writing a compressed block.
59   * <ul>
60   * <li>Each compressed block requires one compression/decompression codec for
61   * I/O.
62   * <li>Temporary space to buffer the key.
63   * <li>Temporary space to buffer the value.
64   * </ul>
65   * <li>HFile index, which is proportional to the total number of Data Blocks.
66   * The total amount of memory needed to hold the index can be estimated as
67   * (56+AvgKeySize)*NumBlocks.
68   * </ul>
69   * Suggestions on performance optimization.
70   * <ul>
71   * <li>Minimum block size. We recommend a setting of minimum block size between
72   * 8KB to 1MB for general usage. Larger block size is preferred if files are
73   * primarily for sequential access. However, it would lead to inefficient random
74   * access (because there are more data to decompress). Smaller blocks are good
75   * for random access, but require more memory to hold the block index, and may
76   * be slower to create (because we must flush the compressor stream at the
77   * conclusion of each data block, which leads to an FS I/O flush). Further, due
78   * to the internal caching in Compression codec, the smallest possible block
79   * size would be around 20KB-30KB.
80   * <li>The current implementation does not offer true multi-threading for
81   * reading. The implementation uses FSDataInputStream seek()+read(), which is
82   * shown to be much faster than positioned-read call in single thread mode.
83   * However, it also means that if multiple threads attempt to access the same
84   * HFile (using multiple scanners) simultaneously, the actual I/O is carried out
85   * sequentially even if they access different DFS blocks (Reexamine! pread seems
86   * to be 10% faster than seek+read in my testing -- stack).
87   * <li>Compression codec. Use "none" if the data is not very compressable (by
88   * compressable, I mean a compression ratio at least 2:1). Generally, use "lzo"
89   * as the starting point for experimenting. "gz" overs slightly better
90   * compression ratio over "lzo" but requires 4x CPU to compress and 2x CPU to
91   * decompress, comparing to "lzo".
92   * </ul>
93   *
94   * For more on the background behind HFile, see <a
95   * href=https://issues.apache.org/jira/browse/HBASE-61>HBASE-61</a>.
96   * <p>
97   * File is made of data blocks followed by meta data blocks (if any), a fileinfo
98   * block, data block index, meta data block index, and a fixed size trailer
99   * which records the offsets at which file changes content type.
100  * <pre>&lt;data blocks>&lt;meta blocks>&lt;fileinfo>&lt;data index>&lt;meta index>&lt;trailer></pre>
101  * Each block has a bit of magic at its start.  Block are comprised of
102  * key/values.  In data blocks, they are both byte arrays.  Metadata blocks are
103  * a String key and a byte array value.  An empty file looks like this:
104  * <pre>&lt;fileinfo>&lt;trailer></pre>.  That is, there are not data nor meta
105  * blocks present.
106  * <p>
107  * TODO: Do scanners need to be able to take a start and end row?
108  * TODO: Should BlockIndex know the name of its file?  Should it have a Path
109  * that points at its file say for the case where an index lives apart from
110  * an HFile instance?
111  */
112 public class HFile {
113   static final Log LOG = LogFactory.getLog(HFile.class);
114 
115   /**
116    * Maximum length of key in HFile.
117    */
118   public final static int MAXIMUM_KEY_LENGTH = Integer.MAX_VALUE;
119 
120   /**
121    * Default block size for an HFile.
122    */
123   public final static int DEFAULT_BLOCKSIZE = 64 * 1024;
124 
125   /**
126    * Default compression: none.
127    */
128   public final static Compression.Algorithm DEFAULT_COMPRESSION_ALGORITHM =
129     Compression.Algorithm.NONE;
130 
131   /** Minimum supported HFile format version */
132   public static final int MIN_FORMAT_VERSION = 1;
133 
134   /** Maximum supported HFile format version */
135   public static final int MAX_FORMAT_VERSION = 2;
136 
137   /** Default compression name: none. */
138   public final static String DEFAULT_COMPRESSION =
139     DEFAULT_COMPRESSION_ALGORITHM.getName();
140 
141   /** Separator between HFile name and offset in block cache key */
142   static final char CACHE_KEY_SEPARATOR = '_';
143 
144   // For measuring latency of "typical" reads and writes
145   static volatile AtomicLong readOps = new AtomicLong();
146   static volatile AtomicLong readTimeNano = new AtomicLong();
147   static volatile AtomicLong writeOps = new AtomicLong();
148   static volatile AtomicLong writeTimeNano = new AtomicLong();
149 
150   public static final long getReadOps() {
151     return readOps.getAndSet(0);
152   }
153 
154   public static final long getReadTimeMs() {
155     return readTimeNano.getAndSet(0) / 1000000;
156   }
157 
158   public static final long getWriteOps() {
159     return writeOps.getAndSet(0);
160   }
161 
162   public static final long getWriteTimeMs() {
163     return writeTimeNano.getAndSet(0) / 1000000;
164   }
165 
166   /** API required to write an {@link HFile} */
167   public interface Writer extends Closeable {
168 
169     /** Add an element to the file info map. */
170     void appendFileInfo(byte[] key, byte[] value) throws IOException;
171 
172     void append(KeyValue kv) throws IOException;
173 
174     void append(byte[] key, byte[] value) throws IOException;
175 
176     /** @return the path to this {@link HFile} */
177     Path getPath();
178 
179     void appendMetaBlock(String bloomFilterMetaKey, Writable metaWriter);
180 
181     /**
182      * Adds an inline block writer such as a multi-level block index writer or
183      * a compound Bloom filter writer.
184      */
185     void addInlineBlockWriter(InlineBlockWriter bloomWriter);
186 
187     /**
188      * Store Bloom filter in the file. This does not deal with Bloom filter
189      * internals but is necessary, since Bloom filters are stored differently
190      * in HFile version 1 and version 2.
191      */
192     void addBloomFilter(BloomFilterWriter bfw);
193   }
194 
195   /**
196    * This variety of ways to construct writers is used throughout the code, and
197    * we want to be able to swap writer implementations.
198    */
199   public static abstract class WriterFactory {
200     protected Configuration conf;
201     protected CacheConfig cacheConf;
202 
203     WriterFactory(Configuration conf, CacheConfig cacheConf) {
204       this.conf = conf;
205       this.cacheConf = cacheConf;
206     }
207 
208     public abstract Writer createWriter(FileSystem fs, Path path)
209         throws IOException;
210 
211     public abstract Writer createWriter(FileSystem fs, Path path,
212         int blockSize, Compression.Algorithm compress,
213         final KeyComparator comparator) throws IOException;
214 
215     public abstract Writer createWriter(FileSystem fs, Path path,
216         int blockSize, String compress,
217         final KeyComparator comparator) throws IOException;
218 
219     public abstract Writer createWriter(final FSDataOutputStream ostream,
220         final int blockSize, final String compress,
221         final KeyComparator comparator) throws IOException;
222 
223     public abstract Writer createWriter(final FSDataOutputStream ostream,
224         final int blockSize, final Compression.Algorithm compress,
225         final KeyComparator c) throws IOException;
226   }
227 
228   /** The configuration key for HFile version to use for new files */
229   public static final String FORMAT_VERSION_KEY = "hfile.format.version";
230 
231   public static int getFormatVersion(Configuration conf) {
232     int version = conf.getInt(FORMAT_VERSION_KEY, MAX_FORMAT_VERSION);
233     checkFormatVersion(version);
234     return version;
235   }
236 
237   /**
238    * Returns the factory to be used to create {@link HFile} writers.
239    */
240   public static final WriterFactory getWriterFactory(Configuration conf) {
241     return HFile.getWriterFactory(conf, new CacheConfig(conf));
242   }
243 
244   /**
245    * Returns the factory to be used to create {@link HFile} writers
246    */
247   public static final WriterFactory getWriterFactory(Configuration conf,
248       CacheConfig cacheConf) {
249     int version = getFormatVersion(conf);
250     switch (version) {
251     case 1:
252       return new HFileWriterV1.WriterFactoryV1(conf, cacheConf);
253     case 2:
254       return new HFileWriterV2.WriterFactoryV2(conf, cacheConf);
255     default:
256       throw new IllegalArgumentException("Cannot create writer for HFile " +
257           "format version " + version);
258     }
259   }
260 
261   /** An abstraction used by the block index */
262   public interface CachingBlockReader {
263     HFileBlock readBlock(long offset, long onDiskBlockSize,
264         boolean cacheBlock, final boolean pread, final boolean isCompaction)
265         throws IOException;
266   }
267 
268   /** An interface used by clients to open and iterate an {@link HFile}. */
269   public interface Reader extends Closeable, CachingBlockReader {
270     /**
271      * Returns this reader's "name". Usually the last component of the path.
272      * Needs to be constant as the file is being moved to support caching on
273      * write.
274      */
275     String getName();
276 
277     String getColumnFamilyName();
278 
279     RawComparator<byte []> getComparator();
280 
281     HFileScanner getScanner(boolean cacheBlocks,
282         final boolean pread, final boolean isCompaction);
283 
284     ByteBuffer getMetaBlock(String metaBlockName,
285         boolean cacheBlock) throws IOException;
286 
287     Map<byte[], byte[]> loadFileInfo() throws IOException;
288 
289     byte[] getLastKey();
290 
291     byte[] midkey() throws IOException;
292 
293     long length();
294 
295     long getEntries();
296 
297     byte[] getFirstKey();
298 
299     long indexSize();
300 
301     byte[] getFirstRowKey();
302 
303     byte[] getLastRowKey();
304 
305     FixedFileTrailer getTrailer();
306 
307     HFileBlockIndex.BlockIndexReader getDataBlockIndexReader();
308 
309     HFileScanner getScanner(boolean cacheBlocks, boolean pread);
310 
311     Compression.Algorithm getCompressionAlgorithm();
312 
313     /**
314      * Retrieves Bloom filter metadata as appropriate for each {@link HFile}
315      * version. Knows nothing about how that metadata is structured.
316      */
317     DataInput getBloomFilterMetadata() throws IOException;
318 
319     Path getPath();
320 
321     /** Close method with optional evictOnClose */
322     void close(boolean evictOnClose) throws IOException;
323   }
324 
325   private static Reader pickReaderVersion(Path path, FSDataInputStream fsdis,
326       long size, boolean closeIStream, CacheConfig cacheConf)
327   throws IOException {
328     FixedFileTrailer trailer = FixedFileTrailer.readFromStream(fsdis, size);
329     switch (trailer.getVersion()) {
330     case 1:
331       return new HFileReaderV1(path, trailer, fsdis, size, closeIStream,
332           cacheConf);
333     case 2:
334       return new HFileReaderV2(path, trailer, fsdis, size, closeIStream,
335           cacheConf);
336     default:
337       throw new IOException("Cannot instantiate reader for HFile version " +
338           trailer.getVersion());
339     }
340   }
341 
342   public static Reader createReader(FileSystem fs, Path path,
343       CacheConfig cacheConf) throws IOException {
344     return pickReaderVersion(path, fs.open(path),
345         fs.getFileStatus(path).getLen(), true, cacheConf);
346   }
347 
348   public static Reader createReader(Path path, FSDataInputStream fsdis,
349       long size, CacheConfig cacheConf)
350       throws IOException {
351     return pickReaderVersion(path, fsdis, size, false, cacheConf);
352   }
353 
354   /*
355    * Metadata for this file.  Conjured by the writer.  Read in by the reader.
356    */
357   static class FileInfo extends HbaseMapWritable<byte [], byte []> {
358     static final String RESERVED_PREFIX = "hfile.";
359     static final byte[] RESERVED_PREFIX_BYTES = Bytes.toBytes(RESERVED_PREFIX);
360     static final byte [] LASTKEY = Bytes.toBytes(RESERVED_PREFIX + "LASTKEY");
361     static final byte [] AVG_KEY_LEN =
362       Bytes.toBytes(RESERVED_PREFIX + "AVG_KEY_LEN");
363     static final byte [] AVG_VALUE_LEN =
364       Bytes.toBytes(RESERVED_PREFIX + "AVG_VALUE_LEN");
365     static final byte [] COMPARATOR =
366       Bytes.toBytes(RESERVED_PREFIX + "COMPARATOR");
367 
368     /**
369      * Append the given key/value pair to the file info, optionally checking the
370      * key prefix.
371      *
372      * @param k key to add
373      * @param v value to add
374      * @param checkPrefix whether to check that the provided key does not start
375      *          with the reserved prefix
376      * @return this file info object
377      * @throws IOException if the key or value is invalid
378      */
379     public FileInfo append(final byte[] k, final byte[] v,
380         final boolean checkPrefix) throws IOException {
381       if (k == null || v == null) {
382         throw new NullPointerException("Key nor value may be null");
383       }
384       if (checkPrefix && isReservedFileInfoKey(k)) {
385         throw new IOException("Keys with a " + FileInfo.RESERVED_PREFIX
386             + " are reserved");
387       }
388       put(k, v);
389       return this;
390     }
391 
392   }
393 
394   /** Return true if the given file info key is reserved for internal use. */
395   public static boolean isReservedFileInfoKey(byte[] key) {
396     return Bytes.startsWith(key, FileInfo.RESERVED_PREFIX_BYTES);
397   }
398 
399   /**
400    * Get names of supported compression algorithms. The names are acceptable by
401    * HFile.Writer.
402    *
403    * @return Array of strings, each represents a supported compression
404    *         algorithm. Currently, the following compression algorithms are
405    *         supported.
406    *         <ul>
407    *         <li>"none" - No compression.
408    *         <li>"gz" - GZIP compression.
409    *         </ul>
410    */
411   public static String[] getSupportedCompressionAlgorithms() {
412     return Compression.getSupportedAlgorithms();
413   }
414 
415   // Utility methods.
416   /*
417    * @param l Long to convert to an int.
418    * @return <code>l</code> cast as an int.
419    */
420   static int longToInt(final long l) {
421     // Expecting the size() of a block not exceeding 4GB. Assuming the
422     // size() will wrap to negative integer if it exceeds 2GB (From tfile).
423     return (int)(l & 0x00000000ffffffffL);
424   }
425 
426   /**
427    * Returns all files belonging to the given region directory. Could return an
428    * empty list.
429    *
430    * @param fs  The file system reference.
431    * @param regionDir  The region directory to scan.
432    * @return The list of files found.
433    * @throws IOException When scanning the files fails.
434    */
435   static List<Path> getStoreFiles(FileSystem fs, Path regionDir)
436       throws IOException {
437     List<Path> res = new ArrayList<Path>();
438     PathFilter dirFilter = new FSUtils.DirFilter(fs);
439     FileStatus[] familyDirs = fs.listStatus(regionDir, dirFilter);
440     for(FileStatus dir : familyDirs) {
441       FileStatus[] files = fs.listStatus(dir.getPath());
442       for (FileStatus file : files) {
443         if (!file.isDir()) {
444           res.add(file.getPath());
445         }
446       }
447     }
448     return res;
449   }
450 
451   public static void main(String[] args) throws IOException {
452     HFilePrettyPrinter prettyPrinter = new HFilePrettyPrinter();
453     System.exit(prettyPrinter.run(args));
454   }
455 
456   public static String getBlockCacheKey(String hfileName, long offset) {
457     return hfileName + CACHE_KEY_SEPARATOR + offset;
458   }
459 
460   /**
461    * Checks the given {@link HFile} format version, and throws an exception if
462    * invalid. Note that if the version number comes from an input file and has
463    * not been verified, the caller needs to re-throw an {@link IOException} to
464    * indicate that this is not a software error, but corrupted input.
465    *
466    * @param version an HFile version
467    * @throws IllegalArgumentException if the version is invalid
468    */
469   public static void checkFormatVersion(int version)
470       throws IllegalArgumentException {
471     if (version < MIN_FORMAT_VERSION || version > MAX_FORMAT_VERSION) {
472       throw new IllegalArgumentException("Invalid HFile version: " + version
473           + " (expected to be " + "between " + MIN_FORMAT_VERSION + " and "
474           + MAX_FORMAT_VERSION + ")");
475     }
476   }
477 
478 }