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