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.Compression;
48  import org.apache.hadoop.hbase.io.hfile.HFile;
49  import org.apache.hadoop.hbase.io.hfile.HFileScanner;
50  import org.apache.hadoop.hbase.util.Bytes;
51  import org.apache.hadoop.hbase.util.ClassSize;
52  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
53  import org.apache.hadoop.util.StringUtils;
54  
55  import com.google.common.collect.ImmutableList;
56  import com.google.common.collect.Iterables;
57  
58  /**
59   * A Store holds a column family in a Region.  Its a memstore and a set of zero
60   * or more StoreFiles, which stretch backwards over time.
61   *
62   * <p>There's no reason to consider append-logging at this level; all logging
63   * and locking is handled at the HRegion level.  Store just provides
64   * services to manage sets of StoreFiles.  One of the most important of those
65   * services is compaction services where files are aggregated once they pass
66   * a configurable threshold.
67   *
68   * <p>The only thing having to do with logs that Store needs to deal with is
69   * the reconstructionLog.  This is a segment of an HRegion's log that might
70   * NOT be present upon startup.  If the param is NULL, there's nothing to do.
71   * If the param is non-NULL, we need to process the log to reconstruct
72   * a TreeMap that might not have been written to disk before the process
73   * died.
74   *
75   * <p>It's assumed that after this constructor returns, the reconstructionLog
76   * file will be deleted (by whoever has instantiated the Store).
77   *
78   * <p>Locking and transactions are handled at a higher level.  This API should
79   * not be called directly but by an HRegion manager.
80   */
81  public class Store implements HeapSize {
82    static final Log LOG = LogFactory.getLog(Store.class);
83    protected final MemStore memstore;
84    // This stores directory in the filesystem.
85    private final Path homedir;
86    private final HRegion region;
87    private final HColumnDescriptor family;
88    final FileSystem fs;
89    final Configuration conf;
90    // ttl in milliseconds.
91    protected long ttl;
92    private long majorCompactionTime;
93    private final int maxFilesToCompact;
94    private final long minCompactSize;
95    // compactRatio: double on purpose!  Float.MAX < Long.MAX < Double.MAX
96    // With float, java will downcast your long to float for comparisons (bad)
97    private double compactRatio;
98    private long lastCompactSize = 0;
99    /* how many bytes to write between status checks */
100   static int closeCheckInterval = 0;
101   private final long desiredMaxFileSize;
102   private final int blockingStoreFileCount;
103   private volatile long storeSize = 0L;
104   private final Object flushLock = new Object();
105   final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
106   private final String storeNameStr;
107   private final boolean inMemory;
108 
109   /*
110    * List of store files inside this store. This is an immutable list that
111    * is atomically replaced when its contents change.
112    */
113   private ImmutableList<StoreFile> storefiles = null;
114 
115 
116   // All access must be synchronized.
117   private final CopyOnWriteArraySet<ChangedReadersObserver> changedReaderObservers =
118     new CopyOnWriteArraySet<ChangedReadersObserver>();
119 
120   private final Object compactLock = new Object();
121   private final int compactionThreshold;
122   private final int blocksize;
123   private final boolean blockcache;
124   /** Compression algorithm for flush files and minor compaction */
125   private final Compression.Algorithm compression;
126   /** Compression algorithm for major compaction */
127   private final Compression.Algorithm compactionCompression;
128 
129   // Comparing KeyValues
130   final KeyValue.KVComparator comparator;
131 
132   /**
133    * Constructor
134    * @param basedir qualified path under which the region directory lives;
135    * generally the table subdirectory
136    * @param region
137    * @param family HColumnDescriptor for this column
138    * @param fs file system object
139    * @param conf configuration object
140    * failed.  Can be null.
141    * @throws IOException
142    */
143   protected Store(Path basedir, HRegion region, HColumnDescriptor family,
144     FileSystem fs, Configuration conf)
145   throws IOException {
146     HRegionInfo info = region.regionInfo;
147     this.fs = fs;
148     this.homedir = getStoreHomedir(basedir, info.getEncodedName(), family.getName());
149     if (!this.fs.exists(this.homedir)) {
150       if (!this.fs.mkdirs(this.homedir))
151         throw new IOException("Failed create of: " + this.homedir.toString());
152     }
153     this.region = region;
154     this.family = family;
155     this.conf = conf;
156     this.blockcache = family.isBlockCacheEnabled();
157     this.blocksize = family.getBlocksize();
158     this.compression = family.getCompression();
159     // avoid overriding compression setting for major compactions if the user
160     // has not specified it separately
161     this.compactionCompression =
162       (family.getCompactionCompression() != Compression.Algorithm.NONE) ?
163         family.getCompactionCompression() : this.compression;
164     this.comparator = info.getComparator();
165     // getTimeToLive returns ttl in seconds.  Convert to milliseconds.
166     this.ttl = family.getTimeToLive();
167     if (ttl == HConstants.FOREVER) {
168       // default is unlimited ttl.
169       ttl = Long.MAX_VALUE;
170     } else if (ttl == -1) {
171       ttl = Long.MAX_VALUE;
172     } else {
173       // second -> ms adjust for user data
174       this.ttl *= 1000;
175     }
176     this.memstore = new MemStore(conf, this.comparator);
177     this.storeNameStr = Bytes.toString(this.family.getName());
178 
179     // By default, we compact if an HStore has more than
180     // MIN_COMMITS_FOR_COMPACTION map files
181     this.compactionThreshold = Math.max(2,
182       conf.getInt("hbase.hstore.compactionThreshold", 3));
183 
184     // Check if this is in-memory store
185     this.inMemory = family.isInMemory();
186 
187     // By default we split region if a file > HConstants.DEFAULT_MAX_FILE_SIZE.
188     long maxFileSize = info.getTableDesc().getMaxFileSize();
189     if (maxFileSize == HConstants.DEFAULT_MAX_FILE_SIZE) {
190       maxFileSize = conf.getLong("hbase.hregion.max.filesize",
191         HConstants.DEFAULT_MAX_FILE_SIZE);
192     }
193     this.desiredMaxFileSize = maxFileSize;
194     this.blockingStoreFileCount =
195       conf.getInt("hbase.hstore.blockingStoreFiles", 7);
196 
197     this.majorCompactionTime = getNextMajorCompactTime();
198 
199     this.maxFilesToCompact = conf.getInt("hbase.hstore.compaction.max", 10);
200     this.minCompactSize = conf.getLong("hbase.hstore.compaction.min.size",
201         this.region.memstoreFlushSize);
202     this.compactRatio = conf.getFloat("hbase.hstore.compaction.ratio", 1.2F);
203 
204     if (Store.closeCheckInterval == 0) {
205       Store.closeCheckInterval = conf.getInt(
206           "hbase.hstore.close.check.interval", 10*1000*1000 /* 10 MB */);
207     }
208     this.storefiles = sortAndClone(loadStoreFiles());
209   }
210 
211   public HColumnDescriptor getFamily() {
212     return this.family;
213   }
214 
215   /**
216    * @return The maximum sequence id in all store files.
217    */
218   long getMaxSequenceId() {
219     return StoreFile.getMaxSequenceIdInList(this.getStorefiles());
220   }
221 
222   /**
223    * @param tabledir
224    * @param encodedName Encoded region name.
225    * @param family
226    * @return Path to family/Store home directory.
227    */
228   public static Path getStoreHomedir(final Path tabledir,
229       final String encodedName, final byte [] family) {
230     return new Path(tabledir, new Path(encodedName,
231       new Path(Bytes.toString(family))));
232   }
233 
234   /**
235    * Return the directory in which this store stores its
236    * StoreFiles
237    */
238   public Path getHomedir() {
239     return homedir;
240   }
241 
242   /*
243    * Creates an unsorted list of StoreFile loaded from the given directory.
244    * @throws IOException
245    */
246   private List<StoreFile> loadStoreFiles()
247   throws IOException {
248     ArrayList<StoreFile> results = new ArrayList<StoreFile>();
249     FileStatus files[] = this.fs.listStatus(this.homedir);
250     for (int i = 0; files != null && i < files.length; i++) {
251       // Skip directories.
252       if (files[i].isDir()) {
253         continue;
254       }
255       Path p = files[i].getPath();
256       // Check for empty file.  Should never be the case but can happen
257       // after data loss in hdfs for whatever reason (upgrade, etc.): HBASE-646
258       if (this.fs.getFileStatus(p).getLen() <= 0) {
259         LOG.warn("Skipping " + p + " because its empty. HBASE-646 DATA LOSS?");
260         continue;
261       }
262       StoreFile curfile = null;
263       try {
264         curfile = new StoreFile(fs, p, blockcache, this.conf,
265             this.family.getBloomFilterType(), this.inMemory);
266         curfile.createReader();
267       } catch (IOException ioe) {
268         LOG.warn("Failed open of " + p + "; presumption is that file was " +
269           "corrupted at flush and lost edits picked up by commit log replay. " +
270           "Verify!", ioe);
271         continue;
272       }
273       long length = curfile.getReader().length();
274       this.storeSize += length;
275       if (LOG.isDebugEnabled()) {
276         LOG.debug("loaded " + curfile.toStringDetailed());
277       }
278       results.add(curfile);
279     }
280     return results;
281   }
282 
283   /**
284    * Adds a value to the memstore
285    *
286    * @param kv
287    * @return memstore size delta
288    */
289   protected long add(final KeyValue kv) {
290     lock.readLock().lock();
291     try {
292       return this.memstore.add(kv);
293     } finally {
294       lock.readLock().unlock();
295     }
296   }
297 
298   /**
299    * Adds a value to the memstore
300    *
301    * @param kv
302    * @return memstore size delta
303    */
304   protected long delete(final KeyValue kv) {
305     lock.readLock().lock();
306     try {
307       return this.memstore.delete(kv);
308     } finally {
309       lock.readLock().unlock();
310     }
311   }
312 
313   /**
314    * @return All store files.
315    */
316   List<StoreFile> getStorefiles() {
317     return this.storefiles;
318   }
319 
320   public void bulkLoadHFile(String srcPathStr) throws IOException {
321     Path srcPath = new Path(srcPathStr);
322 
323     HFile.Reader reader  = null;
324     try {
325       LOG.info("Validating hfile at " + srcPath + " for inclusion in "
326           + "store " + this + " region " + this.region);
327       reader = new HFile.Reader(srcPath.getFileSystem(conf),
328           srcPath, null, false);
329       reader.loadFileInfo();
330 
331       byte[] firstKey = reader.getFirstRowKey();
332       byte[] lk = reader.getLastKey();
333       byte[] lastKey =
334           (lk == null) ? null :
335               KeyValue.createKeyValueFromKey(lk).getRow();
336 
337       LOG.debug("HFile bounds: first=" + Bytes.toStringBinary(firstKey) +
338           " last=" + Bytes.toStringBinary(lastKey));
339       LOG.debug("Region bounds: first=" +
340           Bytes.toStringBinary(region.getStartKey()) +
341           " last=" + Bytes.toStringBinary(region.getEndKey()));
342 
343       HRegionInfo hri = region.getRegionInfo();
344       if (!hri.containsRange(firstKey, lastKey)) {
345         throw new WrongRegionException(
346             "Bulk load file " + srcPathStr + " does not fit inside region "
347             + this.region);
348       }
349     } finally {
350       if (reader != null) reader.close();
351     }
352 
353     // Move the file if it's on another filesystem
354     FileSystem srcFs = srcPath.getFileSystem(conf);
355     if (!srcFs.equals(fs)) {
356       LOG.info("File " + srcPath + " on different filesystem than " +
357           "destination store - moving to this filesystem.");
358       Path tmpPath = getTmpPath();
359       FileUtil.copy(srcFs, srcPath, fs, tmpPath, false, conf);
360       LOG.info("Copied to temporary path on dst filesystem: " + tmpPath);
361       srcPath = tmpPath;
362     }
363 
364     Path dstPath = StoreFile.getRandomFilename(fs, homedir);
365     LOG.info("Renaming bulk load file " + srcPath + " to " + dstPath);
366     StoreFile.rename(fs, srcPath, dstPath);
367 
368     StoreFile sf = new StoreFile(fs, dstPath, blockcache,
369         this.conf, this.family.getBloomFilterType(), this.inMemory);
370     sf.createReader();
371 
372     LOG.info("Moved hfile " + srcPath + " into store directory " +
373         homedir + " - updating store file list.");
374 
375     // Append the new storefile into the list
376     this.lock.writeLock().lock();
377     try {
378       ArrayList<StoreFile> newFiles = new ArrayList<StoreFile>(storefiles);
379       newFiles.add(sf);
380       this.storefiles = sortAndClone(newFiles);
381       notifyChangedReadersObservers();
382     } finally {
383       this.lock.writeLock().unlock();
384     }
385     LOG.info("Successfully loaded store file " + srcPath
386         + " into store " + this + " (new location: " + dstPath + ")");
387   }
388 
389   /**
390    * Get a temporary path in this region. These temporary files
391    * will get cleaned up when the region is re-opened if they are
392    * still around.
393    */
394   private Path getTmpPath() throws IOException {
395     return StoreFile.getRandomFilename(
396         fs, region.getTmpDir());
397   }
398 
399   /**
400    * Close all the readers
401    *
402    * We don't need to worry about subsequent requests because the HRegion holds
403    * a write lock that will prevent any more reads or writes.
404    *
405    * @throws IOException
406    */
407   ImmutableList<StoreFile> close() throws IOException {
408     this.lock.writeLock().lock();
409     try {
410       ImmutableList<StoreFile> result = storefiles;
411 
412       // Clear so metrics doesn't find them.
413       storefiles = ImmutableList.of();
414 
415       for (StoreFile f: result) {
416         f.closeReader();
417       }
418       LOG.debug("closed " + this.storeNameStr);
419       return result;
420     } finally {
421       this.lock.writeLock().unlock();
422     }
423   }
424 
425   /**
426    * Snapshot this stores memstore.  Call before running
427    * {@link #flushCache(long, SortedSet<KeyValue>)} so it has some work to do.
428    */
429   void snapshot() {
430     this.memstore.snapshot();
431   }
432 
433   /**
434    * Write out current snapshot.  Presumes {@link #snapshot()} has been called
435    * previously.
436    * @param logCacheFlushId flush sequence number
437    * @param snapshot
438    * @param snapshotTimeRangeTracker
439    * @return true if a compaction is needed
440    * @throws IOException
441    */
442   private StoreFile flushCache(final long logCacheFlushId,
443       SortedSet<KeyValue> snapshot,
444       TimeRangeTracker snapshotTimeRangeTracker) throws IOException {
445     // If an exception happens flushing, we let it out without clearing
446     // the memstore snapshot.  The old snapshot will be returned when we say
447     // 'snapshot', the next time flush comes around.
448     return internalFlushCache(snapshot, logCacheFlushId, snapshotTimeRangeTracker);
449   }
450 
451   /*
452    * @param cache
453    * @param logCacheFlushId
454    * @return StoreFile created.
455    * @throws IOException
456    */
457   private StoreFile internalFlushCache(final SortedSet<KeyValue> set,
458       final long logCacheFlushId,
459       TimeRangeTracker snapshotTimeRangeTracker)
460       throws IOException {
461     StoreFile.Writer writer = null;
462     long flushed = 0;
463     // Don't flush if there are no entries.
464     if (set.size() == 0) {
465       return null;
466     }
467     long oldestTimestamp = System.currentTimeMillis() - ttl;
468     // TODO:  We can fail in the below block before we complete adding this
469     // flush to list of store files.  Add cleanup of anything put on filesystem
470     // if we fail.
471     synchronized (flushLock) {
472       // A. Write the map out to the disk
473       writer = createWriterInTmp(set.size());
474       writer.setTimeRangeTracker(snapshotTimeRangeTracker);
475       int entries = 0;
476       try {
477         for (KeyValue kv: set) {
478           if (!isExpired(kv, oldestTimestamp)) {
479             writer.append(kv);
480             entries++;
481             flushed += this.memstore.heapSizeChange(kv, true);
482           }
483         }
484       } finally {
485         // Write out the log sequence number that corresponds to this output
486         // hfile.  The hfile is current up to and including logCacheFlushId.
487         writer.appendMetadata(logCacheFlushId, false);
488         writer.close();
489       }
490     }
491 
492     // Write-out finished successfully, move into the right spot
493     Path dstPath = StoreFile.getUniqueFile(fs, homedir);
494     LOG.info("Renaming flushed file at " + writer.getPath() + " to " + dstPath);
495     if (!fs.rename(writer.getPath(), dstPath)) {
496       LOG.warn("Unable to rename " + writer.getPath() + " to " + dstPath);
497     }
498 
499     StoreFile sf = new StoreFile(this.fs, dstPath, blockcache,
500       this.conf, this.family.getBloomFilterType(), this.inMemory);
501     StoreFile.Reader r = sf.createReader();
502     this.storeSize += r.length();
503     if(LOG.isInfoEnabled()) {
504       LOG.info("Added " + sf + ", entries=" + r.getEntries() +
505         ", sequenceid=" + logCacheFlushId +
506         ", memsize=" + StringUtils.humanReadableInt(flushed) +
507         ", filesize=" + StringUtils.humanReadableInt(r.length()));
508     }
509     return sf;
510   }
511 
512   /*
513    * @param maxKeyCount
514    * @return Writer for a new StoreFile in the tmp dir.
515    */
516   private StoreFile.Writer createWriterInTmp(int maxKeyCount)
517   throws IOException {
518     return createWriterInTmp(maxKeyCount, this.compression);
519   }
520 
521   /*
522    * @param maxKeyCount
523    * @param compression Compression algorithm to use
524    * @return Writer for a new StoreFile in the tmp dir.
525    */
526   private StoreFile.Writer createWriterInTmp(int maxKeyCount,
527     Compression.Algorithm compression)
528   throws IOException {
529     return StoreFile.createWriter(this.fs, region.getTmpDir(), this.blocksize,
530         compression, this.comparator, this.conf,
531         this.family.getBloomFilterType(), maxKeyCount);
532   }
533 
534   /*
535    * Change storefiles adding into place the Reader produced by this new flush.
536    * @param sf
537    * @param set That was used to make the passed file <code>p</code>.
538    * @throws IOException
539    * @return Whether compaction is required.
540    */
541   private boolean updateStorefiles(final StoreFile sf,
542                                    final SortedSet<KeyValue> set)
543   throws IOException {
544     this.lock.writeLock().lock();
545     try {
546       ArrayList<StoreFile> newList = new ArrayList<StoreFile>(storefiles);
547       newList.add(sf);
548       storefiles = sortAndClone(newList);
549       this.memstore.clearSnapshot(set);
550 
551       // Tell listeners of the change in readers.
552       notifyChangedReadersObservers();
553 
554       return this.storefiles.size() >= this.compactionThreshold;
555     } finally {
556       this.lock.writeLock().unlock();
557     }
558   }
559 
560   /*
561    * Notify all observers that set of Readers has changed.
562    * @throws IOException
563    */
564   private void notifyChangedReadersObservers() throws IOException {
565     for (ChangedReadersObserver o: this.changedReaderObservers) {
566       o.updateReaders();
567     }
568   }
569 
570   /*
571    * @param o Observer who wants to know about changes in set of Readers
572    */
573   void addChangedReaderObserver(ChangedReadersObserver o) {
574     this.changedReaderObservers.add(o);
575   }
576 
577   /*
578    * @param o Observer no longer interested in changes in set of Readers.
579    */
580   void deleteChangedReaderObserver(ChangedReadersObserver o) {
581     // We don't check if observer present; it may not be (legitimately)
582     this.changedReaderObservers.remove(o);
583   }
584 
585   //////////////////////////////////////////////////////////////////////////////
586   // Compaction
587   //////////////////////////////////////////////////////////////////////////////
588 
589   /**
590    * Compact the StoreFiles.  This method may take some time, so the calling
591    * thread must be able to block for long periods.
592    *
593    * <p>During this time, the Store can work as usual, getting values from
594    * StoreFiles and writing new StoreFiles from the memstore.
595    *
596    * Existing StoreFiles are not destroyed until the new compacted StoreFile is
597    * completely written-out to disk.
598    *
599    * <p>The compactLock prevents multiple simultaneous compactions.
600    * The structureLock prevents us from interfering with other write operations.
601    *
602    * <p>We don't want to hold the structureLock for the whole time, as a compact()
603    * can be lengthy and we want to allow cache-flushes during this period.
604    *
605    * @param forceMajor True to force a major compaction regardless of thresholds
606    * @return row to split around if a split is needed, null otherwise
607    * @throws IOException
608    */
609   StoreSize compact(final boolean forceMajor) throws IOException {
610     boolean forceSplit = this.region.shouldForceSplit();
611     boolean majorcompaction = forceMajor;
612     synchronized (compactLock) {
613       this.lastCompactSize = 0;
614 
615       // filesToCompact are sorted oldest to newest.
616       List<StoreFile> filesToCompact = this.storefiles;
617       if (filesToCompact.isEmpty()) {
618         LOG.debug(this.storeNameStr + ": no store files to compact");
619         return null;
620       }
621 
622       // Check to see if we need to do a major compaction on this region.
623       // If so, change doMajorCompaction to true to skip the incremental
624       // compacting below. Only check if doMajorCompaction is not true.
625       if (!majorcompaction) {
626         majorcompaction = isMajorCompaction(filesToCompact);
627       }
628 
629       boolean references = hasReferences(filesToCompact);
630       if (!majorcompaction && !references &&
631           (forceSplit || (filesToCompact.size() < compactionThreshold))) {
632         return checkSplit(forceSplit);
633       }
634 
635       /* get store file sizes for incremental compacting selection.
636        * normal skew:
637        *
638        *         older ----> newer
639        *     _
640        *    | |   _
641        *    | |  | |   _
642        *  --|-|- |-|- |-|---_-------_-------  minCompactSize
643        *    | |  | |  | |  | |  _  | |
644        *    | |  | |  | |  | | | | | |
645        *    | |  | |  | |  | | | | | |
646        */
647       int countOfFiles = filesToCompact.size();
648       long [] fileSizes = new long[countOfFiles];
649       long [] sumSize = new long[countOfFiles];
650       for (int i = countOfFiles-1; i >= 0; --i) {
651         StoreFile file = filesToCompact.get(i);
652         Path path = file.getPath();
653         if (path == null) {
654           LOG.error("Path is null for " + file);
655           return null;
656         }
657         StoreFile.Reader r = file.getReader();
658         if (r == null) {
659           LOG.error("StoreFile " + file + " has a null Reader");
660           return null;
661         }
662         fileSizes[i] = file.getReader().length();
663         // calculate the sum of fileSizes[i,i+maxFilesToCompact-1) for algo
664         int tooFar = i + this.maxFilesToCompact - 1;
665         sumSize[i] = fileSizes[i]
666                    + ((i+1    < countOfFiles) ? sumSize[i+1]      : 0)
667                    - ((tooFar < countOfFiles) ? fileSizes[tooFar] : 0);
668       }
669 
670       long totalSize = 0;
671       if (!majorcompaction && !references) {
672         // we're doing a minor compaction, let's see what files are applicable
673         int start = 0;
674         double r = this.compactRatio;
675 
676         /* Start at the oldest file and stop when you find the first file that
677          * meets compaction criteria:
678          *   (1) a recently-flushed, small file (i.e. <= minCompactSize)
679          *      OR
680          *   (2) within the compactRatio of sum(newer_files)
681          * Given normal skew, any newer files will also meet this criteria
682          *
683          * Additional Note:
684          * If fileSizes.size() >> maxFilesToCompact, we will recurse on
685          * compact().  Consider the oldest files first to avoid a
686          * situation where we always compact [end-threshold,end).  Then, the
687          * last file becomes an aggregate of the previous compactions.
688          */
689         while(countOfFiles - start >= this.compactionThreshold &&
690               fileSizes[start] >
691                 Math.max(minCompactSize, (long)(sumSize[start+1] * r))) {
692           ++start;
693         }
694         int end = Math.min(countOfFiles, start + this.maxFilesToCompact);
695         totalSize = fileSizes[start]
696                   + ((start+1 < countOfFiles) ? sumSize[start+1] : 0);
697 
698         // if we don't have enough files to compact, just wait
699         if (end - start < this.compactionThreshold) {
700           if (LOG.isDebugEnabled()) {
701             LOG.debug("Skipped compaction of " + this.storeNameStr
702               + " because only " + (end - start) + " file(s) of size "
703               + StringUtils.humanReadableInt(totalSize)
704               + " meet compaction criteria.");
705           }
706           return checkSplit(forceSplit);
707         }
708 
709         if (0 == start && end == countOfFiles) {
710           // we decided all the files were candidates! major compact
711           majorcompaction = true;
712         } else {
713           filesToCompact = new ArrayList<StoreFile>(filesToCompact.subList(start,
714             end));
715         }
716       } else {
717         // all files included in this compaction
718         for (long i : fileSizes) {
719           totalSize += i;
720         }
721       }
722       this.lastCompactSize = totalSize;
723 
724       // Max-sequenceID is the last key in the files we're compacting
725       long maxId = StoreFile.getMaxSequenceIdInList(filesToCompact);
726 
727       // Ready to go.  Have list of files to compact.
728       LOG.info("Started compaction of " + filesToCompact.size() + " file(s) in cf=" +
729           this.storeNameStr +
730         (references? ", hasReferences=true,": " ") + " into " +
731           region.getTmpDir() + ", seqid=" + maxId +
732           ", totalSize=" + StringUtils.humanReadableInt(totalSize));
733       StoreFile.Writer writer = compact(filesToCompact, majorcompaction, maxId);
734       // Move the compaction into place.
735       StoreFile sf = completeCompaction(filesToCompact, writer);
736       if (LOG.isInfoEnabled()) {
737         LOG.info("Completed" + (majorcompaction? " major ": " ") +
738           "compaction of " + filesToCompact.size() +
739           " file(s), new file=" + (sf == null? "none": sf.toString()) +
740           ", size=" + (sf == null? "none": StringUtils.humanReadableInt(sf.getReader().length())) +
741           "; total size for store is " + StringUtils.humanReadableInt(storeSize));
742       }
743     }
744     return checkSplit(forceSplit);
745   }
746 
747   /*
748    * Compact the most recent N files. Essentially a hook for testing.
749    */
750   protected void compactRecent(int N) throws IOException {
751     synchronized(compactLock) {
752       List<StoreFile> filesToCompact = this.storefiles;
753       int count = filesToCompact.size();
754       if (N > count) {
755         throw new RuntimeException("Not enough files");
756       }
757 
758       filesToCompact = new ArrayList<StoreFile>(filesToCompact.subList(count-N, count));
759       long maxId = StoreFile.getMaxSequenceIdInList(filesToCompact);
760       boolean majorcompaction = (N == count);
761 
762       // Ready to go.  Have list of files to compact.
763       StoreFile.Writer writer = compact(filesToCompact, majorcompaction, maxId);
764       // Move the compaction into place.
765       StoreFile sf = completeCompaction(filesToCompact, writer);
766     }
767   }
768 
769   /*
770    * @param files
771    * @return True if any of the files in <code>files</code> are References.
772    */
773   private boolean hasReferences(Collection<StoreFile> files) {
774     if (files != null && files.size() > 0) {
775       for (StoreFile hsf: files) {
776         if (hsf.isReference()) {
777           return true;
778         }
779       }
780     }
781     return false;
782   }
783 
784   /*
785    * Gets lowest timestamp from files in a dir
786    *
787    * @param fs
788    * @param dir
789    * @throws IOException
790    */
791   private static long getLowestTimestamp(FileSystem fs, Path dir) throws IOException {
792     FileStatus[] stats = fs.listStatus(dir);
793     if (stats == null || stats.length == 0) {
794       return 0l;
795     }
796     long lowTimestamp = Long.MAX_VALUE;
797     for (int i = 0; i < stats.length; i++) {
798       long timestamp = stats[i].getModificationTime();
799       if (timestamp < lowTimestamp){
800         lowTimestamp = timestamp;
801       }
802     }
803     return lowTimestamp;
804   }
805 
806   /*
807    * @return True if we should run a major compaction.
808    */
809   boolean isMajorCompaction() throws IOException {
810     return isMajorCompaction(storefiles);
811   }
812 
813   /*
814    * @param filesToCompact Files to compact. Can be null.
815    * @return True if we should run a major compaction.
816    */
817   private boolean isMajorCompaction(final List<StoreFile> filesToCompact) throws IOException {
818     boolean result = false;
819     if (filesToCompact == null || filesToCompact.isEmpty() ||
820         majorCompactionTime == 0) {
821       return result;
822     }
823     // TODO: Use better method for determining stamp of last major (HBASE-2990)
824     long lowTimestamp = getLowestTimestamp(fs,
825       filesToCompact.get(0).getPath().getParent());
826     long now = System.currentTimeMillis();
827     if (lowTimestamp > 0l && lowTimestamp < (now - this.majorCompactionTime)) {
828       // Major compaction time has elapsed.
829       if (filesToCompact.size() == 1) {
830         // Single file
831         StoreFile sf = filesToCompact.get(0);
832         long oldest =
833             (sf.getReader().timeRangeTracker == null) ?
834                 Long.MIN_VALUE :
835                 now - sf.getReader().timeRangeTracker.minimumTimestamp;
836         if (sf.isMajorCompaction() &&
837             (this.ttl == HConstants.FOREVER || oldest < this.ttl)) {
838           if (LOG.isDebugEnabled()) {
839             LOG.debug("Skipping major compaction of " + this.storeNameStr +
840                 " because one (major) compacted file only and oldestTime " +
841                 oldest + "ms is < ttl=" + this.ttl);
842           }
843         }
844       } else {
845         if (LOG.isDebugEnabled()) {
846           LOG.debug("Major compaction triggered on store " + this.storeNameStr +
847             "; time since last major compaction " + (now - lowTimestamp) + "ms");
848         }
849         result = true;
850         this.majorCompactionTime = getNextMajorCompactTime();
851       }
852     }
853     return result;
854   }
855 
856   long getNextMajorCompactTime() {
857     // default = 24hrs
858     long ret = conf.getLong(HConstants.MAJOR_COMPACTION_PERIOD, 1000*60*60*24);
859     if (family.getValue(HConstants.MAJOR_COMPACTION_PERIOD) != null) {
860       String strCompactionTime =
861         family.getValue(HConstants.MAJOR_COMPACTION_PERIOD);
862       ret = (new Long(strCompactionTime)).longValue();
863     }
864 
865     if (ret > 0) {
866       // default = 20% = +/- 4.8 hrs
867       double jitterPct =  conf.getFloat("hbase.hregion.majorcompaction.jitter",
868           0.20F);
869       if (jitterPct > 0) {
870         long jitter = Math.round(ret * jitterPct);
871         ret += jitter - Math.round(2L * jitter * Math.random());
872       }
873     }
874     return ret;
875   }
876 
877   /**
878    * Do a minor/major compaction.  Uses the scan infrastructure to make it easy.
879    *
880    * @param filesToCompact which files to compact
881    * @param majorCompaction true to major compact (prune all deletes, max versions, etc)
882    * @param maxId Readers maximum sequence id.
883    * @return Product of compaction or null if all cells expired or deleted and
884    * nothing made it through the compaction.
885    * @throws IOException
886    */
887   private StoreFile.Writer compact(final List<StoreFile> filesToCompact,
888                                final boolean majorCompaction, final long maxId)
889       throws IOException {
890     // calculate maximum key count after compaction (for blooms)
891     int maxKeyCount = 0;
892     for (StoreFile file : filesToCompact) {
893       StoreFile.Reader r = file.getReader();
894       if (r != null) {
895         // NOTE: getFilterEntries could cause under-sized blooms if the user
896         //       switches bloom type (e.g. from ROW to ROWCOL)
897         long keyCount = (r.getBloomFilterType() == family.getBloomFilterType())
898           ? r.getFilterEntries() : r.getEntries();
899         maxKeyCount += keyCount;
900         if (LOG.isDebugEnabled()) {
901           LOG.debug("Compacting " + file +
902             ", keycount=" + keyCount +
903             ", bloomtype=" + r.getBloomFilterType().toString() +
904             ", size=" + StringUtils.humanReadableInt(r.length()) );
905         }
906       }
907     }
908 
909     // For each file, obtain a scanner:
910     List<StoreFileScanner> scanners = StoreFileScanner
911       .getScannersForStoreFiles(filesToCompact, false, false);
912 
913     // Make the instantiation lazy in case compaction produces no product; i.e.
914     // where all source cells are expired or deleted.
915     StoreFile.Writer writer = null;
916     try {
917       InternalScanner scanner = null;
918       try {
919         Scan scan = new Scan();
920         scan.setMaxVersions(family.getMaxVersions());
921         /* include deletes, unless we are doing a major compaction */
922         scanner = new StoreScanner(this, scan, scanners, !majorCompaction);
923         int bytesWritten = 0;
924         // since scanner.next() can return 'false' but still be delivering data,
925         // we have to use a do/while loop.
926         ArrayList<KeyValue> kvs = new ArrayList<KeyValue>();
927         while (scanner.next(kvs)) {
928           if (writer == null && !kvs.isEmpty()) {
929             writer = createWriterInTmp(maxKeyCount,
930               this.compactionCompression);
931           }
932           if (writer != null) {
933             // output to writer:
934             for (KeyValue kv : kvs) {
935               writer.append(kv);
936 
937               // check periodically to see if a system stop is requested
938               if (Store.closeCheckInterval > 0) {
939                 bytesWritten += kv.getLength();
940                 if (bytesWritten > Store.closeCheckInterval) {
941                   bytesWritten = 0;
942                   if (!this.region.areWritesEnabled()) {
943                     writer.close();
944                     fs.delete(writer.getPath(), false);
945                     throw new InterruptedIOException(
946                         "Aborting compaction of store " + this +
947                         " in region " + this.region +
948                         " because user requested stop.");
949                   }
950                 }
951               }
952             }
953           }
954           kvs.clear();
955         }
956       } finally {
957         if (scanner != null) {
958           scanner.close();
959         }
960       }
961     } finally {
962       if (writer != null) {
963         writer.appendMetadata(maxId, majorCompaction);
964         writer.close();
965       }
966     }
967     return writer;
968   }
969 
970   /*
971    * It's assumed that the compactLock  will be acquired prior to calling this
972    * method!  Otherwise, it is not thread-safe!
973    *
974    * <p>It works by processing a compaction that's been written to disk.
975    *
976    * <p>It is usually invoked at the end of a compaction, but might also be
977    * invoked at HStore startup, if the prior execution died midway through.
978    *
979    * <p>Moving the compacted TreeMap into place means:
980    * <pre>
981    * 1) Moving the new compacted StoreFile into place
982    * 2) Unload all replaced StoreFile, close and collect list to delete.
983    * 3) Loading the new TreeMap.
984    * 4) Compute new store size
985    * </pre>
986    *
987    * @param compactedFiles list of files that were compacted
988    * @param compactedFile StoreFile that is the result of the compaction
989    * @return StoreFile created. May be null.
990    * @throws IOException
991    */
992   private StoreFile completeCompaction(final List<StoreFile> compactedFiles,
993                                        final StoreFile.Writer compactedFile)
994       throws IOException {
995     // 1. Moving the new files into place -- if there is a new file (may not
996     // be if all cells were expired or deleted).
997     StoreFile result = null;
998     if (compactedFile != null) {
999       Path p = null;
1000       try {
1001         p = StoreFile.rename(this.fs, compactedFile.getPath(),
1002           StoreFile.getRandomFilename(fs, this.homedir));
1003       } catch (IOException e) {
1004         LOG.error("Failed move of compacted file " + compactedFile.getPath(), e);
1005         return null;
1006       }
1007       result = new StoreFile(this.fs, p, blockcache, this.conf,
1008           this.family.getBloomFilterType(), this.inMemory);
1009       result.createReader();
1010     }
1011     this.lock.writeLock().lock();
1012     try {
1013       try {
1014         // 2. Unloading
1015         // 3. Loading the new TreeMap.
1016         // Change this.storefiles so it reflects new state but do not
1017         // delete old store files until we have sent out notification of
1018         // change in case old files are still being accessed by outstanding
1019         // scanners.
1020         ArrayList<StoreFile> newStoreFiles = new ArrayList<StoreFile>();
1021         for (StoreFile sf : storefiles) {
1022           if (!compactedFiles.contains(sf)) {
1023             newStoreFiles.add(sf);
1024           }
1025         }
1026 
1027         // If a StoreFile result, move it into place.  May be null.
1028         if (result != null) {
1029           newStoreFiles.add(result);
1030         }
1031 
1032         this.storefiles = sortAndClone(newStoreFiles);
1033 
1034         // Tell observers that list of StoreFiles has changed.
1035         notifyChangedReadersObservers();
1036         // Finally, delete old store files.
1037         for (StoreFile hsf: compactedFiles) {
1038           hsf.deleteReader();
1039         }
1040       } catch (IOException e) {
1041         e = RemoteExceptionHandler.checkIOException(e);
1042         LOG.error("Failed replacing compacted files in " + this.storeNameStr +
1043           ". Compacted file is " + (result == null? "none": result.toString()) +
1044           ".  Files replaced " + compactedFiles.toString() +
1045           " some of which may have been already removed", e);
1046       }
1047       // 4. Compute new store size
1048       this.storeSize = 0L;
1049       for (StoreFile hsf : this.storefiles) {
1050         StoreFile.Reader r = hsf.getReader();
1051         if (r == null) {
1052           LOG.warn("StoreFile " + hsf + " has a null Reader");
1053           continue;
1054         }
1055         this.storeSize += r.length();
1056       }
1057     } finally {
1058       this.lock.writeLock().unlock();
1059     }
1060     return result;
1061   }
1062 
1063   public ImmutableList<StoreFile> sortAndClone(List<StoreFile> storeFiles) {
1064     Collections.sort(storeFiles, StoreFile.Comparators.FLUSH_TIME);
1065     ImmutableList<StoreFile> newList = ImmutableList.copyOf(storeFiles);
1066     return newList;
1067   }
1068 
1069   // ////////////////////////////////////////////////////////////////////////////
1070   // Accessors.
1071   // (This is the only section that is directly useful!)
1072   //////////////////////////////////////////////////////////////////////////////
1073   /**
1074    * @return the number of files in this store
1075    */
1076   public int getNumberOfstorefiles() {
1077     return this.storefiles.size();
1078   }
1079 
1080   /*
1081    * @param wantedVersions How many versions were asked for.
1082    * @return wantedVersions or this families' {@link HConstants#VERSIONS}.
1083    */
1084   int versionsToReturn(final int wantedVersions) {
1085     if (wantedVersions <= 0) {
1086       throw new IllegalArgumentException("Number of versions must be > 0");
1087     }
1088     // Make sure we do not return more than maximum versions for this store.
1089     int maxVersions = this.family.getMaxVersions();
1090     return wantedVersions > maxVersions ? maxVersions: wantedVersions;
1091   }
1092 
1093   static boolean isExpired(final KeyValue key, final long oldestTimestamp) {
1094     return key.getTimestamp() < oldestTimestamp;
1095   }
1096 
1097   /**
1098    * Find the key that matches <i>row</i> exactly, or the one that immediately
1099    * preceeds it. WARNING: Only use this method on a table where writes occur
1100    * with strictly increasing timestamps. This method assumes this pattern of
1101    * writes in order to make it reasonably performant.  Also our search is
1102    * dependent on the axiom that deletes are for cells that are in the container
1103    * that follows whether a memstore snapshot or a storefile, not for the
1104    * current container: i.e. we'll see deletes before we come across cells we
1105    * are to delete. Presumption is that the memstore#kvset is processed before
1106    * memstore#snapshot and so on.
1107    * @param kv First possible item on targeted row; i.e. empty columns, latest
1108    * timestamp and maximum type.
1109    * @return Found keyvalue or null if none found.
1110    * @throws IOException
1111    */
1112   KeyValue getRowKeyAtOrBefore(final KeyValue kv) throws IOException {
1113     GetClosestRowBeforeTracker state = new GetClosestRowBeforeTracker(
1114       this.comparator, kv, this.ttl, this.region.getRegionInfo().isMetaRegion());
1115     this.lock.readLock().lock();
1116     try {
1117       // First go to the memstore.  Pick up deletes and candidates.
1118       this.memstore.getRowKeyAtOrBefore(state);
1119       // Check if match, if we got a candidate on the asked for 'kv' row.
1120       // Process each store file. Run through from newest to oldest.
1121       for (StoreFile sf : Iterables.reverse(storefiles)) {
1122         // Update the candidate keys from the current map file
1123         rowAtOrBeforeFromStoreFile(sf, state);
1124       }
1125       return state.getCandidate();
1126     } finally {
1127       this.lock.readLock().unlock();
1128     }
1129   }
1130 
1131   /*
1132    * Check an individual MapFile for the row at or before a given row.
1133    * @param f
1134    * @param state
1135    * @throws IOException
1136    */
1137   private void rowAtOrBeforeFromStoreFile(final StoreFile f,
1138                                           final GetClosestRowBeforeTracker state)
1139       throws IOException {
1140     StoreFile.Reader r = f.getReader();
1141     if (r == null) {
1142       LOG.warn("StoreFile " + f + " has a null Reader");
1143       return;
1144     }
1145     // TODO: Cache these keys rather than make each time?
1146     byte [] fk = r.getFirstKey();
1147     KeyValue firstKV = KeyValue.createKeyValueFromKey(fk, 0, fk.length);
1148     byte [] lk = r.getLastKey();
1149     KeyValue lastKV = KeyValue.createKeyValueFromKey(lk, 0, lk.length);
1150     KeyValue firstOnRow = state.getTargetKey();
1151     if (this.comparator.compareRows(lastKV, firstOnRow) < 0) {
1152       // If last key in file is not of the target table, no candidates in this
1153       // file.  Return.
1154       if (!state.isTargetTable(lastKV)) return;
1155       // If the row we're looking for is past the end of file, set search key to
1156       // last key. TODO: Cache last and first key rather than make each time.
1157       firstOnRow = new KeyValue(lastKV.getRow(), HConstants.LATEST_TIMESTAMP);
1158     }
1159     // Get a scanner that caches blocks and that uses pread.
1160     HFileScanner scanner = r.getScanner(true, true);
1161     // Seek scanner.  If can't seek it, return.
1162     if (!seekToScanner(scanner, firstOnRow, firstKV)) return;
1163     // If we found candidate on firstOnRow, just return. THIS WILL NEVER HAPPEN!
1164     // Unlikely that there'll be an instance of actual first row in table.
1165     if (walkForwardInSingleRow(scanner, firstOnRow, state)) return;
1166     // If here, need to start backing up.
1167     while (scanner.seekBefore(firstOnRow.getBuffer(), firstOnRow.getKeyOffset(),
1168        firstOnRow.getKeyLength())) {
1169       KeyValue kv = scanner.getKeyValue();
1170       if (!state.isTargetTable(kv)) break;
1171       if (!state.isBetterCandidate(kv)) break;
1172       // Make new first on row.
1173       firstOnRow = new KeyValue(kv.getRow(), HConstants.LATEST_TIMESTAMP);
1174       // Seek scanner.  If can't seek it, break.
1175       if (!seekToScanner(scanner, firstOnRow, firstKV)) break;
1176       // If we find something, break;
1177       if (walkForwardInSingleRow(scanner, firstOnRow, state)) break;
1178     }
1179   }
1180 
1181   /*
1182    * Seek the file scanner to firstOnRow or first entry in file.
1183    * @param scanner
1184    * @param firstOnRow
1185    * @param firstKV
1186    * @return True if we successfully seeked scanner.
1187    * @throws IOException
1188    */
1189   private boolean seekToScanner(final HFileScanner scanner,
1190                                 final KeyValue firstOnRow,
1191                                 final KeyValue firstKV)
1192       throws IOException {
1193     KeyValue kv = firstOnRow;
1194     // If firstOnRow < firstKV, set to firstKV
1195     if (this.comparator.compareRows(firstKV, firstOnRow) == 0) kv = firstKV;
1196     int result = scanner.seekTo(kv.getBuffer(), kv.getKeyOffset(),
1197       kv.getKeyLength());
1198     return result >= 0;
1199   }
1200 
1201   /*
1202    * When we come in here, we are probably at the kv just before we break into
1203    * the row that firstOnRow is on.  Usually need to increment one time to get
1204    * on to the row we are interested in.
1205    * @param scanner
1206    * @param firstOnRow
1207    * @param state
1208    * @return True we found a candidate.
1209    * @throws IOException
1210    */
1211   private boolean walkForwardInSingleRow(final HFileScanner scanner,
1212                                          final KeyValue firstOnRow,
1213                                          final GetClosestRowBeforeTracker state)
1214       throws IOException {
1215     boolean foundCandidate = false;
1216     do {
1217       KeyValue kv = scanner.getKeyValue();
1218       // If we are not in the row, skip.
1219       if (this.comparator.compareRows(kv, firstOnRow) < 0) continue;
1220       // Did we go beyond the target row? If so break.
1221       if (state.isTooFar(kv, firstOnRow)) break;
1222       if (state.isExpired(kv)) {
1223         continue;
1224       }
1225       // If we added something, this row is a contender. break.
1226       if (state.handle(kv)) {
1227         foundCandidate = true;
1228         break;
1229       }
1230     } while(scanner.next());
1231     return foundCandidate;
1232   }
1233 
1234   /**
1235    * Determines if HStore can be split
1236    * @param force Whether to force a split or not.
1237    * @return a StoreSize if store can be split, null otherwise.
1238    */
1239   StoreSize checkSplit(final boolean force) {
1240     this.lock.readLock().lock();
1241     try {
1242       // Iterate through all store files
1243       if (this.storefiles.isEmpty()) {
1244         return null;
1245       }
1246       if (!force && (storeSize < this.desiredMaxFileSize)) {
1247         return null;
1248       }
1249 
1250       if (this.region.getRegionInfo().isMetaRegion()) {
1251         if (force) {
1252           LOG.warn("Cannot split meta regions in HBase 0.20");
1253         }
1254         return null;
1255       }
1256 
1257       // Not splitable if we find a reference store file present in the store.
1258       boolean splitable = true;
1259       long maxSize = 0L;
1260       StoreFile largestSf = null;
1261       for (StoreFile sf : storefiles) {
1262         if (splitable) {
1263           splitable = !sf.isReference();
1264           if (!splitable) {
1265             // RETURN IN MIDDLE OF FUNCTION!!! If not splitable, just return.
1266             if (LOG.isDebugEnabled()) {
1267               LOG.debug(sf +  " is not splittable");
1268             }
1269             return null;
1270           }
1271         }
1272         StoreFile.Reader r = sf.getReader();
1273         if (r == null) {
1274           LOG.warn("Storefile " + sf + " Reader is null");
1275           continue;
1276         }
1277         long size = r.length();
1278         if (size > maxSize) {
1279           // This is the largest one so far
1280           maxSize = size;
1281           largestSf = sf;
1282         }
1283       }
1284       // if the user explicit set a split point, use that
1285       if (this.region.getSplitPoint() != null) {
1286         return new StoreSize(maxSize, this.region.getSplitPoint());
1287       }
1288       StoreFile.Reader r = largestSf.getReader();
1289       if (r == null) {
1290         LOG.warn("Storefile " + largestSf + " Reader is null");
1291         return null;
1292       }
1293       // Get first, last, and mid keys.  Midkey is the key that starts block
1294       // in middle of hfile.  Has column and timestamp.  Need to return just
1295       // the row we want to split on as midkey.
1296       byte [] midkey = r.midkey();
1297       if (midkey != null) {
1298         KeyValue mk = KeyValue.createKeyValueFromKey(midkey, 0, midkey.length);
1299         byte [] fk = r.getFirstKey();
1300         KeyValue firstKey = KeyValue.createKeyValueFromKey(fk, 0, fk.length);
1301         byte [] lk = r.getLastKey();
1302         KeyValue lastKey = KeyValue.createKeyValueFromKey(lk, 0, lk.length);
1303         // if the midkey is the same as the first and last keys, then we cannot
1304         // (ever) split this region.
1305         if (this.comparator.compareRows(mk, firstKey) == 0 &&
1306             this.comparator.compareRows(mk, lastKey) == 0) {
1307           if (LOG.isDebugEnabled()) {
1308             LOG.debug("cannot split because midkey is the same as first or " +
1309               "last row");
1310           }
1311           return null;
1312         }
1313         return new StoreSize(maxSize, mk.getRow());
1314       }
1315     } catch(IOException e) {
1316       LOG.warn("Failed getting store size for " + this.storeNameStr, e);
1317     } finally {
1318       this.lock.readLock().unlock();
1319     }
1320     return null;
1321   }
1322 
1323   /** @return aggregate size of all HStores used in the last compaction */
1324   public long getLastCompactSize() {
1325     return this.lastCompactSize;
1326   }
1327 
1328   /** @return aggregate size of HStore */
1329   public long getSize() {
1330     return storeSize;
1331   }
1332 
1333   //////////////////////////////////////////////////////////////////////////////
1334   // File administration
1335   //////////////////////////////////////////////////////////////////////////////
1336 
1337   /**
1338    * Return a scanner for both the memstore and the HStore files
1339    * @throws IOException
1340    */
1341   public KeyValueScanner getScanner(Scan scan,
1342       final NavigableSet<byte []> targetCols) throws IOException {
1343     lock.readLock().lock();
1344     try {
1345       return new StoreScanner(this, scan, targetCols);
1346     } finally {
1347       lock.readLock().unlock();
1348     }
1349   }
1350 
1351   @Override
1352   public String toString() {
1353     return this.storeNameStr;
1354   }
1355 
1356   /**
1357    * @return Count of store files
1358    */
1359   int getStorefilesCount() {
1360     return this.storefiles.size();
1361   }
1362 
1363   /**
1364    * @return The size of the store files, in bytes.
1365    */
1366   long getStorefilesSize() {
1367     long size = 0;
1368     for (StoreFile s: storefiles) {
1369       StoreFile.Reader r = s.getReader();
1370       if (r == null) {
1371         LOG.warn("StoreFile " + s + " has a null Reader");
1372         continue;
1373       }
1374       size += r.length();
1375     }
1376     return size;
1377   }
1378 
1379   /**
1380    * @return The size of the store file indexes, in bytes.
1381    */
1382   long getStorefilesIndexSize() {
1383     long size = 0;
1384     for (StoreFile s: storefiles) {
1385       StoreFile.Reader r = s.getReader();
1386       if (r == null) {
1387         LOG.warn("StoreFile " + s + " has a null Reader");
1388         continue;
1389       }
1390       size += r.indexSize();
1391     }
1392     return size;
1393   }
1394 
1395   /**
1396    * @return The priority that this store should have in the compaction queue
1397    */
1398   int getCompactPriority() {
1399     return this.blockingStoreFileCount - this.storefiles.size();
1400   }
1401 
1402   /**
1403    * Datastructure that holds size and row to split a file around.
1404    * TODO: Take a KeyValue rather than row.
1405    */
1406   static class StoreSize {
1407     private final long size;
1408     private final byte [] row;
1409 
1410     StoreSize(long size, byte [] row) {
1411       this.size = size;
1412       this.row = row;
1413     }
1414     /* @return the size */
1415     long getSize() {
1416       return size;
1417     }
1418 
1419     byte [] getSplitRow() {
1420       return this.row;
1421     }
1422   }
1423 
1424   HRegion getHRegion() {
1425     return this.region;
1426   }
1427 
1428   HRegionInfo getHRegionInfo() {
1429     return this.region.regionInfo;
1430   }
1431 
1432   /**
1433    * Increments the value for the given row/family/qualifier.
1434    *
1435    * This function will always be seen as atomic by other readers
1436    * because it only puts a single KV to memstore. Thus no
1437    * read/write control necessary.
1438    *
1439    * @param row
1440    * @param f
1441    * @param qualifier
1442    * @param newValue the new value to set into memstore
1443    * @return memstore size delta
1444    * @throws IOException
1445    */
1446   public long updateColumnValue(byte [] row, byte [] f,
1447                                 byte [] qualifier, long newValue)
1448       throws IOException {
1449 
1450     this.lock.readLock().lock();
1451     try {
1452       long now = EnvironmentEdgeManager.currentTimeMillis();
1453 
1454       return this.memstore.updateColumnValue(row,
1455           f,
1456           qualifier,
1457           newValue,
1458           now);
1459 
1460     } finally {
1461       this.lock.readLock().unlock();
1462     }
1463   }
1464 
1465   /**
1466    * Adds or replaces the specified KeyValues.
1467    * <p>
1468    * For each KeyValue specified, if a cell with the same row, family, and
1469    * qualifier exists in MemStore, it will be replaced.  Otherwise, it will just
1470    * be inserted to MemStore.
1471    * <p>
1472    * This operation is atomic on each KeyValue (row/family/qualifier) but not
1473    * necessarily atomic across all of them.
1474    * @param kvs
1475    * @return memstore size delta
1476    * @throws IOException
1477    */
1478   public long upsert(List<KeyValue> kvs)
1479       throws IOException {
1480     this.lock.readLock().lock();
1481     try {
1482       // TODO: Make this operation atomic w/ RWCC
1483       return this.memstore.upsert(kvs);
1484     } finally {
1485       this.lock.readLock().unlock();
1486     }
1487   }
1488 
1489   public StoreFlusher getStoreFlusher(long cacheFlushId) {
1490     return new StoreFlusherImpl(cacheFlushId);
1491   }
1492 
1493   private class StoreFlusherImpl implements StoreFlusher {
1494 
1495     private long cacheFlushId;
1496     private SortedSet<KeyValue> snapshot;
1497     private StoreFile storeFile;
1498     private TimeRangeTracker snapshotTimeRangeTracker;
1499 
1500     private StoreFlusherImpl(long cacheFlushId) {
1501       this.cacheFlushId = cacheFlushId;
1502     }
1503 
1504     @Override
1505     public void prepare() {
1506       memstore.snapshot();
1507       this.snapshot = memstore.getSnapshot();
1508       this.snapshotTimeRangeTracker = memstore.getSnapshotTimeRangeTracker();
1509     }
1510 
1511     @Override
1512     public void flushCache() throws IOException {
1513       storeFile = Store.this.flushCache(cacheFlushId, snapshot, snapshotTimeRangeTracker);
1514     }
1515 
1516     @Override
1517     public boolean commit() throws IOException {
1518       if (storeFile == null) {
1519         return false;
1520       }
1521       // Add new file to store files.  Clear snapshot too while we have
1522       // the Store write lock.
1523       return Store.this.updateStorefiles(storeFile, snapshot);
1524     }
1525   }
1526 
1527   /**
1528    * See if there's too much store files in this store
1529    * @return true if number of store files is greater than
1530    *  the number defined in compactionThreshold
1531    */
1532   public boolean hasTooManyStoreFiles() {
1533     return this.storefiles.size() > this.compactionThreshold;
1534   }
1535 
1536   public static final long FIXED_OVERHEAD = ClassSize.align(
1537       ClassSize.OBJECT + (15 * ClassSize.REFERENCE) +
1538       (6 * Bytes.SIZEOF_LONG) + (1 * Bytes.SIZEOF_DOUBLE) +
1539       (4 * Bytes.SIZEOF_INT) + (Bytes.SIZEOF_BOOLEAN * 2));
1540 
1541   public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD +
1542       ClassSize.OBJECT + ClassSize.REENTRANT_LOCK +
1543       ClassSize.CONCURRENT_SKIPLISTMAP +
1544       ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + ClassSize.OBJECT);
1545 
1546   @Override
1547   public long heapSize() {
1548     return DEEP_OVERHEAD + this.memstore.heapSize();
1549   }
1550 }