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