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