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