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