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