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.regionserver;
20  
21  import java.io.DataInput;
22  import java.io.IOException;
23  import java.net.InetSocketAddress;
24  import java.nio.ByteBuffer;
25  import java.util.Arrays;
26  import java.util.Collection;
27  import java.util.Collections;
28  import java.util.Comparator;
29  import java.util.Map;
30  import java.util.SortedSet;
31  import java.util.UUID;
32  import java.util.concurrent.atomic.AtomicBoolean;
33  
34  import org.apache.commons.logging.Log;
35  import org.apache.commons.logging.LogFactory;
36  import org.apache.hadoop.hbase.classification.InterfaceAudience;
37  import org.apache.hadoop.conf.Configuration;
38  import org.apache.hadoop.fs.FileSystem;
39  import org.apache.hadoop.fs.Path;
40  import org.apache.hadoop.hbase.Cell;
41  import org.apache.hadoop.hbase.CellUtil;
42  import org.apache.hadoop.hbase.HConstants;
43  import org.apache.hadoop.hbase.HDFSBlocksDistribution;
44  import org.apache.hadoop.hbase.KeyValue;
45  import org.apache.hadoop.hbase.KeyValue.KVComparator;
46  import org.apache.hadoop.hbase.KeyValueUtil;
47  import org.apache.hadoop.hbase.client.Scan;
48  import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
49  import org.apache.hadoop.hbase.io.hfile.BlockType;
50  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
51  import org.apache.hadoop.hbase.io.hfile.HFile;
52  import org.apache.hadoop.hbase.io.hfile.HFileContext;
53  import org.apache.hadoop.hbase.io.hfile.HFileScanner;
54  import org.apache.hadoop.hbase.io.hfile.HFileWriterV2;
55  import org.apache.hadoop.hbase.regionserver.compactions.Compactor;
56  import org.apache.hadoop.hbase.util.BloomFilter;
57  import org.apache.hadoop.hbase.util.BloomFilterFactory;
58  import org.apache.hadoop.hbase.util.BloomFilterWriter;
59  import org.apache.hadoop.hbase.util.Bytes;
60  import org.apache.hadoop.hbase.util.Writables;
61  import org.apache.hadoop.io.WritableUtils;
62  
63  import com.google.common.base.Function;
64  import com.google.common.base.Preconditions;
65  import com.google.common.collect.ImmutableList;
66  import com.google.common.collect.Ordering;
67  
68  /**
69   * A Store data file.  Stores usually have one or more of these files.  They
70   * are produced by flushing the memstore to disk.  To
71   * create, instantiate a writer using {@link StoreFile.WriterBuilder}
72   * and append data. Be sure to add any metadata before calling close on the
73   * Writer (Use the appendMetadata convenience methods). On close, a StoreFile
74   * is sitting in the Filesystem.  To refer to it, create a StoreFile instance
75   * passing filesystem and path.  To read, call {@link #createReader()}.
76   * <p>StoreFiles may also reference store files in another Store.
77   *
78   * The reason for this weird pattern where you use a different instance for the
79   * writer and a reader is that we write once but read a lot more.
80   */
81  @InterfaceAudience.LimitedPrivate("Coprocessor")
82  public class StoreFile {
83    static final Log LOG = LogFactory.getLog(StoreFile.class.getName());
84  
85    // Keys for fileinfo values in HFile
86  
87    /** Max Sequence ID in FileInfo */
88    public static final byte [] MAX_SEQ_ID_KEY = Bytes.toBytes("MAX_SEQ_ID_KEY");
89  
90    /** Major compaction flag in FileInfo */
91    public static final byte[] MAJOR_COMPACTION_KEY =
92        Bytes.toBytes("MAJOR_COMPACTION_KEY");
93  
94    /** Minor compaction flag in FileInfo */
95    public static final byte[] EXCLUDE_FROM_MINOR_COMPACTION_KEY =
96        Bytes.toBytes("EXCLUDE_FROM_MINOR_COMPACTION");
97  
98    /** Bloom filter Type in FileInfo */
99    public static final byte[] BLOOM_FILTER_TYPE_KEY =
100       Bytes.toBytes("BLOOM_FILTER_TYPE");
101 
102   /** Delete Family Count in FileInfo */
103   public static final byte[] DELETE_FAMILY_COUNT =
104       Bytes.toBytes("DELETE_FAMILY_COUNT");
105 
106   /** Last Bloom filter key in FileInfo */
107   private static final byte[] LAST_BLOOM_KEY = Bytes.toBytes("LAST_BLOOM_KEY");
108 
109   /** Key for Timerange information in metadata*/
110   public static final byte[] TIMERANGE_KEY = Bytes.toBytes("TIMERANGE");
111 
112   /** Key for timestamp of earliest-put in metadata*/
113   public static final byte[] EARLIEST_PUT_TS = Bytes.toBytes("EARLIEST_PUT_TS");
114 
115   private final StoreFileInfo fileInfo;
116   private final FileSystem fs;
117 
118   // Block cache configuration and reference.
119   private final CacheConfig cacheConf;
120 
121   // Keys for metadata stored in backing HFile.
122   // Set when we obtain a Reader.
123   private long sequenceid = -1;
124 
125   // max of the MemstoreTS in the KV's in this store
126   // Set when we obtain a Reader.
127   private long maxMemstoreTS = -1;
128 
129   public long getMaxMemstoreTS() {
130     return maxMemstoreTS;
131   }
132 
133   public void setMaxMemstoreTS(long maxMemstoreTS) {
134     this.maxMemstoreTS = maxMemstoreTS;
135   }
136 
137   // If true, this file was product of a major compaction.  Its then set
138   // whenever you get a Reader.
139   private AtomicBoolean majorCompaction = null;
140 
141   // If true, this file should not be included in minor compactions.
142   // It's set whenever you get a Reader.
143   private boolean excludeFromMinorCompaction = false;
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   // StoreFile.Reader
157   private volatile Reader reader;
158 
159   /**
160    * Bloom filter type specified in column family configuration. Does not
161    * necessarily correspond to the Bloom filter type present in the HFile.
162    */
163   private final BloomType cfBloomType;
164 
165   // the last modification time stamp
166   private long modificationTimeStamp = 0L;
167 
168   /**
169    * Constructor, loads a reader and it's indices, etc. May allocate a
170    * substantial amount of ram depending on the underlying files (10-20MB?).
171    *
172    * @param fs  The current file system to use.
173    * @param p  The path of the file.
174    * @param conf  The current configuration.
175    * @param cacheConf  The cache configuration and block cache reference.
176    * @param cfBloomType The bloom type to use for this store file as specified
177    *          by column family configuration. This may or may not be the same
178    *          as the Bloom filter type actually present in the HFile, because
179    *          column family configuration might change. If this is
180    *          {@link BloomType#NONE}, the existing Bloom filter is ignored.
181    * @throws IOException When opening the reader fails.
182    */
183   public StoreFile(final FileSystem fs, final Path p, final Configuration conf,
184         final CacheConfig cacheConf, final BloomType cfBloomType) throws IOException {
185     this(fs, new StoreFileInfo(conf, fs, p), conf, cacheConf, cfBloomType);
186   }
187 
188 
189   /**
190    * Constructor, loads a reader and it's indices, etc. May allocate a
191    * substantial amount of ram depending on the underlying files (10-20MB?).
192    *
193    * @param fs  The current file system to use.
194    * @param fileInfo  The store file information.
195    * @param conf  The current configuration.
196    * @param cacheConf  The cache configuration and block cache reference.
197    * @param cfBloomType The bloom type to use for this store file as specified
198    *          by column family configuration. This may or may not be the same
199    *          as the Bloom filter type actually present in the HFile, because
200    *          column family configuration might change. If this is
201    *          {@link BloomType#NONE}, the existing Bloom filter is ignored.
202    * @throws IOException When opening the reader fails.
203    */
204   public StoreFile(final FileSystem fs, final StoreFileInfo fileInfo, final Configuration conf,
205       final CacheConfig cacheConf,  final BloomType cfBloomType) throws IOException {
206     this.fs = fs;
207     this.fileInfo = fileInfo;
208     this.cacheConf = cacheConf;
209 
210     if (BloomFilterFactory.isGeneralBloomEnabled(conf)) {
211       this.cfBloomType = cfBloomType;
212     } else {
213       LOG.info("Ignoring bloom filter check for file " + this.getPath() + ": " +
214           "cfBloomType=" + cfBloomType + " (disabled in config)");
215       this.cfBloomType = BloomType.NONE;
216     }
217 
218     // cache the modification time stamp of this store file
219     this.modificationTimeStamp = fileInfo.getModificationTime();
220   }
221 
222   /**
223    * Clone
224    * @param other The StoreFile to clone from
225    */
226   public StoreFile(final StoreFile other) {
227     this.fs = other.fs;
228     this.fileInfo = other.fileInfo;
229     this.cacheConf = other.cacheConf;
230     this.cfBloomType = other.cfBloomType;
231     this.modificationTimeStamp = other.modificationTimeStamp;
232   }
233 
234   /**
235    * @return the StoreFile object associated to this StoreFile.
236    *         null if the StoreFile is not a reference.
237    */
238   public StoreFileInfo getFileInfo() {
239     return this.fileInfo;
240   }
241 
242   /**
243    * @return Path or null if this StoreFile was made with a Stream.
244    */
245   public Path getPath() {
246     return this.fileInfo.getPath();
247   }
248 
249   /**
250    * @return Returns the qualified path of this StoreFile
251    */
252   public Path getQualifiedPath() {
253     return this.fileInfo.getPath().makeQualified(fs);
254   }
255 
256   /**
257    * @return True if this is a StoreFile Reference; call after {@link #open()}
258    * else may get wrong answer.
259    */
260   public boolean isReference() {
261     return this.fileInfo.isReference();
262   }
263 
264   /**
265    * @return True if this file was made by a major compaction.
266    */
267   public boolean isMajorCompaction() {
268     if (this.majorCompaction == null) {
269       throw new NullPointerException("This has not been set yet");
270     }
271     return this.majorCompaction.get();
272   }
273 
274   /**
275    * @return True if this file should not be part of a minor compaction.
276    */
277   public boolean excludeFromMinorCompaction() {
278     return this.excludeFromMinorCompaction;
279   }
280 
281   /**
282    * @return This files maximum edit sequence id.
283    */
284   public long getMaxSequenceId() {
285     return this.sequenceid;
286   }
287 
288   public long getModificationTimeStamp() {
289     return modificationTimeStamp;
290   }
291 
292   public byte[] getMetadataValue(byte[] key) {
293     return metadataMap.get(key);
294   }
295 
296   /**
297    * Return the largest memstoreTS found across all storefiles in
298    * the given list. Store files that were created by a mapreduce
299    * bulk load are ignored, as they do not correspond to any specific
300    * put operation, and thus do not have a memstoreTS associated with them.
301    * @return 0 if no non-bulk-load files are provided or, this is Store that
302    * does not yet have any store files.
303    */
304   public static long getMaxMemstoreTSInList(Collection<StoreFile> sfs) {
305     long max = 0;
306     for (StoreFile sf : sfs) {
307       if (!sf.isBulkLoadResult()) {
308         max = Math.max(max, sf.getMaxMemstoreTS());
309       }
310     }
311     return max;
312   }
313 
314   /**
315    * Return the highest sequence ID found across all storefiles in
316    * the given list.
317    * @param sfs
318    * @return 0 if no non-bulk-load files are provided or, this is Store that
319    * does not yet have any store files.
320    */
321   public static long getMaxSequenceIdInList(Collection<StoreFile> sfs) {
322     long max = 0;
323     for (StoreFile sf : sfs) {
324       max = Math.max(max, sf.getMaxSequenceId());
325     }
326     return max;
327   }
328 
329   /**
330    * Check if this storefile was created by bulk load.
331    * When a hfile is bulk loaded into HBase, we append
332    * '_SeqId_<id-when-loaded>' to the hfile name, unless
333    * "hbase.mapreduce.bulkload.assign.sequenceNumbers" is
334    * explicitly turned off.
335    * If "hbase.mapreduce.bulkload.assign.sequenceNumbers"
336    * is turned off, fall back to BULKLOAD_TIME_KEY.
337    * @return true if this storefile was created by bulk load.
338    */
339   boolean isBulkLoadResult() {
340     boolean bulkLoadedHFile = false;
341     String fileName = this.getPath().getName();
342     int startPos = fileName.indexOf("SeqId_");
343     if (startPos != -1) {
344       bulkLoadedHFile = true;
345     }
346     return bulkLoadedHFile || metadataMap.containsKey(BULKLOAD_TIME_KEY);
347   }
348 
349   /**
350    * Return the timestamp at which this bulk load file was generated.
351    */
352   public long getBulkLoadTimestamp() {
353     byte[] bulkLoadTimestamp = metadataMap.get(BULKLOAD_TIME_KEY);
354     return (bulkLoadTimestamp == null) ? 0 : Bytes.toLong(bulkLoadTimestamp);
355   }
356 
357   /**
358    * @return the cached value of HDFS blocks distribution. The cached value is
359    * calculated when store file is opened.
360    */
361   public HDFSBlocksDistribution getHDFSBlockDistribution() {
362     return this.fileInfo.getHDFSBlockDistribution();
363   }
364 
365   /**
366    * Opens reader on this store file.  Called by Constructor.
367    * @return Reader for the store file.
368    * @throws IOException
369    * @see #closeReader(boolean)
370    */
371   private Reader open() throws IOException {
372     if (this.reader != null) {
373       throw new IllegalAccessError("Already open");
374     }
375 
376     // Open the StoreFile.Reader
377     this.reader = fileInfo.open(this.fs, this.cacheConf);
378 
379     // Load up indices and fileinfo. This also loads Bloom filter type.
380     metadataMap = Collections.unmodifiableMap(this.reader.loadFileInfo());
381 
382     // Read in our metadata.
383     byte [] b = metadataMap.get(MAX_SEQ_ID_KEY);
384     if (b != null) {
385       // By convention, if halfhfile, top half has a sequence number > bottom
386       // half. Thats why we add one in below. Its done for case the two halves
387       // are ever merged back together --rare.  Without it, on open of store,
388       // since store files are distinguished by sequence id, the one half would
389       // subsume the other.
390       this.sequenceid = Bytes.toLong(b);
391       if (fileInfo.isTopReference()) {
392         this.sequenceid += 1;
393       }
394     }
395 
396     if (isBulkLoadResult()){
397       // generate the sequenceId from the fileName
398       // fileName is of the form <randomName>_SeqId_<id-when-loaded>_
399       String fileName = this.getPath().getName();
400       // Use lastIndexOf() to get the last, most recent bulk load seqId.
401       int startPos = fileName.lastIndexOf("SeqId_");
402       if (startPos != -1) {
403         this.sequenceid = Long.parseLong(fileName.substring(startPos + 6,
404             fileName.indexOf('_', startPos + 6)));
405         // Handle reference files as done above.
406         if (fileInfo.isTopReference()) {
407           this.sequenceid += 1;
408         }
409       }
410       this.reader.setBulkLoaded(true);
411     }
412     this.reader.setSequenceID(this.sequenceid);
413 
414     b = metadataMap.get(HFileWriterV2.MAX_MEMSTORE_TS_KEY);
415     if (b != null) {
416       this.maxMemstoreTS = Bytes.toLong(b);
417     }
418 
419     b = metadataMap.get(MAJOR_COMPACTION_KEY);
420     if (b != null) {
421       boolean mc = Bytes.toBoolean(b);
422       if (this.majorCompaction == null) {
423         this.majorCompaction = new AtomicBoolean(mc);
424       } else {
425         this.majorCompaction.set(mc);
426       }
427     } else {
428       // Presume it is not major compacted if it doesn't explicity say so
429       // HFileOutputFormat explicitly sets the major compacted key.
430       this.majorCompaction = new AtomicBoolean(false);
431     }
432 
433     b = metadataMap.get(EXCLUDE_FROM_MINOR_COMPACTION_KEY);
434     this.excludeFromMinorCompaction = (b != null && Bytes.toBoolean(b));
435 
436     BloomType hfileBloomType = reader.getBloomFilterType();
437     if (cfBloomType != BloomType.NONE) {
438       reader.loadBloomfilter(BlockType.GENERAL_BLOOM_META);
439       if (hfileBloomType != cfBloomType) {
440         LOG.info("HFile Bloom filter type for "
441             + reader.getHFileReader().getName() + ": " + hfileBloomType
442             + ", but " + cfBloomType + " specified in column family "
443             + "configuration");
444       }
445     } else if (hfileBloomType != BloomType.NONE) {
446       LOG.info("Bloom filter turned off by CF config for "
447           + reader.getHFileReader().getName());
448     }
449 
450     // load delete family bloom filter
451     reader.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META);
452 
453     try {
454       byte [] timerangeBytes = metadataMap.get(TIMERANGE_KEY);
455       if (timerangeBytes != null) {
456         this.reader.timeRangeTracker = new TimeRangeTracker();
457         Writables.copyWritable(timerangeBytes, this.reader.timeRangeTracker);
458       }
459     } catch (IllegalArgumentException e) {
460       LOG.error("Error reading timestamp range data from meta -- " +
461           "proceeding without", e);
462       this.reader.timeRangeTracker = null;
463     }
464     return this.reader;
465   }
466 
467   /**
468    * @return Reader for StoreFile. creates if necessary
469    * @throws IOException
470    */
471   public Reader createReader() throws IOException {
472     if (this.reader == null) {
473       try {
474         this.reader = open();
475       } catch (IOException e) {
476         try {
477           this.closeReader(true);
478         } catch (IOException ee) {
479         }
480         throw e;
481       }
482 
483     }
484     return this.reader;
485   }
486 
487   /**
488    * @return Current reader.  Must call createReader first else returns null.
489    * @see #createReader()
490    */
491   public Reader getReader() {
492     return this.reader;
493   }
494 
495   /**
496    * @param evictOnClose whether to evict blocks belonging to this file
497    * @throws IOException
498    */
499   public synchronized void closeReader(boolean evictOnClose)
500       throws IOException {
501     if (this.reader != null) {
502       this.reader.close(evictOnClose);
503       this.reader = null;
504     }
505   }
506 
507   /**
508    * Delete this file
509    * @throws IOException
510    */
511   public void deleteReader() throws IOException {
512     closeReader(true);
513     this.fs.delete(getPath(), true);
514   }
515 
516   @Override
517   public String toString() {
518     return this.fileInfo.toString();
519   }
520 
521   /**
522    * @return a length description of this StoreFile, suitable for debug output
523    */
524   public String toStringDetailed() {
525     StringBuilder sb = new StringBuilder();
526     sb.append(this.getPath().toString());
527     sb.append(", isReference=").append(isReference());
528     sb.append(", isBulkLoadResult=").append(isBulkLoadResult());
529     if (isBulkLoadResult()) {
530       sb.append(", bulkLoadTS=").append(getBulkLoadTimestamp());
531     } else {
532       sb.append(", seqid=").append(getMaxSequenceId());
533     }
534     sb.append(", majorCompaction=").append(isMajorCompaction());
535 
536     return sb.toString();
537   }
538 
539   public static class WriterBuilder {
540     private final Configuration conf;
541     private final CacheConfig cacheConf;
542     private final FileSystem fs;
543 
544     private KeyValue.KVComparator comparator = KeyValue.COMPARATOR;
545     private BloomType bloomType = BloomType.NONE;
546     private long maxKeyCount = 0;
547     private Path dir;
548     private Path filePath;
549     private InetSocketAddress[] favoredNodes;
550     private HFileContext fileContext;
551     public WriterBuilder(Configuration conf, CacheConfig cacheConf,
552         FileSystem fs) {
553       this.conf = conf;
554       this.cacheConf = cacheConf;
555       this.fs = fs;
556     }
557 
558     /**
559      * Use either this method or {@link #withFilePath}, but not both.
560      * @param dir Path to column family directory. The directory is created if
561      *          does not exist. The file is given a unique name within this
562      *          directory.
563      * @return this (for chained invocation)
564      */
565     public WriterBuilder withOutputDir(Path dir) {
566       Preconditions.checkNotNull(dir);
567       this.dir = dir;
568       return this;
569     }
570 
571     /**
572      * Use either this method or {@link #withOutputDir}, but not both.
573      * @param filePath the StoreFile path to write
574      * @return this (for chained invocation)
575      */
576     public WriterBuilder withFilePath(Path filePath) {
577       Preconditions.checkNotNull(filePath);
578       this.filePath = filePath;
579       return this;
580     }
581 
582     /**
583      * @param favoredNodes an array of favored nodes or possibly null
584      * @return this (for chained invocation)
585      */
586     public WriterBuilder withFavoredNodes(InetSocketAddress[] favoredNodes) {
587       this.favoredNodes = favoredNodes;
588       return this;
589     }
590 
591     public WriterBuilder withComparator(KeyValue.KVComparator comparator) {
592       Preconditions.checkNotNull(comparator);
593       this.comparator = comparator;
594       return this;
595     }
596 
597     public WriterBuilder withBloomType(BloomType bloomType) {
598       Preconditions.checkNotNull(bloomType);
599       this.bloomType = bloomType;
600       return this;
601     }
602 
603     /**
604      * @param maxKeyCount estimated maximum number of keys we expect to add
605      * @return this (for chained invocation)
606      */
607     public WriterBuilder withMaxKeyCount(long maxKeyCount) {
608       this.maxKeyCount = maxKeyCount;
609       return this;
610     }
611 
612     public WriterBuilder withFileContext(HFileContext fileContext) {
613       this.fileContext = fileContext;
614       return this;
615     }
616     /**
617      * Create a store file writer. Client is responsible for closing file when
618      * done. If metadata, add BEFORE closing using
619      * {@link Writer#appendMetadata}.
620      */
621     public Writer build() throws IOException {
622       if ((dir == null ? 0 : 1) + (filePath == null ? 0 : 1) != 1) {
623         throw new IllegalArgumentException("Either specify parent directory " +
624             "or file path");
625       }
626 
627       if (dir == null) {
628         dir = filePath.getParent();
629       }
630 
631       if (!fs.exists(dir)) {
632         fs.mkdirs(dir);
633       }
634 
635       if (filePath == null) {
636         filePath = getUniqueFile(fs, dir);
637         if (!BloomFilterFactory.isGeneralBloomEnabled(conf)) {
638           bloomType = BloomType.NONE;
639         }
640       }
641 
642       if (comparator == null) {
643         comparator = KeyValue.COMPARATOR;
644       }
645       return new Writer(fs, filePath,
646           conf, cacheConf, comparator, bloomType, maxKeyCount, favoredNodes, fileContext);
647     }
648   }
649 
650   /**
651    * @param fs
652    * @param dir Directory to create file in.
653    * @return random filename inside passed <code>dir</code>
654    */
655   public static Path getUniqueFile(final FileSystem fs, final Path dir)
656       throws IOException {
657     if (!fs.getFileStatus(dir).isDirectory()) {
658       throw new IOException("Expecting " + dir.toString() +
659         " to be a directory");
660     }
661     return new Path(dir, UUID.randomUUID().toString().replaceAll("-", ""));
662   }
663 
664   public Long getMinimumTimestamp() {
665     return (getReader().timeRangeTracker == null) ?
666         null :
667         getReader().timeRangeTracker.getMinimumTimestamp();
668   }
669 
670   /**
671    * Gets the approximate mid-point of this file that is optimal for use in splitting it.
672    * @param comparator Comparator used to compare KVs.
673    * @return The split point row, or null if splitting is not possible, or reader is null.
674    */
675   @SuppressWarnings("deprecation")
676   byte[] getFileSplitPoint(KVComparator comparator) throws IOException {
677     if (this.reader == null) {
678       LOG.warn("Storefile " + this + " Reader is null; cannot get split point");
679       return null;
680     }
681     // Get first, last, and mid keys.  Midkey is the key that starts block
682     // in middle of hfile.  Has column and timestamp.  Need to return just
683     // the row we want to split on as midkey.
684     byte [] midkey = this.reader.midkey();
685     if (midkey != null) {
686       KeyValue mk = KeyValue.createKeyValueFromKey(midkey, 0, midkey.length);
687       byte [] fk = this.reader.getFirstKey();
688       KeyValue firstKey = KeyValue.createKeyValueFromKey(fk, 0, fk.length);
689       byte [] lk = this.reader.getLastKey();
690       KeyValue lastKey = KeyValue.createKeyValueFromKey(lk, 0, lk.length);
691       // if the midkey is the same as the first or last keys, we cannot (ever) split this region.
692       if (comparator.compareRows(mk, firstKey) == 0 || comparator.compareRows(mk, lastKey) == 0) {
693         if (LOG.isDebugEnabled()) {
694           LOG.debug("cannot split because midkey is the same as first or last row");
695         }
696         return null;
697       }
698       return mk.getRow();
699     }
700     return null;
701   }
702 
703   /**
704    * A StoreFile writer.  Use this to read/write HBase Store Files. It is package
705    * local because it is an implementation detail of the HBase regionserver.
706    */
707   public static class Writer implements Compactor.CellSink {
708     private final BloomFilterWriter generalBloomFilterWriter;
709     private final BloomFilterWriter deleteFamilyBloomFilterWriter;
710     private final BloomType bloomType;
711     private byte[] lastBloomKey;
712     private int lastBloomKeyOffset, lastBloomKeyLen;
713     private KVComparator kvComparator;
714     private Cell lastCell = null;
715     private long earliestPutTs = HConstants.LATEST_TIMESTAMP;
716     private Cell lastDeleteFamilyCell = null;
717     private long deleteFamilyCnt = 0;
718 
719     /** Bytes per Checksum */
720     protected int bytesPerChecksum;
721 
722     TimeRangeTracker timeRangeTracker = new TimeRangeTracker();
723     /* isTimeRangeTrackerSet keeps track if the timeRange has already been set
724      * When flushing a memstore, we set TimeRange and use this variable to
725      * indicate that it doesn't need to be calculated again while
726      * appending KeyValues.
727      * It is not set in cases of compactions when it is recalculated using only
728      * the appended KeyValues*/
729     boolean isTimeRangeTrackerSet = false;
730 
731     protected HFile.Writer writer;
732 
733     /**
734      * Creates an HFile.Writer that also write helpful meta data.
735      * @param fs file system to write to
736      * @param path file name to create
737      * @param conf user configuration
738      * @param comparator key comparator
739      * @param bloomType bloom filter setting
740      * @param maxKeys the expected maximum number of keys to be added. Was used
741      *        for Bloom filter size in {@link HFile} format version 1.
742      * @param favoredNodes
743      * @param fileContext - The HFile context
744      * @throws IOException problem writing to FS
745      */
746     private Writer(FileSystem fs, Path path,
747         final Configuration conf,
748         CacheConfig cacheConf,
749         final KVComparator comparator, BloomType bloomType, long maxKeys,
750         InetSocketAddress[] favoredNodes, HFileContext fileContext)
751             throws IOException {
752       writer = HFile.getWriterFactory(conf, cacheConf)
753           .withPath(fs, path)
754           .withComparator(comparator)
755           .withFavoredNodes(favoredNodes)
756           .withFileContext(fileContext)
757           .create();
758 
759       this.kvComparator = comparator;
760 
761       generalBloomFilterWriter = BloomFilterFactory.createGeneralBloomAtWrite(
762           conf, cacheConf, bloomType,
763           (int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
764 
765       if (generalBloomFilterWriter != null) {
766         this.bloomType = bloomType;
767         if (LOG.isTraceEnabled()) LOG.trace("Bloom filter type for " + path + ": " +
768           this.bloomType + ", " + generalBloomFilterWriter.getClass().getSimpleName());
769       } else {
770         // Not using Bloom filters.
771         this.bloomType = BloomType.NONE;
772       }
773 
774       // initialize delete family Bloom filter when there is NO RowCol Bloom
775       // filter
776       if (this.bloomType != BloomType.ROWCOL) {
777         this.deleteFamilyBloomFilterWriter = BloomFilterFactory
778             .createDeleteBloomAtWrite(conf, cacheConf,
779                 (int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
780       } else {
781         deleteFamilyBloomFilterWriter = null;
782       }
783       if (deleteFamilyBloomFilterWriter != null) {
784         if (LOG.isTraceEnabled()) LOG.trace("Delete Family Bloom filter type for " + path + ": "
785             + deleteFamilyBloomFilterWriter.getClass().getSimpleName());
786       }
787     }
788 
789     /**
790      * Writes meta data.
791      * Call before {@link #close()} since its written as meta data to this file.
792      * @param maxSequenceId Maximum sequence id.
793      * @param majorCompaction True if this file is product of a major compaction
794      * @throws IOException problem writing to FS
795      */
796     public void appendMetadata(final long maxSequenceId, final boolean majorCompaction)
797     throws IOException {
798       writer.appendFileInfo(MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId));
799       writer.appendFileInfo(MAJOR_COMPACTION_KEY,
800           Bytes.toBytes(majorCompaction));
801       appendTrackedTimestampsToMetadata();
802     }
803 
804     /**
805      * Add TimestampRange and earliest put timestamp to Metadata
806      */
807     public void appendTrackedTimestampsToMetadata() throws IOException {
808       appendFileInfo(TIMERANGE_KEY,WritableUtils.toByteArray(timeRangeTracker));
809       appendFileInfo(EARLIEST_PUT_TS, Bytes.toBytes(earliestPutTs));
810     }
811 
812     /**
813      * Set TimeRangeTracker
814      * @param trt
815      */
816     public void setTimeRangeTracker(final TimeRangeTracker trt) {
817       this.timeRangeTracker = trt;
818       isTimeRangeTrackerSet = true;
819     }
820 
821     /**
822      * Record the earlest Put timestamp.
823      *
824      * If the timeRangeTracker is not set,
825      * update TimeRangeTracker to include the timestamp of this key
826      * @param cell
827      */
828     public void trackTimestamps(final Cell cell) {
829       if (KeyValue.Type.Put.getCode() == cell.getTypeByte()) {
830         earliestPutTs = Math.min(earliestPutTs, cell.getTimestamp());
831       }
832       if (!isTimeRangeTrackerSet) {
833         timeRangeTracker.includeTimestamp(cell);
834       }
835     }
836 
837     private void appendGeneralBloomfilter(final Cell cell) throws IOException {
838       if (this.generalBloomFilterWriter != null) {
839         // only add to the bloom filter on a new, unique key
840         boolean newKey = true;
841         if (this.lastCell != null) {
842           switch(bloomType) {
843           case ROW:
844             newKey = ! kvComparator.matchingRows(cell, lastCell);
845             break;
846           case ROWCOL:
847             newKey = ! kvComparator.matchingRowColumn(cell, lastCell);
848             break;
849           case NONE:
850             newKey = false;
851             break;
852           default:
853             throw new IOException("Invalid Bloom filter type: " + bloomType +
854                 " (ROW or ROWCOL expected)");
855           }
856         }
857         if (newKey) {
858           /*
859            * http://2.bp.blogspot.com/_Cib_A77V54U/StZMrzaKufI/AAAAAAAAADo/ZhK7bGoJdMQ/s400/KeyValue.png
860            * Key = RowLen + Row + FamilyLen + Column [Family + Qualifier] + TimeStamp
861            *
862            * 2 Types of Filtering:
863            *  1. Row = Row
864            *  2. RowCol = Row + Qualifier
865            */
866           byte[] bloomKey;
867           int bloomKeyOffset, bloomKeyLen;
868 
869           switch (bloomType) {
870           case ROW:
871             bloomKey = cell.getRowArray();
872             bloomKeyOffset = cell.getRowOffset();
873             bloomKeyLen = cell.getRowLength();
874             break;
875           case ROWCOL:
876             // merge(row, qualifier)
877             // TODO: could save one buffer copy in case of compound Bloom
878             // filters when this involves creating a KeyValue
879             bloomKey = generalBloomFilterWriter.createBloomKey(cell.getRowArray(),
880                 cell.getRowOffset(), cell.getRowLength(), cell.getQualifierArray(),
881                 cell.getQualifierOffset(), cell.getQualifierLength());
882             bloomKeyOffset = 0;
883             bloomKeyLen = bloomKey.length;
884             break;
885           default:
886             throw new IOException("Invalid Bloom filter type: " + bloomType +
887                 " (ROW or ROWCOL expected)");
888           }
889           generalBloomFilterWriter.add(bloomKey, bloomKeyOffset, bloomKeyLen);
890           if (lastBloomKey != null
891               && generalBloomFilterWriter.getComparator().compareFlatKey(bloomKey,
892                   bloomKeyOffset, bloomKeyLen, lastBloomKey,
893                   lastBloomKeyOffset, lastBloomKeyLen) <= 0) {
894             throw new IOException("Non-increasing Bloom keys: "
895                 + Bytes.toStringBinary(bloomKey, bloomKeyOffset, bloomKeyLen)
896                 + " after "
897                 + Bytes.toStringBinary(lastBloomKey, lastBloomKeyOffset,
898                     lastBloomKeyLen));
899           }
900           lastBloomKey = bloomKey;
901           lastBloomKeyOffset = bloomKeyOffset;
902           lastBloomKeyLen = bloomKeyLen;
903           this.lastCell = cell;
904         }
905       }
906     }
907 
908     private void appendDeleteFamilyBloomFilter(final Cell cell)
909         throws IOException {
910       if (!CellUtil.isDeleteFamily(cell) && !CellUtil.isDeleteFamilyVersion(cell)) {
911         return;
912       }
913 
914       // increase the number of delete family in the store file
915       deleteFamilyCnt++;
916       if (null != this.deleteFamilyBloomFilterWriter) {
917         boolean newKey = true;
918         if (lastDeleteFamilyCell != null) {
919           newKey = !kvComparator.matchingRows(cell, lastDeleteFamilyCell);
920         }
921         if (newKey) {
922           this.deleteFamilyBloomFilterWriter.add(cell.getRowArray(),
923               cell.getRowOffset(), cell.getRowLength());
924           this.lastDeleteFamilyCell = cell;
925         }
926       }
927     }
928 
929     public void append(final Cell cell) throws IOException {
930       appendGeneralBloomfilter(cell);
931       appendDeleteFamilyBloomFilter(cell);
932       writer.append(cell);
933       trackTimestamps(cell);
934     }
935 
936     public Path getPath() {
937       return this.writer.getPath();
938     }
939 
940     boolean hasGeneralBloom() {
941       return this.generalBloomFilterWriter != null;
942     }
943 
944     /**
945      * For unit testing only.
946      *
947      * @return the Bloom filter used by this writer.
948      */
949     BloomFilterWriter getGeneralBloomWriter() {
950       return generalBloomFilterWriter;
951     }
952 
953     private boolean closeBloomFilter(BloomFilterWriter bfw) throws IOException {
954       boolean haveBloom = (bfw != null && bfw.getKeyCount() > 0);
955       if (haveBloom) {
956         bfw.compactBloom();
957       }
958       return haveBloom;
959     }
960 
961     private boolean closeGeneralBloomFilter() throws IOException {
962       boolean hasGeneralBloom = closeBloomFilter(generalBloomFilterWriter);
963 
964       // add the general Bloom filter writer and append file info
965       if (hasGeneralBloom) {
966         writer.addGeneralBloomFilter(generalBloomFilterWriter);
967         writer.appendFileInfo(BLOOM_FILTER_TYPE_KEY,
968             Bytes.toBytes(bloomType.toString()));
969         if (lastBloomKey != null) {
970           writer.appendFileInfo(LAST_BLOOM_KEY, Arrays.copyOfRange(
971               lastBloomKey, lastBloomKeyOffset, lastBloomKeyOffset
972                   + lastBloomKeyLen));
973         }
974       }
975       return hasGeneralBloom;
976     }
977 
978     private boolean closeDeleteFamilyBloomFilter() throws IOException {
979       boolean hasDeleteFamilyBloom = closeBloomFilter(deleteFamilyBloomFilterWriter);
980 
981       // add the delete family Bloom filter writer
982       if (hasDeleteFamilyBloom) {
983         writer.addDeleteFamilyBloomFilter(deleteFamilyBloomFilterWriter);
984       }
985 
986       // append file info about the number of delete family kvs
987       // even if there is no delete family Bloom.
988       writer.appendFileInfo(DELETE_FAMILY_COUNT,
989           Bytes.toBytes(this.deleteFamilyCnt));
990 
991       return hasDeleteFamilyBloom;
992     }
993 
994     public void close() throws IOException {
995       boolean hasGeneralBloom = this.closeGeneralBloomFilter();
996       boolean hasDeleteFamilyBloom = this.closeDeleteFamilyBloomFilter();
997 
998       writer.close();
999 
1000       // Log final Bloom filter statistics. This needs to be done after close()
1001       // because compound Bloom filters might be finalized as part of closing.
1002       if (StoreFile.LOG.isTraceEnabled()) {
1003         StoreFile.LOG.trace((hasGeneralBloom ? "" : "NO ") + "General Bloom and " +
1004           (hasDeleteFamilyBloom ? "" : "NO ") + "DeleteFamily" + " was added to HFile " +
1005           getPath());
1006       }
1007 
1008     }
1009 
1010     public void appendFileInfo(byte[] key, byte[] value) throws IOException {
1011       writer.appendFileInfo(key, value);
1012     }
1013 
1014     /** For use in testing, e.g. {@link org.apache.hadoop.hbase.regionserver.CreateRandomStoreFile}
1015      */
1016     HFile.Writer getHFileWriter() {
1017       return writer;
1018     }
1019   }
1020 
1021   /**
1022    * Reader for a StoreFile.
1023    */
1024   public static class Reader {
1025     static final Log LOG = LogFactory.getLog(Reader.class.getName());
1026 
1027     protected BloomFilter generalBloomFilter = null;
1028     protected BloomFilter deleteFamilyBloomFilter = null;
1029     protected BloomType bloomFilterType;
1030     private final HFile.Reader reader;
1031     protected TimeRangeTracker timeRangeTracker = null;
1032     protected long sequenceID = -1;
1033     private byte[] lastBloomKey;
1034     private long deleteFamilyCnt = -1;
1035     private boolean bulkLoadResult = false;
1036 
1037     public Reader(FileSystem fs, Path path, CacheConfig cacheConf, Configuration conf)
1038         throws IOException {
1039       reader = HFile.createReader(fs, path, cacheConf, conf);
1040       bloomFilterType = BloomType.NONE;
1041     }
1042 
1043     public Reader(FileSystem fs, Path path, FSDataInputStreamWrapper in, long size,
1044         CacheConfig cacheConf, Configuration conf) throws IOException {
1045       reader = HFile.createReader(fs, path, in, size, cacheConf, conf);
1046       bloomFilterType = BloomType.NONE;
1047     }
1048 
1049     /**
1050      * ONLY USE DEFAULT CONSTRUCTOR FOR UNIT TESTS
1051      */
1052     Reader() {
1053       this.reader = null;
1054     }
1055 
1056     public KVComparator getComparator() {
1057       return reader.getComparator();
1058     }
1059 
1060     /**
1061      * Get a scanner to scan over this StoreFile. Do not use
1062      * this overload if using this scanner for compactions.
1063      *
1064      * @param cacheBlocks should this scanner cache blocks?
1065      * @param pread use pread (for highly concurrent small readers)
1066      * @return a scanner
1067      */
1068     public StoreFileScanner getStoreFileScanner(boolean cacheBlocks,
1069                                                boolean pread) {
1070       return getStoreFileScanner(cacheBlocks, pread, false,
1071         // 0 is passed as readpoint because this method is only used by test
1072         // where StoreFile is directly operated upon
1073         0);
1074     }
1075 
1076     /**
1077      * Get a scanner to scan over this StoreFile.
1078      *
1079      * @param cacheBlocks should this scanner cache blocks?
1080      * @param pread use pread (for highly concurrent small readers)
1081      * @param isCompaction is scanner being used for compaction?
1082      * @return a scanner
1083      */
1084     public StoreFileScanner getStoreFileScanner(boolean cacheBlocks,
1085                                                boolean pread,
1086                                                boolean isCompaction, long readPt) {
1087       return new StoreFileScanner(this,
1088                                  getScanner(cacheBlocks, pread, isCompaction),
1089                                  !isCompaction, reader.hasMVCCInfo(), readPt);
1090     }
1091 
1092     /**
1093      * Warning: Do not write further code which depends on this call. Instead
1094      * use getStoreFileScanner() which uses the StoreFileScanner class/interface
1095      * which is the preferred way to scan a store with higher level concepts.
1096      *
1097      * @param cacheBlocks should we cache the blocks?
1098      * @param pread use pread (for concurrent small readers)
1099      * @return the underlying HFileScanner
1100      */
1101     @Deprecated
1102     public HFileScanner getScanner(boolean cacheBlocks, boolean pread) {
1103       return getScanner(cacheBlocks, pread, false);
1104     }
1105 
1106     /**
1107      * Warning: Do not write further code which depends on this call. Instead
1108      * use getStoreFileScanner() which uses the StoreFileScanner class/interface
1109      * which is the preferred way to scan a store with higher level concepts.
1110      *
1111      * @param cacheBlocks
1112      *          should we cache the blocks?
1113      * @param pread
1114      *          use pread (for concurrent small readers)
1115      * @param isCompaction
1116      *          is scanner being used for compaction?
1117      * @return the underlying HFileScanner
1118      */
1119     @Deprecated
1120     public HFileScanner getScanner(boolean cacheBlocks, boolean pread,
1121         boolean isCompaction) {
1122       return reader.getScanner(cacheBlocks, pread, isCompaction);
1123     }
1124 
1125     public void close(boolean evictOnClose) throws IOException {
1126       reader.close(evictOnClose);
1127     }
1128 
1129     /**
1130      * Check if this storeFile may contain keys within the TimeRange that
1131      * have not expired (i.e. not older than oldestUnexpiredTS).
1132      * @param scan the current scan
1133      * @param oldestUnexpiredTS the oldest timestamp that is not expired, as
1134      *          determined by the column family's TTL
1135      * @return false if queried keys definitely don't exist in this StoreFile
1136      */
1137     boolean passesTimerangeFilter(Scan scan, long oldestUnexpiredTS) {
1138       if (timeRangeTracker == null) {
1139         return true;
1140       } else {
1141         return timeRangeTracker.includesTimeRange(scan.getTimeRange()) &&
1142             timeRangeTracker.getMaximumTimestamp() >= oldestUnexpiredTS;
1143       }
1144     }
1145 
1146     /**
1147      * Checks whether the given scan passes the Bloom filter (if present). Only
1148      * checks Bloom filters for single-row or single-row-column scans. Bloom
1149      * filter checking for multi-gets is implemented as part of the store
1150      * scanner system (see {@link StoreFileScanner#seekExactly}) and uses
1151      * the lower-level API {@link #passesGeneralBloomFilter(byte[], int, int, byte[],
1152      * int, int)}.
1153      *
1154      * @param scan the scan specification. Used to determine the row, and to
1155      *          check whether this is a single-row ("get") scan.
1156      * @param columns the set of columns. Only used for row-column Bloom
1157      *          filters.
1158      * @return true if the scan with the given column set passes the Bloom
1159      *         filter, or if the Bloom filter is not applicable for the scan.
1160      *         False if the Bloom filter is applicable and the scan fails it.
1161      */
1162      boolean passesBloomFilter(Scan scan,
1163         final SortedSet<byte[]> columns) {
1164       // Multi-column non-get scans will use Bloom filters through the
1165       // lower-level API function that this function calls.
1166       if (!scan.isGetScan()) {
1167         return true;
1168       }
1169 
1170       byte[] row = scan.getStartRow();
1171       switch (this.bloomFilterType) {
1172         case ROW:
1173           return passesGeneralBloomFilter(row, 0, row.length, null, 0, 0);
1174 
1175         case ROWCOL:
1176           if (columns != null && columns.size() == 1) {
1177             byte[] column = columns.first();
1178             return passesGeneralBloomFilter(row, 0, row.length, column, 0,
1179                 column.length);
1180           }
1181 
1182           // For multi-column queries the Bloom filter is checked from the
1183           // seekExact operation.
1184           return true;
1185 
1186         default:
1187           return true;
1188       }
1189     }
1190 
1191     public boolean passesDeleteFamilyBloomFilter(byte[] row, int rowOffset,
1192         int rowLen) {
1193       // Cache Bloom filter as a local variable in case it is set to null by
1194       // another thread on an IO error.
1195       BloomFilter bloomFilter = this.deleteFamilyBloomFilter;
1196 
1197       // Empty file or there is no delete family at all
1198       if (reader.getTrailer().getEntryCount() == 0 || deleteFamilyCnt == 0) {
1199         return false;
1200       }
1201 
1202       if (bloomFilter == null) {
1203         return true;
1204       }
1205 
1206       try {
1207         if (!bloomFilter.supportsAutoLoading()) {
1208           return true;
1209         }
1210         return bloomFilter.contains(row, rowOffset, rowLen, null);
1211       } catch (IllegalArgumentException e) {
1212         LOG.error("Bad Delete Family bloom filter data -- proceeding without",
1213             e);
1214         setDeleteFamilyBloomFilterFaulty();
1215       }
1216 
1217       return true;
1218     }
1219 
1220     /**
1221      * A method for checking Bloom filters. Called directly from
1222      * StoreFileScanner in case of a multi-column query.
1223      *
1224      * @param row
1225      * @param rowOffset
1226      * @param rowLen
1227      * @param col
1228      * @param colOffset
1229      * @param colLen
1230      * @return True if passes
1231      */
1232     public boolean passesGeneralBloomFilter(byte[] row, int rowOffset,
1233         int rowLen, byte[] col, int colOffset, int colLen) {
1234       // Cache Bloom filter as a local variable in case it is set to null by
1235       // another thread on an IO error.
1236       BloomFilter bloomFilter = this.generalBloomFilter;
1237       if (bloomFilter == null) {
1238         return true;
1239       }
1240 
1241       byte[] key;
1242       switch (bloomFilterType) {
1243         case ROW:
1244           if (col != null) {
1245             throw new RuntimeException("Row-only Bloom filter called with " +
1246                 "column specified");
1247           }
1248           if (rowOffset != 0 || rowLen != row.length) {
1249               throw new AssertionError("For row-only Bloom filters the row "
1250                   + "must occupy the whole array");
1251           }
1252           key = row;
1253           break;
1254 
1255         case ROWCOL:
1256           key = bloomFilter.createBloomKey(row, rowOffset, rowLen, col,
1257               colOffset, colLen);
1258           break;
1259 
1260         default:
1261           return true;
1262       }
1263 
1264       // Empty file
1265       if (reader.getTrailer().getEntryCount() == 0)
1266         return false;
1267 
1268       try {
1269         boolean shouldCheckBloom;
1270         ByteBuffer bloom;
1271         if (bloomFilter.supportsAutoLoading()) {
1272           bloom = null;
1273           shouldCheckBloom = true;
1274         } else {
1275           bloom = reader.getMetaBlock(HFile.BLOOM_FILTER_DATA_KEY,
1276               true);
1277           shouldCheckBloom = bloom != null;
1278         }
1279 
1280         if (shouldCheckBloom) {
1281           boolean exists;
1282 
1283           // Whether the primary Bloom key is greater than the last Bloom key
1284           // from the file info. For row-column Bloom filters this is not yet
1285           // a sufficient condition to return false.
1286           boolean keyIsAfterLast = lastBloomKey != null
1287               && bloomFilter.getComparator().compareFlatKey(key, lastBloomKey) > 0;
1288 
1289           if (bloomFilterType == BloomType.ROWCOL) {
1290             // Since a Row Delete is essentially a DeleteFamily applied to all
1291             // columns, a file might be skipped if using row+col Bloom filter.
1292             // In order to ensure this file is included an additional check is
1293             // required looking only for a row bloom.
1294             byte[] rowBloomKey = bloomFilter.createBloomKey(row, 0, row.length,
1295                 null, 0, 0);
1296 
1297             if (keyIsAfterLast
1298                 && bloomFilter.getComparator().compareFlatKey(rowBloomKey,
1299                     lastBloomKey) > 0) {
1300               exists = false;
1301             } else {
1302               exists =
1303                   bloomFilter.contains(key, 0, key.length, bloom) ||
1304                   bloomFilter.contains(rowBloomKey, 0, rowBloomKey.length,
1305                       bloom);
1306             }
1307           } else {
1308             exists = !keyIsAfterLast
1309                 && bloomFilter.contains(key, 0, key.length, bloom);
1310           }
1311 
1312           return exists;
1313         }
1314       } catch (IOException e) {
1315         LOG.error("Error reading bloom filter data -- proceeding without",
1316             e);
1317         setGeneralBloomFilterFaulty();
1318       } catch (IllegalArgumentException e) {
1319         LOG.error("Bad bloom filter data -- proceeding without", e);
1320         setGeneralBloomFilterFaulty();
1321       }
1322 
1323       return true;
1324     }
1325 
1326     /**
1327      * Checks whether the given scan rowkey range overlaps with the current storefile's
1328      * @param scan the scan specification. Used to determine the rowkey range.
1329      * @return true if there is overlap, false otherwise
1330      */
1331     public boolean passesKeyRangeFilter(Scan scan) {
1332       if (this.getFirstKey() == null || this.getLastKey() == null) {
1333         // the file is empty
1334         return false;
1335       }
1336       if (Bytes.equals(scan.getStartRow(), HConstants.EMPTY_START_ROW)
1337           && Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW)) {
1338         return true;
1339       }
1340       KeyValue smallestScanKeyValue = scan.isReversed() ? KeyValueUtil
1341           .createFirstOnRow(scan.getStopRow()) : KeyValueUtil.createFirstOnRow(scan
1342           .getStartRow());
1343       KeyValue largestScanKeyValue = scan.isReversed() ? KeyValueUtil
1344           .createLastOnRow(scan.getStartRow()) : KeyValueUtil.createLastOnRow(scan
1345           .getStopRow());
1346       boolean nonOverLapping = (getComparator().compareFlatKey(
1347           this.getFirstKey(), largestScanKeyValue.getKey()) > 0 && !Bytes
1348           .equals(scan.isReversed() ? scan.getStartRow() : scan.getStopRow(),
1349               HConstants.EMPTY_END_ROW))
1350           || getComparator().compareFlatKey(this.getLastKey(),
1351               smallestScanKeyValue.getKey()) < 0;
1352       return !nonOverLapping;
1353     }
1354 
1355     public Map<byte[], byte[]> loadFileInfo() throws IOException {
1356       Map<byte [], byte []> fi = reader.loadFileInfo();
1357 
1358       byte[] b = fi.get(BLOOM_FILTER_TYPE_KEY);
1359       if (b != null) {
1360         bloomFilterType = BloomType.valueOf(Bytes.toString(b));
1361       }
1362 
1363       lastBloomKey = fi.get(LAST_BLOOM_KEY);
1364       byte[] cnt = fi.get(DELETE_FAMILY_COUNT);
1365       if (cnt != null) {
1366         deleteFamilyCnt = Bytes.toLong(cnt);
1367       }
1368 
1369       return fi;
1370     }
1371 
1372     public void loadBloomfilter() {
1373       this.loadBloomfilter(BlockType.GENERAL_BLOOM_META);
1374       this.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META);
1375     }
1376 
1377     private void loadBloomfilter(BlockType blockType) {
1378       try {
1379         if (blockType == BlockType.GENERAL_BLOOM_META) {
1380           if (this.generalBloomFilter != null)
1381             return; // Bloom has been loaded
1382 
1383           DataInput bloomMeta = reader.getGeneralBloomFilterMetadata();
1384           if (bloomMeta != null) {
1385             // sanity check for NONE Bloom filter
1386             if (bloomFilterType == BloomType.NONE) {
1387               throw new IOException(
1388                   "valid bloom filter type not found in FileInfo");
1389             } else {
1390               generalBloomFilter = BloomFilterFactory.createFromMeta(bloomMeta,
1391                   reader);
1392               if (LOG.isTraceEnabled()) {
1393                 LOG.trace("Loaded " + bloomFilterType.toString() + " "
1394                   + generalBloomFilter.getClass().getSimpleName()
1395                   + " metadata for " + reader.getName());
1396               }
1397             }
1398           }
1399         } else if (blockType == BlockType.DELETE_FAMILY_BLOOM_META) {
1400           if (this.deleteFamilyBloomFilter != null)
1401             return; // Bloom has been loaded
1402 
1403           DataInput bloomMeta = reader.getDeleteBloomFilterMetadata();
1404           if (bloomMeta != null) {
1405             deleteFamilyBloomFilter = BloomFilterFactory.createFromMeta(
1406                 bloomMeta, reader);
1407             LOG.info("Loaded Delete Family Bloom ("
1408                 + deleteFamilyBloomFilter.getClass().getSimpleName()
1409                 + ") metadata for " + reader.getName());
1410           }
1411         } else {
1412           throw new RuntimeException("Block Type: " + blockType.toString()
1413               + "is not supported for Bloom filter");
1414         }
1415       } catch (IOException e) {
1416         LOG.error("Error reading bloom filter meta for " + blockType
1417             + " -- proceeding without", e);
1418         setBloomFilterFaulty(blockType);
1419       } catch (IllegalArgumentException e) {
1420         LOG.error("Bad bloom filter meta " + blockType
1421             + " -- proceeding without", e);
1422         setBloomFilterFaulty(blockType);
1423       }
1424     }
1425 
1426     private void setBloomFilterFaulty(BlockType blockType) {
1427       if (blockType == BlockType.GENERAL_BLOOM_META) {
1428         setGeneralBloomFilterFaulty();
1429       } else if (blockType == BlockType.DELETE_FAMILY_BLOOM_META) {
1430         setDeleteFamilyBloomFilterFaulty();
1431       }
1432     }
1433 
1434     /**
1435      * The number of Bloom filter entries in this store file, or an estimate
1436      * thereof, if the Bloom filter is not loaded. This always returns an upper
1437      * bound of the number of Bloom filter entries.
1438      *
1439      * @return an estimate of the number of Bloom filter entries in this file
1440      */
1441     public long getFilterEntries() {
1442       return generalBloomFilter != null ? generalBloomFilter.getKeyCount()
1443           : reader.getEntries();
1444     }
1445 
1446     public void setGeneralBloomFilterFaulty() {
1447       generalBloomFilter = null;
1448     }
1449 
1450     public void setDeleteFamilyBloomFilterFaulty() {
1451       this.deleteFamilyBloomFilter = null;
1452     }
1453 
1454     public byte[] getLastKey() {
1455       return reader.getLastKey();
1456     }
1457 
1458     public byte[] getLastRowKey() {
1459       return reader.getLastRowKey();
1460     }
1461 
1462     public byte[] midkey() throws IOException {
1463       return reader.midkey();
1464     }
1465 
1466     public long length() {
1467       return reader.length();
1468     }
1469 
1470     public long getTotalUncompressedBytes() {
1471       return reader.getTrailer().getTotalUncompressedBytes();
1472     }
1473 
1474     public long getEntries() {
1475       return reader.getEntries();
1476     }
1477 
1478     public long getDeleteFamilyCnt() {
1479       return deleteFamilyCnt;
1480     }
1481 
1482     public byte[] getFirstKey() {
1483       return reader.getFirstKey();
1484     }
1485 
1486     public long indexSize() {
1487       return reader.indexSize();
1488     }
1489 
1490     public BloomType getBloomFilterType() {
1491       return this.bloomFilterType;
1492     }
1493 
1494     public long getSequenceID() {
1495       return sequenceID;
1496     }
1497 
1498     public void setSequenceID(long sequenceID) {
1499       this.sequenceID = sequenceID;
1500     }
1501 
1502     public void setBulkLoaded(boolean bulkLoadResult) {
1503       this.bulkLoadResult = bulkLoadResult;
1504     }
1505 
1506     public boolean isBulkLoaded() {
1507       return this.bulkLoadResult;
1508     }
1509 
1510     BloomFilter getGeneralBloomFilter() {
1511       return generalBloomFilter;
1512     }
1513 
1514     long getUncompressedDataIndexSize() {
1515       return reader.getTrailer().getUncompressedDataIndexSize();
1516     }
1517 
1518     public long getTotalBloomSize() {
1519       if (generalBloomFilter == null)
1520         return 0;
1521       return generalBloomFilter.getByteSize();
1522     }
1523 
1524     public int getHFileVersion() {
1525       return reader.getTrailer().getMajorVersion();
1526     }
1527 
1528     public int getHFileMinorVersion() {
1529       return reader.getTrailer().getMinorVersion();
1530     }
1531 
1532     public HFile.Reader getHFileReader() {
1533       return reader;
1534     }
1535 
1536     void disableBloomFilterForTesting() {
1537       generalBloomFilter = null;
1538       this.deleteFamilyBloomFilter = null;
1539     }
1540 
1541     public long getMaxTimestamp() {
1542       return timeRangeTracker == null ? Long.MAX_VALUE : timeRangeTracker.getMaximumTimestamp();
1543     }
1544   }
1545 
1546   /**
1547    * Useful comparators for comparing StoreFiles.
1548    */
1549   public abstract static class Comparators {
1550     /**
1551      * Comparator that compares based on the Sequence Ids of the
1552      * the StoreFiles. Bulk loads that did not request a seq ID
1553      * are given a seq id of -1; thus, they are placed before all non-
1554      * bulk loads, and bulk loads with sequence Id. Among these files,
1555      * the size is used to determine the ordering, then bulkLoadTime.
1556      * If there are ties, the path name is used as a tie-breaker.
1557      */
1558     public static final Comparator<StoreFile> SEQ_ID =
1559       Ordering.compound(ImmutableList.of(
1560           Ordering.natural().onResultOf(new GetSeqId()),
1561           Ordering.natural().onResultOf(new GetFileSize()).reverse(),
1562           Ordering.natural().onResultOf(new GetBulkTime()),
1563           Ordering.natural().onResultOf(new GetPathName())
1564       ));
1565 
1566     private static class GetSeqId implements Function<StoreFile, Long> {
1567       @Override
1568       public Long apply(StoreFile sf) {
1569         return sf.getMaxSequenceId();
1570       }
1571     }
1572 
1573     private static class GetFileSize implements Function<StoreFile, Long> {
1574       @Override
1575       public Long apply(StoreFile sf) {
1576         return sf.getReader().length();
1577       }
1578     }
1579 
1580     private static class GetBulkTime implements Function<StoreFile, Long> {
1581       @Override
1582       public Long apply(StoreFile sf) {
1583         if (!sf.isBulkLoadResult()) return Long.MAX_VALUE;
1584         return sf.getBulkLoadTimestamp();
1585       }
1586     }
1587 
1588     private static class GetPathName implements Function<StoreFile, String> {
1589       @Override
1590       public String apply(StoreFile sf) {
1591         return sf.getPath().getName();
1592       }
1593     }
1594   }
1595 }