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   /**
203    * Constructor, loads a reader and it's indices, etc. May allocate a
204    * substantial amount of ram depending on the underlying files (10-20MB?).
205    *
206    * @param fs  The current file system to use.
207    * @param fileInfo  The store file information.
208    * @param conf  The current configuration.
209    * @param cacheConf  The cache configuration and block cache reference.
210    * @param cfBloomType The bloom type to use for this store file as specified
211    *          by column family configuration. This may or may not be the same
212    *          as the Bloom filter type actually present in the HFile, because
213    *          column family configuration might change. If this is
214    *          {@link BloomType#NONE}, the existing Bloom filter is ignored.
215    * @throws IOException When opening the reader fails.
216    */
217   public StoreFile(final FileSystem fs, final StoreFileInfo fileInfo, final Configuration conf,
218       final CacheConfig cacheConf,  final BloomType cfBloomType) throws IOException {
219     this.fs = fs;
220     this.fileInfo = fileInfo;
221     this.cacheConf = cacheConf;
222
223     if (BloomFilterFactory.isGeneralBloomEnabled(conf)) {
224       this.cfBloomType = cfBloomType;
225     } else {
226       LOG.info("Ignoring bloom filter check for file " + this.getPath() + ": " +
227           "cfBloomType=" + cfBloomType + " (disabled in config)");
228       this.cfBloomType = BloomType.NONE;
229     }
230   }
231
232   /**
233    * Clone
234    * @param other The StoreFile to clone from
235    */
236   public StoreFile(final StoreFile other) {
237     this.fs = other.fs;
238     this.fileInfo = other.fileInfo;
239     this.cacheConf = other.cacheConf;
240     this.cfBloomType = other.cfBloomType;
241     this.metadataMap = other.metadataMap;
242   }
243
244   /**
245    * Clone a StoreFile for opening private reader.
246    */
247   public StoreFile cloneForReader() {
248     return new StoreFile(this);
249   }
250
251   /**
252    * @return the StoreFile object associated to this StoreFile.
253    *         null if the StoreFile is not a reference.
254    */
255   public StoreFileInfo getFileInfo() {
256     return this.fileInfo;
257   }
258
259   /**
260    * @return Path or null if this StoreFile was made with a Stream.
261    */
262   public Path getPath() {
263     return this.fileInfo.getPath();
264   }
265
266   /**
267    * @return Returns the qualified path of this StoreFile
268    */
269   public Path getQualifiedPath() {
270     return this.fileInfo.getPath().makeQualified(fs);
271   }
272
273   /**
274    * @return True if this is a StoreFile Reference; call
275    * after {@link #open(boolean canUseDropBehind)} else may get wrong answer.
276    */
277   public boolean isReference() {
278     return this.fileInfo.isReference();
279   }
280
281   /**
282    * @return True if this is HFile.
283    */
284   public boolean isHFile() {
285     return StoreFileInfo.isHFile(this.fileInfo.getPath());
286   }
287
288   /**
289    * @return True if this file was made by a major compaction.
290    */
291   public boolean isMajorCompaction() {
292     if (this.majorCompaction == null) {
293       throw new NullPointerException("This has not been set yet");
294     }
295     return this.majorCompaction.get();
296   }
297
298   /**
299    * @return True if this file should not be part of a minor compaction.
300    */
301   public boolean excludeFromMinorCompaction() {
302     return this.excludeFromMinorCompaction;
303   }
304
305   /**
306    * @return This files maximum edit sequence id.
307    */
308   public long getMaxSequenceId() {
309     return this.sequenceid;
310   }
311
312   public long getModificationTimeStamp() throws IOException {
313     return (fileInfo == null) ? 0 : fileInfo.getModificationTime();
314   }
315
316   /**
317    * Only used by the Striped Compaction Policy
318    * @param key
319    * @return value associated with the metadata key
320    */
321   public byte[] getMetadataValue(byte[] key) {
322     return metadataMap.get(key);
323   }
324
325   /**
326    * Return the largest memstoreTS found across all storefiles in
327    * the given list. Store files that were created by a mapreduce
328    * bulk load are ignored, as they do not correspond to any specific
329    * put operation, and thus do not have a memstoreTS associated with them.
330    * @return 0 if no non-bulk-load files are provided or, this is Store that
331    * does not yet have any store files.
332    */
333   public static long getMaxMemstoreTSInList(Collection<StoreFile> sfs) {
334     long max = 0;
335     for (StoreFile sf : sfs) {
336       if (!sf.isBulkLoadResult()) {
337         max = Math.max(max, sf.getMaxMemstoreTS());
338       }
339     }
340     return max;
341   }
342
343   /**
344    * Return the highest sequence ID found across all storefiles in
345    * the given list.
346    * @param sfs
347    * @return 0 if no non-bulk-load files are provided or, this is Store that
348    * does not yet have any store files.
349    */
350   public static long getMaxSequenceIdInList(Collection<StoreFile> sfs) {
351     long max = 0;
352     for (StoreFile sf : sfs) {
353       max = Math.max(max, sf.getMaxSequenceId());
354     }
355     return max;
356   }
357
358   /**
359    * Check if this storefile was created by bulk load.
360    * When a hfile is bulk loaded into HBase, we append
361    * {@code '_SeqId_<id-when-loaded>'} to the hfile name, unless
362    * "hbase.mapreduce.bulkload.assign.sequenceNumbers" is
363    * explicitly turned off.
364    * If "hbase.mapreduce.bulkload.assign.sequenceNumbers"
365    * is turned off, fall back to BULKLOAD_TIME_KEY.
366    * @return true if this storefile was created by bulk load.
367    */
368   public boolean isBulkLoadResult() {
369     boolean bulkLoadedHFile = false;
370     String fileName = this.getPath().getName();
371     int startPos = fileName.indexOf("SeqId_");
372     if (startPos != -1) {
373       bulkLoadedHFile = true;
374     }
375     return bulkLoadedHFile || (metadataMap != null && metadataMap.containsKey(BULKLOAD_TIME_KEY));
376   }
377
378   @VisibleForTesting
379   public boolean isCompactedAway() {
380     if (this.reader != null) {
381       return this.reader.isCompactedAway();
382     }
383     return true;
384   }
385
386   @VisibleForTesting
387   public int getRefCount() {
388     return this.reader.getRefCount().get();
389   }
390
391   /**
392    * Return the timestamp at which this bulk load file was generated.
393    */
394   public long getBulkLoadTimestamp() {
395     byte[] bulkLoadTimestamp = metadataMap.get(BULKLOAD_TIME_KEY);
396     return (bulkLoadTimestamp == null) ? 0 : Bytes.toLong(bulkLoadTimestamp);
397   }
398
399   /**
400    * @return the cached value of HDFS blocks distribution. The cached value is
401    * calculated when store file is opened.
402    */
403   public HDFSBlocksDistribution getHDFSBlockDistribution() {
404     return this.fileInfo.getHDFSBlockDistribution();
405   }
406
407   /**
408    * Opens reader on this store file.  Called by Constructor.
409    * @return Reader for the store file.
410    * @throws IOException
411    * @see #closeReader(boolean)
412    */
413   private StoreFileReader open(boolean canUseDropBehind) throws IOException {
414     if (this.reader != null) {
415       throw new IllegalAccessError("Already open");
416     }
417
418     // Open the StoreFile.Reader
419     this.reader = fileInfo.open(this.fs, this.cacheConf, canUseDropBehind);
420
421     // Load up indices and fileinfo. This also loads Bloom filter type.
422     metadataMap = Collections.unmodifiableMap(this.reader.loadFileInfo());
423
424     // Read in our metadata.
425     byte [] b = metadataMap.get(MAX_SEQ_ID_KEY);
426     if (b != null) {
427       // By convention, if halfhfile, top half has a sequence number > bottom
428       // half. Thats why we add one in below. Its done for case the two halves
429       // are ever merged back together --rare.  Without it, on open of store,
430       // since store files are distinguished by sequence id, the one half would
431       // subsume the other.
432       this.sequenceid = Bytes.toLong(b);
433       if (fileInfo.isTopReference()) {
434         this.sequenceid += 1;
435       }
436     }
437
438     if (isBulkLoadResult()){
439       // generate the sequenceId from the fileName
440       // fileName is of the form <randomName>_SeqId_<id-when-loaded>_
441       String fileName = this.getPath().getName();
442       // Use lastIndexOf() to get the last, most recent bulk load seqId.
443       int startPos = fileName.lastIndexOf("SeqId_");
444       if (startPos != -1) {
445         this.sequenceid = Long.parseLong(fileName.substring(startPos + 6,
446             fileName.indexOf('_', startPos + 6)));
447         // Handle reference files as done above.
448         if (fileInfo.isTopReference()) {
449           this.sequenceid += 1;
450         }
451       }
452       // SKIP_RESET_SEQ_ID only works in bulk loaded file.
453       // In mob compaction, the hfile where the cells contain the path of a new mob file is bulk
454       // loaded to hbase, these cells have the same seqIds with the old ones. We do not want
455       // to reset new seqIds for them since this might make a mess of the visibility of cells that
456       // have the same row key but different seqIds.
457       this.reader.setSkipResetSeqId(isSkipResetSeqId(metadataMap.get(SKIP_RESET_SEQ_ID)));
458       this.reader.setBulkLoaded(true);
459     }
460     this.reader.setSequenceID(this.sequenceid);
461
462     b = metadataMap.get(HFile.Writer.MAX_MEMSTORE_TS_KEY);
463     if (b != null) {
464       this.maxMemstoreTS = Bytes.toLong(b);
465     }
466
467     b = metadataMap.get(MAJOR_COMPACTION_KEY);
468     if (b != null) {
469       boolean mc = Bytes.toBoolean(b);
470       if (this.majorCompaction == null) {
471         this.majorCompaction = new AtomicBoolean(mc);
472       } else {
473         this.majorCompaction.set(mc);
474       }
475     } else {
476       // Presume it is not major compacted if it doesn't explicity say so
477       // HFileOutputFormat explicitly sets the major compacted key.
478       this.majorCompaction = new AtomicBoolean(false);
479     }
480
481     b = metadataMap.get(EXCLUDE_FROM_MINOR_COMPACTION_KEY);
482     this.excludeFromMinorCompaction = (b != null && Bytes.toBoolean(b));
483
484     BloomType hfileBloomType = reader.getBloomFilterType();
485     if (cfBloomType != BloomType.NONE) {
486       reader.loadBloomfilter(BlockType.GENERAL_BLOOM_META);
487       if (hfileBloomType != cfBloomType) {
488         LOG.info("HFile Bloom filter type for "
489             + reader.getHFileReader().getName() + ": " + hfileBloomType
490             + ", but " + cfBloomType + " specified in column family "
491             + "configuration");
492       }
493     } else if (hfileBloomType != BloomType.NONE) {
494       LOG.info("Bloom filter turned off by CF config for "
495           + reader.getHFileReader().getName());
496     }
497
498     // load delete family bloom filter
499     reader.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META);
500
501     try {
502       this.reader.timeRange = TimeRangeTracker.getTimeRange(metadataMap.get(TIMERANGE_KEY));
503     } catch (IllegalArgumentException e) {
504       LOG.error("Error reading timestamp range data from meta -- " +
505           "proceeding without", e);
506       this.reader.timeRange = null;
507     }
508     // initialize so we can reuse them after reader closed.
509     firstKey = reader.getFirstKey();
510     lastKey = reader.getLastKey();
511     comparator = reader.getComparator();
512     return this.reader;
513   }
514
515   public StoreFileReader createReader() throws IOException {
516     return createReader(false);
517   }
518
519   /**
520    * @return Reader for StoreFile. creates if necessary
521    * @throws IOException
522    */
523   public StoreFileReader createReader(boolean canUseDropBehind) throws IOException {
524     if (this.reader == null) {
525       try {
526         this.reader = open(canUseDropBehind);
527       } catch (IOException e) {
528         try {
529           boolean evictOnClose =
530               cacheConf != null? cacheConf.shouldEvictOnClose(): true;
531           this.closeReader(evictOnClose);
532         } catch (IOException ee) {
533         }
534         throw e;
535       }
536
537     }
538     return this.reader;
539   }
540
541   /**
542    * @return Current reader.  Must call createReader first else returns null.
543    * @see #createReader()
544    */
545   public StoreFileReader getReader() {
546     return this.reader;
547   }
548
549   /**
550    * @param evictOnClose whether to evict blocks belonging to this file
551    * @throws IOException
552    */
553   public synchronized void closeReader(boolean evictOnClose)
554       throws IOException {
555     if (this.reader != null) {
556       this.reader.close(evictOnClose);
557       this.reader = null;
558     }
559   }
560
561   /**
562    * Marks the status of the file as compactedAway.
563    */
564   public void markCompactedAway() {
565     if (this.reader != null) {
566       this.reader.markCompactedAway();
567     }
568   }
569
570   /**
571    * Delete this file
572    * @throws IOException
573    */
574   public void deleteReader() throws IOException {
575     boolean evictOnClose =
576         cacheConf != null? cacheConf.shouldEvictOnClose(): true;
577     closeReader(evictOnClose);
578     this.fs.delete(getPath(), true);
579   }
580
581   @Override
582   public String toString() {
583     return this.fileInfo.toString();
584   }
585
586   /**
587    * @return a length description of this StoreFile, suitable for debug output
588    */
589   public String toStringDetailed() {
590     StringBuilder sb = new StringBuilder();
591     sb.append(this.getPath().toString());
592     sb.append(", isReference=").append(isReference());
593     sb.append(", isBulkLoadResult=").append(isBulkLoadResult());
594     if (isBulkLoadResult()) {
595       sb.append(", bulkLoadTS=").append(getBulkLoadTimestamp());
596     } else {
597       sb.append(", seqid=").append(getMaxSequenceId());
598     }
599     sb.append(", majorCompaction=").append(isMajorCompaction());
600
601     return sb.toString();
602   }
603
604   /**
605    * Gets whether to skip resetting the sequence id for cells.
606    * @param skipResetSeqId The byte array of boolean.
607    * @return Whether to skip resetting the sequence id.
608    */
609   private boolean isSkipResetSeqId(byte[] skipResetSeqId) {
610     if (skipResetSeqId != null && skipResetSeqId.length == 1) {
611       return Bytes.toBoolean(skipResetSeqId);
612     }
613     return false;
614   }
615
616   /**
617    * @param fs
618    * @param dir Directory to create file in.
619    * @return random filename inside passed <code>dir</code>
620    */
621   public static Path getUniqueFile(final FileSystem fs, final Path dir)
622       throws IOException {
623     if (!fs.getFileStatus(dir).isDirectory()) {
624       throw new IOException("Expecting " + dir.toString() +
625         " to be a directory");
626     }
627     return new Path(dir, UUID.randomUUID().toString().replaceAll("-", ""));
628   }
629
630   public Long getMinimumTimestamp() {
631     return getReader().timeRange == null? null: getReader().timeRange.getMin();
632   }
633
634   public Long getMaximumTimestamp() {
635     return getReader().timeRange == null? null: getReader().timeRange.getMax();
636   }
637
638
639   /**
640    * Gets the approximate mid-point of this file that is optimal for use in splitting it.
641    * @param comparator Comparator used to compare KVs.
642    * @return The split point row, or null if splitting is not possible, or reader is null.
643    */
644   byte[] getFileSplitPoint(CellComparator comparator) throws IOException {
645     if (this.reader == null) {
646       LOG.warn("Storefile " + this + " Reader is null; cannot get split point");
647       return null;
648     }
649     // Get first, last, and mid keys.  Midkey is the key that starts block
650     // in middle of hfile.  Has column and timestamp.  Need to return just
651     // the row we want to split on as midkey.
652     Cell midkey = this.reader.midkey();
653     if (midkey != null) {
654       Cell firstKey = this.reader.getFirstKey();
655       Cell lastKey = this.reader.getLastKey();
656       // if the midkey is the same as the first or last keys, we cannot (ever) split this region.
657       if (comparator.compareRows(midkey, firstKey) == 0
658           || comparator.compareRows(midkey, lastKey) == 0) {
659         if (LOG.isDebugEnabled()) {
660           LOG.debug("cannot split because midkey is the same as first or last row");
661         }
662         return null;
663       }
664       return CellUtil.cloneRow(midkey);
665     }
666     return null;
667   }
668
669   /**
670    * Useful comparators for comparing StoreFiles.
671    */
672   public abstract static class Comparators {
673     /**
674      * Comparator that compares based on the Sequence Ids of the
675      * the StoreFiles. Bulk loads that did not request a seq ID
676      * are given a seq id of -1; thus, they are placed before all non-
677      * bulk loads, and bulk loads with sequence Id. Among these files,
678      * the size is used to determine the ordering, then bulkLoadTime.
679      * If there are ties, the path name is used as a tie-breaker.
680      */
681     public static final Comparator<StoreFile> SEQ_ID =
682       Ordering.compound(ImmutableList.of(
683           Ordering.natural().onResultOf(new GetSeqId()),
684           Ordering.natural().onResultOf(new GetFileSize()).reverse(),
685           Ordering.natural().onResultOf(new GetBulkTime()),
686           Ordering.natural().onResultOf(new GetPathName())
687       ));
688
689     /**
690      * Comparator for time-aware compaction. SeqId is still the first
691      *   ordering criterion to maintain MVCC.
692      */
693     public static final Comparator<StoreFile> SEQ_ID_MAX_TIMESTAMP =
694       Ordering.compound(ImmutableList.of(
695         Ordering.natural().onResultOf(new GetSeqId()),
696         Ordering.natural().onResultOf(new GetMaxTimestamp()),
697         Ordering.natural().onResultOf(new GetFileSize()).reverse(),
698         Ordering.natural().onResultOf(new GetBulkTime()),
699         Ordering.natural().onResultOf(new GetPathName())
700       ));
701
702     private static class GetSeqId implements Function<StoreFile, Long> {
703       @Override
704       public Long apply(StoreFile sf) {
705         return sf.getMaxSequenceId();
706       }
707     }
708
709     private static class GetFileSize implements Function<StoreFile, Long> {
710       @Override
711       public Long apply(StoreFile sf) {
712         if (sf.getReader() != null) {
713           return sf.getReader().length();
714         } else {
715           // the reader may be null for the compacted files and if the archiving
716           // had failed.
717           return -1L;
718         }
719       }
720     }
721
722     private static class GetBulkTime implements Function<StoreFile, Long> {
723       @Override
724       public Long apply(StoreFile sf) {
725         if (!sf.isBulkLoadResult()) return Long.MAX_VALUE;
726         return sf.getBulkLoadTimestamp();
727       }
728     }
729
730     private static class GetPathName implements Function<StoreFile, String> {
731       @Override
732       public String apply(StoreFile sf) {
733         return sf.getPath().getName();
734       }
735     }
736
737     private static class GetMaxTimestamp implements Function<StoreFile, Long> {
738       @Override
739       public Long apply(StoreFile sf) {
740         return sf.getMaximumTimestamp() == null? (Long)Long.MAX_VALUE : sf.getMaximumTimestamp();
741       }
742     }
743   }
744 }