View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.regionserver;
20  
21  import java.io.IOException;
22  import java.io.InterruptedIOException;
23  import java.net.InetSocketAddress;
24  import java.security.Key;
25  import java.security.KeyException;
26  import java.util.ArrayList;
27  import java.util.Collection;
28  import java.util.Collections;
29  import java.util.HashMap;
30  import java.util.HashSet;
31  import java.util.Iterator;
32  import java.util.List;
33  import java.util.NavigableSet;
34  import java.util.Set;
35  import java.util.concurrent.Callable;
36  import java.util.concurrent.CompletionService;
37  import java.util.concurrent.ConcurrentHashMap;
38  import java.util.concurrent.ExecutionException;
39  import java.util.concurrent.ExecutorCompletionService;
40  import java.util.concurrent.Future;
41  import java.util.concurrent.ThreadPoolExecutor;
42  import java.util.concurrent.atomic.AtomicBoolean;
43  import java.util.concurrent.locks.ReentrantReadWriteLock;
44  
45  import org.apache.commons.logging.Log;
46  import org.apache.commons.logging.LogFactory;
47  import org.apache.hadoop.conf.Configuration;
48  import org.apache.hadoop.fs.FileSystem;
49  import org.apache.hadoop.fs.Path;
50  import org.apache.hadoop.hbase.Cell;
51  import org.apache.hadoop.hbase.CellComparator;
52  import org.apache.hadoop.hbase.CellUtil;
53  import org.apache.hadoop.hbase.CompoundConfiguration;
54  import org.apache.hadoop.hbase.HColumnDescriptor;
55  import org.apache.hadoop.hbase.HConstants;
56  import org.apache.hadoop.hbase.HRegionInfo;
57  import org.apache.hadoop.hbase.KeyValue;
58  import org.apache.hadoop.hbase.TableName;
59  import org.apache.hadoop.hbase.Tag;
60  import org.apache.hadoop.hbase.TagType;
61  import org.apache.hadoop.hbase.classification.InterfaceAudience;
62  import org.apache.hadoop.hbase.client.Scan;
63  import org.apache.hadoop.hbase.conf.ConfigurationManager;
64  import org.apache.hadoop.hbase.io.compress.Compression;
65  import org.apache.hadoop.hbase.io.crypto.Cipher;
66  import org.apache.hadoop.hbase.io.crypto.Encryption;
67  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
68  import org.apache.hadoop.hbase.io.hfile.HFile;
69  import org.apache.hadoop.hbase.io.hfile.HFileContext;
70  import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
71  import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
72  import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoderImpl;
73  import org.apache.hadoop.hbase.io.hfile.HFileScanner;
74  import org.apache.hadoop.hbase.io.hfile.InvalidHFileException;
75  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
76  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
77  import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
78  import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
79  import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
80  import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
81  import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
82  import org.apache.hadoop.hbase.regionserver.compactions.OffPeakHours;
83  import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
84  import org.apache.hadoop.hbase.security.EncryptionUtil;
85  import org.apache.hadoop.hbase.security.User;
86  import org.apache.hadoop.hbase.util.Bytes;
87  import org.apache.hadoop.hbase.util.ChecksumType;
88  import org.apache.hadoop.hbase.util.ClassSize;
89  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
90  import org.apache.hadoop.hbase.util.Pair;
91  import org.apache.hadoop.hbase.util.ReflectionUtils;
92  import org.apache.hadoop.ipc.RemoteException;
93  import org.apache.hadoop.util.StringUtils;
94  
95  import com.google.common.annotations.VisibleForTesting;
96  import com.google.common.base.Preconditions;
97  import com.google.common.collect.ImmutableCollection;
98  import com.google.common.collect.ImmutableList;
99  import com.google.common.collect.Lists;
100 import com.google.common.collect.Sets;
101 
102 /**
103  * A Store holds a column family in a Region.  Its a memstore and a set of zero
104  * or more StoreFiles, which stretch backwards over time.
105  *
106  * <p>There's no reason to consider append-logging at this level; all logging
107  * and locking is handled at the HRegion level.  Store just provides
108  * services to manage sets of StoreFiles.  One of the most important of those
109  * services is compaction services where files are aggregated once they pass
110  * a configurable threshold.
111  *
112  * <p>The only thing having to do with logs that Store needs to deal with is
113  * the reconstructionLog.  This is a segment of an HRegion's log that might
114  * NOT be present upon startup.  If the param is NULL, there's nothing to do.
115  * If the param is non-NULL, we need to process the log to reconstruct
116  * a TreeMap that might not have been written to disk before the process
117  * died.
118  *
119  * <p>It's assumed that after this constructor returns, the reconstructionLog
120  * file will be deleted (by whoever has instantiated the Store).
121  *
122  * <p>Locking and transactions are handled at a higher level.  This API should
123  * not be called directly but by an HRegion manager.
124  */
125 @InterfaceAudience.Private
126 public class HStore implements Store {
127   private static final String MEMSTORE_CLASS_NAME = "hbase.regionserver.memstore.class";
128   public static final String COMPACTCHECKER_INTERVAL_MULTIPLIER_KEY =
129       "hbase.server.compactchecker.interval.multiplier";
130   public static final String BLOCKING_STOREFILES_KEY = "hbase.hstore.blockingStoreFiles";
131   public static final int DEFAULT_COMPACTCHECKER_INTERVAL_MULTIPLIER = 1000;
132   public static final int DEFAULT_BLOCKING_STOREFILE_COUNT = 7;
133 
134   static final Log LOG = LogFactory.getLog(HStore.class);
135 
136   protected final MemStore memstore;
137   // This stores directory in the filesystem.
138   private final HRegion region;
139   private final HColumnDescriptor family;
140   private final HRegionFileSystem fs;
141   private Configuration conf;
142   private final CacheConfig cacheConf;
143   private long lastCompactSize = 0;
144   volatile boolean forceMajor = false;
145   /* how many bytes to write between status checks */
146   static int closeCheckInterval = 0;
147   private volatile long storeSize = 0L;
148   private volatile long totalUncompressedBytes = 0L;
149 
150   /**
151    * RWLock for store operations.
152    * Locked in shared mode when the list of component stores is looked at:
153    *   - all reads/writes to table data
154    *   - checking for split
155    * Locked in exclusive mode when the list of component stores is modified:
156    *   - closing
157    *   - completing a compaction
158    */
159   final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
160   private final boolean verifyBulkLoads;
161 
162   private ScanInfo scanInfo;
163 
164   // TODO: ideally, this should be part of storeFileManager, as we keep passing this to it.
165   final List<StoreFile> filesCompacting = Lists.newArrayList();
166 
167   // All access must be synchronized.
168   private final Set<ChangedReadersObserver> changedReaderObservers =
169     Collections.newSetFromMap(new ConcurrentHashMap<ChangedReadersObserver, Boolean>());
170 
171   private final int blocksize;
172   private HFileDataBlockEncoder dataBlockEncoder;
173 
174   /** Checksum configuration */
175   private ChecksumType checksumType;
176   private int bytesPerChecksum;
177 
178   // Comparing KeyValues
179   private final KeyValue.KVComparator comparator;
180 
181   final StoreEngine<?, ?, ?, ?> storeEngine;
182 
183   private static final AtomicBoolean offPeakCompactionTracker = new AtomicBoolean();
184   private volatile OffPeakHours offPeakHours;
185 
186   private static final int DEFAULT_FLUSH_RETRIES_NUMBER = 10;
187   private int flushRetriesNumber;
188   private int pauseTime;
189 
190   private long blockingFileCount;
191   private int compactionCheckMultiplier;
192 
193   private Encryption.Context cryptoContext = Encryption.Context.NONE;
194 
195   private volatile long flushedCellsCount = 0;
196   private volatile long compactedCellsCount = 0;
197   private volatile long majorCompactedCellsCount = 0;
198   private volatile long flushedCellsSize = 0;
199   private volatile long compactedCellsSize = 0;
200   private volatile long majorCompactedCellsSize = 0;
201 
202   /**
203    * Constructor
204    * @param region
205    * @param family HColumnDescriptor for this column
206    * @param confParam configuration object
207    * failed.  Can be null.
208    * @throws IOException
209    */
210   protected HStore(final HRegion region, final HColumnDescriptor family,
211       final Configuration confParam) throws IOException {
212 
213     HRegionInfo info = region.getRegionInfo();
214     this.fs = region.getRegionFileSystem();
215 
216     // Assemble the store's home directory and Ensure it exists.
217     fs.createStoreDir(family.getNameAsString());
218     this.region = region;
219     this.family = family;
220     // 'conf' renamed to 'confParam' b/c we use this.conf in the constructor
221     // CompoundConfiguration will look for keys in reverse order of addition, so we'd
222     // add global config first, then table and cf overrides, then cf metadata.
223     this.conf = new CompoundConfiguration()
224       .add(confParam)
225       .addStringMap(region.getTableDesc().getConfiguration())
226       .addStringMap(family.getConfiguration())
227       .addBytesMap(family.getValues());
228     this.blocksize = family.getBlocksize();
229 
230     this.dataBlockEncoder =
231         new HFileDataBlockEncoderImpl(family.getDataBlockEncoding());
232 
233     this.comparator = info.getComparator();
234     // used by ScanQueryMatcher
235     long timeToPurgeDeletes =
236         Math.max(conf.getLong("hbase.hstore.time.to.purge.deletes", 0), 0);
237     LOG.trace("Time to purge deletes set to " + timeToPurgeDeletes +
238         "ms in store " + this);
239     // Get TTL
240     long ttl = determineTTLFromFamily(family);
241     // Why not just pass a HColumnDescriptor in here altogether?  Even if have
242     // to clone it?
243     scanInfo = new ScanInfo(family, ttl, timeToPurgeDeletes, this.comparator);
244     String className = conf.get(MEMSTORE_CLASS_NAME, DefaultMemStore.class.getName());
245     this.memstore = ReflectionUtils.instantiateWithCustomCtor(className, new Class[] {
246         Configuration.class, KeyValue.KVComparator.class }, new Object[] { conf, this.comparator });
247     this.offPeakHours = OffPeakHours.getInstance(conf);
248 
249     // Setting up cache configuration for this family
250     this.cacheConf = new CacheConfig(conf, family);
251 
252     this.verifyBulkLoads = conf.getBoolean("hbase.hstore.bulkload.verify", false);
253 
254     this.blockingFileCount =
255         conf.getInt(BLOCKING_STOREFILES_KEY, DEFAULT_BLOCKING_STOREFILE_COUNT);
256     this.compactionCheckMultiplier = conf.getInt(
257         COMPACTCHECKER_INTERVAL_MULTIPLIER_KEY, DEFAULT_COMPACTCHECKER_INTERVAL_MULTIPLIER);
258     if (this.compactionCheckMultiplier <= 0) {
259       LOG.error("Compaction check period multiplier must be positive, setting default: "
260           + DEFAULT_COMPACTCHECKER_INTERVAL_MULTIPLIER);
261       this.compactionCheckMultiplier = DEFAULT_COMPACTCHECKER_INTERVAL_MULTIPLIER;
262     }
263 
264     if (HStore.closeCheckInterval == 0) {
265       HStore.closeCheckInterval = conf.getInt(
266           "hbase.hstore.close.check.interval", 10*1000*1000 /* 10 MB */);
267     }
268 
269     this.storeEngine = StoreEngine.create(this, this.conf, this.comparator);
270     this.storeEngine.getStoreFileManager().loadFiles(loadStoreFiles());
271 
272     // Initialize checksum type from name. The names are CRC32, CRC32C, etc.
273     this.checksumType = getChecksumType(conf);
274     // initilize bytes per checksum
275     this.bytesPerChecksum = getBytesPerChecksum(conf);
276     flushRetriesNumber = conf.getInt(
277         "hbase.hstore.flush.retries.number", DEFAULT_FLUSH_RETRIES_NUMBER);
278     pauseTime = conf.getInt(HConstants.HBASE_SERVER_PAUSE, HConstants.DEFAULT_HBASE_SERVER_PAUSE);
279     if (flushRetriesNumber <= 0) {
280       throw new IllegalArgumentException(
281           "hbase.hstore.flush.retries.number must be > 0, not "
282               + flushRetriesNumber);
283     }
284 
285     // Crypto context for new store files
286     String cipherName = family.getEncryptionType();
287     if (cipherName != null) {
288       Cipher cipher;
289       Key key;
290       byte[] keyBytes = family.getEncryptionKey();
291       if (keyBytes != null) {
292         // Family provides specific key material
293         String masterKeyName = conf.get(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY,
294           User.getCurrent().getShortName());
295         try {
296           // First try the master key
297           key = EncryptionUtil.unwrapKey(conf, masterKeyName, keyBytes);
298         } catch (KeyException e) {
299           // If the current master key fails to unwrap, try the alternate, if
300           // one is configured
301           if (LOG.isDebugEnabled()) {
302             LOG.debug("Unable to unwrap key with current master key '" + masterKeyName + "'");
303           }
304           String alternateKeyName =
305             conf.get(HConstants.CRYPTO_MASTERKEY_ALTERNATE_NAME_CONF_KEY);
306           if (alternateKeyName != null) {
307             try {
308               key = EncryptionUtil.unwrapKey(conf, alternateKeyName, keyBytes);
309             } catch (KeyException ex) {
310               throw new IOException(ex);
311             }
312           } else {
313             throw new IOException(e);
314           }
315         }
316         // Use the algorithm the key wants
317         cipher = Encryption.getCipher(conf, key.getAlgorithm());
318         if (cipher == null) {
319           throw new RuntimeException("Cipher '" + cipher + "' is not available");
320         }
321         // Fail if misconfigured
322         // We use the encryption type specified in the column schema as a sanity check on
323         // what the wrapped key is telling us
324         if (!cipher.getName().equalsIgnoreCase(cipherName)) {
325           throw new RuntimeException("Encryption for family '" + family.getNameAsString() +
326             "' configured with type '" + cipherName +
327             "' but key specifies algorithm '" + cipher.getName() + "'");
328         }
329       } else {
330         // Family does not provide key material, create a random key
331         cipher = Encryption.getCipher(conf, cipherName);
332         if (cipher == null) {
333           throw new RuntimeException("Cipher '" + cipher + "' is not available");
334         }
335         key = cipher.getRandomKey();
336       }
337       cryptoContext = Encryption.newContext(conf);
338       cryptoContext.setCipher(cipher);
339       cryptoContext.setKey(key);
340     }
341   }
342 
343   /**
344    * @param family
345    * @return TTL in seconds of the specified family
346    */
347   private static long determineTTLFromFamily(final HColumnDescriptor family) {
348     // HCD.getTimeToLive returns ttl in seconds.  Convert to milliseconds.
349     long ttl = family.getTimeToLive();
350     if (ttl == HConstants.FOREVER) {
351       // Default is unlimited ttl.
352       ttl = Long.MAX_VALUE;
353     } else if (ttl == -1) {
354       ttl = Long.MAX_VALUE;
355     } else {
356       // Second -> ms adjust for user data
357       ttl *= 1000;
358     }
359     return ttl;
360   }
361 
362   @Override
363   public String getColumnFamilyName() {
364     return this.family.getNameAsString();
365   }
366 
367   @Override
368   public TableName getTableName() {
369     return this.getRegionInfo().getTable();
370   }
371 
372   @Override
373   public FileSystem getFileSystem() {
374     return this.fs.getFileSystem();
375   }
376 
377   public HRegionFileSystem getRegionFileSystem() {
378     return this.fs;
379   }
380 
381   /* Implementation of StoreConfigInformation */
382   @Override
383   public long getStoreFileTtl() {
384     // TTL only applies if there's no MIN_VERSIONs setting on the column.
385     return (this.scanInfo.getMinVersions() == 0) ? this.scanInfo.getTtl() : Long.MAX_VALUE;
386   }
387 
388   @Override
389   public long getMemstoreFlushSize() {
390     // TODO: Why is this in here?  The flushsize of the region rather than the store?  St.Ack
391     return this.region.memstoreFlushSize;
392   }
393 
394   @Override
395   public long getFlushableSize() {
396     return this.memstore.getFlushableSize();
397   }
398 
399   @Override
400   public long getCompactionCheckMultiplier() {
401     return this.compactionCheckMultiplier;
402   }
403 
404   @Override
405   public long getBlockingFileCount() {
406     return blockingFileCount;
407   }
408   /* End implementation of StoreConfigInformation */
409 
410   /**
411    * Returns the configured bytesPerChecksum value.
412    * @param conf The configuration
413    * @return The bytesPerChecksum that is set in the configuration
414    */
415   public static int getBytesPerChecksum(Configuration conf) {
416     return conf.getInt(HConstants.BYTES_PER_CHECKSUM,
417                        HFile.DEFAULT_BYTES_PER_CHECKSUM);
418   }
419 
420   /**
421    * Returns the configured checksum algorithm.
422    * @param conf The configuration
423    * @return The checksum algorithm that is set in the configuration
424    */
425   public static ChecksumType getChecksumType(Configuration conf) {
426     String checksumName = conf.get(HConstants.CHECKSUM_TYPE_NAME);
427     if (checksumName == null) {
428       return HFile.DEFAULT_CHECKSUM_TYPE;
429     } else {
430       return ChecksumType.nameToType(checksumName);
431     }
432   }
433 
434   /**
435    * @return how many bytes to write between status checks
436    */
437   public static int getCloseCheckInterval() {
438     return closeCheckInterval;
439   }
440 
441   @Override
442   public HColumnDescriptor getFamily() {
443     return this.family;
444   }
445 
446   /**
447    * @return The maximum sequence id in all store files. Used for log replay.
448    */
449   long getMaxSequenceId() {
450     return StoreFile.getMaxSequenceIdInList(this.getStorefiles());
451   }
452 
453   @Override
454   public long getMaxMemstoreTS() {
455     return StoreFile.getMaxMemstoreTSInList(this.getStorefiles());
456   }
457 
458   /**
459    * @param tabledir {@link Path} to where the table is being stored
460    * @param hri {@link HRegionInfo} for the region.
461    * @param family {@link HColumnDescriptor} describing the column family
462    * @return Path to family/Store home directory.
463    */
464   @Deprecated
465   public static Path getStoreHomedir(final Path tabledir,
466       final HRegionInfo hri, final byte[] family) {
467     return getStoreHomedir(tabledir, hri.getEncodedName(), family);
468   }
469 
470   /**
471    * @param tabledir {@link Path} to where the table is being stored
472    * @param encodedName Encoded region name.
473    * @param family {@link HColumnDescriptor} describing the column family
474    * @return Path to family/Store home directory.
475    */
476   @Deprecated
477   public static Path getStoreHomedir(final Path tabledir,
478       final String encodedName, final byte[] family) {
479     return new Path(tabledir, new Path(encodedName, Bytes.toString(family)));
480   }
481 
482   @Override
483   public HFileDataBlockEncoder getDataBlockEncoder() {
484     return dataBlockEncoder;
485   }
486 
487   /**
488    * Should be used only in tests.
489    * @param blockEncoder the block delta encoder to use
490    */
491   void setDataBlockEncoderInTest(HFileDataBlockEncoder blockEncoder) {
492     this.dataBlockEncoder = blockEncoder;
493   }
494 
495   /**
496    * Creates an unsorted list of StoreFile loaded in parallel
497    * from the given directory.
498    * @throws IOException
499    */
500   private List<StoreFile> loadStoreFiles() throws IOException {
501     Collection<StoreFileInfo> files = fs.getStoreFiles(getColumnFamilyName());
502     return openStoreFiles(files);
503   }
504 
505   private List<StoreFile> openStoreFiles(Collection<StoreFileInfo> files) throws IOException {
506     if (files == null || files.size() == 0) {
507       return new ArrayList<StoreFile>();
508     }
509     // initialize the thread pool for opening store files in parallel..
510     ThreadPoolExecutor storeFileOpenerThreadPool =
511       this.region.getStoreFileOpenAndCloseThreadPool("StoreFileOpenerThread-" +
512           this.getColumnFamilyName());
513     CompletionService<StoreFile> completionService =
514       new ExecutorCompletionService<StoreFile>(storeFileOpenerThreadPool);
515 
516     int totalValidStoreFile = 0;
517     for (final StoreFileInfo storeFileInfo: files) {
518       // open each store file in parallel
519       completionService.submit(new Callable<StoreFile>() {
520         @Override
521         public StoreFile call() throws IOException {
522           StoreFile storeFile = createStoreFileAndReader(storeFileInfo);
523           return storeFile;
524         }
525       });
526       totalValidStoreFile++;
527     }
528 
529     ArrayList<StoreFile> results = new ArrayList<StoreFile>(files.size());
530     IOException ioe = null;
531     try {
532       for (int i = 0; i < totalValidStoreFile; i++) {
533         try {
534           Future<StoreFile> future = completionService.take();
535           StoreFile storeFile = future.get();
536           long length = storeFile.getReader().length();
537           this.storeSize += length;
538           this.totalUncompressedBytes +=
539               storeFile.getReader().getTotalUncompressedBytes();
540           if (LOG.isDebugEnabled()) {
541             LOG.debug("loaded " + storeFile.toStringDetailed());
542           }
543           results.add(storeFile);
544         } catch (InterruptedException e) {
545           if (ioe == null) ioe = new InterruptedIOException(e.getMessage());
546         } catch (ExecutionException e) {
547           if (ioe == null) ioe = new IOException(e.getCause());
548         }
549       }
550     } finally {
551       storeFileOpenerThreadPool.shutdownNow();
552     }
553     if (ioe != null) {
554       // close StoreFile readers
555       for (StoreFile file : results) {
556         try {
557           if (file != null) file.closeReader(true);
558         } catch (IOException e) {
559           LOG.warn(e.getMessage());
560         }
561       }
562       throw ioe;
563     }
564 
565     return results;
566   }
567 
568   /**
569    * Checks the underlying store files, and opens the files that  have not
570    * been opened, and removes the store file readers for store files no longer
571    * available. Mainly used by secondary region replicas to keep up to date with
572    * the primary region files.
573    * @throws IOException
574    */
575   @Override
576   public void refreshStoreFiles() throws IOException {
577     StoreFileManager sfm = storeEngine.getStoreFileManager();
578     Collection<StoreFile> currentFiles = sfm.getStorefiles();
579     if (currentFiles == null) currentFiles = new ArrayList<StoreFile>(0);
580 
581     Collection<StoreFileInfo> newFiles = fs.getStoreFiles(getColumnFamilyName());
582     if (newFiles == null) newFiles = new ArrayList<StoreFileInfo>(0);
583 
584     HashMap<StoreFileInfo, StoreFile> currentFilesSet = new HashMap<StoreFileInfo, StoreFile>(currentFiles.size());
585     for (StoreFile sf : currentFiles) {
586       currentFilesSet.put(sf.getFileInfo(), sf);
587     }
588     HashSet<StoreFileInfo> newFilesSet = new HashSet<StoreFileInfo>(newFiles);
589 
590     Set<StoreFileInfo> toBeAddedFiles = Sets.difference(newFilesSet, currentFilesSet.keySet());
591     Set<StoreFileInfo> toBeRemovedFiles = Sets.difference(currentFilesSet.keySet(), newFilesSet);
592 
593     if (toBeAddedFiles.isEmpty() && toBeRemovedFiles.isEmpty()) {
594       return;
595     }
596 
597     LOG.info("Refreshing store files for region " + this.getRegionInfo().getRegionNameAsString()
598       + " files to add: " + toBeAddedFiles + " files to remove: " + toBeRemovedFiles);
599 
600     Set<StoreFile> toBeRemovedStoreFiles = new HashSet<StoreFile>(toBeRemovedFiles.size());
601     for (StoreFileInfo sfi : toBeRemovedFiles) {
602       toBeRemovedStoreFiles.add(currentFilesSet.get(sfi));
603     }
604 
605     // try to open the files
606     List<StoreFile> openedFiles = openStoreFiles(toBeAddedFiles);
607 
608     // propogate the file changes to the underlying store file manager
609     replaceStoreFiles(toBeRemovedStoreFiles, openedFiles); //won't throw an exception
610 
611     // Advance the memstore read point to be at least the new store files seqIds so that
612     // readers might pick it up. This assumes that the store is not getting any writes (otherwise
613     // in-flight transactions might be made visible)
614     if (!toBeAddedFiles.isEmpty()) {
615       region.getMVCC().advanceMemstoreReadPointIfNeeded(this.getMaxSequenceId());
616     }
617 
618     // notify scanners, close file readers, and recompute store size
619     completeCompaction(toBeRemovedStoreFiles, false);
620   }
621 
622   private StoreFile createStoreFileAndReader(final Path p) throws IOException {
623     StoreFileInfo info = new StoreFileInfo(conf, this.getFileSystem(), p);
624     return createStoreFileAndReader(info);
625   }
626 
627   private StoreFile createStoreFileAndReader(final StoreFileInfo info)
628       throws IOException {
629     info.setRegionCoprocessorHost(this.region.getCoprocessorHost());
630     StoreFile storeFile = new StoreFile(this.getFileSystem(), info, this.conf, this.cacheConf,
631       this.family.getBloomFilterType());
632     storeFile.createReader();
633     return storeFile;
634   }
635 
636   @Override
637   public Pair<Long, Cell> add(final Cell cell) {
638     lock.readLock().lock();
639     try {
640        return this.memstore.add(cell);
641     } finally {
642       lock.readLock().unlock();
643     }
644   }
645 
646   @Override
647   public long timeOfOldestEdit() {
648     return memstore.timeOfOldestEdit();
649   }
650 
651   /**
652    * Adds a value to the memstore
653    *
654    * @param kv
655    * @return memstore size delta
656    */
657   protected long delete(final KeyValue kv) {
658     lock.readLock().lock();
659     try {
660       return this.memstore.delete(kv);
661     } finally {
662       lock.readLock().unlock();
663     }
664   }
665 
666   @Override
667   public void rollback(final Cell cell) {
668     lock.readLock().lock();
669     try {
670       this.memstore.rollback(cell);
671     } finally {
672       lock.readLock().unlock();
673     }
674   }
675 
676   /**
677    * @return All store files.
678    */
679   @Override
680   public Collection<StoreFile> getStorefiles() {
681     return this.storeEngine.getStoreFileManager().getStorefiles();
682   }
683 
684   @Override
685   public void assertBulkLoadHFileOk(Path srcPath) throws IOException {
686     HFile.Reader reader  = null;
687     try {
688       LOG.info("Validating hfile at " + srcPath + " for inclusion in "
689           + "store " + this + " region " + this.getRegionInfo().getRegionNameAsString());
690       reader = HFile.createReader(srcPath.getFileSystem(conf),
691           srcPath, cacheConf, conf);
692       reader.loadFileInfo();
693 
694       byte[] firstKey = reader.getFirstRowKey();
695       Preconditions.checkState(firstKey != null, "First key can not be null");
696       byte[] lk = reader.getLastKey();
697       Preconditions.checkState(lk != null, "Last key can not be null");
698       byte[] lastKey =  KeyValue.createKeyValueFromKey(lk).getRow();
699 
700       LOG.debug("HFile bounds: first=" + Bytes.toStringBinary(firstKey) +
701           " last=" + Bytes.toStringBinary(lastKey));
702       LOG.debug("Region bounds: first=" +
703           Bytes.toStringBinary(getRegionInfo().getStartKey()) +
704           " last=" + Bytes.toStringBinary(getRegionInfo().getEndKey()));
705 
706       if (!this.getRegionInfo().containsRange(firstKey, lastKey)) {
707         throw new WrongRegionException(
708             "Bulk load file " + srcPath.toString() + " does not fit inside region "
709             + this.getRegionInfo().getRegionNameAsString());
710       }
711 
712       if (verifyBulkLoads) {
713         long verificationStartTime = EnvironmentEdgeManager.currentTime();
714         LOG.info("Full verification started for bulk load hfile: " + srcPath.toString());
715         Cell prevCell = null;
716         HFileScanner scanner = reader.getScanner(false, false, false);
717         scanner.seekTo();
718         do {
719           Cell cell = scanner.getKeyValue();
720           if (prevCell != null) {
721             if (CellComparator.compareRows(prevCell, cell) > 0) {
722               throw new InvalidHFileException("Previous row is greater than"
723                   + " current row: path=" + srcPath + " previous="
724                   + CellUtil.getCellKeyAsString(prevCell) + " current="
725                   + CellUtil.getCellKeyAsString(cell));
726             }
727             if (CellComparator.compareFamilies(prevCell, cell) != 0) {
728               throw new InvalidHFileException("Previous key had different"
729                   + " family compared to current key: path=" + srcPath
730                   + " previous="
731                   + Bytes.toStringBinary(prevCell.getFamilyArray(), prevCell.getFamilyOffset(),
732                       prevCell.getFamilyLength())
733                   + " current="
734                   + Bytes.toStringBinary(cell.getFamilyArray(), cell.getFamilyOffset(),
735                       cell.getFamilyLength()));
736             }
737           }
738           prevCell = cell;
739         } while (scanner.next());
740       LOG.info("Full verification complete for bulk load hfile: " + srcPath.toString()
741          + " took " + (EnvironmentEdgeManager.currentTime() - verificationStartTime)
742          + " ms");
743       }
744     } finally {
745       if (reader != null) reader.close();
746     }
747   }
748 
749   @Override
750   public void bulkLoadHFile(String srcPathStr, long seqNum) throws IOException {
751     Path srcPath = new Path(srcPathStr);
752     Path dstPath = fs.bulkLoadStoreFile(getColumnFamilyName(), srcPath, seqNum);
753 
754     StoreFile sf = createStoreFileAndReader(dstPath);
755 
756     StoreFile.Reader r = sf.getReader();
757     this.storeSize += r.length();
758     this.totalUncompressedBytes += r.getTotalUncompressedBytes();
759 
760     LOG.info("Loaded HFile " + srcPath + " into store '" + getColumnFamilyName() +
761         "' as " + dstPath + " - updating store file list.");
762 
763     // Append the new storefile into the list
764     this.lock.writeLock().lock();
765     try {
766       this.storeEngine.getStoreFileManager().insertNewFiles(Lists.newArrayList(sf));
767     } finally {
768       // We need the lock, as long as we are updating the storeFiles
769       // or changing the memstore. Let us release it before calling
770       // notifyChangeReadersObservers. See HBASE-4485 for a possible
771       // deadlock scenario that could have happened if continue to hold
772       // the lock.
773       this.lock.writeLock().unlock();
774     }
775     notifyChangedReadersObservers();
776     LOG.info("Successfully loaded store file " + srcPath
777         + " into store " + this + " (new location: " + dstPath + ")");
778     if (LOG.isTraceEnabled()) {
779       String traceMessage = "BULK LOAD time,size,store size,store files ["
780           + EnvironmentEdgeManager.currentTime() + "," + r.length() + "," + storeSize
781           + "," + storeEngine.getStoreFileManager().getStorefileCount() + "]";
782       LOG.trace(traceMessage);
783     }
784   }
785 
786   @Override
787   public ImmutableCollection<StoreFile> close() throws IOException {
788     this.lock.writeLock().lock();
789     try {
790       // Clear so metrics doesn't find them.
791       ImmutableCollection<StoreFile> result = storeEngine.getStoreFileManager().clearFiles();
792 
793       if (!result.isEmpty()) {
794         // initialize the thread pool for closing store files in parallel.
795         ThreadPoolExecutor storeFileCloserThreadPool = this.region
796             .getStoreFileOpenAndCloseThreadPool("StoreFileCloserThread-"
797                 + this.getColumnFamilyName());
798 
799         // close each store file in parallel
800         CompletionService<Void> completionService =
801           new ExecutorCompletionService<Void>(storeFileCloserThreadPool);
802         for (final StoreFile f : result) {
803           completionService.submit(new Callable<Void>() {
804             @Override
805             public Void call() throws IOException {
806               f.closeReader(true);
807               return null;
808             }
809           });
810         }
811 
812         IOException ioe = null;
813         try {
814           for (int i = 0; i < result.size(); i++) {
815             try {
816               Future<Void> future = completionService.take();
817               future.get();
818             } catch (InterruptedException e) {
819               if (ioe == null) {
820                 ioe = new InterruptedIOException();
821                 ioe.initCause(e);
822               }
823             } catch (ExecutionException e) {
824               if (ioe == null) ioe = new IOException(e.getCause());
825             }
826           }
827         } finally {
828           storeFileCloserThreadPool.shutdownNow();
829         }
830         if (ioe != null) throw ioe;
831       }
832       LOG.info("Closed " + this);
833       return result;
834     } finally {
835       this.lock.writeLock().unlock();
836     }
837   }
838 
839   /**
840    * Snapshot this stores memstore. Call before running
841    * {@link #flushCache(long, MemStoreSnapshot, MonitoredTask)}
842    *  so it has some work to do.
843    */
844   void snapshot() {
845     this.lock.writeLock().lock();
846     try {
847       this.memstore.snapshot();
848     } finally {
849       this.lock.writeLock().unlock();
850     }
851   }
852 
853   /**
854    * Write out current snapshot.  Presumes {@link #snapshot()} has been called
855    * previously.
856    * @param logCacheFlushId flush sequence number
857    * @param snapshot
858    * @param status
859    * @return The path name of the tmp file to which the store was flushed
860    * @throws IOException
861    */
862   protected List<Path> flushCache(final long logCacheFlushId, MemStoreSnapshot snapshot,
863       MonitoredTask status) throws IOException {
864     // If an exception happens flushing, we let it out without clearing
865     // the memstore snapshot.  The old snapshot will be returned when we say
866     // 'snapshot', the next time flush comes around.
867     // Retry after catching exception when flushing, otherwise server will abort
868     // itself
869     StoreFlusher flusher = storeEngine.getStoreFlusher();
870     IOException lastException = null;
871     for (int i = 0; i < flushRetriesNumber; i++) {
872       try {
873         List<Path> pathNames = flusher.flushSnapshot(snapshot, logCacheFlushId, status);
874         Path lastPathName = null;
875         try {
876           for (Path pathName : pathNames) {
877             lastPathName = pathName;
878             validateStoreFile(pathName);
879           }
880           return pathNames;
881         } catch (Exception e) {
882           LOG.warn("Failed validating store file " + lastPathName + ", retrying num=" + i, e);
883           if (e instanceof IOException) {
884             lastException = (IOException) e;
885           } else {
886             lastException = new IOException(e);
887           }
888         }
889       } catch (IOException e) {
890         LOG.warn("Failed flushing store file, retrying num=" + i, e);
891         lastException = e;
892       }
893       if (lastException != null && i < (flushRetriesNumber - 1)) {
894         try {
895           Thread.sleep(pauseTime);
896         } catch (InterruptedException e) {
897           IOException iie = new InterruptedIOException();
898           iie.initCause(e);
899           throw iie;
900         }
901       }
902     }
903     throw lastException;
904   }
905 
906   /*
907    * @param path The pathname of the tmp file into which the store was flushed
908    * @param logCacheFlushId
909    * @param status
910    * @return StoreFile created.
911    * @throws IOException
912    */
913   private StoreFile commitFile(final Path path, final long logCacheFlushId, MonitoredTask status)
914       throws IOException {
915     // Write-out finished successfully, move into the right spot
916     Path dstPath = fs.commitStoreFile(getColumnFamilyName(), path);
917 
918     status.setStatus("Flushing " + this + ": reopening flushed file");
919     StoreFile sf = createStoreFileAndReader(dstPath);
920 
921     StoreFile.Reader r = sf.getReader();
922     this.storeSize += r.length();
923     this.totalUncompressedBytes += r.getTotalUncompressedBytes();
924 
925     if (LOG.isInfoEnabled()) {
926       LOG.info("Added " + sf + ", entries=" + r.getEntries() +
927         ", sequenceid=" + logCacheFlushId +
928         ", filesize=" + StringUtils.humanReadableInt(r.length()));
929     }
930     return sf;
931   }
932 
933   /*
934    * @param maxKeyCount
935    * @param compression Compression algorithm to use
936    * @param isCompaction whether we are creating a new file in a compaction
937    * @param includesMVCCReadPoint - whether to include MVCC or not
938    * @param includesTag - includesTag or not
939    * @return Writer for a new StoreFile in the tmp dir.
940    */
941   @Override
942   public StoreFile.Writer createWriterInTmp(long maxKeyCount, Compression.Algorithm compression,
943       boolean isCompaction, boolean includeMVCCReadpoint, boolean includesTag)
944   throws IOException {
945     final CacheConfig writerCacheConf;
946     if (isCompaction) {
947       // Don't cache data on write on compactions.
948       writerCacheConf = new CacheConfig(cacheConf);
949       writerCacheConf.setCacheDataOnWrite(false);
950     } else {
951       writerCacheConf = cacheConf;
952     }
953     InetSocketAddress[] favoredNodes = null;
954     if (region.getRegionServerServices() != null) {
955       favoredNodes = region.getRegionServerServices().getFavoredNodesForRegion(
956           region.getRegionInfo().getEncodedName());
957     }
958     HFileContext hFileContext = createFileContext(compression, includeMVCCReadpoint, includesTag,
959       cryptoContext);
960     StoreFile.Writer w = new StoreFile.WriterBuilder(conf, writerCacheConf,
961         this.getFileSystem())
962             .withFilePath(fs.createTempName())
963             .withComparator(comparator)
964             .withBloomType(family.getBloomFilterType())
965             .withMaxKeyCount(maxKeyCount)
966             .withFavoredNodes(favoredNodes)
967             .withFileContext(hFileContext)
968             .build();
969     return w;
970   }
971 
972   private HFileContext createFileContext(Compression.Algorithm compression,
973       boolean includeMVCCReadpoint, boolean includesTag, Encryption.Context cryptoContext) {
974     if (compression == null) {
975       compression = HFile.DEFAULT_COMPRESSION_ALGORITHM;
976     }
977     HFileContext hFileContext = new HFileContextBuilder()
978                                 .withIncludesMvcc(includeMVCCReadpoint)
979                                 .withIncludesTags(includesTag)
980                                 .withCompression(compression)
981                                 .withCompressTags(family.isCompressTags())
982                                 .withChecksumType(checksumType)
983                                 .withBytesPerCheckSum(bytesPerChecksum)
984                                 .withBlockSize(blocksize)
985                                 .withHBaseCheckSum(true)
986                                 .withDataBlockEncoding(family.getDataBlockEncoding())
987                                 .withEncryptionContext(cryptoContext)
988                                 .build();
989     return hFileContext;
990   }
991 
992 
993   /*
994    * Change storeFiles adding into place the Reader produced by this new flush.
995    * @param sfs Store files
996    * @param snapshotId
997    * @throws IOException
998    * @return Whether compaction is required.
999    */
1000   private boolean updateStorefiles(final List<StoreFile> sfs, final long snapshotId)
1001       throws IOException {
1002     this.lock.writeLock().lock();
1003     try {
1004       this.storeEngine.getStoreFileManager().insertNewFiles(sfs);
1005       this.memstore.clearSnapshot(snapshotId);
1006     } finally {
1007       // We need the lock, as long as we are updating the storeFiles
1008       // or changing the memstore. Let us release it before calling
1009       // notifyChangeReadersObservers. See HBASE-4485 for a possible
1010       // deadlock scenario that could have happened if continue to hold
1011       // the lock.
1012       this.lock.writeLock().unlock();
1013     }
1014 
1015     // Tell listeners of the change in readers.
1016     notifyChangedReadersObservers();
1017 
1018     if (LOG.isTraceEnabled()) {
1019       long totalSize = 0;
1020       for (StoreFile sf : sfs) {
1021         totalSize += sf.getReader().length();
1022       }
1023       String traceMessage = "FLUSH time,count,size,store size,store files ["
1024           + EnvironmentEdgeManager.currentTime() + "," + sfs.size() + "," + totalSize
1025           + "," + storeSize + "," + storeEngine.getStoreFileManager().getStorefileCount() + "]";
1026       LOG.trace(traceMessage);
1027     }
1028     return needsCompaction();
1029   }
1030 
1031   /*
1032    * Notify all observers that set of Readers has changed.
1033    * @throws IOException
1034    */
1035   private void notifyChangedReadersObservers() throws IOException {
1036     for (ChangedReadersObserver o: this.changedReaderObservers) {
1037       o.updateReaders();
1038     }
1039   }
1040 
1041   /**
1042    * Get all scanners with no filtering based on TTL (that happens further down
1043    * the line).
1044    * @return all scanners for this store
1045    */
1046   @Override
1047   public List<KeyValueScanner> getScanners(boolean cacheBlocks, boolean isGet,
1048       boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow,
1049       byte[] stopRow, long readPt) throws IOException {
1050     Collection<StoreFile> storeFilesToScan;
1051     List<KeyValueScanner> memStoreScanners;
1052     this.lock.readLock().lock();
1053     try {
1054       storeFilesToScan =
1055           this.storeEngine.getStoreFileManager().getFilesForScanOrGet(isGet, startRow, stopRow);
1056       memStoreScanners = this.memstore.getScanners(readPt);
1057     } finally {
1058       this.lock.readLock().unlock();
1059     }
1060 
1061     // First the store file scanners
1062 
1063     // TODO this used to get the store files in descending order,
1064     // but now we get them in ascending order, which I think is
1065     // actually more correct, since memstore get put at the end.
1066     List<StoreFileScanner> sfScanners = StoreFileScanner
1067       .getScannersForStoreFiles(storeFilesToScan, cacheBlocks, usePread, isCompaction, matcher,
1068         readPt);
1069     List<KeyValueScanner> scanners =
1070       new ArrayList<KeyValueScanner>(sfScanners.size()+1);
1071     scanners.addAll(sfScanners);
1072     // Then the memstore scanners
1073     scanners.addAll(memStoreScanners);
1074     return scanners;
1075   }
1076 
1077   @Override
1078   public void addChangedReaderObserver(ChangedReadersObserver o) {
1079     this.changedReaderObservers.add(o);
1080   }
1081 
1082   @Override
1083   public void deleteChangedReaderObserver(ChangedReadersObserver o) {
1084     // We don't check if observer present; it may not be (legitimately)
1085     this.changedReaderObservers.remove(o);
1086   }
1087 
1088   //////////////////////////////////////////////////////////////////////////////
1089   // Compaction
1090   //////////////////////////////////////////////////////////////////////////////
1091 
1092   /**
1093    * Compact the StoreFiles.  This method may take some time, so the calling
1094    * thread must be able to block for long periods.
1095    *
1096    * <p>During this time, the Store can work as usual, getting values from
1097    * StoreFiles and writing new StoreFiles from the memstore.
1098    *
1099    * Existing StoreFiles are not destroyed until the new compacted StoreFile is
1100    * completely written-out to disk.
1101    *
1102    * <p>The compactLock prevents multiple simultaneous compactions.
1103    * The structureLock prevents us from interfering with other write operations.
1104    *
1105    * <p>We don't want to hold the structureLock for the whole time, as a compact()
1106    * can be lengthy and we want to allow cache-flushes during this period.
1107    *
1108    * <p> Compaction event should be idempotent, since there is no IO Fencing for
1109    * the region directory in hdfs. A region server might still try to complete the
1110    * compaction after it lost the region. That is why the following events are carefully
1111    * ordered for a compaction:
1112    *  1. Compaction writes new files under region/.tmp directory (compaction output)
1113    *  2. Compaction atomically moves the temporary file under region directory
1114    *  3. Compaction appends a WAL edit containing the compaction input and output files.
1115    *  Forces sync on WAL.
1116    *  4. Compaction deletes the input files from the region directory.
1117    *
1118    * Failure conditions are handled like this:
1119    *  - If RS fails before 2, compaction wont complete. Even if RS lives on and finishes
1120    *  the compaction later, it will only write the new data file to the region directory.
1121    *  Since we already have this data, this will be idempotent but we will have a redundant
1122    *  copy of the data.
1123    *  - If RS fails between 2 and 3, the region will have a redundant copy of the data. The
1124    *  RS that failed won't be able to finish snyc() for WAL because of lease recovery in WAL.
1125    *  - If RS fails after 3, the region region server who opens the region will pick up the
1126    *  the compaction marker from the WAL and replay it by removing the compaction input files.
1127    *  Failed RS can also attempt to delete those files, but the operation will be idempotent
1128    *
1129    * See HBASE-2231 for details.
1130    *
1131    * @param compaction compaction details obtained from requestCompaction()
1132    * @throws IOException
1133    * @return Storefile we compacted into or null if we failed or opted out early.
1134    */
1135   @Override
1136   public List<StoreFile> compact(CompactionContext compaction) throws IOException {
1137     assert compaction != null;
1138     List<StoreFile> sfs = null;
1139     CompactionRequest cr = compaction.getRequest();;
1140     try {
1141       // Do all sanity checking in here if we have a valid CompactionRequest
1142       // because we need to clean up after it on the way out in a finally
1143       // block below
1144       long compactionStartTime = EnvironmentEdgeManager.currentTime();
1145       assert compaction.hasSelection();
1146       Collection<StoreFile> filesToCompact = cr.getFiles();
1147       assert !filesToCompact.isEmpty();
1148       synchronized (filesCompacting) {
1149         // sanity check: we're compacting files that this store knows about
1150         // TODO: change this to LOG.error() after more debugging
1151         Preconditions.checkArgument(filesCompacting.containsAll(filesToCompact));
1152       }
1153 
1154       // Ready to go. Have list of files to compact.
1155       LOG.info("Starting compaction of " + filesToCompact.size() + " file(s) in "
1156           + this + " of " + this.getRegionInfo().getRegionNameAsString()
1157           + " into tmpdir=" + fs.getTempDir() + ", totalSize="
1158           + StringUtils.humanReadableInt(cr.getSize()));
1159 
1160       // Commence the compaction.
1161       List<Path> newFiles = compaction.compact();
1162 
1163       // TODO: get rid of this!
1164       if (!this.conf.getBoolean("hbase.hstore.compaction.complete", true)) {
1165         LOG.warn("hbase.hstore.compaction.complete is set to false");
1166         sfs = new ArrayList<StoreFile>(newFiles.size());
1167         for (Path newFile : newFiles) {
1168           // Create storefile around what we wrote with a reader on it.
1169           StoreFile sf = createStoreFileAndReader(newFile);
1170           sf.closeReader(true);
1171           sfs.add(sf);
1172         }
1173         return sfs;
1174       }
1175       // Do the steps necessary to complete the compaction.
1176       sfs = moveCompatedFilesIntoPlace(cr, newFiles);
1177       writeCompactionWalRecord(filesToCompact, sfs);
1178       replaceStoreFiles(filesToCompact, sfs);
1179       if (cr.isMajor()) {
1180         majorCompactedCellsCount += getCompactionProgress().totalCompactingKVs;
1181         majorCompactedCellsSize += getCompactionProgress().totalCompactedSize;
1182       } else {
1183         compactedCellsCount += getCompactionProgress().totalCompactingKVs;
1184         compactedCellsSize += getCompactionProgress().totalCompactedSize;
1185       }
1186       // At this point the store will use new files for all new scanners.
1187       completeCompaction(filesToCompact, true); // Archive old files & update store size.
1188 
1189       logCompactionEndMessage(cr, sfs, compactionStartTime);
1190       return sfs;
1191     } finally {
1192       finishCompactionRequest(cr);
1193     }
1194   }
1195 
1196   private List<StoreFile> moveCompatedFilesIntoPlace(
1197       CompactionRequest cr, List<Path> newFiles) throws IOException {
1198     List<StoreFile> sfs = new ArrayList<StoreFile>(newFiles.size());
1199     for (Path newFile : newFiles) {
1200       assert newFile != null;
1201       StoreFile sf = moveFileIntoPlace(newFile);
1202       if (this.getCoprocessorHost() != null) {
1203         this.getCoprocessorHost().postCompact(this, sf, cr);
1204       }
1205       assert sf != null;
1206       sfs.add(sf);
1207     }
1208     return sfs;
1209   }
1210 
1211   // Package-visible for tests
1212   StoreFile moveFileIntoPlace(final Path newFile) throws IOException {
1213     validateStoreFile(newFile);
1214     // Move the file into the right spot
1215     Path destPath = fs.commitStoreFile(getColumnFamilyName(), newFile);
1216     return createStoreFileAndReader(destPath);
1217   }
1218 
1219   /**
1220    * Writes the compaction WAL record.
1221    * @param filesCompacted Files compacted (input).
1222    * @param newFiles Files from compaction.
1223    */
1224   private void writeCompactionWalRecord(Collection<StoreFile> filesCompacted,
1225       Collection<StoreFile> newFiles) throws IOException {
1226     if (region.getWAL() == null) return;
1227     List<Path> inputPaths = new ArrayList<Path>(filesCompacted.size());
1228     for (StoreFile f : filesCompacted) {
1229       inputPaths.add(f.getPath());
1230     }
1231     List<Path> outputPaths = new ArrayList<Path>(newFiles.size());
1232     for (StoreFile f : newFiles) {
1233       outputPaths.add(f.getPath());
1234     }
1235     HRegionInfo info = this.region.getRegionInfo();
1236     CompactionDescriptor compactionDescriptor = ProtobufUtil.toCompactionDescriptor(info,
1237         family.getName(), inputPaths, outputPaths, fs.getStoreDir(getFamily().getNameAsString()));
1238     WALUtil.writeCompactionMarker(region.getWAL(), this.region.getTableDesc(),
1239         this.region.getRegionInfo(), compactionDescriptor, this.region.getSequenceId());
1240   }
1241 
1242   @VisibleForTesting
1243   void replaceStoreFiles(final Collection<StoreFile> compactedFiles,
1244       final Collection<StoreFile> result) throws IOException {
1245     this.lock.writeLock().lock();
1246     try {
1247       this.storeEngine.getStoreFileManager().addCompactionResults(compactedFiles, result);
1248       filesCompacting.removeAll(compactedFiles); // safe bc: lock.writeLock();
1249     } finally {
1250       this.lock.writeLock().unlock();
1251     }
1252   }
1253 
1254   /**
1255    * Log a very elaborate compaction completion message.
1256    * @param cr Request.
1257    * @param sfs Resulting files.
1258    * @param compactionStartTime Start time.
1259    */
1260   private void logCompactionEndMessage(
1261       CompactionRequest cr, List<StoreFile> sfs, long compactionStartTime) {
1262     long now = EnvironmentEdgeManager.currentTime();
1263     StringBuilder message = new StringBuilder(
1264       "Completed" + (cr.isMajor() ? " major" : "") + " compaction of "
1265       + cr.getFiles().size() + (cr.isAllFiles() ? " (all)" : "") + " file(s) in "
1266       + this + " of " + this.getRegionInfo().getRegionNameAsString() + " into ");
1267     if (sfs.isEmpty()) {
1268       message.append("none, ");
1269     } else {
1270       for (StoreFile sf: sfs) {
1271         message.append(sf.getPath().getName());
1272         message.append("(size=");
1273         message.append(StringUtils.humanReadableInt(sf.getReader().length()));
1274         message.append("), ");
1275       }
1276     }
1277     message.append("total size for store is ")
1278       .append(StringUtils.humanReadableInt(storeSize))
1279       .append(". This selection was in queue for ")
1280       .append(StringUtils.formatTimeDiff(compactionStartTime, cr.getSelectionTime()))
1281       .append(", and took ").append(StringUtils.formatTimeDiff(now, compactionStartTime))
1282       .append(" to execute.");
1283     LOG.info(message.toString());
1284     if (LOG.isTraceEnabled()) {
1285       int fileCount = storeEngine.getStoreFileManager().getStorefileCount();
1286       long resultSize = 0;
1287       for (StoreFile sf : sfs) {
1288         resultSize += sf.getReader().length();
1289       }
1290       String traceMessage = "COMPACTION start,end,size out,files in,files out,store size,"
1291         + "store files [" + compactionStartTime + "," + now + "," + resultSize + ","
1292           + cr.getFiles().size() + "," + sfs.size() + "," +  storeSize + "," + fileCount + "]";
1293       LOG.trace(traceMessage);
1294     }
1295   }
1296 
1297   /**
1298    * Call to complete a compaction. Its for the case where we find in the WAL a compaction
1299    * that was not finished.  We could find one recovering a WAL after a regionserver crash.
1300    * See HBASE-2231.
1301    * @param compaction
1302    */
1303   @Override
1304   public void completeCompactionMarker(CompactionDescriptor compaction)
1305       throws IOException {
1306     LOG.debug("Completing compaction from the WAL marker");
1307     List<String> compactionInputs = compaction.getCompactionInputList();
1308 
1309     // The Compaction Marker is written after the compaction is completed,
1310     // and the files moved into the region/family folder.
1311     //
1312     // If we crash after the entry is written, we may not have removed the
1313     // input files, but the output file is present.
1314     // (The unremoved input files will be removed by this function)
1315     //
1316     // If we scan the directory and the file is not present, it can mean that:
1317     //   - The file was manually removed by the user
1318     //   - The file was removed as consequence of subsequent compaction
1319     // so, we can't do anything with the "compaction output list" because those
1320     // files have already been loaded when opening the region (by virtue of
1321     // being in the store's folder) or they may be missing due to a compaction.
1322 
1323     String familyName = this.getColumnFamilyName();
1324     List<Path> inputPaths = new ArrayList<Path>(compactionInputs.size());
1325     for (String compactionInput : compactionInputs) {
1326       Path inputPath = fs.getStoreFilePath(familyName, compactionInput);
1327       inputPaths.add(inputPath);
1328     }
1329 
1330     //some of the input files might already be deleted
1331     List<StoreFile> inputStoreFiles = new ArrayList<StoreFile>(compactionInputs.size());
1332     for (StoreFile sf : this.getStorefiles()) {
1333       if (inputPaths.contains(sf.getQualifiedPath())) {
1334         inputStoreFiles.add(sf);
1335       }
1336     }
1337 
1338     this.replaceStoreFiles(inputStoreFiles, Collections.EMPTY_LIST);
1339     this.completeCompaction(inputStoreFiles);
1340   }
1341 
1342   /**
1343    * This method tries to compact N recent files for testing.
1344    * Note that because compacting "recent" files only makes sense for some policies,
1345    * e.g. the default one, it assumes default policy is used. It doesn't use policy,
1346    * but instead makes a compaction candidate list by itself.
1347    * @param N Number of files.
1348    */
1349   public void compactRecentForTestingAssumingDefaultPolicy(int N) throws IOException {
1350     List<StoreFile> filesToCompact;
1351     boolean isMajor;
1352 
1353     this.lock.readLock().lock();
1354     try {
1355       synchronized (filesCompacting) {
1356         filesToCompact = Lists.newArrayList(storeEngine.getStoreFileManager().getStorefiles());
1357         if (!filesCompacting.isEmpty()) {
1358           // exclude all files older than the newest file we're currently
1359           // compacting. this allows us to preserve contiguity (HBASE-2856)
1360           StoreFile last = filesCompacting.get(filesCompacting.size() - 1);
1361           int idx = filesToCompact.indexOf(last);
1362           Preconditions.checkArgument(idx != -1);
1363           filesToCompact.subList(0, idx + 1).clear();
1364         }
1365         int count = filesToCompact.size();
1366         if (N > count) {
1367           throw new RuntimeException("Not enough files");
1368         }
1369 
1370         filesToCompact = filesToCompact.subList(count - N, count);
1371         isMajor = (filesToCompact.size() == storeEngine.getStoreFileManager().getStorefileCount());
1372         filesCompacting.addAll(filesToCompact);
1373         Collections.sort(filesCompacting, StoreFile.Comparators.SEQ_ID);
1374       }
1375     } finally {
1376       this.lock.readLock().unlock();
1377     }
1378 
1379     try {
1380       // Ready to go. Have list of files to compact.
1381       List<Path> newFiles = ((DefaultCompactor)this.storeEngine.getCompactor())
1382           .compactForTesting(filesToCompact, isMajor);
1383       for (Path newFile: newFiles) {
1384         // Move the compaction into place.
1385         StoreFile sf = moveFileIntoPlace(newFile);
1386         if (this.getCoprocessorHost() != null) {
1387           this.getCoprocessorHost().postCompact(this, sf, null);
1388         }
1389         replaceStoreFiles(filesToCompact, Lists.newArrayList(sf));
1390         completeCompaction(filesToCompact, true);
1391       }
1392     } finally {
1393       synchronized (filesCompacting) {
1394         filesCompacting.removeAll(filesToCompact);
1395       }
1396     }
1397   }
1398 
1399   @Override
1400   public boolean hasReferences() {
1401     return StoreUtils.hasReferences(this.storeEngine.getStoreFileManager().getStorefiles());
1402   }
1403 
1404   @Override
1405   public CompactionProgress getCompactionProgress() {
1406     return this.storeEngine.getCompactor().getProgress();
1407   }
1408 
1409   @Override
1410   public boolean isMajorCompaction() throws IOException {
1411     for (StoreFile sf : this.storeEngine.getStoreFileManager().getStorefiles()) {
1412       // TODO: what are these reader checks all over the place?
1413       if (sf.getReader() == null) {
1414         LOG.debug("StoreFile " + sf + " has null Reader");
1415         return false;
1416       }
1417     }
1418     return storeEngine.getCompactionPolicy().isMajorCompaction(
1419         this.storeEngine.getStoreFileManager().getStorefiles());
1420   }
1421 
1422   @Override
1423   public CompactionContext requestCompaction() throws IOException {
1424     return requestCompaction(Store.NO_PRIORITY, null);
1425   }
1426 
1427   @Override
1428   public CompactionContext requestCompaction(int priority, CompactionRequest baseRequest)
1429       throws IOException {
1430     // don't even select for compaction if writes are disabled
1431     if (!this.areWritesEnabled()) {
1432       return null;
1433     }
1434 
1435     // Before we do compaction, try to get rid of unneeded files to simplify things.
1436     removeUnneededFiles();
1437 
1438     CompactionContext compaction = storeEngine.createCompaction();
1439     CompactionRequest request = null;
1440     this.lock.readLock().lock();
1441     try {
1442       synchronized (filesCompacting) {
1443         // First, see if coprocessor would want to override selection.
1444         if (this.getCoprocessorHost() != null) {
1445           List<StoreFile> candidatesForCoproc = compaction.preSelect(this.filesCompacting);
1446           boolean override = this.getCoprocessorHost().preCompactSelection(
1447               this, candidatesForCoproc, baseRequest);
1448           if (override) {
1449             // Coprocessor is overriding normal file selection.
1450             compaction.forceSelect(new CompactionRequest(candidatesForCoproc));
1451           }
1452         }
1453 
1454         // Normal case - coprocessor is not overriding file selection.
1455         if (!compaction.hasSelection()) {
1456           boolean isUserCompaction = priority == Store.PRIORITY_USER;
1457           boolean mayUseOffPeak = offPeakHours.isOffPeakHour() &&
1458               offPeakCompactionTracker.compareAndSet(false, true);
1459           try {
1460             compaction.select(this.filesCompacting, isUserCompaction,
1461               mayUseOffPeak, forceMajor && filesCompacting.isEmpty());
1462           } catch (IOException e) {
1463             if (mayUseOffPeak) {
1464               offPeakCompactionTracker.set(false);
1465             }
1466             throw e;
1467           }
1468           assert compaction.hasSelection();
1469           if (mayUseOffPeak && !compaction.getRequest().isOffPeak()) {
1470             // Compaction policy doesn't want to take advantage of off-peak.
1471             offPeakCompactionTracker.set(false);
1472           }
1473         }
1474         if (this.getCoprocessorHost() != null) {
1475           this.getCoprocessorHost().postCompactSelection(
1476               this, ImmutableList.copyOf(compaction.getRequest().getFiles()), baseRequest);
1477         }
1478 
1479         // Selected files; see if we have a compaction with some custom base request.
1480         if (baseRequest != null) {
1481           // Update the request with what the system thinks the request should be;
1482           // its up to the request if it wants to listen.
1483           compaction.forceSelect(
1484               baseRequest.combineWith(compaction.getRequest()));
1485         }
1486         // Finally, we have the resulting files list. Check if we have any files at all.
1487         request = compaction.getRequest();
1488         final Collection<StoreFile> selectedFiles = request.getFiles();
1489         if (selectedFiles.isEmpty()) {
1490           return null;
1491         }
1492 
1493         addToCompactingFiles(selectedFiles);
1494 
1495         // If we're enqueuing a major, clear the force flag.
1496         this.forceMajor = this.forceMajor && !request.isMajor();
1497 
1498         // Set common request properties.
1499         // Set priority, either override value supplied by caller or from store.
1500         request.setPriority((priority != Store.NO_PRIORITY) ? priority : getCompactPriority());
1501         request.setDescription(getRegionInfo().getRegionNameAsString(), getColumnFamilyName());
1502       }
1503     } finally {
1504       this.lock.readLock().unlock();
1505     }
1506 
1507     LOG.debug(getRegionInfo().getEncodedName() + " - "  + getColumnFamilyName()
1508         + ": Initiating " + (request.isMajor() ? "major" : "minor") + " compaction"
1509         + (request.isAllFiles() ? " (all files)" : ""));
1510     this.region.reportCompactionRequestStart(request.isMajor());
1511     return compaction;
1512   }
1513 
1514   /** Adds the files to compacting files. filesCompacting must be locked. */
1515   private void addToCompactingFiles(final Collection<StoreFile> filesToAdd) {
1516     if (filesToAdd == null) return;
1517     // Check that we do not try to compact the same StoreFile twice.
1518     if (!Collections.disjoint(filesCompacting, filesToAdd)) {
1519       Preconditions.checkArgument(false, "%s overlaps with %s", filesToAdd, filesCompacting);
1520     }
1521     filesCompacting.addAll(filesToAdd);
1522     Collections.sort(filesCompacting, StoreFile.Comparators.SEQ_ID);
1523   }
1524 
1525   private void removeUnneededFiles() throws IOException {
1526     if (!conf.getBoolean("hbase.store.delete.expired.storefile", true)) return;
1527     if (getFamily().getMinVersions() > 0) {
1528       LOG.debug("Skipping expired store file removal due to min version being " +
1529           getFamily().getMinVersions());
1530       return;
1531     }
1532     this.lock.readLock().lock();
1533     Collection<StoreFile> delSfs = null;
1534     try {
1535       synchronized (filesCompacting) {
1536         long cfTtl = getStoreFileTtl();
1537         if (cfTtl != Long.MAX_VALUE) {
1538           delSfs = storeEngine.getStoreFileManager().getUnneededFiles(
1539               EnvironmentEdgeManager.currentTime() - cfTtl, filesCompacting);
1540           addToCompactingFiles(delSfs);
1541         }
1542       }
1543     } finally {
1544       this.lock.readLock().unlock();
1545     }
1546     if (delSfs == null || delSfs.isEmpty()) return;
1547 
1548     Collection<StoreFile> newFiles = new ArrayList<StoreFile>(); // No new files.
1549     writeCompactionWalRecord(delSfs, newFiles);
1550     replaceStoreFiles(delSfs, newFiles);
1551     completeCompaction(delSfs);
1552     LOG.info("Completed removal of " + delSfs.size() + " unnecessary (expired) file(s) in "
1553         + this + " of " + this.getRegionInfo().getRegionNameAsString()
1554         + "; total size for store is " + StringUtils.humanReadableInt(storeSize));
1555   }
1556 
1557   @Override
1558   public void cancelRequestedCompaction(CompactionContext compaction) {
1559     finishCompactionRequest(compaction.getRequest());
1560   }
1561 
1562   private void finishCompactionRequest(CompactionRequest cr) {
1563     this.region.reportCompactionRequestEnd(cr.isMajor(), cr.getFiles().size(), cr.getSize());
1564     if (cr.isOffPeak()) {
1565       offPeakCompactionTracker.set(false);
1566       cr.setOffPeak(false);
1567     }
1568     synchronized (filesCompacting) {
1569       filesCompacting.removeAll(cr.getFiles());
1570     }
1571   }
1572 
1573   /**
1574    * Validates a store file by opening and closing it. In HFileV2 this should
1575    * not be an expensive operation.
1576    *
1577    * @param path the path to the store file
1578    */
1579   private void validateStoreFile(Path path)
1580       throws IOException {
1581     StoreFile storeFile = null;
1582     try {
1583       storeFile = createStoreFileAndReader(path);
1584     } catch (IOException e) {
1585       LOG.error("Failed to open store file : " + path
1586           + ", keeping it in tmp location", e);
1587       throw e;
1588     } finally {
1589       if (storeFile != null) {
1590         storeFile.closeReader(false);
1591       }
1592     }
1593   }
1594 
1595   /**
1596    * <p>It works by processing a compaction that's been written to disk.
1597    *
1598    * <p>It is usually invoked at the end of a compaction, but might also be
1599    * invoked at HStore startup, if the prior execution died midway through.
1600    *
1601    * <p>Moving the compacted TreeMap into place means:
1602    * <pre>
1603    * 1) Unload all replaced StoreFile, close and collect list to delete.
1604    * 2) Compute new store size
1605    * </pre>
1606    *
1607    * @param compactedFiles list of files that were compacted
1608    */
1609   @VisibleForTesting
1610   protected void completeCompaction(final Collection<StoreFile> compactedFiles)
1611     throws IOException {
1612     completeCompaction(compactedFiles, true);
1613   }
1614 
1615 
1616   /**
1617    * <p>It works by processing a compaction that's been written to disk.
1618    *
1619    * <p>It is usually invoked at the end of a compaction, but might also be
1620    * invoked at HStore startup, if the prior execution died midway through.
1621    *
1622    * <p>Moving the compacted TreeMap into place means:
1623    * <pre>
1624    * 1) Unload all replaced StoreFile, close and collect list to delete.
1625    * 2) Compute new store size
1626    * </pre>
1627    *
1628    * @param compactedFiles list of files that were compacted
1629    */
1630   @VisibleForTesting
1631   protected void completeCompaction(final Collection<StoreFile> compactedFiles, boolean removeFiles)
1632       throws IOException {
1633     try {
1634       // Do not delete old store files until we have sent out notification of
1635       // change in case old files are still being accessed by outstanding scanners.
1636       // Don't do this under writeLock; see HBASE-4485 for a possible deadlock
1637       // scenario that could have happened if continue to hold the lock.
1638       notifyChangedReadersObservers();
1639       // At this point the store will use new files for all scanners.
1640 
1641       // let the archive util decide if we should archive or delete the files
1642       LOG.debug("Removing store files after compaction...");
1643       for (StoreFile compactedFile : compactedFiles) {
1644         compactedFile.closeReader(true);
1645       }
1646       if (removeFiles) {
1647         this.fs.removeStoreFiles(this.getColumnFamilyName(), compactedFiles);
1648       }
1649     } catch (IOException e) {
1650       e = e instanceof RemoteException ?
1651                 ((RemoteException)e).unwrapRemoteException() : e;
1652       LOG.error("Failed removing compacted files in " + this +
1653         ". Files we were trying to remove are " + compactedFiles.toString() +
1654         "; some of them may have been already removed", e);
1655     }
1656 
1657     // 4. Compute new store size
1658     this.storeSize = 0L;
1659     this.totalUncompressedBytes = 0L;
1660     for (StoreFile hsf : this.storeEngine.getStoreFileManager().getStorefiles()) {
1661       StoreFile.Reader r = hsf.getReader();
1662       if (r == null) {
1663         LOG.warn("StoreFile " + hsf + " has a null Reader");
1664         continue;
1665       }
1666       this.storeSize += r.length();
1667       this.totalUncompressedBytes += r.getTotalUncompressedBytes();
1668     }
1669   }
1670 
1671   /*
1672    * @param wantedVersions How many versions were asked for.
1673    * @return wantedVersions or this families' {@link HConstants#VERSIONS}.
1674    */
1675   int versionsToReturn(final int wantedVersions) {
1676     if (wantedVersions <= 0) {
1677       throw new IllegalArgumentException("Number of versions must be > 0");
1678     }
1679     // Make sure we do not return more than maximum versions for this store.
1680     int maxVersions = this.family.getMaxVersions();
1681     return wantedVersions > maxVersions ? maxVersions: wantedVersions;
1682   }
1683 
1684   /**
1685    * @param cell
1686    * @param oldestTimestamp
1687    * @return true if the cell is expired
1688    */
1689   static boolean isCellTTLExpired(final Cell cell, final long oldestTimestamp, final long now) {
1690     // Do not create an Iterator or Tag objects unless the cell actually has
1691     // tags
1692     if (cell.getTagsLength() > 0) {
1693       // Look for a TTL tag first. Use it instead of the family setting if
1694       // found. If a cell has multiple TTLs, resolve the conflict by using the
1695       // first tag encountered.
1696       Iterator<Tag> i = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
1697         cell.getTagsLength());
1698       while (i.hasNext()) {
1699         Tag t = i.next();
1700         if (TagType.TTL_TAG_TYPE == t.getType()) {
1701           // Unlike in schema cell TTLs are stored in milliseconds, no need
1702           // to convert
1703           long ts = cell.getTimestamp();
1704           assert t.getTagLength() == Bytes.SIZEOF_LONG;
1705           long ttl = Bytes.toLong(t.getBuffer(), t.getTagOffset(), t.getTagLength());
1706           if (ts + ttl < now) {
1707             return true;
1708           }
1709           // Per cell TTLs cannot extend lifetime beyond family settings, so
1710           // fall through to check that
1711           break;
1712         }
1713       }
1714     }
1715     return false;
1716   }
1717 
1718   @Override
1719   public Cell getRowKeyAtOrBefore(final byte[] row) throws IOException {
1720     // If minVersions is set, we will not ignore expired KVs.
1721     // As we're only looking for the latest matches, that should be OK.
1722     // With minVersions > 0 we guarantee that any KV that has any version
1723     // at all (expired or not) has at least one version that will not expire.
1724     // Note that this method used to take a KeyValue as arguments. KeyValue
1725     // can be back-dated, a row key cannot.
1726     long ttlToUse = scanInfo.getMinVersions() > 0 ? Long.MAX_VALUE : this.scanInfo.getTtl();
1727 
1728     KeyValue kv = new KeyValue(row, HConstants.LATEST_TIMESTAMP);
1729 
1730     GetClosestRowBeforeTracker state = new GetClosestRowBeforeTracker(
1731       this.comparator, kv, ttlToUse, this.getRegionInfo().isMetaRegion());
1732     this.lock.readLock().lock();
1733     try {
1734       // First go to the memstore.  Pick up deletes and candidates.
1735       this.memstore.getRowKeyAtOrBefore(state);
1736       // Check if match, if we got a candidate on the asked for 'kv' row.
1737       // Process each relevant store file. Run through from newest to oldest.
1738       Iterator<StoreFile> sfIterator = this.storeEngine.getStoreFileManager()
1739           .getCandidateFilesForRowKeyBefore(state.getTargetKey());
1740       while (sfIterator.hasNext()) {
1741         StoreFile sf = sfIterator.next();
1742         sfIterator.remove(); // Remove sf from iterator.
1743         boolean haveNewCandidate = rowAtOrBeforeFromStoreFile(sf, state);
1744         Cell candidate = state.getCandidate();
1745         // we have an optimization here which stops the search if we find exact match.
1746         if (candidate != null && CellUtil.matchingRow(candidate, row)) {
1747           return candidate;
1748         }
1749         if (haveNewCandidate) {
1750           sfIterator = this.storeEngine.getStoreFileManager().updateCandidateFilesForRowKeyBefore(
1751               sfIterator, state.getTargetKey(), candidate);
1752         }
1753       }
1754       return state.getCandidate();
1755     } finally {
1756       this.lock.readLock().unlock();
1757     }
1758   }
1759 
1760   /*
1761    * Check an individual MapFile for the row at or before a given row.
1762    * @param f
1763    * @param state
1764    * @throws IOException
1765    * @return True iff the candidate has been updated in the state.
1766    */
1767   private boolean rowAtOrBeforeFromStoreFile(final StoreFile f,
1768                                           final GetClosestRowBeforeTracker state)
1769       throws IOException {
1770     StoreFile.Reader r = f.getReader();
1771     if (r == null) {
1772       LOG.warn("StoreFile " + f + " has a null Reader");
1773       return false;
1774     }
1775     if (r.getEntries() == 0) {
1776       LOG.warn("StoreFile " + f + " is a empty store file");
1777       return false;
1778     }
1779     // TODO: Cache these keys rather than make each time?
1780     byte [] fk = r.getFirstKey();
1781     if (fk == null) return false;
1782     KeyValue firstKV = KeyValue.createKeyValueFromKey(fk, 0, fk.length);
1783     byte [] lk = r.getLastKey();
1784     KeyValue lastKV = KeyValue.createKeyValueFromKey(lk, 0, lk.length);
1785     KeyValue firstOnRow = state.getTargetKey();
1786     if (this.comparator.compareRows(lastKV, firstOnRow) < 0) {
1787       // If last key in file is not of the target table, no candidates in this
1788       // file.  Return.
1789       if (!state.isTargetTable(lastKV)) return false;
1790       // If the row we're looking for is past the end of file, set search key to
1791       // last key. TODO: Cache last and first key rather than make each time.
1792       firstOnRow = new KeyValue(lastKV.getRow(), HConstants.LATEST_TIMESTAMP);
1793     }
1794     // Get a scanner that caches blocks and that uses pread.
1795     HFileScanner scanner = r.getScanner(true, true, false);
1796     // Seek scanner.  If can't seek it, return.
1797     if (!seekToScanner(scanner, firstOnRow, firstKV)) return false;
1798     // If we found candidate on firstOnRow, just return. THIS WILL NEVER HAPPEN!
1799     // Unlikely that there'll be an instance of actual first row in table.
1800     if (walkForwardInSingleRow(scanner, firstOnRow, state)) return true;
1801     // If here, need to start backing up.
1802     while (scanner.seekBefore(firstOnRow.getBuffer(), firstOnRow.getKeyOffset(),
1803        firstOnRow.getKeyLength())) {
1804       Cell kv = scanner.getKeyValue();
1805       if (!state.isTargetTable(kv)) break;
1806       if (!state.isBetterCandidate(kv)) break;
1807       // Make new first on row.
1808       firstOnRow = new KeyValue(kv.getRow(), HConstants.LATEST_TIMESTAMP);
1809       // Seek scanner.  If can't seek it, break.
1810       if (!seekToScanner(scanner, firstOnRow, firstKV)) return false;
1811       // If we find something, break;
1812       if (walkForwardInSingleRow(scanner, firstOnRow, state)) return true;
1813     }
1814     return false;
1815   }
1816 
1817   /*
1818    * Seek the file scanner to firstOnRow or first entry in file.
1819    * @param scanner
1820    * @param firstOnRow
1821    * @param firstKV
1822    * @return True if we successfully seeked scanner.
1823    * @throws IOException
1824    */
1825   private boolean seekToScanner(final HFileScanner scanner,
1826                                 final KeyValue firstOnRow,
1827                                 final KeyValue firstKV)
1828       throws IOException {
1829     KeyValue kv = firstOnRow;
1830     // If firstOnRow < firstKV, set to firstKV
1831     if (this.comparator.compareRows(firstKV, firstOnRow) == 0) kv = firstKV;
1832     int result = scanner.seekTo(kv);
1833     return result != -1;
1834   }
1835 
1836   /*
1837    * When we come in here, we are probably at the kv just before we break into
1838    * the row that firstOnRow is on.  Usually need to increment one time to get
1839    * on to the row we are interested in.
1840    * @param scanner
1841    * @param firstOnRow
1842    * @param state
1843    * @return True we found a candidate.
1844    * @throws IOException
1845    */
1846   private boolean walkForwardInSingleRow(final HFileScanner scanner,
1847                                          final KeyValue firstOnRow,
1848                                          final GetClosestRowBeforeTracker state)
1849       throws IOException {
1850     boolean foundCandidate = false;
1851     do {
1852       Cell kv = scanner.getKeyValue();
1853       // If we are not in the row, skip.
1854       if (this.comparator.compareRows(kv, firstOnRow) < 0) continue;
1855       // Did we go beyond the target row? If so break.
1856       if (state.isTooFar(kv, firstOnRow)) break;
1857       if (state.isExpired(kv)) {
1858         continue;
1859       }
1860       // If we added something, this row is a contender. break.
1861       if (state.handle(kv)) {
1862         foundCandidate = true;
1863         break;
1864       }
1865     } while(scanner.next());
1866     return foundCandidate;
1867   }
1868 
1869   @Override
1870   public boolean canSplit() {
1871     this.lock.readLock().lock();
1872     try {
1873       // Not split-able if we find a reference store file present in the store.
1874       boolean result = !hasReferences();
1875       if (!result && LOG.isDebugEnabled()) {
1876         LOG.debug("Cannot split region due to reference files being there");
1877       }
1878       return result;
1879     } finally {
1880       this.lock.readLock().unlock();
1881     }
1882   }
1883 
1884   @Override
1885   public byte[] getSplitPoint() {
1886     this.lock.readLock().lock();
1887     try {
1888       // Should already be enforced by the split policy!
1889       assert !this.getRegionInfo().isMetaRegion();
1890       // Not split-able if we find a reference store file present in the store.
1891       if (hasReferences()) {
1892         return null;
1893       }
1894       return this.storeEngine.getStoreFileManager().getSplitPoint();
1895     } catch(IOException e) {
1896       LOG.warn("Failed getting store size for " + this, e);
1897     } finally {
1898       this.lock.readLock().unlock();
1899     }
1900     return null;
1901   }
1902 
1903   @Override
1904   public long getLastCompactSize() {
1905     return this.lastCompactSize;
1906   }
1907 
1908   @Override
1909   public long getSize() {
1910     return storeSize;
1911   }
1912 
1913   @Override
1914   public void triggerMajorCompaction() {
1915     this.forceMajor = true;
1916   }
1917 
1918 
1919   //////////////////////////////////////////////////////////////////////////////
1920   // File administration
1921   //////////////////////////////////////////////////////////////////////////////
1922 
1923   @Override
1924   public KeyValueScanner getScanner(Scan scan,
1925       final NavigableSet<byte []> targetCols, long readPt) throws IOException {
1926     lock.readLock().lock();
1927     try {
1928       KeyValueScanner scanner = null;
1929       if (this.getCoprocessorHost() != null) {
1930         scanner = this.getCoprocessorHost().preStoreScannerOpen(this, scan, targetCols);
1931       }
1932       if (scanner == null) {
1933         scanner = scan.isReversed() ? new ReversedStoreScanner(this,
1934             getScanInfo(), scan, targetCols, readPt) : new StoreScanner(this,
1935             getScanInfo(), scan, targetCols, readPt);
1936       }
1937       return scanner;
1938     } finally {
1939       lock.readLock().unlock();
1940     }
1941   }
1942 
1943   @Override
1944   public String toString() {
1945     return this.getColumnFamilyName();
1946   }
1947 
1948   @Override
1949   // TODO: why is there this and also getNumberOfStorefiles?! Remove one.
1950   public int getStorefilesCount() {
1951     return this.storeEngine.getStoreFileManager().getStorefileCount();
1952   }
1953 
1954   @Override
1955   public long getStoreSizeUncompressed() {
1956     return this.totalUncompressedBytes;
1957   }
1958 
1959   @Override
1960   public long getStorefilesSize() {
1961     long size = 0;
1962     for (StoreFile s: this.storeEngine.getStoreFileManager().getStorefiles()) {
1963       StoreFile.Reader r = s.getReader();
1964       if (r == null) {
1965         LOG.warn("StoreFile " + s + " has a null Reader");
1966         continue;
1967       }
1968       size += r.length();
1969     }
1970     return size;
1971   }
1972 
1973   @Override
1974   public long getStorefilesIndexSize() {
1975     long size = 0;
1976     for (StoreFile s: this.storeEngine.getStoreFileManager().getStorefiles()) {
1977       StoreFile.Reader r = s.getReader();
1978       if (r == null) {
1979         LOG.warn("StoreFile " + s + " has a null Reader");
1980         continue;
1981       }
1982       size += r.indexSize();
1983     }
1984     return size;
1985   }
1986 
1987   @Override
1988   public long getTotalStaticIndexSize() {
1989     long size = 0;
1990     for (StoreFile s : this.storeEngine.getStoreFileManager().getStorefiles()) {
1991       size += s.getReader().getUncompressedDataIndexSize();
1992     }
1993     return size;
1994   }
1995 
1996   @Override
1997   public long getTotalStaticBloomSize() {
1998     long size = 0;
1999     for (StoreFile s : this.storeEngine.getStoreFileManager().getStorefiles()) {
2000       StoreFile.Reader r = s.getReader();
2001       size += r.getTotalBloomSize();
2002     }
2003     return size;
2004   }
2005 
2006   @Override
2007   public long getMemStoreSize() {
2008     return this.memstore.size();
2009   }
2010 
2011   @Override
2012   public int getCompactPriority() {
2013     int priority = this.storeEngine.getStoreFileManager().getStoreCompactionPriority();
2014     if (priority == PRIORITY_USER) {
2015       LOG.warn("Compaction priority is USER despite there being no user compaction");
2016     }
2017     return priority;
2018   }
2019 
2020   @Override
2021   public boolean throttleCompaction(long compactionSize) {
2022     return storeEngine.getCompactionPolicy().throttleCompaction(compactionSize);
2023   }
2024 
2025   public HRegion getHRegion() {
2026     return this.region;
2027   }
2028 
2029   @Override
2030   public RegionCoprocessorHost getCoprocessorHost() {
2031     return this.region.getCoprocessorHost();
2032   }
2033 
2034   @Override
2035   public HRegionInfo getRegionInfo() {
2036     return this.fs.getRegionInfo();
2037   }
2038 
2039   @Override
2040   public boolean areWritesEnabled() {
2041     return this.region.areWritesEnabled();
2042   }
2043 
2044   @Override
2045   public long getSmallestReadPoint() {
2046     return this.region.getSmallestReadPoint();
2047   }
2048 
2049   /**
2050    * Used in tests. TODO: Remove
2051    *
2052    * Updates the value for the given row/family/qualifier. This function will always be seen as
2053    * atomic by other readers because it only puts a single KV to memstore. Thus no read/write
2054    * control necessary.
2055    * @param row row to update
2056    * @param f family to update
2057    * @param qualifier qualifier to update
2058    * @param newValue the new value to set into memstore
2059    * @return memstore size delta
2060    * @throws IOException
2061    */
2062   public long updateColumnValue(byte [] row, byte [] f,
2063                                 byte [] qualifier, long newValue)
2064       throws IOException {
2065 
2066     this.lock.readLock().lock();
2067     try {
2068       long now = EnvironmentEdgeManager.currentTime();
2069 
2070       return this.memstore.updateColumnValue(row,
2071           f,
2072           qualifier,
2073           newValue,
2074           now);
2075 
2076     } finally {
2077       this.lock.readLock().unlock();
2078     }
2079   }
2080 
2081   @Override
2082   public long upsert(Iterable<Cell> cells, long readpoint) throws IOException {
2083     this.lock.readLock().lock();
2084     try {
2085       return this.memstore.upsert(cells, readpoint);
2086     } finally {
2087       this.lock.readLock().unlock();
2088     }
2089   }
2090 
2091   @Override
2092   public StoreFlushContext createFlushContext(long cacheFlushId) {
2093     return new StoreFlusherImpl(cacheFlushId);
2094   }
2095 
2096   private class StoreFlusherImpl implements StoreFlushContext {
2097 
2098     private long cacheFlushSeqNum;
2099     private MemStoreSnapshot snapshot;
2100     private List<Path> tempFiles;
2101     private List<Path> committedFiles;
2102     private long cacheFlushCount;
2103     private long cacheFlushSize;
2104 
2105     private StoreFlusherImpl(long cacheFlushSeqNum) {
2106       this.cacheFlushSeqNum = cacheFlushSeqNum;
2107     }
2108 
2109     /**
2110      * This is not thread safe. The caller should have a lock on the region or the store.
2111      * If necessary, the lock can be added with the patch provided in HBASE-10087
2112      */
2113     @Override
2114     public void prepare() {
2115       this.snapshot = memstore.snapshot();
2116       this.cacheFlushCount = snapshot.getCellsCount();
2117       this.cacheFlushSize = snapshot.getSize();
2118       committedFiles = new ArrayList<Path>(1);
2119     }
2120 
2121     @Override
2122     public void flushCache(MonitoredTask status) throws IOException {
2123       tempFiles = HStore.this.flushCache(cacheFlushSeqNum, snapshot, status);
2124     }
2125 
2126     @Override
2127     public boolean commit(MonitoredTask status) throws IOException {
2128       if (this.tempFiles == null || this.tempFiles.isEmpty()) {
2129         return false;
2130       }
2131       List<StoreFile> storeFiles = new ArrayList<StoreFile>(this.tempFiles.size());
2132       for (Path storeFilePath : tempFiles) {
2133         try {
2134           storeFiles.add(HStore.this.commitFile(storeFilePath, cacheFlushSeqNum, status));
2135         } catch (IOException ex) {
2136           LOG.error("Failed to commit store file " + storeFilePath, ex);
2137           // Try to delete the files we have committed before.
2138           for (StoreFile sf : storeFiles) {
2139             Path pathToDelete = sf.getPath();
2140             try {
2141               sf.deleteReader();
2142             } catch (IOException deleteEx) {
2143               LOG.fatal("Failed to delete store file we committed, halting " + pathToDelete, ex);
2144               Runtime.getRuntime().halt(1);
2145             }
2146           }
2147           throw new IOException("Failed to commit the flush", ex);
2148         }
2149       }
2150 
2151       for (StoreFile sf : storeFiles) {
2152         if (HStore.this.getCoprocessorHost() != null) {
2153           HStore.this.getCoprocessorHost().postFlush(HStore.this, sf);
2154         }
2155         committedFiles.add(sf.getPath());
2156       }
2157 
2158       HStore.this.flushedCellsCount += cacheFlushCount;
2159       HStore.this.flushedCellsSize += cacheFlushSize;
2160 
2161       // Add new file to store files.  Clear snapshot too while we have the Store write lock.
2162       return HStore.this.updateStorefiles(storeFiles, snapshot.getId());
2163     }
2164 
2165     @Override
2166     public List<Path> getCommittedFiles() {
2167       return committedFiles;
2168     }
2169   }
2170 
2171   @Override
2172   public boolean needsCompaction() {
2173     return this.storeEngine.needsCompaction(this.filesCompacting);
2174   }
2175 
2176   @Override
2177   public CacheConfig getCacheConfig() {
2178     return this.cacheConf;
2179   }
2180 
2181   public static final long FIXED_OVERHEAD =
2182       ClassSize.align(ClassSize.OBJECT + (16 * ClassSize.REFERENCE) + (10 * Bytes.SIZEOF_LONG)
2183               + (5 * Bytes.SIZEOF_INT) + (2 * Bytes.SIZEOF_BOOLEAN));
2184 
2185   public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD
2186       + ClassSize.OBJECT + ClassSize.REENTRANT_LOCK
2187       + ClassSize.CONCURRENT_SKIPLISTMAP
2188       + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + ClassSize.OBJECT
2189       + ScanInfo.FIXED_OVERHEAD);
2190 
2191   @Override
2192   public long heapSize() {
2193     return DEEP_OVERHEAD + this.memstore.heapSize();
2194   }
2195 
2196   @Override
2197   public KeyValue.KVComparator getComparator() {
2198     return comparator;
2199   }
2200 
2201   @Override
2202   public ScanInfo getScanInfo() {
2203     return scanInfo;
2204   }
2205 
2206   /**
2207    * Set scan info, used by test
2208    * @param scanInfo new scan info to use for test
2209    */
2210   void setScanInfo(ScanInfo scanInfo) {
2211     this.scanInfo = scanInfo;
2212   }
2213 
2214   @Override
2215   public boolean hasTooManyStoreFiles() {
2216     return getStorefilesCount() > this.blockingFileCount;
2217   }
2218 
2219   @Override
2220   public long getFlushedCellsCount() {
2221     return flushedCellsCount;
2222   }
2223 
2224   @Override
2225   public long getFlushedCellsSize() {
2226     return flushedCellsSize;
2227   }
2228 
2229   @Override
2230   public long getCompactedCellsCount() {
2231     return compactedCellsCount;
2232   }
2233 
2234   @Override
2235   public long getCompactedCellsSize() {
2236     return compactedCellsSize;
2237   }
2238 
2239   @Override
2240   public long getMajorCompactedCellsCount() {
2241     return majorCompactedCellsCount;
2242   }
2243 
2244   @Override
2245   public long getMajorCompactedCellsSize() {
2246     return majorCompactedCellsSize;
2247   }
2248 
2249   /**
2250    * Returns the StoreEngine that is backing this concrete implementation of Store.
2251    * @return Returns the {@link StoreEngine} object used internally inside this HStore object.
2252    */
2253   protected StoreEngine<?, ?, ?, ?> getStoreEngine() {
2254     return this.storeEngine;
2255   }
2256 
2257   protected OffPeakHours getOffPeakHours() {
2258     return this.offPeakHours;
2259   }
2260 
2261   /**
2262    * {@inheritDoc}
2263    */
2264   @Override
2265   public void onConfigurationChange(Configuration conf) {
2266     this.conf = new CompoundConfiguration()
2267             .add(conf)
2268             .addBytesMap(family.getValues());
2269     this.storeEngine.compactionPolicy.setConf(conf);
2270     this.offPeakHours = OffPeakHours.getInstance(conf);
2271   }
2272 
2273   /**
2274    * {@inheritDoc}
2275    */
2276   @Override
2277   public void registerChildren(ConfigurationManager manager) {
2278     // No children to register
2279   }
2280 
2281   /**
2282    * {@inheritDoc}
2283    */
2284   @Override
2285   public void deregisterChildren(ConfigurationManager manager) {
2286     // No children to deregister
2287   }
2288 }