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.IOException;
23  import java.io.InterruptedIOException;
24  import java.util.ArrayList;
25  import java.util.Collection;
26  import java.util.Collections;
27  import java.util.List;
28  import java.util.NavigableSet;
29  import java.util.Random;
30  import java.util.SortedSet;
31  import java.util.concurrent.Callable;
32  import java.util.concurrent.CompletionService;
33  import java.util.concurrent.CopyOnWriteArraySet;
34  import java.util.concurrent.ExecutionException;
35  import java.util.concurrent.ExecutorCompletionService;
36  import java.util.concurrent.Future;
37  import java.util.concurrent.ThreadPoolExecutor;
38  import java.util.concurrent.atomic.AtomicLong;
39  import java.util.concurrent.locks.ReentrantReadWriteLock;
40  
41  import org.apache.commons.logging.Log;
42  import org.apache.commons.logging.LogFactory;
43  import org.apache.hadoop.conf.Configuration;
44  import org.apache.hadoop.fs.FileStatus;
45  import org.apache.hadoop.fs.FileSystem;
46  import org.apache.hadoop.fs.FileUtil;
47  import org.apache.hadoop.fs.Path;
48  import org.apache.hadoop.hbase.HBaseFileSystem;
49  import org.apache.hadoop.hbase.HColumnDescriptor;
50  import org.apache.hadoop.hbase.HConstants;
51  import org.apache.hadoop.hbase.HRegionInfo;
52  import org.apache.hadoop.hbase.KeyValue;
53  import org.apache.hadoop.hbase.KeyValue.KVComparator;
54  import org.apache.hadoop.hbase.RemoteExceptionHandler;
55  import org.apache.hadoop.hbase.backup.HFileArchiver;
56  import org.apache.hadoop.hbase.client.Scan;
57  import org.apache.hadoop.hbase.fs.HFileSystem;
58  import org.apache.hadoop.hbase.io.HFileLink;
59  import org.apache.hadoop.hbase.io.HeapSize;
60  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
61  import org.apache.hadoop.hbase.io.hfile.Compression;
62  import org.apache.hadoop.hbase.io.hfile.HFile;
63  import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
64  import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoderImpl;
65  import org.apache.hadoop.hbase.io.hfile.HFileScanner;
66  import org.apache.hadoop.hbase.io.hfile.InvalidHFileException;
67  import org.apache.hadoop.hbase.io.hfile.NoOpDataBlockEncoder;
68  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
69  import org.apache.hadoop.hbase.regionserver.compactions.CompactSelection;
70  import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
71  import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
72  import org.apache.hadoop.hbase.regionserver.metrics.SchemaConfigured;
73  import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
74  import org.apache.hadoop.hbase.util.Bytes;
75  import org.apache.hadoop.hbase.util.ChecksumType;
76  import org.apache.hadoop.hbase.util.ClassSize;
77  import org.apache.hadoop.hbase.util.CollectionBackedScanner;
78  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
79  import org.apache.hadoop.hbase.util.FSUtils;
80  import org.apache.hadoop.util.StringUtils;
81  
82  import com.google.common.base.Preconditions;
83  import com.google.common.base.Predicate;
84  import com.google.common.collect.Collections2;
85  import com.google.common.collect.ImmutableList;
86  import com.google.common.collect.Lists;
87  
88  /**
89   * A Store holds a column family in a Region.  Its a memstore and a set of zero
90   * or more StoreFiles, which stretch backwards over time.
91   *
92   * <p>There's no reason to consider append-logging at this level; all logging
93   * and locking is handled at the HRegion level.  Store just provides
94   * services to manage sets of StoreFiles.  One of the most important of those
95   * services is compaction services where files are aggregated once they pass
96   * a configurable threshold.
97   *
98   * <p>The only thing having to do with logs that Store needs to deal with is
99   * the reconstructionLog.  This is a segment of an HRegion's log that might
100  * NOT be present upon startup.  If the param is NULL, there's nothing to do.
101  * If the param is non-NULL, we need to process the log to reconstruct
102  * a TreeMap that might not have been written to disk before the process
103  * died.
104  *
105  * <p>It's assumed that after this constructor returns, the reconstructionLog
106  * file will be deleted (by whoever has instantiated the Store).
107  *
108  * <p>Locking and transactions are handled at a higher level.  This API should
109  * not be called directly but by an HRegion manager.
110  */
111 public class Store extends SchemaConfigured implements HeapSize {
112   static final Log LOG = LogFactory.getLog(Store.class);
113   
114   public static final String BLOCKING_STOREFILES_KEY = "hbase.hstore.blockingStoreFiles";
115   public static final int DEFAULT_BLOCKING_STOREFILE_COUNT = 7;
116 
117   protected final MemStore memstore;
118   // This stores directory in the filesystem.
119   private final Path homedir;
120   private final HRegion region;
121   private final HColumnDescriptor family;
122   final FileSystem fs;
123   final Configuration conf;
124   final CacheConfig cacheConf;
125   // ttl in milliseconds.
126   private long ttl;
127   private final int minFilesToCompact;
128   private final int maxFilesToCompact;
129   private final long minCompactSize;
130   private final long maxCompactSize;
131   private long lastCompactSize = 0;
132   volatile boolean forceMajor = false;
133   /* how many bytes to write between status checks */
134   static int closeCheckInterval = 0;
135   private final int blockingStoreFileCount;
136   private volatile long storeSize = 0L;
137   private volatile long totalUncompressedBytes = 0L;
138   private final Object flushLock = new Object();
139   final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
140   private final boolean verifyBulkLoads;
141   
142   private long blockingFileCount;
143 
144   /* The default priority for user-specified compaction requests.
145    * The user gets top priority unless we have blocking compactions. (Pri <= 0)
146    */
147   public static final int PRIORITY_USER = 1;
148   public static final int NO_PRIORITY = Integer.MIN_VALUE;
149 
150   // not private for testing
151   /* package */ScanInfo scanInfo;
152   /*
153    * List of store files inside this store. This is an immutable list that
154    * is atomically replaced when its contents change.
155    */
156   private volatile ImmutableList<StoreFile> storefiles = null;
157 
158   List<StoreFile> filesCompacting = Lists.newArrayList();
159 
160   // All access must be synchronized.
161   private final CopyOnWriteArraySet<ChangedReadersObserver> changedReaderObservers =
162     new CopyOnWriteArraySet<ChangedReadersObserver>();
163 
164   private final int blocksize;
165   private HFileDataBlockEncoder dataBlockEncoder;
166 
167   /** Checksum configuration */
168   private ChecksumType checksumType;
169   private int bytesPerChecksum;
170 
171   // Comparing KeyValues
172   final KeyValue.KVComparator comparator;
173 
174   private final Compactor compactor;
175 
176   private static final int DEFAULT_FLUSH_RETRIES_NUMBER = 10;
177   private static int flush_retries_number;
178   private static int pauseTime;
179 
180   /**
181    * Constructor
182    * @param basedir qualified path under which the region directory lives;
183    * generally the table subdirectory
184    * @param region
185    * @param family HColumnDescriptor for this column
186    * @param fs file system object
187    * @param confParam configuration object
188    * failed.  Can be null.
189    * @throws IOException
190    */
191   protected Store(Path basedir, HRegion region, HColumnDescriptor family,
192       FileSystem fs, Configuration confParam)
193   throws IOException {
194     super(new CompoundConfiguration().add(confParam).add(
195         family.getValues()), region.getTableDesc().getNameAsString(),
196         Bytes.toString(family.getName()));
197     HRegionInfo info = region.getRegionInfo();
198     this.fs = fs;
199     Path p = getStoreHomedir(basedir, info.getEncodedName(), family.getName());
200     this.homedir = createStoreHomeDir(this.fs, p);
201     this.region = region;
202     this.family = family;
203     // 'conf' renamed to 'confParam' b/c we use this.conf in the constructor
204     this.conf = new CompoundConfiguration().add(confParam).add(family.getValues());
205     this.blocksize = family.getBlocksize();
206 
207     this.dataBlockEncoder =
208         new HFileDataBlockEncoderImpl(family.getDataBlockEncodingOnDisk(),
209             family.getDataBlockEncoding());
210 
211     this.comparator = info.getComparator();
212     // getTimeToLive returns ttl in seconds.  Convert to milliseconds.
213     this.ttl = family.getTimeToLive();
214     if (ttl == HConstants.FOREVER) {
215       // default is unlimited ttl.
216       ttl = Long.MAX_VALUE;
217     } else if (ttl == -1) {
218       ttl = Long.MAX_VALUE;
219     } else {
220       // second -> ms adjust for user data
221       this.ttl *= 1000;
222     }
223     // used by ScanQueryMatcher
224     long timeToPurgeDeletes =
225         Math.max(conf.getLong("hbase.hstore.time.to.purge.deletes", 0), 0);
226     LOG.info("time to purge deletes set to " + timeToPurgeDeletes +
227         "ms in store " + this);
228     scanInfo = new ScanInfo(family, ttl, timeToPurgeDeletes, this.comparator);
229     this.memstore = new MemStore(conf, this.comparator);
230 
231     // By default, compact if storefile.count >= minFilesToCompact
232     this.minFilesToCompact = Math.max(2,
233       conf.getInt("hbase.hstore.compaction.min",
234         /*old name*/ conf.getInt("hbase.hstore.compactionThreshold", 3)));
235 
236     LOG.info("hbase.hstore.compaction.min = " + this.minFilesToCompact);
237     
238     // Setting up cache configuration for this family
239     this.cacheConf = new CacheConfig(conf, family);
240     this.blockingStoreFileCount =
241       conf.getInt("hbase.hstore.blockingStoreFiles", 7);
242 
243     this.maxFilesToCompact = conf.getInt("hbase.hstore.compaction.max", 10);
244     this.minCompactSize = conf.getLong("hbase.hstore.compaction.min.size",
245       this.region.memstoreFlushSize);
246     this.maxCompactSize
247       = conf.getLong("hbase.hstore.compaction.max.size", Long.MAX_VALUE);
248 
249     this.verifyBulkLoads = conf.getBoolean("hbase.hstore.bulkload.verify", false);
250     
251     this.blockingFileCount =
252                 conf.getInt(BLOCKING_STOREFILES_KEY, DEFAULT_BLOCKING_STOREFILE_COUNT);
253     
254     if (Store.closeCheckInterval == 0) {
255       Store.closeCheckInterval = conf.getInt(
256           "hbase.hstore.close.check.interval", 10*1000*1000 /* 10 MB */);
257     }
258     this.storefiles = sortAndClone(loadStoreFiles());
259 
260     // Initialize checksum type from name. The names are CRC32, CRC32C, etc.
261     this.checksumType = getChecksumType(conf);
262     // initilize bytes per checksum
263     this.bytesPerChecksum = getBytesPerChecksum(conf);
264     // Create a compaction tool instance
265     this.compactor = new Compactor(this.conf);
266     if (Store.flush_retries_number == 0) {
267       Store.flush_retries_number = conf.getInt(
268           "hbase.hstore.flush.retries.number", DEFAULT_FLUSH_RETRIES_NUMBER);
269       Store.pauseTime = conf.getInt(HConstants.HBASE_SERVER_PAUSE,
270           HConstants.DEFAULT_HBASE_SERVER_PAUSE);
271       if (Store.flush_retries_number <= 0) {
272         throw new IllegalArgumentException(
273             "hbase.hstore.flush.retries.number must be > 0, not "
274                 + Store.flush_retries_number);
275       }
276     }
277   }
278 
279   /**
280    * @param family
281    * @return
282    */
283   long getTTL(final HColumnDescriptor family) {
284     // HCD.getTimeToLive returns ttl in seconds.  Convert to milliseconds.
285     long ttl = family.getTimeToLive();
286     if (ttl == HConstants.FOREVER) {
287       // Default is unlimited ttl.
288       ttl = Long.MAX_VALUE;
289     } else if (ttl == -1) {
290       ttl = Long.MAX_VALUE;
291     } else {
292       // Second -> ms adjust for user data
293       ttl *= 1000;
294     }
295     return ttl;
296   }
297 
298   /**
299    * Create this store's homedir
300    * @param fs
301    * @param homedir
302    * @return Return <code>homedir</code>
303    * @throws IOException
304    */
305   Path createStoreHomeDir(final FileSystem fs,
306       final Path homedir) throws IOException {
307     if (!fs.exists(homedir) && !HBaseFileSystem.makeDirOnFileSystem(fs, homedir)) {
308         throw new IOException("Failed create of: " + homedir.toString());
309     }
310     return homedir;
311   }
312 
313   FileSystem getFileSystem() {
314     return this.fs;
315   }
316 
317   /**
318    * Returns the configured bytesPerChecksum value.
319    * @param conf The configuration
320    * @return The bytesPerChecksum that is set in the configuration
321    */
322   public static int getBytesPerChecksum(Configuration conf) {
323     return conf.getInt(HConstants.BYTES_PER_CHECKSUM,
324                        HFile.DEFAULT_BYTES_PER_CHECKSUM);
325   }
326 
327   /**
328    * Returns the configured checksum algorithm.
329    * @param conf The configuration
330    * @return The checksum algorithm that is set in the configuration
331    */
332   public static ChecksumType getChecksumType(Configuration conf) {
333     String checksumName = conf.get(HConstants.CHECKSUM_TYPE_NAME);
334     if (checksumName == null) {
335       return HFile.DEFAULT_CHECKSUM_TYPE;
336     } else {
337       return ChecksumType.nameToType(checksumName);
338     }
339   }
340 
341   public HColumnDescriptor getFamily() {
342     return this.family;
343   }
344 
345   /**
346    * @return The maximum sequence id in all store files.
347    */
348   long getMaxSequenceId() {
349     return StoreFile.getMaxSequenceIdInList(this.getStorefiles());
350   }
351 
352   /**
353    * @return The maximum memstoreTS in all store files.
354    */
355   public long getMaxMemstoreTS() {
356     return StoreFile.getMaxMemstoreTSInList(this.getStorefiles());
357   }
358 
359   /**
360    * @param tabledir
361    * @param encodedName Encoded region name.
362    * @param family
363    * @return Path to family/Store home directory.
364    */
365   public static Path getStoreHomedir(final Path tabledir,
366       final String encodedName, final byte [] family) {
367      return getStoreHomedir(tabledir, encodedName, Bytes.toString(family));
368    }
369 
370   /**
371    * @param tabledir
372    * @param encodedName Encoded region name.
373    * @param family
374    * @return Path to family/Store home directory.
375    */
376   public static Path getStoreHomedir(final Path tabledir,
377       final String encodedName, final String family) {
378     return new Path(tabledir, new Path(encodedName, new Path(family)));
379   }
380 
381   /**
382    * @param parentRegionDirectory directory for the parent region
383    * @param family family name of this store
384    * @return Path to the family/Store home directory
385    */
386   public static Path getStoreHomedir(final Path parentRegionDirectory, final byte[] family) {
387     return new Path(parentRegionDirectory, new Path(Bytes.toString(family)));
388   }
389 
390   /**
391    * Return the directory in which this store stores its
392    * StoreFiles
393    */
394   Path getHomedir() {
395     return homedir;
396   }
397 
398   /**
399    * @return the data block encoder
400    */
401   public HFileDataBlockEncoder getDataBlockEncoder() {
402     return dataBlockEncoder;
403   }
404 
405   /**
406    * Should be used only in tests.
407    * @param blockEncoder the block delta encoder to use
408    */
409   void setDataBlockEncoderInTest(HFileDataBlockEncoder blockEncoder) {
410     this.dataBlockEncoder = blockEncoder;
411   }
412 
413   FileStatus [] getStoreFiles() throws IOException {
414     return FSUtils.listStatus(this.fs, this.homedir, null);
415   }
416 
417   /**
418    * Creates an unsorted list of StoreFile loaded in parallel
419    * from the given directory.
420    * @throws IOException
421    */
422   private List<StoreFile> loadStoreFiles() throws IOException {
423     ArrayList<StoreFile> results = new ArrayList<StoreFile>();
424     FileStatus files[] = getStoreFiles();
425 
426     if (files == null || files.length == 0) {
427       return results;
428     }
429     // initialize the thread pool for opening store files in parallel..
430     ThreadPoolExecutor storeFileOpenerThreadPool =
431       this.region.getStoreFileOpenAndCloseThreadPool("StoreFileOpenerThread-" +
432           this.family.getNameAsString());
433     CompletionService<StoreFile> completionService =
434       new ExecutorCompletionService<StoreFile>(storeFileOpenerThreadPool);
435 
436     int totalValidStoreFile = 0;
437     for (int i = 0; i < files.length; i++) {
438       // Skip directories.
439       if (files[i].isDir()) {
440         continue;
441       }
442       final Path p = files[i].getPath();
443       // Check for empty hfile. Should never be the case but can happen
444       // after data loss in hdfs for whatever reason (upgrade, etc.): HBASE-646
445       // NOTE: that the HFileLink is just a name, so it's an empty file.
446       if (!HFileLink.isHFileLink(p) && this.fs.getFileStatus(p).getLen() <= 0) {
447         LOG.warn("Skipping " + p + " because its empty. HBASE-646 DATA LOSS?");
448         continue;
449       }
450 
451       // open each store file in parallel
452       completionService.submit(new Callable<StoreFile>() {
453         public StoreFile call() throws IOException {
454           StoreFile storeFile = new StoreFile(fs, p, conf, cacheConf,
455               family.getBloomFilterType(), dataBlockEncoder);
456           passSchemaMetricsTo(storeFile);
457           storeFile.createReader();
458           return storeFile;
459         }
460       });
461       totalValidStoreFile++;
462     }
463 
464     IOException ioe = null;
465     try {
466       for (int i = 0; i < totalValidStoreFile; i++) {
467         try {
468           Future<StoreFile> future = completionService.take();
469           StoreFile storeFile = future.get();
470           long length = storeFile.getReader().length();
471           this.storeSize += length;
472           this.totalUncompressedBytes +=
473               storeFile.getReader().getTotalUncompressedBytes();
474           if (LOG.isDebugEnabled()) {
475             LOG.debug("loaded " + storeFile.toStringDetailed());
476           }
477           results.add(storeFile);
478         } catch (InterruptedException e) {
479           if (ioe == null) ioe = new InterruptedIOException(e.getMessage());
480         } catch (ExecutionException e) {
481           if (ioe == null) ioe = new IOException(e.getCause());
482         } 
483       } 
484     } finally {
485       storeFileOpenerThreadPool.shutdownNow();
486     }
487     if (ioe != null) {
488       // close StoreFile readers
489       try {
490         for (StoreFile file : results) {
491           if (file != null) file.closeReader(true);
492         }
493       } catch (IOException e) { }
494       throw ioe;
495     }
496 
497     return results;
498   }
499 
500   /**
501    * Adds a value to the memstore
502    *
503    * @param kv
504    * @return memstore size delta
505    */
506   protected long add(final KeyValue kv) {
507     lock.readLock().lock();
508     try {
509       return this.memstore.add(kv);
510     } finally {
511       lock.readLock().unlock();
512     }
513   }
514 
515   /**
516    * When was the oldest edit done in the memstore
517    */
518   public long timeOfOldestEdit() {
519     return memstore.timeOfOldestEdit();
520   }
521 
522   /**
523    * Adds a value to the memstore
524    *
525    * @param kv
526    * @return memstore size delta
527    */
528   protected long delete(final KeyValue kv) {
529     lock.readLock().lock();
530     try {
531       return this.memstore.delete(kv);
532     } finally {
533       lock.readLock().unlock();
534     }
535   }
536 
537   /**
538    * Removes a kv from the memstore. The KeyValue is removed only
539    * if its key & memstoreTS matches the key & memstoreTS value of the
540    * kv parameter.
541    *
542    * @param kv
543    */
544   protected void rollback(final KeyValue kv) {
545     lock.readLock().lock();
546     try {
547       this.memstore.rollback(kv);
548     } finally {
549       lock.readLock().unlock();
550     }
551   }
552 
553   /**
554    * @return All store files.
555    */
556   public List<StoreFile> getStorefiles() {
557     return this.storefiles;
558   }
559 
560   /**
561    * This throws a WrongRegionException if the HFile does not fit in this
562    * region, or an InvalidHFileException if the HFile is not valid.
563    */
564   void assertBulkLoadHFileOk(Path srcPath) throws IOException {
565     HFile.Reader reader  = null;
566     try {
567       LOG.info("Validating hfile at " + srcPath + " for inclusion in "
568           + "store " + this + " region " + this.region);
569       reader = HFile.createReader(srcPath.getFileSystem(conf),
570           srcPath, cacheConf);
571       reader.loadFileInfo();
572 
573       byte[] firstKey = reader.getFirstRowKey();
574       byte[] lk = reader.getLastKey();
575       byte[] lastKey =
576           (lk == null) ? null :
577               KeyValue.createKeyValueFromKey(lk).getRow();
578 
579       LOG.debug("HFile bounds: first=" + Bytes.toStringBinary(firstKey) +
580           " last=" + Bytes.toStringBinary(lastKey));
581       LOG.debug("Region bounds: first=" +
582           Bytes.toStringBinary(region.getStartKey()) +
583           " last=" + Bytes.toStringBinary(region.getEndKey()));
584 
585       HRegionInfo hri = region.getRegionInfo();
586       if (!hri.containsRange(firstKey, lastKey)) {
587         throw new WrongRegionException(
588             "Bulk load file " + srcPath.toString() + " does not fit inside region "
589             + this.region);
590       }
591 
592       if (verifyBulkLoads) {
593         KeyValue prevKV = null;
594         HFileScanner scanner = reader.getScanner(false, false, false);
595         scanner.seekTo();
596         do {
597           KeyValue kv = scanner.getKeyValue();
598           if (prevKV != null) {
599             if (Bytes.compareTo(prevKV.getBuffer(), prevKV.getRowOffset(),
600                 prevKV.getRowLength(), kv.getBuffer(), kv.getRowOffset(),
601                 kv.getRowLength()) > 0) {
602               throw new InvalidHFileException("Previous row is greater than"
603                   + " current row: path=" + srcPath + " previous="
604                   + Bytes.toStringBinary(prevKV.getKey()) + " current="
605                   + Bytes.toStringBinary(kv.getKey()));
606             }
607             if (Bytes.compareTo(prevKV.getBuffer(), prevKV.getFamilyOffset(),
608                 prevKV.getFamilyLength(), kv.getBuffer(), kv.getFamilyOffset(),
609                 kv.getFamilyLength()) != 0) {
610               throw new InvalidHFileException("Previous key had different"
611                   + " family compared to current key: path=" + srcPath
612                   + " previous=" + Bytes.toStringBinary(prevKV.getFamily())
613                   + " current=" + Bytes.toStringBinary(kv.getFamily()));
614             }
615           }
616           prevKV = kv;
617         } while (scanner.next());
618       }
619     } finally {
620       if (reader != null) reader.close();
621     }
622   }
623 
624   /**
625    * This method should only be called from HRegion.  It is assumed that the
626    * ranges of values in the HFile fit within the stores assigned region.
627    * (assertBulkLoadHFileOk checks this)
628    */
629   void bulkLoadHFile(String srcPathStr) throws IOException {
630     Path srcPath = new Path(srcPathStr);
631 
632     // Move the file if it's on another filesystem
633     FileSystem srcFs = srcPath.getFileSystem(conf);
634     FileSystem desFs = fs instanceof HFileSystem ? ((HFileSystem)fs).getBackingFs() : fs;
635     //We can't compare FileSystem instances as
636     //equals() includes UGI instance as part of the comparison
637     //and won't work when doing SecureBulkLoad
638     //TODO deal with viewFS
639     if (!srcFs.getUri().equals(desFs.getUri())) {
640       LOG.info("File " + srcPath + " on different filesystem than " +
641           "destination store - moving to this filesystem.");
642       Path tmpPath = getTmpPath();
643       FileUtil.copy(srcFs, srcPath, fs, tmpPath, false, conf);
644       LOG.info("Copied to temporary path on dst filesystem: " + tmpPath);
645       srcPath = tmpPath;
646     }
647 
648     Path dstPath = StoreFile.getRandomFilename(fs, homedir);
649     LOG.debug("Renaming bulk load file " + srcPath + " to " + dstPath);
650     StoreFile.rename(fs, srcPath, dstPath);
651 
652     StoreFile sf = new StoreFile(fs, dstPath, this.conf, this.cacheConf,
653         this.family.getBloomFilterType(), this.dataBlockEncoder);
654     passSchemaMetricsTo(sf);
655 
656     StoreFile.Reader r = sf.createReader();
657     this.storeSize += r.length();
658     this.totalUncompressedBytes += r.getTotalUncompressedBytes();
659 
660     LOG.info("Moved hfile " + srcPath + " into store directory " +
661         homedir + " - updating store file list.");
662 
663     // Append the new storefile into the list
664     this.lock.writeLock().lock();
665     try {
666       ArrayList<StoreFile> newFiles = new ArrayList<StoreFile>(storefiles);
667       newFiles.add(sf);
668       this.storefiles = sortAndClone(newFiles);
669     } finally {
670       // We need the lock, as long as we are updating the storefiles
671       // or changing the memstore. Let us release it before calling
672       // notifyChangeReadersObservers. See HBASE-4485 for a possible
673       // deadlock scenario that could have happened if continue to hold
674       // the lock.
675       this.lock.writeLock().unlock();
676     }
677     notifyChangedReadersObservers();
678     LOG.info("Successfully loaded store file " + srcPath
679         + " into store " + this + " (new location: " + dstPath + ")");
680   }
681 
682   /**
683    * Get a temporary path in this region. These temporary files
684    * will get cleaned up when the region is re-opened if they are
685    * still around.
686    */
687   private Path getTmpPath() throws IOException {
688     return StoreFile.getRandomFilename(
689         fs, region.getTmpDir());
690   }
691 
692   /**
693    * Close all the readers
694    *
695    * We don't need to worry about subsequent requests because the HRegion holds
696    * a write lock that will prevent any more reads or writes.
697    *
698    * @throws IOException
699    */
700   ImmutableList<StoreFile> close() throws IOException {
701     this.lock.writeLock().lock();
702     try {
703       ImmutableList<StoreFile> result = storefiles;
704 
705       // Clear so metrics doesn't find them.
706       storefiles = ImmutableList.of();
707 
708       if (!result.isEmpty()) {
709         // initialize the thread pool for closing store files in parallel.
710         ThreadPoolExecutor storeFileCloserThreadPool = this.region
711             .getStoreFileOpenAndCloseThreadPool("StoreFileCloserThread-"
712                 + this.family.getNameAsString());
713 
714         // close each store file in parallel
715         CompletionService<Void> completionService =
716           new ExecutorCompletionService<Void>(storeFileCloserThreadPool);
717         for (final StoreFile f : result) {
718           completionService.submit(new Callable<Void>() {
719             public Void call() throws IOException {
720               f.closeReader(true);
721               return null;
722             }
723           });
724         }
725 
726         IOException ioe = null;
727         try {
728           for (int i = 0; i < result.size(); i++) {
729             try {
730               Future<Void> future = completionService.take();
731               future.get();
732             } catch (InterruptedException e) {
733               if (ioe == null) {
734                 ioe = new InterruptedIOException();
735                 ioe.initCause(e);
736               }
737             } catch (ExecutionException e) {
738               if (ioe == null) ioe = new IOException(e.getCause());
739             }
740           }
741         } finally {
742           storeFileCloserThreadPool.shutdownNow();
743         }
744         if (ioe != null) throw ioe;
745       }
746       LOG.info("Closed " + this);
747       return result;
748     } finally {
749       this.lock.writeLock().unlock();
750     }
751   }
752 
753   /**
754    * Snapshot this stores memstore.  Call before running
755    * {@link #flushCache(long, SortedSet<KeyValue>)} so it has some work to do.
756    */
757   void snapshot() {
758     this.memstore.snapshot();
759   }
760 
761   /**
762    * Write out current snapshot.  Presumes {@link #snapshot()} has been called
763    * previously.
764    * @param logCacheFlushId flush sequence number
765    * @param snapshot
766    * @param snapshotTimeRangeTracker
767    * @param flushedSize The number of bytes flushed
768    * @param status
769    * @return Path The path name of the tmp file to which the store was flushed
770    * @throws IOException
771    */
772   protected Path flushCache(final long logCacheFlushId,
773       SortedSet<KeyValue> snapshot,
774       TimeRangeTracker snapshotTimeRangeTracker,
775       AtomicLong flushedSize,
776       MonitoredTask status) throws IOException {
777     // If an exception happens flushing, we let it out without clearing
778     // the memstore snapshot.  The old snapshot will be returned when we say
779     // 'snapshot', the next time flush comes around.
780     // Retry after catching exception when flushing, otherwise server will abort
781     // itself
782     IOException lastException = null;
783     for (int i = 0; i < Store.flush_retries_number; i++) {
784       try {
785         Path pathName = internalFlushCache(snapshot, logCacheFlushId,
786             snapshotTimeRangeTracker, flushedSize, status);
787         try {
788           // Path name is null if there is no entry to flush
789           if (pathName != null) {
790             validateStoreFile(pathName);
791           }
792           return pathName;
793         } catch (Exception e) {
794           LOG.warn("Failed validating store file " + pathName
795               + ", retring num=" + i, e);
796           if (e instanceof IOException) {
797             lastException = (IOException) e;
798           } else {
799             lastException = new IOException(e);
800           }
801         }
802       } catch (IOException e) {
803         LOG.warn("Failed flushing store file, retring num=" + i, e);
804         lastException = e;
805       }
806       if (lastException != null) {
807         try {
808           Thread.sleep(pauseTime);
809         } catch (InterruptedException e) {
810           IOException iie = new InterruptedIOException();
811           iie.initCause(e);
812           throw iie;
813         }
814       }
815     }
816     throw lastException;
817   }
818 
819   /*
820    * @param cache
821    * @param logCacheFlushId
822    * @param snapshotTimeRangeTracker
823    * @param flushedSize The number of bytes flushed
824    * @return Path The path name of the tmp file to which the store was flushed
825    * @throws IOException
826    */
827   private Path internalFlushCache(final SortedSet<KeyValue> set,
828       final long logCacheFlushId,
829       TimeRangeTracker snapshotTimeRangeTracker,
830       AtomicLong flushedSize,
831       MonitoredTask status)
832       throws IOException {
833     StoreFile.Writer writer;
834     // Find the smallest read point across all the Scanners.
835     long smallestReadPoint = region.getSmallestReadPoint();
836     long flushed = 0;
837     Path pathName;
838     // Don't flush if there are no entries.
839     if (set.size() == 0) {
840       return null;
841     }
842     // Use a store scanner to find which rows to flush.
843     // Note that we need to retain deletes, hence
844     // treat this as a minor compaction.
845     InternalScanner scanner = null;
846     KeyValueScanner memstoreScanner = new CollectionBackedScanner(set, this.comparator);
847     if (getHRegion().getCoprocessorHost() != null) {
848       scanner = getHRegion().getCoprocessorHost().preFlushScannerOpen(this, memstoreScanner);
849     }
850     if (scanner == null) {
851       Scan scan = new Scan();
852       scan.setMaxVersions(scanInfo.getMaxVersions());
853       scanner = new StoreScanner(this, scanInfo, scan,
854           Collections.singletonList(memstoreScanner), ScanType.MINOR_COMPACT,
855           this.region.getSmallestReadPoint(), HConstants.OLDEST_TIMESTAMP);
856     }
857     if (getHRegion().getCoprocessorHost() != null) {
858       InternalScanner cpScanner =
859         getHRegion().getCoprocessorHost().preFlush(this, scanner);
860       // NULL scanner returned from coprocessor hooks means skip normal processing
861       if (cpScanner == null) {
862         return null;
863       }
864       scanner = cpScanner;
865     }
866     try {
867       int compactionKVMax = conf.getInt(HConstants.COMPACTION_KV_MAX, 10);
868       // TODO:  We can fail in the below block before we complete adding this
869       // flush to list of store files.  Add cleanup of anything put on filesystem
870       // if we fail.
871       synchronized (flushLock) {
872         status.setStatus("Flushing " + this + ": creating writer");
873         // A. Write the map out to the disk
874         writer = createWriterInTmp(set.size());
875         writer.setTimeRangeTracker(snapshotTimeRangeTracker);
876         pathName = writer.getPath();
877         try {
878           List<KeyValue> kvs = new ArrayList<KeyValue>();
879           boolean hasMore;
880           do {
881             hasMore = scanner.next(kvs, compactionKVMax);
882             if (!kvs.isEmpty()) {
883               for (KeyValue kv : kvs) {
884                 // If we know that this KV is going to be included always, then let us
885                 // set its memstoreTS to 0. This will help us save space when writing to disk.
886                 if (kv.getMemstoreTS() <= smallestReadPoint) {
887                   // let us not change the original KV. It could be in the memstore
888                   // changing its memstoreTS could affect other threads/scanners.
889                   kv = kv.shallowCopy();
890                   kv.setMemstoreTS(0);
891                 }
892                 writer.append(kv);
893                 flushed += this.memstore.heapSizeChange(kv, true);
894               }
895               kvs.clear();
896             }
897           } while (hasMore);
898         } finally {
899           // Write out the log sequence number that corresponds to this output
900           // hfile.  The hfile is current up to and including logCacheFlushId.
901           status.setStatus("Flushing " + this + ": appending metadata");
902           writer.appendMetadata(logCacheFlushId, false);
903           status.setStatus("Flushing " + this + ": closing flushed file");
904           writer.close();
905         }
906       }
907     } finally {
908       flushedSize.set(flushed);
909       scanner.close();
910     }
911     if (LOG.isInfoEnabled()) {
912       LOG.info("Flushed " +
913                ", sequenceid=" + logCacheFlushId +
914                ", memsize=" + StringUtils.humanReadableInt(flushed) +
915                ", into tmp file " + pathName);
916     }
917     return pathName;
918   }
919 
920   /*
921    * @param path The pathname of the tmp file into which the store was flushed
922    * @param logCacheFlushId
923    * @return StoreFile created.
924    * @throws IOException
925    */
926   private StoreFile commitFile(final Path path,
927       final long logCacheFlushId,
928       TimeRangeTracker snapshotTimeRangeTracker,
929       AtomicLong flushedSize,
930       MonitoredTask status)
931       throws IOException {
932     // Write-out finished successfully, move into the right spot
933     String fileName = path.getName();
934     Path dstPath = new Path(homedir, fileName);
935     String msg = "Renaming flushed file at " + path + " to " + dstPath;
936     LOG.debug(msg);
937     status.setStatus("Flushing " + this + ": " + msg);
938     if (!HBaseFileSystem.renameDirForFileSystem(fs, path, dstPath)) {
939       LOG.warn("Unable to rename " + path + " to " + dstPath);
940     }
941 
942     status.setStatus("Flushing " + this + ": reopening flushed file");
943     StoreFile sf = new StoreFile(this.fs, dstPath, this.conf, this.cacheConf,
944         this.family.getBloomFilterType(), this.dataBlockEncoder);
945     passSchemaMetricsTo(sf);
946 
947     StoreFile.Reader r = sf.createReader();
948     this.storeSize += r.length();
949     this.totalUncompressedBytes += r.getTotalUncompressedBytes();
950 
951     // This increments the metrics associated with total flushed bytes for this
952     // family. The overall flush count is stored in the static metrics and
953     // retrieved from HRegion.recentFlushes, which is set within
954     // HRegion.internalFlushcache, which indirectly calls this to actually do
955     // the flushing through the StoreFlusherImpl class
956     getSchemaMetrics().updatePersistentStoreMetric(
957         SchemaMetrics.StoreMetricType.FLUSH_SIZE, flushedSize.longValue());
958     if (LOG.isInfoEnabled()) {
959       LOG.info("Added " + sf + ", entries=" + r.getEntries() +
960         ", sequenceid=" + logCacheFlushId +
961         ", filesize=" + StringUtils.humanReadableInt(r.length()));
962     }
963     return sf;
964   }
965 
966   /*
967    * @param maxKeyCount
968    * @return Writer for a new StoreFile in the tmp dir.
969    */
970   private StoreFile.Writer createWriterInTmp(int maxKeyCount)
971   throws IOException {
972     return createWriterInTmp(maxKeyCount, this.family.getCompression(), false, true);
973   }
974 
975   /*
976    * @param maxKeyCount
977    * @param compression Compression algorithm to use
978    * @param isCompaction whether we are creating a new file in a compaction
979    * @return Writer for a new StoreFile in the tmp dir.
980    */
981   public StoreFile.Writer createWriterInTmp(int maxKeyCount,
982     Compression.Algorithm compression, boolean isCompaction, boolean includeMVCCReadpoint)
983   throws IOException {
984     final CacheConfig writerCacheConf;
985     if (isCompaction) {
986       // Don't cache data on write on compactions.
987       writerCacheConf = new CacheConfig(cacheConf);
988       writerCacheConf.setCacheDataOnWrite(false);
989     } else {
990       writerCacheConf = cacheConf;
991     }
992     StoreFile.Writer w = new StoreFile.WriterBuilder(conf, writerCacheConf,
993         fs, blocksize)
994             .withOutputDir(region.getTmpDir())
995             .withDataBlockEncoder(dataBlockEncoder)
996             .withComparator(comparator)
997             .withBloomType(family.getBloomFilterType())
998             .withMaxKeyCount(maxKeyCount)
999             .withChecksumType(checksumType)
1000             .withBytesPerChecksum(bytesPerChecksum)
1001             .withCompression(compression)
1002             .includeMVCCReadpoint(includeMVCCReadpoint)
1003             .build();
1004     // The store file writer's path does not include the CF name, so we need
1005     // to configure the HFile writer directly.
1006     SchemaConfigured sc = (SchemaConfigured) w.writer;
1007     SchemaConfigured.resetSchemaMetricsConf(sc);
1008     passSchemaMetricsTo(sc);
1009     return w;
1010   }
1011 
1012   /*
1013    * Change storefiles adding into place the Reader produced by this new flush.
1014    * @param sf
1015    * @param set That was used to make the passed file <code>p</code>.
1016    * @throws IOException
1017    * @return Whether compaction is required.
1018    */
1019   private boolean updateStorefiles(final StoreFile sf,
1020                                    final SortedSet<KeyValue> set)
1021   throws IOException {
1022     this.lock.writeLock().lock();
1023     try {
1024       ArrayList<StoreFile> newList = new ArrayList<StoreFile>(storefiles);
1025       newList.add(sf);
1026       storefiles = sortAndClone(newList);
1027 
1028       this.memstore.clearSnapshot(set);
1029     } finally {
1030       // We need the lock, as long as we are updating the storefiles
1031       // or changing the memstore. Let us release it before calling
1032       // notifyChangeReadersObservers. See HBASE-4485 for a possible
1033       // deadlock scenario that could have happened if continue to hold
1034       // the lock.
1035       this.lock.writeLock().unlock();
1036     }
1037 
1038     // Tell listeners of the change in readers.
1039     notifyChangedReadersObservers();
1040 
1041     return needsCompaction();
1042   }
1043 
1044   /*
1045    * Notify all observers that set of Readers has changed.
1046    * @throws IOException
1047    */
1048   private void notifyChangedReadersObservers() throws IOException {
1049     for (ChangedReadersObserver o: this.changedReaderObservers) {
1050       o.updateReaders();
1051     }
1052   }
1053 
1054   /**
1055    * Get all scanners with no filtering based on TTL (that happens further down
1056    * the line).
1057    * @return all scanners for this store
1058    */
1059   protected List<KeyValueScanner> getScanners(boolean cacheBlocks,
1060       boolean isGet,
1061       boolean isCompaction,
1062       ScanQueryMatcher matcher) throws IOException {
1063     List<StoreFile> storeFiles;
1064     List<KeyValueScanner> memStoreScanners;
1065     this.lock.readLock().lock();
1066     try {
1067       storeFiles = this.getStorefiles();
1068       memStoreScanners = this.memstore.getScanners();
1069     } finally {
1070       this.lock.readLock().unlock();
1071     }
1072 
1073     // First the store file scanners
1074 
1075     // TODO this used to get the store files in descending order,
1076     // but now we get them in ascending order, which I think is
1077     // actually more correct, since memstore get put at the end.
1078     List<StoreFileScanner> sfScanners = StoreFileScanner
1079       .getScannersForStoreFiles(storeFiles, cacheBlocks, isGet, isCompaction, matcher);
1080     List<KeyValueScanner> scanners =
1081       new ArrayList<KeyValueScanner>(sfScanners.size()+1);
1082     scanners.addAll(sfScanners);
1083     // Then the memstore scanners
1084     scanners.addAll(memStoreScanners);
1085     return scanners;
1086   }
1087 
1088   /*
1089    * @param o Observer who wants to know about changes in set of Readers
1090    */
1091   void addChangedReaderObserver(ChangedReadersObserver o) {
1092     this.changedReaderObservers.add(o);
1093   }
1094 
1095   /*
1096    * @param o Observer no longer interested in changes in set of Readers.
1097    */
1098   void deleteChangedReaderObserver(ChangedReadersObserver o) {
1099     // We don't check if observer present; it may not be (legitimately)
1100     this.changedReaderObservers.remove(o);
1101   }
1102 
1103   //////////////////////////////////////////////////////////////////////////////
1104   // Compaction
1105   //////////////////////////////////////////////////////////////////////////////
1106 
1107   /**
1108    * Compact the StoreFiles.  This method may take some time, so the calling
1109    * thread must be able to block for long periods.
1110    *
1111    * <p>During this time, the Store can work as usual, getting values from
1112    * StoreFiles and writing new StoreFiles from the memstore.
1113    *
1114    * Existing StoreFiles are not destroyed until the new compacted StoreFile is
1115    * completely written-out to disk.
1116    *
1117    * <p>The compactLock prevents multiple simultaneous compactions.
1118    * The structureLock prevents us from interfering with other write operations.
1119    *
1120    * <p>We don't want to hold the structureLock for the whole time, as a compact()
1121    * can be lengthy and we want to allow cache-flushes during this period.
1122    *
1123    * @param cr
1124    *          compaction details obtained from requestCompaction()
1125    * @throws IOException
1126    * @return Storefile we compacted into or null if we failed or opted out early.
1127    */
1128   StoreFile compact(CompactionRequest cr) throws IOException {
1129     if (cr == null || cr.getFiles().isEmpty()) return null;
1130     Preconditions.checkArgument(cr.getStore().toString().equals(this.toString()));
1131     List<StoreFile> filesToCompact = cr.getFiles();
1132     synchronized (filesCompacting) {
1133       // sanity check: we're compacting files that this store knows about
1134       // TODO: change this to LOG.error() after more debugging
1135       Preconditions.checkArgument(filesCompacting.containsAll(filesToCompact));
1136     }
1137 
1138     // Max-sequenceID is the last key in the files we're compacting
1139     long maxId = StoreFile.getMaxSequenceIdInList(filesToCompact);
1140 
1141     // Ready to go. Have list of files to compact.
1142     LOG.info("Starting compaction of " + filesToCompact.size() + " file(s) in "
1143         + this + " of "
1144         + this.region.getRegionInfo().getRegionNameAsString()
1145         + " into tmpdir=" + region.getTmpDir() + ", seqid=" + maxId + ", totalSize="
1146         + StringUtils.humanReadableInt(cr.getSize()));
1147 
1148     StoreFile sf = null;
1149     try {
1150       StoreFile.Writer writer = this.compactor.compact(cr, maxId);
1151       // Move the compaction into place.
1152       if (this.conf.getBoolean("hbase.hstore.compaction.complete", true)) {
1153         sf = completeCompaction(filesToCompact, writer);
1154         if (region.getCoprocessorHost() != null) {
1155           region.getCoprocessorHost().postCompact(this, sf, cr);
1156         }
1157       } else {
1158         // Create storefile around what we wrote with a reader on it.
1159         sf = new StoreFile(this.fs, writer.getPath(), this.conf, this.cacheConf,
1160           this.family.getBloomFilterType(), this.dataBlockEncoder);
1161         sf.createReader();
1162       }
1163     } finally {
1164       synchronized (filesCompacting) {
1165         filesCompacting.removeAll(filesToCompact);
1166       }
1167     }
1168 
1169     LOG.info("Completed" + (cr.isMajor() ? " major " : " ") + "compaction of "
1170         + filesToCompact.size() + " file(s) in " + this + " of "
1171         + this.region.getRegionInfo().getRegionNameAsString()
1172         + " into " +
1173         (sf == null ? "none" : sf.getPath().getName()) +
1174         ", size=" + (sf == null ? "none" :
1175           StringUtils.humanReadableInt(sf.getReader().length()))
1176         + "; total size for store is "
1177         + StringUtils.humanReadableInt(storeSize));
1178     return sf;
1179   }
1180 
1181   /**
1182    * Compact the most recent N files. Used in testing.
1183    */
1184   public void compactRecentForTesting(int N) throws IOException {
1185     List<StoreFile> filesToCompact;
1186     long maxId;
1187     boolean isMajor;
1188 
1189     this.lock.readLock().lock();
1190     try {
1191       synchronized (filesCompacting) {
1192         filesToCompact = Lists.newArrayList(storefiles);
1193         if (!filesCompacting.isEmpty()) {
1194           // exclude all files older than the newest file we're currently
1195           // compacting. this allows us to preserve contiguity (HBASE-2856)
1196           StoreFile last = filesCompacting.get(filesCompacting.size() - 1);
1197           int idx = filesToCompact.indexOf(last);
1198           Preconditions.checkArgument(idx != -1);
1199           filesToCompact.subList(0, idx + 1).clear();
1200         }
1201         int count = filesToCompact.size();
1202         if (N > count) {
1203           throw new RuntimeException("Not enough files");
1204         }
1205 
1206         filesToCompact = filesToCompact.subList(count - N, count);
1207         maxId = StoreFile.getMaxSequenceIdInList(filesToCompact);
1208         isMajor = (filesToCompact.size() == storefiles.size());
1209         filesCompacting.addAll(filesToCompact);
1210         Collections.sort(filesCompacting, StoreFile.Comparators.FLUSH_TIME);
1211       }
1212     } finally {
1213       this.lock.readLock().unlock();
1214     }
1215 
1216     try {
1217       // Ready to go. Have list of files to compact.
1218       StoreFile.Writer writer = this.compactor.compactForTesting(this, conf, filesToCompact,
1219         isMajor, maxId);
1220       // Move the compaction into place.
1221       StoreFile sf = completeCompaction(filesToCompact, writer);
1222       if (region.getCoprocessorHost() != null) {
1223         region.getCoprocessorHost().postCompact(this, sf, null);
1224       }
1225     } finally {
1226       synchronized (filesCompacting) {
1227         filesCompacting.removeAll(filesToCompact);
1228       }
1229     }
1230   }
1231 
1232   boolean hasReferences() {
1233     return hasReferences(this.storefiles);
1234   }
1235 
1236   /*
1237    * @param files
1238    * @return True if any of the files in <code>files</code> are References.
1239    */
1240   private boolean hasReferences(Collection<StoreFile> files) {
1241     if (files != null && files.size() > 0) {
1242       for (StoreFile hsf: files) {
1243         if (hsf.isReference()) {
1244           return true;
1245         }
1246       }
1247     }
1248     return false;
1249   }
1250 
1251   /*
1252    * Gets lowest timestamp from candidate StoreFiles
1253    *
1254    * @param fs
1255    * @param dir
1256    * @throws IOException
1257    */
1258   public static long getLowestTimestamp(final List<StoreFile> candidates)
1259       throws IOException {
1260     long minTs = Long.MAX_VALUE;
1261     for (StoreFile storeFile : candidates) {
1262       minTs = Math.min(minTs, storeFile.getModificationTimeStamp());
1263     }
1264     return minTs;
1265   }
1266 
1267   /** getter for CompactionProgress object
1268    * @return CompactionProgress object; can be null
1269    */
1270   public CompactionProgress getCompactionProgress() {
1271     return this.compactor.getProgress();
1272   }
1273 
1274   /*
1275    * @return True if we should run a major compaction.
1276    */
1277   boolean isMajorCompaction() throws IOException {
1278     for (StoreFile sf : this.storefiles) {
1279       if (sf.getReader() == null) {
1280         LOG.debug("StoreFile " + sf + " has null Reader");
1281         return false;
1282       }
1283     }
1284 
1285     List<StoreFile> candidates = new ArrayList<StoreFile>(this.storefiles);
1286 
1287     // exclude files above the max compaction threshold
1288     // except: save all references. we MUST compact them
1289     int pos = 0;
1290     while (pos < candidates.size() &&
1291            candidates.get(pos).getReader().length() > this.maxCompactSize &&
1292            !candidates.get(pos).isReference()) ++pos;
1293     candidates.subList(0, pos).clear();
1294 
1295     return isMajorCompaction(candidates);
1296   }
1297 
1298   /*
1299    * @param filesToCompact Files to compact. Can be null.
1300    * @return True if we should run a major compaction.
1301    */
1302   private boolean isMajorCompaction(final List<StoreFile> filesToCompact) throws IOException {
1303     boolean result = false;
1304     long mcTime = getNextMajorCompactTime();
1305     if (filesToCompact == null || filesToCompact.isEmpty() || mcTime == 0) {
1306       return result;
1307     }
1308     // TODO: Use better method for determining stamp of last major (HBASE-2990)
1309     long lowTimestamp = getLowestTimestamp(filesToCompact);
1310     long now = System.currentTimeMillis();
1311     if (lowTimestamp > 0l && lowTimestamp < (now - mcTime)) {
1312       // Major compaction time has elapsed.
1313       if (filesToCompact.size() == 1) {
1314         // Single file
1315         StoreFile sf = filesToCompact.get(0);
1316         long oldest =
1317             (sf.getReader().timeRangeTracker == null) ?
1318                 Long.MIN_VALUE :
1319                 now - sf.getReader().timeRangeTracker.minimumTimestamp;
1320         if (sf.isMajorCompaction() &&
1321             (this.ttl == HConstants.FOREVER || oldest < this.ttl)) {
1322           if (LOG.isDebugEnabled()) {
1323             LOG.debug("Skipping major compaction of " + this +
1324                 " because one (major) compacted file only and oldestTime " +
1325                 oldest + "ms is < ttl=" + this.ttl);
1326           }
1327         } else if (this.ttl != HConstants.FOREVER && oldest > this.ttl) {
1328           LOG.debug("Major compaction triggered on store " + this +
1329             ", because keyvalues outdated; time since last major compaction " +
1330             (now - lowTimestamp) + "ms");
1331           result = true;
1332         }
1333       } else {
1334         if (LOG.isDebugEnabled()) {
1335           LOG.debug("Major compaction triggered on store " + this +
1336               "; time since last major compaction " + (now - lowTimestamp) + "ms");
1337         }
1338         result = true;
1339       }
1340     }
1341     return result;
1342   }
1343 
1344   long getNextMajorCompactTime() {
1345     // default = 24hrs
1346     long ret = conf.getLong(HConstants.MAJOR_COMPACTION_PERIOD, 1000*60*60*24);
1347     if (family.getValue(HConstants.MAJOR_COMPACTION_PERIOD) != null) {
1348       String strCompactionTime =
1349         family.getValue(HConstants.MAJOR_COMPACTION_PERIOD);
1350       ret = (new Long(strCompactionTime)).longValue();
1351     }
1352 
1353     if (ret > 0) {
1354       // default = 20% = +/- 4.8 hrs
1355       double jitterPct =  conf.getFloat("hbase.hregion.majorcompaction.jitter",
1356           0.20F);
1357       if (jitterPct > 0) {
1358         long jitter = Math.round(ret * jitterPct);
1359         // deterministic jitter avoids a major compaction storm on restart
1360         ImmutableList<StoreFile> snapshot = storefiles;
1361         if (snapshot != null && !snapshot.isEmpty()) {
1362           String seed = snapshot.get(0).getPath().getName();
1363           double curRand = new Random(seed.hashCode()).nextDouble();
1364           ret += jitter - Math.round(2L * jitter * curRand);
1365         } else {
1366           ret = 0; // no storefiles == no major compaction
1367         }
1368       }
1369     }
1370     return ret;
1371   }
1372 
1373   public CompactionRequest requestCompaction() throws IOException {
1374     return requestCompaction(NO_PRIORITY, null);
1375   }
1376 
1377   public CompactionRequest requestCompaction(int priority, CompactionRequest request)
1378       throws IOException {
1379     // don't even select for compaction if writes are disabled
1380     if (!this.region.areWritesEnabled()) {
1381       return null;
1382     }
1383 
1384     this.lock.readLock().lock();
1385     try {
1386       synchronized (filesCompacting) {
1387         // candidates = all storefiles not already in compaction queue
1388         List<StoreFile> candidates = Lists.newArrayList(storefiles);
1389         if (!filesCompacting.isEmpty()) {
1390           // exclude all files older than the newest file we're currently
1391           // compacting. this allows us to preserve contiguity (HBASE-2856)
1392           StoreFile last = filesCompacting.get(filesCompacting.size() - 1);
1393           int idx = candidates.indexOf(last);
1394           Preconditions.checkArgument(idx != -1);
1395           candidates.subList(0, idx + 1).clear();
1396         }
1397 
1398         boolean override = false;
1399         if (region.getCoprocessorHost() != null) {
1400           override = region.getCoprocessorHost().preCompactSelection(this, candidates, request);
1401         }
1402         CompactSelection filesToCompact;
1403         if (override) {
1404           // coprocessor is overriding normal file selection
1405           filesToCompact = new CompactSelection(conf, candidates);
1406         } else {
1407           filesToCompact = compactSelection(candidates, priority);
1408         }
1409 
1410         if (region.getCoprocessorHost() != null) {
1411           region.getCoprocessorHost().postCompactSelection(this,
1412             ImmutableList.copyOf(filesToCompact.getFilesToCompact()), request);
1413         }
1414 
1415         // no files to compact
1416         if (filesToCompact.getFilesToCompact().isEmpty()) {
1417           return null;
1418         }
1419 
1420         // basic sanity check: do not try to compact the same StoreFile twice.
1421         if (!Collections.disjoint(filesCompacting, filesToCompact.getFilesToCompact())) {
1422           // TODO: change this from an IAE to LOG.error after sufficient testing
1423           Preconditions.checkArgument(false, "%s overlaps with %s",
1424               filesToCompact, filesCompacting);
1425         }
1426         filesCompacting.addAll(filesToCompact.getFilesToCompact());
1427         Collections.sort(filesCompacting, StoreFile.Comparators.FLUSH_TIME);
1428 
1429         // major compaction iff all StoreFiles are included
1430         boolean isMajor = (filesToCompact.getFilesToCompact().size() == this.storefiles.size());
1431         if (isMajor) {
1432           // since we're enqueuing a major, update the compaction wait interval
1433           this.forceMajor = false;
1434         }
1435 
1436         // everything went better than expected. create a compaction request
1437         int pri = getCompactPriority(priority);
1438         //not a special compaction request, so we need to make one
1439         if(request == null){
1440           request = new CompactionRequest(region, this, filesToCompact, isMajor, pri);
1441         } else {
1442           // update the request with what the system thinks the request should be
1443           // its up to the request if it wants to listen
1444           request.setSelection(filesToCompact);
1445           request.setIsMajor(isMajor);
1446           request.setPriority(pri);
1447         }
1448       }
1449     } finally {
1450       this.lock.readLock().unlock();
1451     }
1452     if (request != null) {
1453       CompactionRequest.preRequest(request);
1454     }
1455     return request;
1456   }
1457 
1458   public void finishRequest(CompactionRequest cr) {
1459     CompactionRequest.postRequest(cr);
1460     cr.finishRequest();
1461     synchronized (filesCompacting) {
1462       filesCompacting.removeAll(cr.getFiles());
1463     }
1464   }
1465 
1466   /**
1467    * Algorithm to choose which files to compact, see {@link #compactSelection(java.util.List, int)}
1468    * @param candidates
1469    * @return
1470    * @throws IOException
1471    */
1472   CompactSelection compactSelection(List<StoreFile> candidates) throws IOException {
1473     return compactSelection(candidates,NO_PRIORITY);
1474   }
1475 
1476   /**
1477    * Algorithm to choose which files to compact
1478    *
1479    * Configuration knobs:
1480    *  "hbase.hstore.compaction.ratio"
1481    *    normal case: minor compact when file <= sum(smaller_files) * ratio
1482    *  "hbase.hstore.compaction.min.size"
1483    *    unconditionally compact individual files below this size
1484    *  "hbase.hstore.compaction.max.size"
1485    *    never compact individual files above this size (unless splitting)
1486    *  "hbase.hstore.compaction.min"
1487    *    min files needed to minor compact
1488    *  "hbase.hstore.compaction.max"
1489    *    max files to compact at once (avoids OOM)
1490    *
1491    * @param candidates candidate files, ordered from oldest to newest
1492    * @return subset copy of candidate list that meets compaction criteria
1493    * @throws IOException
1494    */
1495   CompactSelection compactSelection(List<StoreFile> candidates, int priority)
1496       throws IOException {
1497     // ASSUMPTION!!! filesCompacting is locked when calling this function
1498 
1499     /* normal skew:
1500      *
1501      *         older ----> newer
1502      *     _
1503      *    | |   _
1504      *    | |  | |   _
1505      *  --|-|- |-|- |-|---_-------_-------  minCompactSize
1506      *    | |  | |  | |  | |  _  | |
1507      *    | |  | |  | |  | | | | | |
1508      *    | |  | |  | |  | | | | | |
1509      */
1510     CompactSelection compactSelection = new CompactSelection(conf, candidates);
1511 
1512     boolean forcemajor = this.forceMajor && filesCompacting.isEmpty();
1513     if (!forcemajor) {
1514       // Delete the expired store files before the compaction selection.
1515       if (conf.getBoolean("hbase.store.delete.expired.storefile", true)
1516           && (ttl != Long.MAX_VALUE) && (this.scanInfo.minVersions == 0)) {
1517         CompactSelection expiredSelection = compactSelection
1518             .selectExpiredStoreFilesToCompact(
1519                 EnvironmentEdgeManager.currentTimeMillis() - this.ttl);
1520 
1521         // If there is any expired store files, delete them  by compaction.
1522         if (expiredSelection != null) {
1523           return expiredSelection;
1524         }
1525       }
1526       // do not compact old files above a configurable threshold
1527       // save all references. we MUST compact them
1528       int pos = 0;
1529       while (pos < compactSelection.getFilesToCompact().size() &&
1530              compactSelection.getFilesToCompact().get(pos).getReader().length()
1531                > maxCompactSize &&
1532              !compactSelection.getFilesToCompact().get(pos).isReference()) ++pos;
1533       if (pos != 0) compactSelection.clearSubList(0, pos);
1534     }
1535 
1536     if (compactSelection.getFilesToCompact().isEmpty()) {
1537       LOG.debug(this.getHRegionInfo().getEncodedName() + " - " +
1538         this + ": no store files to compact");
1539       compactSelection.emptyFileList();
1540       return compactSelection;
1541     }
1542 
1543     // Force a major compaction if this is a user-requested major compaction,
1544     // or if we do not have too many files to compact and this was requested
1545     // as a major compaction
1546     boolean majorcompaction = (forcemajor && priority == PRIORITY_USER) ||
1547       (forcemajor || isMajorCompaction(compactSelection.getFilesToCompact())) &&
1548       (compactSelection.getFilesToCompact().size() < this.maxFilesToCompact
1549     );
1550     LOG.debug(this.getHRegionInfo().getEncodedName() + " - " +
1551       this.getColumnFamilyName() + ": Initiating " +
1552       (majorcompaction ? "major" : "minor") + "compaction");
1553 
1554     if (!majorcompaction &&
1555         !hasReferences(compactSelection.getFilesToCompact())) {
1556 
1557       // remove bulk import files that request to be excluded from minors
1558       compactSelection.getFilesToCompact().removeAll(Collections2.filter(
1559           compactSelection.getFilesToCompact(),
1560           new Predicate<StoreFile>() {
1561             public boolean apply(StoreFile input) {
1562               return input.excludeFromMinorCompaction();
1563             }
1564           }));
1565 
1566       // skip selection algorithm if we don't have enough files
1567       if (compactSelection.getFilesToCompact().size() < this.minFilesToCompact) {
1568         if(LOG.isDebugEnabled()) {
1569           LOG.debug("Not compacting files because we only have " +
1570             compactSelection.getFilesToCompact().size() +
1571             " files ready for compaction.  Need " + this.minFilesToCompact + " to initiate.");
1572         }
1573         compactSelection.emptyFileList();
1574         return compactSelection;
1575       }
1576       if (conf.getBoolean("hbase.hstore.useExploringCompation", false)) {
1577         compactSelection = exploringCompactionSelection(compactSelection);
1578       } else {
1579         compactSelection = defaultCompactionSelection(compactSelection);
1580       }
1581     } else {
1582       if(majorcompaction) {
1583         if (compactSelection.getFilesToCompact().size() > this.maxFilesToCompact) {
1584           LOG.debug("Warning, compacting more than " + this.maxFilesToCompact +
1585             " files, probably because of a user-requested major compaction");
1586           if(priority != PRIORITY_USER) {
1587             LOG.error("Compacting more than max files on a non user-requested compaction");
1588           }
1589         }
1590       } else if (compactSelection.getFilesToCompact().size() > this.maxFilesToCompact) {
1591         // all files included in this compaction, up to max
1592         int pastMax = compactSelection.getFilesToCompact().size() - this.maxFilesToCompact;
1593         compactSelection.getFilesToCompact().subList(0, pastMax).clear();
1594       }
1595     }
1596     return compactSelection;
1597   }
1598 
1599   private CompactSelection defaultCompactionSelection(CompactSelection compactSelection) {
1600     // we're doing a minor compaction, let's see what files are applicable
1601     int start = 0;
1602 
1603     double r = compactSelection.getCompactSelectionRatio();
1604 
1605     // get store file sizes for incremental compacting selection.
1606     int countOfFiles = compactSelection.getFilesToCompact().size();
1607     long [] fileSizes = new long[countOfFiles];
1608     long [] sumSize = new long[countOfFiles];
1609     for (int i = countOfFiles-1; i >= 0; --i) {
1610       StoreFile file = compactSelection.getFilesToCompact().get(i);
1611       fileSizes[i] = file.getReader().length();
1612       // calculate the sum of fileSizes[i,i+maxFilesToCompact-1) for algo
1613       int tooFar = i + this.maxFilesToCompact - 1;
1614       sumSize[i] = fileSizes[i]
1615           + ((i+1    < countOfFiles) ? sumSize[i+1]      : 0)
1616           - ((tooFar < countOfFiles) ? fileSizes[tooFar] : 0);
1617     }
1618 
1619       /* Start at the oldest file and stop when you find the first file that
1620        * meets compaction criteria:
1621        *   (1) a recently-flushed, small file (i.e. <= minCompactSize)
1622        *      OR
1623        *   (2) within the compactRatio of sum(newer_files)
1624        * Given normal skew, any newer files will also meet this criteria
1625        *
1626        * Additional Note:
1627        * If fileSizes.size() >> maxFilesToCompact, we will recurse on
1628        * compact().  Consider the oldest files first to avoid a
1629        * situation where we always compact [end-threshold,end).  Then, the
1630        * last file becomes an aggregate of the previous compactions.
1631        */
1632     while(countOfFiles - start >= this.minFilesToCompact &&
1633         fileSizes[start] >
1634             Math.max(minCompactSize, (long)(sumSize[start+1] * r))) {
1635       ++start;
1636     }
1637     int end = Math.min(countOfFiles, start + this.maxFilesToCompact);
1638     long totalSize = fileSizes[start]
1639         + ((start+1 < countOfFiles) ? sumSize[start+1] : 0);
1640     compactSelection = compactSelection.getSubList(start, end);
1641 
1642     // if we don't have enough files to compact, just wait
1643     if (compactSelection.getFilesToCompact().size() < this.minFilesToCompact) {
1644       if (LOG.isDebugEnabled()) {
1645         LOG.debug("Skipped compaction of " + this
1646             + ".  Only " + (end - start) + " file(s) of size "
1647             + StringUtils.humanReadableInt(totalSize)
1648             + " have met compaction criteria.");
1649       }
1650       compactSelection.emptyFileList();
1651       return compactSelection;
1652     }
1653     return compactSelection;
1654   }
1655 
1656   private CompactSelection exploringCompactionSelection(CompactSelection compactSelection) {
1657 
1658     List<StoreFile> candidates = compactSelection.getFilesToCompact();
1659     int futureFiles = filesCompacting.isEmpty() ? 0 : 1;
1660     boolean mayBeStuck = (candidates.size() - filesCompacting.size() + futureFiles)
1661         >= blockingStoreFileCount;
1662     // Start off choosing nothing.
1663     List<StoreFile> bestSelection = new ArrayList<StoreFile>(0);
1664     List<StoreFile> smallest = new ArrayList<StoreFile>(0);
1665     long bestSize = 0;
1666     long smallestSize = Long.MAX_VALUE;
1667     double r = compactSelection.getCompactSelectionRatio();
1668 
1669     // Consider every starting place.
1670     for (int startIndex = 0; startIndex < candidates.size(); startIndex++) {
1671       // Consider every different sub list permutation in between start and end with min files.
1672       for (int currentEnd = startIndex + minFilesToCompact - 1;
1673            currentEnd < candidates.size(); currentEnd++) {
1674         List<StoreFile> potentialMatchFiles = candidates.subList(startIndex, currentEnd + 1);
1675 
1676         // Sanity checks
1677         if (potentialMatchFiles.size() < minFilesToCompact) {
1678           continue;
1679         }
1680         if (potentialMatchFiles.size() > maxFilesToCompact) {
1681           continue;
1682         }
1683 
1684         // Compute the total size of files that will
1685         // have to be read if this set of files is compacted.
1686         long size = getCompactionSize(potentialMatchFiles);
1687 
1688         // Store the smallest set of files.  This stored set of files will be used
1689         // if it looks like the algorithm is stuck.
1690         if (size < smallestSize) {
1691           smallest = potentialMatchFiles;
1692           smallestSize = size;
1693         }
1694 
1695         if (size >= minCompactSize
1696             && !filesInRatio(potentialMatchFiles, r)) {
1697           continue;
1698         }
1699 
1700         if (size > maxCompactSize) {
1701           continue;
1702         }
1703 
1704         // Keep if this gets rid of more files.  Or the same number of files for less io.
1705         if (potentialMatchFiles.size() > bestSelection.size()
1706             || (potentialMatchFiles.size() == bestSelection.size() && size < bestSize)) {
1707           bestSelection = potentialMatchFiles;
1708           bestSize = size;
1709         }
1710       }
1711     }
1712 
1713     if (bestSelection.size() == 0 && mayBeStuck) {
1714       smallest = new ArrayList<StoreFile>(smallest);
1715       compactSelection.getFilesToCompact().clear();
1716       compactSelection.getFilesToCompact().addAll(smallest);
1717     } else {
1718       bestSelection = new ArrayList<StoreFile>(bestSelection);
1719       compactSelection.getFilesToCompact().clear();
1720       compactSelection.getFilesToCompact().addAll(bestSelection);
1721     }
1722 
1723     return compactSelection;
1724 
1725   }
1726 
1727   /**
1728    * Check that all files satisfy the ratio
1729    *
1730    * @param files set of files to examine.
1731    * @param currentRatio The raio
1732    * @return if all files are in ratio.
1733    */
1734   private boolean filesInRatio(final List<StoreFile> files, final double currentRatio) {
1735     if (files.size() < 2) {
1736       return true;
1737     }
1738     long totalFileSize = 0;
1739     for (int i = 0; i < files.size(); i++) {
1740       totalFileSize += files.get(i).getReader().length();
1741     }
1742     for (int i = 0; i < files.size(); i++) {
1743       long singleFileSize = files.get(i).getReader().length();
1744       long sumAllOtherFilesize = totalFileSize - singleFileSize;
1745 
1746       if ((singleFileSize > sumAllOtherFilesize * currentRatio)
1747           && (sumAllOtherFilesize >= this.minCompactSize)) {
1748         return false;
1749       }
1750     }
1751     return true;
1752   }
1753 
1754   /**
1755    * Get the number of bytes a proposed compaction would have to read.
1756    *
1757    * @param files Set of files in a proposed compaction.
1758    * @return size in bytes.
1759    */
1760   private long getCompactionSize(final List<StoreFile> files) {
1761     long size = 0;
1762     if (files == null) {
1763       return size;
1764     }
1765     for (StoreFile f : files) {
1766       size += f.getReader().length();
1767     }
1768     return size;
1769   }
1770 
1771   /**
1772    * Validates a store file by opening and closing it. In HFileV2 this should
1773    * not be an expensive operation.
1774    *
1775    * @param path the path to the store file
1776    */
1777   private void validateStoreFile(Path path)
1778       throws IOException {
1779     StoreFile storeFile = null;
1780     try {
1781       storeFile = new StoreFile(this.fs, path, this.conf,
1782           this.cacheConf, this.family.getBloomFilterType(),
1783           NoOpDataBlockEncoder.INSTANCE);
1784       passSchemaMetricsTo(storeFile);
1785       storeFile.createReader();
1786     } catch (IOException e) {
1787       LOG.error("Failed to open store file : " + path
1788           + ", keeping it in tmp location", e);
1789       throw e;
1790     } finally {
1791       if (storeFile != null) {
1792         storeFile.closeReader(false);
1793       }
1794     }
1795   }
1796 
1797   /*
1798    * <p>It works by processing a compaction that's been written to disk.
1799    *
1800    * <p>It is usually invoked at the end of a compaction, but might also be
1801    * invoked at HStore startup, if the prior execution died midway through.
1802    *
1803    * <p>Moving the compacted TreeMap into place means:
1804    * <pre>
1805    * 1) Moving the new compacted StoreFile into place
1806    * 2) Unload all replaced StoreFile, close and collect list to delete.
1807    * 3) Loading the new TreeMap.
1808    * 4) Compute new store size
1809    * </pre>
1810    *
1811    * @param compactedFiles list of files that were compacted
1812    * @param compactedFile StoreFile that is the result of the compaction
1813    * @return StoreFile created. May be null.
1814    * @throws IOException
1815    */
1816   StoreFile completeCompaction(final Collection<StoreFile> compactedFiles,
1817                                        final StoreFile.Writer compactedFile)
1818       throws IOException {
1819     // 1. Moving the new files into place -- if there is a new file (may not
1820     // be if all cells were expired or deleted).
1821     StoreFile result = null;
1822     if (compactedFile != null) {
1823       validateStoreFile(compactedFile.getPath());
1824       // Move the file into the right spot
1825       Path origPath = compactedFile.getPath();
1826       Path destPath = new Path(homedir, origPath.getName());
1827       LOG.info("Renaming compacted file at " + origPath + " to " + destPath);
1828       if (!HBaseFileSystem.renameDirForFileSystem(fs, origPath, destPath)) {
1829         LOG.error("Failed move of compacted file " + origPath + " to " +
1830             destPath);
1831         throw new IOException("Failed move of compacted file " + origPath +
1832             " to " + destPath);
1833       }
1834       result = new StoreFile(this.fs, destPath, this.conf, this.cacheConf,
1835           this.family.getBloomFilterType(), this.dataBlockEncoder);
1836       passSchemaMetricsTo(result);
1837       result.createReader();
1838     }
1839     try {
1840       this.lock.writeLock().lock();
1841       try {
1842         // Change this.storefiles so it reflects new state but do not
1843         // delete old store files until we have sent out notification of
1844         // change in case old files are still being accessed by outstanding
1845         // scanners.
1846         ArrayList<StoreFile> newStoreFiles = Lists.newArrayList(storefiles);
1847         newStoreFiles.removeAll(compactedFiles);
1848         filesCompacting.removeAll(compactedFiles); // safe bc: lock.writeLock()
1849 
1850         // If a StoreFile result, move it into place.  May be null.
1851         if (result != null) {
1852           newStoreFiles.add(result);
1853         }
1854 
1855         this.storefiles = sortAndClone(newStoreFiles);
1856       } finally {
1857         // We need the lock, as long as we are updating the storefiles
1858         // or changing the memstore. Let us release it before calling
1859         // notifyChangeReadersObservers. See HBASE-4485 for a possible
1860         // deadlock scenario that could have happened if continue to hold
1861         // the lock.
1862         this.lock.writeLock().unlock();
1863       }
1864 
1865       // Tell observers that list of StoreFiles has changed.
1866       notifyChangedReadersObservers();
1867 
1868       // let the archive util decide if we should archive or delete the files
1869       LOG.debug("Removing store files after compaction...");
1870       HFileArchiver.archiveStoreFiles(this.conf, this.fs, this.region, this.family.getName(),
1871         compactedFiles);
1872 
1873     } catch (IOException e) {
1874       e = RemoteExceptionHandler.checkIOException(e);
1875       LOG.error("Failed replacing compacted files in " + this +
1876         ". Compacted file is " + (result == null? "none": result.toString()) +
1877         ".  Files replaced " + compactedFiles.toString() +
1878         " some of which may have been already removed", e);
1879     }
1880 
1881     // 4. Compute new store size
1882     this.storeSize = 0L;
1883     this.totalUncompressedBytes = 0L;
1884     for (StoreFile hsf : this.storefiles) {
1885       StoreFile.Reader r = hsf.getReader();
1886       if (r == null) {
1887         LOG.warn("StoreFile " + hsf + " has a null Reader");
1888         continue;
1889       }
1890       this.storeSize += r.length();
1891       this.totalUncompressedBytes += r.getTotalUncompressedBytes();
1892     }
1893     return result;
1894   }
1895 
1896   public ImmutableList<StoreFile> sortAndClone(List<StoreFile> storeFiles) {
1897     Collections.sort(storeFiles, StoreFile.Comparators.FLUSH_TIME);
1898     ImmutableList<StoreFile> newList = ImmutableList.copyOf(storeFiles);
1899     return newList;
1900   }
1901 
1902   // ////////////////////////////////////////////////////////////////////////////
1903   // Accessors.
1904   // (This is the only section that is directly useful!)
1905   //////////////////////////////////////////////////////////////////////////////
1906   /**
1907    * @return the number of files in this store
1908    */
1909   public int getNumberOfStoreFiles() {
1910     return this.storefiles.size();
1911   }
1912 
1913   /*
1914    * @param wantedVersions How many versions were asked for.
1915    * @return wantedVersions or this families' {@link HConstants#VERSIONS}.
1916    */
1917   int versionsToReturn(final int wantedVersions) {
1918     if (wantedVersions <= 0) {
1919       throw new IllegalArgumentException("Number of versions must be > 0");
1920     }
1921     // Make sure we do not return more than maximum versions for this store.
1922     int maxVersions = this.family.getMaxVersions();
1923     return wantedVersions > maxVersions ? maxVersions: wantedVersions;
1924   }
1925 
1926   static boolean isExpired(final KeyValue key, final long oldestTimestamp) {
1927     return key.getTimestamp() < oldestTimestamp;
1928   }
1929 
1930   /**
1931    * Find the key that matches <i>row</i> exactly, or the one that immediately
1932    * precedes it. WARNING: Only use this method on a table where writes occur
1933    * with strictly increasing timestamps. This method assumes this pattern of
1934    * writes in order to make it reasonably performant.  Also our search is
1935    * dependent on the axiom that deletes are for cells that are in the container
1936    * that follows whether a memstore snapshot or a storefile, not for the
1937    * current container: i.e. we'll see deletes before we come across cells we
1938    * are to delete. Presumption is that the memstore#kvset is processed before
1939    * memstore#snapshot and so on.
1940    * @param row The row key of the targeted row.
1941    * @return Found keyvalue or null if none found.
1942    * @throws IOException
1943    */
1944   KeyValue getRowKeyAtOrBefore(final byte[] row) throws IOException {
1945     // If minVersions is set, we will not ignore expired KVs.
1946     // As we're only looking for the latest matches, that should be OK.
1947     // With minVersions > 0 we guarantee that any KV that has any version
1948     // at all (expired or not) has at least one version that will not expire.
1949     // Note that this method used to take a KeyValue as arguments. KeyValue
1950     // can be back-dated, a row key cannot.
1951     long ttlToUse = scanInfo.getMinVersions() > 0 ? Long.MAX_VALUE : this.ttl;
1952 
1953     KeyValue kv = new KeyValue(row, HConstants.LATEST_TIMESTAMP);
1954 
1955     GetClosestRowBeforeTracker state = new GetClosestRowBeforeTracker(
1956       this.comparator, kv, ttlToUse, this.region.getRegionInfo().isMetaRegion());
1957     this.lock.readLock().lock();
1958     try {
1959       // First go to the memstore.  Pick up deletes and candidates.
1960       this.memstore.getRowKeyAtOrBefore(state);
1961       // Check if match, if we got a candidate on the asked for 'kv' row.
1962       // Process each store file. Run through from newest to oldest.
1963       for (StoreFile sf : Lists.reverse(storefiles)) {
1964         // Update the candidate keys from the current map file
1965         rowAtOrBeforeFromStoreFile(sf, state);
1966       }
1967       return state.getCandidate();
1968     } finally {
1969       this.lock.readLock().unlock();
1970     }
1971   }
1972 
1973   /*
1974    * Check an individual MapFile for the row at or before a given row.
1975    * @param f
1976    * @param state
1977    * @throws IOException
1978    */
1979   private void rowAtOrBeforeFromStoreFile(final StoreFile f,
1980                                           final GetClosestRowBeforeTracker state)
1981       throws IOException {
1982     StoreFile.Reader r = f.getReader();
1983     if (r == null) {
1984       LOG.warn("StoreFile " + f + " has a null Reader");
1985       return;
1986     }
1987     if (r.getEntries() == 0) {
1988       LOG.warn("StoreFile " + f + " is a empty store file");
1989       return;
1990     }
1991     // TODO: Cache these keys rather than make each time?
1992     byte [] fk = r.getFirstKey();
1993     if (fk == null) return;
1994     KeyValue firstKV = KeyValue.createKeyValueFromKey(fk, 0, fk.length);
1995     byte [] lk = r.getLastKey();
1996     KeyValue lastKV = KeyValue.createKeyValueFromKey(lk, 0, lk.length);
1997     KeyValue firstOnRow = state.getTargetKey();
1998     if (this.comparator.compareRows(lastKV, firstOnRow) < 0) {
1999       // If last key in file is not of the target table, no candidates in this
2000       // file.  Return.
2001       if (!state.isTargetTable(lastKV)) return;
2002       // If the row we're looking for is past the end of file, set search key to
2003       // last key. TODO: Cache last and first key rather than make each time.
2004       firstOnRow = new KeyValue(lastKV.getRow(), HConstants.LATEST_TIMESTAMP);
2005     }
2006     // Get a scanner that caches blocks and that uses pread.
2007     HFileScanner scanner = r.getScanner(true, true, false);
2008     // Seek scanner.  If can't seek it, return.
2009     if (!seekToScanner(scanner, firstOnRow, firstKV)) return;
2010     // If we found candidate on firstOnRow, just return. THIS WILL NEVER HAPPEN!
2011     // Unlikely that there'll be an instance of actual first row in table.
2012     if (walkForwardInSingleRow(scanner, firstOnRow, state)) return;
2013     // If here, need to start backing up.
2014     while (scanner.seekBefore(firstOnRow.getBuffer(), firstOnRow.getKeyOffset(),
2015        firstOnRow.getKeyLength())) {
2016       KeyValue kv = scanner.getKeyValue();
2017       if (!state.isTargetTable(kv)) break;
2018       if (!state.isBetterCandidate(kv)) break;
2019       // Make new first on row.
2020       firstOnRow = new KeyValue(kv.getRow(), HConstants.LATEST_TIMESTAMP);
2021       // Seek scanner.  If can't seek it, break.
2022       if (!seekToScanner(scanner, firstOnRow, firstKV)) break;
2023       // If we find something, break;
2024       if (walkForwardInSingleRow(scanner, firstOnRow, state)) break;
2025     }
2026   }
2027 
2028   /*
2029    * Seek the file scanner to firstOnRow or first entry in file.
2030    * @param scanner
2031    * @param firstOnRow
2032    * @param firstKV
2033    * @return True if we successfully seeked scanner.
2034    * @throws IOException
2035    */
2036   private boolean seekToScanner(final HFileScanner scanner,
2037                                 final KeyValue firstOnRow,
2038                                 final KeyValue firstKV)
2039       throws IOException {
2040     KeyValue kv = firstOnRow;
2041     // If firstOnRow < firstKV, set to firstKV
2042     if (this.comparator.compareRows(firstKV, firstOnRow) == 0) kv = firstKV;
2043     int result = scanner.seekTo(kv.getBuffer(), kv.getKeyOffset(),
2044       kv.getKeyLength());
2045     return result >= 0;
2046   }
2047 
2048   /*
2049    * When we come in here, we are probably at the kv just before we break into
2050    * the row that firstOnRow is on.  Usually need to increment one time to get
2051    * on to the row we are interested in.
2052    * @param scanner
2053    * @param firstOnRow
2054    * @param state
2055    * @return True we found a candidate.
2056    * @throws IOException
2057    */
2058   private boolean walkForwardInSingleRow(final HFileScanner scanner,
2059                                          final KeyValue firstOnRow,
2060                                          final GetClosestRowBeforeTracker state)
2061       throws IOException {
2062     boolean foundCandidate = false;
2063     do {
2064       KeyValue kv = scanner.getKeyValue();
2065       // If we are not in the row, skip.
2066       if (this.comparator.compareRows(kv, firstOnRow) < 0) continue;
2067       // Did we go beyond the target row? If so break.
2068       if (state.isTooFar(kv, firstOnRow)) break;
2069       if (state.isExpired(kv)) {
2070         continue;
2071       }
2072       // If we added something, this row is a contender. break.
2073       if (state.handle(kv)) {
2074         foundCandidate = true;
2075         break;
2076       }
2077     } while(scanner.next());
2078     return foundCandidate;
2079   }
2080 
2081   public boolean canSplit() {
2082     this.lock.readLock().lock();
2083     try {
2084       // Not splitable if we find a reference store file present in the store.
2085       for (StoreFile sf : storefiles) {
2086         if (sf.isReference()) {
2087           if (LOG.isDebugEnabled()) {
2088             LOG.debug(sf + " is not splittable");
2089           }
2090           return false;
2091         }
2092       }
2093 
2094       return true;
2095     } finally {
2096       this.lock.readLock().unlock();
2097     }
2098   }
2099   /**
2100    * Determines if Store should be split
2101    * @return byte[] if store should be split, null otherwise.
2102    */
2103   public byte[] getSplitPoint() {
2104     this.lock.readLock().lock();
2105     try {
2106       // sanity checks
2107       if (this.storefiles.isEmpty()) {
2108         return null;
2109       }
2110       // Should already be enforced by the split policy!
2111       assert !this.region.getRegionInfo().isMetaRegion();
2112 
2113       // Not splitable if we find a reference store file present in the store.
2114       long maxSize = 0L;
2115       StoreFile largestSf = null;
2116       for (StoreFile sf : storefiles) {
2117         if (sf.isReference()) {
2118           // Should already be enforced since we return false in this case
2119           assert false : "getSplitPoint() called on a region that can't split!";
2120           return null;
2121         }
2122 
2123         StoreFile.Reader r = sf.getReader();
2124         if (r == null) {
2125           LOG.warn("Storefile " + sf + " Reader is null");
2126           continue;
2127         }
2128 
2129         long size = r.length();
2130         if (size > maxSize) {
2131           // This is the largest one so far
2132           maxSize = size;
2133           largestSf = sf;
2134         }
2135       }
2136 
2137       StoreFile.Reader r = largestSf.getReader();
2138       if (r == null) {
2139         LOG.warn("Storefile " + largestSf + " Reader is null");
2140         return null;
2141       }
2142       // Get first, last, and mid keys.  Midkey is the key that starts block
2143       // in middle of hfile.  Has column and timestamp.  Need to return just
2144       // the row we want to split on as midkey.
2145       byte [] midkey = r.midkey();
2146       if (midkey != null) {
2147         KeyValue mk = KeyValue.createKeyValueFromKey(midkey, 0, midkey.length);
2148         byte [] fk = r.getFirstKey();
2149         KeyValue firstKey = KeyValue.createKeyValueFromKey(fk, 0, fk.length);
2150         byte [] lk = r.getLastKey();
2151         KeyValue lastKey = KeyValue.createKeyValueFromKey(lk, 0, lk.length);
2152         // if the midkey is the same as the first or last keys, then we cannot
2153         // (ever) split this region.
2154         if (this.comparator.compareRows(mk, firstKey) == 0 ||
2155             this.comparator.compareRows(mk, lastKey) == 0) {
2156           if (LOG.isDebugEnabled()) {
2157             LOG.debug("cannot split because midkey is the same as first or " +
2158               "last row");
2159           }
2160           return null;
2161         }
2162         return mk.getRow();
2163       }
2164     } catch(IOException e) {
2165       LOG.warn("Failed getting store size for " + this, e);
2166     } finally {
2167       this.lock.readLock().unlock();
2168     }
2169     return null;
2170   }
2171 
2172   /** @return aggregate size of all HStores used in the last compaction */
2173   public long getLastCompactSize() {
2174     return this.lastCompactSize;
2175   }
2176 
2177   /** @return aggregate size of HStore */
2178   public long getSize() {
2179     return storeSize;
2180   }
2181 
2182   public void triggerMajorCompaction() {
2183     this.forceMajor = true;
2184   }
2185 
2186   boolean getForceMajorCompaction() {
2187     return this.forceMajor;
2188   }
2189 
2190   //////////////////////////////////////////////////////////////////////////////
2191   // File administration
2192   //////////////////////////////////////////////////////////////////////////////
2193 
2194   /**
2195    * Return a scanner for both the memstore and the HStore files. Assumes we
2196    * are not in a compaction.
2197    * @throws IOException
2198    */
2199   public KeyValueScanner getScanner(Scan scan,
2200       final NavigableSet<byte []> targetCols) throws IOException {
2201     lock.readLock().lock();
2202     try {
2203       KeyValueScanner scanner = null;
2204       if (getHRegion().getCoprocessorHost() != null) {
2205         scanner = getHRegion().getCoprocessorHost().preStoreScannerOpen(this, scan, targetCols);
2206       }
2207       if (scanner == null) {
2208         scanner = new StoreScanner(this, getScanInfo(), scan, targetCols);
2209       }
2210       return scanner;
2211     } finally {
2212       lock.readLock().unlock();
2213     }
2214   }
2215 
2216   @Override
2217   public String toString() {
2218     return getColumnFamilyName();
2219   }
2220 
2221   /**
2222    * @return Count of store files
2223    */
2224   int getStorefilesCount() {
2225     return this.storefiles.size();
2226   }
2227 
2228   /**
2229    * @return The size of the store files, in bytes, uncompressed.
2230    */
2231   long getStoreSizeUncompressed() {
2232     return this.totalUncompressedBytes;
2233   }
2234 
2235   /**
2236    * @return The size of the store files, in bytes.
2237    */
2238   long getStorefilesSize() {
2239     long size = 0;
2240     for (StoreFile s: storefiles) {
2241       StoreFile.Reader r = s.getReader();
2242       if (r == null) {
2243         LOG.warn("StoreFile " + s + " has a null Reader");
2244         continue;
2245       }
2246       size += r.length();
2247     }
2248     return size;
2249   }
2250 
2251   /**
2252    * @return The size of the store file indexes, in bytes.
2253    */
2254   long getStorefilesIndexSize() {
2255     long size = 0;
2256     for (StoreFile s: storefiles) {
2257       StoreFile.Reader r = s.getReader();
2258       if (r == null) {
2259         LOG.warn("StoreFile " + s + " has a null Reader");
2260         continue;
2261       }
2262       size += r.indexSize();
2263     }
2264     return size;
2265   }
2266 
2267   /**
2268    * Returns the total size of all index blocks in the data block indexes,
2269    * including the root level, intermediate levels, and the leaf level for
2270    * multi-level indexes, or just the root level for single-level indexes.
2271    *
2272    * @return the total size of block indexes in the store
2273    */
2274   long getTotalStaticIndexSize() {
2275     long size = 0;
2276     for (StoreFile s : storefiles) {
2277       size += s.getReader().getUncompressedDataIndexSize();
2278     }
2279     return size;
2280   }
2281 
2282   /**
2283    * Returns the total byte size of all Bloom filter bit arrays. For compound
2284    * Bloom filters even the Bloom blocks currently not loaded into the block
2285    * cache are counted.
2286    *
2287    * @return the total size of all Bloom filters in the store
2288    */
2289   long getTotalStaticBloomSize() {
2290     long size = 0;
2291     for (StoreFile s : storefiles) {
2292       StoreFile.Reader r = s.getReader();
2293       size += r.getTotalBloomSize();
2294     }
2295     return size;
2296   }
2297 
2298   /**
2299    * @return The size of this store's memstore, in bytes
2300    */
2301   long getMemStoreSize() {
2302     return this.memstore.heapSize();
2303   }
2304 
2305   public int getCompactPriority() {
2306     return getCompactPriority(NO_PRIORITY);
2307   }
2308 
2309   /**
2310    * @return The priority that this store should have in the compaction queue
2311    * @param priority
2312    */
2313   public int getCompactPriority(int priority) {
2314     // If this is a user-requested compaction, leave this at the highest priority
2315     if(priority == PRIORITY_USER) {
2316       return PRIORITY_USER;
2317     } else {
2318       return this.blockingStoreFileCount - this.storefiles.size();
2319     }
2320   }
2321 
2322   boolean throttleCompaction(long compactionSize) {
2323     long throttlePoint = conf.getLong(
2324         "hbase.regionserver.thread.compaction.throttle",
2325         2 * this.minFilesToCompact * this.region.memstoreFlushSize);
2326     return compactionSize > throttlePoint;
2327   }
2328 
2329   public HRegion getHRegion() {
2330     return this.region;
2331   }
2332 
2333   HRegionInfo getHRegionInfo() {
2334     return this.region.getRegionInfo();
2335   }
2336 
2337   /**
2338    * Increments the value for the given row/family/qualifier.
2339    *
2340    * This function will always be seen as atomic by other readers
2341    * because it only puts a single KV to memstore. Thus no
2342    * read/write control necessary.
2343    *
2344    * @param row
2345    * @param f
2346    * @param qualifier
2347    * @param newValue the new value to set into memstore
2348    * @return memstore size delta
2349    * @throws IOException
2350    */
2351   public long updateColumnValue(byte [] row, byte [] f,
2352                                 byte [] qualifier, long newValue)
2353       throws IOException {
2354 
2355     this.lock.readLock().lock();
2356     try {
2357       long now = EnvironmentEdgeManager.currentTimeMillis();
2358 
2359       return this.memstore.updateColumnValue(row,
2360           f,
2361           qualifier,
2362           newValue,
2363           now);
2364 
2365     } finally {
2366       this.lock.readLock().unlock();
2367     }
2368   }
2369 
2370   /**
2371    * Adds or replaces the specified KeyValues.
2372    * <p>
2373    * For each KeyValue specified, if a cell with the same row, family, and
2374    * qualifier exists in MemStore, it will be replaced.  Otherwise, it will just
2375    * be inserted to MemStore.
2376    * <p>
2377    * This operation is atomic on each KeyValue (row/family/qualifier) but not
2378    * necessarily atomic across all of them.
2379    * @param kvs
2380    * @return memstore size delta
2381    * @throws IOException
2382    */
2383   public long upsert(List<KeyValue> kvs)
2384       throws IOException {
2385     this.lock.readLock().lock();
2386     try {
2387       // TODO: Make this operation atomic w/ MVCC
2388       return this.memstore.upsert(kvs);
2389     } finally {
2390       this.lock.readLock().unlock();
2391     }
2392   }
2393 
2394   public StoreFlusher getStoreFlusher(long cacheFlushId) {
2395     return new StoreFlusherImpl(cacheFlushId);
2396   }
2397 
2398   private class StoreFlusherImpl implements StoreFlusher {
2399 
2400     private long cacheFlushId;
2401     private SortedSet<KeyValue> snapshot;
2402     private StoreFile storeFile;
2403     private Path storeFilePath;
2404     private TimeRangeTracker snapshotTimeRangeTracker;
2405     private AtomicLong flushedSize;
2406 
2407     private StoreFlusherImpl(long cacheFlushId) {
2408       this.cacheFlushId = cacheFlushId;
2409       this.flushedSize = new AtomicLong();
2410     }
2411 
2412     @Override
2413     public void prepare() {
2414       memstore.snapshot();
2415       this.snapshot = memstore.getSnapshot();
2416       this.snapshotTimeRangeTracker = memstore.getSnapshotTimeRangeTracker();
2417     }
2418 
2419     @Override
2420     public void flushCache(MonitoredTask status) throws IOException {
2421       storeFilePath = Store.this.flushCache(
2422         cacheFlushId, snapshot, snapshotTimeRangeTracker, flushedSize, status);
2423     }
2424 
2425     @Override
2426     public boolean commit(MonitoredTask status) throws IOException {
2427       if (storeFilePath == null) {
2428         return false;
2429       }
2430       storeFile = Store.this.commitFile(storeFilePath, cacheFlushId,
2431                                snapshotTimeRangeTracker, flushedSize, status);
2432       if (Store.this.getHRegion().getCoprocessorHost() != null) {
2433         Store.this.getHRegion()
2434             .getCoprocessorHost()
2435             .postFlush(Store.this, storeFile);
2436       }
2437 
2438       // Add new file to store files.  Clear snapshot too while we have
2439       // the Store write lock.
2440       return Store.this.updateStorefiles(storeFile, snapshot);
2441     }
2442   }
2443 
2444   /**
2445    * See if there's too much store files in this store
2446    * @return true if number of store files is greater than
2447    *  the number defined in minFilesToCompact
2448    */
2449   public boolean needsCompaction() {
2450     return (storefiles.size() - filesCompacting.size()) > minFilesToCompact;
2451   }
2452 
2453   /**
2454    * Used for tests. Get the cache configuration for this Store.
2455    */
2456   public CacheConfig getCacheConfig() {
2457     return this.cacheConf;
2458   }
2459 
2460   public static final long FIXED_OVERHEAD =
2461       ClassSize.align(SchemaConfigured.SCHEMA_CONFIGURED_UNALIGNED_HEAP_SIZE +
2462           + (17 * ClassSize.REFERENCE) + (7 * Bytes.SIZEOF_LONG)
2463           + (5 * Bytes.SIZEOF_INT) + Bytes.SIZEOF_BOOLEAN);
2464 
2465   public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD
2466       + ClassSize.OBJECT + ClassSize.REENTRANT_LOCK
2467       + ClassSize.CONCURRENT_SKIPLISTMAP
2468       + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + ClassSize.OBJECT
2469       + ScanInfo.FIXED_OVERHEAD);
2470 
2471   @Override
2472   public long heapSize() {
2473     return DEEP_OVERHEAD + this.memstore.heapSize();
2474   }
2475 
2476   public KeyValue.KVComparator getComparator() {
2477     return comparator;
2478   }
2479 
2480   public ScanInfo getScanInfo() {
2481     return scanInfo;
2482   }
2483   
2484   public boolean hasTooManyStoreFiles() {
2485     return getStorefilesCount() > this.blockingFileCount;
2486   }
2487 
2488   /**
2489    * Immutable information for scans over a store.
2490    */
2491   public static class ScanInfo {
2492     private byte[] family;
2493     private int minVersions;
2494     private int maxVersions;
2495     private long ttl;
2496     private boolean keepDeletedCells;
2497     private long timeToPurgeDeletes;
2498     private KVComparator comparator;
2499 
2500     public static final long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT
2501         + (2 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_INT)
2502         + Bytes.SIZEOF_LONG + Bytes.SIZEOF_BOOLEAN);
2503 
2504     /**
2505      * @param family {@link HColumnDescriptor} describing the column family
2506      * @param ttl Store's TTL (in ms)
2507      * @param timeToPurgeDeletes duration in ms after which a delete marker can
2508      *        be purged during a major compaction.
2509      * @param comparator The store's comparator
2510      */
2511     public ScanInfo(HColumnDescriptor family, long ttl, long timeToPurgeDeletes, KVComparator comparator) {
2512       this(family.getName(), family.getMinVersions(), family.getMaxVersions(), ttl, family
2513           .getKeepDeletedCells(), timeToPurgeDeletes, comparator);
2514     }
2515     /**
2516      * @param family Name of this store's column family
2517      * @param minVersions Store's MIN_VERSIONS setting
2518      * @param maxVersions Store's VERSIONS setting
2519      * @param ttl Store's TTL (in ms)
2520      * @param timeToPurgeDeletes duration in ms after which a delete marker can
2521      *        be purged during a major compaction.
2522      * @param keepDeletedCells Store's keepDeletedCells setting
2523      * @param comparator The store's comparator
2524      */
2525     public ScanInfo(byte[] family, int minVersions, int maxVersions, long ttl,
2526         boolean keepDeletedCells, long timeToPurgeDeletes,
2527         KVComparator comparator) {
2528 
2529       this.family = family;
2530       this.minVersions = minVersions;
2531       this.maxVersions = maxVersions;
2532       this.ttl = ttl;
2533       this.keepDeletedCells = keepDeletedCells;
2534       this.timeToPurgeDeletes = timeToPurgeDeletes;
2535       this.comparator = comparator;
2536     }
2537 
2538     public byte[] getFamily() {
2539       return family;
2540     }
2541 
2542     public int getMinVersions() {
2543       return minVersions;
2544     }
2545 
2546     public int getMaxVersions() {
2547       return maxVersions;
2548     }
2549 
2550     public long getTtl() {
2551       return ttl;
2552     }
2553 
2554     public boolean getKeepDeletedCells() {
2555       return keepDeletedCells;
2556     }
2557 
2558     public long getTimeToPurgeDeletes() {
2559       return timeToPurgeDeletes;
2560     }
2561 
2562     public KVComparator getComparator() {
2563       return comparator;
2564     }
2565   }
2566 
2567 }