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       for (StoreFile file : results) {
547         try {
548           if (file != null) file.closeReader(true);
549         } catch (IOException e) {
550           LOG.warn(e.getMessage());
551         }
552       }
553       throw ioe;
554     }
555 
556     return results;
557   }
558 
559   private StoreFile createStoreFileAndReader(final Path p) throws IOException {
560     StoreFileInfo info = new StoreFileInfo(conf, this.getFileSystem(), p);
561     return createStoreFileAndReader(info);
562   }
563 
564   private StoreFile createStoreFileAndReader(final StoreFileInfo info)
565       throws IOException {
566     info.setRegionCoprocessorHost(this.region.getCoprocessorHost());
567     StoreFile storeFile = new StoreFile(this.getFileSystem(), info, this.conf, this.cacheConf,
568       this.family.getBloomFilterType());
569     storeFile.createReader();
570     return storeFile;
571   }
572 
573   @Override
574   public long add(final KeyValue kv) {
575     lock.readLock().lock();
576     try {
577       return this.memstore.add(kv);
578     } finally {
579       lock.readLock().unlock();
580     }
581   }
582 
583   @Override
584   public long timeOfOldestEdit() {
585     return memstore.timeOfOldestEdit();
586   }
587 
588   /**
589    * Adds a value to the memstore
590    *
591    * @param kv
592    * @return memstore size delta
593    */
594   protected long delete(final KeyValue kv) {
595     lock.readLock().lock();
596     try {
597       return this.memstore.delete(kv);
598     } finally {
599       lock.readLock().unlock();
600     }
601   }
602 
603   @Override
604   public void rollback(final KeyValue kv) {
605     lock.readLock().lock();
606     try {
607       this.memstore.rollback(kv);
608     } finally {
609       lock.readLock().unlock();
610     }
611   }
612 
613   /**
614    * @return All store files.
615    */
616   @Override
617   public Collection<StoreFile> getStorefiles() {
618     return this.storeEngine.getStoreFileManager().getStorefiles();
619   }
620 
621   @Override
622   public void assertBulkLoadHFileOk(Path srcPath) throws IOException {
623     HFile.Reader reader  = null;
624     try {
625       LOG.info("Validating hfile at " + srcPath + " for inclusion in "
626           + "store " + this + " region " + this.getRegionInfo().getRegionNameAsString());
627       reader = HFile.createReader(srcPath.getFileSystem(conf),
628           srcPath, cacheConf, conf);
629       reader.loadFileInfo();
630 
631       byte[] firstKey = reader.getFirstRowKey();
632       Preconditions.checkState(firstKey != null, "First key can not be null");
633       byte[] lk = reader.getLastKey();
634       Preconditions.checkState(lk != null, "Last key can not be null");
635       byte[] lastKey =  KeyValue.createKeyValueFromKey(lk).getRow();
636 
637       LOG.debug("HFile bounds: first=" + Bytes.toStringBinary(firstKey) +
638           " last=" + Bytes.toStringBinary(lastKey));
639       LOG.debug("Region bounds: first=" +
640           Bytes.toStringBinary(getRegionInfo().getStartKey()) +
641           " last=" + Bytes.toStringBinary(getRegionInfo().getEndKey()));
642 
643       if (!this.getRegionInfo().containsRange(firstKey, lastKey)) {
644         throw new WrongRegionException(
645             "Bulk load file " + srcPath.toString() + " does not fit inside region "
646             + this.getRegionInfo().getRegionNameAsString());
647       }
648 
649       if(reader.length() > conf.getLong(HConstants.HREGION_MAX_FILESIZE,
650           HConstants.DEFAULT_MAX_FILE_SIZE)) {
651         LOG.warn("Trying to bulk load hfile " + srcPath.toString() + " with size: " +
652             reader.length() + " bytes can be problematic as it may lead to oversplitting.");
653       }
654 
655       if (verifyBulkLoads) {
656         long verificationStartTime = EnvironmentEdgeManager.currentTimeMillis();
657         LOG.info("Full verification started for bulk load hfile: " + srcPath.toString());
658         KeyValue prevKV = null;
659         HFileScanner scanner = reader.getScanner(false, false, false);
660         scanner.seekTo();
661         do {
662           KeyValue kv = scanner.getKeyValue();
663           if (prevKV != null) {
664             if (Bytes.compareTo(prevKV.getBuffer(), prevKV.getRowOffset(),
665                 prevKV.getRowLength(), kv.getBuffer(), kv.getRowOffset(),
666                 kv.getRowLength()) > 0) {
667               throw new InvalidHFileException("Previous row is greater than"
668                   + " current row: path=" + srcPath + " previous="
669                   + Bytes.toStringBinary(prevKV.getKey()) + " current="
670                   + Bytes.toStringBinary(kv.getKey()));
671             }
672             if (Bytes.compareTo(prevKV.getBuffer(), prevKV.getFamilyOffset(),
673                 prevKV.getFamilyLength(), kv.getBuffer(), kv.getFamilyOffset(),
674                 kv.getFamilyLength()) != 0) {
675               throw new InvalidHFileException("Previous key had different"
676                   + " family compared to current key: path=" + srcPath
677                   + " previous=" + Bytes.toStringBinary(prevKV.getFamily())
678                   + " current=" + Bytes.toStringBinary(kv.getFamily()));
679             }
680           }
681           prevKV = kv;
682         } while (scanner.next());
683       LOG.info("Full verification complete for bulk load hfile: " + srcPath.toString()
684          + " took " + (EnvironmentEdgeManager.currentTimeMillis() - verificationStartTime)
685          + " ms");
686       }
687     } finally {
688       if (reader != null) reader.close();
689     }
690   }
691 
692   @Override
693   public void bulkLoadHFile(String srcPathStr, long seqNum) throws IOException {
694     Path srcPath = new Path(srcPathStr);
695     Path dstPath = fs.bulkLoadStoreFile(getColumnFamilyName(), srcPath, seqNum);
696 
697     StoreFile sf = createStoreFileAndReader(dstPath);
698 
699     StoreFile.Reader r = sf.getReader();
700     this.storeSize += r.length();
701     this.totalUncompressedBytes += r.getTotalUncompressedBytes();
702 
703     LOG.info("Loaded HFile " + srcPath + " into store '" + getColumnFamilyName() +
704         "' as " + dstPath + " - updating store file list.");
705 
706     // Append the new storefile into the list
707     this.lock.writeLock().lock();
708     try {
709       this.storeEngine.getStoreFileManager().insertNewFiles(Lists.newArrayList(sf));
710     } finally {
711       // We need the lock, as long as we are updating the storeFiles
712       // or changing the memstore. Let us release it before calling
713       // notifyChangeReadersObservers. See HBASE-4485 for a possible
714       // deadlock scenario that could have happened if continue to hold
715       // the lock.
716       this.lock.writeLock().unlock();
717     }
718     notifyChangedReadersObservers();
719     LOG.info("Successfully loaded store file " + srcPath
720         + " into store " + this + " (new location: " + dstPath + ")");
721     if (LOG.isTraceEnabled()) {
722       String traceMessage = "BULK LOAD time,size,store size,store files ["
723           + EnvironmentEdgeManager.currentTimeMillis() + "," + r.length() + "," + storeSize
724           + "," + storeEngine.getStoreFileManager().getStorefileCount() + "]";
725       LOG.trace(traceMessage);
726     }
727   }
728 
729   @Override
730   public ImmutableCollection<StoreFile> close() throws IOException {
731     this.lock.writeLock().lock();
732     try {
733       // Clear so metrics doesn't find them.
734       ImmutableCollection<StoreFile> result = storeEngine.getStoreFileManager().clearFiles();
735 
736       if (!result.isEmpty()) {
737         // initialize the thread pool for closing store files in parallel.
738         ThreadPoolExecutor storeFileCloserThreadPool = this.region
739             .getStoreFileOpenAndCloseThreadPool("StoreFileCloserThread-"
740                 + this.getColumnFamilyName());
741 
742         // close each store file in parallel
743         CompletionService<Void> completionService =
744           new ExecutorCompletionService<Void>(storeFileCloserThreadPool);
745         for (final StoreFile f : result) {
746           completionService.submit(new Callable<Void>() {
747             @Override
748             public Void call() throws IOException {
749               boolean evictOnClose = 
750                   cacheConf != null? cacheConf.shouldEvictOnClose(): true; 
751               f.closeReader(evictOnClose);
752               return null;
753             }
754           });
755         }
756 
757         IOException ioe = null;
758         try {
759           for (int i = 0; i < result.size(); i++) {
760             try {
761               Future<Void> future = completionService.take();
762               future.get();
763             } catch (InterruptedException e) {
764               if (ioe == null) {
765                 ioe = new InterruptedIOException();
766                 ioe.initCause(e);
767               }
768             } catch (ExecutionException e) {
769               if (ioe == null) ioe = new IOException(e.getCause());
770             }
771           }
772         } finally {
773           storeFileCloserThreadPool.shutdownNow();
774         }
775         if (ioe != null) throw ioe;
776       }
777       LOG.info("Closed " + this);
778       return result;
779     } finally {
780       this.lock.writeLock().unlock();
781     }
782   }
783 
784   /**
785    * Snapshot this stores memstore. Call before running
786    * {@link #flushCache(long, SortedSet, TimeRangeTracker, AtomicLong, MonitoredTask)}
787    *  so it has some work to do.
788    */
789   void snapshot() {
790     this.lock.writeLock().lock();
791     try {
792       this.memstore.snapshot();
793     } finally {
794       this.lock.writeLock().unlock();
795     }
796   }
797 
798   /**
799    * Write out current snapshot.  Presumes {@link #snapshot()} has been called
800    * previously.
801    * @param logCacheFlushId flush sequence number
802    * @param snapshot
803    * @param snapshotTimeRangeTracker
804    * @param flushedSize The number of bytes flushed
805    * @param status
806    * @return The path name of the tmp file to which the store was flushed
807    * @throws IOException
808    */
809   protected List<Path> flushCache(final long logCacheFlushId,
810       SortedSet<KeyValue> snapshot,
811       TimeRangeTracker snapshotTimeRangeTracker,
812       AtomicLong flushedSize,
813       MonitoredTask status) throws IOException {
814     // If an exception happens flushing, we let it out without clearing
815     // the memstore snapshot.  The old snapshot will be returned when we say
816     // 'snapshot', the next time flush comes around.
817     // Retry after catching exception when flushing, otherwise server will abort
818     // itself
819     StoreFlusher flusher = storeEngine.getStoreFlusher();
820     IOException lastException = null;
821     for (int i = 0; i < flushRetriesNumber; i++) {
822       try {
823         List<Path> pathNames = flusher.flushSnapshot(
824             snapshot, logCacheFlushId, snapshotTimeRangeTracker, flushedSize, status);
825         Path lastPathName = null;
826         try {
827           for (Path pathName : pathNames) {
828             lastPathName = pathName;
829             validateStoreFile(pathName);
830           }
831           return pathNames;
832         } catch (Exception e) {
833           LOG.warn("Failed validating store file " + lastPathName + ", retrying num=" + i, e);
834           if (e instanceof IOException) {
835             lastException = (IOException) e;
836           } else {
837             lastException = new IOException(e);
838           }
839         }
840       } catch (IOException e) {
841         LOG.warn("Failed flushing store file, retrying num=" + i, e);
842         lastException = e;
843       }
844       if (lastException != null && i < (flushRetriesNumber - 1)) {
845         try {
846           Thread.sleep(pauseTime);
847         } catch (InterruptedException e) {
848           IOException iie = new InterruptedIOException();
849           iie.initCause(e);
850           throw iie;
851         }
852       }
853     }
854     throw lastException;
855   }
856 
857   /*
858    * @param path The pathname of the tmp file into which the store was flushed
859    * @param logCacheFlushId
860    * @return StoreFile created.
861    * @throws IOException
862    */
863   private StoreFile commitFile(final Path path,
864       final long logCacheFlushId,
865       TimeRangeTracker snapshotTimeRangeTracker,
866       AtomicLong flushedSize,
867       MonitoredTask status)
868       throws IOException {
869     // Write-out finished successfully, move into the right spot
870     Path dstPath = fs.commitStoreFile(getColumnFamilyName(), path);
871 
872     status.setStatus("Flushing " + this + ": reopening flushed file");
873     StoreFile sf = createStoreFileAndReader(dstPath);
874 
875     StoreFile.Reader r = sf.getReader();
876     this.storeSize += r.length();
877     this.totalUncompressedBytes += r.getTotalUncompressedBytes();
878 
879     if (LOG.isInfoEnabled()) {
880       LOG.info("Added " + sf + ", entries=" + r.getEntries() +
881         ", sequenceid=" + logCacheFlushId +
882         ", filesize=" + StringUtils.humanReadableInt(r.length()));
883     }
884     return sf;
885   }
886 
887   @Override
888   public StoreFile.Writer createWriterInTmp(long maxKeyCount, Compression.Algorithm compression,
889                                             boolean isCompaction, boolean includeMVCCReadpoint,
890                                             boolean includesTag)
891       throws IOException {
892     return createWriterInTmp(maxKeyCount, compression, isCompaction, includeMVCCReadpoint,
893         includesTag, false);
894   }
895 
896   /*
897    * @param maxKeyCount
898    * @param compression Compression algorithm to use
899    * @param isCompaction whether we are creating a new file in a compaction
900    * @param includesMVCCReadPoint - whether to include MVCC or not
901    * @param includesTag - includesTag or not
902    * @return Writer for a new StoreFile in the tmp dir.
903    */
904   @Override
905   public StoreFile.Writer createWriterInTmp(long maxKeyCount, Compression.Algorithm compression,
906       boolean isCompaction, boolean includeMVCCReadpoint, boolean includesTag,
907       boolean shouldDropBehind)
908   throws IOException {
909     final CacheConfig writerCacheConf;
910     if (isCompaction) {
911       // Don't cache data on write on compactions.
912       writerCacheConf = new CacheConfig(cacheConf);
913       writerCacheConf.setCacheDataOnWrite(false);
914     } else {
915       writerCacheConf = cacheConf;
916     }
917     InetSocketAddress[] favoredNodes = null;
918     if (region.getRegionServerServices() != null) {
919       favoredNodes = region.getRegionServerServices().getFavoredNodesForRegion(
920           region.getRegionInfo().getEncodedName());
921     }
922     HFileContext hFileContext = createFileContext(compression, includeMVCCReadpoint, includesTag,
923       cryptoContext);
924     StoreFile.Writer w = new StoreFile.WriterBuilder(conf, writerCacheConf,
925         this.getFileSystem())
926             .withFilePath(fs.createTempName())
927             .withComparator(comparator)
928             .withBloomType(family.getBloomFilterType())
929             .withMaxKeyCount(maxKeyCount)
930             .withFavoredNodes(favoredNodes)
931             .withFileContext(hFileContext)
932             .withShouldDropCacheBehind(shouldDropBehind)
933             .build();
934     return w;
935   }
936 
937   private HFileContext createFileContext(Compression.Algorithm compression,
938       boolean includeMVCCReadpoint, boolean includesTag, Encryption.Context cryptoContext) {
939     if (compression == null) {
940       compression = HFile.DEFAULT_COMPRESSION_ALGORITHM;
941     }
942     HFileContext hFileContext = new HFileContextBuilder()
943                                 .withIncludesMvcc(includeMVCCReadpoint)
944                                 .withIncludesTags(includesTag)
945                                 .withCompression(compression)
946                                 .withCompressTags(family.shouldCompressTags())
947                                 .withChecksumType(checksumType)
948                                 .withBytesPerCheckSum(bytesPerChecksum)
949                                 .withBlockSize(blocksize)
950                                 .withHBaseCheckSum(true)
951                                 .withDataBlockEncoding(family.getDataBlockEncoding())
952                                 .withEncryptionContext(cryptoContext)
953                                 .build();
954     return hFileContext;
955   }
956 
957 
958   /*
959    * Change storeFiles adding into place the Reader produced by this new flush.
960    * @param sfs Store files
961    * @param set That was used to make the passed file.
962    * @throws IOException
963    * @return Whether compaction is required.
964    */
965   private boolean updateStorefiles(
966       final List<StoreFile> sfs, final SortedSet<KeyValue> set) throws IOException {
967     this.lock.writeLock().lock();
968     try {
969       this.storeEngine.getStoreFileManager().insertNewFiles(sfs);
970       this.memstore.clearSnapshot(set);
971     } finally {
972       // We need the lock, as long as we are updating the storeFiles
973       // or changing the memstore. Let us release it before calling
974       // notifyChangeReadersObservers. See HBASE-4485 for a possible
975       // deadlock scenario that could have happened if continue to hold
976       // the lock.
977       this.lock.writeLock().unlock();
978     }
979 
980     // Tell listeners of the change in readers.
981     notifyChangedReadersObservers();
982 
983     if (LOG.isTraceEnabled()) {
984       long totalSize = 0;
985       for (StoreFile sf : sfs) {
986         totalSize += sf.getReader().length();
987       }
988       String traceMessage = "FLUSH time,count,size,store size,store files ["
989           + EnvironmentEdgeManager.currentTimeMillis() + "," + sfs.size() + "," + totalSize
990           + "," + storeSize + "," + storeEngine.getStoreFileManager().getStorefileCount() + "]";
991       LOG.trace(traceMessage);
992     }
993     return needsCompaction();
994   }
995 
996   /*
997    * Notify all observers that set of Readers has changed.
998    * @throws IOException
999    */
1000   private void notifyChangedReadersObservers() throws IOException {
1001     for (ChangedReadersObserver o: this.changedReaderObservers) {
1002       o.updateReaders();
1003     }
1004   }
1005 
1006   /**
1007    * Get all scanners with no filtering based on TTL (that happens further down
1008    * the line).
1009    * @return all scanners for this store
1010    */
1011   @Override
1012   public List<KeyValueScanner> getScanners(boolean cacheBlocks, boolean isGet,
1013       boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow,
1014       byte[] stopRow, long readPt) throws IOException {
1015     Collection<StoreFile> storeFilesToScan;
1016     List<KeyValueScanner> memStoreScanners;
1017     this.lock.readLock().lock();
1018     try {
1019       storeFilesToScan =
1020           this.storeEngine.getStoreFileManager().getFilesForScanOrGet(isGet, startRow, stopRow);
1021       memStoreScanners = this.memstore.getScanners(readPt);
1022     } finally {
1023       this.lock.readLock().unlock();
1024     }
1025 
1026     // First the store file scanners
1027 
1028     // TODO this used to get the store files in descending order,
1029     // but now we get them in ascending order, which I think is
1030     // actually more correct, since memstore get put at the end.
1031     List<StoreFileScanner> sfScanners = StoreFileScanner.getScannersForStoreFiles(storeFilesToScan,
1032         cacheBlocks, usePread, isCompaction, false, matcher, readPt);
1033     List<KeyValueScanner> scanners =
1034       new ArrayList<KeyValueScanner>(sfScanners.size()+1);
1035     scanners.addAll(sfScanners);
1036     // Then the memstore scanners
1037     scanners.addAll(memStoreScanners);
1038     return scanners;
1039   }
1040 
1041   @Override
1042   public void addChangedReaderObserver(ChangedReadersObserver o) {
1043     this.changedReaderObservers.add(o);
1044   }
1045 
1046   @Override
1047   public void deleteChangedReaderObserver(ChangedReadersObserver o) {
1048     // We don't check if observer present; it may not be (legitimately)
1049     this.changedReaderObservers.remove(o);
1050   }
1051 
1052   //////////////////////////////////////////////////////////////////////////////
1053   // Compaction
1054   //////////////////////////////////////////////////////////////////////////////
1055 
1056   /**
1057    * Compact the StoreFiles.  This method may take some time, so the calling
1058    * thread must be able to block for long periods.
1059    *
1060    * <p>During this time, the Store can work as usual, getting values from
1061    * StoreFiles and writing new StoreFiles from the memstore.
1062    *
1063    * Existing StoreFiles are not destroyed until the new compacted StoreFile is
1064    * completely written-out to disk.
1065    *
1066    * <p>The compactLock prevents multiple simultaneous compactions.
1067    * The structureLock prevents us from interfering with other write operations.
1068    *
1069    * <p>We don't want to hold the structureLock for the whole time, as a compact()
1070    * can be lengthy and we want to allow cache-flushes during this period.
1071    *
1072    * <p> Compaction event should be idempotent, since there is no IO Fencing for
1073    * the region directory in hdfs. A region server might still try to complete the
1074    * compaction after it lost the region. That is why the following events are carefully
1075    * ordered for a compaction:
1076    *  1. Compaction writes new files under region/.tmp directory (compaction output)
1077    *  2. Compaction atomically moves the temporary file under region directory
1078    *  3. Compaction appends a WAL edit containing the compaction input and output files.
1079    *  Forces sync on WAL.
1080    *  4. Compaction deletes the input files from the region directory.
1081    *
1082    * Failure conditions are handled like this:
1083    *  - If RS fails before 2, compaction wont complete. Even if RS lives on and finishes
1084    *  the compaction later, it will only write the new data file to the region directory.
1085    *  Since we already have this data, this will be idempotent but we will have a redundant
1086    *  copy of the data.
1087    *  - If RS fails between 2 and 3, the region will have a redundant copy of the data. The
1088    *  RS that failed won't be able to finish snyc() for WAL because of lease recovery in WAL.
1089    *  - If RS fails after 3, the region region server who opens the region will pick up the
1090    *  the compaction marker from the WAL and replay it by removing the compaction input files.
1091    *  Failed RS can also attempt to delete those files, but the operation will be idempotent
1092    *
1093    * See HBASE-2231 for details.
1094    *
1095    * @param compaction compaction details obtained from requestCompaction()
1096    * @throws IOException
1097    * @return Storefile we compacted into or null if we failed or opted out early.
1098    */
1099   @Override
1100   public List<StoreFile> compact(CompactionContext compaction,
1101       CompactionThroughputController throughputController) throws IOException {
1102     return compact(compaction, throughputController, null);
1103   }
1104 
1105   @Override
1106   public List<StoreFile> compact(CompactionContext compaction,
1107     CompactionThroughputController throughputController, User user) throws IOException {
1108     assert compaction != null;
1109     List<StoreFile> sfs = null;
1110     CompactionRequest cr = compaction.getRequest();
1111     try {
1112       // Do all sanity checking in here if we have a valid CompactionRequest
1113       // because we need to clean up after it on the way out in a finally
1114       // block below
1115       long compactionStartTime = EnvironmentEdgeManager.currentTimeMillis();
1116       assert compaction.hasSelection();
1117       Collection<StoreFile> filesToCompact = cr.getFiles();
1118       assert !filesToCompact.isEmpty();
1119       synchronized (filesCompacting) {
1120         // sanity check: we're compacting files that this store knows about
1121         // TODO: change this to LOG.error() after more debugging
1122         Preconditions.checkArgument(filesCompacting.containsAll(filesToCompact));
1123       }
1124 
1125       // Ready to go. Have list of files to compact.
1126       LOG.info("Starting compaction of " + filesToCompact.size() + " file(s) in "
1127           + this + " of " + this.getRegionInfo().getRegionNameAsString()
1128           + " into tmpdir=" + fs.getTempDir() + ", totalSize="
1129           + StringUtils.humanReadableInt(cr.getSize()));
1130 
1131       // Commence the compaction.
1132       List<Path> newFiles = compaction.compact(throughputController, user);
1133 
1134       // TODO: get rid of this!
1135       if (!this.conf.getBoolean("hbase.hstore.compaction.complete", true)) {
1136         LOG.warn("hbase.hstore.compaction.complete is set to false");
1137         sfs = new ArrayList<StoreFile>(newFiles.size());
1138         for (Path newFile : newFiles) {
1139           // Create storefile around what we wrote with a reader on it.
1140           StoreFile sf = createStoreFileAndReader(newFile);
1141           sf.closeReader(true);
1142           sfs.add(sf);
1143         }
1144         return sfs;
1145       }
1146       // Do the steps necessary to complete the compaction.
1147       sfs = moveCompatedFilesIntoPlace(cr, newFiles, user);
1148       writeCompactionWalRecord(filesToCompact, sfs);
1149       replaceStoreFiles(filesToCompact, sfs);
1150       if (cr.isMajor()) {
1151         majorCompactedCellsCount += getCompactionProgress().totalCompactingKVs;
1152         majorCompactedCellsSize += getCompactionProgress().totalCompactedSize;
1153       } else {
1154         compactedCellsCount += getCompactionProgress().totalCompactingKVs;
1155         compactedCellsSize += getCompactionProgress().totalCompactedSize;
1156       }
1157       // At this point the store will use new files for all new scanners.
1158       completeCompaction(filesToCompact); // Archive old files & update store size.
1159 
1160       logCompactionEndMessage(cr, sfs, compactionStartTime);
1161       return sfs;
1162     } finally {
1163       finishCompactionRequest(cr);
1164     }
1165   }
1166 
1167   private List<StoreFile> moveCompatedFilesIntoPlace(
1168       final CompactionRequest cr, List<Path> newFiles, User user) throws IOException {
1169     List<StoreFile> sfs = new ArrayList<StoreFile>(newFiles.size());
1170     for (Path newFile : newFiles) {
1171       assert newFile != null;
1172       final StoreFile sf = moveFileIntoPlace(newFile);
1173       if (this.getCoprocessorHost() != null) {
1174         final Store thisStore = this;
1175         if (user == null) {
1176           getCoprocessorHost().postCompact(thisStore, sf, cr);
1177         } else {
1178           try {
1179             user.getUGI().doAs(new PrivilegedExceptionAction<Void>() {
1180               @Override
1181               public Void run() throws Exception {
1182                 getCoprocessorHost().postCompact(thisStore, sf, cr);
1183                 return null;
1184               }
1185             });
1186           } catch (InterruptedException ie) {
1187             InterruptedIOException iioe = new InterruptedIOException();
1188             iioe.initCause(ie);
1189             throw iioe;
1190           }
1191         }
1192       }
1193       assert sf != null;
1194       sfs.add(sf);
1195     }
1196     return sfs;
1197   }
1198 
1199   // Package-visible for tests
1200   StoreFile moveFileIntoPlace(final Path newFile) throws IOException {
1201     validateStoreFile(newFile);
1202     // Move the file into the right spot
1203     Path destPath = fs.commitStoreFile(getColumnFamilyName(), newFile);
1204     return createStoreFileAndReader(destPath);
1205   }
1206 
1207   /**
1208    * Writes the compaction WAL record.
1209    * @param filesCompacted Files compacted (input).
1210    * @param newFiles Files from compaction.
1211    */
1212   private void writeCompactionWalRecord(Collection<StoreFile> filesCompacted,
1213       Collection<StoreFile> newFiles) throws IOException {
1214     if (region.getLog() == null) return;
1215     List<Path> inputPaths = new ArrayList<Path>(filesCompacted.size());
1216     for (StoreFile f : filesCompacted) {
1217       inputPaths.add(f.getPath());
1218     }
1219     List<Path> outputPaths = new ArrayList<Path>(newFiles.size());
1220     for (StoreFile f : newFiles) {
1221       outputPaths.add(f.getPath());
1222     }
1223     HRegionInfo info = this.region.getRegionInfo();
1224     CompactionDescriptor compactionDescriptor = ProtobufUtil.toCompactionDescriptor(info,
1225         family.getName(), inputPaths, outputPaths, fs.getStoreDir(getFamily().getNameAsString()));
1226     HLogUtil.writeCompactionMarker(region.getLog(), this.region.getTableDesc(),
1227         this.region.getRegionInfo(), compactionDescriptor, this.region.getSequenceId());
1228   }
1229 
1230   private void replaceStoreFiles(final Collection<StoreFile> compactedFiles,
1231       final Collection<StoreFile> result) throws IOException {
1232     this.lock.writeLock().lock();
1233     try {
1234       this.storeEngine.getStoreFileManager().addCompactionResults(compactedFiles, result);
1235       filesCompacting.removeAll(compactedFiles); // safe bc: lock.writeLock();
1236     } finally {
1237       this.lock.writeLock().unlock();
1238     }
1239   }
1240 
1241   /**
1242    * Log a very elaborate compaction completion message.
1243    * @param cr Request.
1244    * @param sfs Resulting files.
1245    * @param compactionStartTime Start time.
1246    */
1247   private void logCompactionEndMessage(
1248       CompactionRequest cr, List<StoreFile> sfs, long compactionStartTime) {
1249     long now = EnvironmentEdgeManager.currentTimeMillis();
1250     StringBuilder message = new StringBuilder(
1251       "Completed" + (cr.isMajor() ? " major " : " ") + "compaction of "
1252       + cr.getFiles().size() + " file(s) in " + this + " of "
1253       + this.getRegionInfo().getRegionNameAsString()
1254       + " into ");
1255     if (sfs.isEmpty()) {
1256       message.append("none, ");
1257     } else {
1258       for (StoreFile sf: sfs) {
1259         message.append(sf.getPath().getName());
1260         message.append("(size=");
1261         message.append(StringUtils.humanReadableInt(sf.getReader().length()));
1262         message.append("), ");
1263       }
1264     }
1265     message.append("total size for store is ")
1266       .append(StringUtils.humanReadableInt(storeSize))
1267       .append(". This selection was in queue for ")
1268       .append(StringUtils.formatTimeDiff(compactionStartTime, cr.getSelectionTime()))
1269       .append(", and took ").append(StringUtils.formatTimeDiff(now, compactionStartTime))
1270       .append(" to execute.");
1271     LOG.info(message.toString());
1272     if (LOG.isTraceEnabled()) {
1273       int fileCount = storeEngine.getStoreFileManager().getStorefileCount();
1274       long resultSize = 0;
1275       for (StoreFile sf : sfs) {
1276         resultSize += sf.getReader().length();
1277       }
1278       String traceMessage = "COMPACTION start,end,size out,files in,files out,store size,"
1279         + "store files [" + compactionStartTime + "," + now + "," + resultSize + ","
1280           + cr.getFiles().size() + "," + sfs.size() + "," +  storeSize + "," + fileCount + "]";
1281       LOG.trace(traceMessage);
1282     }
1283   }
1284 
1285   /**
1286    * Call to complete a compaction. Its for the case where we find in the WAL a compaction
1287    * that was not finished.  We could find one recovering a WAL after a regionserver crash.
1288    * See HBASE-2231.
1289    * @param compaction
1290    */
1291   @Override
1292   public void completeCompactionMarker(CompactionDescriptor compaction)
1293       throws IOException {
1294     LOG.debug("Completing compaction from the WAL marker");
1295     List<String> compactionInputs = compaction.getCompactionInputList();
1296 
1297     // The Compaction Marker is written after the compaction is completed,
1298     // and the files moved into the region/family folder.
1299     //
1300     // If we crash after the entry is written, we may not have removed the
1301     // input files, but the output file is present.
1302     // (The unremoved input files will be removed by this function)
1303     //
1304     // If we scan the directory and the file is not present, it can mean that:
1305     //   - The file was manually removed by the user
1306     //   - The file was removed as consequence of subsequent compaction
1307     // so, we can't do anything with the "compaction output list" because those
1308     // files have already been loaded when opening the region (by virtue of
1309     // being in the store's folder) or they may be missing due to a compaction.
1310 
1311     String familyName = this.getColumnFamilyName();
1312     List<Path> inputPaths = new ArrayList<Path>(compactionInputs.size());
1313     for (String compactionInput : compactionInputs) {
1314       Path inputPath = fs.getStoreFilePath(familyName, compactionInput);
1315       inputPaths.add(inputPath);
1316     }
1317 
1318     //some of the input files might already be deleted
1319     List<StoreFile> inputStoreFiles = new ArrayList<StoreFile>(compactionInputs.size());
1320     for (StoreFile sf : this.getStorefiles()) {
1321       if (inputPaths.contains(sf.getQualifiedPath())) {
1322         inputStoreFiles.add(sf);
1323       }
1324     }
1325 
1326     this.replaceStoreFiles(inputStoreFiles, Collections.<StoreFile>emptyList());
1327     this.completeCompaction(inputStoreFiles);
1328   }
1329 
1330   /**
1331    * This method tries to compact N recent files for testing.
1332    * Note that because compacting "recent" files only makes sense for some policies,
1333    * e.g. the default one, it assumes default policy is used. It doesn't use policy,
1334    * but instead makes a compaction candidate list by itself.
1335    * @param N Number of files.
1336    */
1337   public void compactRecentForTestingAssumingDefaultPolicy(int N) throws IOException {
1338     List<StoreFile> filesToCompact;
1339     boolean isMajor;
1340 
1341     this.lock.readLock().lock();
1342     try {
1343       synchronized (filesCompacting) {
1344         filesToCompact = Lists.newArrayList(storeEngine.getStoreFileManager().getStorefiles());
1345         if (!filesCompacting.isEmpty()) {
1346           // exclude all files older than the newest file we're currently
1347           // compacting. this allows us to preserve contiguity (HBASE-2856)
1348           StoreFile last = filesCompacting.get(filesCompacting.size() - 1);
1349           int idx = filesToCompact.indexOf(last);
1350           Preconditions.checkArgument(idx != -1);
1351           filesToCompact.subList(0, idx + 1).clear();
1352         }
1353         int count = filesToCompact.size();
1354         if (N > count) {
1355           throw new RuntimeException("Not enough files");
1356         }
1357 
1358         filesToCompact = filesToCompact.subList(count - N, count);
1359         isMajor = (filesToCompact.size() == storeEngine.getStoreFileManager().getStorefileCount());
1360         filesCompacting.addAll(filesToCompact);
1361         Collections.sort(filesCompacting, StoreFile.Comparators.SEQ_ID);
1362       }
1363     } finally {
1364       this.lock.readLock().unlock();
1365     }
1366 
1367     try {
1368       // Ready to go. Have list of files to compact.
1369       List<Path> newFiles = ((DefaultCompactor)this.storeEngine.getCompactor())
1370           .compactForTesting(filesToCompact, isMajor);
1371       for (Path newFile: newFiles) {
1372         // Move the compaction into place.
1373         StoreFile sf = moveFileIntoPlace(newFile);
1374         if (this.getCoprocessorHost() != null) {
1375           this.getCoprocessorHost().postCompact(this, sf, null);
1376         }
1377         replaceStoreFiles(filesToCompact, Lists.newArrayList(sf));
1378         completeCompaction(filesToCompact);
1379       }
1380     } finally {
1381       synchronized (filesCompacting) {
1382         filesCompacting.removeAll(filesToCompact);
1383       }
1384     }
1385   }
1386 
1387   @Override
1388   public boolean hasReferences() {
1389     return StoreUtils.hasReferences(this.storeEngine.getStoreFileManager().getStorefiles());
1390   }
1391 
1392   @Override
1393   public CompactionProgress getCompactionProgress() {
1394     return this.storeEngine.getCompactor().getProgress();
1395   }
1396 
1397   @Override
1398   public boolean isMajorCompaction() throws IOException {
1399     for (StoreFile sf : this.storeEngine.getStoreFileManager().getStorefiles()) {
1400       // TODO: what are these reader checks all over the place?
1401       if (sf.getReader() == null) {
1402         LOG.debug("StoreFile " + sf + " has null Reader");
1403         return false;
1404       }
1405     }
1406     return storeEngine.getCompactionPolicy().isMajorCompaction(
1407         this.storeEngine.getStoreFileManager().getStorefiles());
1408   }
1409 
1410   @Override
1411   public CompactionContext requestCompaction() throws IOException {
1412     return requestCompaction(Store.NO_PRIORITY, null);
1413   }
1414 
1415   @Override
1416   public CompactionContext requestCompaction(int priority, CompactionRequest baseRequest)
1417       throws IOException {
1418     return requestCompaction(priority, baseRequest, null);
1419   }
1420   @Override
1421   public CompactionContext requestCompaction(int priority, final CompactionRequest baseRequest,
1422       User user) throws IOException {
1423     // don't even select for compaction if writes are disabled
1424     if (!this.areWritesEnabled()) {
1425       return null;
1426     }
1427 
1428     // Before we do compaction, try to get rid of unneeded files to simplify things.
1429     removeUnneededFiles();
1430 
1431     final CompactionContext compaction = storeEngine.createCompaction();
1432     this.lock.readLock().lock();
1433     try {
1434       synchronized (filesCompacting) {
1435         final Store thisStore = this;
1436         // First, see if coprocessor would want to override selection.
1437         if (this.getCoprocessorHost() != null) {
1438           final List<StoreFile> candidatesForCoproc = compaction.preSelect(this.filesCompacting);
1439           boolean override = false;
1440           if (user == null) {
1441             override = getCoprocessorHost().preCompactSelection(this, candidatesForCoproc,
1442               baseRequest);
1443           } else {
1444             try {
1445               override = user.getUGI().doAs(new PrivilegedExceptionAction<Boolean>() {
1446                 @Override
1447                 public Boolean run() throws Exception {
1448                   return getCoprocessorHost().preCompactSelection(thisStore, candidatesForCoproc,
1449                     baseRequest);
1450                 }
1451               });
1452             } catch (InterruptedException ie) {
1453               InterruptedIOException iioe = new InterruptedIOException();
1454               iioe.initCause(ie);
1455               throw iioe;
1456             }
1457           }
1458           if (override) {
1459             // Coprocessor is overriding normal file selection.
1460             compaction.forceSelect(new CompactionRequest(candidatesForCoproc));
1461           }
1462         }
1463 
1464         // Normal case - coprocessor is not overriding file selection.
1465         if (!compaction.hasSelection()) {
1466           boolean isUserCompaction = priority == Store.PRIORITY_USER;
1467           boolean mayUseOffPeak = offPeakHours.isOffPeakHour() &&
1468               offPeakCompactionTracker.compareAndSet(false, true);
1469           try {
1470             compaction.select(this.filesCompacting, isUserCompaction,
1471               mayUseOffPeak, forceMajor && filesCompacting.isEmpty());
1472           } catch (IOException e) {
1473             if (mayUseOffPeak) {
1474               offPeakCompactionTracker.set(false);
1475             }
1476             throw e;
1477           }
1478           assert compaction.hasSelection();
1479           if (mayUseOffPeak && !compaction.getRequest().isOffPeak()) {
1480             // Compaction policy doesn't want to take advantage of off-peak.
1481             offPeakCompactionTracker.set(false);
1482           }
1483         }
1484         if (this.getCoprocessorHost() != null) {
1485           if (user == null) {
1486             this.getCoprocessorHost().postCompactSelection(
1487               this, ImmutableList.copyOf(compaction.getRequest().getFiles()), baseRequest);
1488           } else {
1489             try {
1490               user.getUGI().doAs(new PrivilegedExceptionAction<Void>() {
1491                 @Override
1492                 public Void run() throws Exception {
1493                   getCoprocessorHost().postCompactSelection(
1494                     thisStore,ImmutableList.copyOf(compaction.getRequest().getFiles()),baseRequest);
1495                   return null;
1496                 }
1497               });
1498             } catch (InterruptedException ie) {
1499               InterruptedIOException iioe = new InterruptedIOException();
1500               iioe.initCause(ie);
1501               throw iioe;
1502             }
1503           }
1504         }
1505 
1506         // Selected files; see if we have a compaction with some custom base request.
1507         if (baseRequest != null) {
1508           // Update the request with what the system thinks the request should be;
1509           // its up to the request if it wants to listen.
1510           compaction.forceSelect(
1511               baseRequest.combineWith(compaction.getRequest()));
1512         }
1513 
1514         // Finally, we have the resulting files list. Check if we have any files at all.
1515         final Collection<StoreFile> selectedFiles = compaction.getRequest().getFiles();
1516         if (selectedFiles.isEmpty()) {
1517           return null;
1518         }
1519 
1520         addToCompactingFiles(selectedFiles);
1521 
1522         // If we're enqueuing a major, clear the force flag.
1523         boolean isMajor = selectedFiles.size() == this.getStorefilesCount();
1524         this.forceMajor = this.forceMajor && !isMajor;
1525 
1526         // Set common request properties.
1527         // Set priority, either override value supplied by caller or from store.
1528         compaction.getRequest().setPriority(
1529             (priority != Store.NO_PRIORITY) ? priority : getCompactPriority());
1530         compaction.getRequest().setIsMajor(isMajor);
1531         compaction.getRequest().setDescription(
1532             getRegionInfo().getRegionNameAsString(), getColumnFamilyName());
1533       }
1534     } finally {
1535       this.lock.readLock().unlock();
1536     }
1537 
1538     LOG.debug(getRegionInfo().getEncodedName() + " - " + getColumnFamilyName() + ": Initiating "
1539         + (compaction.getRequest().isMajor() ? "major" : "minor") + " compaction");
1540     this.region.reportCompactionRequestStart(compaction.getRequest().isMajor());
1541     return compaction;
1542   }
1543 
1544   /** Adds the files to compacting files. filesCompacting must be locked. */
1545   private void addToCompactingFiles(final Collection<StoreFile> filesToAdd) {
1546     if (filesToAdd == null) return;
1547     // Check that we do not try to compact the same StoreFile twice.
1548     if (!Collections.disjoint(filesCompacting, filesToAdd)) {
1549       Preconditions.checkArgument(false, "%s overlaps with %s", filesToAdd, filesCompacting);
1550     }
1551     filesCompacting.addAll(filesToAdd);
1552     Collections.sort(filesCompacting, StoreFile.Comparators.SEQ_ID);
1553   }
1554 
1555   private void removeUnneededFiles() throws IOException {
1556     if (!conf.getBoolean("hbase.store.delete.expired.storefile", true)) return;
1557     if (getFamily().getMinVersions() > 0) {
1558       LOG.debug("Skipping expired store file removal due to min version being " +
1559           getFamily().getMinVersions());
1560       return;
1561     }
1562     this.lock.readLock().lock();
1563     Collection<StoreFile> delSfs = null;
1564     try {
1565       synchronized (filesCompacting) {
1566         long cfTtl = getStoreFileTtl();
1567         if (cfTtl != Long.MAX_VALUE) {
1568           delSfs = storeEngine.getStoreFileManager().getUnneededFiles(
1569               EnvironmentEdgeManager.currentTimeMillis() - cfTtl, filesCompacting);
1570           addToCompactingFiles(delSfs);
1571         }
1572       }
1573     } finally {
1574       this.lock.readLock().unlock();
1575     }
1576     if (delSfs == null || delSfs.isEmpty()) return;
1577 
1578     Collection<StoreFile> newFiles = new ArrayList<StoreFile>(); // No new files.
1579     writeCompactionWalRecord(delSfs, newFiles);
1580     replaceStoreFiles(delSfs, newFiles);
1581     completeCompaction(delSfs);
1582     LOG.info("Completed removal of " + delSfs.size() + " unnecessary (expired) file(s) in "
1583         + this + " of " + this.getRegionInfo().getRegionNameAsString()
1584         + "; total size for store is " + StringUtils.humanReadableInt(storeSize));
1585   }
1586 
1587   @Override
1588   public void cancelRequestedCompaction(CompactionContext compaction) {
1589     finishCompactionRequest(compaction.getRequest());
1590   }
1591 
1592   private void finishCompactionRequest(CompactionRequest cr) {
1593     this.region.reportCompactionRequestEnd(cr.isMajor(), cr.getFiles().size(), cr.getSize());
1594     if (cr.isOffPeak()) {
1595       offPeakCompactionTracker.set(false);
1596       cr.setOffPeak(false);
1597     }
1598     synchronized (filesCompacting) {
1599       filesCompacting.removeAll(cr.getFiles());
1600     }
1601   }
1602 
1603   /**
1604    * Validates a store file by opening and closing it. In HFileV2 this should
1605    * not be an expensive operation.
1606    *
1607    * @param path the path to the store file
1608    */
1609   private void validateStoreFile(Path path)
1610       throws IOException {
1611     StoreFile storeFile = null;
1612     try {
1613       storeFile = createStoreFileAndReader(path);
1614     } catch (IOException e) {
1615       LOG.error("Failed to open store file : " + path
1616           + ", keeping it in tmp location", e);
1617       throw e;
1618     } finally {
1619       if (storeFile != null) {
1620         storeFile.closeReader(false);
1621       }
1622     }
1623   }
1624 
1625   /*
1626    * <p>It works by processing a compaction that's been written to disk.
1627    *
1628    * <p>It is usually invoked at the end of a compaction, but might also be
1629    * invoked at HStore startup, if the prior execution died midway through.
1630    *
1631    * <p>Moving the compacted TreeMap into place means:
1632    * <pre>
1633    * 1) Unload all replaced StoreFile, close and collect list to delete.
1634    * 2) Compute new store size
1635    * </pre>
1636    *
1637    * @param compactedFiles list of files that were compacted
1638    * @param newFile StoreFile that is the result of the compaction
1639    */
1640   @VisibleForTesting
1641   protected void completeCompaction(final Collection<StoreFile> compactedFiles)
1642       throws IOException {
1643     try {
1644       // Do not delete old store files until we have sent out notification of
1645       // change in case old files are still being accessed by outstanding scanners.
1646       // Don't do this under writeLock; see HBASE-4485 for a possible deadlock
1647       // scenario that could have happened if continue to hold the lock.
1648       notifyChangedReadersObservers();
1649       // At this point the store will use new files for all scanners.
1650 
1651       // let the archive util decide if we should archive or delete the files
1652       LOG.debug("Removing store files after compaction...");
1653       for (StoreFile compactedFile : compactedFiles) {
1654         compactedFile.closeReader(true);
1655       }
1656       this.fs.removeStoreFiles(this.getColumnFamilyName(), compactedFiles);
1657     } catch (IOException e) {
1658       e = RemoteExceptionHandler.checkIOException(e);
1659       LOG.error("Failed removing compacted files in " + this +
1660         ". Files we were trying to remove are " + compactedFiles.toString() +
1661         "; some of them may have been already removed", e);
1662     }
1663 
1664     // 4. Compute new store size
1665     this.storeSize = 0L;
1666     this.totalUncompressedBytes = 0L;
1667     for (StoreFile hsf : this.storeEngine.getStoreFileManager().getStorefiles()) {
1668       StoreFile.Reader r = hsf.getReader();
1669       if (r == null) {
1670         LOG.warn("StoreFile " + hsf + " has a null Reader");
1671         continue;
1672       }
1673       this.storeSize += r.length();
1674       this.totalUncompressedBytes += r.getTotalUncompressedBytes();
1675     }
1676   }
1677 
1678   /*
1679    * @param wantedVersions How many versions were asked for.
1680    * @return wantedVersions or this families' {@link HConstants#VERSIONS}.
1681    */
1682   int versionsToReturn(final int wantedVersions) {
1683     if (wantedVersions <= 0) {
1684       throw new IllegalArgumentException("Number of versions must be > 0");
1685     }
1686     // Make sure we do not return more than maximum versions for this store.
1687     int maxVersions = this.family.getMaxVersions();
1688     return wantedVersions > maxVersions ? maxVersions: wantedVersions;
1689   }
1690 
1691   /**
1692    * @param kv
1693    * @param oldestTimestamp
1694    * @return true if the cell is expired
1695    */
1696   static boolean isCellTTLExpired(final Cell cell, final long oldestTimestamp, final long now) {
1697     // Do not create an Iterator or Tag objects unless the cell actually has
1698     // tags
1699     if (cell.getTagsLengthUnsigned() > 0) {
1700       // Look for a TTL tag first. Use it instead of the family setting if
1701       // found. If a cell has multiple TTLs, resolve the conflict by using the
1702       // first tag encountered.
1703       Iterator<Tag> i = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
1704         cell.getTagsLengthUnsigned());
1705       while (i.hasNext()) {
1706         Tag t = i.next();
1707         if (TagType.TTL_TAG_TYPE == t.getType()) {
1708           // Unlike in schema cell TTLs are stored in milliseconds, no need
1709           // to convert
1710           long ts = cell.getTimestamp();
1711           assert t.getTagLength() == Bytes.SIZEOF_LONG;
1712           long ttl = Bytes.toLong(t.getBuffer(), t.getTagOffset(), t.getTagLength());
1713           if (ts + ttl < now) {
1714             return true;
1715           }
1716           // Per cell TTLs cannot extend lifetime beyond family settings, so
1717           // fall through to check that
1718           break;
1719         }
1720       }
1721     }
1722     return false;
1723   }
1724 
1725   @Override
1726   public KeyValue getRowKeyAtOrBefore(final byte[] row) throws IOException {
1727     // If minVersions is set, we will not ignore expired KVs.
1728     // As we're only looking for the latest matches, that should be OK.
1729     // With minVersions > 0 we guarantee that any KV that has any version
1730     // at all (expired or not) has at least one version that will not expire.
1731     // Note that this method used to take a KeyValue as arguments. KeyValue
1732     // can be back-dated, a row key cannot.
1733     long ttlToUse = scanInfo.getMinVersions() > 0 ? Long.MAX_VALUE : this.scanInfo.getTtl();
1734 
1735     KeyValue kv = new KeyValue(row, HConstants.LATEST_TIMESTAMP);
1736 
1737     GetClosestRowBeforeTracker state = new GetClosestRowBeforeTracker(
1738       this.comparator, kv, ttlToUse, this.getRegionInfo().isMetaRegion());
1739     this.lock.readLock().lock();
1740     try {
1741       // First go to the memstore.  Pick up deletes and candidates.
1742       this.memstore.getRowKeyAtOrBefore(state);
1743       // Check if match, if we got a candidate on the asked for 'kv' row.
1744       // Process each relevant store file. Run through from newest to oldest.
1745       Iterator<StoreFile> sfIterator = this.storeEngine.getStoreFileManager()
1746           .getCandidateFilesForRowKeyBefore(state.getTargetKey());
1747       while (sfIterator.hasNext()) {
1748         StoreFile sf = sfIterator.next();
1749         sfIterator.remove(); // Remove sf from iterator.
1750         boolean haveNewCandidate = rowAtOrBeforeFromStoreFile(sf, state);
1751         KeyValue keyv = state.getCandidate();
1752         // we have an optimization here which stops the search if we find exact match.
1753         if (keyv != null && keyv.matchingRow(row)) return state.getCandidate();
1754         if (haveNewCandidate) {
1755           sfIterator = this.storeEngine.getStoreFileManager().updateCandidateFilesForRowKeyBefore(
1756               sfIterator, state.getTargetKey(), state.getCandidate());
1757         }
1758       }
1759       return state.getCandidate();
1760     } finally {
1761       this.lock.readLock().unlock();
1762     }
1763   }
1764 
1765   /*
1766    * Check an individual MapFile for the row at or before a given row.
1767    * @param f
1768    * @param state
1769    * @throws IOException
1770    * @return True iff the candidate has been updated in the state.
1771    */
1772   private boolean rowAtOrBeforeFromStoreFile(final StoreFile f,
1773                                           final GetClosestRowBeforeTracker state)
1774       throws IOException {
1775     StoreFile.Reader r = f.getReader();
1776     if (r == null) {
1777       LOG.warn("StoreFile " + f + " has a null Reader");
1778       return false;
1779     }
1780     if (r.getEntries() == 0) {
1781       LOG.warn("StoreFile " + f + " is a empty store file");
1782       return false;
1783     }
1784     // TODO: Cache these keys rather than make each time?
1785     byte [] fk = r.getFirstKey();
1786     if (fk == null) return false;
1787     KeyValue firstKV = KeyValue.createKeyValueFromKey(fk, 0, fk.length);
1788     byte [] lk = r.getLastKey();
1789     KeyValue lastKV = KeyValue.createKeyValueFromKey(lk, 0, lk.length);
1790     KeyValue firstOnRow = state.getTargetKey();
1791     if (this.comparator.compareRows(lastKV, firstOnRow) < 0) {
1792       // If last key in file is not of the target table, no candidates in this
1793       // file.  Return.
1794       if (!state.isTargetTable(lastKV)) return false;
1795       // If the row we're looking for is past the end of file, set search key to
1796       // last key. TODO: Cache last and first key rather than make each time.
1797       firstOnRow = new KeyValue(lastKV.getRow(), HConstants.LATEST_TIMESTAMP);
1798     }
1799     // Get a scanner that caches blocks and that uses pread.
1800     HFileScanner scanner = r.getScanner(true, true, false);
1801     // Seek scanner.  If can't seek it, return.
1802     if (!seekToScanner(scanner, firstOnRow, firstKV)) return false;
1803     // If we found candidate on firstOnRow, just return. THIS WILL NEVER HAPPEN!
1804     // Unlikely that there'll be an instance of actual first row in table.
1805     if (walkForwardInSingleRow(scanner, firstOnRow, state)) return true;
1806     // If here, need to start backing up.
1807     while (scanner.seekBefore(firstOnRow.getBuffer(), firstOnRow.getKeyOffset(),
1808        firstOnRow.getKeyLength())) {
1809       KeyValue kv = scanner.getKeyValue();
1810       if (!state.isTargetTable(kv)) break;
1811       if (!state.isBetterCandidate(kv)) break;
1812       // Make new first on row.
1813       firstOnRow = new KeyValue(kv.getRow(), HConstants.LATEST_TIMESTAMP);
1814       // Seek scanner.  If can't seek it, break.
1815       if (!seekToScanner(scanner, firstOnRow, firstKV)) return false;
1816       // If we find something, break;
1817       if (walkForwardInSingleRow(scanner, firstOnRow, state)) return true;
1818     }
1819     return false;
1820   }
1821 
1822   /*
1823    * Seek the file scanner to firstOnRow or first entry in file.
1824    * @param scanner
1825    * @param firstOnRow
1826    * @param firstKV
1827    * @return True if we successfully seeked scanner.
1828    * @throws IOException
1829    */
1830   private boolean seekToScanner(final HFileScanner scanner,
1831                                 final KeyValue firstOnRow,
1832                                 final KeyValue firstKV)
1833       throws IOException {
1834     KeyValue kv = firstOnRow;
1835     // If firstOnRow < firstKV, set to firstKV
1836     if (this.comparator.compareRows(firstKV, firstOnRow) == 0) kv = firstKV;
1837     int result = scanner.seekTo(kv.getBuffer(), kv.getKeyOffset(),
1838       kv.getKeyLength());
1839     return result != -1;
1840   }
1841 
1842   /*
1843    * When we come in here, we are probably at the kv just before we break into
1844    * the row that firstOnRow is on.  Usually need to increment one time to get
1845    * on to the row we are interested in.
1846    * @param scanner
1847    * @param firstOnRow
1848    * @param state
1849    * @return True we found a candidate.
1850    * @throws IOException
1851    */
1852   private boolean walkForwardInSingleRow(final HFileScanner scanner,
1853                                          final KeyValue firstOnRow,
1854                                          final GetClosestRowBeforeTracker state)
1855       throws IOException {
1856     boolean foundCandidate = false;
1857     do {
1858       KeyValue kv = scanner.getKeyValue();
1859       // If we are not in the row, skip.
1860       if (this.comparator.compareRows(kv, firstOnRow) < 0) continue;
1861       // Did we go beyond the target row? If so break.
1862       if (state.isTooFar(kv, firstOnRow)) break;
1863       if (state.isExpired(kv)) {
1864         continue;
1865       }
1866       // If we added something, this row is a contender. break.
1867       if (state.handle(kv)) {
1868         foundCandidate = true;
1869         break;
1870       }
1871     } while(scanner.next());
1872     return foundCandidate;
1873   }
1874 
1875   @Override
1876   public boolean canSplit() {
1877     this.lock.readLock().lock();
1878     try {
1879       // Not split-able if we find a reference store file present in the store.
1880       boolean result = !hasReferences();
1881       if (!result && LOG.isDebugEnabled()) {
1882         LOG.debug("Cannot split region due to reference files being there");
1883       }
1884       return result;
1885     } finally {
1886       this.lock.readLock().unlock();
1887     }
1888   }
1889 
1890   @Override
1891   public byte[] getSplitPoint() {
1892     this.lock.readLock().lock();
1893     try {
1894       // Should already be enforced by the split policy!
1895       assert !this.getRegionInfo().isMetaRegion();
1896       // Not split-able if we find a reference store file present in the store.
1897       if (hasReferences()) {
1898         return null;
1899       }
1900       return this.storeEngine.getStoreFileManager().getSplitPoint();
1901     } catch(IOException e) {
1902       LOG.warn("Failed getting store size for " + this, e);
1903     } finally {
1904       this.lock.readLock().unlock();
1905     }
1906     return null;
1907   }
1908 
1909   @Override
1910   public long getLastCompactSize() {
1911     return this.lastCompactSize;
1912   }
1913 
1914   @Override
1915   public long getSize() {
1916     return storeSize;
1917   }
1918 
1919   @Override
1920   public void triggerMajorCompaction() {
1921     this.forceMajor = true;
1922   }
1923 
1924   boolean getForceMajorCompaction() {
1925     return this.forceMajor;
1926   }
1927 
1928   //////////////////////////////////////////////////////////////////////////////
1929   // File administration
1930   //////////////////////////////////////////////////////////////////////////////
1931 
1932   @Override
1933   public KeyValueScanner getScanner(Scan scan,
1934       final NavigableSet<byte []> targetCols, long readPt) throws IOException {
1935     lock.readLock().lock();
1936     try {
1937       KeyValueScanner scanner = null;
1938       if (this.getCoprocessorHost() != null) {
1939         scanner = this.getCoprocessorHost().preStoreScannerOpen(this, scan, targetCols);
1940       }
1941       if (scanner == null) {
1942         scanner = scan.isReversed() ? new ReversedStoreScanner(this,
1943             getScanInfo(), scan, targetCols, readPt) : new StoreScanner(this,
1944             getScanInfo(), scan, targetCols, readPt);
1945       }
1946       return scanner;
1947     } finally {
1948       lock.readLock().unlock();
1949     }
1950   }
1951 
1952   @Override
1953   public String toString() {
1954     return this.getColumnFamilyName();
1955   }
1956 
1957   @Override
1958   public int getStorefilesCount() {
1959     return this.storeEngine.getStoreFileManager().getStorefileCount();
1960   }
1961 
1962   @Override
1963   public long getStoreSizeUncompressed() {
1964     return this.totalUncompressedBytes;
1965   }
1966 
1967   @Override
1968   public long getStorefilesSize() {
1969     long size = 0;
1970     for (StoreFile s: this.storeEngine.getStoreFileManager().getStorefiles()) {
1971       StoreFile.Reader r = s.getReader();
1972       if (r == null) {
1973         LOG.warn("StoreFile " + s + " has a null Reader");
1974         continue;
1975       }
1976       size += r.length();
1977     }
1978     return size;
1979   }
1980 
1981   @Override
1982   public long getStorefilesIndexSize() {
1983     long size = 0;
1984     for (StoreFile s: this.storeEngine.getStoreFileManager().getStorefiles()) {
1985       StoreFile.Reader r = s.getReader();
1986       if (r == null) {
1987         LOG.warn("StoreFile " + s + " has a null Reader");
1988         continue;
1989       }
1990       size += r.indexSize();
1991     }
1992     return size;
1993   }
1994 
1995   @Override
1996   public long getTotalStaticIndexSize() {
1997     long size = 0;
1998     for (StoreFile s : this.storeEngine.getStoreFileManager().getStorefiles()) {
1999       size += s.getReader().getUncompressedDataIndexSize();
2000     }
2001     return size;
2002   }
2003 
2004   @Override
2005   public long getTotalStaticBloomSize() {
2006     long size = 0;
2007     for (StoreFile s : this.storeEngine.getStoreFileManager().getStorefiles()) {
2008       StoreFile.Reader r = s.getReader();
2009       size += r.getTotalBloomSize();
2010     }
2011     return size;
2012   }
2013 
2014   @Override
2015   public long getMemStoreSize() {
2016     return this.memstore.heapSize();
2017   }
2018 
2019   @Override
2020   public int getCompactPriority() {
2021     int priority = this.storeEngine.getStoreFileManager().getStoreCompactionPriority();
2022     if (priority == PRIORITY_USER) {
2023       LOG.warn("Compaction priority is USER despite there being no user compaction");
2024     }
2025     return priority;
2026   }
2027 
2028   @Override
2029   public boolean throttleCompaction(long compactionSize) {
2030     return storeEngine.getCompactionPolicy().throttleCompaction(compactionSize);
2031   }
2032 
2033   public HRegion getHRegion() {
2034     return this.region;
2035   }
2036 
2037   @Override
2038   public RegionCoprocessorHost getCoprocessorHost() {
2039     return this.region.getCoprocessorHost();
2040   }
2041 
2042   @Override
2043   public HRegionInfo getRegionInfo() {
2044     return this.fs.getRegionInfo();
2045   }
2046 
2047   @Override
2048   public boolean areWritesEnabled() {
2049     return this.region.areWritesEnabled();
2050   }
2051 
2052   @Override
2053   public long getSmallestReadPoint() {
2054     return this.region.getSmallestReadPoint();
2055   }
2056 
2057   /**
2058    * Used in tests. TODO: Remove
2059    *
2060    * Updates the value for the given row/family/qualifier. This function will always be seen as
2061    * atomic by other readers because it only puts a single KV to memstore. Thus no read/write
2062    * control necessary.
2063    * @param row row to update
2064    * @param f family to update
2065    * @param qualifier qualifier to update
2066    * @param newValue the new value to set into memstore
2067    * @return memstore size delta
2068    * @throws IOException
2069    */
2070   public long updateColumnValue(byte [] row, byte [] f,
2071                                 byte [] qualifier, long newValue)
2072       throws IOException {
2073 
2074     this.lock.readLock().lock();
2075     try {
2076       long now = EnvironmentEdgeManager.currentTimeMillis();
2077 
2078       return this.memstore.updateColumnValue(row,
2079           f,
2080           qualifier,
2081           newValue,
2082           now);
2083 
2084     } finally {
2085       this.lock.readLock().unlock();
2086     }
2087   }
2088 
2089   @Override
2090   public long upsert(Iterable<Cell> cells, long readpoint) throws IOException {
2091     this.lock.readLock().lock();
2092     try {
2093       return this.memstore.upsert(cells, readpoint);
2094     } finally {
2095       this.lock.readLock().unlock();
2096     }
2097   }
2098 
2099   @Override
2100   public StoreFlushContext createFlushContext(long cacheFlushId) {
2101     return new StoreFlusherImpl(cacheFlushId);
2102   }
2103 
2104   private final class StoreFlusherImpl implements StoreFlushContext {
2105 
2106     private long cacheFlushSeqNum;
2107     private SortedSet<KeyValue> snapshot;
2108     private List<Path> tempFiles;
2109     private List<Path> committedFiles;
2110     private TimeRangeTracker snapshotTimeRangeTracker;
2111     private long flushedCount;
2112     private final AtomicLong flushedSize = new AtomicLong();
2113 
2114     private StoreFlusherImpl(long cacheFlushSeqNum) {
2115       this.cacheFlushSeqNum = cacheFlushSeqNum;
2116     }
2117 
2118     /**
2119      * This is not thread safe. The caller should have a lock on the region or the store.
2120      * If necessary, the lock can be added with the patch provided in HBASE-10087
2121      */
2122     @Override
2123     public void prepare() {
2124       memstore.snapshot();
2125       this.snapshot = memstore.getSnapshot();
2126       this.snapshotTimeRangeTracker = memstore.getSnapshotTimeRangeTracker();
2127       this.flushedCount = this.snapshot.size();
2128       this.committedFiles = new ArrayList<Path>(1);
2129     }
2130 
2131     @Override
2132     public void flushCache(MonitoredTask status) throws IOException {
2133       tempFiles = HStore.this.flushCache(
2134         cacheFlushSeqNum, snapshot, snapshotTimeRangeTracker, flushedSize, status);
2135     }
2136 
2137     @Override
2138     public boolean commit(MonitoredTask status) throws IOException {
2139       if (this.tempFiles == null || this.tempFiles.isEmpty()) {
2140         return false;
2141       }
2142       List<StoreFile> storeFiles = new ArrayList<StoreFile>(this.tempFiles.size());
2143       for (Path storeFilePath : tempFiles) {
2144         try {
2145           storeFiles.add(HStore.this.commitFile(storeFilePath, cacheFlushSeqNum,
2146               snapshotTimeRangeTracker, flushedSize, status));
2147         } catch (IOException ex) {
2148           LOG.error("Failed to commit store file " + storeFilePath, ex);
2149           // Try to delete the files we have committed before.
2150           for (StoreFile sf : storeFiles) {
2151             Path pathToDelete = sf.getPath();
2152             try {
2153               sf.deleteReader();
2154             } catch (IOException deleteEx) {
2155               LOG.fatal("Failed to delete store file we committed, halting " + pathToDelete, ex);
2156               Runtime.getRuntime().halt(1);
2157             }
2158           }
2159           throw new IOException("Failed to commit the flush", ex);
2160         }
2161       }
2162 
2163       for (StoreFile sf : storeFiles) {
2164         if (HStore.this.getCoprocessorHost() != null) {
2165           HStore.this.getCoprocessorHost().postFlush(HStore.this, sf);
2166         }
2167         committedFiles.add(sf.getPath());
2168       }
2169 
2170       HStore.this.flushedCellsCount += flushedCount;
2171       HStore.this.flushedCellsSize += flushedSize.get();
2172 
2173       // Add new file to store files.  Clear snapshot too while we have the Store write lock.
2174       return HStore.this.updateStorefiles(storeFiles, snapshot);
2175     }
2176 
2177     @Override
2178     public List<Path> getCommittedFiles() {
2179       return this.committedFiles;
2180     }
2181   }
2182 
2183   @Override
2184   public boolean needsCompaction() {
2185     return this.storeEngine.needsCompaction(this.filesCompacting);
2186   }
2187 
2188   @Override
2189   public CacheConfig getCacheConfig() {
2190     return this.cacheConf;
2191   }
2192 
2193   public static final long FIXED_OVERHEAD =
2194       ClassSize.align(ClassSize.OBJECT + (16 * ClassSize.REFERENCE) + (10 * Bytes.SIZEOF_LONG)
2195               + (5 * Bytes.SIZEOF_INT) + (2 * Bytes.SIZEOF_BOOLEAN));
2196 
2197   public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD
2198       + ClassSize.OBJECT + ClassSize.REENTRANT_LOCK
2199       + ClassSize.CONCURRENT_SKIPLISTMAP
2200       + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + ClassSize.OBJECT
2201       + ScanInfo.FIXED_OVERHEAD);
2202 
2203   @Override
2204   public long heapSize() {
2205     return DEEP_OVERHEAD + this.memstore.heapSize();
2206   }
2207 
2208   @Override
2209   public KeyValue.KVComparator getComparator() {
2210     return comparator;
2211   }
2212 
2213   @Override
2214   public ScanInfo getScanInfo() {
2215     return scanInfo;
2216   }
2217 
2218   /**
2219    * Set scan info, used by test
2220    * @param scanInfo new scan info to use for test
2221    */
2222   void setScanInfo(ScanInfo scanInfo) {
2223     this.scanInfo = scanInfo;
2224   }
2225 
2226   @Override
2227   public boolean hasTooManyStoreFiles() {
2228     return getStorefilesCount() > this.blockingFileCount;
2229   }
2230 
2231   @Override
2232   public long getFlushedCellsCount() {
2233     return flushedCellsCount;
2234   }
2235 
2236   @Override
2237   public long getFlushedCellsSize() {
2238     return flushedCellsSize;
2239   }
2240 
2241   @Override
2242   public long getCompactedCellsCount() {
2243     return compactedCellsCount;
2244   }
2245 
2246   @Override
2247   public long getCompactedCellsSize() {
2248     return compactedCellsSize;
2249   }
2250 
2251   @Override
2252   public long getMajorCompactedCellsCount() {
2253     return majorCompactedCellsCount;
2254   }
2255 
2256   @Override
2257   public long getMajorCompactedCellsSize() {
2258     return majorCompactedCellsSize;
2259   }
2260 
2261   @Override
2262   public double getCompactionPressure() {
2263     return storeEngine.getStoreFileManager().getCompactionPressure();
2264   }
2265 }