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