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.FileStatus;
41  import org.apache.hadoop.fs.FileSystem;
42  import org.apache.hadoop.fs.Path;
43  import org.apache.hadoop.hbase.HDFSBlocksDistribution;
44  import org.apache.hadoop.hbase.KeyValue;
45  import org.apache.hadoop.hbase.KeyValue.KVComparator;
46  import org.apache.hadoop.hbase.client.Scan;
47  import org.apache.hadoop.hbase.io.HalfStoreFileReader;
48  import org.apache.hadoop.hbase.io.Reference;
49  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
50  import org.apache.hadoop.hbase.io.hfile.Compression;
51  import org.apache.hadoop.hbase.io.hfile.HFile;
52  import org.apache.hadoop.hbase.io.hfile.HFileScanner;
53  import org.apache.hadoop.hbase.io.hfile.HFileWriterV1;
54  import org.apache.hadoop.hbase.io.hfile.HFileWriterV2;
55  import org.apache.hadoop.hbase.util.BloomFilter;
56  import org.apache.hadoop.hbase.util.BloomFilterFactory;
57  import org.apache.hadoop.hbase.util.BloomFilterWriter;
58  import org.apache.hadoop.hbase.util.Bytes;
59  import org.apache.hadoop.hbase.util.FSUtils;
60  import org.apache.hadoop.hbase.util.Writables;
61  import org.apache.hadoop.io.RawComparator;
62  import org.apache.hadoop.io.WritableUtils;
63  
64  import com.google.common.base.Function;
65  import com.google.common.collect.ImmutableList;
66  import com.google.common.collect.Ordering;
67  
68  /**
69   * A Store data file.  Stores usually have one or more of these files.  They
70   * are produced by flushing the memstore to disk.  To
71   * create, call {@link #createWriter(FileSystem, Path, int, Configuration, CacheConfig)}
72   * and append data. Be sure to add any metadata before calling close on the
73   * Writer (Use the appendMetadata convenience methods). On close, a StoreFile
74   * is sitting in the Filesystem.  To refer to it, create a StoreFile instance
75   * passing filesystem and path.  To read, call {@link #createReader()}.
76   * <p>StoreFiles may also reference store files in another Store.
77   *
78   * The reason for this weird pattern where you use a different instance for the
79   * writer and a reader is that we write once but read a lot more.
80   */
81  public class StoreFile {
82    static final Log LOG = LogFactory.getLog(StoreFile.class.getName());
83  
84    static final String HFILE_BLOCK_CACHE_SIZE_KEY = "hfile.block.cache.size";
85  
86    public static enum BloomType {
87      /**
88       * Bloomfilters disabled
89       */
90      NONE,
91      /**
92       * Bloom enabled with Table row as Key
93       */
94      ROW,
95      /**
96       * Bloom enabled with Table row & column (family+qualifier) as Key
97       */
98      ROWCOL
99    }
100 
101   // Keys for fileinfo values in HFile
102 
103   /** Max Sequence ID in FileInfo */
104   public static final byte [] MAX_SEQ_ID_KEY = Bytes.toBytes("MAX_SEQ_ID_KEY");
105 
106   /** Major compaction flag in FileInfo */
107   public static final byte[] MAJOR_COMPACTION_KEY =
108       Bytes.toBytes("MAJOR_COMPACTION_KEY");
109 
110   /** Bloom filter Type in FileInfo */
111   static final byte[] BLOOM_FILTER_TYPE_KEY =
112       Bytes.toBytes("BLOOM_FILTER_TYPE");
113 
114   /** Last Bloom filter key in FileInfo */
115   private static final byte[] LAST_BLOOM_KEY = Bytes.toBytes("LAST_BLOOM_KEY");
116 
117   /** Key for Timerange information in metadata*/
118   public static final byte[] TIMERANGE_KEY = Bytes.toBytes("TIMERANGE");
119 
120   // Make default block size for StoreFiles 8k while testing.  TODO: FIX!
121   // Need to make it 8k for testing.
122   public static final int DEFAULT_BLOCKSIZE_SMALL = 8 * 1024;
123 
124   private final FileSystem fs;
125 
126   // This file's path.
127   private final Path path;
128 
129   // If this storefile references another, this is the reference instance.
130   private Reference reference;
131 
132   // If this StoreFile references another, this is the other files path.
133   private Path referencePath;
134 
135   // Block cache configuration and reference.
136   private final CacheConfig cacheConf;
137 
138   // HDFS blocks distribuion information
139   private HDFSBlocksDistribution hdfsBlocksDistribution;
140 
141   // Keys for metadata stored in backing HFile.
142   // Set when we obtain a Reader.
143   private long sequenceid = -1;
144 
145   // max of the MemstoreTS in the KV's in this store
146   // Set when we obtain a Reader.
147   private long maxMemstoreTS = -1;
148 
149   public long getMaxMemstoreTS() {
150     return maxMemstoreTS;
151   }
152 
153   public void setMaxMemstoreTS(long maxMemstoreTS) {
154     this.maxMemstoreTS = maxMemstoreTS;
155   }
156 
157   // If true, this file was product of a major compaction.  Its then set
158   // whenever you get a Reader.
159   private AtomicBoolean majorCompaction = null;
160 
161   /** Meta key set when store file is a result of a bulk load */
162   public static final byte[] BULKLOAD_TASK_KEY =
163     Bytes.toBytes("BULKLOAD_SOURCE_TASK");
164   public static final byte[] BULKLOAD_TIME_KEY =
165     Bytes.toBytes("BULKLOAD_TIMESTAMP");
166 
167   /**
168    * Map of the metadata entries in the corresponding HFile
169    */
170   private Map<byte[], byte[]> metadataMap;
171 
172   /*
173    * Regex that will work for straight filenames and for reference names.
174    * If reference, then the regex has more than just one group.  Group 1 is
175    * this files id.  Group 2 the referenced region name, etc.
176    */
177   private static final Pattern REF_NAME_PARSER =
178     Pattern.compile("^([0-9a-f]+)(?:\\.(.+))?$");
179 
180   // StoreFile.Reader
181   private volatile Reader reader;
182 
183   /**
184    * Bloom filter type specified in column family configuration. Does not
185    * necessarily correspond to the Bloom filter type present in the HFile.
186    */
187   private final BloomType cfBloomType;
188 
189   // the last modification time stamp
190   private long modificationTimeStamp = 0L;
191 
192   /**
193    * Constructor, loads a reader and it's indices, etc. May allocate a
194    * substantial amount of ram depending on the underlying files (10-20MB?).
195    *
196    * @param fs  The current file system to use.
197    * @param p  The path of the file.
198    * @param blockcache  <code>true</code> if the block cache is enabled.
199    * @param conf  The current configuration.
200    * @param cacheConf  The cache configuration and block cache reference.
201    * @param cfBloomType The bloom type to use for this store file as specified
202    *          by column family configuration. This may or may not be the same
203    *          as the Bloom filter type actually present in the HFile, because
204    *          column family configuration might change. If this is
205    *          {@link BloomType#NONE}, the existing Bloom filter is ignored.
206    * @throws IOException When opening the reader fails.
207    */
208   StoreFile(final FileSystem fs,
209             final Path p,
210             final Configuration conf,
211             final CacheConfig cacheConf,
212             final BloomType cfBloomType)
213       throws IOException {
214     this.fs = fs;
215     this.path = p;
216     this.cacheConf = cacheConf;
217     if (isReference(p)) {
218       this.reference = Reference.read(fs, p);
219       this.referencePath = getReferredToFile(this.path);
220     }
221 
222     if (BloomFilterFactory.isBloomEnabled(conf)) {
223       this.cfBloomType = cfBloomType;
224     } else {
225       LOG.info("Ignoring bloom filter check for file " + path + ": " +
226           "cfBloomType=" + cfBloomType + " (disabled in config)");
227       this.cfBloomType = BloomType.NONE;
228     }
229 
230     // cache the modification time stamp of this store file
231     FileStatus[] stats = FSUtils.listStatus(fs, p, null);
232     if (stats != null && stats.length == 1) {
233       this.modificationTimeStamp = stats[0].getModificationTime();
234     } else {
235       this.modificationTimeStamp = 0;
236     }
237   }
238 
239   /**
240    * @return Path or null if this StoreFile was made with a Stream.
241    */
242   Path getPath() {
243     return this.path;
244   }
245 
246   /**
247    * @return The Store/ColumnFamily this file belongs to.
248    */
249   byte [] getFamily() {
250     return Bytes.toBytes(this.path.getParent().getName());
251   }
252 
253   /**
254    * @return True if this is a StoreFile Reference; call after {@link #open()}
255    * else may get wrong answer.
256    */
257   boolean isReference() {
258     return this.reference != null;
259   }
260 
261   /**
262    * @param p Path to check.
263    * @return True if the path has format of a HStoreFile reference.
264    */
265   public static boolean isReference(final Path p) {
266     return !p.getName().startsWith("_") &&
267       isReference(p, REF_NAME_PARSER.matcher(p.getName()));
268   }
269 
270   /**
271    * @param p Path to check.
272    * @param m Matcher to use.
273    * @return True if the path has format of a HStoreFile reference.
274    */
275   public static boolean isReference(final Path p, final Matcher m) {
276     if (m == null || !m.matches()) {
277       LOG.warn("Failed match of store file name " + p.toString());
278       throw new RuntimeException("Failed match of store file name " +
279           p.toString());
280     }
281     return m.groupCount() > 1 && m.group(2) != null;
282   }
283 
284   /*
285    * Return path to the file referred to by a Reference.  Presumes a directory
286    * hierarchy of <code>${hbase.rootdir}/tablename/regionname/familyname</code>.
287    * @param p Path to a Reference file.
288    * @return Calculated path to parent region file.
289    * @throws IOException
290    */
291   static Path getReferredToFile(final Path p) {
292     Matcher m = REF_NAME_PARSER.matcher(p.getName());
293     if (m == null || !m.matches()) {
294       LOG.warn("Failed match of store file name " + p.toString());
295       throw new RuntimeException("Failed match of store file name " +
296           p.toString());
297     }
298     // Other region name is suffix on the passed Reference file name
299     String otherRegion = m.group(2);
300     // Tabledir is up two directories from where Reference was written.
301     Path tableDir = p.getParent().getParent().getParent();
302     String nameStrippedOfSuffix = m.group(1);
303     // Build up new path with the referenced region in place of our current
304     // region in the reference path.  Also strip regionname suffix from name.
305     return new Path(new Path(new Path(tableDir, otherRegion),
306       p.getParent().getName()), nameStrippedOfSuffix);
307   }
308 
309   /**
310    * @return True if this file was made by a major compaction.
311    */
312   boolean isMajorCompaction() {
313     if (this.majorCompaction == null) {
314       throw new NullPointerException("This has not been set yet");
315     }
316     return this.majorCompaction.get();
317   }
318 
319   /**
320    * @return This files maximum edit sequence id.
321    */
322   public long getMaxSequenceId() {
323     return this.sequenceid;
324   }
325 
326   public long getModificationTimeStamp() {
327     return modificationTimeStamp;
328   }
329 
330   /**
331    * Return the largest memstoreTS found across all storefiles in
332    * the given list. Store files that were created by a mapreduce
333    * bulk load are ignored, as they do not correspond to any specific
334    * put operation, and thus do not have a memstoreTS associated with them.
335    * @return 0 if no non-bulk-load files are provided or, this is Store that
336    * does not yet have any store files.
337    */
338   public static long getMaxMemstoreTSInList(Collection<StoreFile> sfs) {
339     long max = 0;
340     for (StoreFile sf : sfs) {
341       if (!sf.isBulkLoadResult()) {
342         max = Math.max(max, sf.getMaxMemstoreTS());
343       }
344     }
345     return max;
346   }
347 
348   /**
349    * Return the highest sequence ID found across all storefiles in
350    * the given list. Store files that were created by a mapreduce
351    * bulk load are ignored, as they do not correspond to any edit
352    * log items.
353    * @return 0 if no non-bulk-load files are provided or, this is Store that
354    * does not yet have any store files.
355    */
356   public static long getMaxSequenceIdInList(Collection<StoreFile> sfs) {
357     long max = 0;
358     for (StoreFile sf : sfs) {
359       if (!sf.isBulkLoadResult()) {
360         max = Math.max(max, sf.getMaxSequenceId());
361       }
362     }
363     return max;
364   }
365 
366   /**
367    * @return true if this storefile was created by HFileOutputFormat
368    * for a bulk load.
369    */
370   boolean isBulkLoadResult() {
371     return metadataMap.containsKey(BULKLOAD_TIME_KEY);
372   }
373 
374   /**
375    * Return the timestamp at which this bulk load file was generated.
376    */
377   public long getBulkLoadTimestamp() {
378     return Bytes.toLong(metadataMap.get(BULKLOAD_TIME_KEY));
379   }
380 
381   /**
382    * @return the cached value of HDFS blocks distribution. The cached value is
383    * calculated when store file is opened.
384    */
385   public HDFSBlocksDistribution getHDFSBlockDistribution() {
386     return this.hdfsBlocksDistribution;
387   }
388 
389   /**
390    * helper function to compute HDFS blocks distribution of a given reference
391    * file.For reference file, we don't compute the exact value. We use some
392    * estimate instead given it might be good enough. we assume bottom part
393    * takes the first half of reference file, top part takes the second half
394    * of the reference file. This is just estimate, given
395    * midkey ofregion != midkey of HFile, also the number and size of keys vary.
396    * If this estimate isn't good enough, we can improve it later.
397    * @param fs  The FileSystem
398    * @param reference  The reference
399    * @param reference  The referencePath
400    * @return HDFS blocks distribution
401    */
402   static private HDFSBlocksDistribution computeRefFileHDFSBlockDistribution(
403     FileSystem fs, Reference reference, Path referencePath) throws IOException {
404     if ( referencePath == null) {
405       return null;
406     }
407 
408     FileStatus status = fs.getFileStatus(referencePath);
409     long start = 0;
410     long length = 0;
411 
412     if (Reference.isTopFileRegion(reference.getFileRegion())) {
413       start = status.getLen()/2;
414       length = status.getLen() - status.getLen()/2;
415     } else {
416       start = 0;
417       length = status.getLen()/2;
418     }
419     return FSUtils.computeHDFSBlocksDistribution(fs, status, start, length);
420   }
421 
422   /**
423    * helper function to compute HDFS blocks distribution of a given file.
424    * For reference file, it is an estimate
425    * @param fs  The FileSystem
426    * @param p  The path of the file
427    * @return HDFS blocks distribution
428    */
429   static public HDFSBlocksDistribution computeHDFSBlockDistribution(
430     FileSystem fs, Path p) throws IOException {
431     if (isReference(p)) {
432       Reference reference = Reference.read(fs, p);
433       Path referencePath = getReferredToFile(p);
434       return computeRefFileHDFSBlockDistribution(fs, reference, referencePath);
435     } else {
436       FileStatus status = fs.getFileStatus(p);
437       long length = status.getLen();
438       return FSUtils.computeHDFSBlocksDistribution(fs, status, 0, length);
439     }
440   }
441 
442 
443   /**
444    * compute HDFS block distribution, for reference file, it is an estimate
445    */
446   private void computeHDFSBlockDistribution() throws IOException {
447     if (isReference()) {
448       this.hdfsBlocksDistribution = computeRefFileHDFSBlockDistribution(
449         this.fs, this.reference, this.referencePath);
450     } else {
451       FileStatus status = this.fs.getFileStatus(this.path);
452       long length = status.getLen();
453       this.hdfsBlocksDistribution = FSUtils.computeHDFSBlocksDistribution(
454         this.fs, status, 0, length);
455     }
456   }
457 
458   /**
459    * Opens reader on this store file.  Called by Constructor.
460    * @return Reader for the store file.
461    * @throws IOException
462    * @see #closeReader()
463    */
464   private Reader open() throws IOException {
465     if (this.reader != null) {
466       throw new IllegalAccessError("Already open");
467     }
468     if (isReference()) {
469       this.reader = new HalfStoreFileReader(this.fs, this.referencePath,
470           this.cacheConf, this.reference);
471     } else {
472       this.reader = new Reader(this.fs, this.path, this.cacheConf);
473     }
474 
475     computeHDFSBlockDistribution();
476 
477     // Load up indices and fileinfo. This also loads Bloom filter type.
478     metadataMap = Collections.unmodifiableMap(this.reader.loadFileInfo());
479 
480     // Read in our metadata.
481     byte [] b = metadataMap.get(MAX_SEQ_ID_KEY);
482     if (b != null) {
483       // By convention, if halfhfile, top half has a sequence number > bottom
484       // half. Thats why we add one in below. Its done for case the two halves
485       // are ever merged back together --rare.  Without it, on open of store,
486       // since store files are distingushed by sequence id, the one half would
487       // subsume the other.
488       this.sequenceid = Bytes.toLong(b);
489       if (isReference()) {
490         if (Reference.isTopFileRegion(this.reference.getFileRegion())) {
491           this.sequenceid += 1;
492         }
493       }
494     }
495     this.reader.setSequenceID(this.sequenceid);
496 
497     b = metadataMap.get(HFileWriterV2.MAX_MEMSTORE_TS_KEY);
498     if (b != null) {
499       this.maxMemstoreTS = Bytes.toLong(b);
500     }
501 
502     b = metadataMap.get(MAJOR_COMPACTION_KEY);
503     if (b != null) {
504       boolean mc = Bytes.toBoolean(b);
505       if (this.majorCompaction == null) {
506         this.majorCompaction = new AtomicBoolean(mc);
507       } else {
508         this.majorCompaction.set(mc);
509       }
510     } else {
511       // Presume it is not major compacted if it doesn't explicity say so
512       // HFileOutputFormat explicitly sets the major compacted key.
513       this.majorCompaction = new AtomicBoolean(false);
514     }
515 
516     BloomType hfileBloomType = reader.getBloomFilterType();
517     if (cfBloomType != BloomType.NONE) {
518       reader.loadBloomfilter();
519       if (hfileBloomType != cfBloomType) {
520         LOG.info("HFile Bloom filter type for "
521             + reader.getHFileReader().getName() + ": " + hfileBloomType
522             + ", but " + cfBloomType + " specified in column family "
523             + "configuration");
524       }
525     } else if (hfileBloomType != BloomType.NONE) {
526       LOG.info("Bloom filter turned off by CF config for "
527           + reader.getHFileReader().getName());
528     }
529 
530     try {
531       byte [] timerangeBytes = metadataMap.get(TIMERANGE_KEY);
532       if (timerangeBytes != null) {
533         this.reader.timeRangeTracker = new TimeRangeTracker();
534         Writables.copyWritable(timerangeBytes, this.reader.timeRangeTracker);
535       }
536     } catch (IllegalArgumentException e) {
537       LOG.error("Error reading timestamp range data from meta -- " +
538           "proceeding without", e);
539       this.reader.timeRangeTracker = null;
540     }
541     return this.reader;
542   }
543 
544   /**
545    * @return Reader for StoreFile. creates if necessary
546    * @throws IOException
547    */
548   public Reader createReader() throws IOException {
549     if (this.reader == null) {
550       this.reader = open();
551     }
552     return this.reader;
553   }
554 
555   /**
556    * @return Current reader.  Must call createReader first else returns null.
557    * @see #createReader()
558    */
559   public Reader getReader() {
560     return this.reader;
561   }
562 
563   /**
564    * @param evictOnClose 
565    * @throws IOException
566    */
567   public synchronized void closeReader(boolean evictOnClose)
568   throws IOException {
569     if (this.reader != null) {
570       this.reader.close(evictOnClose);
571       this.reader = null;
572     }
573   }
574 
575   /**
576    * Delete this file
577    * @throws IOException
578    */
579   public void deleteReader() throws IOException {
580     closeReader(true);
581     this.fs.delete(getPath(), true);
582   }
583 
584   @Override
585   public String toString() {
586     return this.path.toString() +
587       (isReference()? "-" + this.referencePath + "-" + reference.toString(): "");
588   }
589 
590   /**
591    * @return a length description of this StoreFile, suitable for debug output
592    */
593   public String toStringDetailed() {
594     StringBuilder sb = new StringBuilder();
595     sb.append(this.path.toString());
596     sb.append(", isReference=").append(isReference());
597     sb.append(", isBulkLoadResult=").append(isBulkLoadResult());
598     if (isBulkLoadResult()) {
599       sb.append(", bulkLoadTS=").append(getBulkLoadTimestamp());
600     } else {
601       sb.append(", seqid=").append(getMaxSequenceId());
602     }
603     sb.append(", majorCompaction=").append(isMajorCompaction());
604 
605     return sb.toString();
606   }
607 
608   /**
609    * Utility to help with rename.
610    * @param fs
611    * @param src
612    * @param tgt
613    * @return True if succeeded.
614    * @throws IOException
615    */
616   public static Path rename(final FileSystem fs,
617                             final Path src,
618                             final Path tgt)
619       throws IOException {
620 
621     if (!fs.exists(src)) {
622       throw new FileNotFoundException(src.toString());
623     }
624     if (!fs.rename(src, tgt)) {
625       throw new IOException("Failed rename of " + src + " to " + tgt);
626     }
627     return tgt;
628   }
629 
630   /**
631    * Get a store file writer. Client is responsible for closing file when done.
632    *
633    * @param fs
634    * @param dir Path to family directory.  Makes the directory if doesn't exist.
635    * Creates a file with a unique name in this directory.
636    * @param blocksize size per filesystem block
637    * @return StoreFile.Writer
638    * @throws IOException
639    */
640   public static Writer createWriter(final FileSystem fs, final Path dir,
641       final int blocksize, Configuration conf, CacheConfig cacheConf)
642   throws IOException {
643     return createWriter(fs, dir, blocksize, null, null, conf, cacheConf,
644         BloomType.NONE, 0);
645   }
646 
647   /**
648    * Create a store file writer. Client is responsible for closing file when done.
649    * If metadata, add BEFORE closing using appendMetadata()
650    * @param fs
651    * @param dir Path to family directory.  Makes the directory if doesn't exist.
652    * Creates a file with a unique name in this directory.
653    * @param blocksize
654    * @param algorithm Pass null to get default.
655    * @param c Pass null to get default.
656    * @param conf HBase system configuration. used with bloom filters
657    * @param cacheConf Cache configuration and reference.
658    * @param bloomType column family setting for bloom filters
659    * @param maxKeyCount estimated maximum number of keys we expect to add
660    * @return HFile.Writer
661    * @throws IOException
662    */
663   public static StoreFile.Writer createWriter(final FileSystem fs,
664                                               final Path dir,
665                                               final int blocksize,
666                                               final Compression.Algorithm algorithm,
667                                               final KeyValue.KVComparator c,
668                                               final Configuration conf,
669                                               final CacheConfig cacheConf,
670                                               BloomType bloomType,
671                                               long maxKeyCount)
672       throws IOException {
673 
674     if (!fs.exists(dir)) {
675       fs.mkdirs(dir);
676     }
677     Path path = getUniqueFile(fs, dir);
678     if (!BloomFilterFactory.isBloomEnabled(conf)) {
679       bloomType = BloomType.NONE;
680     }
681 
682     return new Writer(fs, path, blocksize,
683         algorithm == null? HFile.DEFAULT_COMPRESSION_ALGORITHM: algorithm,
684         conf, cacheConf, c == null ? KeyValue.COMPARATOR: c, bloomType,
685         maxKeyCount);
686   }
687 
688   /**
689    * @param fs
690    * @param dir Directory to create file in.
691    * @return random filename inside passed <code>dir</code>
692    */
693   public static Path getUniqueFile(final FileSystem fs, final Path dir)
694       throws IOException {
695     if (!fs.getFileStatus(dir).isDir()) {
696       throw new IOException("Expecting " + dir.toString() +
697         " to be a directory");
698     }
699     return getRandomFilename(fs, dir);
700   }
701 
702   /**
703    *
704    * @param fs
705    * @param dir
706    * @return Path to a file that doesn't exist at time of this invocation.
707    * @throws IOException
708    */
709   static Path getRandomFilename(final FileSystem fs, final Path dir)
710       throws IOException {
711     return getRandomFilename(fs, dir, null);
712   }
713 
714   /**
715    *
716    * @param fs
717    * @param dir
718    * @param suffix
719    * @return Path to a file that doesn't exist at time of this invocation.
720    * @throws IOException
721    */
722   static Path getRandomFilename(final FileSystem fs,
723                                 final Path dir,
724                                 final String suffix)
725       throws IOException {
726     return new Path(dir, UUID.randomUUID().toString().replaceAll("-", "")
727         + (suffix == null ? "" : suffix));
728   }
729 
730   /**
731    * Write out a split reference.
732    *
733    * Package local so it doesnt leak out of regionserver.
734    *
735    * @param fs
736    * @param splitDir Presumes path format is actually
737    * <code>SOME_DIRECTORY/REGIONNAME/FAMILY</code>.
738    * @param f File to split.
739    * @param splitRow
740    * @param range
741    * @return Path to created reference.
742    * @throws IOException
743    */
744   static Path split(final FileSystem fs,
745                     final Path splitDir,
746                     final StoreFile f,
747                     final byte [] splitRow,
748                     final Reference.Range range)
749       throws IOException {
750     // A reference to the bottom half of the hsf store file.
751     Reference r = new Reference(splitRow, range);
752     // Add the referred-to regions name as a dot separated suffix.
753     // See REF_NAME_PARSER regex above.  The referred-to regions name is
754     // up in the path of the passed in <code>f</code> -- parentdir is family,
755     // then the directory above is the region name.
756     String parentRegionName = f.getPath().getParent().getParent().getName();
757     // Write reference with same file id only with the other region name as
758     // suffix and into the new region location (under same family).
759     Path p = new Path(splitDir, f.getPath().getName() + "." + parentRegionName);
760     return r.write(fs, p);
761   }
762 
763 
764   /**
765    * A StoreFile writer.  Use this to read/write HBase Store Files. It is package
766    * local because it is an implementation detail of the HBase regionserver.
767    */
768   public static class Writer {
769     private final BloomFilterWriter bloomFilterWriter;
770     private final BloomType bloomType;
771     private byte[] lastBloomKey;
772     private int lastBloomKeyOffset, lastBloomKeyLen;
773     private KVComparator kvComparator;
774     private KeyValue lastKv = null;
775 
776     TimeRangeTracker timeRangeTracker = new TimeRangeTracker();
777     /* isTimeRangeTrackerSet keeps track if the timeRange has already been set
778      * When flushing a memstore, we set TimeRange and use this variable to
779      * indicate that it doesn't need to be calculated again while
780      * appending KeyValues.
781      * It is not set in cases of compactions when it is recalculated using only
782      * the appended KeyValues*/
783     boolean isTimeRangeTrackerSet = false;
784 
785     protected HFile.Writer writer;
786     /**
787      * Creates an HFile.Writer that also write helpful meta data.
788      * @param fs file system to write to
789      * @param path file name to create
790      * @param blocksize HDFS block size
791      * @param compress HDFS block compression
792      * @param conf user configuration
793      * @param comparator key comparator
794      * @param bloomType bloom filter setting
795      * @param maxKeys the expected maximum number of keys to be added. Was used
796      *        for Bloom filter size in {@link HFile} format version 1.
797      * @throws IOException problem writing to FS
798      */
799     public Writer(FileSystem fs, Path path, int blocksize,
800         Compression.Algorithm compress, final Configuration conf,
801         CacheConfig cacheConf,
802         final KVComparator comparator, BloomType bloomType, long maxKeys)
803         throws IOException {
804       writer = HFile.getWriterFactory(conf, cacheConf).createWriter(
805           fs, path, blocksize,
806           compress, comparator.getRawComparator());
807 
808       this.kvComparator = comparator;
809 
810       bloomFilterWriter = BloomFilterFactory.createBloomAtWrite(conf, cacheConf,
811           bloomType, (int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
812       if (bloomFilterWriter != null) {
813         this.bloomType = bloomType;
814         LOG.info("Bloom filter type for " + path + ": " + this.bloomType +
815             ", "+ bloomFilterWriter.getClass().getSimpleName());
816       } else {
817         // Not using Bloom filters.
818         this.bloomType = BloomType.NONE;
819       }
820     }
821 
822     /**
823      * Writes meta data.
824      * Call before {@link #close()} since its written as meta data to this file.
825      * @param maxSequenceId Maximum sequence id.
826      * @param majorCompaction True if this file is product of a major compaction
827      * @throws IOException problem writing to FS
828      */
829     public void appendMetadata(final long maxSequenceId, final boolean majorCompaction)
830     throws IOException {
831       writer.appendFileInfo(MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId));
832       writer.appendFileInfo(MAJOR_COMPACTION_KEY,
833           Bytes.toBytes(majorCompaction));
834       appendTimeRangeMetadata();
835     }
836 
837     /**
838      * Add TimestampRange to Metadata
839      */
840     public void appendTimeRangeMetadata() throws IOException {
841       appendFileInfo(TIMERANGE_KEY,WritableUtils.toByteArray(timeRangeTracker));
842     }
843 
844     /**
845      * Set TimeRangeTracker
846      * @param trt
847      */
848     public void setTimeRangeTracker(final TimeRangeTracker trt) {
849       this.timeRangeTracker = trt;
850       isTimeRangeTrackerSet = true;
851     }
852 
853     /**
854      * If the timeRangeTracker is not set,
855      * update TimeRangeTracker to include the timestamp of this key
856      * @param kv
857      */
858     public void includeInTimeRangeTracker(final KeyValue kv) {
859       if (!isTimeRangeTrackerSet) {
860         timeRangeTracker.includeTimestamp(kv);
861       }
862     }
863 
864     /**
865      * If the timeRangeTracker is not set,
866      * update TimeRangeTracker to include the timestamp of this key
867      * @param key
868      * @throws IOException
869      */
870     public void includeInTimeRangeTracker(final byte [] key) {
871       if (!isTimeRangeTrackerSet) {
872         timeRangeTracker.includeTimestamp(key);
873       }
874     }
875 
876     public void append(final KeyValue kv) throws IOException {
877       if (this.bloomFilterWriter != null) {
878         // only add to the bloom filter on a new, unique key
879         boolean newKey = true;
880         if (this.lastKv != null) {
881           switch(bloomType) {
882           case ROW:
883             newKey = ! kvComparator.matchingRows(kv, lastKv);
884             break;
885           case ROWCOL:
886             newKey = ! kvComparator.matchingRowColumn(kv, lastKv);
887             break;
888           case NONE:
889             newKey = false;
890             break;
891           default:
892             throw new IOException("Invalid Bloom filter type: " + bloomType);
893           }
894         }
895         if (newKey) {
896           /*
897            * http://2.bp.blogspot.com/_Cib_A77V54U/StZMrzaKufI/AAAAAAAAADo/ZhK7bGoJdMQ/s400/KeyValue.png
898            * Key = RowLen + Row + FamilyLen + Column [Family + Qualifier] + TimeStamp
899            *
900            * 2 Types of Filtering:
901            *  1. Row = Row
902            *  2. RowCol = Row + Qualifier
903            */
904           byte[] bloomKey;
905           int bloomKeyOffset, bloomKeyLen;
906 
907           switch (bloomType) {
908           case ROW:
909             bloomKey = kv.getBuffer();
910             bloomKeyOffset = kv.getRowOffset();
911             bloomKeyLen = kv.getRowLength();
912             break;
913           case ROWCOL:
914             // merge(row, qualifier)
915             // TODO: could save one buffer copy in case of compound Bloom
916             // filters when this involves creating a KeyValue
917             bloomKey = bloomFilterWriter.createBloomKey(kv.getBuffer(),
918                 kv.getRowOffset(), kv.getRowLength(), kv.getBuffer(),
919                 kv.getQualifierOffset(), kv.getQualifierLength());
920             bloomKeyOffset = 0;
921             bloomKeyLen = bloomKey.length;
922             break;
923           default:
924             throw new IOException("Invalid Bloom filter type: " + bloomType +
925                 " (ROW or ROWCOL expected)");
926           }
927           bloomFilterWriter.add(bloomKey, bloomKeyOffset, bloomKeyLen);
928           if (lastBloomKey != null
929               && bloomFilterWriter.getComparator().compare(bloomKey,
930                   bloomKeyOffset, bloomKeyLen, lastBloomKey,
931                   lastBloomKeyOffset, lastBloomKeyLen) <= 0) {
932             throw new IOException("Non-increasing Bloom keys: "
933                 + Bytes.toStringBinary(bloomKey, bloomKeyOffset, bloomKeyLen)
934                 + " after "
935                 + Bytes.toStringBinary(lastBloomKey, lastBloomKeyOffset,
936                     lastBloomKeyLen));
937           }
938           lastBloomKey = bloomKey;
939           lastBloomKeyOffset = bloomKeyOffset;
940           lastBloomKeyLen = bloomKeyLen;
941           this.lastKv = kv;
942         }
943       }
944       writer.append(kv);
945       includeInTimeRangeTracker(kv);
946     }
947 
948     public Path getPath() {
949       return this.writer.getPath();
950     }
951 
952     boolean hasBloom() {
953       return this.bloomFilterWriter != null;
954     }
955 
956     /**
957      * For unit testing only.
958      * @return the Bloom filter used by this writer.
959      */
960     BloomFilterWriter getBloomWriter() {
961       return bloomFilterWriter;
962     }
963 
964     public void close() throws IOException {
965       // Make sure we wrote something to the Bloom filter before adding it.
966       boolean haveBloom = bloomFilterWriter != null &&
967           bloomFilterWriter.getKeyCount() > 0;
968       if (haveBloom) {
969         bloomFilterWriter.compactBloom();
970         writer.addBloomFilter(bloomFilterWriter);
971         writer.appendFileInfo(BLOOM_FILTER_TYPE_KEY,
972             Bytes.toBytes(bloomType.toString()));
973         if (lastBloomKey != null) {
974           writer.appendFileInfo(LAST_BLOOM_KEY, Arrays.copyOfRange(
975               lastBloomKey, lastBloomKeyOffset, lastBloomKeyOffset
976                   + lastBloomKeyLen));
977         }
978       }
979       writer.close();
980 
981       // Log final Bloom filter statistics. This needs to be done after close()
982       // because compound Bloom filters might be finalized as part of closing.
983       if (haveBloom && bloomFilterWriter.getMaxKeys() > 0) {
984         StoreFile.LOG.info("Bloom added to HFile ("
985             + getPath() + "): " +
986             bloomFilterWriter.toString().replace("\n", "; "));
987       }
988     }
989 
990     public void appendFileInfo(byte[] key, byte[] value) throws IOException {
991       writer.appendFileInfo(key, value);
992     }
993   }
994 
995   /**
996    * Reader for a StoreFile.
997    */
998   public static class Reader {
999     static final Log LOG = LogFactory.getLog(Reader.class.getName());
1000 
1001     protected BloomFilter bloomFilter = null;
1002     protected BloomType bloomFilterType;
1003     private final HFile.Reader reader;
1004     protected TimeRangeTracker timeRangeTracker = null;
1005     protected long sequenceID = -1;
1006     private byte[] lastBloomKey;
1007 
1008     public Reader(FileSystem fs, Path path, CacheConfig cacheConf)
1009         throws IOException {
1010       reader = HFile.createReader(fs, path, cacheConf);
1011       bloomFilterType = BloomType.NONE;
1012     }
1013 
1014     /**
1015      * ONLY USE DEFAULT CONSTRUCTOR FOR UNIT TESTS
1016      */
1017     Reader() {
1018       this.reader = null;
1019     }
1020 
1021     public RawComparator<byte []> getComparator() {
1022       return reader.getComparator();
1023     }
1024 
1025     /**
1026      * Get a scanner to scan over this StoreFile.
1027      *
1028      * @param cacheBlocks should this scanner cache blocks?
1029      * @param pread use pread (for highly concurrent small readers)
1030      * @return a scanner
1031      */
1032     public StoreFileScanner getStoreFileScanner(boolean cacheBlocks, boolean pread) {
1033       return getStoreFileScanner(cacheBlocks, pread, false);
1034     }
1035         /**
1036      * Get a scanner to scan over this StoreFile.
1037      *
1038      * @param cacheBlocks should this scanner cache blocks?
1039      * @param pread use pread (for highly concurrent small readers)
1040      * @param isCompaction is scanner being used for compaction?
1041      * @return a scanner
1042      */
1043     public StoreFileScanner getStoreFileScanner(boolean cacheBlocks,
1044         boolean pread, boolean isCompaction) {
1045       return new StoreFileScanner(this, getScanner(cacheBlocks, pread,
1046           isCompaction), !isCompaction);
1047     }
1048 
1049     /**
1050      * Warning: Do not write further code which depends on this call. Instead
1051      * use getStoreFileScanner() which uses the StoreFileScanner class/interface
1052      * which is the preferred way to scan a store with higher level concepts.
1053      *
1054      * @param cacheBlocks should we cache the blocks?
1055      * @param pread use pread (for concurrent small readers)
1056      * @return the underlying HFileScanner
1057      */
1058     @Deprecated
1059     public HFileScanner getScanner(boolean cacheBlocks, boolean pread) {
1060       return getScanner(cacheBlocks, pread, false);
1061     }
1062 
1063     /**
1064      * Warning: Do not write further code which depends on this call. Instead
1065      * use getStoreFileScanner() which uses the StoreFileScanner class/interface
1066      * which is the preferred way to scan a store with higher level concepts.
1067      * 
1068      * @param cacheBlocks
1069      *          should we cache the blocks?
1070      * @param pread
1071      *          use pread (for concurrent small readers)
1072      * @param isCompaction
1073      *          is scanner being used for compaction?
1074      * @return the underlying HFileScanner
1075      */
1076     @Deprecated
1077     public HFileScanner getScanner(boolean cacheBlocks, boolean pread,
1078         boolean isCompaction) {
1079       return reader.getScanner(cacheBlocks, pread, isCompaction);
1080     }
1081 
1082     public void close(boolean evictOnClose) throws IOException {
1083       reader.close(evictOnClose);
1084     }
1085 
1086     public boolean shouldSeek(Scan scan, final SortedSet<byte[]> columns) {
1087       return (passesTimerangeFilter(scan) && passesBloomFilter(scan, columns));
1088     }
1089 
1090     /**
1091      * Check if this storeFile may contain keys within the TimeRange
1092      * @param scan
1093      * @return False if it definitely does not exist in this StoreFile
1094      */
1095     private boolean passesTimerangeFilter(Scan scan) {
1096       if (timeRangeTracker == null) {
1097         return true;
1098       } else {
1099         return timeRangeTracker.includesTimeRange(scan.getTimeRange());
1100       }
1101     }
1102 
1103     /**
1104      * Checks whether the given scan passes the Bloom filter (if present). Only
1105      * checks Bloom filters for single-row or single-row-column scans. Bloom
1106      * filter checking for multi-gets is implemented as part of the store
1107      * scanner system (see {@link StoreFileScanner#seekExactly}) and uses
1108      * the lower-level API {@link #passesBloomFilter(byte[], int, int, byte[],
1109      * int, int)}.
1110      *
1111      * @param scan the scan specification. Used to determine the row, and to
1112      *          check whether this is a single-row ("get") scan.
1113      * @param columns the set of columns. Only used for row-column Bloom
1114      *          filters.
1115      * @return true if the scan with the given column set passes the Bloom
1116      *         filter, or if the Bloom filter is not applicable for the scan.
1117      *         False if the Bloom filter is applicable and the scan fails it.
1118      */
1119     private boolean passesBloomFilter(Scan scan,
1120         final SortedSet<byte[]> columns) {
1121       // Multi-column non-get scans will use Bloom filters through the
1122       // lower-level API function that this function calls.
1123       if (!scan.isGetScan()) {
1124         return true;
1125       }
1126 
1127       byte[] row = scan.getStartRow();
1128       switch (this.bloomFilterType) {
1129         case ROW:
1130           return passesBloomFilter(row, 0, row.length, null, 0, 0);
1131 
1132         case ROWCOL:
1133           if (columns != null && columns.size() == 1) {
1134             byte[] column = columns.first();
1135             return passesBloomFilter(row, 0, row.length, column, 0, 
1136                 column.length);
1137           }
1138 
1139           // For multi-column queries the Bloom filter is checked from the
1140           // seekExact operation.
1141           return true;
1142 
1143         default:
1144           return true;
1145       }      
1146     }
1147 
1148     /**
1149      * A method for checking Bloom filters. Called directly from
1150      * StoreFileScanner in case of a multi-column query.
1151      *
1152      * @param row
1153      * @param rowOffset
1154      * @param rowLen
1155      * @param col
1156      * @param colOffset
1157      * @param colLen
1158      * @return True if passes
1159      */
1160     public boolean passesBloomFilter(byte[] row, int rowOffset, int rowLen,
1161         byte[] col, int colOffset, int colLen) {
1162       if (bloomFilter == null)
1163         return true;
1164 
1165       byte[] key;
1166       switch (bloomFilterType) { 
1167         case ROW:
1168           if (col != null) {
1169             throw new RuntimeException("Row-only Bloom filter called with " +
1170                 "column specified");
1171           }
1172           if (rowOffset != 0 || rowLen != row.length) {
1173               throw new AssertionError("For row-only Bloom filters the row "
1174                   + "must occupy the whole array");
1175           }
1176           key = row;
1177           break;
1178 
1179         case ROWCOL:
1180           key = bloomFilter.createBloomKey(row, rowOffset, rowLen, col,
1181               colOffset, colLen);
1182           break;
1183 
1184         default:
1185           return true;
1186       }
1187 
1188       // Cache Bloom filter as a local variable in case it is set to null by
1189       // another thread on an IO error.
1190       BloomFilter bloomFilter = this.bloomFilter;
1191 
1192       if (bloomFilter == null) {
1193         return true;
1194       }
1195 
1196       // Empty file?
1197       if (reader.getTrailer().getEntryCount() == 0)
1198         return false;
1199 
1200       try {
1201         boolean shouldCheckBloom;
1202         ByteBuffer bloom;
1203         if (bloomFilter.supportsAutoLoading()) {
1204           bloom = null;
1205           shouldCheckBloom = true;
1206         } else {
1207           bloom = reader.getMetaBlock(HFileWriterV1.BLOOM_FILTER_DATA_KEY,
1208               true);
1209           shouldCheckBloom = bloom != null;
1210         }
1211 
1212         if (shouldCheckBloom) {
1213           boolean exists;
1214 
1215           // Whether the primary Bloom key is greater than the last Bloom key
1216           // from the file info. For row-column Bloom filters this is not yet
1217           // a sufficient condition to return false.
1218           boolean keyIsAfterLast = lastBloomKey != null
1219               && bloomFilter.getComparator().compare(key, lastBloomKey) > 0;
1220 
1221           if (bloomFilterType == BloomType.ROWCOL) {
1222             // Since a Row Delete is essentially a DeleteFamily applied to all
1223             // columns, a file might be skipped if using row+col Bloom filter.
1224             // In order to ensure this file is included an additional check is
1225             // required looking only for a row bloom.
1226             byte[] rowBloomKey = bloomFilter.createBloomKey(row, 0, row.length,
1227                 null, 0, 0);
1228 
1229             if (keyIsAfterLast
1230                 && bloomFilter.getComparator().compare(rowBloomKey,
1231                     lastBloomKey) > 0) {
1232               exists = false;
1233             } else {
1234               exists =
1235                   this.bloomFilter.contains(key, 0, key.length, bloom) ||
1236                   this.bloomFilter.contains(rowBloomKey, 0, rowBloomKey.length,
1237                       bloom);
1238             }
1239           } else {
1240             exists = !keyIsAfterLast
1241                 && this.bloomFilter.contains(key, 0, key.length, bloom);
1242           }
1243 
1244           return exists;
1245         }
1246       } catch (IOException e) {
1247         LOG.error("Error reading bloom filter data -- proceeding without",
1248             e);
1249         setBloomFilterFaulty();
1250       } catch (IllegalArgumentException e) {
1251         LOG.error("Bad bloom filter data -- proceeding without", e);
1252         setBloomFilterFaulty();
1253       }
1254 
1255       return true;
1256     }
1257 
1258     public Map<byte[], byte[]> loadFileInfo() throws IOException {
1259       Map<byte [], byte []> fi = reader.loadFileInfo();
1260 
1261       byte[] b = fi.get(BLOOM_FILTER_TYPE_KEY);
1262       if (b != null) {
1263         bloomFilterType = BloomType.valueOf(Bytes.toString(b));
1264       }
1265 
1266       lastBloomKey = fi.get(LAST_BLOOM_KEY);
1267 
1268       return fi;
1269     }
1270 
1271     public void loadBloomfilter() {
1272       if (this.bloomFilter != null) {
1273         return; // already loaded
1274       }
1275 
1276       try {
1277         DataInput bloomMeta = reader.getBloomFilterMetadata();
1278         if (bloomMeta != null) {
1279           if (bloomFilterType == BloomType.NONE) {
1280             throw new IOException(
1281                 "valid bloom filter type not found in FileInfo");
1282           }
1283 
1284           bloomFilter = BloomFilterFactory.createFromMeta(bloomMeta, reader);
1285           LOG.info("Loaded " + bloomFilterType + " " +
1286               bloomFilter.getClass().getSimpleName() + " metadata for " +
1287               reader.getName());
1288         }
1289       } catch (IOException e) {
1290         LOG.error("Error reading bloom filter meta -- proceeding without", e);
1291         this.bloomFilter = null;
1292       } catch (IllegalArgumentException e) {
1293         LOG.error("Bad bloom filter meta -- proceeding without", e);
1294         this.bloomFilter = null;
1295       }
1296     }
1297 
1298     /**
1299      * The number of Bloom filter entries in this store file, or an estimate
1300      * thereof, if the Bloom filter is not loaded. This always returns an upper
1301      * bound of the number of Bloom filter entries.
1302      *
1303      * @return an estimate of the number of Bloom filter entries in this file
1304      */
1305     public long getFilterEntries() {
1306       return bloomFilter != null ? bloomFilter.getKeyCount()
1307           : reader.getEntries();
1308     }
1309 
1310     public void setBloomFilterFaulty() {
1311       bloomFilter = null;
1312     }
1313 
1314     public byte[] getLastKey() {
1315       return reader.getLastKey();
1316     }
1317 
1318     public byte[] midkey() throws IOException {
1319       return reader.midkey();
1320     }
1321 
1322     public long length() {
1323       return reader.length();
1324     }
1325 
1326     public long getTotalUncompressedBytes() {
1327       return reader.getTrailer().getTotalUncompressedBytes();
1328     }
1329 
1330     public long getEntries() {
1331       return reader.getEntries();
1332     }
1333 
1334     public byte[] getFirstKey() {
1335       return reader.getFirstKey();
1336     }
1337 
1338     public long indexSize() {
1339       return reader.indexSize();
1340     }
1341 
1342     public BloomType getBloomFilterType() {
1343       return this.bloomFilterType;
1344     }
1345 
1346     public long getSequenceID() {
1347       return sequenceID;
1348     }
1349 
1350     public void setSequenceID(long sequenceID) {
1351       this.sequenceID = sequenceID;
1352     }
1353 
1354     BloomFilter getBloomFilter() {
1355       return bloomFilter;
1356     }
1357 
1358     long getUncompressedDataIndexSize() {
1359       return reader.getTrailer().getUncompressedDataIndexSize();
1360     }
1361 
1362     public long getTotalBloomSize() {
1363       if (bloomFilter == null)
1364         return 0;
1365       return bloomFilter.getByteSize();
1366     }
1367 
1368     public int getHFileVersion() {
1369       return reader.getTrailer().getVersion();
1370     }
1371 
1372     HFile.Reader getHFileReader() {
1373       return reader;
1374     }
1375 
1376     void disableBloomFilterForTesting() {
1377       bloomFilter = null;
1378     }
1379   }
1380 
1381   /**
1382    * Useful comparators for comparing StoreFiles.
1383    */
1384   abstract static class Comparators {
1385     /**
1386      * Comparator that compares based on the flush time of
1387      * the StoreFiles. All bulk loads are placed before all non-
1388      * bulk loads, and then all files are sorted by sequence ID.
1389      * If there are ties, the path name is used as a tie-breaker.
1390      */
1391     static final Comparator<StoreFile> FLUSH_TIME =
1392       Ordering.compound(ImmutableList.of(
1393           Ordering.natural().onResultOf(new GetBulkTime()),
1394           Ordering.natural().onResultOf(new GetSeqId()),
1395           Ordering.natural().onResultOf(new GetPathName())
1396       ));
1397 
1398     private static class GetBulkTime implements Function<StoreFile, Long> {
1399       @Override
1400       public Long apply(StoreFile sf) {
1401         if (!sf.isBulkLoadResult()) return Long.MAX_VALUE;
1402         return sf.getBulkLoadTimestamp();
1403       }
1404     }
1405     private static class GetSeqId implements Function<StoreFile, Long> {
1406       @Override
1407       public Long apply(StoreFile sf) {
1408         if (sf.isBulkLoadResult()) return -1L;
1409         return sf.getMaxSequenceId();
1410       }
1411     }
1412     private static class GetPathName implements Function<StoreFile, String> {
1413       @Override
1414       public String apply(StoreFile sf) {
1415         return sf.getPath().getName();
1416       }
1417     }
1418 
1419     /**
1420      * FILE_SIZE = descending sort StoreFiles (largest --> smallest in size)
1421      */
1422     static final Comparator<StoreFile> FILE_SIZE =
1423       Ordering.natural().reverse().onResultOf(new Function<StoreFile, Long>() {
1424         @Override
1425         public Long apply(StoreFile sf) {
1426           return sf.getReader().length();
1427         }
1428       });
1429   }
1430 }