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