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