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.SortedSet;
30  import java.util.concurrent.CopyOnWriteArraySet;
31  import java.util.concurrent.locks.ReentrantReadWriteLock;
32  
33  import org.apache.commons.logging.Log;
34  import org.apache.commons.logging.LogFactory;
35  import org.apache.hadoop.conf.Configuration;
36  import org.apache.hadoop.fs.FileStatus;
37  import org.apache.hadoop.fs.FileSystem;
38  import org.apache.hadoop.fs.FileUtil;
39  import org.apache.hadoop.fs.Path;
40  import org.apache.hadoop.hbase.HColumnDescriptor;
41  import org.apache.hadoop.hbase.HConstants;
42  import org.apache.hadoop.hbase.HRegionInfo;
43  import org.apache.hadoop.hbase.KeyValue;
44  import org.apache.hadoop.hbase.RemoteExceptionHandler;
45  import org.apache.hadoop.hbase.client.Scan;
46  import org.apache.hadoop.hbase.io.HeapSize;
47  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
48  import org.apache.hadoop.hbase.io.hfile.Compression;
49  import org.apache.hadoop.hbase.io.hfile.HFile;
50  import org.apache.hadoop.hbase.io.hfile.HFileScanner;
51  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
52  import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
53  import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
54  import org.apache.hadoop.hbase.util.Bytes;
55  import org.apache.hadoop.hbase.util.ClassSize;
56  import org.apache.hadoop.hbase.util.CollectionBackedScanner;
57  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
58  import org.apache.hadoop.hbase.util.FSUtils;
59  import org.apache.hadoop.util.StringUtils;
60  
61  import com.google.common.base.Preconditions;
62  import com.google.common.collect.ImmutableList;
63  import com.google.common.collect.Lists;
64  
65  /**
66   * A Store holds a column family in a Region.  Its a memstore and a set of zero
67   * or more StoreFiles, which stretch backwards over time.
68   *
69   * <p>There's no reason to consider append-logging at this level; all logging
70   * and locking is handled at the HRegion level.  Store just provides
71   * services to manage sets of StoreFiles.  One of the most important of those
72   * services is compaction services where files are aggregated once they pass
73   * a configurable threshold.
74   *
75   * <p>The only thing having to do with logs that Store needs to deal with is
76   * the reconstructionLog.  This is a segment of an HRegion's log that might
77   * NOT be present upon startup.  If the param is NULL, there's nothing to do.
78   * If the param is non-NULL, we need to process the log to reconstruct
79   * a TreeMap that might not have been written to disk before the process
80   * died.
81   *
82   * <p>It's assumed that after this constructor returns, the reconstructionLog
83   * file will be deleted (by whoever has instantiated the Store).
84   *
85   * <p>Locking and transactions are handled at a higher level.  This API should
86   * not be called directly but by an HRegion manager.
87   */
88  public class Store implements HeapSize {
89    static final Log LOG = LogFactory.getLog(Store.class);
90    protected final MemStore memstore;
91    // This stores directory in the filesystem.
92    private final Path homedir;
93    private final HRegion region;
94    private final HColumnDescriptor family;
95    final FileSystem fs;
96    final Configuration conf;
97    final CacheConfig cacheConf;
98    // ttl in milliseconds.
99    protected long ttl;
100   protected int minVersions;
101   protected int maxVersions;
102   long majorCompactionTime;
103   private final int minFilesToCompact;
104   private final int maxFilesToCompact;
105   private final long minCompactSize;
106   private final long maxCompactSize;
107   // compactRatio: double on purpose!  Float.MAX < Long.MAX < Double.MAX
108   // With float, java will downcast your long to float for comparisons (bad)
109   private double compactRatio;
110   private long lastCompactSize = 0;
111   volatile boolean forceMajor = false;
112   /* how many bytes to write between status checks */
113   static int closeCheckInterval = 0;
114   private final int blockingStoreFileCount;
115   private volatile long storeSize = 0L;
116   private volatile long totalUncompressedBytes = 0L;
117   private final Object flushLock = new Object();
118   final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
119   private final String storeNameStr;
120   private CompactionProgress progress;
121   private final int compactionKVMax;
122 
123   /*
124    * List of store files inside this store. This is an immutable list that
125    * is atomically replaced when its contents change.
126    */
127   private ImmutableList<StoreFile> storefiles = null;
128 
129   List<StoreFile> filesCompacting = Lists.newArrayList();
130 
131   // All access must be synchronized.
132   private final CopyOnWriteArraySet<ChangedReadersObserver> changedReaderObservers =
133     new CopyOnWriteArraySet<ChangedReadersObserver>();
134 
135   private final int blocksize;
136   /** Compression algorithm for flush files and minor compaction */
137   private final Compression.Algorithm compression;
138   /** Compression algorithm for major compaction */
139   private final Compression.Algorithm compactionCompression;
140 
141   // Comparing KeyValues
142   final KeyValue.KVComparator comparator;
143 
144   /**
145    * Constructor
146    * @param basedir qualified path under which the region directory lives;
147    * generally the table subdirectory
148    * @param region
149    * @param family HColumnDescriptor for this column
150    * @param fs file system object
151    * @param conf configuration object
152    * failed.  Can be null.
153    * @throws IOException
154    */
155   protected Store(Path basedir, HRegion region, HColumnDescriptor family,
156     FileSystem fs, Configuration conf)
157   throws IOException {
158     HRegionInfo info = region.regionInfo;
159     this.fs = fs;
160     this.homedir = getStoreHomedir(basedir, info.getEncodedName(), family.getName());
161     if (!this.fs.exists(this.homedir)) {
162       if (!this.fs.mkdirs(this.homedir))
163         throw new IOException("Failed create of: " + this.homedir.toString());
164     }
165     this.region = region;
166     this.family = family;
167     this.conf = conf;
168     this.blocksize = family.getBlocksize();
169     this.compression = family.getCompression();
170     // avoid overriding compression setting for major compactions if the user
171     // has not specified it separately
172     this.compactionCompression =
173       (family.getCompactionCompression() != Compression.Algorithm.NONE) ?
174         family.getCompactionCompression() : this.compression;
175     this.comparator = info.getComparator();
176     // getTimeToLive returns ttl in seconds.  Convert to milliseconds.
177     this.ttl = family.getTimeToLive();
178     if (ttl == HConstants.FOREVER) {
179       // default is unlimited ttl.
180       ttl = Long.MAX_VALUE;
181     } else if (ttl == -1) {
182       ttl = Long.MAX_VALUE;
183     } else {
184       // second -> ms adjust for user data
185       this.ttl *= 1000;
186     }
187     this.minVersions = family.getMinVersions();
188     this.maxVersions = family.getMaxVersions();
189     this.memstore = new MemStore(conf, this.comparator);
190     this.storeNameStr = Bytes.toString(this.family.getName());
191 
192     // By default, compact if storefile.count >= minFilesToCompact
193     this.minFilesToCompact = Math.max(2,
194       conf.getInt("hbase.hstore.compaction.min",
195         /*old name*/ conf.getInt("hbase.hstore.compactionThreshold", 3)));
196 
197     // Setting up cache configuration for this family
198     this.cacheConf = new CacheConfig(conf, family);
199     this.blockingStoreFileCount =
200       conf.getInt("hbase.hstore.blockingStoreFiles", 7);
201 
202     this.majorCompactionTime = getNextMajorCompactTime();
203 
204     this.maxFilesToCompact = conf.getInt("hbase.hstore.compaction.max", 10);
205     this.minCompactSize = conf.getLong("hbase.hstore.compaction.min.size",
206       this.region.memstoreFlushSize);
207     this.maxCompactSize
208       = conf.getLong("hbase.hstore.compaction.max.size", Long.MAX_VALUE);
209     this.compactRatio = conf.getFloat("hbase.hstore.compaction.ratio", 1.2F);
210     this.compactionKVMax = conf.getInt("hbase.hstore.compaction.kv.max", 10);
211 
212     if (Store.closeCheckInterval == 0) {
213       Store.closeCheckInterval = conf.getInt(
214           "hbase.hstore.close.check.interval", 10*1000*1000 /* 10 MB */);
215     }
216     this.storefiles = sortAndClone(loadStoreFiles());
217   }
218 
219   public HColumnDescriptor getFamily() {
220     return this.family;
221   }
222 
223   /**
224    * @return The maximum sequence id in all store files.
225    */
226   long getMaxSequenceId() {
227     return StoreFile.getMaxSequenceIdInList(this.getStorefiles());
228   }
229 
230   /**
231    * @return The maximum memstoreTS in all store files.
232    */
233   public long getMaxMemstoreTS() {
234     return StoreFile.getMaxMemstoreTSInList(this.getStorefiles());
235   }
236 
237   /**
238    * @param tabledir
239    * @param encodedName Encoded region name.
240    * @param family
241    * @return Path to family/Store home directory.
242    */
243   public static Path getStoreHomedir(final Path tabledir,
244       final String encodedName, final byte [] family) {
245     return new Path(tabledir, new Path(encodedName,
246       new Path(Bytes.toString(family))));
247   }
248 
249   /**
250    * Return the directory in which this store stores its
251    * StoreFiles
252    */
253   public Path getHomedir() {
254     return homedir;
255   }
256 
257   /*
258    * Creates an unsorted list of StoreFile loaded from the given directory.
259    * @throws IOException
260    */
261   private List<StoreFile> loadStoreFiles()
262   throws IOException {
263     ArrayList<StoreFile> results = new ArrayList<StoreFile>();
264     FileStatus files[] = FSUtils.listStatus(this.fs, this.homedir, null);
265     for (int i = 0; files != null && i < files.length; i++) {
266       // Skip directories.
267       if (files[i].isDir()) {
268         continue;
269       }
270       Path p = files[i].getPath();
271       // Check for empty file.  Should never be the case but can happen
272       // after data loss in hdfs for whatever reason (upgrade, etc.): HBASE-646
273       if (this.fs.getFileStatus(p).getLen() <= 0) {
274         LOG.warn("Skipping " + p + " because its empty. HBASE-646 DATA LOSS?");
275         continue;
276       }
277       StoreFile curfile = new StoreFile(fs, p, this.conf, this.cacheConf,
278           this.family.getBloomFilterType());
279       curfile.createReader();
280       long length = curfile.getReader().length();
281       this.storeSize += length;
282       this.totalUncompressedBytes += curfile.getReader().getTotalUncompressedBytes();
283       if (LOG.isDebugEnabled()) {
284         LOG.debug("loaded " + curfile.toStringDetailed());
285       }
286       results.add(curfile);
287     }
288     return results;
289   }
290 
291   /**
292    * Adds a value to the memstore
293    *
294    * @param kv
295    * @return memstore size delta
296    */
297   protected long add(final KeyValue kv) {
298     lock.readLock().lock();
299     try {
300       return this.memstore.add(kv);
301     } finally {
302       lock.readLock().unlock();
303     }
304   }
305 
306   /**
307    * Adds a value to the memstore
308    *
309    * @param kv
310    * @return memstore size delta
311    */
312   protected long delete(final KeyValue kv) {
313     lock.readLock().lock();
314     try {
315       return this.memstore.delete(kv);
316     } finally {
317       lock.readLock().unlock();
318     }
319   }
320 
321   /**
322    * @return All store files.
323    */
324   List<StoreFile> getStorefiles() {
325     return this.storefiles;
326   }
327 
328   /**
329    * This throws a WrongRegionException if the bulkHFile does not fit in this
330    * region.
331    *
332    */
333   void assertBulkLoadHFileOk(Path srcPath) throws IOException {
334     HFile.Reader reader  = null;
335     try {
336       LOG.info("Validating hfile at " + srcPath + " for inclusion in "
337           + "store " + this + " region " + this.region);
338       reader = HFile.createReader(srcPath.getFileSystem(conf),
339           srcPath, cacheConf);
340       reader.loadFileInfo();
341 
342       byte[] firstKey = reader.getFirstRowKey();
343       byte[] lk = reader.getLastKey();
344       byte[] lastKey =
345           (lk == null) ? null :
346               KeyValue.createKeyValueFromKey(lk).getRow();
347 
348       LOG.debug("HFile bounds: first=" + Bytes.toStringBinary(firstKey) +
349           " last=" + Bytes.toStringBinary(lastKey));
350       LOG.debug("Region bounds: first=" +
351           Bytes.toStringBinary(region.getStartKey()) +
352           " last=" + Bytes.toStringBinary(region.getEndKey()));
353 
354       HRegionInfo hri = region.getRegionInfo();
355       if (!hri.containsRange(firstKey, lastKey)) {
356         throw new WrongRegionException(
357             "Bulk load file " + srcPath.toString() + " does not fit inside region "
358             + this.region);
359       }
360     } finally {
361       if (reader != null) reader.close();
362     }
363   }
364 
365   /**
366    * This method should only be called from HRegion.  It is assumed that the 
367    * ranges of values in the HFile fit within the stores assigned region. 
368    * (assertBulkLoadHFileOk checks this)
369    */
370   void bulkLoadHFile(String srcPathStr) throws IOException {
371     Path srcPath = new Path(srcPathStr);
372 
373     // Move the file if it's on another filesystem
374     FileSystem srcFs = srcPath.getFileSystem(conf);
375     if (!srcFs.equals(fs)) {
376       LOG.info("File " + srcPath + " on different filesystem than " +
377           "destination store - moving to this filesystem.");
378       Path tmpPath = getTmpPath();
379       FileUtil.copy(srcFs, srcPath, fs, tmpPath, false, conf);
380       LOG.info("Copied to temporary path on dst filesystem: " + tmpPath);
381       srcPath = tmpPath;
382     }
383 
384     Path dstPath = StoreFile.getRandomFilename(fs, homedir);
385     LOG.debug("Renaming bulk load file " + srcPath + " to " + dstPath);
386     StoreFile.rename(fs, srcPath, dstPath);
387 
388     StoreFile sf = new StoreFile(fs, dstPath, this.conf, this.cacheConf,
389         this.family.getBloomFilterType());
390     sf.createReader();
391 
392     LOG.info("Moved hfile " + srcPath + " into store directory " +
393         homedir + " - updating store file list.");
394 
395     // Append the new storefile into the list
396     this.lock.writeLock().lock();
397     try {
398       ArrayList<StoreFile> newFiles = new ArrayList<StoreFile>(storefiles);
399       newFiles.add(sf);
400       this.storefiles = sortAndClone(newFiles);
401     } finally {
402       // We need the lock, as long as we are updating the storefiles
403       // or changing the memstore. Let us release it before calling
404       // notifyChangeReadersObservers. See HBASE-4485 for a possible
405       // deadlock scenario that could have happened if continue to hold
406       // the lock.
407       this.lock.writeLock().unlock();
408     }
409     notifyChangedReadersObservers();
410     LOG.info("Successfully loaded store file " + srcPath
411         + " into store " + this + " (new location: " + dstPath + ")");
412   }
413 
414   /**
415    * Get a temporary path in this region. These temporary files
416    * will get cleaned up when the region is re-opened if they are
417    * still around.
418    */
419   private Path getTmpPath() throws IOException {
420     return StoreFile.getRandomFilename(
421         fs, region.getTmpDir());
422   }
423 
424   /**
425    * Close all the readers
426    *
427    * We don't need to worry about subsequent requests because the HRegion holds
428    * a write lock that will prevent any more reads or writes.
429    *
430    * @throws IOException
431    */
432   ImmutableList<StoreFile> close() throws IOException {
433     this.lock.writeLock().lock();
434     try {
435       ImmutableList<StoreFile> result = storefiles;
436 
437       // Clear so metrics doesn't find them.
438       storefiles = ImmutableList.of();
439 
440       for (StoreFile f: result) {
441         f.closeReader(true);
442       }
443       LOG.debug("closed " + this.storeNameStr);
444       return result;
445     } finally {
446       this.lock.writeLock().unlock();
447     }
448   }
449 
450   /**
451    * Snapshot this stores memstore.  Call before running
452    * {@link #flushCache(long, SortedSet<KeyValue>)} so it has some work to do.
453    */
454   void snapshot() {
455     this.memstore.snapshot();
456   }
457 
458   /**
459    * Write out current snapshot.  Presumes {@link #snapshot()} has been called
460    * previously.
461    * @param logCacheFlushId flush sequence number
462    * @param snapshot
463    * @param snapshotTimeRangeTracker
464    * @return true if a compaction is needed
465    * @throws IOException
466    */
467   private StoreFile flushCache(final long logCacheFlushId,
468       SortedSet<KeyValue> snapshot,
469       TimeRangeTracker snapshotTimeRangeTracker,
470       MonitoredTask status) throws IOException {
471     // If an exception happens flushing, we let it out without clearing
472     // the memstore snapshot.  The old snapshot will be returned when we say
473     // 'snapshot', the next time flush comes around.
474     return internalFlushCache(
475         snapshot, logCacheFlushId, snapshotTimeRangeTracker, status);
476   }
477 
478   /*
479    * @param cache
480    * @param logCacheFlushId
481    * @return StoreFile created.
482    * @throws IOException
483    */
484   private StoreFile internalFlushCache(final SortedSet<KeyValue> set,
485       final long logCacheFlushId,
486       TimeRangeTracker snapshotTimeRangeTracker,
487       MonitoredTask status)
488       throws IOException {
489     StoreFile.Writer writer;
490     String fileName;
491     // Find the smallest read point across all the Scanners.
492     long smallestReadPoint = region.getSmallestReadPoint();
493     long flushed = 0;
494     // Don't flush if there are no entries.
495     if (set.size() == 0) {
496       return null;
497     }
498     Scan scan = new Scan();
499     scan.setMaxVersions(maxVersions);
500     // Use a store scanner to find which rows to flush.
501     // Note that we need to retain deletes, hence
502     // pass true as the StoreScanner's retainDeletesInOutput argument.
503     InternalScanner scanner = new StoreScanner(this, scan,
504         Collections.singletonList(new CollectionBackedScanner(set,
505             this.comparator)), true, this.region.getSmallestReadPoint());
506     try {
507       // TODO:  We can fail in the below block before we complete adding this
508       // flush to list of store files.  Add cleanup of anything put on filesystem
509       // if we fail.
510       synchronized (flushLock) {
511         status.setStatus("Flushing " + this + ": creating writer");
512         // A. Write the map out to the disk
513         writer = createWriterInTmp(set.size());
514         writer.setTimeRangeTracker(snapshotTimeRangeTracker);
515         fileName = writer.getPath().getName();
516         try {
517           List<KeyValue> kvs = new ArrayList<KeyValue>();
518           boolean hasMore;
519           do {
520             hasMore = scanner.next(kvs);
521             if (!kvs.isEmpty()) {
522               for (KeyValue kv : kvs) {
523                 // If we know that this KV is going to be included always, then let us
524                 // set its memstoreTS to 0. This will help us save space when writing to disk.
525                 if (kv.getMemstoreTS() <= smallestReadPoint) {
526                   // let us not change the original KV. It could be in the memstore
527                   // changing its memstoreTS could affect other threads/scanners.
528                   kv = kv.shallowCopy();
529                   kv.setMemstoreTS(0);
530                 }
531                 writer.append(kv);
532                 flushed += this.memstore.heapSizeChange(kv, true);
533               }
534               kvs.clear();
535             }
536           } while (hasMore);
537         } finally {
538           // Write out the log sequence number that corresponds to this output
539           // hfile.  The hfile is current up to and including logCacheFlushId.
540           status.setStatus("Flushing " + this + ": appending metadata");
541           writer.appendMetadata(logCacheFlushId, false);
542           status.setStatus("Flushing " + this + ": closing flushed file");
543           writer.close();
544         }
545       }
546     } finally {
547       scanner.close();
548     }
549 
550     // Write-out finished successfully, move into the right spot
551     Path dstPath = new Path(homedir, fileName);
552     validateStoreFile(writer.getPath());
553     String msg = "Renaming flushed file at " + writer.getPath() + " to " + dstPath;
554     LOG.debug(msg);
555     status.setStatus("Flushing " + this + ": " + msg);
556     if (!fs.rename(writer.getPath(), dstPath)) {
557       LOG.warn("Unable to rename " + writer.getPath() + " to " + dstPath);
558     }
559 
560     status.setStatus("Flushing " + this + ": reopening flushed file");
561     StoreFile sf = new StoreFile(this.fs, dstPath, this.conf, this.cacheConf,
562         this.family.getBloomFilterType());
563     StoreFile.Reader r = sf.createReader();
564     this.storeSize += r.length();
565     this.totalUncompressedBytes += r.getTotalUncompressedBytes();
566     if (LOG.isInfoEnabled()) {
567       LOG.info("Added " + sf + ", entries=" + r.getEntries() +
568         ", sequenceid=" + logCacheFlushId +
569         ", memsize=" + StringUtils.humanReadableInt(flushed) +
570         ", filesize=" + StringUtils.humanReadableInt(r.length()));
571     }
572     return sf;
573   }
574 
575   /*
576    * @param maxKeyCount
577    * @return Writer for a new StoreFile in the tmp dir.
578    */
579   private StoreFile.Writer createWriterInTmp(int maxKeyCount)
580   throws IOException {
581     return createWriterInTmp(maxKeyCount, this.compression);
582   }
583 
584   /*
585    * @param maxKeyCount
586    * @param compression Compression algorithm to use
587    * @return Writer for a new StoreFile in the tmp dir.
588    */
589   private StoreFile.Writer createWriterInTmp(int maxKeyCount,
590     Compression.Algorithm compression)
591   throws IOException {
592     return StoreFile.createWriter(this.fs, region.getTmpDir(), this.blocksize,
593         compression, this.comparator, this.conf, this.cacheConf,
594         this.family.getBloomFilterType(), maxKeyCount);
595   }
596 
597   /*
598    * Change storefiles adding into place the Reader produced by this new flush.
599    * @param sf
600    * @param set That was used to make the passed file <code>p</code>.
601    * @throws IOException
602    * @return Whether compaction is required.
603    */
604   private boolean updateStorefiles(final StoreFile sf,
605                                    final SortedSet<KeyValue> set)
606   throws IOException {
607     this.lock.writeLock().lock();
608     try {
609       ArrayList<StoreFile> newList = new ArrayList<StoreFile>(storefiles);
610       newList.add(sf);
611       storefiles = sortAndClone(newList);
612 
613       this.memstore.clearSnapshot(set);
614     } finally {
615       // We need the lock, as long as we are updating the storefiles
616       // or changing the memstore. Let us release it before calling
617       // notifyChangeReadersObservers. See HBASE-4485 for a possible
618       // deadlock scenario that could have happened if continue to hold
619       // the lock.
620       this.lock.writeLock().unlock();
621     }
622 
623     // Tell listeners of the change in readers.
624     notifyChangedReadersObservers();
625 
626     return needsCompaction();
627   }
628 
629   /*
630    * Notify all observers that set of Readers has changed.
631    * @throws IOException
632    */
633   private void notifyChangedReadersObservers() throws IOException {
634     for (ChangedReadersObserver o: this.changedReaderObservers) {
635       o.updateReaders();
636     }
637   }
638 
639   protected List<KeyValueScanner> getScanners(boolean cacheBlocks,
640       boolean isGet,
641       boolean isCompaction) throws IOException {
642     List<StoreFile> storeFiles;
643     List<KeyValueScanner> memStoreScanners;
644     this.lock.readLock().lock();
645     try {
646       storeFiles = this.getStorefiles();
647       memStoreScanners = this.memstore.getScanners();
648     } finally {
649       this.lock.readLock().unlock();
650     }
651 
652     // First the store file scanners
653 
654     // TODO this used to get the store files in descending order,
655     // but now we get them in ascending order, which I think is
656     // actually more correct, since memstore get put at the end.
657     List<StoreFileScanner> sfScanners = StoreFileScanner
658       .getScannersForStoreFiles(storeFiles, cacheBlocks, isGet, isCompaction);
659     List<KeyValueScanner> scanners =
660       new ArrayList<KeyValueScanner>(sfScanners.size()+1);
661     scanners.addAll(sfScanners);
662     // Then the memstore scanners
663     scanners.addAll(memStoreScanners);
664     return scanners;
665   }
666 
667   /*
668    * @param o Observer who wants to know about changes in set of Readers
669    */
670   void addChangedReaderObserver(ChangedReadersObserver o) {
671     this.changedReaderObservers.add(o);
672   }
673 
674   /*
675    * @param o Observer no longer interested in changes in set of Readers.
676    */
677   void deleteChangedReaderObserver(ChangedReadersObserver o) {
678     // We don't check if observer present; it may not be (legitimately)
679     this.changedReaderObservers.remove(o);
680   }
681 
682   //////////////////////////////////////////////////////////////////////////////
683   // Compaction
684   //////////////////////////////////////////////////////////////////////////////
685 
686   /**
687    * Compact the StoreFiles.  This method may take some time, so the calling
688    * thread must be able to block for long periods.
689    *
690    * <p>During this time, the Store can work as usual, getting values from
691    * StoreFiles and writing new StoreFiles from the memstore.
692    *
693    * Existing StoreFiles are not destroyed until the new compacted StoreFile is
694    * completely written-out to disk.
695    *
696    * <p>The compactLock prevents multiple simultaneous compactions.
697    * The structureLock prevents us from interfering with other write operations.
698    *
699    * <p>We don't want to hold the structureLock for the whole time, as a compact()
700    * can be lengthy and we want to allow cache-flushes during this period.
701    *
702    * @param CompactionRequest
703    *          compaction details obtained from requestCompaction()
704    * @throws IOException
705    */
706   void compact(CompactionRequest cr) throws IOException {
707     if (cr == null || cr.getFiles().isEmpty()) {
708       return;
709     }
710     Preconditions.checkArgument(cr.getStore().toString()
711         .equals(this.toString()));
712 
713     List<StoreFile> filesToCompact = cr.getFiles();
714 
715     synchronized (filesCompacting) {
716       // sanity check: we're compacting files that this store knows about
717       // TODO: change this to LOG.error() after more debugging
718       Preconditions.checkArgument(filesCompacting.containsAll(filesToCompact));
719     }
720 
721     // Max-sequenceID is the last key in the files we're compacting
722     long maxId = StoreFile.getMaxSequenceIdInList(filesToCompact);
723 
724     // Ready to go. Have list of files to compact.
725     LOG.info("Starting compaction of " + filesToCompact.size() + " file(s) in "
726         + this.storeNameStr + " of "
727         + this.region.getRegionInfo().getRegionNameAsString()
728         + " into tmpdir=" + region.getTmpDir() + ", seqid=" + maxId + ", totalSize="
729         + StringUtils.humanReadableInt(cr.getSize()));
730 
731     StoreFile sf = null;
732     try {
733       StoreFile.Writer writer = compactStore(filesToCompact, cr.isMajor(),
734           maxId);
735       // Move the compaction into place.
736       sf = completeCompaction(filesToCompact, writer);
737       if (region.getCoprocessorHost() != null) {
738         region.getCoprocessorHost().postCompact(this, sf);
739       }
740     } finally {
741       synchronized (filesCompacting) {
742         filesCompacting.removeAll(filesToCompact);
743       }
744     }
745 
746     LOG.info("Completed" + (cr.isMajor() ? " major " : " ") + "compaction of "
747         + filesToCompact.size() + " file(s) in " + this.storeNameStr + " of "
748         + this.region.getRegionInfo().getRegionNameAsString()
749         + " into " +
750         (sf == null ? "none" : sf.getPath().getName()) +
751         ", size=" + (sf == null ? "none" :
752           StringUtils.humanReadableInt(sf.getReader().length()))
753         + "; total size for store is "
754         + StringUtils.humanReadableInt(storeSize));
755   }
756 
757   /*
758    * Compact the most recent N files. Essentially a hook for testing.
759    */
760   protected void compactRecent(int N) throws IOException {
761     List<StoreFile> filesToCompact;
762     long maxId;
763     boolean isMajor;
764 
765     this.lock.readLock().lock();
766     try {
767       synchronized (filesCompacting) {
768         filesToCompact = Lists.newArrayList(storefiles);
769         if (!filesCompacting.isEmpty()) {
770           // exclude all files older than the newest file we're currently
771           // compacting. this allows us to preserve contiguity (HBASE-2856)
772           StoreFile last = filesCompacting.get(filesCompacting.size() - 1);
773           int idx = filesToCompact.indexOf(last);
774           Preconditions.checkArgument(idx != -1);
775           filesToCompact.subList(0, idx + 1).clear();
776         }
777         int count = filesToCompact.size();
778         if (N > count) {
779           throw new RuntimeException("Not enough files");
780         }
781 
782         filesToCompact = filesToCompact.subList(count - N, count);
783         maxId = StoreFile.getMaxSequenceIdInList(filesToCompact);
784         isMajor = (filesToCompact.size() == storefiles.size());
785         filesCompacting.addAll(filesToCompact);
786         Collections.sort(filesCompacting, StoreFile.Comparators.FLUSH_TIME);
787       }
788     } finally {
789       this.lock.readLock().unlock();
790     }
791 
792     try {
793       // Ready to go. Have list of files to compact.
794       StoreFile.Writer writer = compactStore(filesToCompact, isMajor, maxId);
795       // Move the compaction into place.
796       StoreFile sf = completeCompaction(filesToCompact, writer);
797       if (region.getCoprocessorHost() != null) {
798         region.getCoprocessorHost().postCompact(this, sf);
799       }
800     } finally {
801       synchronized (filesCompacting) {
802         filesCompacting.removeAll(filesToCompact);
803       }
804     }
805   }
806 
807   boolean hasReferences() {
808     return hasReferences(this.storefiles);
809   }
810 
811   /*
812    * @param files
813    * @return True if any of the files in <code>files</code> are References.
814    */
815   private boolean hasReferences(Collection<StoreFile> files) {
816     if (files != null && files.size() > 0) {
817       for (StoreFile hsf: files) {
818         if (hsf.isReference()) {
819           return true;
820         }
821       }
822     }
823     return false;
824   }
825 
826   /*
827    * Gets lowest timestamp from candidate StoreFiles
828    *
829    * @param fs
830    * @param dir
831    * @throws IOException
832    */
833   public static long getLowestTimestamp(final List<StoreFile> candidates)
834       throws IOException {
835     long minTs = Long.MAX_VALUE;
836     for (StoreFile storeFile : candidates) {
837       minTs = Math.min(minTs, storeFile.getModificationTimeStamp());
838     }
839     return minTs;
840   }
841 
842   /** getter for CompactionProgress object
843    * @return CompactionProgress object
844    */
845   public CompactionProgress getCompactionProgress() {
846     return this.progress;
847   }
848 
849   /*
850    * @return True if we should run a major compaction.
851    */
852   boolean isMajorCompaction() throws IOException {
853     for (StoreFile sf : this.storefiles) {
854       if (sf.getReader() == null) {
855         LOG.debug("StoreFile " + sf + " has null Reader");
856         return false;
857       }
858     }
859 
860     List<StoreFile> candidates = new ArrayList<StoreFile>(this.storefiles);
861 
862     // exclude files above the max compaction threshold
863     // except: save all references. we MUST compact them
864     int pos = 0;
865     while (pos < candidates.size() &&
866            candidates.get(pos).getReader().length() > this.maxCompactSize &&
867            !candidates.get(pos).isReference()) ++pos;
868     candidates.subList(0, pos).clear();
869 
870     return isMajorCompaction(candidates);
871   }
872 
873   /*
874    * @param filesToCompact Files to compact. Can be null.
875    * @return True if we should run a major compaction.
876    */
877   private boolean isMajorCompaction(final List<StoreFile> filesToCompact) throws IOException {
878     boolean result = false;
879     if (filesToCompact == null || filesToCompact.isEmpty() ||
880         majorCompactionTime == 0) {
881       return result;
882         }
883     // TODO: Use better method for determining stamp of last major (HBASE-2990)
884     long lowTimestamp = getLowestTimestamp(filesToCompact);
885     long now = System.currentTimeMillis();
886     if (lowTimestamp > 0l && lowTimestamp < (now - this.majorCompactionTime)) {
887       // Major compaction time has elapsed.
888       if (filesToCompact.size() == 1) {
889         // Single file
890         StoreFile sf = filesToCompact.get(0);
891         long oldest =
892             (sf.getReader().timeRangeTracker == null) ?
893                 Long.MIN_VALUE :
894                 now - sf.getReader().timeRangeTracker.minimumTimestamp;
895         if (sf.isMajorCompaction() &&
896             (this.ttl == HConstants.FOREVER || oldest < this.ttl)) {
897           if (LOG.isDebugEnabled()) {
898             LOG.debug("Skipping major compaction of " + this.storeNameStr +
899                 " because one (major) compacted file only and oldestTime " +
900                 oldest + "ms is < ttl=" + this.ttl);
901           }
902         } else if (this.ttl != HConstants.FOREVER && oldest > this.ttl) {
903           LOG.debug("Major compaction triggered on store " + this.storeNameStr +
904             ", because keyvalues outdated; time since last major compaction " +
905             (now - lowTimestamp) + "ms");
906           result = true;
907         }
908       } else {
909         if (LOG.isDebugEnabled()) {
910           LOG.debug("Major compaction triggered on store " + this.storeNameStr +
911               "; time since last major compaction " + (now - lowTimestamp) + "ms");
912         }
913         result = true;
914       }
915     }
916     return result;
917   }
918 
919   long getNextMajorCompactTime() {
920     // default = 24hrs
921     long ret = conf.getLong(HConstants.MAJOR_COMPACTION_PERIOD, 1000*60*60*24);
922     if (family.getValue(HConstants.MAJOR_COMPACTION_PERIOD) != null) {
923       String strCompactionTime =
924         family.getValue(HConstants.MAJOR_COMPACTION_PERIOD);
925       ret = (new Long(strCompactionTime)).longValue();
926     }
927 
928     if (ret > 0) {
929       // default = 20% = +/- 4.8 hrs
930       double jitterPct =  conf.getFloat("hbase.hregion.majorcompaction.jitter",
931           0.20F);
932       if (jitterPct > 0) {
933         long jitter = Math.round(ret * jitterPct);
934         ret += jitter - Math.round(2L * jitter * Math.random());
935       }
936     }
937     return ret;
938   }
939 
940   public CompactionRequest requestCompaction() {
941     // don't even select for compaction if writes are disabled
942     if (!this.region.areWritesEnabled()) {
943       return null;
944     }
945 
946     CompactionRequest ret = null;
947     this.lock.readLock().lock();
948     try {
949       synchronized (filesCompacting) {
950         // candidates = all storefiles not already in compaction queue
951         List<StoreFile> candidates = Lists.newArrayList(storefiles);
952         if (!filesCompacting.isEmpty()) {
953           // exclude all files older than the newest file we're currently
954           // compacting. this allows us to preserve contiguity (HBASE-2856)
955           StoreFile last = filesCompacting.get(filesCompacting.size() - 1);
956           int idx = candidates.indexOf(last);
957           Preconditions.checkArgument(idx != -1);
958           candidates.subList(0, idx + 1).clear();
959         }
960 
961         boolean override = false;
962         if (region.getCoprocessorHost() != null) {
963           override = region.getCoprocessorHost().preCompactSelection(
964               this, candidates);
965         }
966         List<StoreFile> filesToCompact;
967         if (override) {
968           // coprocessor is overriding normal file selection
969           filesToCompact = candidates;
970         } else {
971           filesToCompact = compactSelection(candidates);
972         }
973 
974         if (region.getCoprocessorHost() != null) {
975           region.getCoprocessorHost().postCompactSelection(this,
976               ImmutableList.copyOf(filesToCompact));
977         }
978 
979         // no files to compact
980         if (filesToCompact.isEmpty()) {
981           return null;
982         }
983 
984         // basic sanity check: do not try to compact the same StoreFile twice.
985         if (!Collections.disjoint(filesCompacting, filesToCompact)) {
986           // TODO: change this from an IAE to LOG.error after sufficient testing
987           Preconditions.checkArgument(false, "%s overlaps with %s",
988               filesToCompact, filesCompacting);
989         }
990         filesCompacting.addAll(filesToCompact);
991         Collections.sort(filesCompacting, StoreFile.Comparators.FLUSH_TIME);
992 
993         // major compaction iff all StoreFiles are included
994         boolean isMajor = (filesToCompact.size() == this.storefiles.size());
995         if (isMajor) {
996           // since we're enqueuing a major, update the compaction wait interval
997           this.forceMajor = false;
998           this.majorCompactionTime = getNextMajorCompactTime();
999         }
1000 
1001         // everything went better than expected. create a compaction request
1002         int pri = getCompactPriority();
1003         ret = new CompactionRequest(region, this, filesToCompact, isMajor, pri);
1004       }
1005     } catch (IOException ex) {
1006       LOG.error("Compaction Request failed for region " + region + ", store "
1007           + this, RemoteExceptionHandler.checkIOException(ex));
1008     } finally {
1009       this.lock.readLock().unlock();
1010     }
1011     return ret;
1012   }
1013 
1014   public void finishRequest(CompactionRequest cr) {
1015     synchronized (filesCompacting) {
1016       filesCompacting.removeAll(cr.getFiles());
1017     }
1018   }
1019 
1020   /**
1021    * Algorithm to choose which files to compact
1022    *
1023    * Configuration knobs:
1024    *  "hbase.hstore.compaction.ratio"
1025    *    normal case: minor compact when file <= sum(smaller_files) * ratio
1026    *  "hbase.hstore.compaction.min.size"
1027    *    unconditionally compact individual files below this size
1028    *  "hbase.hstore.compaction.max.size"
1029    *    never compact individual files above this size (unless splitting)
1030    *  "hbase.hstore.compaction.min"
1031    *    min files needed to minor compact
1032    *  "hbase.hstore.compaction.max"
1033    *    max files to compact at once (avoids OOM)
1034    *
1035    * @param candidates candidate files, ordered from oldest to newest
1036    * @return subset copy of candidate list that meets compaction criteria
1037    * @throws IOException
1038    */
1039   List<StoreFile> compactSelection(List<StoreFile> candidates)
1040       throws IOException {
1041     // ASSUMPTION!!! filesCompacting is locked when calling this function
1042 
1043     /* normal skew:
1044      *
1045      *         older ----> newer
1046      *     _
1047      *    | |   _
1048      *    | |  | |   _
1049      *  --|-|- |-|- |-|---_-------_-------  minCompactSize
1050      *    | |  | |  | |  | |  _  | |
1051      *    | |  | |  | |  | | | | | |
1052      *    | |  | |  | |  | | | | | |
1053      */
1054     List<StoreFile> filesToCompact = new ArrayList<StoreFile>(candidates);
1055 
1056     boolean forcemajor = this.forceMajor && filesCompacting.isEmpty();
1057     if (!forcemajor) {
1058       // do not compact old files above a configurable threshold
1059       // save all references. we MUST compact them
1060       int pos = 0;
1061       while (pos < filesToCompact.size() &&
1062              filesToCompact.get(pos).getReader().length() > maxCompactSize &&
1063              !filesToCompact.get(pos).isReference()) ++pos;
1064       filesToCompact.subList(0, pos).clear();
1065     }
1066 
1067     if (filesToCompact.isEmpty()) {
1068       LOG.debug(this.getHRegionInfo().getEncodedName() + " - " +
1069         this.storeNameStr + ": no store files to compact");
1070       return filesToCompact;
1071     }
1072 
1073     // major compact on user action or age (caveat: we have too many files)
1074     boolean majorcompaction = filesToCompact.size() < this.maxFilesToCompact
1075       && (forcemajor || isMajorCompaction(filesToCompact));
1076 
1077     if (!majorcompaction && !hasReferences(filesToCompact)) {
1078       // we're doing a minor compaction, let's see what files are applicable
1079       int start = 0;
1080       double r = this.compactRatio;
1081 
1082       // skip selection algorithm if we don't have enough files
1083       if (filesToCompact.size() < this.minFilesToCompact) {
1084         return Collections.emptyList();
1085       }
1086 
1087       /* TODO: add sorting + unit test back in when HBASE-2856 is fixed
1088       // Sort files by size to correct when normal skew is altered by bulk load.
1089       Collections.sort(filesToCompact, StoreFile.Comparators.FILE_SIZE);
1090        */
1091 
1092       // get store file sizes for incremental compacting selection.
1093       int countOfFiles = filesToCompact.size();
1094       long [] fileSizes = new long[countOfFiles];
1095       long [] sumSize = new long[countOfFiles];
1096       for (int i = countOfFiles-1; i >= 0; --i) {
1097         StoreFile file = filesToCompact.get(i);
1098         fileSizes[i] = file.getReader().length();
1099         // calculate the sum of fileSizes[i,i+maxFilesToCompact-1) for algo
1100         int tooFar = i + this.maxFilesToCompact - 1;
1101         sumSize[i] = fileSizes[i]
1102                    + ((i+1    < countOfFiles) ? sumSize[i+1]      : 0)
1103                    - ((tooFar < countOfFiles) ? fileSizes[tooFar] : 0);
1104       }
1105 
1106       /* Start at the oldest file and stop when you find the first file that
1107        * meets compaction criteria:
1108        *   (1) a recently-flushed, small file (i.e. <= minCompactSize)
1109        *      OR
1110        *   (2) within the compactRatio of sum(newer_files)
1111        * Given normal skew, any newer files will also meet this criteria
1112        *
1113        * Additional Note:
1114        * If fileSizes.size() >> maxFilesToCompact, we will recurse on
1115        * compact().  Consider the oldest files first to avoid a
1116        * situation where we always compact [end-threshold,end).  Then, the
1117        * last file becomes an aggregate of the previous compactions.
1118        */
1119       while(countOfFiles - start >= this.minFilesToCompact &&
1120             fileSizes[start] >
1121               Math.max(minCompactSize, (long)(sumSize[start+1] * r))) {
1122         ++start;
1123       }
1124       int end = Math.min(countOfFiles, start + this.maxFilesToCompact);
1125       long totalSize = fileSizes[start]
1126                      + ((start+1 < countOfFiles) ? sumSize[start+1] : 0);
1127       filesToCompact = filesToCompact.subList(start, end);
1128 
1129       // if we don't have enough files to compact, just wait
1130       if (filesToCompact.size() < this.minFilesToCompact) {
1131         if (LOG.isDebugEnabled()) {
1132           LOG.debug("Skipped compaction of " + this.storeNameStr
1133             + ".  Only " + (end - start) + " file(s) of size "
1134             + StringUtils.humanReadableInt(totalSize)
1135             + " have met compaction criteria.");
1136         }
1137         return Collections.emptyList();
1138       }
1139     } else {
1140       // all files included in this compaction, up to max
1141       if (filesToCompact.size() > this.maxFilesToCompact) {
1142         int pastMax = filesToCompact.size() - this.maxFilesToCompact;
1143         filesToCompact.subList(0, pastMax).clear();
1144       }
1145     }
1146     return filesToCompact;
1147   }
1148 
1149   /**
1150    * Do a minor/major compaction on an explicit set of storefiles in a Store.
1151    * Uses the scan infrastructure to make it easy.
1152    *
1153    * @param filesToCompact which files to compact
1154    * @param majorCompaction true to major compact (prune all deletes, max versions, etc)
1155    * @param maxId Readers maximum sequence id.
1156    * @return Product of compaction or null if all cells expired or deleted and
1157    * nothing made it through the compaction.
1158    * @throws IOException
1159    */
1160   StoreFile.Writer compactStore(final Collection<StoreFile> filesToCompact,
1161                                final boolean majorCompaction, final long maxId)
1162       throws IOException {
1163     // calculate maximum key count after compaction (for blooms)
1164     int maxKeyCount = 0;
1165     for (StoreFile file : filesToCompact) {
1166       StoreFile.Reader r = file.getReader();
1167       if (r != null) {
1168         // NOTE: getFilterEntries could cause under-sized blooms if the user
1169         //       switches bloom type (e.g. from ROW to ROWCOL)
1170         long keyCount = (r.getBloomFilterType() == family.getBloomFilterType())
1171           ? r.getFilterEntries() : r.getEntries();
1172         maxKeyCount += keyCount;
1173         if (LOG.isDebugEnabled()) {
1174           LOG.debug("Compacting " + file +
1175             ", keycount=" + keyCount +
1176             ", bloomtype=" + r.getBloomFilterType().toString() +
1177             ", size=" + StringUtils.humanReadableInt(r.length()) );
1178         }
1179       }
1180     }
1181 
1182     // keep track of compaction progress
1183     progress = new CompactionProgress(maxKeyCount);
1184 
1185     // For each file, obtain a scanner:
1186     List<StoreFileScanner> scanners = StoreFileScanner
1187       .getScannersForStoreFiles(filesToCompact, false, false, true);
1188 
1189     // Make the instantiation lazy in case compaction produces no product; i.e.
1190     // where all source cells are expired or deleted.
1191     StoreFile.Writer writer = null;
1192     // Find the smallest read point across all the Scanners.
1193     long smallestReadPoint = region.getSmallestReadPoint();
1194     MultiVersionConsistencyControl.setThreadReadPoint(smallestReadPoint);
1195     try {
1196       InternalScanner scanner = null;
1197       try {
1198         Scan scan = new Scan();
1199         scan.setMaxVersions(family.getMaxVersions());
1200         /* include deletes, unless we are doing a major compaction */
1201         scanner = new StoreScanner(this, scan, scanners, !majorCompaction, smallestReadPoint);
1202         if (region.getCoprocessorHost() != null) {
1203           InternalScanner cpScanner = region.getCoprocessorHost().preCompact(
1204               this, scanner);
1205           // NULL scanner returned from coprocessor hooks means skip normal processing
1206           if (cpScanner == null) {
1207             return null;
1208           }
1209 
1210           scanner = cpScanner;
1211         }
1212 
1213         int bytesWritten = 0;
1214         // since scanner.next() can return 'false' but still be delivering data,
1215         // we have to use a do/while loop.
1216         ArrayList<KeyValue> kvs = new ArrayList<KeyValue>();
1217         // Limit to "hbase.hstore.compaction.kv.max" (default 10) to avoid OOME
1218         boolean hasMore;
1219         do {
1220           hasMore = scanner.next(kvs, this.compactionKVMax);
1221           if (writer == null && !kvs.isEmpty()) {
1222             writer = createWriterInTmp(maxKeyCount,
1223               this.compactionCompression);
1224           }
1225           if (writer != null) {
1226             // output to writer:
1227             for (KeyValue kv : kvs) {
1228               if (kv.getMemstoreTS() <= smallestReadPoint) {
1229                 kv.setMemstoreTS(0);
1230               }
1231               writer.append(kv);
1232               // update progress per key
1233               ++progress.currentCompactedKVs;
1234 
1235               // check periodically to see if a system stop is requested
1236               if (Store.closeCheckInterval > 0) {
1237                 bytesWritten += kv.getLength();
1238                 if (bytesWritten > Store.closeCheckInterval) {
1239                   bytesWritten = 0;
1240                   if (!this.region.areWritesEnabled()) {
1241                     writer.close();
1242                     fs.delete(writer.getPath(), false);
1243                     throw new InterruptedIOException(
1244                         "Aborting compaction of store " + this +
1245                         " in region " + this.region +
1246                         " because user requested stop.");
1247                   }
1248                 }
1249               }
1250             }
1251           }
1252           kvs.clear();
1253         } while (hasMore);
1254       } finally {
1255         if (scanner != null) {
1256           scanner.close();
1257         }
1258       }
1259     } finally {
1260       if (writer != null) {
1261         writer.appendMetadata(maxId, majorCompaction);
1262         writer.close();
1263       }
1264     }
1265     return writer;
1266   }
1267 
1268   /**
1269    * Validates a store file by opening and closing it. In HFileV2 this should
1270    * not be an expensive operation.
1271    *
1272    * @param path the path to the store file
1273    */
1274   private void validateStoreFile(Path path)
1275       throws IOException {
1276     StoreFile storeFile = null;
1277     try {
1278       storeFile = new StoreFile(this.fs, path, this.conf,
1279           this.cacheConf, this.family.getBloomFilterType());
1280       storeFile.createReader();
1281     } catch (IOException e) {
1282       LOG.error("Failed to open store file : " + path
1283           + ", keeping it in tmp location", e);
1284       throw e;
1285     } finally {
1286       if (storeFile != null) {
1287         storeFile.closeReader(false);
1288       }
1289     }
1290   }
1291 
1292   /*
1293    * <p>It works by processing a compaction that's been written to disk.
1294    *
1295    * <p>It is usually invoked at the end of a compaction, but might also be
1296    * invoked at HStore startup, if the prior execution died midway through.
1297    *
1298    * <p>Moving the compacted TreeMap into place means:
1299    * <pre>
1300    * 1) Moving the new compacted StoreFile into place
1301    * 2) Unload all replaced StoreFile, close and collect list to delete.
1302    * 3) Loading the new TreeMap.
1303    * 4) Compute new store size
1304    * </pre>
1305    *
1306    * @param compactedFiles list of files that were compacted
1307    * @param compactedFile StoreFile that is the result of the compaction
1308    * @return StoreFile created. May be null.
1309    * @throws IOException
1310    */
1311   StoreFile completeCompaction(final Collection<StoreFile> compactedFiles,
1312                                        final StoreFile.Writer compactedFile)
1313       throws IOException {
1314     // 1. Moving the new files into place -- if there is a new file (may not
1315     // be if all cells were expired or deleted).
1316     StoreFile result = null;
1317     if (compactedFile != null) {
1318       validateStoreFile(compactedFile.getPath());
1319       // Move the file into the right spot
1320       Path origPath = compactedFile.getPath();
1321       Path destPath = new Path(homedir, origPath.getName());
1322       LOG.info("Renaming compacted file at " + origPath + " to " + destPath);
1323       if (!fs.rename(origPath, destPath)) {
1324         LOG.error("Failed move of compacted file " + origPath + " to " +
1325             destPath);
1326         throw new IOException("Failed move of compacted file " + origPath +
1327             " to " + destPath);
1328       }
1329       result = new StoreFile(this.fs, destPath, this.conf, this.cacheConf,
1330           this.family.getBloomFilterType());
1331       result.createReader();
1332     }
1333     try {
1334       this.lock.writeLock().lock();
1335       try {
1336         // Change this.storefiles so it reflects new state but do not
1337         // delete old store files until we have sent out notification of
1338         // change in case old files are still being accessed by outstanding
1339         // scanners.
1340         ArrayList<StoreFile> newStoreFiles = Lists.newArrayList(storefiles);
1341         newStoreFiles.removeAll(compactedFiles);
1342         filesCompacting.removeAll(compactedFiles); // safe bc: lock.writeLock()
1343 
1344         // If a StoreFile result, move it into place.  May be null.
1345         if (result != null) {
1346           newStoreFiles.add(result);
1347         }
1348 
1349         this.storefiles = sortAndClone(newStoreFiles);
1350       } finally {
1351         // We need the lock, as long as we are updating the storefiles
1352         // or changing the memstore. Let us release it before calling
1353         // notifyChangeReadersObservers. See HBASE-4485 for a possible
1354         // deadlock scenario that could have happened if continue to hold
1355         // the lock.
1356         this.lock.writeLock().unlock();
1357       }
1358 
1359       // Tell observers that list of StoreFiles has changed.
1360       notifyChangedReadersObservers();
1361       // Finally, delete old store files.
1362       for (StoreFile hsf: compactedFiles) {
1363         hsf.deleteReader();
1364       }
1365     } catch (IOException e) {
1366       e = RemoteExceptionHandler.checkIOException(e);
1367       LOG.error("Failed replacing compacted files in " + this.storeNameStr +
1368         ". Compacted file is " + (result == null? "none": result.toString()) +
1369         ".  Files replaced " + compactedFiles.toString() +
1370         " some of which may have been already removed", e);
1371     }
1372 
1373     // 4. Compute new store size
1374     this.storeSize = 0L;
1375     this.totalUncompressedBytes = 0L;
1376     for (StoreFile hsf : this.storefiles) {
1377       StoreFile.Reader r = hsf.getReader();
1378       if (r == null) {
1379         LOG.warn("StoreFile " + hsf + " has a null Reader");
1380         continue;
1381       }
1382       this.storeSize += r.length();
1383       this.totalUncompressedBytes += r.getTotalUncompressedBytes();
1384     }
1385     return result;
1386   }
1387 
1388   public ImmutableList<StoreFile> sortAndClone(List<StoreFile> storeFiles) {
1389     Collections.sort(storeFiles, StoreFile.Comparators.FLUSH_TIME);
1390     ImmutableList<StoreFile> newList = ImmutableList.copyOf(storeFiles);
1391     return newList;
1392   }
1393 
1394   // ////////////////////////////////////////////////////////////////////////////
1395   // Accessors.
1396   // (This is the only section that is directly useful!)
1397   //////////////////////////////////////////////////////////////////////////////
1398   /**
1399    * @return the number of files in this store
1400    */
1401   public int getNumberOfstorefiles() {
1402     return this.storefiles.size();
1403   }
1404 
1405   /*
1406    * @param wantedVersions How many versions were asked for.
1407    * @return wantedVersions or this families' {@link HConstants#VERSIONS}.
1408    */
1409   int versionsToReturn(final int wantedVersions) {
1410     if (wantedVersions <= 0) {
1411       throw new IllegalArgumentException("Number of versions must be > 0");
1412     }
1413     // Make sure we do not return more than maximum versions for this store.
1414     int maxVersions = this.family.getMaxVersions();
1415     return wantedVersions > maxVersions ? maxVersions: wantedVersions;
1416   }
1417 
1418   static boolean isExpired(final KeyValue key, final long oldestTimestamp) {
1419     return key.getTimestamp() < oldestTimestamp;
1420   }
1421 
1422   /**
1423    * Find the key that matches <i>row</i> exactly, or the one that immediately
1424    * preceeds it. WARNING: Only use this method on a table where writes occur
1425    * with strictly increasing timestamps. This method assumes this pattern of
1426    * writes in order to make it reasonably performant.  Also our search is
1427    * dependent on the axiom that deletes are for cells that are in the container
1428    * that follows whether a memstore snapshot or a storefile, not for the
1429    * current container: i.e. we'll see deletes before we come across cells we
1430    * are to delete. Presumption is that the memstore#kvset is processed before
1431    * memstore#snapshot and so on.
1432    * @param row The row key of the targeted row.
1433    * @return Found keyvalue or null if none found.
1434    * @throws IOException
1435    */
1436   KeyValue getRowKeyAtOrBefore(final byte[] row) throws IOException {
1437     // If minVersions is set, we will not ignore expired KVs.
1438     // As we're only looking for the latest matches, that should be OK.
1439     // With minVersions > 0 we guarantee that any KV that has any version
1440     // at all (expired or not) has at least one version that will not expire.
1441     // Note that this method used to take a KeyValue as arguments. KeyValue
1442     // can be back-dated, a row key cannot.
1443     long ttlToUse = this.minVersions > 0 ? Long.MAX_VALUE : this.ttl;
1444 
1445     KeyValue kv = new KeyValue(row, HConstants.LATEST_TIMESTAMP);
1446 
1447     GetClosestRowBeforeTracker state = new GetClosestRowBeforeTracker(
1448       this.comparator, kv, ttlToUse, this.region.getRegionInfo().isMetaRegion());
1449     this.lock.readLock().lock();
1450     try {
1451       // First go to the memstore.  Pick up deletes and candidates.
1452       this.memstore.getRowKeyAtOrBefore(state);
1453       // Check if match, if we got a candidate on the asked for 'kv' row.
1454       // Process each store file. Run through from newest to oldest.
1455       for (StoreFile sf : Lists.reverse(storefiles)) {
1456         // Update the candidate keys from the current map file
1457         rowAtOrBeforeFromStoreFile(sf, state);
1458       }
1459       return state.getCandidate();
1460     } finally {
1461       this.lock.readLock().unlock();
1462     }
1463   }
1464 
1465   /*
1466    * Check an individual MapFile for the row at or before a given row.
1467    * @param f
1468    * @param state
1469    * @throws IOException
1470    */
1471   private void rowAtOrBeforeFromStoreFile(final StoreFile f,
1472                                           final GetClosestRowBeforeTracker state)
1473       throws IOException {
1474     StoreFile.Reader r = f.getReader();
1475     if (r == null) {
1476       LOG.warn("StoreFile " + f + " has a null Reader");
1477       return;
1478     }
1479     // TODO: Cache these keys rather than make each time?
1480     byte [] fk = r.getFirstKey();
1481     KeyValue firstKV = KeyValue.createKeyValueFromKey(fk, 0, fk.length);
1482     byte [] lk = r.getLastKey();
1483     KeyValue lastKV = KeyValue.createKeyValueFromKey(lk, 0, lk.length);
1484     KeyValue firstOnRow = state.getTargetKey();
1485     if (this.comparator.compareRows(lastKV, firstOnRow) < 0) {
1486       // If last key in file is not of the target table, no candidates in this
1487       // file.  Return.
1488       if (!state.isTargetTable(lastKV)) return;
1489       // If the row we're looking for is past the end of file, set search key to
1490       // last key. TODO: Cache last and first key rather than make each time.
1491       firstOnRow = new KeyValue(lastKV.getRow(), HConstants.LATEST_TIMESTAMP);
1492     }
1493     // Get a scanner that caches blocks and that uses pread.
1494     HFileScanner scanner = r.getHFileReader().getScanner(true, true, false);
1495     // Seek scanner.  If can't seek it, return.
1496     if (!seekToScanner(scanner, firstOnRow, firstKV)) return;
1497     // If we found candidate on firstOnRow, just return. THIS WILL NEVER HAPPEN!
1498     // Unlikely that there'll be an instance of actual first row in table.
1499     if (walkForwardInSingleRow(scanner, firstOnRow, state)) return;
1500     // If here, need to start backing up.
1501     while (scanner.seekBefore(firstOnRow.getBuffer(), firstOnRow.getKeyOffset(),
1502        firstOnRow.getKeyLength())) {
1503       KeyValue kv = scanner.getKeyValue();
1504       if (!state.isTargetTable(kv)) break;
1505       if (!state.isBetterCandidate(kv)) break;
1506       // Make new first on row.
1507       firstOnRow = new KeyValue(kv.getRow(), HConstants.LATEST_TIMESTAMP);
1508       // Seek scanner.  If can't seek it, break.
1509       if (!seekToScanner(scanner, firstOnRow, firstKV)) break;
1510       // If we find something, break;
1511       if (walkForwardInSingleRow(scanner, firstOnRow, state)) break;
1512     }
1513   }
1514 
1515   /*
1516    * Seek the file scanner to firstOnRow or first entry in file.
1517    * @param scanner
1518    * @param firstOnRow
1519    * @param firstKV
1520    * @return True if we successfully seeked scanner.
1521    * @throws IOException
1522    */
1523   private boolean seekToScanner(final HFileScanner scanner,
1524                                 final KeyValue firstOnRow,
1525                                 final KeyValue firstKV)
1526       throws IOException {
1527     KeyValue kv = firstOnRow;
1528     // If firstOnRow < firstKV, set to firstKV
1529     if (this.comparator.compareRows(firstKV, firstOnRow) == 0) kv = firstKV;
1530     int result = scanner.seekTo(kv.getBuffer(), kv.getKeyOffset(),
1531       kv.getKeyLength());
1532     return result >= 0;
1533   }
1534 
1535   /*
1536    * When we come in here, we are probably at the kv just before we break into
1537    * the row that firstOnRow is on.  Usually need to increment one time to get
1538    * on to the row we are interested in.
1539    * @param scanner
1540    * @param firstOnRow
1541    * @param state
1542    * @return True we found a candidate.
1543    * @throws IOException
1544    */
1545   private boolean walkForwardInSingleRow(final HFileScanner scanner,
1546                                          final KeyValue firstOnRow,
1547                                          final GetClosestRowBeforeTracker state)
1548       throws IOException {
1549     boolean foundCandidate = false;
1550     do {
1551       KeyValue kv = scanner.getKeyValue();
1552       // If we are not in the row, skip.
1553       if (this.comparator.compareRows(kv, firstOnRow) < 0) continue;
1554       // Did we go beyond the target row? If so break.
1555       if (state.isTooFar(kv, firstOnRow)) break;
1556       if (state.isExpired(kv)) {
1557         continue;
1558       }
1559       // If we added something, this row is a contender. break.
1560       if (state.handle(kv)) {
1561         foundCandidate = true;
1562         break;
1563       }
1564     } while(scanner.next());
1565     return foundCandidate;
1566   }
1567 
1568   public boolean canSplit() {
1569     this.lock.readLock().lock();
1570     try {
1571       // Not splitable if we find a reference store file present in the store.
1572       for (StoreFile sf : storefiles) {
1573         if (sf.isReference()) {
1574           if (LOG.isDebugEnabled()) {
1575             LOG.debug(sf + " is not splittable");
1576           }
1577           return false;
1578         }
1579       }
1580 
1581       return true;
1582     } finally {
1583       this.lock.readLock().unlock();
1584     }
1585   }
1586   /**
1587    * Determines if Store should be split
1588    * @return byte[] if store should be split, null otherwise.
1589    */
1590   public byte[] getSplitPoint() {
1591     this.lock.readLock().lock();
1592     try {
1593       // sanity checks
1594       if (this.storefiles.isEmpty()) {
1595         return null;
1596       }
1597       // Should already be enforced by the split policy!
1598       assert !this.region.getRegionInfo().isMetaRegion();
1599 
1600       // Not splitable if we find a reference store file present in the store.
1601       long maxSize = 0L;
1602       StoreFile largestSf = null;
1603       for (StoreFile sf : storefiles) {
1604         if (sf.isReference()) {
1605           // Should already be enforced since we return false in this case
1606           assert false : "getSplitPoint() called on a region that can't split!";
1607           return null;
1608         }
1609 
1610         StoreFile.Reader r = sf.getReader();
1611         if (r == null) {
1612           LOG.warn("Storefile " + sf + " Reader is null");
1613           continue;
1614         }
1615 
1616         long size = r.length();
1617         if (size > maxSize) {
1618           // This is the largest one so far
1619           maxSize = size;
1620           largestSf = sf;
1621         }
1622       }
1623 
1624       StoreFile.Reader r = largestSf.getReader();
1625       if (r == null) {
1626         LOG.warn("Storefile " + largestSf + " Reader is null");
1627         return null;
1628       }
1629       // Get first, last, and mid keys.  Midkey is the key that starts block
1630       // in middle of hfile.  Has column and timestamp.  Need to return just
1631       // the row we want to split on as midkey.
1632       byte [] midkey = r.midkey();
1633       if (midkey != null) {
1634         KeyValue mk = KeyValue.createKeyValueFromKey(midkey, 0, midkey.length);
1635         byte [] fk = r.getFirstKey();
1636         KeyValue firstKey = KeyValue.createKeyValueFromKey(fk, 0, fk.length);
1637         byte [] lk = r.getLastKey();
1638         KeyValue lastKey = KeyValue.createKeyValueFromKey(lk, 0, lk.length);
1639         // if the midkey is the same as the first and last keys, then we cannot
1640         // (ever) split this region.
1641         if (this.comparator.compareRows(mk, firstKey) == 0 &&
1642             this.comparator.compareRows(mk, lastKey) == 0) {
1643           if (LOG.isDebugEnabled()) {
1644             LOG.debug("cannot split because midkey is the same as first or " +
1645               "last row");
1646           }
1647           return null;
1648         }
1649         return mk.getRow();
1650       }
1651     } catch(IOException e) {
1652       LOG.warn("Failed getting store size for " + this.storeNameStr, e);
1653     } finally {
1654       this.lock.readLock().unlock();
1655     }
1656     return null;
1657   }
1658 
1659   /** @return aggregate size of all HStores used in the last compaction */
1660   public long getLastCompactSize() {
1661     return this.lastCompactSize;
1662   }
1663 
1664   /** @return aggregate size of HStore */
1665   public long getSize() {
1666     return storeSize;
1667   }
1668 
1669   public void triggerMajorCompaction() {
1670     this.forceMajor = true;
1671   }
1672 
1673   boolean getForceMajorCompaction() {
1674     return this.forceMajor;
1675   }
1676 
1677   //////////////////////////////////////////////////////////////////////////////
1678   // File administration
1679   //////////////////////////////////////////////////////////////////////////////
1680 
1681   /**
1682    * Return a scanner for both the memstore and the HStore files
1683    * @throws IOException
1684    */
1685   public StoreScanner getScanner(Scan scan,
1686       final NavigableSet<byte []> targetCols) throws IOException {
1687     lock.readLock().lock();
1688     try {
1689       return new StoreScanner(this, scan, targetCols);
1690     } finally {
1691       lock.readLock().unlock();
1692     }
1693   }
1694 
1695   @Override
1696   public String toString() {
1697     return this.storeNameStr;
1698   }
1699 
1700   /**
1701    * @return Count of store files
1702    */
1703   int getStorefilesCount() {
1704     return this.storefiles.size();
1705   }
1706 
1707   /**
1708    * @return The size of the store files, in bytes, uncompressed.
1709    */
1710   long getStoreSizeUncompressed() {
1711     return this.totalUncompressedBytes;
1712   }
1713 
1714   /**
1715    * @return The size of the store files, in bytes.
1716    */
1717   long getStorefilesSize() {
1718     long size = 0;
1719     for (StoreFile s: storefiles) {
1720       StoreFile.Reader r = s.getReader();
1721       if (r == null) {
1722         LOG.warn("StoreFile " + s + " has a null Reader");
1723         continue;
1724       }
1725       size += r.length();
1726     }
1727     return size;
1728   }
1729 
1730   /**
1731    * @return The size of the store file indexes, in bytes.
1732    */
1733   long getStorefilesIndexSize() {
1734     long size = 0;
1735     for (StoreFile s: storefiles) {
1736       StoreFile.Reader r = s.getReader();
1737       if (r == null) {
1738         LOG.warn("StoreFile " + s + " has a null Reader");
1739         continue;
1740       }
1741       size += r.indexSize();
1742     }
1743     return size;
1744   }
1745 
1746   /**
1747    * Returns the total size of all index blocks in the data block indexes,
1748    * including the root level, intermediate levels, and the leaf level for
1749    * multi-level indexes, or just the root level for single-level indexes.
1750    *
1751    * @return the total size of block indexes in the store
1752    */
1753   long getTotalStaticIndexSize() {
1754     long size = 0;
1755     for (StoreFile s : storefiles) {
1756       size += s.getReader().getUncompressedDataIndexSize();
1757     }
1758     return size;
1759   }
1760 
1761   /**
1762    * Returns the total byte size of all Bloom filter bit arrays. For compound
1763    * Bloom filters even the Bloom blocks currently not loaded into the block
1764    * cache are counted.
1765    *
1766    * @return the total size of all Bloom filters in the store
1767    */
1768   long getTotalStaticBloomSize() {
1769     long size = 0;
1770     for (StoreFile s : storefiles) {
1771       StoreFile.Reader r = s.getReader();
1772       size += r.getTotalBloomSize();
1773     }
1774     return size;
1775   }
1776 
1777   /**
1778    * @return The priority that this store should have in the compaction queue
1779    */
1780   public int getCompactPriority() {
1781     return this.blockingStoreFileCount - this.storefiles.size();
1782   }
1783 
1784   HRegion getHRegion() {
1785     return this.region;
1786   }
1787 
1788   HRegionInfo getHRegionInfo() {
1789     return this.region.regionInfo;
1790   }
1791 
1792   /**
1793    * Increments the value for the given row/family/qualifier.
1794    *
1795    * This function will always be seen as atomic by other readers
1796    * because it only puts a single KV to memstore. Thus no
1797    * read/write control necessary.
1798    *
1799    * @param row
1800    * @param f
1801    * @param qualifier
1802    * @param newValue the new value to set into memstore
1803    * @return memstore size delta
1804    * @throws IOException
1805    */
1806   public long updateColumnValue(byte [] row, byte [] f,
1807                                 byte [] qualifier, long newValue)
1808       throws IOException {
1809 
1810     this.lock.readLock().lock();
1811     try {
1812       long now = EnvironmentEdgeManager.currentTimeMillis();
1813 
1814       return this.memstore.updateColumnValue(row,
1815           f,
1816           qualifier,
1817           newValue,
1818           now);
1819 
1820     } finally {
1821       this.lock.readLock().unlock();
1822     }
1823   }
1824 
1825   /**
1826    * Adds or replaces the specified KeyValues.
1827    * <p>
1828    * For each KeyValue specified, if a cell with the same row, family, and
1829    * qualifier exists in MemStore, it will be replaced.  Otherwise, it will just
1830    * be inserted to MemStore.
1831    * <p>
1832    * This operation is atomic on each KeyValue (row/family/qualifier) but not
1833    * necessarily atomic across all of them.
1834    * @param kvs
1835    * @return memstore size delta
1836    * @throws IOException
1837    */
1838   public long upsert(List<KeyValue> kvs)
1839       throws IOException {
1840     this.lock.readLock().lock();
1841     try {
1842       // TODO: Make this operation atomic w/ MVCC
1843       return this.memstore.upsert(kvs);
1844     } finally {
1845       this.lock.readLock().unlock();
1846     }
1847   }
1848 
1849   public StoreFlusher getStoreFlusher(long cacheFlushId) {
1850     return new StoreFlusherImpl(cacheFlushId);
1851   }
1852 
1853   private class StoreFlusherImpl implements StoreFlusher {
1854 
1855     private long cacheFlushId;
1856     private SortedSet<KeyValue> snapshot;
1857     private StoreFile storeFile;
1858     private TimeRangeTracker snapshotTimeRangeTracker;
1859 
1860     private StoreFlusherImpl(long cacheFlushId) {
1861       this.cacheFlushId = cacheFlushId;
1862     }
1863 
1864     @Override
1865     public void prepare() {
1866       memstore.snapshot();
1867       this.snapshot = memstore.getSnapshot();
1868       this.snapshotTimeRangeTracker = memstore.getSnapshotTimeRangeTracker();
1869     }
1870 
1871     @Override
1872     public void flushCache(MonitoredTask status) throws IOException {
1873       storeFile = Store.this.flushCache(
1874           cacheFlushId, snapshot, snapshotTimeRangeTracker, status);
1875     }
1876 
1877     @Override
1878     public boolean commit() throws IOException {
1879       if (storeFile == null) {
1880         return false;
1881       }
1882       // Add new file to store files.  Clear snapshot too while we have
1883       // the Store write lock.
1884       return Store.this.updateStorefiles(storeFile, snapshot);
1885     }
1886   }
1887 
1888   /**
1889    * See if there's too much store files in this store
1890    * @return true if number of store files is greater than
1891    *  the number defined in minFilesToCompact
1892    */
1893   public boolean needsCompaction() {
1894     return (storefiles.size() - filesCompacting.size()) > minFilesToCompact;
1895   }
1896 
1897   /**
1898    * Used for tests. Get the cache configuration for this Store.
1899    */
1900   public CacheConfig getCacheConfig() {
1901     return this.cacheConf;
1902   }
1903 
1904   public static final long FIXED_OVERHEAD = ClassSize.align(
1905       ClassSize.OBJECT + (17 * ClassSize.REFERENCE) +
1906       (7 * Bytes.SIZEOF_LONG) + (1 * Bytes.SIZEOF_DOUBLE) +
1907       (7 * Bytes.SIZEOF_INT) + (1 * Bytes.SIZEOF_BOOLEAN));
1908 
1909   public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD +
1910       ClassSize.OBJECT + ClassSize.REENTRANT_LOCK +
1911       ClassSize.CONCURRENT_SKIPLISTMAP +
1912       ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + ClassSize.OBJECT);
1913 
1914   @Override
1915   public long heapSize() {
1916     return DEEP_OVERHEAD + this.memstore.heapSize();
1917   }
1918 
1919   public KeyValue.KVComparator getComparator() {
1920     return comparator;
1921   }
1922 
1923 }