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