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