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() {
351     return StoreFile.getMaxSequenceIdInList(this.getStorefiles());
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   void bulkLoadHFile(String srcPathStr) 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 = StoreFile.getRandomFilename(fs, homedir);
651     LOG.debug("Renaming bulk load file " + srcPath + " to " + dstPath);
652     StoreFile.rename(fs, srcPath, dstPath);
653 
654     StoreFile sf = new StoreFile(fs, dstPath, this.conf, this.cacheConf,
655         this.family.getBloomFilterType(), this.dataBlockEncoder);
656     passSchemaMetricsTo(sf);
657 
658     StoreFile.Reader r = sf.createReader();
659     this.storeSize += r.length();
660     this.totalUncompressedBytes += r.getTotalUncompressedBytes();
661 
662     LOG.info("Moved hfile " + srcPath + " into store directory " +
663         homedir + " - updating store file list.");
664 
665     // Append the new storefile into the list
666     this.lock.writeLock().lock();
667     try {
668       ArrayList<StoreFile> newFiles = new ArrayList<StoreFile>(storefiles);
669       newFiles.add(sf);
670       this.storefiles = sortAndClone(newFiles);
671     } finally {
672       // We need the lock, as long as we are updating the storefiles
673       // or changing the memstore. Let us release it before calling
674       // notifyChangeReadersObservers. See HBASE-4485 for a possible
675       // deadlock scenario that could have happened if continue to hold
676       // the lock.
677       this.lock.writeLock().unlock();
678     }
679     notifyChangedReadersObservers();
680     LOG.info("Successfully loaded store file " + srcPath
681         + " into store " + this + " (new location: " + dstPath + ")");
682   }
683 
684   /**
685    * Get a temporary path in this region. These temporary files
686    * will get cleaned up when the region is re-opened if they are
687    * still around.
688    */
689   private Path getTmpPath() throws IOException {
690     return StoreFile.getRandomFilename(
691         fs, region.getTmpDir());
692   }
693 
694   /**
695    * Close all the readers
696    *
697    * We don't need to worry about subsequent requests because the HRegion holds
698    * a write lock that will prevent any more reads or writes.
699    *
700    * @throws IOException
701    */
702   ImmutableList<StoreFile> close() throws IOException {
703     this.lock.writeLock().lock();
704     try {
705       ImmutableList<StoreFile> result = storefiles;
706 
707       // Clear so metrics doesn't find them.
708       storefiles = ImmutableList.of();
709 
710       if (!result.isEmpty()) {
711         // initialize the thread pool for closing store files in parallel.
712         ThreadPoolExecutor storeFileCloserThreadPool = this.region
713             .getStoreFileOpenAndCloseThreadPool("StoreFileCloserThread-"
714                 + this.family.getNameAsString());
715 
716         // close each store file in parallel
717         CompletionService<Void> completionService =
718           new ExecutorCompletionService<Void>(storeFileCloserThreadPool);
719         for (final StoreFile f : result) {
720           completionService.submit(new Callable<Void>() {
721             public Void call() throws IOException {
722               f.closeReader(true);
723               return null;
724             }
725           });
726         }
727 
728         IOException ioe = null;
729         try {
730           for (int i = 0; i < result.size(); i++) {
731             try {
732               Future<Void> future = completionService.take();
733               future.get();
734             } catch (InterruptedException e) {
735               if (ioe == null) {
736                 ioe = new InterruptedIOException();
737                 ioe.initCause(e);
738               }
739             } catch (ExecutionException e) {
740               if (ioe == null) ioe = new IOException(e.getCause());
741             }
742           }
743         } finally {
744           storeFileCloserThreadPool.shutdownNow();
745         }
746         if (ioe != null) throw ioe;
747       }
748       LOG.info("Closed " + this);
749       return result;
750     } finally {
751       this.lock.writeLock().unlock();
752     }
753   }
754 
755   /**
756    * Snapshot this stores memstore.  Call before running
757    * {@link #flushCache(long, SortedSet<KeyValue>)} so it has some work to do.
758    */
759   void snapshot() {
760     this.memstore.snapshot();
761   }
762 
763   /**
764    * Write out current snapshot.  Presumes {@link #snapshot()} has been called
765    * previously.
766    * @param logCacheFlushId flush sequence number
767    * @param snapshot
768    * @param snapshotTimeRangeTracker
769    * @param flushedSize The number of bytes flushed
770    * @param status
771    * @return Path The path name of the tmp file to which the store was flushed
772    * @throws IOException
773    */
774   protected Path flushCache(final long logCacheFlushId,
775       SortedSet<KeyValue> snapshot,
776       TimeRangeTracker snapshotTimeRangeTracker,
777       AtomicLong flushedSize,
778       MonitoredTask status) throws IOException {
779     // If an exception happens flushing, we let it out without clearing
780     // the memstore snapshot.  The old snapshot will be returned when we say
781     // 'snapshot', the next time flush comes around.
782     // Retry after catching exception when flushing, otherwise server will abort
783     // itself
784     IOException lastException = null;
785     for (int i = 0; i < Store.flush_retries_number; i++) {
786       try {
787         Path pathName = internalFlushCache(snapshot, logCacheFlushId,
788             snapshotTimeRangeTracker, flushedSize, status);
789         try {
790           // Path name is null if there is no entry to flush
791           if (pathName != null) {
792             validateStoreFile(pathName);
793           }
794           return pathName;
795         } catch (Exception e) {
796           LOG.warn("Failed validating store file " + pathName
797               + ", retring num=" + i, e);
798           if (e instanceof IOException) {
799             lastException = (IOException) e;
800           } else {
801             lastException = new IOException(e);
802           }
803         }
804       } catch (IOException e) {
805         LOG.warn("Failed flushing store file, retring num=" + i, e);
806         lastException = e;
807       }
808       if (lastException != null) {
809         try {
810           Thread.sleep(pauseTime);
811         } catch (InterruptedException e) {
812           IOException iie = new InterruptedIOException();
813           iie.initCause(e);
814           throw iie;
815         }
816       }
817     }
818     throw lastException;
819   }
820 
821   /*
822    * @param cache
823    * @param logCacheFlushId
824    * @param snapshotTimeRangeTracker
825    * @param flushedSize The number of bytes flushed
826    * @return Path The path name of the tmp file to which the store was flushed
827    * @throws IOException
828    */
829   private Path internalFlushCache(final SortedSet<KeyValue> set,
830       final long logCacheFlushId,
831       TimeRangeTracker snapshotTimeRangeTracker,
832       AtomicLong flushedSize,
833       MonitoredTask status)
834       throws IOException {
835     StoreFile.Writer writer;
836     // Find the smallest read point across all the Scanners.
837     long smallestReadPoint = region.getSmallestReadPoint();
838     long flushed = 0;
839     Path pathName;
840     // Don't flush if there are no entries.
841     if (set.size() == 0) {
842       return null;
843     }
844     // Use a store scanner to find which rows to flush.
845     // Note that we need to retain deletes, hence
846     // treat this as a minor compaction.
847     InternalScanner scanner = null;
848     KeyValueScanner memstoreScanner = new CollectionBackedScanner(set, this.comparator);
849     if (getHRegion().getCoprocessorHost() != null) {
850       scanner = getHRegion().getCoprocessorHost().preFlushScannerOpen(this, memstoreScanner);
851     }
852     if (scanner == null) {
853       Scan scan = new Scan();
854       scan.setMaxVersions(scanInfo.getMaxVersions());
855       scanner = new StoreScanner(this, scanInfo, scan,
856           Collections.singletonList(memstoreScanner), ScanType.MINOR_COMPACT,
857           this.region.getSmallestReadPoint(), HConstants.OLDEST_TIMESTAMP);
858     }
859     if (getHRegion().getCoprocessorHost() != null) {
860       InternalScanner cpScanner =
861         getHRegion().getCoprocessorHost().preFlush(this, scanner);
862       // NULL scanner returned from coprocessor hooks means skip normal processing
863       if (cpScanner == null) {
864         return null;
865       }
866       scanner = cpScanner;
867     }
868     try {
869       int compactionKVMax = conf.getInt(HConstants.COMPACTION_KV_MAX, 10);
870       // TODO:  We can fail in the below block before we complete adding this
871       // flush to list of store files.  Add cleanup of anything put on filesystem
872       // if we fail.
873       synchronized (flushLock) {
874         status.setStatus("Flushing " + this + ": creating writer");
875         // A. Write the map out to the disk
876         writer = createWriterInTmp(set.size());
877         writer.setTimeRangeTracker(snapshotTimeRangeTracker);
878         pathName = writer.getPath();
879         try {
880           List<KeyValue> kvs = new ArrayList<KeyValue>();
881           boolean hasMore;
882           do {
883             hasMore = scanner.next(kvs, compactionKVMax);
884             if (!kvs.isEmpty()) {
885               for (KeyValue kv : kvs) {
886                 // If we know that this KV is going to be included always, then let us
887                 // set its memstoreTS to 0. This will help us save space when writing to disk.
888                 if (kv.getMemstoreTS() <= smallestReadPoint) {
889                   // let us not change the original KV. It could be in the memstore
890                   // changing its memstoreTS could affect other threads/scanners.
891                   kv = kv.shallowCopy();
892                   kv.setMemstoreTS(0);
893                 }
894                 writer.append(kv);
895                 flushed += this.memstore.heapSizeChange(kv, true);
896               }
897               kvs.clear();
898             }
899           } while (hasMore);
900         } finally {
901           // Write out the log sequence number that corresponds to this output
902           // hfile.  The hfile is current up to and including logCacheFlushId.
903           status.setStatus("Flushing " + this + ": appending metadata");
904           writer.appendMetadata(logCacheFlushId, false);
905           status.setStatus("Flushing " + this + ": closing flushed file");
906           writer.close();
907         }
908       }
909     } finally {
910       flushedSize.set(flushed);
911       scanner.close();
912     }
913     if (LOG.isInfoEnabled()) {
914       LOG.info("Flushed " +
915                ", sequenceid=" + logCacheFlushId +
916                ", memsize=" + StringUtils.humanReadableInt(flushed) +
917                ", into tmp file " + pathName);
918     }
919     return pathName;
920   }
921 
922   /*
923    * @param path The pathname of the tmp file into which the store was flushed
924    * @param logCacheFlushId
925    * @return StoreFile created.
926    * @throws IOException
927    */
928   private StoreFile commitFile(final Path path,
929       final long logCacheFlushId,
930       TimeRangeTracker snapshotTimeRangeTracker,
931       AtomicLong flushedSize,
932       MonitoredTask status)
933       throws IOException {
934     // Write-out finished successfully, move into the right spot
935     String fileName = path.getName();
936     Path dstPath = new Path(homedir, fileName);
937     String msg = "Renaming flushed file at " + path + " to " + dstPath;
938     LOG.debug(msg);
939     status.setStatus("Flushing " + this + ": " + msg);
940     if (!HBaseFileSystem.renameDirForFileSystem(fs, path, dstPath)) {
941       LOG.warn("Unable to rename " + path + " to " + dstPath);
942     }
943 
944     status.setStatus("Flushing " + this + ": reopening flushed file");
945     StoreFile sf = new StoreFile(this.fs, dstPath, this.conf, this.cacheConf,
946         this.family.getBloomFilterType(), this.dataBlockEncoder);
947     passSchemaMetricsTo(sf);
948 
949     StoreFile.Reader r = sf.createReader();
950     this.storeSize += r.length();
951     this.totalUncompressedBytes += r.getTotalUncompressedBytes();
952 
953     // This increments the metrics associated with total flushed bytes for this
954     // family. The overall flush count is stored in the static metrics and
955     // retrieved from HRegion.recentFlushes, which is set within
956     // HRegion.internalFlushcache, which indirectly calls this to actually do
957     // the flushing through the StoreFlusherImpl class
958     getSchemaMetrics().updatePersistentStoreMetric(
959         SchemaMetrics.StoreMetricType.FLUSH_SIZE, flushedSize.longValue());
960     if (LOG.isInfoEnabled()) {
961       LOG.info("Added " + sf + ", entries=" + r.getEntries() +
962         ", sequenceid=" + logCacheFlushId +
963         ", filesize=" + StringUtils.humanReadableInt(r.length()));
964     }
965     return sf;
966   }
967 
968   /*
969    * @param maxKeyCount
970    * @return Writer for a new StoreFile in the tmp dir.
971    */
972   private StoreFile.Writer createWriterInTmp(int maxKeyCount)
973   throws IOException {
974     return createWriterInTmp(maxKeyCount, this.family.getCompression(), false, true);
975   }
976 
977   /*
978    * @param maxKeyCount
979    * @param compression Compression algorithm to use
980    * @param isCompaction whether we are creating a new file in a compaction
981    * @return Writer for a new StoreFile in the tmp dir.
982    */
983   public StoreFile.Writer createWriterInTmp(int maxKeyCount,
984     Compression.Algorithm compression, boolean isCompaction, boolean includeMVCCReadpoint)
985   throws IOException {
986     final CacheConfig writerCacheConf;
987     if (isCompaction) {
988       // Don't cache data on write on compactions.
989       writerCacheConf = new CacheConfig(cacheConf);
990       writerCacheConf.setCacheDataOnWrite(false);
991     } else {
992       writerCacheConf = cacheConf;
993     }
994     StoreFile.Writer w = new StoreFile.WriterBuilder(conf, writerCacheConf,
995         fs, blocksize)
996             .withOutputDir(region.getTmpDir())
997             .withDataBlockEncoder(dataBlockEncoder)
998             .withComparator(comparator)
999             .withBloomType(family.getBloomFilterType())
1000             .withMaxKeyCount(maxKeyCount)
1001             .withChecksumType(checksumType)
1002             .withBytesPerChecksum(bytesPerChecksum)
1003             .withCompression(compression)
1004             .includeMVCCReadpoint(includeMVCCReadpoint)
1005             .build();
1006     // The store file writer's path does not include the CF name, so we need
1007     // to configure the HFile writer directly.
1008     SchemaConfigured sc = (SchemaConfigured) w.writer;
1009     SchemaConfigured.resetSchemaMetricsConf(sc);
1010     passSchemaMetricsTo(sc);
1011     return w;
1012   }
1013 
1014   /*
1015    * Change storefiles adding into place the Reader produced by this new flush.
1016    * @param sf
1017    * @param set That was used to make the passed file <code>p</code>.
1018    * @throws IOException
1019    * @return Whether compaction is required.
1020    */
1021   private boolean updateStorefiles(final StoreFile sf,
1022                                    final SortedSet<KeyValue> set)
1023   throws IOException {
1024     this.lock.writeLock().lock();
1025     try {
1026       ArrayList<StoreFile> newList = new ArrayList<StoreFile>(storefiles);
1027       newList.add(sf);
1028       storefiles = sortAndClone(newList);
1029 
1030       this.memstore.clearSnapshot(set);
1031     } finally {
1032       // We need the lock, as long as we are updating the storefiles
1033       // or changing the memstore. Let us release it before calling
1034       // notifyChangeReadersObservers. See HBASE-4485 for a possible
1035       // deadlock scenario that could have happened if continue to hold
1036       // the lock.
1037       this.lock.writeLock().unlock();
1038     }
1039 
1040     // Tell listeners of the change in readers.
1041     notifyChangedReadersObservers();
1042 
1043     return needsCompaction();
1044   }
1045 
1046   /*
1047    * Notify all observers that set of Readers has changed.
1048    * @throws IOException
1049    */
1050   private void notifyChangedReadersObservers() throws IOException {
1051     for (ChangedReadersObserver o: this.changedReaderObservers) {
1052       o.updateReaders();
1053     }
1054   }
1055 
1056   /**
1057    * Get all scanners with no filtering based on TTL (that happens further down
1058    * the line).
1059    * @return all scanners for this store
1060    */
1061   protected List<KeyValueScanner> getScanners(boolean cacheBlocks,
1062       boolean isGet,
1063       boolean isCompaction,
1064       ScanQueryMatcher matcher) throws IOException {
1065     List<StoreFile> storeFiles;
1066     List<KeyValueScanner> memStoreScanners;
1067     this.lock.readLock().lock();
1068     try {
1069       storeFiles = this.getStorefiles();
1070       memStoreScanners = this.memstore.getScanners();
1071     } finally {
1072       this.lock.readLock().unlock();
1073     }
1074 
1075     // First the store file scanners
1076 
1077     // TODO this used to get the store files in descending order,
1078     // but now we get them in ascending order, which I think is
1079     // actually more correct, since memstore get put at the end.
1080     List<StoreFileScanner> sfScanners = StoreFileScanner
1081       .getScannersForStoreFiles(storeFiles, cacheBlocks, isGet, isCompaction, matcher);
1082     List<KeyValueScanner> scanners =
1083       new ArrayList<KeyValueScanner>(sfScanners.size()+1);
1084     scanners.addAll(sfScanners);
1085     // Then the memstore scanners
1086     scanners.addAll(memStoreScanners);
1087     return scanners;
1088   }
1089 
1090   /*
1091    * @param o Observer who wants to know about changes in set of Readers
1092    */
1093   void addChangedReaderObserver(ChangedReadersObserver o) {
1094     this.changedReaderObservers.add(o);
1095   }
1096 
1097   /*
1098    * @param o Observer no longer interested in changes in set of Readers.
1099    */
1100   void deleteChangedReaderObserver(ChangedReadersObserver o) {
1101     // We don't check if observer present; it may not be (legitimately)
1102     this.changedReaderObservers.remove(o);
1103   }
1104 
1105   //////////////////////////////////////////////////////////////////////////////
1106   // Compaction
1107   //////////////////////////////////////////////////////////////////////////////
1108 
1109   /**
1110    * Compact the StoreFiles.  This method may take some time, so the calling
1111    * thread must be able to block for long periods.
1112    *
1113    * <p>During this time, the Store can work as usual, getting values from
1114    * StoreFiles and writing new StoreFiles from the memstore.
1115    *
1116    * Existing StoreFiles are not destroyed until the new compacted StoreFile is
1117    * completely written-out to disk.
1118    *
1119    * <p>The compactLock prevents multiple simultaneous compactions.
1120    * The structureLock prevents us from interfering with other write operations.
1121    *
1122    * <p>We don't want to hold the structureLock for the whole time, as a compact()
1123    * can be lengthy and we want to allow cache-flushes during this period.
1124    *
1125    * @param cr
1126    *          compaction details obtained from requestCompaction()
1127    * @throws IOException
1128    * @return Storefile we compacted into or null if we failed or opted out early.
1129    */
1130   StoreFile compact(CompactionRequest cr) throws IOException {
1131     if (cr == null || cr.getFiles().isEmpty()) return null;
1132     Preconditions.checkArgument(cr.getStore().toString().equals(this.toString()));
1133     List<StoreFile> filesToCompact = cr.getFiles();
1134     synchronized (filesCompacting) {
1135       // sanity check: we're compacting files that this store knows about
1136       // TODO: change this to LOG.error() after more debugging
1137       Preconditions.checkArgument(filesCompacting.containsAll(filesToCompact));
1138     }
1139 
1140     // Max-sequenceID is the last key in the files we're compacting
1141     long maxId = StoreFile.getMaxSequenceIdInList(filesToCompact);
1142 
1143     // Ready to go. Have list of files to compact.
1144     LOG.info("Starting compaction of " + filesToCompact.size() + " file(s) in "
1145         + this + " of "
1146         + this.region.getRegionInfo().getRegionNameAsString()
1147         + " into tmpdir=" + region.getTmpDir() + ", seqid=" + maxId + ", totalSize="
1148         + StringUtils.humanReadableInt(cr.getSize()));
1149 
1150     StoreFile sf = null;
1151     try {
1152       StoreFile.Writer writer = this.compactor.compact(cr, maxId);
1153       // Move the compaction into place.
1154       if (this.conf.getBoolean("hbase.hstore.compaction.complete", true)) {
1155         sf = completeCompaction(filesToCompact, writer);
1156         if (region.getCoprocessorHost() != null) {
1157           region.getCoprocessorHost().postCompact(this, sf, cr);
1158         }
1159       } else {
1160         // Create storefile around what we wrote with a reader on it.
1161         sf = new StoreFile(this.fs, writer.getPath(), this.conf, this.cacheConf,
1162           this.family.getBloomFilterType(), this.dataBlockEncoder);
1163         sf.createReader();
1164       }
1165     } finally {
1166       synchronized (filesCompacting) {
1167         filesCompacting.removeAll(filesToCompact);
1168       }
1169     }
1170 
1171     LOG.info("Completed" + (cr.isMajor() ? " major " : " ") + "compaction of "
1172         + filesToCompact.size() + " file(s) in " + this + " of "
1173         + this.region.getRegionInfo().getRegionNameAsString()
1174         + " into " +
1175         (sf == null ? "none" : sf.getPath().getName()) +
1176         ", size=" + (sf == null ? "none" :
1177           StringUtils.humanReadableInt(sf.getReader().length()))
1178         + "; total size for store is "
1179         + StringUtils.humanReadableInt(storeSize));
1180     return sf;
1181   }
1182 
1183   /**
1184    * Compact the most recent N files. Used in testing.
1185    */
1186   public void compactRecentForTesting(int N) throws IOException {
1187     List<StoreFile> filesToCompact;
1188     long maxId;
1189     boolean isMajor;
1190 
1191     this.lock.readLock().lock();
1192     try {
1193       synchronized (filesCompacting) {
1194         filesToCompact = Lists.newArrayList(storefiles);
1195         if (!filesCompacting.isEmpty()) {
1196           // exclude all files older than the newest file we're currently
1197           // compacting. this allows us to preserve contiguity (HBASE-2856)
1198           StoreFile last = filesCompacting.get(filesCompacting.size() - 1);
1199           int idx = filesToCompact.indexOf(last);
1200           Preconditions.checkArgument(idx != -1);
1201           filesToCompact.subList(0, idx + 1).clear();
1202         }
1203         int count = filesToCompact.size();
1204         if (N > count) {
1205           throw new RuntimeException("Not enough files");
1206         }
1207 
1208         filesToCompact = filesToCompact.subList(count - N, count);
1209         maxId = StoreFile.getMaxSequenceIdInList(filesToCompact);
1210         isMajor = (filesToCompact.size() == storefiles.size());
1211         filesCompacting.addAll(filesToCompact);
1212         Collections.sort(filesCompacting, StoreFile.Comparators.FLUSH_TIME);
1213       }
1214     } finally {
1215       this.lock.readLock().unlock();
1216     }
1217 
1218     try {
1219       // Ready to go. Have list of files to compact.
1220       StoreFile.Writer writer = this.compactor.compactForTesting(this, conf, filesToCompact,
1221         isMajor, maxId);
1222       // Move the compaction into place.
1223       StoreFile sf = completeCompaction(filesToCompact, writer);
1224       if (region.getCoprocessorHost() != null) {
1225         region.getCoprocessorHost().postCompact(this, sf, null);
1226       }
1227     } finally {
1228       synchronized (filesCompacting) {
1229         filesCompacting.removeAll(filesToCompact);
1230       }
1231     }
1232   }
1233 
1234   boolean hasReferences() {
1235     return hasReferences(this.storefiles);
1236   }
1237 
1238   /*
1239    * @param files
1240    * @return True if any of the files in <code>files</code> are References.
1241    */
1242   private boolean hasReferences(Collection<StoreFile> files) {
1243     if (files != null && files.size() > 0) {
1244       for (StoreFile hsf: files) {
1245         if (hsf.isReference()) {
1246           return true;
1247         }
1248       }
1249     }
1250     return false;
1251   }
1252 
1253   /*
1254    * Gets lowest timestamp from candidate StoreFiles
1255    *
1256    * @param fs
1257    * @param dir
1258    * @throws IOException
1259    */
1260   public static long getLowestTimestamp(final List<StoreFile> candidates)
1261       throws IOException {
1262     long minTs = Long.MAX_VALUE;
1263     for (StoreFile storeFile : candidates) {
1264       minTs = Math.min(minTs, storeFile.getModificationTimeStamp());
1265     }
1266     return minTs;
1267   }
1268 
1269   /** getter for CompactionProgress object
1270    * @return CompactionProgress object; can be null
1271    */
1272   public CompactionProgress getCompactionProgress() {
1273     return this.compactor.getProgress();
1274   }
1275 
1276   /*
1277    * @return True if we should run a major compaction.
1278    */
1279   boolean isMajorCompaction() throws IOException {
1280     for (StoreFile sf : this.storefiles) {
1281       if (sf.getReader() == null) {
1282         LOG.debug("StoreFile " + sf + " has null Reader");
1283         return false;
1284       }
1285     }
1286 
1287     List<StoreFile> candidates = new ArrayList<StoreFile>(this.storefiles);
1288 
1289     // exclude files above the max compaction threshold
1290     // except: save all references. we MUST compact them
1291     int pos = 0;
1292     while (pos < candidates.size() &&
1293            candidates.get(pos).getReader().length() > this.maxCompactSize &&
1294            !candidates.get(pos).isReference()) ++pos;
1295     candidates.subList(0, pos).clear();
1296 
1297     return isMajorCompaction(candidates);
1298   }
1299 
1300   /*
1301    * @param filesToCompact Files to compact. Can be null.
1302    * @return True if we should run a major compaction.
1303    */
1304   private boolean isMajorCompaction(final List<StoreFile> filesToCompact) throws IOException {
1305     boolean result = false;
1306     long mcTime = getNextMajorCompactTime();
1307     if (filesToCompact == null || filesToCompact.isEmpty() || mcTime == 0) {
1308       return result;
1309     }
1310     // TODO: Use better method for determining stamp of last major (HBASE-2990)
1311     long lowTimestamp = getLowestTimestamp(filesToCompact);
1312     long now = System.currentTimeMillis();
1313     if (lowTimestamp > 0l && lowTimestamp < (now - mcTime)) {
1314       // Major compaction time has elapsed.
1315       if (filesToCompact.size() == 1) {
1316         // Single file
1317         StoreFile sf = filesToCompact.get(0);
1318         long oldest =
1319             (sf.getReader().timeRangeTracker == null) ?
1320                 Long.MIN_VALUE :
1321                 now - sf.getReader().timeRangeTracker.minimumTimestamp;
1322         if (sf.isMajorCompaction() &&
1323             (this.ttl == HConstants.FOREVER || oldest < this.ttl)) {
1324           if (LOG.isDebugEnabled()) {
1325             LOG.debug("Skipping major compaction of " + this +
1326                 " because one (major) compacted file only and oldestTime " +
1327                 oldest + "ms is < ttl=" + this.ttl);
1328           }
1329         } else if (this.ttl != HConstants.FOREVER && oldest > this.ttl) {
1330           LOG.debug("Major compaction triggered on store " + this +
1331             ", because keyvalues outdated; time since last major compaction " +
1332             (now - lowTimestamp) + "ms");
1333           result = true;
1334         }
1335       } else {
1336         if (LOG.isDebugEnabled()) {
1337           LOG.debug("Major compaction triggered on store " + this +
1338               "; time since last major compaction " + (now - lowTimestamp) + "ms");
1339         }
1340         result = true;
1341       }
1342     }
1343     return result;
1344   }
1345 
1346   long getNextMajorCompactTime() {
1347     // default = 24hrs
1348     long ret = conf.getLong(HConstants.MAJOR_COMPACTION_PERIOD, 1000*60*60*24);
1349     if (family.getValue(HConstants.MAJOR_COMPACTION_PERIOD) != null) {
1350       String strCompactionTime =
1351         family.getValue(HConstants.MAJOR_COMPACTION_PERIOD);
1352       ret = (new Long(strCompactionTime)).longValue();
1353     }
1354 
1355     if (ret > 0) {
1356       // default = 20% = +/- 4.8 hrs
1357       double jitterPct =  conf.getFloat("hbase.hregion.majorcompaction.jitter",
1358           0.20F);
1359       if (jitterPct > 0) {
1360         long jitter = Math.round(ret * jitterPct);
1361         // deterministic jitter avoids a major compaction storm on restart
1362         ImmutableList<StoreFile> snapshot = storefiles;
1363         if (snapshot != null && !snapshot.isEmpty()) {
1364           String seed = snapshot.get(0).getPath().getName();
1365           double curRand = new Random(seed.hashCode()).nextDouble();
1366           ret += jitter - Math.round(2L * jitter * curRand);
1367         } else {
1368           ret = 0; // no storefiles == no major compaction
1369         }
1370       }
1371     }
1372     return ret;
1373   }
1374 
1375   public CompactionRequest requestCompaction() throws IOException {
1376     return requestCompaction(NO_PRIORITY, null);
1377   }
1378 
1379   public CompactionRequest requestCompaction(int priority, CompactionRequest request)
1380       throws IOException {
1381     // don't even select for compaction if writes are disabled
1382     if (!this.region.areWritesEnabled()) {
1383       return null;
1384     }
1385 
1386     this.lock.readLock().lock();
1387     try {
1388       synchronized (filesCompacting) {
1389         // candidates = all storefiles not already in compaction queue
1390         List<StoreFile> candidates = Lists.newArrayList(storefiles);
1391         if (!filesCompacting.isEmpty()) {
1392           // exclude all files older than the newest file we're currently
1393           // compacting. this allows us to preserve contiguity (HBASE-2856)
1394           StoreFile last = filesCompacting.get(filesCompacting.size() - 1);
1395           int idx = candidates.indexOf(last);
1396           Preconditions.checkArgument(idx != -1);
1397           candidates.subList(0, idx + 1).clear();
1398         }
1399 
1400         boolean override = false;
1401         if (region.getCoprocessorHost() != null) {
1402           override = region.getCoprocessorHost().preCompactSelection(this, candidates, request);
1403         }
1404         CompactSelection filesToCompact;
1405         if (override) {
1406           // coprocessor is overriding normal file selection
1407           filesToCompact = new CompactSelection(conf, candidates);
1408         } else {
1409           filesToCompact = compactSelection(candidates, priority);
1410         }
1411 
1412         if (region.getCoprocessorHost() != null) {
1413           region.getCoprocessorHost().postCompactSelection(this,
1414             ImmutableList.copyOf(filesToCompact.getFilesToCompact()), request);
1415         }
1416 
1417         // no files to compact
1418         if (filesToCompact.getFilesToCompact().isEmpty()) {
1419           return null;
1420         }
1421 
1422         // basic sanity check: do not try to compact the same StoreFile twice.
1423         if (!Collections.disjoint(filesCompacting, filesToCompact.getFilesToCompact())) {
1424           // TODO: change this from an IAE to LOG.error after sufficient testing
1425           Preconditions.checkArgument(false, "%s overlaps with %s",
1426               filesToCompact, filesCompacting);
1427         }
1428         filesCompacting.addAll(filesToCompact.getFilesToCompact());
1429         Collections.sort(filesCompacting, StoreFile.Comparators.FLUSH_TIME);
1430 
1431         // major compaction iff all StoreFiles are included
1432         boolean isMajor = (filesToCompact.getFilesToCompact().size() == this.storefiles.size());
1433         if (isMajor) {
1434           // since we're enqueuing a major, update the compaction wait interval
1435           this.forceMajor = false;
1436         }
1437 
1438         // everything went better than expected. create a compaction request
1439         int pri = getCompactPriority(priority);
1440         //not a special compaction request, so we need to make one
1441         if(request == null){
1442           request = new CompactionRequest(region, this, filesToCompact, isMajor, pri);
1443         } else {
1444           // update the request with what the system thinks the request should be
1445           // its up to the request if it wants to listen
1446           request.setSelection(filesToCompact);
1447           request.setIsMajor(isMajor);
1448           request.setPriority(pri);
1449         }
1450       }
1451     } finally {
1452       this.lock.readLock().unlock();
1453     }
1454     if (request != null) {
1455       CompactionRequest.preRequest(request);
1456     }
1457     return request;
1458   }
1459 
1460   public void finishRequest(CompactionRequest cr) {
1461     CompactionRequest.postRequest(cr);
1462     cr.finishRequest();
1463     synchronized (filesCompacting) {
1464       filesCompacting.removeAll(cr.getFiles());
1465     }
1466   }
1467 
1468   /**
1469    * Algorithm to choose which files to compact, see {@link #compactSelection(java.util.List, int)}
1470    * @param candidates
1471    * @return
1472    * @throws IOException
1473    */
1474   CompactSelection compactSelection(List<StoreFile> candidates) throws IOException {
1475     return compactSelection(candidates,NO_PRIORITY);
1476   }
1477 
1478   /**
1479    * Algorithm to choose which files to compact
1480    *
1481    * Configuration knobs:
1482    *  "hbase.hstore.compaction.ratio"
1483    *    normal case: minor compact when file <= sum(smaller_files) * ratio
1484    *  "hbase.hstore.compaction.min.size"
1485    *    unconditionally compact individual files below this size
1486    *  "hbase.hstore.compaction.max.size"
1487    *    never compact individual files above this size (unless splitting)
1488    *  "hbase.hstore.compaction.min"
1489    *    min files needed to minor compact
1490    *  "hbase.hstore.compaction.max"
1491    *    max files to compact at once (avoids OOM)
1492    *
1493    * @param candidates candidate files, ordered from oldest to newest
1494    * @return subset copy of candidate list that meets compaction criteria
1495    * @throws IOException
1496    */
1497   CompactSelection compactSelection(List<StoreFile> candidates, int priority)
1498       throws IOException {
1499     // ASSUMPTION!!! filesCompacting is locked when calling this function
1500 
1501     /* normal skew:
1502      *
1503      *         older ----> newer
1504      *     _
1505      *    | |   _
1506      *    | |  | |   _
1507      *  --|-|- |-|- |-|---_-------_-------  minCompactSize
1508      *    | |  | |  | |  | |  _  | |
1509      *    | |  | |  | |  | | | | | |
1510      *    | |  | |  | |  | | | | | |
1511      */
1512     CompactSelection compactSelection = new CompactSelection(conf, candidates);
1513 
1514     boolean forcemajor = this.forceMajor && filesCompacting.isEmpty();
1515     if (!forcemajor) {
1516       // Delete the expired store files before the compaction selection.
1517       if (conf.getBoolean("hbase.store.delete.expired.storefile", true)
1518           && (ttl != Long.MAX_VALUE) && (this.scanInfo.minVersions == 0)) {
1519         CompactSelection expiredSelection = compactSelection
1520             .selectExpiredStoreFilesToCompact(
1521                 EnvironmentEdgeManager.currentTimeMillis() - this.ttl);
1522 
1523         // If there is any expired store files, delete them  by compaction.
1524         if (expiredSelection != null) {
1525           return expiredSelection;
1526         }
1527       }
1528       // do not compact old files above a configurable threshold
1529       // save all references. we MUST compact them
1530       int pos = 0;
1531       while (pos < compactSelection.getFilesToCompact().size() &&
1532              compactSelection.getFilesToCompact().get(pos).getReader().length()
1533                > maxCompactSize &&
1534              !compactSelection.getFilesToCompact().get(pos).isReference()) ++pos;
1535       if (pos != 0) compactSelection.clearSubList(0, pos);
1536     }
1537 
1538     if (compactSelection.getFilesToCompact().isEmpty()) {
1539       LOG.debug(this.getHRegionInfo().getEncodedName() + " - " +
1540         this + ": no store files to compact");
1541       compactSelection.emptyFileList();
1542       return compactSelection;
1543     }
1544 
1545     // Force a major compaction if this is a user-requested major compaction,
1546     // or if we do not have too many files to compact and this was requested
1547     // as a major compaction
1548     boolean majorcompaction = (forcemajor && priority == PRIORITY_USER) ||
1549       (forcemajor || isMajorCompaction(compactSelection.getFilesToCompact())) &&
1550       (compactSelection.getFilesToCompact().size() < this.maxFilesToCompact
1551     );
1552     LOG.debug(this.getHRegionInfo().getEncodedName() + " - " +
1553       this.getColumnFamilyName() + ": Initiating " +
1554       (majorcompaction ? "major" : "minor") + "compaction");
1555 
1556     if (!majorcompaction &&
1557         !hasReferences(compactSelection.getFilesToCompact())) {
1558 
1559       // remove bulk import files that request to be excluded from minors
1560       compactSelection.getFilesToCompact().removeAll(Collections2.filter(
1561           compactSelection.getFilesToCompact(),
1562           new Predicate<StoreFile>() {
1563             public boolean apply(StoreFile input) {
1564               return input.excludeFromMinorCompaction();
1565             }
1566           }));
1567 
1568       // skip selection algorithm if we don't have enough files
1569       if (compactSelection.getFilesToCompact().size() < this.minFilesToCompact) {
1570         if(LOG.isDebugEnabled()) {
1571           LOG.debug("Not compacting files because we only have " +
1572             compactSelection.getFilesToCompact().size() +
1573             " files ready for compaction.  Need " + this.minFilesToCompact + " to initiate.");
1574         }
1575         compactSelection.emptyFileList();
1576         return compactSelection;
1577       }
1578       if (conf.getBoolean("hbase.hstore.useExploringCompation", false)) {
1579         compactSelection = exploringCompactionSelection(compactSelection);
1580       } else {
1581         compactSelection = defaultCompactionSelection(compactSelection);
1582       }
1583     } else {
1584       if(majorcompaction) {
1585         if (compactSelection.getFilesToCompact().size() > this.maxFilesToCompact) {
1586           LOG.debug("Warning, compacting more than " + this.maxFilesToCompact +
1587             " files, probably because of a user-requested major compaction");
1588           if(priority != PRIORITY_USER) {
1589             LOG.error("Compacting more than max files on a non user-requested compaction");
1590           }
1591         }
1592       } else if (compactSelection.getFilesToCompact().size() > this.maxFilesToCompact) {
1593         // all files included in this compaction, up to max
1594         int pastMax = compactSelection.getFilesToCompact().size() - this.maxFilesToCompact;
1595         compactSelection.getFilesToCompact().subList(0, pastMax).clear();
1596       }
1597     }
1598     return compactSelection;
1599   }
1600 
1601   private CompactSelection defaultCompactionSelection(CompactSelection compactSelection) {
1602     // we're doing a minor compaction, let's see what files are applicable
1603     int start = 0;
1604 
1605     double r = compactSelection.getCompactSelectionRatio();
1606 
1607     // get store file sizes for incremental compacting selection.
1608     int countOfFiles = compactSelection.getFilesToCompact().size();
1609     long [] fileSizes = new long[countOfFiles];
1610     long [] sumSize = new long[countOfFiles];
1611     for (int i = countOfFiles-1; i >= 0; --i) {
1612       StoreFile file = compactSelection.getFilesToCompact().get(i);
1613       fileSizes[i] = file.getReader().length();
1614       // calculate the sum of fileSizes[i,i+maxFilesToCompact-1) for algo
1615       int tooFar = i + this.maxFilesToCompact - 1;
1616       sumSize[i] = fileSizes[i]
1617           + ((i+1    < countOfFiles) ? sumSize[i+1]      : 0)
1618           - ((tooFar < countOfFiles) ? fileSizes[tooFar] : 0);
1619     }
1620 
1621       /* Start at the oldest file and stop when you find the first file that
1622        * meets compaction criteria:
1623        *   (1) a recently-flushed, small file (i.e. <= minCompactSize)
1624        *      OR
1625        *   (2) within the compactRatio of sum(newer_files)
1626        * Given normal skew, any newer files will also meet this criteria
1627        *
1628        * Additional Note:
1629        * If fileSizes.size() >> maxFilesToCompact, we will recurse on
1630        * compact().  Consider the oldest files first to avoid a
1631        * situation where we always compact [end-threshold,end).  Then, the
1632        * last file becomes an aggregate of the previous compactions.
1633        */
1634     while(countOfFiles - start >= this.minFilesToCompact &&
1635         fileSizes[start] >
1636             Math.max(minCompactSize, (long)(sumSize[start+1] * r))) {
1637       ++start;
1638     }
1639     int end = Math.min(countOfFiles, start + this.maxFilesToCompact);
1640     long totalSize = fileSizes[start]
1641         + ((start+1 < countOfFiles) ? sumSize[start+1] : 0);
1642     compactSelection = compactSelection.getSubList(start, end);
1643 
1644     // if we don't have enough files to compact, just wait
1645     if (compactSelection.getFilesToCompact().size() < this.minFilesToCompact) {
1646       if (LOG.isDebugEnabled()) {
1647         LOG.debug("Skipped compaction of " + this
1648             + ".  Only " + (end - start) + " file(s) of size "
1649             + StringUtils.humanReadableInt(totalSize)
1650             + " have met compaction criteria.");
1651       }
1652       compactSelection.emptyFileList();
1653       return compactSelection;
1654     }
1655     return compactSelection;
1656   }
1657 
1658   private CompactSelection exploringCompactionSelection(CompactSelection compactSelection) {
1659 
1660     List<StoreFile> candidates = compactSelection.getFilesToCompact();
1661     int futureFiles = filesCompacting.isEmpty() ? 0 : 1;
1662     boolean mayBeStuck = (candidates.size() - filesCompacting.size() + futureFiles)
1663         >= blockingStoreFileCount;
1664     // Start off choosing nothing.
1665     List<StoreFile> bestSelection = new ArrayList<StoreFile>(0);
1666     List<StoreFile> smallest = new ArrayList<StoreFile>(0);
1667     long bestSize = 0;
1668     long smallestSize = Long.MAX_VALUE;
1669     double r = compactSelection.getCompactSelectionRatio();
1670 
1671     // Consider every starting place.
1672     for (int startIndex = 0; startIndex < candidates.size(); startIndex++) {
1673       // Consider every different sub list permutation in between start and end with min files.
1674       for (int currentEnd = startIndex + minFilesToCompact - 1;
1675            currentEnd < candidates.size(); currentEnd++) {
1676         List<StoreFile> potentialMatchFiles = candidates.subList(startIndex, currentEnd + 1);
1677 
1678         // Sanity checks
1679         if (potentialMatchFiles.size() < minFilesToCompact) {
1680           continue;
1681         }
1682         if (potentialMatchFiles.size() > maxFilesToCompact) {
1683           continue;
1684         }
1685 
1686         // Compute the total size of files that will
1687         // have to be read if this set of files is compacted.
1688         long size = getCompactionSize(potentialMatchFiles);
1689 
1690         // Store the smallest set of files.  This stored set of files will be used
1691         // if it looks like the algorithm is stuck.
1692         if (size < smallestSize) {
1693           smallest = potentialMatchFiles;
1694           smallestSize = size;
1695         }
1696 
1697         if (size >= minCompactSize
1698             && !filesInRatio(potentialMatchFiles, r)) {
1699           continue;
1700         }
1701 
1702         if (size > maxCompactSize) {
1703           continue;
1704         }
1705 
1706         // Keep if this gets rid of more files.  Or the same number of files for less io.
1707         if (potentialMatchFiles.size() > bestSelection.size()
1708             || (potentialMatchFiles.size() == bestSelection.size() && size < bestSize)) {
1709           bestSelection = potentialMatchFiles;
1710           bestSize = size;
1711         }
1712       }
1713     }
1714 
1715     if (bestSelection.size() == 0 && mayBeStuck) {
1716       smallest = new ArrayList<StoreFile>(smallest);
1717       compactSelection.getFilesToCompact().clear();
1718       compactSelection.getFilesToCompact().addAll(smallest);
1719     } else {
1720       bestSelection = new ArrayList<StoreFile>(bestSelection);
1721       compactSelection.getFilesToCompact().clear();
1722       compactSelection.getFilesToCompact().addAll(bestSelection);
1723     }
1724 
1725     return compactSelection;
1726 
1727   }
1728 
1729   /**
1730    * Check that all files satisfy the ratio
1731    *
1732    * @param files set of files to examine.
1733    * @param currentRatio The raio
1734    * @return if all files are in ratio.
1735    */
1736   private boolean filesInRatio(final List<StoreFile> files, final double currentRatio) {
1737     if (files.size() < 2) {
1738       return true;
1739     }
1740     long totalFileSize = 0;
1741     for (int i = 0; i < files.size(); i++) {
1742       totalFileSize += files.get(i).getReader().length();
1743     }
1744     for (int i = 0; i < files.size(); i++) {
1745       long singleFileSize = files.get(i).getReader().length();
1746       long sumAllOtherFilesize = totalFileSize - singleFileSize;
1747 
1748       if ((singleFileSize > sumAllOtherFilesize * currentRatio)
1749           && (sumAllOtherFilesize >= this.minCompactSize)) {
1750         return false;
1751       }
1752     }
1753     return true;
1754   }
1755 
1756   /**
1757    * Get the number of bytes a proposed compaction would have to read.
1758    *
1759    * @param files Set of files in a proposed compaction.
1760    * @return size in bytes.
1761    */
1762   private long getCompactionSize(final List<StoreFile> files) {
1763     long size = 0;
1764     if (files == null) {
1765       return size;
1766     }
1767     for (StoreFile f : files) {
1768       size += f.getReader().length();
1769     }
1770     return size;
1771   }
1772 
1773   /**
1774    * Validates a store file by opening and closing it. In HFileV2 this should
1775    * not be an expensive operation.
1776    *
1777    * @param path the path to the store file
1778    */
1779   private void validateStoreFile(Path path)
1780       throws IOException {
1781     StoreFile storeFile = null;
1782     try {
1783       storeFile = new StoreFile(this.fs, path, this.conf,
1784           this.cacheConf, this.family.getBloomFilterType(),
1785           NoOpDataBlockEncoder.INSTANCE);
1786       passSchemaMetricsTo(storeFile);
1787       storeFile.createReader();
1788     } catch (IOException e) {
1789       LOG.error("Failed to open store file : " + path
1790           + ", keeping it in tmp location", e);
1791       throw e;
1792     } finally {
1793       if (storeFile != null) {
1794         storeFile.closeReader(false);
1795       }
1796     }
1797   }
1798 
1799   /*
1800    * <p>It works by processing a compaction that's been written to disk.
1801    *
1802    * <p>It is usually invoked at the end of a compaction, but might also be
1803    * invoked at HStore startup, if the prior execution died midway through.
1804    *
1805    * <p>Moving the compacted TreeMap into place means:
1806    * <pre>
1807    * 1) Moving the new compacted StoreFile into place
1808    * 2) Unload all replaced StoreFile, close and collect list to delete.
1809    * 3) Loading the new TreeMap.
1810    * 4) Compute new store size
1811    * </pre>
1812    *
1813    * @param compactedFiles list of files that were compacted
1814    * @param compactedFile StoreFile that is the result of the compaction
1815    * @return StoreFile created. May be null.
1816    * @throws IOException
1817    */
1818   StoreFile completeCompaction(final Collection<StoreFile> compactedFiles,
1819                                        final StoreFile.Writer compactedFile)
1820       throws IOException {
1821     // 1. Moving the new files into place -- if there is a new file (may not
1822     // be if all cells were expired or deleted).
1823     StoreFile result = null;
1824     if (compactedFile != null) {
1825       validateStoreFile(compactedFile.getPath());
1826       // Move the file into the right spot
1827       Path origPath = compactedFile.getPath();
1828       Path destPath = new Path(homedir, origPath.getName());
1829       LOG.info("Renaming compacted file at " + origPath + " to " + destPath);
1830       if (!HBaseFileSystem.renameDirForFileSystem(fs, origPath, destPath)) {
1831         LOG.error("Failed move of compacted file " + origPath + " to " +
1832             destPath);
1833         throw new IOException("Failed move of compacted file " + origPath +
1834             " to " + destPath);
1835       }
1836       result = new StoreFile(this.fs, destPath, this.conf, this.cacheConf,
1837           this.family.getBloomFilterType(), this.dataBlockEncoder);
1838       passSchemaMetricsTo(result);
1839       result.createReader();
1840     }
1841     try {
1842       this.lock.writeLock().lock();
1843       try {
1844         // Change this.storefiles so it reflects new state but do not
1845         // delete old store files until we have sent out notification of
1846         // change in case old files are still being accessed by outstanding
1847         // scanners.
1848         ArrayList<StoreFile> newStoreFiles = Lists.newArrayList(storefiles);
1849         newStoreFiles.removeAll(compactedFiles);
1850         filesCompacting.removeAll(compactedFiles); // safe bc: lock.writeLock()
1851 
1852         // If a StoreFile result, move it into place.  May be null.
1853         if (result != null) {
1854           newStoreFiles.add(result);
1855         }
1856 
1857         this.storefiles = sortAndClone(newStoreFiles);
1858       } finally {
1859         // We need the lock, as long as we are updating the storefiles
1860         // or changing the memstore. Let us release it before calling
1861         // notifyChangeReadersObservers. See HBASE-4485 for a possible
1862         // deadlock scenario that could have happened if continue to hold
1863         // the lock.
1864         this.lock.writeLock().unlock();
1865       }
1866 
1867       // Tell observers that list of StoreFiles has changed.
1868       notifyChangedReadersObservers();
1869 
1870       // let the archive util decide if we should archive or delete the files
1871       LOG.debug("Removing store files after compaction...");
1872       HFileArchiver.archiveStoreFiles(this.conf, this.fs, this.region, this.family.getName(),
1873         compactedFiles);
1874 
1875     } catch (IOException e) {
1876       e = RemoteExceptionHandler.checkIOException(e);
1877       LOG.error("Failed replacing compacted files in " + this +
1878         ". Compacted file is " + (result == null? "none": result.toString()) +
1879         ".  Files replaced " + compactedFiles.toString() +
1880         " some of which may have been already removed", e);
1881     }
1882 
1883     // 4. Compute new store size
1884     this.storeSize = 0L;
1885     this.totalUncompressedBytes = 0L;
1886     for (StoreFile hsf : this.storefiles) {
1887       StoreFile.Reader r = hsf.getReader();
1888       if (r == null) {
1889         LOG.warn("StoreFile " + hsf + " has a null Reader");
1890         continue;
1891       }
1892       this.storeSize += r.length();
1893       this.totalUncompressedBytes += r.getTotalUncompressedBytes();
1894     }
1895     return result;
1896   }
1897 
1898   public ImmutableList<StoreFile> sortAndClone(List<StoreFile> storeFiles) {
1899     Collections.sort(storeFiles, StoreFile.Comparators.FLUSH_TIME);
1900     ImmutableList<StoreFile> newList = ImmutableList.copyOf(storeFiles);
1901     return newList;
1902   }
1903 
1904   // ////////////////////////////////////////////////////////////////////////////
1905   // Accessors.
1906   // (This is the only section that is directly useful!)
1907   //////////////////////////////////////////////////////////////////////////////
1908   /**
1909    * @return the number of files in this store
1910    */
1911   public int getNumberOfStoreFiles() {
1912     return this.storefiles.size();
1913   }
1914 
1915   /*
1916    * @param wantedVersions How many versions were asked for.
1917    * @return wantedVersions or this families' {@link HConstants#VERSIONS}.
1918    */
1919   int versionsToReturn(final int wantedVersions) {
1920     if (wantedVersions <= 0) {
1921       throw new IllegalArgumentException("Number of versions must be > 0");
1922     }
1923     // Make sure we do not return more than maximum versions for this store.
1924     int maxVersions = this.family.getMaxVersions();
1925     return wantedVersions > maxVersions ? maxVersions: wantedVersions;
1926   }
1927 
1928   static boolean isExpired(final KeyValue key, final long oldestTimestamp) {
1929     return key.getTimestamp() < oldestTimestamp;
1930   }
1931 
1932   /**
1933    * Find the key that matches <i>row</i> exactly, or the one that immediately
1934    * precedes it. WARNING: Only use this method on a table where writes occur
1935    * with strictly increasing timestamps. This method assumes this pattern of
1936    * writes in order to make it reasonably performant.  Also our search is
1937    * dependent on the axiom that deletes are for cells that are in the container
1938    * that follows whether a memstore snapshot or a storefile, not for the
1939    * current container: i.e. we'll see deletes before we come across cells we
1940    * are to delete. Presumption is that the memstore#kvset is processed before
1941    * memstore#snapshot and so on.
1942    * @param row The row key of the targeted row.
1943    * @return Found keyvalue or null if none found.
1944    * @throws IOException
1945    */
1946   KeyValue getRowKeyAtOrBefore(final byte[] row) throws IOException {
1947     // If minVersions is set, we will not ignore expired KVs.
1948     // As we're only looking for the latest matches, that should be OK.
1949     // With minVersions > 0 we guarantee that any KV that has any version
1950     // at all (expired or not) has at least one version that will not expire.
1951     // Note that this method used to take a KeyValue as arguments. KeyValue
1952     // can be back-dated, a row key cannot.
1953     long ttlToUse = scanInfo.getMinVersions() > 0 ? Long.MAX_VALUE : this.ttl;
1954 
1955     KeyValue kv = new KeyValue(row, HConstants.LATEST_TIMESTAMP);
1956 
1957     GetClosestRowBeforeTracker state = new GetClosestRowBeforeTracker(
1958       this.comparator, kv, ttlToUse, this.region.getRegionInfo().isMetaRegion());
1959     this.lock.readLock().lock();
1960     try {
1961       // First go to the memstore.  Pick up deletes and candidates.
1962       this.memstore.getRowKeyAtOrBefore(state);
1963       // Check if match, if we got a candidate on the asked for 'kv' row.
1964       // Process each store file. Run through from newest to oldest.
1965       for (StoreFile sf : Lists.reverse(storefiles)) {
1966         // Update the candidate keys from the current map file
1967         rowAtOrBeforeFromStoreFile(sf, state);
1968       }
1969       return state.getCandidate();
1970     } finally {
1971       this.lock.readLock().unlock();
1972     }
1973   }
1974 
1975   /*
1976    * Check an individual MapFile for the row at or before a given row.
1977    * @param f
1978    * @param state
1979    * @throws IOException
1980    */
1981   private void rowAtOrBeforeFromStoreFile(final StoreFile f,
1982                                           final GetClosestRowBeforeTracker state)
1983       throws IOException {
1984     StoreFile.Reader r = f.getReader();
1985     if (r == null) {
1986       LOG.warn("StoreFile " + f + " has a null Reader");
1987       return;
1988     }
1989     if (r.getEntries() == 0) {
1990       LOG.warn("StoreFile " + f + " is a empty store file");
1991       return;
1992     }
1993     // TODO: Cache these keys rather than make each time?
1994     byte [] fk = r.getFirstKey();
1995     if (fk == null) return;
1996     KeyValue firstKV = KeyValue.createKeyValueFromKey(fk, 0, fk.length);
1997     byte [] lk = r.getLastKey();
1998     KeyValue lastKV = KeyValue.createKeyValueFromKey(lk, 0, lk.length);
1999     KeyValue firstOnRow = state.getTargetKey();
2000     if (this.comparator.compareRows(lastKV, firstOnRow) < 0) {
2001       // If last key in file is not of the target table, no candidates in this
2002       // file.  Return.
2003       if (!state.isTargetTable(lastKV)) return;
2004       // If the row we're looking for is past the end of file, set search key to
2005       // last key. TODO: Cache last and first key rather than make each time.
2006       firstOnRow = new KeyValue(lastKV.getRow(), HConstants.LATEST_TIMESTAMP);
2007     }
2008     // Get a scanner that caches blocks and that uses pread.
2009     HFileScanner scanner = r.getScanner(true, true, false);
2010     // Seek scanner.  If can't seek it, return.
2011     if (!seekToScanner(scanner, firstOnRow, firstKV)) return;
2012     // If we found candidate on firstOnRow, just return. THIS WILL NEVER HAPPEN!
2013     // Unlikely that there'll be an instance of actual first row in table.
2014     if (walkForwardInSingleRow(scanner, firstOnRow, state)) return;
2015     // If here, need to start backing up.
2016     while (scanner.seekBefore(firstOnRow.getBuffer(), firstOnRow.getKeyOffset(),
2017        firstOnRow.getKeyLength())) {
2018       KeyValue kv = scanner.getKeyValue();
2019       if (!state.isTargetTable(kv)) break;
2020       if (!state.isBetterCandidate(kv)) break;
2021       // Make new first on row.
2022       firstOnRow = new KeyValue(kv.getRow(), HConstants.LATEST_TIMESTAMP);
2023       // Seek scanner.  If can't seek it, break.
2024       if (!seekToScanner(scanner, firstOnRow, firstKV)) break;
2025       // If we find something, break;
2026       if (walkForwardInSingleRow(scanner, firstOnRow, state)) break;
2027     }
2028   }
2029 
2030   /*
2031    * Seek the file scanner to firstOnRow or first entry in file.
2032    * @param scanner
2033    * @param firstOnRow
2034    * @param firstKV
2035    * @return True if we successfully seeked scanner.
2036    * @throws IOException
2037    */
2038   private boolean seekToScanner(final HFileScanner scanner,
2039                                 final KeyValue firstOnRow,
2040                                 final KeyValue firstKV)
2041       throws IOException {
2042     KeyValue kv = firstOnRow;
2043     // If firstOnRow < firstKV, set to firstKV
2044     if (this.comparator.compareRows(firstKV, firstOnRow) == 0) kv = firstKV;
2045     int result = scanner.seekTo(kv.getBuffer(), kv.getKeyOffset(),
2046       kv.getKeyLength());
2047     return result >= 0;
2048   }
2049 
2050   /*
2051    * When we come in here, we are probably at the kv just before we break into
2052    * the row that firstOnRow is on.  Usually need to increment one time to get
2053    * on to the row we are interested in.
2054    * @param scanner
2055    * @param firstOnRow
2056    * @param state
2057    * @return True we found a candidate.
2058    * @throws IOException
2059    */
2060   private boolean walkForwardInSingleRow(final HFileScanner scanner,
2061                                          final KeyValue firstOnRow,
2062                                          final GetClosestRowBeforeTracker state)
2063       throws IOException {
2064     boolean foundCandidate = false;
2065     do {
2066       KeyValue kv = scanner.getKeyValue();
2067       // If we are not in the row, skip.
2068       if (this.comparator.compareRows(kv, firstOnRow) < 0) continue;
2069       // Did we go beyond the target row? If so break.
2070       if (state.isTooFar(kv, firstOnRow)) break;
2071       if (state.isExpired(kv)) {
2072         continue;
2073       }
2074       // If we added something, this row is a contender. break.
2075       if (state.handle(kv)) {
2076         foundCandidate = true;
2077         break;
2078       }
2079     } while(scanner.next());
2080     return foundCandidate;
2081   }
2082 
2083   public boolean canSplit() {
2084     this.lock.readLock().lock();
2085     try {
2086       // Not splitable if we find a reference store file present in the store.
2087       for (StoreFile sf : storefiles) {
2088         if (sf.isReference()) {
2089           if (LOG.isDebugEnabled()) {
2090             LOG.debug(sf + " is not splittable");
2091           }
2092           return false;
2093         }
2094       }
2095 
2096       return true;
2097     } finally {
2098       this.lock.readLock().unlock();
2099     }
2100   }
2101   /**
2102    * Determines if Store should be split
2103    * @return byte[] if store should be split, null otherwise.
2104    */
2105   public byte[] getSplitPoint() {
2106     this.lock.readLock().lock();
2107     try {
2108       // sanity checks
2109       if (this.storefiles.isEmpty()) {
2110         return null;
2111       }
2112       // Should already be enforced by the split policy!
2113       assert !this.region.getRegionInfo().isMetaRegion();
2114 
2115       // Not splitable if we find a reference store file present in the store.
2116       long maxSize = 0L;
2117       StoreFile largestSf = null;
2118       for (StoreFile sf : storefiles) {
2119         if (sf.isReference()) {
2120           // Should already be enforced since we return false in this case
2121           assert false : "getSplitPoint() called on a region that can't split!";
2122           return null;
2123         }
2124 
2125         StoreFile.Reader r = sf.getReader();
2126         if (r == null) {
2127           LOG.warn("Storefile " + sf + " Reader is null");
2128           continue;
2129         }
2130 
2131         long size = r.length();
2132         if (size > maxSize) {
2133           // This is the largest one so far
2134           maxSize = size;
2135           largestSf = sf;
2136         }
2137       }
2138 
2139       StoreFile.Reader r = largestSf.getReader();
2140       if (r == null) {
2141         LOG.warn("Storefile " + largestSf + " Reader is null");
2142         return null;
2143       }
2144       // Get first, last, and mid keys.  Midkey is the key that starts block
2145       // in middle of hfile.  Has column and timestamp.  Need to return just
2146       // the row we want to split on as midkey.
2147       byte [] midkey = r.midkey();
2148       if (midkey != null) {
2149         KeyValue mk = KeyValue.createKeyValueFromKey(midkey, 0, midkey.length);
2150         byte [] fk = r.getFirstKey();
2151         KeyValue firstKey = KeyValue.createKeyValueFromKey(fk, 0, fk.length);
2152         byte [] lk = r.getLastKey();
2153         KeyValue lastKey = KeyValue.createKeyValueFromKey(lk, 0, lk.length);
2154         // if the midkey is the same as the first or last keys, then we cannot
2155         // (ever) split this region.
2156         if (this.comparator.compareRows(mk, firstKey) == 0 ||
2157             this.comparator.compareRows(mk, lastKey) == 0) {
2158           if (LOG.isDebugEnabled()) {
2159             LOG.debug("cannot split because midkey is the same as first or " +
2160               "last row");
2161           }
2162           return null;
2163         }
2164         return mk.getRow();
2165       }
2166     } catch(IOException e) {
2167       LOG.warn("Failed getting store size for " + this, e);
2168     } finally {
2169       this.lock.readLock().unlock();
2170     }
2171     return null;
2172   }
2173 
2174   /** @return aggregate size of all HStores used in the last compaction */
2175   public long getLastCompactSize() {
2176     return this.lastCompactSize;
2177   }
2178 
2179   /** @return aggregate size of HStore */
2180   public long getSize() {
2181     return storeSize;
2182   }
2183 
2184   public void triggerMajorCompaction() {
2185     this.forceMajor = true;
2186   }
2187 
2188   boolean getForceMajorCompaction() {
2189     return this.forceMajor;
2190   }
2191 
2192   //////////////////////////////////////////////////////////////////////////////
2193   // File administration
2194   //////////////////////////////////////////////////////////////////////////////
2195 
2196   /**
2197    * Return a scanner for both the memstore and the HStore files. Assumes we
2198    * are not in a compaction.
2199    * @throws IOException
2200    */
2201   public KeyValueScanner getScanner(Scan scan,
2202       final NavigableSet<byte []> targetCols) throws IOException {
2203     lock.readLock().lock();
2204     try {
2205       KeyValueScanner scanner = null;
2206       if (getHRegion().getCoprocessorHost() != null) {
2207         scanner = getHRegion().getCoprocessorHost().preStoreScannerOpen(this, scan, targetCols);
2208       }
2209       if (scanner == null) {
2210         scanner = new StoreScanner(this, getScanInfo(), scan, targetCols);
2211       }
2212       return scanner;
2213     } finally {
2214       lock.readLock().unlock();
2215     }
2216   }
2217 
2218   @Override
2219   public String toString() {
2220     return getColumnFamilyName();
2221   }
2222 
2223   /**
2224    * @return Count of store files
2225    */
2226   int getStorefilesCount() {
2227     return this.storefiles.size();
2228   }
2229 
2230   /**
2231    * @return The size of the store files, in bytes, uncompressed.
2232    */
2233   long getStoreSizeUncompressed() {
2234     return this.totalUncompressedBytes;
2235   }
2236 
2237   /**
2238    * @return The size of the store files, in bytes.
2239    */
2240   long getStorefilesSize() {
2241     long size = 0;
2242     for (StoreFile s: storefiles) {
2243       StoreFile.Reader r = s.getReader();
2244       if (r == null) {
2245         LOG.warn("StoreFile " + s + " has a null Reader");
2246         continue;
2247       }
2248       size += r.length();
2249     }
2250     return size;
2251   }
2252 
2253   /**
2254    * @return The size of the store file indexes, in bytes.
2255    */
2256   long getStorefilesIndexSize() {
2257     long size = 0;
2258     for (StoreFile s: storefiles) {
2259       StoreFile.Reader r = s.getReader();
2260       if (r == null) {
2261         LOG.warn("StoreFile " + s + " has a null Reader");
2262         continue;
2263       }
2264       size += r.indexSize();
2265     }
2266     return size;
2267   }
2268 
2269   /**
2270    * Returns the total size of all index blocks in the data block indexes,
2271    * including the root level, intermediate levels, and the leaf level for
2272    * multi-level indexes, or just the root level for single-level indexes.
2273    *
2274    * @return the total size of block indexes in the store
2275    */
2276   long getTotalStaticIndexSize() {
2277     long size = 0;
2278     for (StoreFile s : storefiles) {
2279       size += s.getReader().getUncompressedDataIndexSize();
2280     }
2281     return size;
2282   }
2283 
2284   /**
2285    * Returns the total byte size of all Bloom filter bit arrays. For compound
2286    * Bloom filters even the Bloom blocks currently not loaded into the block
2287    * cache are counted.
2288    *
2289    * @return the total size of all Bloom filters in the store
2290    */
2291   long getTotalStaticBloomSize() {
2292     long size = 0;
2293     for (StoreFile s : storefiles) {
2294       StoreFile.Reader r = s.getReader();
2295       size += r.getTotalBloomSize();
2296     }
2297     return size;
2298   }
2299 
2300   /**
2301    * @return The size of this store's memstore, in bytes
2302    */
2303   long getMemStoreSize() {
2304     return this.memstore.heapSize();
2305   }
2306 
2307   public int getCompactPriority() {
2308     return getCompactPriority(NO_PRIORITY);
2309   }
2310 
2311   /**
2312    * @return The priority that this store should have in the compaction queue
2313    * @param priority
2314    */
2315   public int getCompactPriority(int priority) {
2316     // If this is a user-requested compaction, leave this at the highest priority
2317     if(priority == PRIORITY_USER) {
2318       return PRIORITY_USER;
2319     } else {
2320       return this.blockingStoreFileCount - this.storefiles.size();
2321     }
2322   }
2323 
2324   boolean throttleCompaction(long compactionSize) {
2325     long throttlePoint = conf.getLong(
2326         "hbase.regionserver.thread.compaction.throttle",
2327         2 * this.minFilesToCompact * this.region.memstoreFlushSize);
2328     return compactionSize > throttlePoint;
2329   }
2330 
2331   public HRegion getHRegion() {
2332     return this.region;
2333   }
2334 
2335   HRegionInfo getHRegionInfo() {
2336     return this.region.getRegionInfo();
2337   }
2338 
2339   /**
2340    * Increments the value for the given row/family/qualifier.
2341    *
2342    * This function will always be seen as atomic by other readers
2343    * because it only puts a single KV to memstore. Thus no
2344    * read/write control necessary.
2345    *
2346    * @param row
2347    * @param f
2348    * @param qualifier
2349    * @param newValue the new value to set into memstore
2350    * @return memstore size delta
2351    * @throws IOException
2352    */
2353   public long updateColumnValue(byte [] row, byte [] f,
2354                                 byte [] qualifier, long newValue)
2355       throws IOException {
2356 
2357     this.lock.readLock().lock();
2358     try {
2359       long now = EnvironmentEdgeManager.currentTimeMillis();
2360 
2361       return this.memstore.updateColumnValue(row,
2362           f,
2363           qualifier,
2364           newValue,
2365           now);
2366 
2367     } finally {
2368       this.lock.readLock().unlock();
2369     }
2370   }
2371 
2372   /**
2373    * Adds or replaces the specified KeyValues.
2374    * <p>
2375    * For each KeyValue specified, if a cell with the same row, family, and
2376    * qualifier exists in MemStore, it will be replaced.  Otherwise, it will just
2377    * be inserted to MemStore.
2378    * <p>
2379    * This operation is atomic on each KeyValue (row/family/qualifier) but not
2380    * necessarily atomic across all of them.
2381    * @param kvs
2382    * @return memstore size delta
2383    * @throws IOException
2384    */
2385   public long upsert(List<KeyValue> kvs)
2386       throws IOException {
2387     this.lock.readLock().lock();
2388     try {
2389       // TODO: Make this operation atomic w/ MVCC
2390       return this.memstore.upsert(kvs);
2391     } finally {
2392       this.lock.readLock().unlock();
2393     }
2394   }
2395 
2396   public StoreFlusher getStoreFlusher(long cacheFlushId) {
2397     return new StoreFlusherImpl(cacheFlushId);
2398   }
2399 
2400   private class StoreFlusherImpl implements StoreFlusher {
2401 
2402     private long cacheFlushId;
2403     private SortedSet<KeyValue> snapshot;
2404     private StoreFile storeFile;
2405     private Path storeFilePath;
2406     private TimeRangeTracker snapshotTimeRangeTracker;
2407     private AtomicLong flushedSize;
2408 
2409     private StoreFlusherImpl(long cacheFlushId) {
2410       this.cacheFlushId = cacheFlushId;
2411       this.flushedSize = new AtomicLong();
2412     }
2413 
2414     @Override
2415     public void prepare() {
2416       memstore.snapshot();
2417       this.snapshot = memstore.getSnapshot();
2418       this.snapshotTimeRangeTracker = memstore.getSnapshotTimeRangeTracker();
2419     }
2420 
2421     @Override
2422     public void flushCache(MonitoredTask status) throws IOException {
2423       storeFilePath = Store.this.flushCache(
2424         cacheFlushId, snapshot, snapshotTimeRangeTracker, flushedSize, status);
2425     }
2426 
2427     @Override
2428     public boolean commit(MonitoredTask status) throws IOException {
2429       if (storeFilePath == null) {
2430         return false;
2431       }
2432       storeFile = Store.this.commitFile(storeFilePath, cacheFlushId,
2433                                snapshotTimeRangeTracker, flushedSize, status);
2434       if (Store.this.getHRegion().getCoprocessorHost() != null) {
2435         Store.this.getHRegion()
2436             .getCoprocessorHost()
2437             .postFlush(Store.this, storeFile);
2438       }
2439 
2440       // Add new file to store files.  Clear snapshot too while we have
2441       // the Store write lock.
2442       return Store.this.updateStorefiles(storeFile, snapshot);
2443     }
2444   }
2445 
2446   /**
2447    * See if there's too much store files in this store
2448    * @return true if number of store files is greater than
2449    *  the number defined in minFilesToCompact
2450    */
2451   public boolean needsCompaction() {
2452     return (storefiles.size() - filesCompacting.size()) > minFilesToCompact;
2453   }
2454 
2455   /**
2456    * Used for tests. Get the cache configuration for this Store.
2457    */
2458   public CacheConfig getCacheConfig() {
2459     return this.cacheConf;
2460   }
2461 
2462   public static final long FIXED_OVERHEAD =
2463       ClassSize.align(SchemaConfigured.SCHEMA_CONFIGURED_UNALIGNED_HEAP_SIZE +
2464           + (17 * ClassSize.REFERENCE) + (7 * Bytes.SIZEOF_LONG)
2465           + (5 * Bytes.SIZEOF_INT) + Bytes.SIZEOF_BOOLEAN);
2466 
2467   public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD
2468       + ClassSize.OBJECT + ClassSize.REENTRANT_LOCK
2469       + ClassSize.CONCURRENT_SKIPLISTMAP
2470       + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + ClassSize.OBJECT
2471       + ScanInfo.FIXED_OVERHEAD);
2472 
2473   @Override
2474   public long heapSize() {
2475     return DEEP_OVERHEAD + this.memstore.heapSize();
2476   }
2477 
2478   public KeyValue.KVComparator getComparator() {
2479     return comparator;
2480   }
2481 
2482   public ScanInfo getScanInfo() {
2483     return scanInfo;
2484   }
2485   
2486   public boolean hasTooManyStoreFiles() {
2487     return getStorefilesCount() > this.blockingFileCount;
2488   }
2489 
2490   /**
2491    * Immutable information for scans over a store.
2492    */
2493   public static class ScanInfo {
2494     private byte[] family;
2495     private int minVersions;
2496     private int maxVersions;
2497     private long ttl;
2498     private boolean keepDeletedCells;
2499     private long timeToPurgeDeletes;
2500     private KVComparator comparator;
2501 
2502     public static final long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT
2503         + (2 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_INT)
2504         + Bytes.SIZEOF_LONG + Bytes.SIZEOF_BOOLEAN);
2505 
2506     /**
2507      * @param family {@link HColumnDescriptor} describing the column family
2508      * @param ttl Store's TTL (in ms)
2509      * @param timeToPurgeDeletes duration in ms after which a delete marker can
2510      *        be purged during a major compaction.
2511      * @param comparator The store's comparator
2512      */
2513     public ScanInfo(HColumnDescriptor family, long ttl, long timeToPurgeDeletes, KVComparator comparator) {
2514       this(family.getName(), family.getMinVersions(), family.getMaxVersions(), ttl, family
2515           .getKeepDeletedCells(), timeToPurgeDeletes, comparator);
2516     }
2517     /**
2518      * @param family Name of this store's column family
2519      * @param minVersions Store's MIN_VERSIONS setting
2520      * @param maxVersions Store's VERSIONS setting
2521      * @param ttl Store's TTL (in ms)
2522      * @param timeToPurgeDeletes duration in ms after which a delete marker can
2523      *        be purged during a major compaction.
2524      * @param keepDeletedCells Store's keepDeletedCells setting
2525      * @param comparator The store's comparator
2526      */
2527     public ScanInfo(byte[] family, int minVersions, int maxVersions, long ttl,
2528         boolean keepDeletedCells, long timeToPurgeDeletes,
2529         KVComparator comparator) {
2530 
2531       this.family = family;
2532       this.minVersions = minVersions;
2533       this.maxVersions = maxVersions;
2534       this.ttl = ttl;
2535       this.keepDeletedCells = keepDeletedCells;
2536       this.timeToPurgeDeletes = timeToPurgeDeletes;
2537       this.comparator = comparator;
2538     }
2539 
2540     public byte[] getFamily() {
2541       return family;
2542     }
2543 
2544     public int getMinVersions() {
2545       return minVersions;
2546     }
2547 
2548     public int getMaxVersions() {
2549       return maxVersions;
2550     }
2551 
2552     public long getTtl() {
2553       return ttl;
2554     }
2555 
2556     public boolean getKeepDeletedCells() {
2557       return keepDeletedCells;
2558     }
2559 
2560     public long getTimeToPurgeDeletes() {
2561       return timeToPurgeDeletes;
2562     }
2563 
2564     public KVComparator getComparator() {
2565       return comparator;
2566     }
2567   }
2568 
2569 }