View Javadoc

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