View Javadoc

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