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