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.HBaseFileSystem;
45  import org.apache.hadoop.hbase.HConstants;
46  import org.apache.hadoop.hbase.HDFSBlocksDistribution;
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   public 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     HBaseFileSystem.deleteDirFromFileSystem(fs, getPath());
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 (!HBaseFileSystem.renameDirForFileSystem(fs, 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     private boolean includeMVCCReadpoint = true;
745 
746     public WriterBuilder(Configuration conf, CacheConfig cacheConf,
747         FileSystem fs, int blockSize) {
748       this.conf = conf;
749       this.cacheConf = cacheConf;
750       this.fs = fs;
751       this.blockSize = blockSize;
752     }
753 
754     /**
755      * Use either this method or {@link #withFilePath}, but not both.
756      * @param dir Path to column family directory. The directory is created if
757      *          does not exist. The file is given a unique name within this
758      *          directory.
759      * @return this (for chained invocation)
760      */
761     public WriterBuilder withOutputDir(Path dir) {
762       Preconditions.checkNotNull(dir);
763       this.dir = dir;
764       return this;
765     }
766 
767     /**
768      * Use either this method or {@link #withOutputDir}, but not both.
769      * @param filePath the StoreFile path to write
770      * @return this (for chained invocation)
771      */
772     public WriterBuilder withFilePath(Path filePath) {
773       Preconditions.checkNotNull(filePath);
774       this.filePath = filePath;
775       return this;
776     }
777 
778     public WriterBuilder withCompression(Compression.Algorithm compressAlgo) {
779       Preconditions.checkNotNull(compressAlgo);
780       this.compressAlgo = compressAlgo;
781       return this;
782     }
783 
784     public WriterBuilder withDataBlockEncoder(HFileDataBlockEncoder encoder) {
785       Preconditions.checkNotNull(encoder);
786       this.dataBlockEncoder = encoder;
787       return this;
788     }
789 
790     public WriterBuilder withComparator(KeyValue.KVComparator comparator) {
791       Preconditions.checkNotNull(comparator);
792       this.comparator = comparator;
793       return this;
794     }
795 
796     public WriterBuilder withBloomType(BloomType bloomType) {
797       Preconditions.checkNotNull(bloomType);
798       this.bloomType = bloomType;
799       return this;
800     }
801 
802     /**
803      * @param maxKeyCount estimated maximum number of keys we expect to add
804      * @return this (for chained invocation)
805      */
806     public WriterBuilder withMaxKeyCount(long maxKeyCount) {
807       this.maxKeyCount = maxKeyCount;
808       return this;
809     }
810 
811     /**
812      * @param checksumType the type of checksum
813      * @return this (for chained invocation)
814      */
815     public WriterBuilder withChecksumType(ChecksumType checksumType) {
816       this.checksumType = checksumType;
817       return this;
818     }
819 
820     /**
821      * @param bytesPerChecksum the number of bytes per checksum chunk
822      * @return this (for chained invocation)
823      */
824     public WriterBuilder withBytesPerChecksum(int bytesPerChecksum) {
825       this.bytesPerChecksum = bytesPerChecksum;
826       return this;
827     }
828 
829     /**
830      * @param includeMVCCReadpoint whether to write the mvcc readpoint to the file for each KV
831      * @return this (for chained invocation)
832      */
833     public WriterBuilder includeMVCCReadpoint(boolean includeMVCCReadpoint) {
834       this.includeMVCCReadpoint = includeMVCCReadpoint;
835       return this;
836     }
837 
838     /**
839      * Create a store file writer. Client is responsible for closing file when
840      * done. If metadata, add BEFORE closing using
841      * {@link Writer#appendMetadata}.
842      */
843     public Writer build() throws IOException {
844       if ((dir == null ? 0 : 1) + (filePath == null ? 0 : 1) != 1) {
845         throw new IllegalArgumentException("Either specify parent directory " +
846             "or file path");
847       }
848 
849       if (dir == null) {
850         dir = filePath.getParent();
851       }
852 
853       if (!fs.exists(dir)) {
854         HBaseFileSystem.makeDirOnFileSystem(fs, dir);
855       }
856 
857       if (filePath == null) {
858         filePath = getUniqueFile(fs, dir);
859         if (!BloomFilterFactory.isGeneralBloomEnabled(conf)) {
860           bloomType = BloomType.NONE;
861         }
862       }
863 
864       if (compressAlgo == null) {
865         compressAlgo = HFile.DEFAULT_COMPRESSION_ALGORITHM;
866       }
867       if (comparator == null) {
868         comparator = KeyValue.COMPARATOR;
869       }
870       return new Writer(fs, filePath, blockSize, compressAlgo, dataBlockEncoder,
871           conf, cacheConf, comparator, bloomType, maxKeyCount, checksumType,
872           bytesPerChecksum, includeMVCCReadpoint);
873     }
874   }
875 
876   /**
877    * @param fs
878    * @param dir Directory to create file in.
879    * @return random filename inside passed <code>dir</code>
880    */
881   public static Path getUniqueFile(final FileSystem fs, final Path dir)
882       throws IOException {
883     if (!fs.getFileStatus(dir).isDir()) {
884       throw new IOException("Expecting " + dir.toString() +
885         " to be a directory");
886     }
887     return getRandomFilename(fs, dir);
888   }
889 
890   /**
891    *
892    * @param fs
893    * @param dir
894    * @return Path to a file that doesn't exist at time of this invocation.
895    * @throws IOException
896    */
897   static Path getRandomFilename(final FileSystem fs, final Path dir)
898       throws IOException {
899     return getRandomFilename(fs, dir, null);
900   }
901 
902   /**
903    *
904    * @param fs
905    * @param dir
906    * @param suffix
907    * @return Path to a file that doesn't exist at time of this invocation.
908    * @throws IOException
909    */
910   static Path getRandomFilename(final FileSystem fs,
911                                 final Path dir,
912                                 final String suffix)
913       throws IOException {
914     return new Path(dir, UUID.randomUUID().toString().replaceAll("-", "")
915         + (suffix == null ? "" : suffix));
916   }
917 
918   /**
919    * Validate the store file name.
920    * @param fileName name of the file to validate
921    * @return <tt>true</tt> if the file could be a valid store file, <tt>false</tt> otherwise
922    */
923   public static boolean validateStoreFileName(String fileName) {
924     if (HFileLink.isHFileLink(fileName))
925       return true;
926     if (isReference(fileName))
927       return true;
928     return !fileName.contains("-");
929   }
930 
931   /**
932    * Write out a split reference. Package local so it doesnt leak out of
933    * regionserver.
934    * @param fs
935    * @param splitDir Presumes path format is actually
936    *          <code>SOME_DIRECTORY/REGIONNAME/FAMILY</code>.
937    * @param f File to split.
938    * @param splitRow
939    * @param range
940    * @return Path to created reference.
941    * @throws IOException
942    */
943   static Path split(final FileSystem fs,
944                     final Path splitDir,
945                     final StoreFile f,
946                     final byte [] splitRow,
947                     final Reference.Range range)
948       throws IOException {
949 	    
950     // Check whether the split row lies in the range of the store file
951     // If it is outside the range, return directly.
952     if (range == Reference.Range.bottom) {
953       //check if smaller than first key
954       KeyValue splitKey = KeyValue.createLastOnRow(splitRow);
955       byte[] firstKey = f.createReader().getFirstKey();
956       // If firstKey is null means storefile is empty.
957       if (firstKey == null) return null;
958       if (f.getReader().getComparator().compare(splitKey.getBuffer(), 
959           splitKey.getKeyOffset(), splitKey.getKeyLength(), 
960           firstKey, 0, firstKey.length) < 0) {
961         return null;
962       }      
963     }
964     else {
965       //check if larger than last key.
966       KeyValue splitKey = KeyValue.createFirstOnRow(splitRow);
967       byte[] lastKey = f.createReader().getLastKey();      
968       // If lastKey is null means storefile is empty.
969       if (lastKey == null) return null;
970       if (f.getReader().getComparator().compare(splitKey.getBuffer(), 
971           splitKey.getKeyOffset(), splitKey.getKeyLength(), 
972           lastKey, 0, lastKey.length) > 0) {
973         return null;
974       }
975     }
976     
977     // A reference to the bottom half of the hsf store file.
978     Reference r = new Reference(splitRow, range);
979     // Add the referred-to regions name as a dot separated suffix.
980     // See REF_NAME_REGEX regex above.  The referred-to regions name is
981     // up in the path of the passed in <code>f</code> -- parentdir is family,
982     // then the directory above is the region name.
983     String parentRegionName = f.getPath().getParent().getParent().getName();
984     // Write reference with same file id only with the other region name as
985     // suffix and into the new region location (under same family).
986     Path p = new Path(splitDir, f.getPath().getName() + "." + parentRegionName);
987     return r.write(fs, p);
988   }
989 
990 
991   /**
992    * A StoreFile writer.  Use this to read/write HBase Store Files. It is package
993    * local because it is an implementation detail of the HBase regionserver.
994    */
995   public static class Writer {
996     private final BloomFilterWriter generalBloomFilterWriter;
997     private final BloomFilterWriter deleteFamilyBloomFilterWriter;
998     private final BloomType bloomType;
999     private byte[] lastBloomKey;
1000     private int lastBloomKeyOffset, lastBloomKeyLen;
1001     private KVComparator kvComparator;
1002     private KeyValue lastKv = null;
1003     private long earliestPutTs = HConstants.LATEST_TIMESTAMP;
1004     private KeyValue lastDeleteFamilyKV = null;
1005     private long deleteFamilyCnt = 0;
1006 
1007     protected HFileDataBlockEncoder dataBlockEncoder;
1008 
1009     /** Checksum type */
1010     protected ChecksumType checksumType;
1011 
1012     /** Bytes per Checksum */
1013     protected int bytesPerChecksum;
1014     
1015     TimeRangeTracker timeRangeTracker = new TimeRangeTracker();
1016     /* isTimeRangeTrackerSet keeps track if the timeRange has already been set
1017      * When flushing a memstore, we set TimeRange and use this variable to
1018      * indicate that it doesn't need to be calculated again while
1019      * appending KeyValues.
1020      * It is not set in cases of compactions when it is recalculated using only
1021      * the appended KeyValues*/
1022     boolean isTimeRangeTrackerSet = false;
1023 
1024     protected HFile.Writer writer;
1025 
1026     /**
1027      * Creates an HFile.Writer that also write helpful meta data.
1028      * @param fs file system to write to
1029      * @param path file name to create
1030      * @param blocksize HDFS block size
1031      * @param compress HDFS block compression
1032      * @param conf user configuration
1033      * @param comparator key comparator
1034      * @param bloomType bloom filter setting
1035      * @param maxKeys the expected maximum number of keys to be added. Was used
1036      *        for Bloom filter size in {@link HFile} format version 1.
1037      * @param checksumType the checksum type
1038      * @param bytesPerChecksum the number of bytes per checksum value
1039      * @param includeMVCCReadpoint whether to write the mvcc readpoint to the file for each KV
1040      * @throws IOException problem writing to FS
1041      */
1042     private Writer(FileSystem fs, Path path, int blocksize,
1043         Compression.Algorithm compress,
1044         HFileDataBlockEncoder dataBlockEncoder, final Configuration conf,
1045         CacheConfig cacheConf,
1046         final KVComparator comparator, BloomType bloomType, long maxKeys,
1047         final ChecksumType checksumType, final int bytesPerChecksum, boolean includeMVCCReadpoint)
1048         throws IOException {
1049       this.dataBlockEncoder = dataBlockEncoder != null ?
1050           dataBlockEncoder : NoOpDataBlockEncoder.INSTANCE;
1051       writer = HFile.getWriterFactory(conf, cacheConf)
1052           .withPath(fs, path)
1053           .withBlockSize(blocksize)
1054           .withCompression(compress)
1055           .withDataBlockEncoder(dataBlockEncoder)
1056           .withComparator(comparator.getRawComparator())
1057           .withChecksumType(checksumType)
1058           .withBytesPerChecksum(bytesPerChecksum)
1059           .includeMVCCReadpoint(includeMVCCReadpoint)
1060           .create();
1061 
1062       this.kvComparator = comparator;
1063 
1064       generalBloomFilterWriter = BloomFilterFactory.createGeneralBloomAtWrite(
1065           conf, cacheConf, bloomType,
1066           (int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
1067 
1068       if (generalBloomFilterWriter != null) {
1069         this.bloomType = bloomType;
1070         LOG.info("Bloom filter type for " + path + ": " + this.bloomType + ", "
1071             + generalBloomFilterWriter.getClass().getSimpleName());
1072       } else {
1073         // Not using Bloom filters.
1074         this.bloomType = BloomType.NONE;
1075       }
1076 
1077       // initialize delete family Bloom filter when there is NO RowCol Bloom
1078       // filter
1079       if (this.bloomType != BloomType.ROWCOL) {
1080         this.deleteFamilyBloomFilterWriter = BloomFilterFactory
1081             .createDeleteBloomAtWrite(conf, cacheConf,
1082                 (int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
1083       } else {
1084         deleteFamilyBloomFilterWriter = null;
1085       }
1086       if (deleteFamilyBloomFilterWriter != null) {
1087         LOG.info("Delete Family Bloom filter type for " + path + ": "
1088             + deleteFamilyBloomFilterWriter.getClass().getSimpleName());
1089       }
1090       this.checksumType = checksumType;
1091       this.bytesPerChecksum = bytesPerChecksum;
1092     }
1093 
1094     /**
1095      * Writes meta data.
1096      * Call before {@link #close()} since its written as meta data to this file.
1097      * @param maxSequenceId Maximum sequence id.
1098      * @param majorCompaction True if this file is product of a major compaction
1099      * @throws IOException problem writing to FS
1100      */
1101     public void appendMetadata(final long maxSequenceId, final boolean majorCompaction)
1102     throws IOException {
1103       writer.appendFileInfo(MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId));
1104       writer.appendFileInfo(MAJOR_COMPACTION_KEY,
1105           Bytes.toBytes(majorCompaction));
1106       appendTrackedTimestampsToMetadata();
1107     }
1108 
1109     /**
1110      * Add TimestampRange and earliest put timestamp to Metadata
1111      */
1112     public void appendTrackedTimestampsToMetadata() throws IOException {
1113       appendFileInfo(TIMERANGE_KEY,WritableUtils.toByteArray(timeRangeTracker));
1114       appendFileInfo(EARLIEST_PUT_TS, Bytes.toBytes(earliestPutTs));
1115     }
1116 
1117     /**
1118      * Set TimeRangeTracker
1119      * @param trt
1120      */
1121     public void setTimeRangeTracker(final TimeRangeTracker trt) {
1122       this.timeRangeTracker = trt;
1123       isTimeRangeTrackerSet = true;
1124     }
1125 
1126     /**
1127      * Record the earlest Put timestamp.
1128      *
1129      * If the timeRangeTracker is not set,
1130      * update TimeRangeTracker to include the timestamp of this key
1131      * @param kv
1132      */
1133     public void trackTimestamps(final KeyValue kv) {
1134       if (KeyValue.Type.Put.getCode() == kv.getType()) {
1135         earliestPutTs = Math.min(earliestPutTs, kv.getTimestamp());
1136       }
1137       if (!isTimeRangeTrackerSet) {
1138         timeRangeTracker.includeTimestamp(kv);
1139       }
1140     }
1141 
1142     private void appendGeneralBloomfilter(final KeyValue kv) throws IOException {
1143       if (this.generalBloomFilterWriter != null) {
1144         // only add to the bloom filter on a new, unique key
1145         boolean newKey = true;
1146         if (this.lastKv != null) {
1147           switch(bloomType) {
1148           case ROW:
1149             newKey = ! kvComparator.matchingRows(kv, lastKv);
1150             break;
1151           case ROWCOL:
1152             newKey = ! kvComparator.matchingRowColumn(kv, lastKv);
1153             break;
1154           case NONE:
1155             newKey = false;
1156             break;
1157           default:
1158             throw new IOException("Invalid Bloom filter type: " + bloomType +
1159                 " (ROW or ROWCOL expected)");
1160           }
1161         }
1162         if (newKey) {
1163           /*
1164            * http://2.bp.blogspot.com/_Cib_A77V54U/StZMrzaKufI/AAAAAAAAADo/ZhK7bGoJdMQ/s400/KeyValue.png
1165            * Key = RowLen + Row + FamilyLen + Column [Family + Qualifier] + TimeStamp
1166            *
1167            * 2 Types of Filtering:
1168            *  1. Row = Row
1169            *  2. RowCol = Row + Qualifier
1170            */
1171           byte[] bloomKey;
1172           int bloomKeyOffset, bloomKeyLen;
1173 
1174           switch (bloomType) {
1175           case ROW:
1176             bloomKey = kv.getBuffer();
1177             bloomKeyOffset = kv.getRowOffset();
1178             bloomKeyLen = kv.getRowLength();
1179             break;
1180           case ROWCOL:
1181             // merge(row, qualifier)
1182             // TODO: could save one buffer copy in case of compound Bloom
1183             // filters when this involves creating a KeyValue
1184             bloomKey = generalBloomFilterWriter.createBloomKey(kv.getBuffer(),
1185                 kv.getRowOffset(), kv.getRowLength(), kv.getBuffer(),
1186                 kv.getQualifierOffset(), kv.getQualifierLength());
1187             bloomKeyOffset = 0;
1188             bloomKeyLen = bloomKey.length;
1189             break;
1190           default:
1191             throw new IOException("Invalid Bloom filter type: " + bloomType +
1192                 " (ROW or ROWCOL expected)");
1193           }
1194           generalBloomFilterWriter.add(bloomKey, bloomKeyOffset, bloomKeyLen);
1195           if (lastBloomKey != null
1196               && generalBloomFilterWriter.getComparator().compare(bloomKey,
1197                   bloomKeyOffset, bloomKeyLen, lastBloomKey,
1198                   lastBloomKeyOffset, lastBloomKeyLen) <= 0) {
1199             throw new IOException("Non-increasing Bloom keys: "
1200                 + Bytes.toStringBinary(bloomKey, bloomKeyOffset, bloomKeyLen)
1201                 + " after "
1202                 + Bytes.toStringBinary(lastBloomKey, lastBloomKeyOffset,
1203                     lastBloomKeyLen));
1204           }
1205           lastBloomKey = bloomKey;
1206           lastBloomKeyOffset = bloomKeyOffset;
1207           lastBloomKeyLen = bloomKeyLen;
1208           this.lastKv = kv;
1209         }
1210       }
1211     }
1212 
1213     private void appendDeleteFamilyBloomFilter(final KeyValue kv)
1214         throws IOException {
1215       if (!kv.isDeleteFamily()) {
1216         return;
1217       }
1218 
1219       // increase the number of delete family in the store file
1220       deleteFamilyCnt++;
1221       if (null != this.deleteFamilyBloomFilterWriter) {
1222         boolean newKey = true;
1223         if (lastDeleteFamilyKV != null) {
1224           newKey = !kvComparator.matchingRows(kv, lastDeleteFamilyKV);
1225         }
1226         if (newKey) {
1227           this.deleteFamilyBloomFilterWriter.add(kv.getBuffer(),
1228               kv.getRowOffset(), kv.getRowLength());
1229           this.lastDeleteFamilyKV = kv;
1230         }
1231       }
1232     }
1233 
1234     public void append(final KeyValue kv) throws IOException {
1235       appendGeneralBloomfilter(kv);
1236       appendDeleteFamilyBloomFilter(kv);
1237       writer.append(kv);
1238       trackTimestamps(kv);
1239     }
1240 
1241     public Path getPath() {
1242       return this.writer.getPath();
1243     }
1244 
1245     boolean hasGeneralBloom() {
1246       return this.generalBloomFilterWriter != null;
1247     }
1248 
1249     /**
1250      * For unit testing only.
1251      *
1252      * @return the Bloom filter used by this writer.
1253      */
1254     BloomFilterWriter getGeneralBloomWriter() {
1255       return generalBloomFilterWriter;
1256     }
1257 
1258     private boolean closeBloomFilter(BloomFilterWriter bfw) throws IOException {
1259       boolean haveBloom = (bfw != null && bfw.getKeyCount() > 0);
1260       if (haveBloom) {
1261         bfw.compactBloom();
1262       }
1263       return haveBloom;
1264     }
1265 
1266     private boolean closeGeneralBloomFilter() throws IOException {
1267       boolean hasGeneralBloom = closeBloomFilter(generalBloomFilterWriter);
1268 
1269       // add the general Bloom filter writer and append file info
1270       if (hasGeneralBloom) {
1271         writer.addGeneralBloomFilter(generalBloomFilterWriter);
1272         writer.appendFileInfo(BLOOM_FILTER_TYPE_KEY,
1273             Bytes.toBytes(bloomType.toString()));
1274         if (lastBloomKey != null) {
1275           writer.appendFileInfo(LAST_BLOOM_KEY, Arrays.copyOfRange(
1276               lastBloomKey, lastBloomKeyOffset, lastBloomKeyOffset
1277                   + lastBloomKeyLen));
1278         }
1279       }
1280       return hasGeneralBloom;
1281     }
1282 
1283     private boolean closeDeleteFamilyBloomFilter() throws IOException {
1284       boolean hasDeleteFamilyBloom = closeBloomFilter(deleteFamilyBloomFilterWriter);
1285 
1286       // add the delete family Bloom filter writer
1287       if (hasDeleteFamilyBloom) {
1288         writer.addDeleteFamilyBloomFilter(deleteFamilyBloomFilterWriter);
1289       }
1290 
1291       // append file info about the number of delete family kvs
1292       // even if there is no delete family Bloom.
1293       writer.appendFileInfo(DELETE_FAMILY_COUNT,
1294           Bytes.toBytes(this.deleteFamilyCnt));
1295 
1296       return hasDeleteFamilyBloom;
1297     }
1298 
1299     public void close() throws IOException {
1300       boolean hasGeneralBloom = this.closeGeneralBloomFilter();
1301       boolean hasDeleteFamilyBloom = this.closeDeleteFamilyBloomFilter();
1302 
1303       writer.close();
1304 
1305       // Log final Bloom filter statistics. This needs to be done after close()
1306       // because compound Bloom filters might be finalized as part of closing.
1307       StoreFile.LOG.info((hasGeneralBloom ? "" : "NO ") + "General Bloom and "
1308           + (hasDeleteFamilyBloom ? "" : "NO ") + "DeleteFamily"
1309           + " was added to HFile (" + getPath() + ") ");
1310 
1311     }
1312 
1313     public void appendFileInfo(byte[] key, byte[] value) throws IOException {
1314       writer.appendFileInfo(key, value);
1315     }
1316 
1317     /** For use in testing, e.g. {@link CreateRandomStoreFile} */
1318     HFile.Writer getHFileWriter() {
1319       return writer;
1320     }
1321   }
1322 
1323   /**
1324    * Reader for a StoreFile.
1325    */
1326   public static class Reader extends SchemaConfigured {
1327     static final Log LOG = LogFactory.getLog(Reader.class.getName());
1328 
1329     protected BloomFilter generalBloomFilter = null;
1330     protected BloomFilter deleteFamilyBloomFilter = null;
1331     protected BloomType bloomFilterType;
1332     private final HFile.Reader reader;
1333     protected TimeRangeTracker timeRangeTracker = null;
1334     protected long sequenceID = -1;
1335     private byte[] lastBloomKey;
1336     private long deleteFamilyCnt = -1;
1337 
1338     public Reader(FileSystem fs, Path path, CacheConfig cacheConf,
1339         DataBlockEncoding preferredEncodingInCache) throws IOException {
1340       super(path);
1341       reader = HFile.createReaderWithEncoding(fs, path, cacheConf,
1342           preferredEncodingInCache);
1343       bloomFilterType = BloomType.NONE;
1344     }
1345 
1346     public Reader(FileSystem fs, Path path, HFileLink hfileLink, long size,
1347         CacheConfig cacheConf, DataBlockEncoding preferredEncodingInCache,
1348         boolean closeIStream) throws IOException {
1349       super(path);
1350 
1351       FSDataInputStream in = hfileLink.open(fs);
1352       FSDataInputStream inNoChecksum = in;
1353       if (fs instanceof HFileSystem) {
1354         FileSystem noChecksumFs = ((HFileSystem)fs).getNoChecksumFs();
1355         inNoChecksum = hfileLink.open(noChecksumFs);
1356       }
1357 
1358       reader = HFile.createReaderWithEncoding(fs, path, in, inNoChecksum,
1359                   size, cacheConf, preferredEncodingInCache, closeIStream);
1360       bloomFilterType = BloomType.NONE;
1361     }
1362 
1363     /**
1364      * ONLY USE DEFAULT CONSTRUCTOR FOR UNIT TESTS
1365      */
1366     Reader() {
1367       this.reader = null;
1368     }
1369 
1370     public RawComparator<byte []> getComparator() {
1371       return reader.getComparator();
1372     }
1373 
1374     /**
1375      * Get a scanner to scan over this StoreFile. Do not use
1376      * this overload if using this scanner for compactions.
1377      *
1378      * @param cacheBlocks should this scanner cache blocks?
1379      * @param pread use pread (for highly concurrent small readers)
1380      * @return a scanner
1381      */
1382     public StoreFileScanner getStoreFileScanner(boolean cacheBlocks,
1383                                                boolean pread) {
1384       return getStoreFileScanner(cacheBlocks, pread, false);
1385     }
1386 
1387     /**
1388      * Get a scanner to scan over this StoreFile.
1389      *
1390      * @param cacheBlocks should this scanner cache blocks?
1391      * @param pread use pread (for highly concurrent small readers)
1392      * @param isCompaction is scanner being used for compaction?
1393      * @return a scanner
1394      */
1395     public StoreFileScanner getStoreFileScanner(boolean cacheBlocks,
1396                                                boolean pread,
1397                                                boolean isCompaction) {
1398       return new StoreFileScanner(this,
1399                                  getScanner(cacheBlocks, pread,
1400                                             isCompaction), !isCompaction);
1401     }
1402 
1403     /**
1404      * Warning: Do not write further code which depends on this call. Instead
1405      * use getStoreFileScanner() which uses the StoreFileScanner class/interface
1406      * which is the preferred way to scan a store with higher level concepts.
1407      *
1408      * @param cacheBlocks should we cache the blocks?
1409      * @param pread use pread (for concurrent small readers)
1410      * @return the underlying HFileScanner
1411      */
1412     @Deprecated
1413     public HFileScanner getScanner(boolean cacheBlocks, boolean pread) {
1414       return getScanner(cacheBlocks, pread, false);
1415     }
1416 
1417     /**
1418      * Warning: Do not write further code which depends on this call. Instead
1419      * use getStoreFileScanner() which uses the StoreFileScanner class/interface
1420      * which is the preferred way to scan a store with higher level concepts.
1421      *
1422      * @param cacheBlocks
1423      *          should we cache the blocks?
1424      * @param pread
1425      *          use pread (for concurrent small readers)
1426      * @param isCompaction
1427      *          is scanner being used for compaction?
1428      * @return the underlying HFileScanner
1429      */
1430     @Deprecated
1431     public HFileScanner getScanner(boolean cacheBlocks, boolean pread,
1432         boolean isCompaction) {
1433       return reader.getScanner(cacheBlocks, pread, isCompaction);
1434     }
1435 
1436     public void close(boolean evictOnClose) throws IOException {
1437       reader.close(evictOnClose);
1438     }
1439 
1440     /**
1441      * Check if this storeFile may contain keys within the TimeRange that
1442      * have not expired (i.e. not older than oldestUnexpiredTS).
1443      * @param scan the current scan
1444      * @param oldestUnexpiredTS the oldest timestamp that is not expired, as
1445      *          determined by the column family's TTL
1446      * @return false if queried keys definitely don't exist in this StoreFile
1447      */
1448     boolean passesTimerangeFilter(Scan scan, long oldestUnexpiredTS) {
1449       if (timeRangeTracker == null) {
1450         return true;
1451       } else {
1452         return timeRangeTracker.includesTimeRange(scan.getTimeRange()) &&
1453             timeRangeTracker.getMaximumTimestamp() >= oldestUnexpiredTS;
1454       }
1455     }
1456 
1457     /**
1458      * Checks whether the given scan passes the Bloom filter (if present). Only
1459      * checks Bloom filters for single-row or single-row-column scans. Bloom
1460      * filter checking for multi-gets is implemented as part of the store
1461      * scanner system (see {@link StoreFileScanner#seekExactly}) and uses
1462      * the lower-level API {@link #passesGeneralBloomFilter(byte[], int, int, byte[],
1463      * int, int)}.
1464      *
1465      * @param scan the scan specification. Used to determine the row, and to
1466      *          check whether this is a single-row ("get") scan.
1467      * @param columns the set of columns. Only used for row-column Bloom
1468      *          filters.
1469      * @return true if the scan with the given column set passes the Bloom
1470      *         filter, or if the Bloom filter is not applicable for the scan.
1471      *         False if the Bloom filter is applicable and the scan fails it.
1472      */
1473      boolean passesBloomFilter(Scan scan,
1474         final SortedSet<byte[]> columns) {
1475       // Multi-column non-get scans will use Bloom filters through the
1476       // lower-level API function that this function calls.
1477       if (!scan.isGetScan()) {
1478         return true;
1479       }
1480 
1481       byte[] row = scan.getStartRow();
1482       switch (this.bloomFilterType) {
1483         case ROW:
1484           return passesGeneralBloomFilter(row, 0, row.length, null, 0, 0);
1485 
1486         case ROWCOL:
1487           if (columns != null && columns.size() == 1) {
1488             byte[] column = columns.first();
1489             return passesGeneralBloomFilter(row, 0, row.length, column, 0,
1490                 column.length);
1491           }
1492 
1493           // For multi-column queries the Bloom filter is checked from the
1494           // seekExact operation.
1495           return true;
1496 
1497         default:
1498           return true;
1499       }
1500     }
1501 
1502     public boolean passesDeleteFamilyBloomFilter(byte[] row, int rowOffset,
1503         int rowLen) {
1504       // Cache Bloom filter as a local variable in case it is set to null by
1505       // another thread on an IO error.
1506       BloomFilter bloomFilter = this.deleteFamilyBloomFilter;
1507 
1508       // Empty file or there is no delete family at all
1509       if (reader.getTrailer().getEntryCount() == 0 || deleteFamilyCnt == 0) {
1510         return false;
1511       }
1512 
1513       if (bloomFilter == null) {
1514         return true;
1515       }
1516 
1517       try {
1518         if (!bloomFilter.supportsAutoLoading()) {
1519           return true;
1520         }
1521         return bloomFilter.contains(row, rowOffset, rowLen, null);
1522       } catch (IllegalArgumentException e) {
1523         LOG.error("Bad Delete Family bloom filter data -- proceeding without",
1524             e);
1525         setDeleteFamilyBloomFilterFaulty();
1526       }
1527 
1528       return true;
1529     }
1530 
1531     /**
1532      * A method for checking Bloom filters. Called directly from
1533      * StoreFileScanner in case of a multi-column query.
1534      *
1535      * @param row
1536      * @param rowOffset
1537      * @param rowLen
1538      * @param col
1539      * @param colOffset
1540      * @param colLen
1541      * @return True if passes
1542      */
1543     public boolean passesGeneralBloomFilter(byte[] row, int rowOffset,
1544         int rowLen, byte[] col, int colOffset, int colLen) {
1545       if (generalBloomFilter == null)
1546         return true;
1547 
1548       byte[] key;
1549       switch (bloomFilterType) {
1550         case ROW:
1551           if (col != null) {
1552             throw new RuntimeException("Row-only Bloom filter called with " +
1553                 "column specified");
1554           }
1555           if (rowOffset != 0 || rowLen != row.length) {
1556               throw new AssertionError("For row-only Bloom filters the row "
1557                   + "must occupy the whole array");
1558           }
1559           key = row;
1560           break;
1561 
1562         case ROWCOL:
1563           key = generalBloomFilter.createBloomKey(row, rowOffset, rowLen, col,
1564               colOffset, colLen);
1565           break;
1566 
1567         default:
1568           return true;
1569       }
1570 
1571       // Cache Bloom filter as a local variable in case it is set to null by
1572       // another thread on an IO error.
1573       BloomFilter bloomFilter = this.generalBloomFilter;
1574 
1575       if (bloomFilter == null) {
1576         return true;
1577       }
1578 
1579       // Empty file
1580       if (reader.getTrailer().getEntryCount() == 0)
1581         return false;
1582 
1583       try {
1584         boolean shouldCheckBloom;
1585         ByteBuffer bloom;
1586         if (bloomFilter.supportsAutoLoading()) {
1587           bloom = null;
1588           shouldCheckBloom = true;
1589         } else {
1590           bloom = reader.getMetaBlock(HFileWriterV1.BLOOM_FILTER_DATA_KEY,
1591               true);
1592           shouldCheckBloom = bloom != null;
1593         }
1594 
1595         if (shouldCheckBloom) {
1596           boolean exists;
1597 
1598           // Whether the primary Bloom key is greater than the last Bloom key
1599           // from the file info. For row-column Bloom filters this is not yet
1600           // a sufficient condition to return false.
1601           boolean keyIsAfterLast = lastBloomKey != null
1602               && bloomFilter.getComparator().compare(key, lastBloomKey) > 0;
1603 
1604           if (bloomFilterType == BloomType.ROWCOL) {
1605             // Since a Row Delete is essentially a DeleteFamily applied to all
1606             // columns, a file might be skipped if using row+col Bloom filter.
1607             // In order to ensure this file is included an additional check is
1608             // required looking only for a row bloom.
1609             byte[] rowBloomKey = bloomFilter.createBloomKey(row, 0, row.length,
1610                 null, 0, 0);
1611 
1612             if (keyIsAfterLast
1613                 && bloomFilter.getComparator().compare(rowBloomKey,
1614                     lastBloomKey) > 0) {
1615               exists = false;
1616             } else {
1617               exists =
1618                   bloomFilter.contains(key, 0, key.length, bloom) ||
1619                   bloomFilter.contains(rowBloomKey, 0, rowBloomKey.length,
1620                       bloom);
1621             }
1622           } else {
1623             exists = !keyIsAfterLast
1624                 && bloomFilter.contains(key, 0, key.length, bloom);
1625           }
1626 
1627           getSchemaMetrics().updateBloomMetrics(exists);
1628           return exists;
1629         }
1630       } catch (IOException e) {
1631         LOG.error("Error reading bloom filter data -- proceeding without",
1632             e);
1633         setGeneralBloomFilterFaulty();
1634       } catch (IllegalArgumentException e) {
1635         LOG.error("Bad bloom filter data -- proceeding without", e);
1636         setGeneralBloomFilterFaulty();
1637       }
1638 
1639       return true;
1640     }
1641 
1642     /**
1643      * Checks whether the given scan rowkey range overlaps with the current storefile's
1644      * @param scan the scan specification. Used to determine the rowkey range.
1645      * @return true if there is overlap, false otherwise
1646      */
1647     public boolean passesKeyRangeFilter(Scan scan) {
1648       if (this.getFirstKey() == null || this.getLastKey() == null) {
1649         // the file is empty
1650         return false;
1651       }
1652       if (Bytes.equals(scan.getStartRow(), HConstants.EMPTY_START_ROW)
1653           && Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW)) {
1654         return true;
1655       }
1656       KeyValue startKeyValue = KeyValue.createFirstOnRow(scan.getStartRow());
1657       KeyValue stopKeyValue = KeyValue.createLastOnRow(scan.getStopRow());
1658       boolean nonOverLapping = (getComparator().compare(this.getFirstKey(),
1659         stopKeyValue.getKey()) > 0 && !Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW))
1660           || getComparator().compare(this.getLastKey(), startKeyValue.getKey()) < 0;
1661       return !nonOverLapping;
1662     }
1663 
1664     public Map<byte[], byte[]> loadFileInfo() throws IOException {
1665       Map<byte [], byte []> fi = reader.loadFileInfo();
1666 
1667       byte[] b = fi.get(BLOOM_FILTER_TYPE_KEY);
1668       if (b != null) {
1669         bloomFilterType = BloomType.valueOf(Bytes.toString(b));
1670       }
1671 
1672       lastBloomKey = fi.get(LAST_BLOOM_KEY);
1673       byte[] cnt = fi.get(DELETE_FAMILY_COUNT);
1674       if (cnt != null) {
1675         deleteFamilyCnt = Bytes.toLong(cnt);
1676       }
1677 
1678       return fi;
1679     }
1680 
1681     public void loadBloomfilter() {
1682       this.loadBloomfilter(BlockType.GENERAL_BLOOM_META);
1683       this.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META);
1684     }
1685 
1686     private void loadBloomfilter(BlockType blockType) {
1687       try {
1688         if (blockType == BlockType.GENERAL_BLOOM_META) {
1689           if (this.generalBloomFilter != null)
1690             return; // Bloom has been loaded
1691 
1692           DataInput bloomMeta = reader.getGeneralBloomFilterMetadata();
1693           if (bloomMeta != null) {
1694             // sanity check for NONE Bloom filter
1695             if (bloomFilterType == BloomType.NONE) {
1696               throw new IOException(
1697                   "valid bloom filter type not found in FileInfo");
1698             } else {
1699               generalBloomFilter = BloomFilterFactory.createFromMeta(bloomMeta,
1700                   reader);
1701               LOG.info("Loaded " + bloomFilterType.toString() + " ("
1702                   + generalBloomFilter.getClass().getSimpleName()
1703                   + ") metadata for " + reader.getName());
1704             }
1705           }
1706         } else if (blockType == BlockType.DELETE_FAMILY_BLOOM_META) {
1707           if (this.deleteFamilyBloomFilter != null)
1708             return; // Bloom has been loaded
1709 
1710           DataInput bloomMeta = reader.getDeleteBloomFilterMetadata();
1711           if (bloomMeta != null) {
1712             deleteFamilyBloomFilter = BloomFilterFactory.createFromMeta(
1713                 bloomMeta, reader);
1714             LOG.info("Loaded Delete Family Bloom ("
1715                 + deleteFamilyBloomFilter.getClass().getSimpleName()
1716                 + ") metadata for " + reader.getName());
1717           }
1718         } else {
1719           throw new RuntimeException("Block Type: " + blockType.toString()
1720               + "is not supported for Bloom filter");
1721         }
1722       } catch (IOException e) {
1723         LOG.error("Error reading bloom filter meta for " + blockType
1724             + " -- proceeding without", e);
1725         setBloomFilterFaulty(blockType);
1726       } catch (IllegalArgumentException e) {
1727         LOG.error("Bad bloom filter meta " + blockType
1728             + " -- proceeding without", e);
1729         setBloomFilterFaulty(blockType);
1730       }
1731     }
1732 
1733     private void setBloomFilterFaulty(BlockType blockType) {
1734       if (blockType == BlockType.GENERAL_BLOOM_META) {
1735         setGeneralBloomFilterFaulty();
1736       } else if (blockType == BlockType.DELETE_FAMILY_BLOOM_META) {
1737         setDeleteFamilyBloomFilterFaulty();
1738       }
1739     }
1740 
1741     /**
1742      * The number of Bloom filter entries in this store file, or an estimate
1743      * thereof, if the Bloom filter is not loaded. This always returns an upper
1744      * bound of the number of Bloom filter entries.
1745      *
1746      * @return an estimate of the number of Bloom filter entries in this file
1747      */
1748     public long getFilterEntries() {
1749       return generalBloomFilter != null ? generalBloomFilter.getKeyCount()
1750           : reader.getEntries();
1751     }
1752 
1753     public void setGeneralBloomFilterFaulty() {
1754       generalBloomFilter = null;
1755     }
1756 
1757     public void setDeleteFamilyBloomFilterFaulty() {
1758       this.deleteFamilyBloomFilter = null;
1759     }
1760 
1761     public byte[] getLastKey() {
1762       return reader.getLastKey();
1763     }
1764 
1765     public byte[] midkey() throws IOException {
1766       return reader.midkey();
1767     }
1768 
1769     public long length() {
1770       return reader.length();
1771     }
1772 
1773     public long getTotalUncompressedBytes() {
1774       return reader.getTrailer().getTotalUncompressedBytes();
1775     }
1776 
1777     public long getEntries() {
1778       return reader.getEntries();
1779     }
1780 
1781     public long getDeleteFamilyCnt() {
1782       return deleteFamilyCnt;
1783     }
1784 
1785     public byte[] getFirstKey() {
1786       return reader.getFirstKey();
1787     }
1788 
1789     public long indexSize() {
1790       return reader.indexSize();
1791     }
1792 
1793     public String getColumnFamilyName() {
1794       return reader.getColumnFamilyName();
1795     }
1796 
1797     public BloomType getBloomFilterType() {
1798       return this.bloomFilterType;
1799     }
1800 
1801     public long getSequenceID() {
1802       return sequenceID;
1803     }
1804 
1805     public void setSequenceID(long sequenceID) {
1806       this.sequenceID = sequenceID;
1807     }
1808 
1809     BloomFilter getGeneralBloomFilter() {
1810       return generalBloomFilter;
1811     }
1812 
1813     long getUncompressedDataIndexSize() {
1814       return reader.getTrailer().getUncompressedDataIndexSize();
1815     }
1816 
1817     public long getTotalBloomSize() {
1818       if (generalBloomFilter == null)
1819         return 0;
1820       return generalBloomFilter.getByteSize();
1821     }
1822 
1823     public int getHFileVersion() {
1824       return reader.getTrailer().getMajorVersion();
1825     }
1826 
1827     HFile.Reader getHFileReader() {
1828       return reader;
1829     }
1830 
1831     void disableBloomFilterForTesting() {
1832       generalBloomFilter = null;
1833       this.deleteFamilyBloomFilter = null;
1834     }
1835 
1836     public long getMaxTimestamp() {
1837       return timeRangeTracker == null ? Long.MAX_VALUE : timeRangeTracker.maximumTimestamp;
1838     }
1839 
1840     @Override
1841     public void schemaConfigurationChanged() {
1842       passSchemaMetricsTo((SchemaConfigured) reader);
1843     }
1844   }
1845 
1846   /**
1847    * Useful comparators for comparing StoreFiles.
1848    */
1849   abstract static class Comparators {
1850     /**
1851      * Comparator that compares based on the flush time of
1852      * the StoreFiles. All bulk loads are placed before all non-
1853      * bulk loads, and then all files are sorted by sequence ID.
1854      * If there are ties, the path name is used as a tie-breaker.
1855      */
1856     static final Comparator<StoreFile> FLUSH_TIME =
1857       Ordering.compound(ImmutableList.of(
1858           Ordering.natural().onResultOf(new GetBulkTime()),
1859           Ordering.natural().onResultOf(new GetSeqId()),
1860           Ordering.natural().onResultOf(new GetPathName())
1861       ));
1862 
1863     private static class GetBulkTime implements Function<StoreFile, Long> {
1864       @Override
1865       public Long apply(StoreFile sf) {
1866         if (!sf.isBulkLoadResult()) return Long.MAX_VALUE;
1867         return sf.getBulkLoadTimestamp();
1868       }
1869     }
1870     private static class GetSeqId implements Function<StoreFile, Long> {
1871       @Override
1872       public Long apply(StoreFile sf) {
1873         if (sf.isBulkLoadResult()) return -1L;
1874         return sf.getMaxSequenceId();
1875       }
1876     }
1877     private static class GetPathName implements Function<StoreFile, String> {
1878       @Override
1879       public String apply(StoreFile sf) {
1880         return sf.getPath().getName();
1881       }
1882     }
1883 
1884     /**
1885      * FILE_SIZE = descending sort StoreFiles (largest --> smallest in size)
1886      */
1887     static final Comparator<StoreFile> FILE_SIZE =
1888       Ordering.natural().reverse().onResultOf(new Function<StoreFile, Long>() {
1889         @Override
1890         public Long apply(StoreFile sf) {
1891           return sf.getReader().length();
1892         }
1893       });
1894   }
1895 }