View Javadoc

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