View Javadoc

1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.regionserver;
21  
22  import java.io.DataInput;
23  import java.io.FileNotFoundException;
24  import java.io.IOException;
25  import java.nio.ByteBuffer;
26  import java.util.Arrays;
27  import java.util.Collection;
28  import java.util.Collections;
29  import java.util.Comparator;
30  import java.util.Map;
31  import java.util.SortedSet;
32  import java.util.UUID;
33  import java.util.concurrent.atomic.AtomicBoolean;
34  import java.util.regex.Matcher;
35  import java.util.regex.Pattern;
36  
37  import org.apache.commons.logging.Log;
38  import org.apache.commons.logging.LogFactory;
39  import org.apache.hadoop.conf.Configuration;
40  import org.apache.hadoop.fs.FSDataInputStream;
41  import org.apache.hadoop.fs.FileStatus;
42  import org.apache.hadoop.fs.FileSystem;
43  import org.apache.hadoop.fs.Path;
44  import org.apache.hadoop.hbase.HConstants;
45  import org.apache.hadoop.hbase.HDFSBlocksDistribution;
46  import org.apache.hadoop.hbase.HTableDescriptor;
47  import org.apache.hadoop.hbase.KeyValue;
48  import org.apache.hadoop.hbase.KeyValue.KVComparator;
49  import org.apache.hadoop.hbase.client.Scan;
50  import org.apache.hadoop.hbase.fs.HFileSystem;
51  import org.apache.hadoop.hbase.io.HFileLink;
52  import org.apache.hadoop.hbase.io.HalfStoreFileReader;
53  import org.apache.hadoop.hbase.io.Reference;
54  import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
55  import org.apache.hadoop.hbase.io.hfile.BlockType;
56  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
57  import org.apache.hadoop.hbase.io.hfile.Compression;
58  import org.apache.hadoop.hbase.io.hfile.HFile;
59  import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
60  import org.apache.hadoop.hbase.io.hfile.HFileScanner;
61  import org.apache.hadoop.hbase.io.hfile.HFileWriterV1;
62  import org.apache.hadoop.hbase.io.hfile.HFileWriterV2;
63  import org.apache.hadoop.hbase.io.hfile.NoOpDataBlockEncoder;
64  import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
65  import org.apache.hadoop.hbase.regionserver.metrics.SchemaConfigured;
66  import org.apache.hadoop.hbase.util.BloomFilter;
67  import org.apache.hadoop.hbase.util.BloomFilterFactory;
68  import org.apache.hadoop.hbase.util.BloomFilterWriter;
69  import org.apache.hadoop.hbase.util.Bytes;
70  import org.apache.hadoop.hbase.util.ChecksumType;
71  import org.apache.hadoop.hbase.util.FSUtils;
72  import org.apache.hadoop.hbase.util.Writables;
73  import org.apache.hadoop.io.RawComparator;
74  import org.apache.hadoop.io.WritableUtils;
75  
76  import com.google.common.base.Function;
77  import com.google.common.base.Preconditions;
78  import com.google.common.collect.ImmutableList;
79  import com.google.common.collect.Ordering;
80  
81  /**
82   * A Store data file.  Stores usually have one or more of these files.  They
83   * are produced by flushing the memstore to disk.  To
84   * create, instantiate a writer using {@link StoreFile#WriterBuilder}
85   * and append data. Be sure to add any metadata before calling close on the
86   * Writer (Use the appendMetadata convenience methods). On close, a StoreFile
87   * is sitting in the Filesystem.  To refer to it, create a StoreFile instance
88   * passing filesystem and path.  To read, call {@link #createReader()}.
89   * <p>StoreFiles may also reference store files in another Store.
90   *
91   * The reason for this weird pattern where you use a different instance for the
92   * writer and a reader is that we write once but read a lot more.
93   */
94  public class StoreFile extends SchemaConfigured {
95    static final Log LOG = LogFactory.getLog(StoreFile.class.getName());
96  
97    public static enum BloomType {
98      /**
99       * Bloomfilters disabled
100      */
101     NONE,
102     /**
103      * Bloom enabled with Table row as Key
104      */
105     ROW,
106     /**
107      * Bloom enabled with Table row & column (family+qualifier) as Key
108      */
109     ROWCOL
110   }
111 
112   // Keys for fileinfo values in HFile
113 
114   /** Max Sequence ID in FileInfo */
115   public static final byte [] MAX_SEQ_ID_KEY = Bytes.toBytes("MAX_SEQ_ID_KEY");
116 
117   /** Major compaction flag in FileInfo */
118   public static final byte[] MAJOR_COMPACTION_KEY =
119       Bytes.toBytes("MAJOR_COMPACTION_KEY");
120 
121   /** Major compaction flag in FileInfo */
122   public static final byte[] EXCLUDE_FROM_MINOR_COMPACTION_KEY =
123       Bytes.toBytes("EXCLUDE_FROM_MINOR_COMPACTION");
124 
125   /** Bloom filter Type in FileInfo */
126   static final byte[] BLOOM_FILTER_TYPE_KEY =
127       Bytes.toBytes("BLOOM_FILTER_TYPE");
128 
129   /** Delete Family Count in FileInfo */
130   public static final byte[] DELETE_FAMILY_COUNT =
131       Bytes.toBytes("DELETE_FAMILY_COUNT");
132 
133   /** Last Bloom filter key in FileInfo */
134   private static final byte[] LAST_BLOOM_KEY = Bytes.toBytes("LAST_BLOOM_KEY");
135 
136   /** Key for Timerange information in metadata*/
137   public static final byte[] TIMERANGE_KEY = Bytes.toBytes("TIMERANGE");
138 
139   /** Key for timestamp of earliest-put in metadata*/
140   public static final byte[] EARLIEST_PUT_TS = Bytes.toBytes("EARLIEST_PUT_TS");
141 
142   // Make default block size for StoreFiles 8k while testing.  TODO: FIX!
143   // Need to make it 8k for testing.
144   public static final int DEFAULT_BLOCKSIZE_SMALL = 8 * 1024;
145 
146   private final FileSystem fs;
147 
148   // This file's path.
149   private final Path path;
150 
151   // If this storefile references another, this is the reference instance.
152   private Reference reference;
153 
154   // If this StoreFile references another, this is the other files path.
155   private Path referencePath;
156 
157   // If this storefile is a link to another, this is the link instance.
158   private HFileLink link;
159 
160   // Block cache configuration and reference.
161   private final CacheConfig cacheConf;
162 
163   // What kind of data block encoding will be used
164   private final HFileDataBlockEncoder dataBlockEncoder;
165 
166   // HDFS blocks distribution information
167   private HDFSBlocksDistribution hdfsBlocksDistribution;
168 
169   // Keys for metadata stored in backing HFile.
170   // Set when we obtain a Reader.
171   private long sequenceid = -1;
172 
173   // max of the MemstoreTS in the KV's in this store
174   // Set when we obtain a Reader.
175   private long maxMemstoreTS = -1;
176 
177   public long getMaxMemstoreTS() {
178     return maxMemstoreTS;
179   }
180 
181   public void setMaxMemstoreTS(long maxMemstoreTS) {
182     this.maxMemstoreTS = maxMemstoreTS;
183   }
184 
185   // If true, this file was product of a major compaction.  Its then set
186   // whenever you get a Reader.
187   private AtomicBoolean majorCompaction = null;
188 
189   // If true, this file should not be included in minor compactions.
190   // It's set whenever you get a Reader.
191   private boolean excludeFromMinorCompaction = false;
192 
193   /** Meta key set when store file is a result of a bulk load */
194   public static final byte[] BULKLOAD_TASK_KEY =
195     Bytes.toBytes("BULKLOAD_SOURCE_TASK");
196   public static final byte[] BULKLOAD_TIME_KEY =
197     Bytes.toBytes("BULKLOAD_TIMESTAMP");
198 
199   /**
200    * Map of the metadata entries in the corresponding HFile
201    */
202   private Map<byte[], byte[]> metadataMap;
203 
204   /**
205    * A non-capture group, for hfiles, so that this can be embedded.
206    * HFiles are uuid ([0-9a-z]+). Bulk loaded hfiles has (_SeqId_[0-9]+_) has suffix.
207    */
208   public static final String HFILE_NAME_REGEX = "[0-9a-f]+(?:_SeqId_[0-9]+_)?";
209 
210   /** Regex that will work for hfiles */
211   private static final Pattern HFILE_NAME_PATTERN =
212     Pattern.compile("^(" + HFILE_NAME_REGEX + ")");
213 
214   /**
215    * Regex that will work for straight reference names (<hfile>.<parentEncRegion>)
216    * and hfilelink reference names (<table>=<region>-<hfile>.<parentEncRegion>)
217    * If reference, then the regex has more than just one group.
218    * Group 1, hfile/hfilelink pattern, is this file's id.
219    * Group 2 '(.+)' is the reference's parent region name.
220    */
221   private static final Pattern REF_NAME_PATTERN =
222     Pattern.compile(String.format("^(%s|%s)\\.(.+)$",
223       HFILE_NAME_REGEX, HFileLink.LINK_NAME_REGEX));
224 
225   // StoreFile.Reader
226   private volatile Reader reader;
227 
228   /**
229    * Bloom filter type specified in column family configuration. Does not
230    * necessarily correspond to the Bloom filter type present in the HFile.
231    */
232   private final BloomType cfBloomType;
233 
234   // the last modification time stamp
235   private long modificationTimeStamp = 0L;
236 
237   /**
238    * Constructor, loads a reader and it's indices, etc. May allocate a
239    * substantial amount of ram depending on the underlying files (10-20MB?).
240    *
241    * @param fs  The current file system to use.
242    * @param p  The path of the file.
243    * @param blockcache  <code>true</code> if the block cache is enabled.
244    * @param conf  The current configuration.
245    * @param cacheConf  The cache configuration and block cache reference.
246    * @param cfBloomType The bloom type to use for this store file as specified
247    *          by column family configuration. This may or may not be the same
248    *          as the Bloom filter type actually present in the HFile, because
249    *          column family configuration might change. If this is
250    *          {@link BloomType#NONE}, the existing Bloom filter is ignored.
251    * @param dataBlockEncoder data block encoding algorithm.
252    * @throws IOException When opening the reader fails.
253    */
254   public StoreFile(final FileSystem fs,
255             final Path p,
256             final Configuration conf,
257             final CacheConfig cacheConf,
258             final BloomType cfBloomType,
259             final HFileDataBlockEncoder dataBlockEncoder)
260       throws IOException {
261     this.fs = fs;
262     this.path = p;
263     this.cacheConf = cacheConf;
264     this.dataBlockEncoder =
265         dataBlockEncoder == null ? NoOpDataBlockEncoder.INSTANCE
266             : dataBlockEncoder;
267 
268     if (HFileLink.isHFileLink(p)) {
269       this.link = new HFileLink(conf, p);
270       LOG.debug("Store file " + p + " is a link");
271     } else if (isReference(p)) {
272       this.reference = Reference.read(fs, p);
273       this.referencePath = getReferredToFile(this.path);
274       if (HFileLink.isHFileLink(this.referencePath)) {
275         this.link = new HFileLink(conf, this.referencePath);
276       }
277       LOG.debug("Store file " + p + " is a " + reference.getFileRegion() +
278         " reference to " + this.referencePath);
279     } else if (!isHFile(p)) {
280       throw new IOException("path=" + path + " doesn't look like a valid StoreFile");
281     }
282 
283     if (BloomFilterFactory.isGeneralBloomEnabled(conf)) {
284       this.cfBloomType = cfBloomType;
285     } else {
286       LOG.info("Ignoring bloom filter check for file " + path + ": " +
287           "cfBloomType=" + cfBloomType + " (disabled in config)");
288       this.cfBloomType = BloomType.NONE;
289     }
290 
291     // cache the modification time stamp of this store file
292     FileStatus[] stats = FSUtils.listStatus(fs, p, null);
293     if (stats != null && stats.length == 1) {
294       this.modificationTimeStamp = stats[0].getModificationTime();
295     } else {
296       this.modificationTimeStamp = 0;
297     }
298 
299     SchemaMetrics.configureGlobally(conf);
300   }
301 
302   /**
303    * @return Path or null if this StoreFile was made with a Stream.
304    */
305   public Path getPath() {
306     return this.path;
307   }
308 
309   /**
310    * @return The Store/ColumnFamily this file belongs to.
311    */
312   byte [] getFamily() {
313     return Bytes.toBytes(this.path.getParent().getName());
314   }
315 
316   /**
317    * @return True if this is a StoreFile Reference; call after {@link #open()}
318    * else may get wrong answer.
319    */
320   boolean isReference() {
321     return this.reference != null;
322   }
323 
324   /**
325    * @return <tt>true</tt> if this StoreFile is an HFileLink
326    */
327   boolean isLink() {
328     return this.link != null && this.reference == null;
329   }
330 
331   private static boolean isHFile(final Path path) {
332     Matcher m = HFILE_NAME_PATTERN.matcher(path.getName());
333     return m.matches() && m.groupCount() > 0;
334   }
335 
336   /**
337    * @param p Path to check.
338    * @return True if the path has format of a HStoreFile reference.
339    */
340   public static boolean isReference(final Path p) {
341     return isReference(p.getName());
342   }
343 
344   /**
345    * @param name file name to check.
346    * @return True if the path has format of a HStoreFile reference.
347    */
348   public static boolean isReference(final String name) {
349     Matcher m = REF_NAME_PATTERN.matcher(name);
350     return m.matches() && m.groupCount() > 1;
351   }
352 
353   /*
354    * Return path to the file referred to by a Reference.  Presumes a directory
355    * hierarchy of <code>${hbase.rootdir}/tablename/regionname/familyname</code>.
356    * @param p Path to a Reference file.
357    * @return Calculated path to parent region file.
358    * @throws IllegalArgumentException when path regex fails to match.
359    */
360   public static Path getReferredToFile(final Path p) {
361     Matcher m = REF_NAME_PATTERN.matcher(p.getName());
362     if (m == null || !m.matches()) {
363       LOG.warn("Failed match of store file name " + p.toString());
364       throw new IllegalArgumentException("Failed match of store file name " +
365           p.toString());
366     }
367     // Other region name is suffix on the passed Reference file name
368     String otherRegion = m.group(2);
369     // Tabledir is up two directories from where Reference was written.
370     Path tableDir = p.getParent().getParent().getParent();
371     String nameStrippedOfSuffix = m.group(1);
372     LOG.debug("reference '" + p + "' to region=" + otherRegion + " hfile=" + nameStrippedOfSuffix);
373 
374     // Build up new path with the referenced region in place of our current
375     // region in the reference path.  Also strip regionname suffix from name.
376     return new Path(new Path(new Path(tableDir, otherRegion),
377       p.getParent().getName()), nameStrippedOfSuffix);
378   }
379 
380   /**
381    * @return True if this file was made by a major compaction.
382    */
383   boolean isMajorCompaction() {
384     if (this.majorCompaction == null) {
385       throw new NullPointerException("This has not been set yet");
386     }
387     return this.majorCompaction.get();
388   }
389 
390   /**
391    * @return True if this file should not be part of a minor compaction.
392    */
393   boolean excludeFromMinorCompaction() {
394     return this.excludeFromMinorCompaction;
395   }
396 
397   /**
398    * @return This files maximum edit sequence id.
399    */
400   public long getMaxSequenceId() {
401     return this.sequenceid;
402   }
403 
404   public long getModificationTimeStamp() {
405     return modificationTimeStamp;
406   }
407 
408   /**
409    * Return the largest memstoreTS found across all storefiles in
410    * the given list. Store files that were created by a mapreduce
411    * bulk load are ignored, as they do not correspond to any specific
412    * put operation, and thus do not have a memstoreTS associated with them.
413    * @return 0 if no non-bulk-load files are provided or, this is Store that
414    * does not yet have any store files.
415    */
416   public static long getMaxMemstoreTSInList(Collection<StoreFile> sfs) {
417     long max = 0;
418     for (StoreFile sf : sfs) {
419       if (!sf.isBulkLoadResult()) {
420         max = Math.max(max, sf.getMaxMemstoreTS());
421       }
422     }
423     return max;
424   }
425 
426   /**
427    * Return the highest sequence ID found across all storefiles in
428    * the given list. Store files that were created by a mapreduce
429    * bulk load are ignored, as they do not correspond to any edit
430    * log items.
431    * @return 0 if no non-bulk-load files are provided or, this is Store that
432    * does not yet have any store files.
433    */
434   public static long getMaxSequenceIdInList(Collection<StoreFile> sfs) {
435     long max = 0;
436     for (StoreFile sf : sfs) {
437       if (!sf.isBulkLoadResult()) {
438         max = Math.max(max, sf.getMaxSequenceId());
439       }
440     }
441     return max;
442   }
443 
444   /**
445    * @return true if this storefile was created by HFileOutputFormat
446    * for a bulk load.
447    */
448   boolean isBulkLoadResult() {
449     return metadataMap.containsKey(BULKLOAD_TIME_KEY);
450   }
451 
452   /**
453    * Return the timestamp at which this bulk load file was generated.
454    */
455   public long getBulkLoadTimestamp() {
456     return Bytes.toLong(metadataMap.get(BULKLOAD_TIME_KEY));
457   }
458 
459   /**
460    * @return the cached value of HDFS blocks distribution. The cached value is
461    * calculated when store file is opened.
462    */
463   public HDFSBlocksDistribution getHDFSBlockDistribution() {
464     return this.hdfsBlocksDistribution;
465   }
466 
467   /**
468    * helper function to compute HDFS blocks distribution of a given reference
469    * file.For reference file, we don't compute the exact value. We use some
470    * estimate instead given it might be good enough. we assume bottom part
471    * takes the first half of reference file, top part takes the second half
472    * of the reference file. This is just estimate, given
473    * midkey ofregion != midkey of HFile, also the number and size of keys vary.
474    * If this estimate isn't good enough, we can improve it later.
475    * @param fs  The FileSystem
476    * @param reference  The reference
477    * @param status  The reference FileStatus
478    * @return HDFS blocks distribution
479    */
480   static private HDFSBlocksDistribution computeRefFileHDFSBlockDistribution(
481     FileSystem fs, Reference reference, FileStatus status) throws IOException {
482     if (status == null) {
483       return null;
484     }
485 
486     long start = 0;
487     long length = 0;
488 
489     if (Reference.isTopFileRegion(reference.getFileRegion())) {
490       start = status.getLen()/2;
491       length = status.getLen() - status.getLen()/2;
492     } else {
493       start = 0;
494       length = status.getLen()/2;
495     }
496     return FSUtils.computeHDFSBlocksDistribution(fs, status, start, length);
497   }
498 
499   /**
500    * compute HDFS block distribution, for reference file, it is an estimate
501    */
502   private void computeHDFSBlockDistribution() throws IOException {
503     if (isReference()) {
504       FileStatus status;
505       if (this.link != null) {
506         status = this.link.getFileStatus(fs);
507       } else {
508         status = fs.getFileStatus(this.referencePath);
509       }
510       this.hdfsBlocksDistribution = computeRefFileHDFSBlockDistribution(
511         this.fs, this.reference, status);
512     } else {
513       FileStatus status;
514       if (isLink()) {
515         status = link.getFileStatus(fs);
516       } else {
517         status = this.fs.getFileStatus(path);
518       }
519       long length = status.getLen();
520       this.hdfsBlocksDistribution = FSUtils.computeHDFSBlocksDistribution(
521         this.fs, status, 0, length);
522     }
523   }
524 
525   /**
526    * Opens reader on this store file.  Called by Constructor.
527    * @return Reader for the store file.
528    * @throws IOException
529    * @see #closeReader()
530    */
531   private Reader open() throws IOException {
532     if (this.reader != null) {
533       throw new IllegalAccessError("Already open");
534     }
535     if (isReference()) {
536       if (this.link != null) {
537         this.reader = new HalfStoreFileReader(this.fs, this.referencePath, this.link,
538           this.cacheConf, this.reference, dataBlockEncoder.getEncodingInCache());
539       } else {
540         this.reader = new HalfStoreFileReader(this.fs, this.referencePath,
541           this.cacheConf, this.reference, dataBlockEncoder.getEncodingInCache());
542       }
543     } else if (isLink()) {
544       long size = link.getFileStatus(fs).getLen();
545       this.reader = new Reader(this.fs, this.path, link, size, this.cacheConf,
546           dataBlockEncoder.getEncodingInCache(), true);
547     } else {
548       this.reader = new Reader(this.fs, this.path, this.cacheConf,
549           dataBlockEncoder.getEncodingInCache());
550     }
551 
552     if (isSchemaConfigured()) {
553       SchemaConfigured.resetSchemaMetricsConf(reader);
554       passSchemaMetricsTo(reader);
555     }
556 
557     computeHDFSBlockDistribution();
558 
559     // Load up indices and fileinfo. This also loads Bloom filter type.
560     metadataMap = Collections.unmodifiableMap(this.reader.loadFileInfo());
561 
562     // Read in our metadata.
563     byte [] b = metadataMap.get(MAX_SEQ_ID_KEY);
564     if (b != null) {
565       // By convention, if halfhfile, top half has a sequence number > bottom
566       // half. Thats why we add one in below. Its done for case the two halves
567       // are ever merged back together --rare.  Without it, on open of store,
568       // since store files are distinguished by sequence id, the one half would
569       // subsume the other.
570       this.sequenceid = Bytes.toLong(b);
571       if (isReference()) {
572         if (Reference.isTopFileRegion(this.reference.getFileRegion())) {
573           this.sequenceid += 1;
574         }
575       }
576     }
577     this.reader.setSequenceID(this.sequenceid);
578 
579     b = metadataMap.get(HFileWriterV2.MAX_MEMSTORE_TS_KEY);
580     if (b != null) {
581       this.maxMemstoreTS = Bytes.toLong(b);
582     }
583 
584     b = metadataMap.get(MAJOR_COMPACTION_KEY);
585     if (b != null) {
586       boolean mc = Bytes.toBoolean(b);
587       if (this.majorCompaction == null) {
588         this.majorCompaction = new AtomicBoolean(mc);
589       } else {
590         this.majorCompaction.set(mc);
591       }
592     } else {
593       // Presume it is not major compacted if it doesn't explicity say so
594       // HFileOutputFormat explicitly sets the major compacted key.
595       this.majorCompaction = new AtomicBoolean(false);
596     }
597 
598     b = metadataMap.get(EXCLUDE_FROM_MINOR_COMPACTION_KEY);
599     this.excludeFromMinorCompaction = (b != null && Bytes.toBoolean(b));
600 
601     BloomType hfileBloomType = reader.getBloomFilterType();
602     if (cfBloomType != BloomType.NONE) {
603       reader.loadBloomfilter(BlockType.GENERAL_BLOOM_META);
604       if (hfileBloomType != cfBloomType) {
605         LOG.info("HFile Bloom filter type for "
606             + reader.getHFileReader().getName() + ": " + hfileBloomType
607             + ", but " + cfBloomType + " specified in column family "
608             + "configuration");
609       }
610     } else if (hfileBloomType != BloomType.NONE) {
611       LOG.info("Bloom filter turned off by CF config for "
612           + reader.getHFileReader().getName());
613     }
614 
615     // load delete family bloom filter
616     reader.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META);
617 
618     try {
619       byte [] timerangeBytes = metadataMap.get(TIMERANGE_KEY);
620       if (timerangeBytes != null) {
621         this.reader.timeRangeTracker = new TimeRangeTracker();
622         Writables.copyWritable(timerangeBytes, this.reader.timeRangeTracker);
623       }
624     } catch (IllegalArgumentException e) {
625       LOG.error("Error reading timestamp range data from meta -- " +
626           "proceeding without", e);
627       this.reader.timeRangeTracker = null;
628     }
629     return this.reader;
630   }
631 
632   /**
633    * @return Reader for StoreFile. creates if necessary
634    * @throws IOException
635    */
636   public Reader createReader() throws IOException {
637     if (this.reader == null) {
638       try {
639         this.reader = open();
640       } catch (IOException e) {
641         try {
642           this.closeReader(true);
643         } catch (IOException ee) {              
644         }
645         throw e;
646       }
647 
648     }
649     return this.reader;
650   }
651 
652   /**
653    * @return Current reader.  Must call createReader first else returns null.
654    * @see #createReader()
655    */
656   public Reader getReader() {
657     return this.reader;
658   }
659 
660   /**
661    * @param evictOnClose whether to evict blocks belonging to this file
662    * @throws IOException
663    */
664   public synchronized void closeReader(boolean evictOnClose)
665       throws IOException {
666     if (this.reader != null) {
667       this.reader.close(evictOnClose);
668       this.reader = null;
669     }
670   }
671 
672   /**
673    * Delete this file
674    * @throws IOException
675    */
676   public void deleteReader() throws IOException {
677     closeReader(true);
678     this.fs.delete(getPath(), true);
679   }
680 
681   @Override
682   public String toString() {
683     return this.path.toString() +
684       (isReference()? "-" + this.referencePath + "-" + reference.toString(): "");
685   }
686 
687   /**
688    * @return a length description of this StoreFile, suitable for debug output
689    */
690   public String toStringDetailed() {
691     StringBuilder sb = new StringBuilder();
692     sb.append(this.path.toString());
693     sb.append(", isReference=").append(isReference());
694     sb.append(", isBulkLoadResult=").append(isBulkLoadResult());
695     if (isBulkLoadResult()) {
696       sb.append(", bulkLoadTS=").append(getBulkLoadTimestamp());
697     } else {
698       sb.append(", seqid=").append(getMaxSequenceId());
699     }
700     sb.append(", majorCompaction=").append(isMajorCompaction());
701 
702     return sb.toString();
703   }
704 
705   /**
706    * Utility to help with rename.
707    * @param fs
708    * @param src
709    * @param tgt
710    * @return True if succeeded.
711    * @throws IOException
712    */
713   public static Path rename(final FileSystem fs,
714                             final Path src,
715                             final Path tgt)
716       throws IOException {
717 
718     if (!fs.exists(src)) {
719       throw new FileNotFoundException(src.toString());
720     }
721     if (!fs.rename(src, tgt)) {
722       throw new IOException("Failed rename of " + src + " to " + tgt);
723     }
724     return tgt;
725   }
726 
727   public static class WriterBuilder {
728     private final Configuration conf;
729     private final CacheConfig cacheConf;
730     private final FileSystem fs;
731     private final int blockSize;
732 
733     private Compression.Algorithm compressAlgo =
734         HFile.DEFAULT_COMPRESSION_ALGORITHM;
735     private HFileDataBlockEncoder dataBlockEncoder =
736         NoOpDataBlockEncoder.INSTANCE;
737     private KeyValue.KVComparator comparator = KeyValue.COMPARATOR;
738     private BloomType bloomType = BloomType.NONE;
739     private long maxKeyCount = 0;
740     private Path dir;
741     private Path filePath;
742     private ChecksumType checksumType = HFile.DEFAULT_CHECKSUM_TYPE;
743     private int bytesPerChecksum = HFile.DEFAULT_BYTES_PER_CHECKSUM;
744 
745     public WriterBuilder(Configuration conf, CacheConfig cacheConf,
746         FileSystem fs, int blockSize) {
747       this.conf = conf;
748       this.cacheConf = cacheConf;
749       this.fs = fs;
750       this.blockSize = blockSize;
751     }
752 
753     /**
754      * Use either this method or {@link #withFilePath}, but not both.
755      * @param dir Path to column family directory. The directory is created if
756      *          does not exist. The file is given a unique name within this
757      *          directory.
758      * @return this (for chained invocation)
759      */
760     public WriterBuilder withOutputDir(Path dir) {
761       Preconditions.checkNotNull(dir);
762       this.dir = dir;
763       return this;
764     }
765 
766     /**
767      * Use either this method or {@link #withOutputDir}, but not both.
768      * @param filePath the StoreFile path to write
769      * @return this (for chained invocation)
770      */
771     public WriterBuilder withFilePath(Path filePath) {
772       Preconditions.checkNotNull(filePath);
773       this.filePath = filePath;
774       return this;
775     }
776 
777     public WriterBuilder withCompression(Compression.Algorithm compressAlgo) {
778       Preconditions.checkNotNull(compressAlgo);
779       this.compressAlgo = compressAlgo;
780       return this;
781     }
782 
783     public WriterBuilder withDataBlockEncoder(HFileDataBlockEncoder encoder) {
784       Preconditions.checkNotNull(encoder);
785       this.dataBlockEncoder = encoder;
786       return this;
787     }
788 
789     public WriterBuilder withComparator(KeyValue.KVComparator comparator) {
790       Preconditions.checkNotNull(comparator);
791       this.comparator = comparator;
792       return this;
793     }
794 
795     public WriterBuilder withBloomType(BloomType bloomType) {
796       Preconditions.checkNotNull(bloomType);
797       this.bloomType = bloomType;
798       return this;
799     }
800 
801     /**
802      * @param maxKeyCount estimated maximum number of keys we expect to add
803      * @return this (for chained invocation)
804      */
805     public WriterBuilder withMaxKeyCount(long maxKeyCount) {
806       this.maxKeyCount = maxKeyCount;
807       return this;
808     }
809 
810     /**
811      * @param checksumType the type of checksum
812      * @return this (for chained invocation)
813      */
814     public WriterBuilder withChecksumType(ChecksumType checksumType) {
815       this.checksumType = checksumType;
816       return this;
817     }
818 
819     /**
820      * @param bytesPerChecksum the number of bytes per checksum chunk
821      * @return this (for chained invocation)
822      */
823     public WriterBuilder withBytesPerChecksum(int bytesPerChecksum) {
824       this.bytesPerChecksum = bytesPerChecksum;
825       return this;
826     }
827 
828     /**
829      * Create a store file writer. Client is responsible for closing file when
830      * done. If metadata, add BEFORE closing using
831      * {@link Writer#appendMetadata}.
832      */
833     public Writer build() throws IOException {
834       if ((dir == null ? 0 : 1) + (filePath == null ? 0 : 1) != 1) {
835         throw new IllegalArgumentException("Either specify parent directory " +
836             "or file path");
837       }
838 
839       if (dir == null) {
840         dir = filePath.getParent();
841       }
842 
843       if (!fs.exists(dir)) {
844         fs.mkdirs(dir);
845       }
846 
847       if (filePath == null) {
848         filePath = getUniqueFile(fs, dir);
849         if (!BloomFilterFactory.isGeneralBloomEnabled(conf)) {
850           bloomType = BloomType.NONE;
851         }
852       }
853 
854       if (compressAlgo == null) {
855         compressAlgo = HFile.DEFAULT_COMPRESSION_ALGORITHM;
856       }
857       if (comparator == null) {
858         comparator = KeyValue.COMPARATOR;
859       }
860       return new Writer(fs, filePath, blockSize, compressAlgo, dataBlockEncoder,
861           conf, cacheConf, comparator, bloomType, maxKeyCount, checksumType,
862           bytesPerChecksum);
863     }
864   }
865 
866   /**
867    * @param fs
868    * @param dir Directory to create file in.
869    * @return random filename inside passed <code>dir</code>
870    */
871   public static Path getUniqueFile(final FileSystem fs, final Path dir)
872       throws IOException {
873     if (!fs.getFileStatus(dir).isDir()) {
874       throw new IOException("Expecting " + dir.toString() +
875         " to be a directory");
876     }
877     return getRandomFilename(fs, dir);
878   }
879 
880   /**
881    *
882    * @param fs
883    * @param dir
884    * @return Path to a file that doesn't exist at time of this invocation.
885    * @throws IOException
886    */
887   static Path getRandomFilename(final FileSystem fs, final Path dir)
888       throws IOException {
889     return getRandomFilename(fs, dir, null);
890   }
891 
892   /**
893    *
894    * @param fs
895    * @param dir
896    * @param suffix
897    * @return Path to a file that doesn't exist at time of this invocation.
898    * @throws IOException
899    */
900   static Path getRandomFilename(final FileSystem fs,
901                                 final Path dir,
902                                 final String suffix)
903       throws IOException {
904     return new Path(dir, UUID.randomUUID().toString().replaceAll("-", "")
905         + (suffix == null ? "" : suffix));
906   }
907 
908   /**
909    * Validate the store file name.
910    * @param fileName name of the file to validate
911    * @return <tt>true</tt> if the file could be a valid store file, <tt>false</tt> otherwise
912    */
913   public static boolean validateStoreFileName(String fileName) {
914     if (HFileLink.isHFileLink(fileName))
915       return true;
916     if (isReference(fileName))
917       return true;
918     return !fileName.contains("-");
919   }
920 
921   /**
922    * Write out a split reference. Package local so it doesnt leak out of
923    * regionserver.
924    * @param fs
925    * @param splitDir Presumes path format is actually
926    *          <code>SOME_DIRECTORY/REGIONNAME/FAMILY</code>.
927    * @param f File to split.
928    * @param splitRow
929    * @param range
930    * @return Path to created reference.
931    * @throws IOException
932    */
933   static Path split(final FileSystem fs,
934                     final Path splitDir,
935                     final StoreFile f,
936                     final byte [] splitRow,
937                     final Reference.Range range)
938       throws IOException {
939     // A reference to the bottom half of the hsf store file.
940     Reference r = new Reference(splitRow, range);
941     // Add the referred-to regions name as a dot separated suffix.
942     // See REF_NAME_REGEX regex above.  The referred-to regions name is
943     // up in the path of the passed in <code>f</code> -- parentdir is family,
944     // then the directory above is the region name.
945     String parentRegionName = f.getPath().getParent().getParent().getName();
946     // Write reference with same file id only with the other region name as
947     // suffix and into the new region location (under same family).
948     Path p = new Path(splitDir, f.getPath().getName() + "." + parentRegionName);
949     return r.write(fs, p);
950   }
951 
952 
953   /**
954    * A StoreFile writer.  Use this to read/write HBase Store Files. It is package
955    * local because it is an implementation detail of the HBase regionserver.
956    */
957   public static class Writer {
958     private final BloomFilterWriter generalBloomFilterWriter;
959     private final BloomFilterWriter deleteFamilyBloomFilterWriter;
960     private final BloomType bloomType;
961     private byte[] lastBloomKey;
962     private int lastBloomKeyOffset, lastBloomKeyLen;
963     private KVComparator kvComparator;
964     private KeyValue lastKv = null;
965     private long earliestPutTs = HConstants.LATEST_TIMESTAMP;
966     private KeyValue lastDeleteFamilyKV = null;
967     private long deleteFamilyCnt = 0;
968 
969     protected HFileDataBlockEncoder dataBlockEncoder;
970 
971     /** Checksum type */
972     protected ChecksumType checksumType;
973 
974     /** Bytes per Checksum */
975     protected int bytesPerChecksum;
976     
977     TimeRangeTracker timeRangeTracker = new TimeRangeTracker();
978     /* isTimeRangeTrackerSet keeps track if the timeRange has already been set
979      * When flushing a memstore, we set TimeRange and use this variable to
980      * indicate that it doesn't need to be calculated again while
981      * appending KeyValues.
982      * It is not set in cases of compactions when it is recalculated using only
983      * the appended KeyValues*/
984     boolean isTimeRangeTrackerSet = false;
985 
986     protected HFile.Writer writer;
987 
988     /**
989      * Creates an HFile.Writer that also write helpful meta data.
990      * @param fs file system to write to
991      * @param path file name to create
992      * @param blocksize HDFS block size
993      * @param compress HDFS block compression
994      * @param conf user configuration
995      * @param comparator key comparator
996      * @param bloomType bloom filter setting
997      * @param maxKeys the expected maximum number of keys to be added. Was used
998      *        for Bloom filter size in {@link HFile} format version 1.
999      * @param checksumType the checksum type
1000      * @param bytesPerChecksum the number of bytes per checksum value
1001      * @throws IOException problem writing to FS
1002      */
1003     private Writer(FileSystem fs, Path path, int blocksize,
1004         Compression.Algorithm compress,
1005         HFileDataBlockEncoder dataBlockEncoder, final Configuration conf,
1006         CacheConfig cacheConf,
1007         final KVComparator comparator, BloomType bloomType, long maxKeys,
1008         final ChecksumType checksumType, final int bytesPerChecksum)
1009         throws IOException {
1010       this.dataBlockEncoder = dataBlockEncoder != null ?
1011           dataBlockEncoder : NoOpDataBlockEncoder.INSTANCE;
1012       writer = HFile.getWriterFactory(conf, cacheConf)
1013           .withPath(fs, path)
1014           .withBlockSize(blocksize)
1015           .withCompression(compress)
1016           .withDataBlockEncoder(dataBlockEncoder)
1017           .withComparator(comparator.getRawComparator())
1018           .withChecksumType(checksumType)
1019           .withBytesPerChecksum(bytesPerChecksum)
1020           .create();
1021 
1022       this.kvComparator = comparator;
1023 
1024       generalBloomFilterWriter = BloomFilterFactory.createGeneralBloomAtWrite(
1025           conf, cacheConf, bloomType,
1026           (int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
1027 
1028       if (generalBloomFilterWriter != null) {
1029         this.bloomType = bloomType;
1030         LOG.info("Bloom filter type for " + path + ": " + this.bloomType + ", "
1031             + generalBloomFilterWriter.getClass().getSimpleName());
1032       } else {
1033         // Not using Bloom filters.
1034         this.bloomType = BloomType.NONE;
1035       }
1036 
1037       // initialize delete family Bloom filter when there is NO RowCol Bloom
1038       // filter
1039       if (this.bloomType != BloomType.ROWCOL) {
1040         this.deleteFamilyBloomFilterWriter = BloomFilterFactory
1041             .createDeleteBloomAtWrite(conf, cacheConf,
1042                 (int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
1043       } else {
1044         deleteFamilyBloomFilterWriter = null;
1045       }
1046       if (deleteFamilyBloomFilterWriter != null) {
1047         LOG.info("Delete Family Bloom filter type for " + path + ": "
1048             + deleteFamilyBloomFilterWriter.getClass().getSimpleName());
1049       }
1050       this.checksumType = checksumType;
1051       this.bytesPerChecksum = bytesPerChecksum;
1052     }
1053 
1054     /**
1055      * Writes meta data.
1056      * Call before {@link #close()} since its written as meta data to this file.
1057      * @param maxSequenceId Maximum sequence id.
1058      * @param majorCompaction True if this file is product of a major compaction
1059      * @throws IOException problem writing to FS
1060      */
1061     public void appendMetadata(final long maxSequenceId, final boolean majorCompaction)
1062     throws IOException {
1063       writer.appendFileInfo(MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId));
1064       writer.appendFileInfo(MAJOR_COMPACTION_KEY,
1065           Bytes.toBytes(majorCompaction));
1066       appendTrackedTimestampsToMetadata();
1067     }
1068 
1069     /**
1070      * Add TimestampRange and earliest put timestamp to Metadata
1071      */
1072     public void appendTrackedTimestampsToMetadata() throws IOException {
1073       appendFileInfo(TIMERANGE_KEY,WritableUtils.toByteArray(timeRangeTracker));
1074       appendFileInfo(EARLIEST_PUT_TS, Bytes.toBytes(earliestPutTs));
1075     }
1076 
1077     /**
1078      * Set TimeRangeTracker
1079      * @param trt
1080      */
1081     public void setTimeRangeTracker(final TimeRangeTracker trt) {
1082       this.timeRangeTracker = trt;
1083       isTimeRangeTrackerSet = true;
1084     }
1085 
1086     /**
1087      * Record the earlest Put timestamp.
1088      *
1089      * If the timeRangeTracker is not set,
1090      * update TimeRangeTracker to include the timestamp of this key
1091      * @param kv
1092      */
1093     public void trackTimestamps(final KeyValue kv) {
1094       if (KeyValue.Type.Put.getCode() == kv.getType()) {
1095         earliestPutTs = Math.min(earliestPutTs, kv.getTimestamp());
1096       }
1097       if (!isTimeRangeTrackerSet) {
1098         timeRangeTracker.includeTimestamp(kv);
1099       }
1100     }
1101 
1102     private void appendGeneralBloomfilter(final KeyValue kv) throws IOException {
1103       if (this.generalBloomFilterWriter != null) {
1104         // only add to the bloom filter on a new, unique key
1105         boolean newKey = true;
1106         if (this.lastKv != null) {
1107           switch(bloomType) {
1108           case ROW:
1109             newKey = ! kvComparator.matchingRows(kv, lastKv);
1110             break;
1111           case ROWCOL:
1112             newKey = ! kvComparator.matchingRowColumn(kv, lastKv);
1113             break;
1114           case NONE:
1115             newKey = false;
1116             break;
1117           default:
1118             throw new IOException("Invalid Bloom filter type: " + bloomType +
1119                 " (ROW or ROWCOL expected)");
1120           }
1121         }
1122         if (newKey) {
1123           /*
1124            * http://2.bp.blogspot.com/_Cib_A77V54U/StZMrzaKufI/AAAAAAAAADo/ZhK7bGoJdMQ/s400/KeyValue.png
1125            * Key = RowLen + Row + FamilyLen + Column [Family + Qualifier] + TimeStamp
1126            *
1127            * 2 Types of Filtering:
1128            *  1. Row = Row
1129            *  2. RowCol = Row + Qualifier
1130            */
1131           byte[] bloomKey;
1132           int bloomKeyOffset, bloomKeyLen;
1133 
1134           switch (bloomType) {
1135           case ROW:
1136             bloomKey = kv.getBuffer();
1137             bloomKeyOffset = kv.getRowOffset();
1138             bloomKeyLen = kv.getRowLength();
1139             break;
1140           case ROWCOL:
1141             // merge(row, qualifier)
1142             // TODO: could save one buffer copy in case of compound Bloom
1143             // filters when this involves creating a KeyValue
1144             bloomKey = generalBloomFilterWriter.createBloomKey(kv.getBuffer(),
1145                 kv.getRowOffset(), kv.getRowLength(), kv.getBuffer(),
1146                 kv.getQualifierOffset(), kv.getQualifierLength());
1147             bloomKeyOffset = 0;
1148             bloomKeyLen = bloomKey.length;
1149             break;
1150           default:
1151             throw new IOException("Invalid Bloom filter type: " + bloomType +
1152                 " (ROW or ROWCOL expected)");
1153           }
1154           generalBloomFilterWriter.add(bloomKey, bloomKeyOffset, bloomKeyLen);
1155           if (lastBloomKey != null
1156               && generalBloomFilterWriter.getComparator().compare(bloomKey,
1157                   bloomKeyOffset, bloomKeyLen, lastBloomKey,
1158                   lastBloomKeyOffset, lastBloomKeyLen) <= 0) {
1159             throw new IOException("Non-increasing Bloom keys: "
1160                 + Bytes.toStringBinary(bloomKey, bloomKeyOffset, bloomKeyLen)
1161                 + " after "
1162                 + Bytes.toStringBinary(lastBloomKey, lastBloomKeyOffset,
1163                     lastBloomKeyLen));
1164           }
1165           lastBloomKey = bloomKey;
1166           lastBloomKeyOffset = bloomKeyOffset;
1167           lastBloomKeyLen = bloomKeyLen;
1168           this.lastKv = kv;
1169         }
1170       }
1171     }
1172 
1173     private void appendDeleteFamilyBloomFilter(final KeyValue kv)
1174         throws IOException {
1175       if (!kv.isDeleteFamily()) {
1176         return;
1177       }
1178 
1179       // increase the number of delete family in the store file
1180       deleteFamilyCnt++;
1181       if (null != this.deleteFamilyBloomFilterWriter) {
1182         boolean newKey = true;
1183         if (lastDeleteFamilyKV != null) {
1184           newKey = !kvComparator.matchingRows(kv, lastDeleteFamilyKV);
1185         }
1186         if (newKey) {
1187           this.deleteFamilyBloomFilterWriter.add(kv.getBuffer(),
1188               kv.getRowOffset(), kv.getRowLength());
1189           this.lastDeleteFamilyKV = kv;
1190         }
1191       }
1192     }
1193 
1194     public void append(final KeyValue kv) throws IOException {
1195       appendGeneralBloomfilter(kv);
1196       appendDeleteFamilyBloomFilter(kv);
1197       writer.append(kv);
1198       trackTimestamps(kv);
1199     }
1200 
1201     public Path getPath() {
1202       return this.writer.getPath();
1203     }
1204 
1205     boolean hasGeneralBloom() {
1206       return this.generalBloomFilterWriter != null;
1207     }
1208 
1209     /**
1210      * For unit testing only.
1211      *
1212      * @return the Bloom filter used by this writer.
1213      */
1214     BloomFilterWriter getGeneralBloomWriter() {
1215       return generalBloomFilterWriter;
1216     }
1217 
1218     private boolean closeBloomFilter(BloomFilterWriter bfw) throws IOException {
1219       boolean haveBloom = (bfw != null && bfw.getKeyCount() > 0);
1220       if (haveBloom) {
1221         bfw.compactBloom();
1222       }
1223       return haveBloom;
1224     }
1225 
1226     private boolean closeGeneralBloomFilter() throws IOException {
1227       boolean hasGeneralBloom = closeBloomFilter(generalBloomFilterWriter);
1228 
1229       // add the general Bloom filter writer and append file info
1230       if (hasGeneralBloom) {
1231         writer.addGeneralBloomFilter(generalBloomFilterWriter);
1232         writer.appendFileInfo(BLOOM_FILTER_TYPE_KEY,
1233             Bytes.toBytes(bloomType.toString()));
1234         if (lastBloomKey != null) {
1235           writer.appendFileInfo(LAST_BLOOM_KEY, Arrays.copyOfRange(
1236               lastBloomKey, lastBloomKeyOffset, lastBloomKeyOffset
1237                   + lastBloomKeyLen));
1238         }
1239       }
1240       return hasGeneralBloom;
1241     }
1242 
1243     private boolean closeDeleteFamilyBloomFilter() throws IOException {
1244       boolean hasDeleteFamilyBloom = closeBloomFilter(deleteFamilyBloomFilterWriter);
1245 
1246       // add the delete family Bloom filter writer
1247       if (hasDeleteFamilyBloom) {
1248         writer.addDeleteFamilyBloomFilter(deleteFamilyBloomFilterWriter);
1249       }
1250 
1251       // append file info about the number of delete family kvs
1252       // even if there is no delete family Bloom.
1253       writer.appendFileInfo(DELETE_FAMILY_COUNT,
1254           Bytes.toBytes(this.deleteFamilyCnt));
1255 
1256       return hasDeleteFamilyBloom;
1257     }
1258 
1259     public void close() throws IOException {
1260       boolean hasGeneralBloom = this.closeGeneralBloomFilter();
1261       boolean hasDeleteFamilyBloom = this.closeDeleteFamilyBloomFilter();
1262 
1263       writer.close();
1264 
1265       // Log final Bloom filter statistics. This needs to be done after close()
1266       // because compound Bloom filters might be finalized as part of closing.
1267       StoreFile.LOG.info((hasGeneralBloom ? "" : "NO ") + "General Bloom and "
1268           + (hasDeleteFamilyBloom ? "" : "NO ") + "DeleteFamily"
1269           + " was added to HFile (" + getPath() + ") ");
1270 
1271     }
1272 
1273     public void appendFileInfo(byte[] key, byte[] value) throws IOException {
1274       writer.appendFileInfo(key, value);
1275     }
1276 
1277     /** For use in testing, e.g. {@link CreateRandomStoreFile} */
1278     HFile.Writer getHFileWriter() {
1279       return writer;
1280     }
1281   }
1282 
1283   /**
1284    * Reader for a StoreFile.
1285    */
1286   public static class Reader extends SchemaConfigured {
1287     static final Log LOG = LogFactory.getLog(Reader.class.getName());
1288 
1289     protected BloomFilter generalBloomFilter = null;
1290     protected BloomFilter deleteFamilyBloomFilter = null;
1291     protected BloomType bloomFilterType;
1292     private final HFile.Reader reader;
1293     protected TimeRangeTracker timeRangeTracker = null;
1294     protected long sequenceID = -1;
1295     private byte[] lastBloomKey;
1296     private long deleteFamilyCnt = -1;
1297 
1298     public Reader(FileSystem fs, Path path, CacheConfig cacheConf,
1299         DataBlockEncoding preferredEncodingInCache) throws IOException {
1300       super(path);
1301       reader = HFile.createReaderWithEncoding(fs, path, cacheConf,
1302           preferredEncodingInCache);
1303       bloomFilterType = BloomType.NONE;
1304     }
1305 
1306     public Reader(FileSystem fs, Path path, HFileLink hfileLink, long size,
1307         CacheConfig cacheConf, DataBlockEncoding preferredEncodingInCache,
1308         boolean closeIStream) throws IOException {
1309       super(path);
1310 
1311       FSDataInputStream in = hfileLink.open(fs);
1312       FSDataInputStream inNoChecksum = in;
1313       if (fs instanceof HFileSystem) {
1314         FileSystem noChecksumFs = ((HFileSystem)fs).getNoChecksumFs();
1315         inNoChecksum = hfileLink.open(noChecksumFs);
1316       }
1317 
1318       reader = HFile.createReaderWithEncoding(fs, path, in, inNoChecksum,
1319                   size, cacheConf, preferredEncodingInCache, closeIStream);
1320       bloomFilterType = BloomType.NONE;
1321     }
1322 
1323     /**
1324      * ONLY USE DEFAULT CONSTRUCTOR FOR UNIT TESTS
1325      */
1326     Reader() {
1327       this.reader = null;
1328     }
1329 
1330     public RawComparator<byte []> getComparator() {
1331       return reader.getComparator();
1332     }
1333 
1334     /**
1335      * Get a scanner to scan over this StoreFile. Do not use
1336      * this overload if using this scanner for compactions.
1337      *
1338      * @param cacheBlocks should this scanner cache blocks?
1339      * @param pread use pread (for highly concurrent small readers)
1340      * @return a scanner
1341      */
1342     public StoreFileScanner getStoreFileScanner(boolean cacheBlocks,
1343                                                boolean pread) {
1344       return getStoreFileScanner(cacheBlocks, pread, false);
1345     }
1346 
1347     /**
1348      * Get a scanner to scan over this StoreFile.
1349      *
1350      * @param cacheBlocks should this scanner cache blocks?
1351      * @param pread use pread (for highly concurrent small readers)
1352      * @param isCompaction is scanner being used for compaction?
1353      * @return a scanner
1354      */
1355     public StoreFileScanner getStoreFileScanner(boolean cacheBlocks,
1356                                                boolean pread,
1357                                                boolean isCompaction) {
1358       return new StoreFileScanner(this,
1359                                  getScanner(cacheBlocks, pread,
1360                                             isCompaction), !isCompaction);
1361     }
1362 
1363     /**
1364      * Warning: Do not write further code which depends on this call. Instead
1365      * use getStoreFileScanner() which uses the StoreFileScanner class/interface
1366      * which is the preferred way to scan a store with higher level concepts.
1367      *
1368      * @param cacheBlocks should we cache the blocks?
1369      * @param pread use pread (for concurrent small readers)
1370      * @return the underlying HFileScanner
1371      */
1372     @Deprecated
1373     public HFileScanner getScanner(boolean cacheBlocks, boolean pread) {
1374       return getScanner(cacheBlocks, pread, false);
1375     }
1376 
1377     /**
1378      * Warning: Do not write further code which depends on this call. Instead
1379      * use getStoreFileScanner() which uses the StoreFileScanner class/interface
1380      * which is the preferred way to scan a store with higher level concepts.
1381      *
1382      * @param cacheBlocks
1383      *          should we cache the blocks?
1384      * @param pread
1385      *          use pread (for concurrent small readers)
1386      * @param isCompaction
1387      *          is scanner being used for compaction?
1388      * @return the underlying HFileScanner
1389      */
1390     @Deprecated
1391     public HFileScanner getScanner(boolean cacheBlocks, boolean pread,
1392         boolean isCompaction) {
1393       return reader.getScanner(cacheBlocks, pread, isCompaction);
1394     }
1395 
1396     public void close(boolean evictOnClose) throws IOException {
1397       reader.close(evictOnClose);
1398     }
1399 
1400     /**
1401      * Check if this storeFile may contain keys within the TimeRange that
1402      * have not expired (i.e. not older than oldestUnexpiredTS).
1403      * @param scan the current scan
1404      * @param oldestUnexpiredTS the oldest timestamp that is not expired, as
1405      *          determined by the column family's TTL
1406      * @return false if queried keys definitely don't exist in this StoreFile
1407      */
1408     boolean passesTimerangeFilter(Scan scan, long oldestUnexpiredTS) {
1409       if (timeRangeTracker == null) {
1410         return true;
1411       } else {
1412         return timeRangeTracker.includesTimeRange(scan.getTimeRange()) &&
1413             timeRangeTracker.getMaximumTimestamp() >= oldestUnexpiredTS;
1414       }
1415     }
1416 
1417     /**
1418      * Checks whether the given scan passes the Bloom filter (if present). Only
1419      * checks Bloom filters for single-row or single-row-column scans. Bloom
1420      * filter checking for multi-gets is implemented as part of the store
1421      * scanner system (see {@link StoreFileScanner#seekExactly}) and uses
1422      * the lower-level API {@link #passesGeneralBloomFilter(byte[], int, int, byte[],
1423      * int, int)}.
1424      *
1425      * @param scan the scan specification. Used to determine the row, and to
1426      *          check whether this is a single-row ("get") scan.
1427      * @param columns the set of columns. Only used for row-column Bloom
1428      *          filters.
1429      * @return true if the scan with the given column set passes the Bloom
1430      *         filter, or if the Bloom filter is not applicable for the scan.
1431      *         False if the Bloom filter is applicable and the scan fails it.
1432      */
1433      boolean passesBloomFilter(Scan scan,
1434         final SortedSet<byte[]> columns) {
1435       // Multi-column non-get scans will use Bloom filters through the
1436       // lower-level API function that this function calls.
1437       if (!scan.isGetScan()) {
1438         return true;
1439       }
1440 
1441       byte[] row = scan.getStartRow();
1442       switch (this.bloomFilterType) {
1443         case ROW:
1444           return passesGeneralBloomFilter(row, 0, row.length, null, 0, 0);
1445 
1446         case ROWCOL:
1447           if (columns != null && columns.size() == 1) {
1448             byte[] column = columns.first();
1449             return passesGeneralBloomFilter(row, 0, row.length, column, 0,
1450                 column.length);
1451           }
1452 
1453           // For multi-column queries the Bloom filter is checked from the
1454           // seekExact operation.
1455           return true;
1456 
1457         default:
1458           return true;
1459       }
1460     }
1461 
1462     public boolean passesDeleteFamilyBloomFilter(byte[] row, int rowOffset,
1463         int rowLen) {
1464       // Cache Bloom filter as a local variable in case it is set to null by
1465       // another thread on an IO error.
1466       BloomFilter bloomFilter = this.deleteFamilyBloomFilter;
1467 
1468       // Empty file or there is no delete family at all
1469       if (reader.getTrailer().getEntryCount() == 0 || deleteFamilyCnt == 0) {
1470         return false;
1471       }
1472 
1473       if (bloomFilter == null) {
1474         return true;
1475       }
1476 
1477       try {
1478         if (!bloomFilter.supportsAutoLoading()) {
1479           return true;
1480         }
1481         return bloomFilter.contains(row, rowOffset, rowLen, null);
1482       } catch (IllegalArgumentException e) {
1483         LOG.error("Bad Delete Family bloom filter data -- proceeding without",
1484             e);
1485         setDeleteFamilyBloomFilterFaulty();
1486       }
1487 
1488       return true;
1489     }
1490 
1491     /**
1492      * A method for checking Bloom filters. Called directly from
1493      * StoreFileScanner in case of a multi-column query.
1494      *
1495      * @param row
1496      * @param rowOffset
1497      * @param rowLen
1498      * @param col
1499      * @param colOffset
1500      * @param colLen
1501      * @return True if passes
1502      */
1503     public boolean passesGeneralBloomFilter(byte[] row, int rowOffset,
1504         int rowLen, byte[] col, int colOffset, int colLen) {
1505       if (generalBloomFilter == null)
1506         return true;
1507 
1508       byte[] key;
1509       switch (bloomFilterType) {
1510         case ROW:
1511           if (col != null) {
1512             throw new RuntimeException("Row-only Bloom filter called with " +
1513                 "column specified");
1514           }
1515           if (rowOffset != 0 || rowLen != row.length) {
1516               throw new AssertionError("For row-only Bloom filters the row "
1517                   + "must occupy the whole array");
1518           }
1519           key = row;
1520           break;
1521 
1522         case ROWCOL:
1523           key = generalBloomFilter.createBloomKey(row, rowOffset, rowLen, col,
1524               colOffset, colLen);
1525           break;
1526 
1527         default:
1528           return true;
1529       }
1530 
1531       // Cache Bloom filter as a local variable in case it is set to null by
1532       // another thread on an IO error.
1533       BloomFilter bloomFilter = this.generalBloomFilter;
1534 
1535       if (bloomFilter == null) {
1536         return true;
1537       }
1538 
1539       // Empty file
1540       if (reader.getTrailer().getEntryCount() == 0)
1541         return false;
1542 
1543       try {
1544         boolean shouldCheckBloom;
1545         ByteBuffer bloom;
1546         if (bloomFilter.supportsAutoLoading()) {
1547           bloom = null;
1548           shouldCheckBloom = true;
1549         } else {
1550           bloom = reader.getMetaBlock(HFileWriterV1.BLOOM_FILTER_DATA_KEY,
1551               true);
1552           shouldCheckBloom = bloom != null;
1553         }
1554 
1555         if (shouldCheckBloom) {
1556           boolean exists;
1557 
1558           // Whether the primary Bloom key is greater than the last Bloom key
1559           // from the file info. For row-column Bloom filters this is not yet
1560           // a sufficient condition to return false.
1561           boolean keyIsAfterLast = lastBloomKey != null
1562               && bloomFilter.getComparator().compare(key, lastBloomKey) > 0;
1563 
1564           if (bloomFilterType == BloomType.ROWCOL) {
1565             // Since a Row Delete is essentially a DeleteFamily applied to all
1566             // columns, a file might be skipped if using row+col Bloom filter.
1567             // In order to ensure this file is included an additional check is
1568             // required looking only for a row bloom.
1569             byte[] rowBloomKey = bloomFilter.createBloomKey(row, 0, row.length,
1570                 null, 0, 0);
1571 
1572             if (keyIsAfterLast
1573                 && bloomFilter.getComparator().compare(rowBloomKey,
1574                     lastBloomKey) > 0) {
1575               exists = false;
1576             } else {
1577               exists =
1578                   bloomFilter.contains(key, 0, key.length, bloom) ||
1579                   bloomFilter.contains(rowBloomKey, 0, rowBloomKey.length,
1580                       bloom);
1581             }
1582           } else {
1583             exists = !keyIsAfterLast
1584                 && bloomFilter.contains(key, 0, key.length, bloom);
1585           }
1586 
1587           getSchemaMetrics().updateBloomMetrics(exists);
1588           return exists;
1589         }
1590       } catch (IOException e) {
1591         LOG.error("Error reading bloom filter data -- proceeding without",
1592             e);
1593         setGeneralBloomFilterFaulty();
1594       } catch (IllegalArgumentException e) {
1595         LOG.error("Bad bloom filter data -- proceeding without", e);
1596         setGeneralBloomFilterFaulty();
1597       }
1598 
1599       return true;
1600     }
1601 
1602     public Map<byte[], byte[]> loadFileInfo() throws IOException {
1603       Map<byte [], byte []> fi = reader.loadFileInfo();
1604 
1605       byte[] b = fi.get(BLOOM_FILTER_TYPE_KEY);
1606       if (b != null) {
1607         bloomFilterType = BloomType.valueOf(Bytes.toString(b));
1608       }
1609 
1610       lastBloomKey = fi.get(LAST_BLOOM_KEY);
1611       byte[] cnt = fi.get(DELETE_FAMILY_COUNT);
1612       if (cnt != null) {
1613         deleteFamilyCnt = Bytes.toLong(cnt);
1614       }
1615 
1616       return fi;
1617     }
1618 
1619     public void loadBloomfilter() {
1620       this.loadBloomfilter(BlockType.GENERAL_BLOOM_META);
1621       this.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META);
1622     }
1623 
1624     private void loadBloomfilter(BlockType blockType) {
1625       try {
1626         if (blockType == BlockType.GENERAL_BLOOM_META) {
1627           if (this.generalBloomFilter != null)
1628             return; // Bloom has been loaded
1629 
1630           DataInput bloomMeta = reader.getGeneralBloomFilterMetadata();
1631           if (bloomMeta != null) {
1632             // sanity check for NONE Bloom filter
1633             if (bloomFilterType == BloomType.NONE) {
1634               throw new IOException(
1635                   "valid bloom filter type not found in FileInfo");
1636             } else {
1637               generalBloomFilter = BloomFilterFactory.createFromMeta(bloomMeta,
1638                   reader);
1639               LOG.info("Loaded " + bloomFilterType.toString() + " ("
1640                   + generalBloomFilter.getClass().getSimpleName()
1641                   + ") metadata for " + reader.getName());
1642             }
1643           }
1644         } else if (blockType == BlockType.DELETE_FAMILY_BLOOM_META) {
1645           if (this.deleteFamilyBloomFilter != null)
1646             return; // Bloom has been loaded
1647 
1648           DataInput bloomMeta = reader.getDeleteBloomFilterMetadata();
1649           if (bloomMeta != null) {
1650             deleteFamilyBloomFilter = BloomFilterFactory.createFromMeta(
1651                 bloomMeta, reader);
1652             LOG.info("Loaded Delete Family Bloom ("
1653                 + deleteFamilyBloomFilter.getClass().getSimpleName()
1654                 + ") metadata for " + reader.getName());
1655           }
1656         } else {
1657           throw new RuntimeException("Block Type: " + blockType.toString()
1658               + "is not supported for Bloom filter");
1659         }
1660       } catch (IOException e) {
1661         LOG.error("Error reading bloom filter meta for " + blockType
1662             + " -- proceeding without", e);
1663         setBloomFilterFaulty(blockType);
1664       } catch (IllegalArgumentException e) {
1665         LOG.error("Bad bloom filter meta " + blockType
1666             + " -- proceeding without", e);
1667         setBloomFilterFaulty(blockType);
1668       }
1669     }
1670 
1671     private void setBloomFilterFaulty(BlockType blockType) {
1672       if (blockType == BlockType.GENERAL_BLOOM_META) {
1673         setGeneralBloomFilterFaulty();
1674       } else if (blockType == BlockType.DELETE_FAMILY_BLOOM_META) {
1675         setDeleteFamilyBloomFilterFaulty();
1676       }
1677     }
1678 
1679     /**
1680      * The number of Bloom filter entries in this store file, or an estimate
1681      * thereof, if the Bloom filter is not loaded. This always returns an upper
1682      * bound of the number of Bloom filter entries.
1683      *
1684      * @return an estimate of the number of Bloom filter entries in this file
1685      */
1686     public long getFilterEntries() {
1687       return generalBloomFilter != null ? generalBloomFilter.getKeyCount()
1688           : reader.getEntries();
1689     }
1690 
1691     public void setGeneralBloomFilterFaulty() {
1692       generalBloomFilter = null;
1693     }
1694 
1695     public void setDeleteFamilyBloomFilterFaulty() {
1696       this.deleteFamilyBloomFilter = null;
1697     }
1698 
1699     public byte[] getLastKey() {
1700       return reader.getLastKey();
1701     }
1702 
1703     public byte[] midkey() throws IOException {
1704       return reader.midkey();
1705     }
1706 
1707     public long length() {
1708       return reader.length();
1709     }
1710 
1711     public long getTotalUncompressedBytes() {
1712       return reader.getTrailer().getTotalUncompressedBytes();
1713     }
1714 
1715     public long getEntries() {
1716       return reader.getEntries();
1717     }
1718 
1719     public long getDeleteFamilyCnt() {
1720       return deleteFamilyCnt;
1721     }
1722 
1723     public byte[] getFirstKey() {
1724       return reader.getFirstKey();
1725     }
1726 
1727     public long indexSize() {
1728       return reader.indexSize();
1729     }
1730 
1731     public String getColumnFamilyName() {
1732       return reader.getColumnFamilyName();
1733     }
1734 
1735     public BloomType getBloomFilterType() {
1736       return this.bloomFilterType;
1737     }
1738 
1739     public long getSequenceID() {
1740       return sequenceID;
1741     }
1742 
1743     public void setSequenceID(long sequenceID) {
1744       this.sequenceID = sequenceID;
1745     }
1746 
1747     BloomFilter getGeneralBloomFilter() {
1748       return generalBloomFilter;
1749     }
1750 
1751     long getUncompressedDataIndexSize() {
1752       return reader.getTrailer().getUncompressedDataIndexSize();
1753     }
1754 
1755     public long getTotalBloomSize() {
1756       if (generalBloomFilter == null)
1757         return 0;
1758       return generalBloomFilter.getByteSize();
1759     }
1760 
1761     public int getHFileVersion() {
1762       return reader.getTrailer().getMajorVersion();
1763     }
1764 
1765     HFile.Reader getHFileReader() {
1766       return reader;
1767     }
1768 
1769     void disableBloomFilterForTesting() {
1770       generalBloomFilter = null;
1771       this.deleteFamilyBloomFilter = null;
1772     }
1773 
1774     public long getMaxTimestamp() {
1775       return timeRangeTracker == null ? Long.MAX_VALUE : timeRangeTracker.maximumTimestamp;
1776     }
1777 
1778     @Override
1779     public void schemaConfigurationChanged() {
1780       passSchemaMetricsTo((SchemaConfigured) reader);
1781     }
1782   }
1783 
1784   /**
1785    * Useful comparators for comparing StoreFiles.
1786    */
1787   abstract static class Comparators {
1788     /**
1789      * Comparator that compares based on the flush time of
1790      * the StoreFiles. All bulk loads are placed before all non-
1791      * bulk loads, and then all files are sorted by sequence ID.
1792      * If there are ties, the path name is used as a tie-breaker.
1793      */
1794     static final Comparator<StoreFile> FLUSH_TIME =
1795       Ordering.compound(ImmutableList.of(
1796           Ordering.natural().onResultOf(new GetBulkTime()),
1797           Ordering.natural().onResultOf(new GetSeqId()),
1798           Ordering.natural().onResultOf(new GetPathName())
1799       ));
1800 
1801     private static class GetBulkTime implements Function<StoreFile, Long> {
1802       @Override
1803       public Long apply(StoreFile sf) {
1804         if (!sf.isBulkLoadResult()) return Long.MAX_VALUE;
1805         return sf.getBulkLoadTimestamp();
1806       }
1807     }
1808     private static class GetSeqId implements Function<StoreFile, Long> {
1809       @Override
1810       public Long apply(StoreFile sf) {
1811         if (sf.isBulkLoadResult()) return -1L;
1812         return sf.getMaxSequenceId();
1813       }
1814     }
1815     private static class GetPathName implements Function<StoreFile, String> {
1816       @Override
1817       public String apply(StoreFile sf) {
1818         return sf.getPath().getName();
1819       }
1820     }
1821 
1822     /**
1823      * FILE_SIZE = descending sort StoreFiles (largest --> smallest in size)
1824      */
1825     static final Comparator<StoreFile> FILE_SIZE =
1826       Ordering.natural().reverse().onResultOf(new Function<StoreFile, Long>() {
1827         @Override
1828         public Long apply(StoreFile sf) {
1829           return sf.getReader().length();
1830         }
1831       });
1832   }
1833 }