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