View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.regionserver;
20  
21  import com.google.common.annotations.VisibleForTesting;
22  import com.google.common.base.Function;
23  import com.google.common.collect.ImmutableList;
24  import com.google.common.collect.Ordering;
25  
26  import java.io.IOException;
27  import java.util.Collection;
28  import java.util.Collections;
29  import java.util.Comparator;
30  import java.util.Map;
31  import java.util.UUID;
32  import java.util.concurrent.atomic.AtomicBoolean;
33  
34  import org.apache.commons.logging.Log;
35  import org.apache.commons.logging.LogFactory;
36  import org.apache.hadoop.conf.Configuration;
37  import org.apache.hadoop.fs.FileSystem;
38  import org.apache.hadoop.fs.Path;
39  import org.apache.hadoop.hbase.Cell;
40  import org.apache.hadoop.hbase.CellComparator;
41  import org.apache.hadoop.hbase.CellUtil;
42  import org.apache.hadoop.hbase.HDFSBlocksDistribution;
43  import org.apache.hadoop.hbase.classification.InterfaceAudience;
44  import org.apache.hadoop.hbase.io.hfile.BlockType;
45  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
46  import org.apache.hadoop.hbase.io.hfile.HFile;
47  import org.apache.hadoop.hbase.util.BloomFilterFactory;
48  import org.apache.hadoop.hbase.util.Bytes;
49  
50  /**
51   * A Store data file.  Stores usually have one or more of these files.  They
52   * are produced by flushing the memstore to disk.  To
53   * create, instantiate a writer using {@link StoreFileWriter.Builder}
54   * and append data. Be sure to add any metadata before calling close on the
55   * Writer (Use the appendMetadata convenience methods). On close, a StoreFile
56   * is sitting in the Filesystem.  To refer to it, create a StoreFile instance
57   * passing filesystem and path.  To read, call {@link #createReader()}.
58   * <p>StoreFiles may also reference store files in another Store.
59   *
60   * The reason for this weird pattern where you use a different instance for the
61   * writer and a reader is that we write once but read a lot more.
62   */
63  @InterfaceAudience.LimitedPrivate("Coprocessor")
64  public class StoreFile {
65    private static final Log LOG = LogFactory.getLog(StoreFile.class.getName());
66  
67    // Keys for fileinfo values in HFile
68  
69    /** Max Sequence ID in FileInfo */
70    public static final byte [] MAX_SEQ_ID_KEY = Bytes.toBytes("MAX_SEQ_ID_KEY");
71  
72    /** Major compaction flag in FileInfo */
73    public static final byte[] MAJOR_COMPACTION_KEY =
74        Bytes.toBytes("MAJOR_COMPACTION_KEY");
75  
76    /** Minor compaction flag in FileInfo */
77    public static final byte[] EXCLUDE_FROM_MINOR_COMPACTION_KEY =
78        Bytes.toBytes("EXCLUDE_FROM_MINOR_COMPACTION");
79  
80    /** Bloom filter Type in FileInfo */
81    public static final byte[] BLOOM_FILTER_TYPE_KEY =
82        Bytes.toBytes("BLOOM_FILTER_TYPE");
83  
84    /** Delete Family Count in FileInfo */
85    public static final byte[] DELETE_FAMILY_COUNT =
86        Bytes.toBytes("DELETE_FAMILY_COUNT");
87  
88    /** Last Bloom filter key in FileInfo */
89    public static final byte[] LAST_BLOOM_KEY = Bytes.toBytes("LAST_BLOOM_KEY");
90  
91    /** Key for Timerange information in metadata*/
92    public static final byte[] TIMERANGE_KEY = Bytes.toBytes("TIMERANGE");
93  
94    /** Key for timestamp of earliest-put in metadata*/
95    public static final byte[] EARLIEST_PUT_TS = Bytes.toBytes("EARLIEST_PUT_TS");
96  
97    /** Key for the number of mob cells in metadata*/
98    public static final byte[] MOB_CELLS_COUNT = Bytes.toBytes("MOB_CELLS_COUNT");
99  
100   private final StoreFileInfo fileInfo;
101   private final FileSystem fs;
102 
103   // Block cache configuration and reference.
104   private final CacheConfig cacheConf;
105 
106   // Keys for metadata stored in backing HFile.
107   // Set when we obtain a Reader.
108   private long sequenceid = -1;
109 
110   // max of the MemstoreTS in the KV's in this store
111   // Set when we obtain a Reader.
112   private long maxMemstoreTS = -1;
113 
114   // firstKey, lastkey and cellComparator will be set when openReader.
115   private Cell firstKey;
116 
117   private Cell lastKey;
118 
119   private Comparator comparator;
120 
121   CacheConfig getCacheConf() {
122     return cacheConf;
123   }
124 
125   public Cell getFirstKey() {
126     return firstKey;
127   }
128 
129   public Cell getLastKey() {
130     return lastKey;
131   }
132 
133   public Comparator getComparator() {
134     return comparator;
135   }
136 
137   public long getMaxMemstoreTS() {
138     return maxMemstoreTS;
139   }
140 
141   public void setMaxMemstoreTS(long maxMemstoreTS) {
142     this.maxMemstoreTS = maxMemstoreTS;
143   }
144 
145   // If true, this file was product of a major compaction.  Its then set
146   // whenever you get a Reader.
147   private AtomicBoolean majorCompaction = null;
148 
149   // If true, this file should not be included in minor compactions.
150   // It's set whenever you get a Reader.
151   private boolean excludeFromMinorCompaction = false;
152 
153   /** Meta key set when store file is a result of a bulk load */
154   public static final byte[] BULKLOAD_TASK_KEY =
155     Bytes.toBytes("BULKLOAD_SOURCE_TASK");
156   public static final byte[] BULKLOAD_TIME_KEY =
157     Bytes.toBytes("BULKLOAD_TIMESTAMP");
158 
159   /**
160    * Map of the metadata entries in the corresponding HFile. Populated when Reader is opened
161    * after which it is not modified again.
162    */
163   private Map<byte[], byte[]> metadataMap;
164
165   // StoreFile.Reader
166   private volatile StoreFileReader reader;
167
168   /**
169    * Bloom filter type specified in column family configuration. Does not
170    * necessarily correspond to the Bloom filter type present in the HFile.
171    */
172   private final BloomType cfBloomType;
173
174   /**
175    * Key for skipping resetting sequence id in metadata.
176    * For bulk loaded hfiles, the scanner resets the cell seqId with the latest one,
177    * if this metadata is set as true, the reset is skipped.
178    */
179   public static final byte[] SKIP_RESET_SEQ_ID = Bytes.toBytes("SKIP_RESET_SEQ_ID");
180
181   /**
182    * Constructor, loads a reader and it's indices, etc. May allocate a
183    * substantial amount of ram depending on the underlying files (10-20MB?).
184    *
185    * @param fs  The current file system to use.
186    * @param p  The path of the file.
187    * @param conf  The current configuration.
188    * @param cacheConf  The cache configuration and block cache reference.
189    * @param cfBloomType The bloom type to use for this store file as specified
190    *          by column family configuration. This may or may not be the same
191    *          as the Bloom filter type actually present in the HFile, because
192    *          column family configuration might change. If this is
193    *          {@link BloomType#NONE}, the existing Bloom filter is ignored.
194    * @throws IOException When opening the reader fails.
195    */
196   public StoreFile(final FileSystem fs, final Path p, final Configuration conf,
197         final CacheConfig cacheConf, final BloomType cfBloomType) throws IOException {
198     this(fs, new StoreFileInfo(conf, fs, p), conf, cacheConf, cfBloomType);
199   }
200 
201   /**
202    * Constructor, loads a reader and it's indices, etc. May allocate a
203    * substantial amount of ram depending on the underlying files (10-20MB?).
204    *
205    * @param fs  The current file system to use.
206    * @param fileInfo  The store file information.
207    * @param conf  The current configuration.
208    * @param cacheConf  The cache configuration and block cache reference.
209    * @param cfBloomType The bloom type to use for this store file as specified
210    *          by column family configuration. This may or may not be the same
211    *          as the Bloom filter type actually present in the HFile, because
212    *          column family configuration might change. If this is
213    *          {@link BloomType#NONE}, the existing Bloom filter is ignored.
214    * @throws IOException When opening the reader fails.
215    */
216   public StoreFile(final FileSystem fs, final StoreFileInfo fileInfo, final Configuration conf,
217       final CacheConfig cacheConf,  final BloomType cfBloomType) throws IOException {
218     this.fs = fs;
219     this.fileInfo = fileInfo;
220     this.cacheConf = cacheConf;
221
222     if (BloomFilterFactory.isGeneralBloomEnabled(conf)) {
223       this.cfBloomType = cfBloomType;
224     } else {
225       LOG.info("Ignoring bloom filter check for file " + this.getPath() + ": " +
226           "cfBloomType=" + cfBloomType + " (disabled in config)");
227       this.cfBloomType = BloomType.NONE;
228     }
229   }
230
231   /**
232    * Clone
233    * @param other The StoreFile to clone from
234    */
235   public StoreFile(final StoreFile other) {
236     this.fs = other.fs;
237     this.fileInfo = other.fileInfo;
238     this.cacheConf = other.cacheConf;
239     this.cfBloomType = other.cfBloomType;
240     this.metadataMap = other.metadataMap;
241   }
242
243   /**
244    * Clone a StoreFile for opening private reader.
245    */
246   public StoreFile cloneForReader() {
247     return new StoreFile(this);
248   }
249
250   /**
251    * @return the StoreFile object associated to this StoreFile.
252    *         null if the StoreFile is not a reference.
253    */
254   public StoreFileInfo getFileInfo() {
255     return this.fileInfo;
256   }
257
258   /**
259    * @return Path or null if this StoreFile was made with a Stream.
260    */
261   public Path getPath() {
262     return this.fileInfo.getPath();
263   }
264
265   /**
266    * @return Returns the qualified path of this StoreFile
267    */
268   public Path getQualifiedPath() {
269     return this.fileInfo.getPath().makeQualified(fs);
270   }
271
272   /**
273    * @return True if this is a StoreFile Reference; call
274    * after {@link #open(boolean canUseDropBehind)} else may get wrong answer.
275    */
276   public boolean isReference() {
277     return this.fileInfo.isReference();
278   }
279
280   /**
281    * @return True if this is HFile.
282    */
283   public boolean isHFile() {
284     return StoreFileInfo.isHFile(this.fileInfo.getPath());
285   }
286
287   /**
288    * @return True if this file was made by a major compaction.
289    */
290   public boolean isMajorCompaction() {
291     if (this.majorCompaction == null) {
292       throw new NullPointerException("This has not been set yet");
293     }
294     return this.majorCompaction.get();
295   }
296
297   /**
298    * @return True if this file should not be part of a minor compaction.
299    */
300   public boolean excludeFromMinorCompaction() {
301     return this.excludeFromMinorCompaction;
302   }
303
304   /**
305    * @return This files maximum edit sequence id.
306    */
307   public long getMaxSequenceId() {
308     return this.sequenceid;
309   }
310
311   public long getModificationTimeStamp() throws IOException {
312     return (fileInfo == null) ? 0 : fileInfo.getModificationTime();
313   }
314
315   /**
316    * Only used by the Striped Compaction Policy
317    * @param key
318    * @return value associated with the metadata key
319    */
320   public byte[] getMetadataValue(byte[] key) {
321     return metadataMap.get(key);
322   }
323
324   /**
325    * Return the largest memstoreTS found across all storefiles in
326    * the given list. Store files that were created by a mapreduce
327    * bulk load are ignored, as they do not correspond to any specific
328    * put operation, and thus do not have a memstoreTS associated with them.
329    * @return 0 if no non-bulk-load files are provided or, this is Store that
330    * does not yet have any store files.
331    */
332   public static long getMaxMemstoreTSInList(Collection<StoreFile> sfs) {
333     long max = 0;
334     for (StoreFile sf : sfs) {
335       if (!sf.isBulkLoadResult()) {
336         max = Math.max(max, sf.getMaxMemstoreTS());
337       }
338     }
339     return max;
340   }
341
342   /**
343    * Return the highest sequence ID found across all storefiles in
344    * the given list.
345    * @param sfs
346    * @return 0 if no non-bulk-load files are provided or, this is Store that
347    * does not yet have any store files.
348    */
349   public static long getMaxSequenceIdInList(Collection<StoreFile> sfs) {
350     long max = 0;
351     for (StoreFile sf : sfs) {
352       max = Math.max(max, sf.getMaxSequenceId());
353     }
354     return max;
355   }
356
357   /**
358    * Check if this storefile was created by bulk load.
359    * When a hfile is bulk loaded into HBase, we append
360    * {@code '_SeqId_<id-when-loaded>'} to the hfile name, unless
361    * "hbase.mapreduce.bulkload.assign.sequenceNumbers" is
362    * explicitly turned off.
363    * If "hbase.mapreduce.bulkload.assign.sequenceNumbers"
364    * is turned off, fall back to BULKLOAD_TIME_KEY.
365    * @return true if this storefile was created by bulk load.
366    */
367   public boolean isBulkLoadResult() {
368     boolean bulkLoadedHFile = false;
369     String fileName = this.getPath().getName();
370     int startPos = fileName.indexOf("SeqId_");
371     if (startPos != -1) {
372       bulkLoadedHFile = true;
373     }
374     return bulkLoadedHFile || (metadataMap != null && metadataMap.containsKey(BULKLOAD_TIME_KEY));
375   }
376
377   @VisibleForTesting
378   public boolean isCompactedAway() {
379     if (this.reader != null) {
380       return this.reader.isCompactedAway();
381     }
382     return true;
383   }
384
385   @VisibleForTesting
386   public int getRefCount() {
387     return this.reader.getRefCount().get();
388   }
389
390   /**
391    * Return the timestamp at which this bulk load file was generated.
392    */
393   public long getBulkLoadTimestamp() {
394     byte[] bulkLoadTimestamp = metadataMap.get(BULKLOAD_TIME_KEY);
395     return (bulkLoadTimestamp == null) ? 0 : Bytes.toLong(bulkLoadTimestamp);
396   }
397
398   /**
399    * @return the cached value of HDFS blocks distribution. The cached value is
400    * calculated when store file is opened.
401    */
402   public HDFSBlocksDistribution getHDFSBlockDistribution() {
403     return this.fileInfo.getHDFSBlockDistribution();
404   }
405
406   /**
407    * Opens reader on this store file.  Called by Constructor.
408    * @return Reader for the store file.
409    * @throws IOException
410    * @see #closeReader(boolean)
411    */
412   private StoreFileReader open(boolean canUseDropBehind) throws IOException {
413     if (this.reader != null) {
414       throw new IllegalAccessError("Already open");
415     }
416
417     // Open the StoreFile.Reader
418     this.reader = fileInfo.open(this.fs, this.cacheConf, canUseDropBehind);
419
420     // Load up indices and fileinfo. This also loads Bloom filter type.
421     metadataMap = Collections.unmodifiableMap(this.reader.loadFileInfo());
422
423     // Read in our metadata.
424     byte [] b = metadataMap.get(MAX_SEQ_ID_KEY);
425     if (b != null) {
426       // By convention, if halfhfile, top half has a sequence number > bottom
427       // half. Thats why we add one in below. Its done for case the two halves
428       // are ever merged back together --rare.  Without it, on open of store,
429       // since store files are distinguished by sequence id, the one half would
430       // subsume the other.
431       this.sequenceid = Bytes.toLong(b);
432       if (fileInfo.isTopReference()) {
433         this.sequenceid += 1;
434       }
435     }
436
437     if (isBulkLoadResult()){
438       // generate the sequenceId from the fileName
439       // fileName is of the form <randomName>_SeqId_<id-when-loaded>_
440       String fileName = this.getPath().getName();
441       // Use lastIndexOf() to get the last, most recent bulk load seqId.
442       int startPos = fileName.lastIndexOf("SeqId_");
443       if (startPos != -1) {
444         this.sequenceid = Long.parseLong(fileName.substring(startPos + 6,
445             fileName.indexOf('_', startPos + 6)));
446         // Handle reference files as done above.
447         if (fileInfo.isTopReference()) {
448           this.sequenceid += 1;
449         }
450       }
451       // SKIP_RESET_SEQ_ID only works in bulk loaded file.
452       // In mob compaction, the hfile where the cells contain the path of a new mob file is bulk
453       // loaded to hbase, these cells have the same seqIds with the old ones. We do not want
454       // to reset new seqIds for them since this might make a mess of the visibility of cells that
455       // have the same row key but different seqIds.
456       this.reader.setSkipResetSeqId(isSkipResetSeqId(metadataMap.get(SKIP_RESET_SEQ_ID)));
457       this.reader.setBulkLoaded(true);
458     }
459     this.reader.setSequenceID(this.sequenceid);
460
461     b = metadataMap.get(HFile.Writer.MAX_MEMSTORE_TS_KEY);
462     if (b != null) {
463       this.maxMemstoreTS = Bytes.toLong(b);
464     }
465
466     b = metadataMap.get(MAJOR_COMPACTION_KEY);
467     if (b != null) {
468       boolean mc = Bytes.toBoolean(b);
469       if (this.majorCompaction == null) {
470         this.majorCompaction = new AtomicBoolean(mc);
471       } else {
472         this.majorCompaction.set(mc);
473       }
474     } else {
475       // Presume it is not major compacted if it doesn't explicity say so
476       // HFileOutputFormat explicitly sets the major compacted key.
477       this.majorCompaction = new AtomicBoolean(false);
478     }
479
480     b = metadataMap.get(EXCLUDE_FROM_MINOR_COMPACTION_KEY);
481     this.excludeFromMinorCompaction = (b != null && Bytes.toBoolean(b));
482
483     BloomType hfileBloomType = reader.getBloomFilterType();
484     if (cfBloomType != BloomType.NONE) {
485       reader.loadBloomfilter(BlockType.GENERAL_BLOOM_META);
486       if (hfileBloomType != cfBloomType) {
487         LOG.info("HFile Bloom filter type for "
488             + reader.getHFileReader().getName() + ": " + hfileBloomType
489             + ", but " + cfBloomType + " specified in column family "
490             + "configuration");
491       }
492     } else if (hfileBloomType != BloomType.NONE) {
493       LOG.info("Bloom filter turned off by CF config for "
494           + reader.getHFileReader().getName());
495     }
496
497     // load delete family bloom filter
498     reader.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META);
499
500     try {
501       this.reader.timeRange = TimeRangeTracker.getTimeRange(metadataMap.get(TIMERANGE_KEY));
502     } catch (IllegalArgumentException e) {
503       LOG.error("Error reading timestamp range data from meta -- " +
504           "proceeding without", e);
505       this.reader.timeRange = null;
506     }
507     // initialize so we can reuse them after reader closed.
508     firstKey = reader.getFirstKey();
509     lastKey = reader.getLastKey();
510     comparator = reader.getComparator();
511     return this.reader;
512   }
513
514   public StoreFileReader createReader() throws IOException {
515     return createReader(false);
516   }
517
518   /**
519    * @return Reader for StoreFile. creates if necessary
520    * @throws IOException
521    */
522   public StoreFileReader createReader(boolean canUseDropBehind) throws IOException {
523     if (this.reader == null) {
524       try {
525         this.reader = open(canUseDropBehind);
526       } catch (IOException e) {
527         try {
528           boolean evictOnClose =
529               cacheConf != null? cacheConf.shouldEvictOnClose(): true;
530           this.closeReader(evictOnClose);
531         } catch (IOException ee) {
532         }
533         throw e;
534       }
535
536     }
537     return this.reader;
538   }
539
540   /**
541    * @return Current reader.  Must call createReader first else returns null.
542    * @see #createReader()
543    */
544   public StoreFileReader getReader() {
545     return this.reader;
546   }
547
548   /**
549    * @param evictOnClose whether to evict blocks belonging to this file
550    * @throws IOException
551    */
552   public synchronized void closeReader(boolean evictOnClose)
553       throws IOException {
554     if (this.reader != null) {
555       this.reader.close(evictOnClose);
556       this.reader = null;
557     }
558   }
559
560   /**
561    * Marks the status of the file as compactedAway.
562    */
563   public void markCompactedAway() {
564     if (this.reader != null) {
565       this.reader.markCompactedAway();
566     }
567   }
568
569   /**
570    * Delete this file
571    * @throws IOException
572    */
573   public void deleteReader() throws IOException {
574     boolean evictOnClose =
575         cacheConf != null? cacheConf.shouldEvictOnClose(): true;
576     closeReader(evictOnClose);
577     this.fs.delete(getPath(), true);
578   }
579
580   @Override
581   public String toString() {
582     return this.fileInfo.toString();
583   }
584
585   /**
586    * @return a length description of this StoreFile, suitable for debug output
587    */
588   public String toStringDetailed() {
589     StringBuilder sb = new StringBuilder();
590     sb.append(this.getPath().toString());
591     sb.append(", isReference=").append(isReference());
592     sb.append(", isBulkLoadResult=").append(isBulkLoadResult());
593     if (isBulkLoadResult()) {
594       sb.append(", bulkLoadTS=").append(getBulkLoadTimestamp());
595     } else {
596       sb.append(", seqid=").append(getMaxSequenceId());
597     }
598     sb.append(", majorCompaction=").append(isMajorCompaction());
599
600     return sb.toString();
601   }
602
603   /**
604    * Gets whether to skip resetting the sequence id for cells.
605    * @param skipResetSeqId The byte array of boolean.
606    * @return Whether to skip resetting the sequence id.
607    */
608   private boolean isSkipResetSeqId(byte[] skipResetSeqId) {
609     if (skipResetSeqId != null && skipResetSeqId.length == 1) {
610       return Bytes.toBoolean(skipResetSeqId);
611     }
612     return false;
613   }
614
615   /**
616    * @param fs
617    * @param dir Directory to create file in.
618    * @return random filename inside passed <code>dir</code>
619    */
620   public static Path getUniqueFile(final FileSystem fs, final Path dir)
621       throws IOException {
622     if (!fs.getFileStatus(dir).isDirectory()) {
623       throw new IOException("Expecting " + dir.toString() +
624         " to be a directory");
625     }
626     return new Path(dir, UUID.randomUUID().toString().replaceAll("-", ""));
627   }
628
629   public Long getMinimumTimestamp() {
630     return getReader().timeRange == null? null: getReader().timeRange.getMin();
631   }
632
633   public Long getMaximumTimestamp() {
634     return getReader().timeRange == null? null: getReader().timeRange.getMax();
635   }
636
637
638   /**
639    * Gets the approximate mid-point of this file that is optimal for use in splitting it.
640    * @param comparator Comparator used to compare KVs.
641    * @return The split point row, or null if splitting is not possible, or reader is null.
642    */
643   byte[] getFileSplitPoint(CellComparator comparator) throws IOException {
644     if (this.reader == null) {
645       LOG.warn("Storefile " + this + " Reader is null; cannot get split point");
646       return null;
647     }
648     // Get first, last, and mid keys.  Midkey is the key that starts block
649     // in middle of hfile.  Has column and timestamp.  Need to return just
650     // the row we want to split on as midkey.
651     Cell midkey = this.reader.midkey();
652     if (midkey != null) {
653       Cell firstKey = this.reader.getFirstKey();
654       Cell lastKey = this.reader.getLastKey();
655       // if the midkey is the same as the first or last keys, we cannot (ever) split this region.
656       if (comparator.compareRows(midkey, firstKey) == 0
657           || comparator.compareRows(midkey, lastKey) == 0) {
658         if (LOG.isDebugEnabled()) {
659           LOG.debug("cannot split because midkey is the same as first or last row");
660         }
661         return null;
662       }
663       return CellUtil.cloneRow(midkey);
664     }
665     return null;
666   }
667
668   /**
669    * Useful comparators for comparing StoreFiles.
670    */
671   public abstract static class Comparators {
672     /**
673      * Comparator that compares based on the Sequence Ids of the
674      * the StoreFiles. Bulk loads that did not request a seq ID
675      * are given a seq id of -1; thus, they are placed before all non-
676      * bulk loads, and bulk loads with sequence Id. Among these files,
677      * the size is used to determine the ordering, then bulkLoadTime.
678      * If there are ties, the path name is used as a tie-breaker.
679      */
680     public static final Comparator<StoreFile> SEQ_ID =
681       Ordering.compound(ImmutableList.of(
682           Ordering.natural().onResultOf(new GetSeqId()),
683           Ordering.natural().onResultOf(new GetFileSize()).reverse(),
684           Ordering.natural().onResultOf(new GetBulkTime()),
685           Ordering.natural().onResultOf(new GetPathName())
686       ));
687
688     /**
689      * Comparator for time-aware compaction. SeqId is still the first
690      *   ordering criterion to maintain MVCC.
691      */
692     public static final Comparator<StoreFile> SEQ_ID_MAX_TIMESTAMP =
693       Ordering.compound(ImmutableList.of(
694         Ordering.natural().onResultOf(new GetSeqId()),
695         Ordering.natural().onResultOf(new GetMaxTimestamp()),
696         Ordering.natural().onResultOf(new GetFileSize()).reverse(),
697         Ordering.natural().onResultOf(new GetBulkTime()),
698         Ordering.natural().onResultOf(new GetPathName())
699       ));
700
701     private static class GetSeqId implements Function<StoreFile, Long> {
702       @Override
703       public Long apply(StoreFile sf) {
704         return sf.getMaxSequenceId();
705       }
706     }
707
708     private static class GetFileSize implements Function<StoreFile, Long> {
709       @Override
710       public Long apply(StoreFile sf) {
711         if (sf.getReader() != null) {
712           return sf.getReader().length();
713         } else {
714           // the reader may be null for the compacted files and if the archiving
715           // had failed.
716           return -1L;
717         }
718       }
719     }
720
721     private static class GetBulkTime implements Function<StoreFile, Long> {
722       @Override
723       public Long apply(StoreFile sf) {
724         if (!sf.isBulkLoadResult()) return Long.MAX_VALUE;
725         return sf.getBulkLoadTimestamp();
726       }
727     }
728
729     private static class GetPathName implements Function<StoreFile, String> {
730       @Override
731       public String apply(StoreFile sf) {
732         return sf.getPath().getName();
733       }
734     }
735
736     private static class GetMaxTimestamp implements Function<StoreFile, Long> {
737       @Override
738       public Long apply(StoreFile sf) {
739         return sf.getMaximumTimestamp() == null? (Long)Long.MAX_VALUE : sf.getMaximumTimestamp();
740       }
741     }
742   }
743 }