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