View Javadoc

1   /**
2    * Copyright 2010 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.regionserver;
21  
22  import org.apache.commons.logging.Log;
23  import org.apache.commons.logging.LogFactory;
24  import org.apache.hadoop.conf.Configuration;
25  import org.apache.hadoop.fs.FileSystem;
26  import org.apache.hadoop.fs.Path;
27  import org.apache.hadoop.hbase.KeyValue;
28  import org.apache.hadoop.hbase.KeyValue.KVComparator;
29  import org.apache.hadoop.hbase.client.Scan;
30  import org.apache.hadoop.hbase.io.HalfStoreFileReader;
31  import org.apache.hadoop.hbase.io.Reference;
32  import org.apache.hadoop.hbase.io.hfile.BlockCache;
33  import org.apache.hadoop.hbase.io.hfile.Compression;
34  import org.apache.hadoop.hbase.io.hfile.HFile;
35  import org.apache.hadoop.hbase.io.hfile.HFileScanner;
36  import org.apache.hadoop.hbase.io.hfile.LruBlockCache;
37  import org.apache.hadoop.hbase.util.BloomFilter;
38  import org.apache.hadoop.hbase.util.ByteBloomFilter;
39  import org.apache.hadoop.hbase.util.Bytes;
40  import org.apache.hadoop.hbase.util.Hash;
41  import org.apache.hadoop.hbase.util.Writables;
42  import org.apache.hadoop.io.RawComparator;
43  import org.apache.hadoop.io.WritableUtils;
44  import org.apache.hadoop.util.StringUtils;
45  
46  import com.google.common.base.Function;
47  import com.google.common.collect.ImmutableList;
48  import com.google.common.collect.Ordering;
49  
50  import java.io.FileNotFoundException;
51  import java.io.IOException;
52  import java.lang.management.ManagementFactory;
53  import java.lang.management.MemoryUsage;
54  import java.nio.ByteBuffer;
55  import java.text.NumberFormat;
56  import java.util.Arrays;
57  import java.util.Collections;
58  import java.util.Comparator;
59  import java.util.List;
60  import java.util.Map;
61  import java.util.Random;
62  import java.util.SortedSet;
63  import java.util.concurrent.atomic.AtomicBoolean;
64  import java.util.regex.Matcher;
65  import java.util.regex.Pattern;
66  
67  /**
68   * A Store data file.  Stores usually have one or more of these files.  They
69   * are produced by flushing the memstore to disk.  To
70   * create, call {@link #createWriter(FileSystem, Path, int)} and append data.  Be
71   * sure to add any metadata before calling close on the Writer
72   * (Use the appendMetadata convenience methods). On close, a StoreFile is
73   * sitting in the Filesystem.  To refer to it, create a StoreFile instance
74   * passing filesystem and path.  To read, call {@link #createReader()}.
75   * <p>StoreFiles may also reference store files in another Store.
76   *
77   * The reason for this weird pattern where you use a different instance for the
78   * writer and a reader is that we write once but read a lot more.
79   */
80  public class StoreFile {
81    static final Log LOG = LogFactory.getLog(StoreFile.class.getName());
82  
83    // Config keys.
84    static final String IO_STOREFILE_BLOOM_ERROR_RATE = "io.storefile.bloom.error.rate";
85    static final String IO_STOREFILE_BLOOM_MAX_FOLD = "io.storefile.bloom.max.fold";
86    static final String IO_STOREFILE_BLOOM_ENABLED = "io.storefile.bloom.enabled";
87    static final String HFILE_BLOCK_CACHE_SIZE_KEY = "hfile.block.cache.size";
88  
89    public static enum BloomType {
90      /**
91       * Bloomfilters disabled
92       */
93      NONE,
94      /**
95       * Bloom enabled with Table row as Key
96       */
97      ROW,
98      /**
99       * Bloom enabled with Table row & column (family+qualifier) as Key
100      */
101     ROWCOL
102   }
103   // Keys for fileinfo values in HFile
104   /** Max Sequence ID in FileInfo */
105   public static final byte [] MAX_SEQ_ID_KEY = Bytes.toBytes("MAX_SEQ_ID_KEY");
106   /** Major compaction flag in FileInfo */
107   public static final byte [] MAJOR_COMPACTION_KEY = Bytes.toBytes("MAJOR_COMPACTION_KEY");
108   /** Bloom filter Type in FileInfo */
109   static final byte[] BLOOM_FILTER_TYPE_KEY = Bytes.toBytes("BLOOM_FILTER_TYPE");
110   /** Key for Timerange information in metadata*/
111   static final byte[] TIMERANGE_KEY = Bytes.toBytes("TIMERANGE");
112 
113   /** Meta data block name for bloom filter meta-info (ie: bloom params/specs) */
114   static final String BLOOM_FILTER_META_KEY = "BLOOM_FILTER_META";
115   /** Meta data block name for bloom filter data (ie: bloom bits) */
116   static final String BLOOM_FILTER_DATA_KEY = "BLOOM_FILTER_DATA";
117 
118   // Make default block size for StoreFiles 8k while testing.  TODO: FIX!
119   // Need to make it 8k for testing.
120   public static final int DEFAULT_BLOCKSIZE_SMALL = 8 * 1024;
121 
122 
123   private static BlockCache hfileBlockCache = null;
124 
125   private final FileSystem fs;
126   // This file's path.
127   private final Path path;
128   // If this storefile references another, this is the reference instance.
129   private Reference reference;
130   // If this StoreFile references another, this is the other files path.
131   private Path referencePath;
132   // Should the block cache be used or not.
133   private boolean blockcache;
134   // Is this from an in-memory store
135   private boolean inMemory;
136 
137   // Keys for metadata stored in backing HFile.
138   // Set when we obtain a Reader.
139   private long sequenceid = -1;
140 
141   // If true, this file was product of a major compaction.  Its then set
142   // whenever you get a Reader.
143   private AtomicBoolean majorCompaction = null;
144 
145   /** Meta key set when store file is a result of a bulk load */
146   public static final byte[] BULKLOAD_TASK_KEY =
147     Bytes.toBytes("BULKLOAD_SOURCE_TASK");
148   public static final byte[] BULKLOAD_TIME_KEY =
149     Bytes.toBytes("BULKLOAD_TIMESTAMP");
150 
151   /**
152    * Map of the metadata entries in the corresponding HFile
153    */
154   private Map<byte[], byte[]> metadataMap;
155 
156   /*
157    * Regex that will work for straight filenames and for reference names.
158    * If reference, then the regex has more than just one group.  Group 1 is
159    * this files id.  Group 2 the referenced region name, etc.
160    */
161   private static final Pattern REF_NAME_PARSER =
162     Pattern.compile("^(\\d+)(?:\\.(.+))?$");
163 
164   // StoreFile.Reader
165   private volatile Reader reader;
166 
167   // Used making file ids.
168   private final static Random rand = new Random();
169   private final Configuration conf;
170   private final BloomType bloomType;
171 
172 
173   /**
174    * Constructor, loads a reader and it's indices, etc. May allocate a
175    * substantial amount of ram depending on the underlying files (10-20MB?).
176    *
177    * @param fs  The current file system to use.
178    * @param p  The path of the file.
179    * @param blockcache  <code>true</code> if the block cache is enabled.
180    * @param conf  The current configuration.
181    * @param bt The bloom type to use for this store file
182    * @throws IOException When opening the reader fails.
183    */
184   StoreFile(final FileSystem fs,
185             final Path p,
186             final boolean blockcache,
187             final Configuration conf,
188             final BloomType bt,
189             final boolean inMemory)
190       throws IOException {
191     this.conf = conf;
192     this.fs = fs;
193     this.path = p;
194     this.blockcache = blockcache;
195     this.inMemory = inMemory;
196     if (isReference(p)) {
197       this.reference = Reference.read(fs, p);
198       this.referencePath = getReferredToFile(this.path);
199     }
200     // ignore if the column family config says "no bloom filter"
201     // even if there is one in the hfile.
202     if (conf.getBoolean(IO_STOREFILE_BLOOM_ENABLED, true)) {
203       this.bloomType = bt;
204     } else {
205       this.bloomType = BloomType.NONE;
206       LOG.info("Ignoring bloom filter check for file (disabled in config)");
207     }
208   }
209 
210   /**
211    * @return Path or null if this StoreFile was made with a Stream.
212    */
213   Path getPath() {
214     return this.path;
215   }
216 
217   /**
218    * @return The Store/ColumnFamily this file belongs to.
219    */
220   byte [] getFamily() {
221     return Bytes.toBytes(this.path.getParent().getName());
222   }
223 
224   /**
225    * @return True if this is a StoreFile Reference; call after {@link #open()}
226    * else may get wrong answer.
227    */
228   boolean isReference() {
229     return this.reference != null;
230   }
231 
232   /**
233    * @param p Path to check.
234    * @return True if the path has format of a HStoreFile reference.
235    */
236   public static boolean isReference(final Path p) {
237     return !p.getName().startsWith("_") &&
238       isReference(p, REF_NAME_PARSER.matcher(p.getName()));
239   }
240 
241   /**
242    * @param p Path to check.
243    * @param m Matcher to use.
244    * @return True if the path has format of a HStoreFile reference.
245    */
246   public static boolean isReference(final Path p, final Matcher m) {
247     if (m == null || !m.matches()) {
248       LOG.warn("Failed match of store file name " + p.toString());
249       throw new RuntimeException("Failed match of store file name " +
250           p.toString());
251     }
252     return m.groupCount() > 1 && m.group(2) != null;
253   }
254 
255   /*
256    * Return path to the file referred to by a Reference.  Presumes a directory
257    * hierarchy of <code>${hbase.rootdir}/tablename/regionname/familyname</code>.
258    * @param p Path to a Reference file.
259    * @return Calculated path to parent region file.
260    * @throws IOException
261    */
262   static Path getReferredToFile(final Path p) {
263     Matcher m = REF_NAME_PARSER.matcher(p.getName());
264     if (m == null || !m.matches()) {
265       LOG.warn("Failed match of store file name " + p.toString());
266       throw new RuntimeException("Failed match of store file name " +
267           p.toString());
268     }
269     // Other region name is suffix on the passed Reference file name
270     String otherRegion = m.group(2);
271     // Tabledir is up two directories from where Reference was written.
272     Path tableDir = p.getParent().getParent().getParent();
273     String nameStrippedOfSuffix = m.group(1);
274     // Build up new path with the referenced region in place of our current
275     // region in the reference path.  Also strip regionname suffix from name.
276     return new Path(new Path(new Path(tableDir, otherRegion),
277       p.getParent().getName()), nameStrippedOfSuffix);
278   }
279 
280   /**
281    * @return True if this file was made by a major compaction.
282    */
283   boolean isMajorCompaction() {
284     if (this.majorCompaction == null) {
285       throw new NullPointerException("This has not been set yet");
286     }
287     return this.majorCompaction.get();
288   }
289 
290   /**
291    * @return This files maximum edit sequence id.
292    */
293   public long getMaxSequenceId() {
294     if (this.sequenceid == -1) {
295       throw new IllegalAccessError("Has not been initialized");
296     }
297     return this.sequenceid;
298   }
299 
300   /**
301    * Return the highest sequence ID found across all storefiles in
302    * the given list. Store files that were created by a mapreduce
303    * bulk load are ignored, as they do not correspond to any edit
304    * log items.
305    * @return 0 if no non-bulk-load files are provided or, this is Store that
306    * does not yet have any store files.
307    */
308   public static long getMaxSequenceIdInList(List<StoreFile> sfs) {
309     long max = 0;
310     for (StoreFile sf : sfs) {
311       if (!sf.isBulkLoadResult()) {
312         max = Math.max(max, sf.getMaxSequenceId());
313       }
314     }
315     return max;
316   }
317 
318   /**
319    * @return true if this storefile was created by HFileOutputFormat
320    * for a bulk load.
321    */
322   boolean isBulkLoadResult() {
323     return metadataMap.containsKey(BULKLOAD_TIME_KEY);
324   }
325 
326   /**
327    * Return the timestamp at which this bulk load file was generated.
328    */
329   public long getBulkLoadTimestamp() {
330     return Bytes.toLong(metadataMap.get(BULKLOAD_TIME_KEY));
331   }
332 
333   /**
334    * Returns the block cache or <code>null</code> in case none should be used.
335    *
336    * @param conf  The current configuration.
337    * @return The block cache or <code>null</code>.
338    */
339   public static synchronized BlockCache getBlockCache(Configuration conf) {
340     if (hfileBlockCache != null) return hfileBlockCache;
341 
342     float cachePercentage = conf.getFloat(HFILE_BLOCK_CACHE_SIZE_KEY, 0.0f);
343     // There should be a better way to optimize this. But oh well.
344     if (cachePercentage == 0L) return null;
345     if (cachePercentage > 1.0) {
346       throw new IllegalArgumentException(HFILE_BLOCK_CACHE_SIZE_KEY +
347         " must be between 0.0 and 1.0, not > 1.0");
348     }
349 
350     // Calculate the amount of heap to give the heap.
351     MemoryUsage mu = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage();
352     long cacheSize = (long)(mu.getMax() * cachePercentage);
353     LOG.info("Allocating LruBlockCache with maximum size " +
354       StringUtils.humanReadableInt(cacheSize));
355     hfileBlockCache = new LruBlockCache(cacheSize, DEFAULT_BLOCKSIZE_SMALL);
356     return hfileBlockCache;
357   }
358 
359   /**
360    * @return the blockcache
361    */
362   public BlockCache getBlockCache() {
363     return blockcache ? getBlockCache(conf) : null;
364   }
365 
366   /**
367    * Opens reader on this store file.  Called by Constructor.
368    * @return Reader for the store file.
369    * @throws IOException
370    * @see #closeReader()
371    */
372   private Reader open() throws IOException {
373 
374     if (this.reader != null) {
375       throw new IllegalAccessError("Already open");
376     }
377 
378     if (isReference()) {
379       this.reader = new HalfStoreFileReader(this.fs, this.referencePath,
380           getBlockCache(), this.reference);
381     } else {
382       this.reader = new Reader(this.fs, this.path, getBlockCache(),
383           this.inMemory);
384     }
385 
386     // Load up indices and fileinfo.
387     metadataMap = Collections.unmodifiableMap(this.reader.loadFileInfo());
388     // Read in our metadata.
389     byte [] b = metadataMap.get(MAX_SEQ_ID_KEY);
390     if (b != null) {
391       // By convention, if halfhfile, top half has a sequence number > bottom
392       // half. Thats why we add one in below. Its done for case the two halves
393       // are ever merged back together --rare.  Without it, on open of store,
394       // since store files are distingushed by sequence id, the one half would
395       // subsume the other.
396       this.sequenceid = Bytes.toLong(b);
397       if (isReference()) {
398         if (Reference.isTopFileRegion(this.reference.getFileRegion())) {
399           this.sequenceid += 1;
400         }
401       }
402     }
403 
404     b = metadataMap.get(MAJOR_COMPACTION_KEY);
405     if (b != null) {
406       boolean mc = Bytes.toBoolean(b);
407       if (this.majorCompaction == null) {
408         this.majorCompaction = new AtomicBoolean(mc);
409       } else {
410         this.majorCompaction.set(mc);
411       }
412     }
413 
414     if (this.bloomType != BloomType.NONE) {
415       this.reader.loadBloomfilter();
416     }
417 
418     try {
419       byte [] timerangeBytes = metadataMap.get(TIMERANGE_KEY);
420       if (timerangeBytes != null) {
421         this.reader.timeRangeTracker = new TimeRangeTracker();
422         Writables.copyWritable(timerangeBytes, this.reader.timeRangeTracker);
423       }
424     } catch (IllegalArgumentException e) {
425       LOG.error("Error reading timestamp range data from meta -- " +
426           "proceeding without", e);
427       this.reader.timeRangeTracker = null;
428     }
429     return this.reader;
430   }
431 
432   /**
433    * @return Reader for StoreFile. creates if necessary
434    * @throws IOException
435    */
436   public Reader createReader() throws IOException {
437     if (this.reader == null) {
438       this.reader = open();
439     }
440     return this.reader;
441   }
442 
443   /**
444    * @return Current reader.  Must call createReader first else returns null.
445    * @throws IOException
446    * @see {@link #createReader()}
447    */
448   public Reader getReader() {
449     return this.reader;
450   }
451 
452   /**
453    * @throws IOException
454    */
455   public synchronized void closeReader() throws IOException {
456     if (this.reader != null) {
457       this.reader.close();
458       this.reader = null;
459     }
460   }
461 
462   /**
463    * Delete this file
464    * @throws IOException
465    */
466   public void deleteReader() throws IOException {
467     closeReader();
468     this.fs.delete(getPath(), true);
469   }
470 
471   @Override
472   public String toString() {
473     return this.path.toString() +
474       (isReference()? "-" + this.referencePath + "-" + reference.toString(): "");
475   }
476 
477   /**
478    * @return a length description of this StoreFile, suitable for debug output
479    */
480   public String toStringDetailed() {
481     StringBuilder sb = new StringBuilder();
482     sb.append(this.path.toString());
483     sb.append(", isReference=").append(isReference());
484     sb.append(", isBulkLoadResult=").append(isBulkLoadResult());
485     if (isBulkLoadResult()) {
486       sb.append(", bulkLoadTS=").append(getBulkLoadTimestamp());
487     } else {
488       sb.append(", seqid=").append(getMaxSequenceId());
489     }
490     sb.append(", majorCompaction=").append(isMajorCompaction());
491 
492     return sb.toString();
493   }
494 
495   /**
496    * Utility to help with rename.
497    * @param fs
498    * @param src
499    * @param tgt
500    * @return True if succeeded.
501    * @throws IOException
502    */
503   public static Path rename(final FileSystem fs,
504                             final Path src,
505                             final Path tgt)
506       throws IOException {
507 
508     if (!fs.exists(src)) {
509       throw new FileNotFoundException(src.toString());
510     }
511     if (!fs.rename(src, tgt)) {
512       throw new IOException("Failed rename of " + src + " to " + tgt);
513     }
514     return tgt;
515   }
516 
517   /**
518    * Get a store file writer. Client is responsible for closing file when done.
519    *
520    * @param fs
521    * @param dir Path to family directory.  Makes the directory if doesn't exist.
522    * Creates a file with a unique name in this directory.
523    * @param blocksize size per filesystem block
524    * @return StoreFile.Writer
525    * @throws IOException
526    */
527   public static Writer createWriter(final FileSystem fs,
528                                               final Path dir,
529                                               final int blocksize)
530       throws IOException {
531 
532     return createWriter(fs, dir, blocksize, null, null, null, BloomType.NONE, 0);
533   }
534 
535   /**
536    * Create a store file writer. Client is responsible for closing file when done.
537    * If metadata, add BEFORE closing using appendMetadata()
538    * @param fs
539    * @param dir Path to family directory.  Makes the directory if doesn't exist.
540    * Creates a file with a unique name in this directory.
541    * @param blocksize
542    * @param algorithm Pass null to get default.
543    * @param conf HBase system configuration. used with bloom filters
544    * @param bloomType column family setting for bloom filters
545    * @param c Pass null to get default.
546    * @param maxKeySize peak theoretical entry size (maintains error rate)
547    * @return HFile.Writer
548    * @throws IOException
549    */
550   public static StoreFile.Writer createWriter(final FileSystem fs,
551                                               final Path dir,
552                                               final int blocksize,
553                                               final Compression.Algorithm algorithm,
554                                               final KeyValue.KVComparator c,
555                                               final Configuration conf,
556                                               BloomType bloomType,
557                                               int maxKeySize)
558       throws IOException {
559 
560     if (!fs.exists(dir)) {
561       fs.mkdirs(dir);
562     }
563     Path path = getUniqueFile(fs, dir);
564     if(conf == null || !conf.getBoolean(IO_STOREFILE_BLOOM_ENABLED, true)) {
565       bloomType = BloomType.NONE;
566     }
567 
568     return new Writer(fs, path, blocksize,
569         algorithm == null? HFile.DEFAULT_COMPRESSION_ALGORITHM: algorithm,
570         conf, c == null? KeyValue.COMPARATOR: c, bloomType, maxKeySize);
571   }
572 
573   /**
574    * @param fs
575    * @param dir Directory to create file in.
576    * @return random filename inside passed <code>dir</code>
577    */
578   public static Path getUniqueFile(final FileSystem fs, final Path dir)
579       throws IOException {
580     if (!fs.getFileStatus(dir).isDir()) {
581       throw new IOException("Expecting " + dir.toString() +
582         " to be a directory");
583     }
584     return fs.getFileStatus(dir).isDir()? getRandomFilename(fs, dir): dir;
585   }
586 
587   /**
588    *
589    * @param fs
590    * @param dir
591    * @return Path to a file that doesn't exist at time of this invocation.
592    * @throws IOException
593    */
594   static Path getRandomFilename(final FileSystem fs, final Path dir)
595       throws IOException {
596     return getRandomFilename(fs, dir, null);
597   }
598 
599   /**
600    *
601    * @param fs
602    * @param dir
603    * @param suffix
604    * @return Path to a file that doesn't exist at time of this invocation.
605    * @throws IOException
606    */
607   static Path getRandomFilename(final FileSystem fs,
608                                 final Path dir,
609                                 final String suffix)
610       throws IOException {
611     long id = -1;
612     Path p = null;
613     do {
614       id = Math.abs(rand.nextLong());
615       p = new Path(dir, Long.toString(id) +
616         ((suffix == null || suffix.length() <= 0)? "": suffix));
617     } while(fs.exists(p));
618     return p;
619   }
620 
621   /**
622    * Write out a split reference.
623    *
624    * Package local so it doesnt leak out of regionserver.
625    *
626    * @param fs
627    * @param splitDir Presumes path format is actually
628    * <code>SOME_DIRECTORY/REGIONNAME/FAMILY</code>.
629    * @param f File to split.
630    * @param splitRow
631    * @param range
632    * @return Path to created reference.
633    * @throws IOException
634    */
635   static Path split(final FileSystem fs,
636                     final Path splitDir,
637                     final StoreFile f,
638                     final byte [] splitRow,
639                     final Reference.Range range)
640       throws IOException {
641     // A reference to the bottom half of the hsf store file.
642     Reference r = new Reference(splitRow, range);
643     // Add the referred-to regions name as a dot separated suffix.
644     // See REF_NAME_PARSER regex above.  The referred-to regions name is
645     // up in the path of the passed in <code>f</code> -- parentdir is family,
646     // then the directory above is the region name.
647     String parentRegionName = f.getPath().getParent().getParent().getName();
648     // Write reference with same file id only with the other region name as
649     // suffix and into the new region location (under same family).
650     Path p = new Path(splitDir, f.getPath().getName() + "." + parentRegionName);
651     return r.write(fs, p);
652   }
653 
654 
655   /**
656    * A StoreFile writer.  Use this to read/write HBase Store Files. It is package
657    * local because it is an implementation detail of the HBase regionserver.
658    */
659   public static class Writer {
660     private final BloomFilter bloomFilter;
661     private final BloomType bloomType;
662     private KVComparator kvComparator;
663     private KeyValue lastKv = null;
664     private byte[] lastByteArray = null;
665     TimeRangeTracker timeRangeTracker = new TimeRangeTracker();
666     /* isTimeRangeTrackerSet keeps track if the timeRange has already been set
667      * When flushing a memstore, we set TimeRange and use this variable to
668      * indicate that it doesn't need to be calculated again while
669      * appending KeyValues.
670      * It is not set in cases of compactions when it is recalculated using only
671      * the appended KeyValues*/
672     boolean isTimeRangeTrackerSet = false;
673 
674     protected HFile.Writer writer;
675     /**
676      * Creates an HFile.Writer that also write helpful meta data.
677      * @param fs file system to write to
678      * @param path file name to create
679      * @param blocksize HDFS block size
680      * @param compress HDFS block compression
681      * @param conf user configuration
682      * @param comparator key comparator
683      * @param bloomType bloom filter setting
684      * @param maxKeys maximum amount of keys to add (for blooms)
685      * @throws IOException problem writing to FS
686      */
687     public Writer(FileSystem fs, Path path, int blocksize,
688         Compression.Algorithm compress, final Configuration conf,
689         final KVComparator comparator, BloomType bloomType, int maxKeys)
690         throws IOException {
691       writer = new HFile.Writer(fs, path, blocksize, compress, comparator.getRawComparator());
692 
693       this.kvComparator = comparator;
694 
695       if (bloomType != BloomType.NONE && conf != null) {
696         float err = conf.getFloat(IO_STOREFILE_BLOOM_ERROR_RATE, (float)0.01);
697         // Since in row+col blooms we have 2 calls to shouldSeek() instead of 1
698         // and the false positives are adding up, we should keep the error rate
699         // twice as low in order to maintain the number of false positives as
700         // desired by the user
701         if (bloomType == BloomType.ROWCOL) {
702           err /= 2;
703         }
704         int maxFold = conf.getInt(IO_STOREFILE_BLOOM_MAX_FOLD, 7);
705 
706         this.bloomFilter = new ByteBloomFilter(maxKeys, err,
707             Hash.getHashType(conf), maxFold);
708         this.bloomFilter.allocBloom();
709         this.bloomType = bloomType;
710       } else {
711         this.bloomFilter = null;
712         this.bloomType = BloomType.NONE;
713       }
714     }
715 
716     /**
717      * Writes meta data.
718      * Call before {@link #close()} since its written as meta data to this file.
719      * @param maxSequenceId Maximum sequence id.
720      * @param majorCompaction True if this file is product of a major compaction
721      * @throws IOException problem writing to FS
722      */
723     public void appendMetadata(final long maxSequenceId, final boolean majorCompaction)
724     throws IOException {
725       writer.appendFileInfo(MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId));
726       writer.appendFileInfo(MAJOR_COMPACTION_KEY,
727           Bytes.toBytes(majorCompaction));
728       appendTimeRangeMetadata();
729     }
730 
731     /**
732      * Add TimestampRange to Metadata
733      */
734     public void appendTimeRangeMetadata() throws IOException {
735       appendFileInfo(TIMERANGE_KEY,WritableUtils.toByteArray(timeRangeTracker));
736     }
737 
738     /**
739      * Set TimeRangeTracker
740      * @param trt
741      */
742     public void setTimeRangeTracker(final TimeRangeTracker trt) {
743       this.timeRangeTracker = trt;
744       isTimeRangeTrackerSet = true;
745     }
746 
747     /**
748      * If the timeRangeTracker is not set,
749      * update TimeRangeTracker to include the timestamp of this key
750      * @param kv
751      * @throws IOException
752      */
753     public void includeInTimeRangeTracker(final KeyValue kv) {
754       if (!isTimeRangeTrackerSet) {
755         timeRangeTracker.includeTimestamp(kv);
756       }
757     }
758 
759     /**
760      * If the timeRangeTracker is not set,
761      * update TimeRangeTracker to include the timestamp of this key
762      * @param key
763      * @throws IOException
764      */
765     public void includeInTimeRangeTracker(final byte [] key) {
766       if (!isTimeRangeTrackerSet) {
767         timeRangeTracker.includeTimestamp(key);
768       }
769     }
770 
771     public void append(final KeyValue kv) throws IOException {
772       if (this.bloomFilter != null) {
773         // only add to the bloom filter on a new, unique key
774         boolean newKey = true;
775         if (this.lastKv != null) {
776           switch(bloomType) {
777           case ROW:
778             newKey = ! kvComparator.matchingRows(kv, lastKv);
779             break;
780           case ROWCOL:
781             newKey = ! kvComparator.matchingRowColumn(kv, lastKv);
782             break;
783           case NONE:
784             newKey = false;
785           }
786         }
787         if (newKey) {
788           /*
789            * http://2.bp.blogspot.com/_Cib_A77V54U/StZMrzaKufI/AAAAAAAAADo/ZhK7bGoJdMQ/s400/KeyValue.png
790            * Key = RowLen + Row + FamilyLen + Column [Family + Qualifier] + TimeStamp
791            *
792            * 2 Types of Filtering:
793            *  1. Row = Row
794            *  2. RowCol = Row + Qualifier
795            */
796           switch (bloomType) {
797           case ROW:
798             this.bloomFilter.add(kv.getBuffer(), kv.getRowOffset(),
799                 kv.getRowLength());
800             break;
801           case ROWCOL:
802             // merge(row, qualifier)
803             int ro = kv.getRowOffset();
804             int rl = kv.getRowLength();
805             int qo = kv.getQualifierOffset();
806             int ql = kv.getQualifierLength();
807             byte [] result = new byte[rl + ql];
808             System.arraycopy(kv.getBuffer(), ro, result, 0,  rl);
809             System.arraycopy(kv.getBuffer(), qo, result, rl, ql);
810             this.bloomFilter.add(result);
811             break;
812           default:
813           }
814           this.lastKv = kv;
815         }
816       }
817       writer.append(kv);
818       includeInTimeRangeTracker(kv);
819     }
820 
821     public Path getPath() {
822       return this.writer.getPath();
823     }
824 
825     public void append(final byte [] key, final byte [] value) throws IOException {
826       if (this.bloomFilter != null) {
827         // only add to the bloom filter on a new row
828         if (this.lastByteArray == null || !Arrays.equals(key, lastByteArray)) {
829           this.bloomFilter.add(key);
830           this.lastByteArray = key;
831         }
832       }
833       writer.append(key, value);
834       includeInTimeRangeTracker(key);
835     }
836 
837     public void close() throws IOException {
838       // make sure we wrote something to the bloom before adding it
839       if (this.bloomFilter != null && this.bloomFilter.getKeyCount() > 0) {
840         bloomFilter.compactBloom();
841         if (this.bloomFilter.getMaxKeys() > 0) {
842           int b = this.bloomFilter.getByteSize();
843           int k = this.bloomFilter.getKeyCount();
844           int m = this.bloomFilter.getMaxKeys();
845           StoreFile.LOG.info("Bloom added to HFile.  " + b + "B, " +
846               k + "/" + m + " (" + NumberFormat.getPercentInstance().format(
847                 ((double)k) / ((double)m)) + ")");
848         }
849         writer.appendMetaBlock(BLOOM_FILTER_META_KEY, bloomFilter.getMetaWriter());
850         writer.appendMetaBlock(BLOOM_FILTER_DATA_KEY, bloomFilter.getDataWriter());
851         writer.appendFileInfo(BLOOM_FILTER_TYPE_KEY, Bytes.toBytes(bloomType.toString()));
852       }
853       writer.close();
854     }
855 
856     public void appendFileInfo(byte[] key, byte[] value) throws IOException {
857       writer.appendFileInfo(key, value);
858     }
859   }
860 
861   /**
862    * Reader for a StoreFile.
863    */
864   public static class Reader {
865     static final Log LOG = LogFactory.getLog(Reader.class.getName());
866 
867     protected BloomFilter bloomFilter = null;
868     protected BloomType bloomFilterType;
869     private final HFile.Reader reader;
870     protected TimeRangeTracker timeRangeTracker = null;
871 
872     public Reader(FileSystem fs, Path path, BlockCache blockCache, boolean inMemory)
873         throws IOException {
874       reader = new HFile.Reader(fs, path, blockCache, inMemory);
875       bloomFilterType = BloomType.NONE;
876     }
877 
878     public RawComparator<byte []> getComparator() {
879       return reader.getComparator();
880     }
881 
882     /**
883      * Get a scanner to scan over this StoreFile.
884      *
885      * @param cacheBlocks should this scanner cache blocks?
886      * @param pread use pread (for highly concurrent small readers)
887      * @return a scanner
888      */
889     public StoreFileScanner getStoreFileScanner(boolean cacheBlocks, boolean pread) {
890       return new StoreFileScanner(this, getScanner(cacheBlocks, pread));
891     }
892 
893     /**
894      * Warning: Do not write further code which depends on this call. Instead
895      * use getStoreFileScanner() which uses the StoreFileScanner class/interface
896      * which is the preferred way to scan a store with higher level concepts.
897      *
898      * @param cacheBlocks should we cache the blocks?
899      * @param pread use pread (for concurrent small readers)
900      * @return the underlying HFileScanner
901      */
902     @Deprecated
903     public HFileScanner getScanner(boolean cacheBlocks, boolean pread) {
904       return reader.getScanner(cacheBlocks, pread);
905     }
906 
907     public void close() throws IOException {
908       reader.close();
909     }
910 
911     public boolean shouldSeek(Scan scan, final SortedSet<byte[]> columns) {
912         return (passesTimerangeFilter(scan) && passesBloomFilter(scan,columns));
913     }
914 
915     /**
916      * Check if this storeFile may contain keys within the TimeRange
917      * @param scan
918      * @return False if it definitely does not exist in this StoreFile
919      */
920     private boolean passesTimerangeFilter(Scan scan) {
921       if (timeRangeTracker == null) {
922         return true;
923       } else {
924         return timeRangeTracker.includesTimeRange(scan.getTimeRange());
925       }
926     }
927 
928     private boolean passesBloomFilter(Scan scan, final SortedSet<byte[]> columns) {
929       if (this.bloomFilter == null || !scan.isGetScan()) {
930         return true;
931       }
932       byte[] row = scan.getStartRow();
933       byte[] key;
934       switch (this.bloomFilterType) {
935         case ROW:
936           key = row;
937           break;
938         case ROWCOL:
939           if (columns != null && columns.size() == 1) {
940             byte[] col = columns.first();
941             key = Bytes.add(row, col);
942             break;
943           }
944           //$FALL-THROUGH$
945         default:
946           return true;
947       }
948 
949       try {
950         ByteBuffer bloom = reader.getMetaBlock(BLOOM_FILTER_DATA_KEY, true);
951         if (bloom != null) {
952           if (this.bloomFilterType == BloomType.ROWCOL) {
953             // Since a Row Delete is essentially a DeleteFamily applied to all
954             // columns, a file might be skipped if using row+col Bloom filter.
955             // In order to ensure this file is included an additional check is
956             // required looking only for a row bloom.
957             return this.bloomFilter.contains(key, bloom) ||
958                 this.bloomFilter.contains(row, bloom);
959           }
960           else {
961             return this.bloomFilter.contains(key, bloom);
962           }
963         }
964       } catch (IOException e) {
965         LOG.error("Error reading bloom filter data -- proceeding without",
966             e);
967         setBloomFilterFaulty();
968       } catch (IllegalArgumentException e) {
969         LOG.error("Bad bloom filter data -- proceeding without", e);
970         setBloomFilterFaulty();
971       }
972 
973       return true;
974     }
975 
976     public Map<byte[], byte[]> loadFileInfo() throws IOException {
977       Map<byte [], byte []> fi = reader.loadFileInfo();
978 
979       byte[] b = fi.get(BLOOM_FILTER_TYPE_KEY);
980       if (b != null) {
981         bloomFilterType = BloomType.valueOf(Bytes.toString(b));
982       }
983 
984       return fi;
985     }
986 
987     public void loadBloomfilter() {
988       if (this.bloomFilter != null) {
989         return; // already loaded
990       }
991 
992       try {
993         ByteBuffer b = reader.getMetaBlock(BLOOM_FILTER_META_KEY, false);
994         if (b != null) {
995           if (bloomFilterType == BloomType.NONE) {
996             throw new IOException("valid bloom filter type not found in FileInfo");
997           }
998 
999 
1000           this.bloomFilter = new ByteBloomFilter(b);
1001           LOG.info("Loaded " + (bloomFilterType== BloomType.ROW? "row":"col")
1002                  + " bloom filter metadata for " + reader.getName());
1003         }
1004       } catch (IOException e) {
1005         LOG.error("Error reading bloom filter meta -- proceeding without", e);
1006         this.bloomFilter = null;
1007       } catch (IllegalArgumentException e) {
1008         LOG.error("Bad bloom filter meta -- proceeding without", e);
1009         this.bloomFilter = null;
1010       }
1011     }
1012 
1013     public int getFilterEntries() {
1014       return (this.bloomFilter != null) ? this.bloomFilter.getKeyCount()
1015           : reader.getFilterEntries();
1016     }
1017 
1018     public ByteBuffer getMetaBlock(String bloomFilterDataKey, boolean cacheBlock) throws IOException {
1019       return reader.getMetaBlock(bloomFilterDataKey, cacheBlock);
1020     }
1021 
1022     public void setBloomFilterFaulty() {
1023       bloomFilter = null;
1024     }
1025 
1026     public byte[] getLastKey() {
1027       return reader.getLastKey();
1028     }
1029 
1030     public byte[] midkey() throws IOException {
1031       return reader.midkey();
1032     }
1033 
1034     public long length() {
1035       return reader.length();
1036     }
1037 
1038     public int getEntries() {
1039       return reader.getEntries();
1040     }
1041 
1042     public byte[] getFirstKey() {
1043       return reader.getFirstKey();
1044     }
1045 
1046     public long indexSize() {
1047       return reader.indexSize();
1048     }
1049 
1050     public BloomType getBloomFilterType() {
1051       return this.bloomFilterType;
1052     }
1053   }
1054 
1055   /**
1056    * Useful comparators for comparing StoreFiles.
1057    */
1058   abstract static class Comparators {
1059     /**
1060      * Comparator that compares based on the flush time of
1061      * the StoreFiles. All bulk loads are placed before all non-
1062      * bulk loads, and then all files are sorted by sequence ID.
1063      * If there are ties, the path name is used as a tie-breaker.
1064      */
1065     static final Comparator<StoreFile> FLUSH_TIME =
1066       Ordering.compound(ImmutableList.of(
1067           Ordering.natural().onResultOf(new GetBulkTime()),
1068           Ordering.natural().onResultOf(new GetSeqId()),
1069           Ordering.natural().onResultOf(new GetPathName())
1070       ));
1071 
1072     private static class GetBulkTime implements Function<StoreFile, Long> {
1073       @Override
1074       public Long apply(StoreFile sf) {
1075         if (!sf.isBulkLoadResult()) return Long.MAX_VALUE;
1076         return sf.getBulkLoadTimestamp();
1077       }
1078     }
1079     private static class GetSeqId implements Function<StoreFile, Long> {
1080       @Override
1081       public Long apply(StoreFile sf) {
1082         if (sf.isBulkLoadResult()) return -1L;
1083         return sf.getMaxSequenceId();
1084       }
1085     }
1086     private static class GetPathName implements Function<StoreFile, String> {
1087       @Override
1088       public String apply(StoreFile sf) {
1089         return sf.getPath().getName();
1090       }
1091     }
1092 
1093   }
1094 }