001/*
002 *
003 * Licensed to the Apache Software Foundation (ASF) under one
004 * or more contributor license agreements.  See the NOTICE file
005 * distributed with this work for additional information
006 * regarding copyright ownership.  The ASF licenses this file
007 * to you under the Apache License, Version 2.0 (the
008 * "License"); you may not use this file except in compliance
009 * with the License.  You may obtain a copy of the License at
010 *
011 *     http://www.apache.org/licenses/LICENSE-2.0
012 *
013 * Unless required by applicable law or agreed to in writing, software
014 * distributed under the License is distributed on an "AS IS" BASIS,
015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
016 * See the License for the specific language governing permissions and
017 * limitations under the License.
018 */
019package org.apache.hadoop.hbase.regionserver;
020
021import java.io.IOException;
022import java.io.InterruptedIOException;
023import java.net.InetSocketAddress;
024import java.util.ArrayList;
025import java.util.Collection;
026import java.util.Collections;
027import java.util.HashMap;
028import java.util.HashSet;
029import java.util.Iterator;
030import java.util.List;
031import java.util.Map;
032import java.util.NavigableSet;
033import java.util.Optional;
034import java.util.OptionalDouble;
035import java.util.OptionalLong;
036import java.util.Set;
037import java.util.concurrent.Callable;
038import java.util.concurrent.CompletionService;
039import java.util.concurrent.ConcurrentHashMap;
040import java.util.concurrent.ExecutionException;
041import java.util.concurrent.ExecutorCompletionService;
042import java.util.concurrent.Future;
043import java.util.concurrent.ThreadPoolExecutor;
044import java.util.concurrent.atomic.AtomicBoolean;
045import java.util.concurrent.atomic.AtomicInteger;
046import java.util.concurrent.atomic.AtomicLong;
047import java.util.concurrent.locks.ReentrantLock;
048import java.util.concurrent.locks.ReentrantReadWriteLock;
049import java.util.function.Predicate;
050import java.util.function.ToLongFunction;
051import java.util.stream.Collectors;
052import java.util.stream.LongStream;
053
054import org.apache.hadoop.conf.Configuration;
055import org.apache.hadoop.fs.FileSystem;
056import org.apache.hadoop.fs.Path;
057import org.apache.hadoop.fs.permission.FsAction;
058import org.apache.hadoop.hbase.Cell;
059import org.apache.hadoop.hbase.CellComparator;
060import org.apache.hadoop.hbase.CellUtil;
061import org.apache.hadoop.hbase.CompoundConfiguration;
062import org.apache.hadoop.hbase.HConstants;
063import org.apache.hadoop.hbase.MemoryCompactionPolicy;
064import org.apache.hadoop.hbase.TableName;
065import org.apache.hadoop.hbase.backup.FailedArchiveException;
066import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
067import org.apache.hadoop.hbase.client.RegionInfo;
068import org.apache.hadoop.hbase.client.Scan;
069import org.apache.hadoop.hbase.conf.ConfigurationManager;
070import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
071import org.apache.hadoop.hbase.io.HeapSize;
072import org.apache.hadoop.hbase.io.compress.Compression;
073import org.apache.hadoop.hbase.io.crypto.Encryption;
074import org.apache.hadoop.hbase.io.hfile.CacheConfig;
075import org.apache.hadoop.hbase.io.hfile.HFile;
076import org.apache.hadoop.hbase.io.hfile.HFileContext;
077import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
078import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
079import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoderImpl;
080import org.apache.hadoop.hbase.io.hfile.HFileScanner;
081import org.apache.hadoop.hbase.io.hfile.InvalidHFileException;
082import org.apache.hadoop.hbase.log.HBaseMarkers;
083import org.apache.hadoop.hbase.monitoring.MonitoredTask;
084import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
085import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
086import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
087import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl;
088import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
089import org.apache.hadoop.hbase.regionserver.compactions.OffPeakHours;
090import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher;
091import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
092import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
093import org.apache.hadoop.hbase.security.EncryptionUtil;
094import org.apache.hadoop.hbase.security.User;
095import org.apache.hadoop.hbase.util.Bytes;
096import org.apache.hadoop.hbase.util.ChecksumType;
097import org.apache.hadoop.hbase.util.ClassSize;
098import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
099import org.apache.hadoop.hbase.util.Pair;
100import org.apache.hadoop.hbase.util.ReflectionUtils;
101import org.apache.hadoop.util.StringUtils;
102import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
103import org.apache.yetus.audience.InterfaceAudience;
104import org.slf4j.Logger;
105import org.slf4j.LoggerFactory;
106import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
107import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
108import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableCollection;
109import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
110import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
111import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
112import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
113import org.apache.hbase.thirdparty.org.apache.commons.collections4.IterableUtils;
114import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
115import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
116
117/**
118 * A Store holds a column family in a Region.  Its a memstore and a set of zero
119 * or more StoreFiles, which stretch backwards over time.
120 *
121 * <p>There's no reason to consider append-logging at this level; all logging
122 * and locking is handled at the HRegion level.  Store just provides
123 * services to manage sets of StoreFiles.  One of the most important of those
124 * services is compaction services where files are aggregated once they pass
125 * a configurable threshold.
126 *
127 * <p>Locking and transactions are handled at a higher level.  This API should
128 * not be called directly but by an HRegion manager.
129 */
130@InterfaceAudience.Private
131public class HStore implements Store, HeapSize, StoreConfigInformation, PropagatingConfigurationObserver {
132  public static final String MEMSTORE_CLASS_NAME = "hbase.regionserver.memstore.class";
133  public static final String COMPACTCHECKER_INTERVAL_MULTIPLIER_KEY =
134      "hbase.server.compactchecker.interval.multiplier";
135  public static final String BLOCKING_STOREFILES_KEY = "hbase.hstore.blockingStoreFiles";
136  public static final String BLOCK_STORAGE_POLICY_KEY = "hbase.hstore.block.storage.policy";
137  // keep in accordance with HDFS default storage policy
138  public static final String DEFAULT_BLOCK_STORAGE_POLICY = "HOT";
139  public static final int DEFAULT_COMPACTCHECKER_INTERVAL_MULTIPLIER = 1000;
140  public static final int DEFAULT_BLOCKING_STOREFILE_COUNT = 16;
141
142  private static final Logger LOG = LoggerFactory.getLogger(HStore.class);
143
144  protected final MemStore memstore;
145  // This stores directory in the filesystem.
146  protected final HRegion region;
147  private final ColumnFamilyDescriptor family;
148  private final HRegionFileSystem fs;
149  protected Configuration conf;
150  protected CacheConfig cacheConf;
151  private long lastCompactSize = 0;
152  volatile boolean forceMajor = false;
153  /* how many bytes to write between status checks */
154  static int closeCheckInterval = 0;
155  private AtomicLong storeSize = new AtomicLong();
156  private AtomicLong totalUncompressedBytes = new AtomicLong();
157
158  /**
159   * RWLock for store operations.
160   * Locked in shared mode when the list of component stores is looked at:
161   *   - all reads/writes to table data
162   *   - checking for split
163   * Locked in exclusive mode when the list of component stores is modified:
164   *   - closing
165   *   - completing a compaction
166   */
167  final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
168  /**
169   * Lock specific to archiving compacted store files.  This avoids races around
170   * the combination of retrieving the list of compacted files and moving them to
171   * the archive directory.  Since this is usually a background process (other than
172   * on close), we don't want to handle this with the store write lock, which would
173   * block readers and degrade performance.
174   *
175   * Locked by:
176   *   - CompactedHFilesDispatchHandler via closeAndArchiveCompactedFiles()
177   *   - close()
178   */
179  final ReentrantLock archiveLock = new ReentrantLock();
180
181  private final boolean verifyBulkLoads;
182
183  /**
184   * Use this counter to track concurrent puts. If TRACE-log is enabled, if we are over the
185   * threshold set by hbase.region.store.parallel.put.print.threshold (Default is 50) we will
186   * log a message that identifies the Store experience this high-level of concurrency.
187   */
188  private final AtomicInteger currentParallelPutCount = new AtomicInteger(0);
189  private final int parallelPutCountPrintThreshold;
190
191  private ScanInfo scanInfo;
192
193  // All access must be synchronized.
194  // TODO: ideally, this should be part of storeFileManager, as we keep passing this to it.
195  private final List<HStoreFile> filesCompacting = Lists.newArrayList();
196
197  // All access must be synchronized.
198  private final Set<ChangedReadersObserver> changedReaderObservers =
199    Collections.newSetFromMap(new ConcurrentHashMap<ChangedReadersObserver, Boolean>());
200
201  protected final int blocksize;
202  private HFileDataBlockEncoder dataBlockEncoder;
203
204  /** Checksum configuration */
205  protected ChecksumType checksumType;
206  protected int bytesPerChecksum;
207
208  // Comparing KeyValues
209  protected final CellComparator comparator;
210
211  final StoreEngine<?, ?, ?, ?> storeEngine;
212
213  private static final AtomicBoolean offPeakCompactionTracker = new AtomicBoolean();
214  private volatile OffPeakHours offPeakHours;
215
216  private static final int DEFAULT_FLUSH_RETRIES_NUMBER = 10;
217  private int flushRetriesNumber;
218  private int pauseTime;
219
220  private long blockingFileCount;
221  private int compactionCheckMultiplier;
222  protected Encryption.Context cryptoContext = Encryption.Context.NONE;
223
224  private AtomicLong flushedCellsCount = new AtomicLong();
225  private AtomicLong compactedCellsCount = new AtomicLong();
226  private AtomicLong majorCompactedCellsCount = new AtomicLong();
227  private AtomicLong flushedCellsSize = new AtomicLong();
228  private AtomicLong flushedOutputFileSize = new AtomicLong();
229  private AtomicLong compactedCellsSize = new AtomicLong();
230  private AtomicLong majorCompactedCellsSize = new AtomicLong();
231
232  /**
233   * Constructor
234   * @param region
235   * @param family HColumnDescriptor for this column
236   * @param confParam configuration object
237   * failed.  Can be null.
238   * @throws IOException
239   */
240  protected HStore(final HRegion region, final ColumnFamilyDescriptor family,
241      final Configuration confParam) throws IOException {
242
243    this.fs = region.getRegionFileSystem();
244
245    // Assemble the store's home directory and Ensure it exists.
246    fs.createStoreDir(family.getNameAsString());
247    this.region = region;
248    this.family = family;
249    // 'conf' renamed to 'confParam' b/c we use this.conf in the constructor
250    // CompoundConfiguration will look for keys in reverse order of addition, so we'd
251    // add global config first, then table and cf overrides, then cf metadata.
252    this.conf = new CompoundConfiguration()
253      .add(confParam)
254      .addBytesMap(region.getTableDescriptor().getValues())
255      .addStringMap(family.getConfiguration())
256      .addBytesMap(family.getValues());
257    this.blocksize = family.getBlocksize();
258
259    // set block storage policy for store directory
260    String policyName = family.getStoragePolicy();
261    if (null == policyName) {
262      policyName = this.conf.get(BLOCK_STORAGE_POLICY_KEY, DEFAULT_BLOCK_STORAGE_POLICY);
263    }
264    this.fs.setStoragePolicy(family.getNameAsString(), policyName.trim());
265
266    this.dataBlockEncoder = new HFileDataBlockEncoderImpl(family.getDataBlockEncoding());
267
268    this.comparator = region.getCellComparator();
269    // used by ScanQueryMatcher
270    long timeToPurgeDeletes =
271        Math.max(conf.getLong("hbase.hstore.time.to.purge.deletes", 0), 0);
272    LOG.trace("Time to purge deletes set to {}ms in store {}", timeToPurgeDeletes, this);
273    // Get TTL
274    long ttl = determineTTLFromFamily(family);
275    // Why not just pass a HColumnDescriptor in here altogether?  Even if have
276    // to clone it?
277    scanInfo = new ScanInfo(conf, family, ttl, timeToPurgeDeletes, this.comparator);
278    this.memstore = getMemstore();
279
280    this.offPeakHours = OffPeakHours.getInstance(conf);
281
282    // Setting up cache configuration for this family
283    createCacheConf(family);
284
285    this.verifyBulkLoads = conf.getBoolean("hbase.hstore.bulkload.verify", false);
286
287    this.blockingFileCount =
288        conf.getInt(BLOCKING_STOREFILES_KEY, DEFAULT_BLOCKING_STOREFILE_COUNT);
289    this.compactionCheckMultiplier = conf.getInt(
290        COMPACTCHECKER_INTERVAL_MULTIPLIER_KEY, DEFAULT_COMPACTCHECKER_INTERVAL_MULTIPLIER);
291    if (this.compactionCheckMultiplier <= 0) {
292      LOG.error("Compaction check period multiplier must be positive, setting default: {}",
293          DEFAULT_COMPACTCHECKER_INTERVAL_MULTIPLIER);
294      this.compactionCheckMultiplier = DEFAULT_COMPACTCHECKER_INTERVAL_MULTIPLIER;
295    }
296
297    if (HStore.closeCheckInterval == 0) {
298      HStore.closeCheckInterval = conf.getInt(
299          "hbase.hstore.close.check.interval", 10*1000*1000 /* 10 MB */);
300    }
301
302    this.storeEngine = createStoreEngine(this, this.conf, this.comparator);
303    List<HStoreFile> hStoreFiles = loadStoreFiles();
304    // Move the storeSize calculation out of loadStoreFiles() method, because the secondary read
305    // replica's refreshStoreFiles() will also use loadStoreFiles() to refresh its store files and
306    // update the storeSize in the completeCompaction(..) finally (just like compaction) , so
307    // no need calculate the storeSize twice.
308    this.storeSize.addAndGet(getStorefilesSize(hStoreFiles, sf -> true));
309    this.totalUncompressedBytes.addAndGet(getTotalUmcompressedBytes(hStoreFiles));
310    this.storeEngine.getStoreFileManager().loadFiles(hStoreFiles);
311
312    // Initialize checksum type from name. The names are CRC32, CRC32C, etc.
313    this.checksumType = getChecksumType(conf);
314    // Initialize bytes per checksum
315    this.bytesPerChecksum = getBytesPerChecksum(conf);
316    flushRetriesNumber = conf.getInt(
317        "hbase.hstore.flush.retries.number", DEFAULT_FLUSH_RETRIES_NUMBER);
318    pauseTime = conf.getInt(HConstants.HBASE_SERVER_PAUSE, HConstants.DEFAULT_HBASE_SERVER_PAUSE);
319    if (flushRetriesNumber <= 0) {
320      throw new IllegalArgumentException(
321          "hbase.hstore.flush.retries.number must be > 0, not "
322              + flushRetriesNumber);
323    }
324    cryptoContext = EncryptionUtil.createEncryptionContext(conf, family);
325
326    int confPrintThreshold =
327        this.conf.getInt("hbase.region.store.parallel.put.print.threshold", 50);
328    if (confPrintThreshold < 10) {
329      confPrintThreshold = 10;
330    }
331    this.parallelPutCountPrintThreshold = confPrintThreshold;
332    LOG.info("Store={},  memstore type={}, storagePolicy={}, verifyBulkLoads={}, "
333            + "parallelPutCountPrintThreshold={}, encoding={}, compression={}",
334        getColumnFamilyName(), memstore.getClass().getSimpleName(), policyName, verifyBulkLoads,
335        parallelPutCountPrintThreshold, family.getDataBlockEncoding(),
336        family.getCompressionType());
337  }
338
339  /**
340   * @return MemStore Instance to use in this store.
341   */
342  private MemStore getMemstore() {
343    MemStore ms = null;
344    // Check if in-memory-compaction configured. Note MemoryCompactionPolicy is an enum!
345    MemoryCompactionPolicy inMemoryCompaction = null;
346    if (this.getTableName().isSystemTable()) {
347      inMemoryCompaction = MemoryCompactionPolicy.valueOf(
348          conf.get("hbase.systemtables.compacting.memstore.type", "NONE"));
349    } else {
350      inMemoryCompaction = family.getInMemoryCompaction();
351    }
352    if (inMemoryCompaction == null) {
353      inMemoryCompaction =
354          MemoryCompactionPolicy.valueOf(conf.get(CompactingMemStore.COMPACTING_MEMSTORE_TYPE_KEY,
355              CompactingMemStore.COMPACTING_MEMSTORE_TYPE_DEFAULT));
356    }
357    switch (inMemoryCompaction) {
358      case NONE:
359        ms = ReflectionUtils.newInstance(DefaultMemStore.class,
360            new Object[] { conf, this.comparator,
361                this.getHRegion().getRegionServicesForStores()});
362        break;
363      default:
364        Class<? extends CompactingMemStore> clz = conf.getClass(MEMSTORE_CLASS_NAME,
365            CompactingMemStore.class, CompactingMemStore.class);
366        ms = ReflectionUtils.newInstance(clz, new Object[]{conf, this.comparator, this,
367            this.getHRegion().getRegionServicesForStores(), inMemoryCompaction});
368    }
369    return ms;
370  }
371
372  /**
373   * Creates the cache config.
374   * @param family The current column family.
375   */
376  protected void createCacheConf(final ColumnFamilyDescriptor family) {
377    this.cacheConf = new CacheConfig(conf, family);
378  }
379
380  /**
381   * Creates the store engine configured for the given Store.
382   * @param store The store. An unfortunate dependency needed due to it
383   *              being passed to coprocessors via the compactor.
384   * @param conf Store configuration.
385   * @param kvComparator KVComparator for storeFileManager.
386   * @return StoreEngine to use.
387   */
388  protected StoreEngine<?, ?, ?, ?> createStoreEngine(HStore store, Configuration conf,
389      CellComparator kvComparator) throws IOException {
390    return StoreEngine.create(store, conf, comparator);
391  }
392
393  /**
394   * @param family
395   * @return TTL in seconds of the specified family
396   */
397  public static long determineTTLFromFamily(final ColumnFamilyDescriptor family) {
398    // HCD.getTimeToLive returns ttl in seconds.  Convert to milliseconds.
399    long ttl = family.getTimeToLive();
400    if (ttl == HConstants.FOREVER) {
401      // Default is unlimited ttl.
402      ttl = Long.MAX_VALUE;
403    } else if (ttl == -1) {
404      ttl = Long.MAX_VALUE;
405    } else {
406      // Second -> ms adjust for user data
407      ttl *= 1000;
408    }
409    return ttl;
410  }
411
412  @Override
413  public String getColumnFamilyName() {
414    return this.family.getNameAsString();
415  }
416
417  @Override
418  public TableName getTableName() {
419    return this.getRegionInfo().getTable();
420  }
421
422  @Override
423  public FileSystem getFileSystem() {
424    return this.fs.getFileSystem();
425  }
426
427  public HRegionFileSystem getRegionFileSystem() {
428    return this.fs;
429  }
430
431  /* Implementation of StoreConfigInformation */
432  @Override
433  public long getStoreFileTtl() {
434    // TTL only applies if there's no MIN_VERSIONs setting on the column.
435    return (this.scanInfo.getMinVersions() == 0) ? this.scanInfo.getTtl() : Long.MAX_VALUE;
436  }
437
438  @Override
439  public long getMemStoreFlushSize() {
440    // TODO: Why is this in here?  The flushsize of the region rather than the store?  St.Ack
441    return this.region.memstoreFlushSize;
442  }
443
444  @Override
445  public MemStoreSize getFlushableSize() {
446    return this.memstore.getFlushableSize();
447  }
448
449  @Override
450  public MemStoreSize getSnapshotSize() {
451    return this.memstore.getSnapshotSize();
452  }
453
454  @Override
455  public long getCompactionCheckMultiplier() {
456    return this.compactionCheckMultiplier;
457  }
458
459  @Override
460  public long getBlockingFileCount() {
461    return blockingFileCount;
462  }
463  /* End implementation of StoreConfigInformation */
464
465  /**
466   * Returns the configured bytesPerChecksum value.
467   * @param conf The configuration
468   * @return The bytesPerChecksum that is set in the configuration
469   */
470  public static int getBytesPerChecksum(Configuration conf) {
471    return conf.getInt(HConstants.BYTES_PER_CHECKSUM,
472                       HFile.DEFAULT_BYTES_PER_CHECKSUM);
473  }
474
475  /**
476   * Returns the configured checksum algorithm.
477   * @param conf The configuration
478   * @return The checksum algorithm that is set in the configuration
479   */
480  public static ChecksumType getChecksumType(Configuration conf) {
481    String checksumName = conf.get(HConstants.CHECKSUM_TYPE_NAME);
482    if (checksumName == null) {
483      return ChecksumType.getDefaultChecksumType();
484    } else {
485      return ChecksumType.nameToType(checksumName);
486    }
487  }
488
489  /**
490   * @return how many bytes to write between status checks
491   */
492  public static int getCloseCheckInterval() {
493    return closeCheckInterval;
494  }
495
496  @Override
497  public ColumnFamilyDescriptor getColumnFamilyDescriptor() {
498    return this.family;
499  }
500
501  @Override
502  public OptionalLong getMaxSequenceId() {
503    return StoreUtils.getMaxSequenceIdInList(this.getStorefiles());
504  }
505
506  @Override
507  public OptionalLong getMaxMemStoreTS() {
508    return StoreUtils.getMaxMemStoreTSInList(this.getStorefiles());
509  }
510
511  /**
512   * @param tabledir {@link Path} to where the table is being stored
513   * @param hri {@link RegionInfo} for the region.
514   * @param family {@link ColumnFamilyDescriptor} describing the column family
515   * @return Path to family/Store home directory.
516   */
517  @Deprecated
518  public static Path getStoreHomedir(final Path tabledir,
519      final RegionInfo hri, final byte[] family) {
520    return getStoreHomedir(tabledir, hri.getEncodedName(), family);
521  }
522
523  /**
524   * @param tabledir {@link Path} to where the table is being stored
525   * @param encodedName Encoded region name.
526   * @param family {@link ColumnFamilyDescriptor} describing the column family
527   * @return Path to family/Store home directory.
528   */
529  @Deprecated
530  public static Path getStoreHomedir(final Path tabledir,
531      final String encodedName, final byte[] family) {
532    return new Path(tabledir, new Path(encodedName, Bytes.toString(family)));
533  }
534
535  /**
536   * @return the data block encoder
537   */
538  public HFileDataBlockEncoder getDataBlockEncoder() {
539    return dataBlockEncoder;
540  }
541
542  /**
543   * Should be used only in tests.
544   * @param blockEncoder the block delta encoder to use
545   */
546  void setDataBlockEncoderInTest(HFileDataBlockEncoder blockEncoder) {
547    this.dataBlockEncoder = blockEncoder;
548  }
549
550  /**
551   * Creates an unsorted list of StoreFile loaded in parallel
552   * from the given directory.
553   * @throws IOException
554   */
555  private List<HStoreFile> loadStoreFiles() throws IOException {
556    Collection<StoreFileInfo> files = fs.getStoreFiles(getColumnFamilyName());
557    return openStoreFiles(files);
558  }
559
560  private List<HStoreFile> openStoreFiles(Collection<StoreFileInfo> files) throws IOException {
561    if (CollectionUtils.isEmpty(files)) {
562      return Collections.emptyList();
563    }
564    // initialize the thread pool for opening store files in parallel..
565    ThreadPoolExecutor storeFileOpenerThreadPool =
566      this.region.getStoreFileOpenAndCloseThreadPool("StoreFileOpenerThread-" +
567          this.getColumnFamilyName());
568    CompletionService<HStoreFile> completionService = new ExecutorCompletionService<>(storeFileOpenerThreadPool);
569
570    int totalValidStoreFile = 0;
571    for (StoreFileInfo storeFileInfo : files) {
572      // open each store file in parallel
573      completionService.submit(() -> this.createStoreFileAndReader(storeFileInfo));
574      totalValidStoreFile++;
575    }
576
577    ArrayList<HStoreFile> results = new ArrayList<>(files.size());
578    IOException ioe = null;
579    try {
580      for (int i = 0; i < totalValidStoreFile; i++) {
581        try {
582          HStoreFile storeFile = completionService.take().get();
583          if (storeFile != null) {
584            LOG.debug("loaded {}", storeFile);
585            results.add(storeFile);
586          }
587        } catch (InterruptedException e) {
588          if (ioe == null) ioe = new InterruptedIOException(e.getMessage());
589        } catch (ExecutionException e) {
590          if (ioe == null) ioe = new IOException(e.getCause());
591        }
592      }
593    } finally {
594      storeFileOpenerThreadPool.shutdownNow();
595    }
596    if (ioe != null) {
597      // close StoreFile readers
598      boolean evictOnClose =
599          cacheConf != null? cacheConf.shouldEvictOnClose(): true;
600      for (HStoreFile file : results) {
601        try {
602          if (file != null) {
603            file.closeStoreFile(evictOnClose);
604          }
605        } catch (IOException e) {
606          LOG.warn("Could not close store file", e);
607        }
608      }
609      throw ioe;
610    }
611
612    return results;
613  }
614
615  @Override
616  public void refreshStoreFiles() throws IOException {
617    Collection<StoreFileInfo> newFiles = fs.getStoreFiles(getColumnFamilyName());
618    refreshStoreFilesInternal(newFiles);
619  }
620
621  /**
622   * Replaces the store files that the store has with the given files. Mainly used by secondary
623   * region replicas to keep up to date with the primary region files.
624   * @throws IOException
625   */
626  public void refreshStoreFiles(Collection<String> newFiles) throws IOException {
627    List<StoreFileInfo> storeFiles = new ArrayList<>(newFiles.size());
628    for (String file : newFiles) {
629      storeFiles.add(fs.getStoreFileInfo(getColumnFamilyName(), file));
630    }
631    refreshStoreFilesInternal(storeFiles);
632  }
633
634  /**
635   * Checks the underlying store files, and opens the files that  have not
636   * been opened, and removes the store file readers for store files no longer
637   * available. Mainly used by secondary region replicas to keep up to date with
638   * the primary region files.
639   * @throws IOException
640   */
641  private void refreshStoreFilesInternal(Collection<StoreFileInfo> newFiles) throws IOException {
642    StoreFileManager sfm = storeEngine.getStoreFileManager();
643    Collection<HStoreFile> currentFiles = sfm.getStorefiles();
644    Collection<HStoreFile> compactedFiles = sfm.getCompactedfiles();
645    if (currentFiles == null) currentFiles = Collections.emptySet();
646    if (newFiles == null) newFiles = Collections.emptySet();
647    if (compactedFiles == null) compactedFiles = Collections.emptySet();
648
649    HashMap<StoreFileInfo, HStoreFile> currentFilesSet = new HashMap<>(currentFiles.size());
650    for (HStoreFile sf : currentFiles) {
651      currentFilesSet.put(sf.getFileInfo(), sf);
652    }
653    HashMap<StoreFileInfo, HStoreFile> compactedFilesSet = new HashMap<>(compactedFiles.size());
654    for (HStoreFile sf : compactedFiles) {
655      compactedFilesSet.put(sf.getFileInfo(), sf);
656    }
657
658    Set<StoreFileInfo> newFilesSet = new HashSet<StoreFileInfo>(newFiles);
659    // Exclude the files that have already been compacted
660    newFilesSet = Sets.difference(newFilesSet, compactedFilesSet.keySet());
661    Set<StoreFileInfo> toBeAddedFiles = Sets.difference(newFilesSet, currentFilesSet.keySet());
662    Set<StoreFileInfo> toBeRemovedFiles = Sets.difference(currentFilesSet.keySet(), newFilesSet);
663
664    if (toBeAddedFiles.isEmpty() && toBeRemovedFiles.isEmpty()) {
665      return;
666    }
667
668    LOG.info("Refreshing store files for region " + this.getRegionInfo().getRegionNameAsString()
669      + " files to add: " + toBeAddedFiles + " files to remove: " + toBeRemovedFiles);
670
671    Set<HStoreFile> toBeRemovedStoreFiles = new HashSet<>(toBeRemovedFiles.size());
672    for (StoreFileInfo sfi : toBeRemovedFiles) {
673      toBeRemovedStoreFiles.add(currentFilesSet.get(sfi));
674    }
675
676    // try to open the files
677    List<HStoreFile> openedFiles = openStoreFiles(toBeAddedFiles);
678
679    // propogate the file changes to the underlying store file manager
680    replaceStoreFiles(toBeRemovedStoreFiles, openedFiles); //won't throw an exception
681
682    // Advance the memstore read point to be at least the new store files seqIds so that
683    // readers might pick it up. This assumes that the store is not getting any writes (otherwise
684    // in-flight transactions might be made visible)
685    if (!toBeAddedFiles.isEmpty()) {
686      // we must have the max sequence id here as we do have several store files
687      region.getMVCC().advanceTo(this.getMaxSequenceId().getAsLong());
688    }
689
690    completeCompaction(toBeRemovedStoreFiles);
691  }
692
693  @VisibleForTesting
694  protected HStoreFile createStoreFileAndReader(final Path p) throws IOException {
695    StoreFileInfo info = new StoreFileInfo(conf, this.getFileSystem(), p);
696    return createStoreFileAndReader(info);
697  }
698
699  private HStoreFile createStoreFileAndReader(StoreFileInfo info) throws IOException {
700    info.setRegionCoprocessorHost(this.region.getCoprocessorHost());
701    HStoreFile storeFile = new HStoreFile(this.getFileSystem(), info, this.conf, this.cacheConf,
702        this.family.getBloomFilterType(), isPrimaryReplicaStore());
703    storeFile.initReader();
704    return storeFile;
705  }
706
707  /**
708   * This message intends to inform the MemStore that next coming updates
709   * are going to be part of the replaying edits from WAL
710   */
711  public void startReplayingFromWAL(){
712    this.memstore.startReplayingFromWAL();
713  }
714
715  /**
716   * This message intends to inform the MemStore that the replaying edits from WAL
717   * are done
718   */
719  public void stopReplayingFromWAL(){
720    this.memstore.stopReplayingFromWAL();
721  }
722
723  /**
724   * Adds a value to the memstore
725   */
726  public void add(final Cell cell, MemStoreSizing memstoreSizing) {
727    lock.readLock().lock();
728    try {
729      if (this.currentParallelPutCount.getAndIncrement() > this.parallelPutCountPrintThreshold) {
730        LOG.trace(this.getTableName() + "tableName={}, encodedName={}, columnFamilyName={} is " +
731          "too busy!", this.getRegionInfo().getEncodedName(), this .getColumnFamilyName());
732      }
733      this.memstore.add(cell, memstoreSizing);
734    } finally {
735      lock.readLock().unlock();
736      currentParallelPutCount.decrementAndGet();
737    }
738  }
739
740  /**
741   * Adds the specified value to the memstore
742   */
743  public void add(final Iterable<Cell> cells, MemStoreSizing memstoreSizing) {
744    lock.readLock().lock();
745    try {
746      if (this.currentParallelPutCount.getAndIncrement() > this.parallelPutCountPrintThreshold) {
747        LOG.trace(this.getTableName() + "tableName={}, encodedName={}, columnFamilyName={} is " +
748            "too busy!", this.getRegionInfo().getEncodedName(), this .getColumnFamilyName());
749      }
750      memstore.add(cells, memstoreSizing);
751    } finally {
752      lock.readLock().unlock();
753      currentParallelPutCount.decrementAndGet();
754    }
755  }
756
757  @Override
758  public long timeOfOldestEdit() {
759    return memstore.timeOfOldestEdit();
760  }
761
762  /**
763   * @return All store files.
764   */
765  @Override
766  public Collection<HStoreFile> getStorefiles() {
767    return this.storeEngine.getStoreFileManager().getStorefiles();
768  }
769
770  @Override
771  public Collection<HStoreFile> getCompactedFiles() {
772    return this.storeEngine.getStoreFileManager().getCompactedfiles();
773  }
774
775  /**
776   * This throws a WrongRegionException if the HFile does not fit in this region, or an
777   * InvalidHFileException if the HFile is not valid.
778   */
779  public void assertBulkLoadHFileOk(Path srcPath) throws IOException {
780    HFile.Reader reader  = null;
781    try {
782      LOG.info("Validating hfile at " + srcPath + " for inclusion in "
783          + "store " + this + " region " + this.getRegionInfo().getRegionNameAsString());
784      FileSystem srcFs = srcPath.getFileSystem(conf);
785      srcFs.access(srcPath, FsAction.READ_WRITE);
786      reader = HFile.createReader(srcFs, srcPath, cacheConf, isPrimaryReplicaStore(), conf);
787      reader.loadFileInfo();
788
789      Optional<byte[]> firstKey = reader.getFirstRowKey();
790      Preconditions.checkState(firstKey.isPresent(), "First key can not be null");
791      Optional<Cell> lk = reader.getLastKey();
792      Preconditions.checkState(lk.isPresent(), "Last key can not be null");
793      byte[] lastKey =  CellUtil.cloneRow(lk.get());
794
795      if (LOG.isDebugEnabled()) {
796        LOG.debug("HFile bounds: first=" + Bytes.toStringBinary(firstKey.get()) +
797            " last=" + Bytes.toStringBinary(lastKey));
798        LOG.debug("Region bounds: first=" +
799            Bytes.toStringBinary(getRegionInfo().getStartKey()) +
800            " last=" + Bytes.toStringBinary(getRegionInfo().getEndKey()));
801      }
802
803      if (!this.getRegionInfo().containsRange(firstKey.get(), lastKey)) {
804        throw new WrongRegionException(
805            "Bulk load file " + srcPath.toString() + " does not fit inside region "
806            + this.getRegionInfo().getRegionNameAsString());
807      }
808
809      if(reader.length() > conf.getLong(HConstants.HREGION_MAX_FILESIZE,
810          HConstants.DEFAULT_MAX_FILE_SIZE)) {
811        LOG.warn("Trying to bulk load hfile " + srcPath + " with size: " +
812            reader.length() + " bytes can be problematic as it may lead to oversplitting.");
813      }
814
815      if (verifyBulkLoads) {
816        long verificationStartTime = EnvironmentEdgeManager.currentTime();
817        LOG.info("Full verification started for bulk load hfile: {}", srcPath);
818        Cell prevCell = null;
819        HFileScanner scanner = reader.getScanner(false, false, false);
820        scanner.seekTo();
821        do {
822          Cell cell = scanner.getCell();
823          if (prevCell != null) {
824            if (comparator.compareRows(prevCell, cell) > 0) {
825              throw new InvalidHFileException("Previous row is greater than"
826                  + " current row: path=" + srcPath + " previous="
827                  + CellUtil.getCellKeyAsString(prevCell) + " current="
828                  + CellUtil.getCellKeyAsString(cell));
829            }
830            if (CellComparator.getInstance().compareFamilies(prevCell, cell) != 0) {
831              throw new InvalidHFileException("Previous key had different"
832                  + " family compared to current key: path=" + srcPath
833                  + " previous="
834                  + Bytes.toStringBinary(prevCell.getFamilyArray(), prevCell.getFamilyOffset(),
835                      prevCell.getFamilyLength())
836                  + " current="
837                  + Bytes.toStringBinary(cell.getFamilyArray(), cell.getFamilyOffset(),
838                      cell.getFamilyLength()));
839            }
840          }
841          prevCell = cell;
842        } while (scanner.next());
843      LOG.info("Full verification complete for bulk load hfile: " + srcPath.toString()
844         + " took " + (EnvironmentEdgeManager.currentTime() - verificationStartTime)
845         + " ms");
846      }
847    } finally {
848      if (reader != null) reader.close();
849    }
850  }
851
852  /**
853   * This method should only be called from Region. It is assumed that the ranges of values in the
854   * HFile fit within the stores assigned region. (assertBulkLoadHFileOk checks this)
855   *
856   * @param srcPathStr
857   * @param seqNum sequence Id associated with the HFile
858   */
859  public Pair<Path, Path> preBulkLoadHFile(String srcPathStr, long seqNum) throws IOException {
860    Path srcPath = new Path(srcPathStr);
861    return fs.bulkLoadStoreFile(getColumnFamilyName(), srcPath, seqNum);
862  }
863
864  public Path bulkLoadHFile(byte[] family, String srcPathStr, Path dstPath) throws IOException {
865    Path srcPath = new Path(srcPathStr);
866    try {
867      fs.commitStoreFile(srcPath, dstPath);
868    } finally {
869      if (this.getCoprocessorHost() != null) {
870        this.getCoprocessorHost().postCommitStoreFile(family, srcPath, dstPath);
871      }
872    }
873
874    LOG.info("Loaded HFile " + srcPath + " into store '" + getColumnFamilyName() + "' as "
875        + dstPath + " - updating store file list.");
876
877    HStoreFile sf = createStoreFileAndReader(dstPath);
878    bulkLoadHFile(sf);
879
880    LOG.info("Successfully loaded store file {} into store {} (new location: {})",
881        srcPath, this, dstPath);
882
883    return dstPath;
884  }
885
886  public void bulkLoadHFile(StoreFileInfo fileInfo) throws IOException {
887    HStoreFile sf = createStoreFileAndReader(fileInfo);
888    bulkLoadHFile(sf);
889  }
890
891  private void bulkLoadHFile(HStoreFile sf) throws IOException {
892    StoreFileReader r = sf.getReader();
893    this.storeSize.addAndGet(r.length());
894    this.totalUncompressedBytes.addAndGet(r.getTotalUncompressedBytes());
895
896    // Append the new storefile into the list
897    this.lock.writeLock().lock();
898    try {
899      this.storeEngine.getStoreFileManager().insertNewFiles(Lists.newArrayList(sf));
900    } finally {
901      // We need the lock, as long as we are updating the storeFiles
902      // or changing the memstore. Let us release it before calling
903      // notifyChangeReadersObservers. See HBASE-4485 for a possible
904      // deadlock scenario that could have happened if continue to hold
905      // the lock.
906      this.lock.writeLock().unlock();
907    }
908    LOG.info("Loaded HFile " + sf.getFileInfo() + " into store '" + getColumnFamilyName());
909    if (LOG.isTraceEnabled()) {
910      String traceMessage = "BULK LOAD time,size,store size,store files ["
911          + EnvironmentEdgeManager.currentTime() + "," + r.length() + "," + storeSize
912          + "," + storeEngine.getStoreFileManager().getStorefileCount() + "]";
913      LOG.trace(traceMessage);
914    }
915  }
916
917  /**
918   * Close all the readers We don't need to worry about subsequent requests because the Region holds
919   * a write lock that will prevent any more reads or writes.
920   * @return the {@link StoreFile StoreFiles} that were previously being used.
921   * @throws IOException on failure
922   */
923  public ImmutableCollection<HStoreFile> close() throws IOException {
924    this.archiveLock.lock();
925    this.lock.writeLock().lock();
926    try {
927      // Clear so metrics doesn't find them.
928      ImmutableCollection<HStoreFile> result = storeEngine.getStoreFileManager().clearFiles();
929      Collection<HStoreFile> compactedfiles =
930          storeEngine.getStoreFileManager().clearCompactedFiles();
931      // clear the compacted files
932      if (CollectionUtils.isNotEmpty(compactedfiles)) {
933        removeCompactedfiles(compactedfiles, true);
934      }
935      if (!result.isEmpty()) {
936        // initialize the thread pool for closing store files in parallel.
937        ThreadPoolExecutor storeFileCloserThreadPool = this.region
938            .getStoreFileOpenAndCloseThreadPool("StoreFileCloserThread-"
939                + this.getColumnFamilyName());
940
941        // close each store file in parallel
942        CompletionService<Void> completionService =
943          new ExecutorCompletionService<>(storeFileCloserThreadPool);
944        for (HStoreFile f : result) {
945          completionService.submit(new Callable<Void>() {
946            @Override
947            public Void call() throws IOException {
948              boolean evictOnClose =
949                  cacheConf != null? cacheConf.shouldEvictOnClose(): true;
950              f.closeStoreFile(evictOnClose);
951              return null;
952            }
953          });
954        }
955
956        IOException ioe = null;
957        try {
958          for (int i = 0; i < result.size(); i++) {
959            try {
960              Future<Void> future = completionService.take();
961              future.get();
962            } catch (InterruptedException e) {
963              if (ioe == null) {
964                ioe = new InterruptedIOException();
965                ioe.initCause(e);
966              }
967            } catch (ExecutionException e) {
968              if (ioe == null) ioe = new IOException(e.getCause());
969            }
970          }
971        } finally {
972          storeFileCloserThreadPool.shutdownNow();
973        }
974        if (ioe != null) throw ioe;
975      }
976      LOG.trace("Closed {}", this);
977      return result;
978    } finally {
979      this.lock.writeLock().unlock();
980      this.archiveLock.unlock();
981    }
982  }
983
984  /**
985   * Snapshot this stores memstore. Call before running
986   * {@link #flushCache(long, MemStoreSnapshot, MonitoredTask, ThroughputController,
987   * FlushLifeCycleTracker)}
988   *  so it has some work to do.
989   */
990  void snapshot() {
991    this.lock.writeLock().lock();
992    try {
993      this.memstore.snapshot();
994    } finally {
995      this.lock.writeLock().unlock();
996    }
997  }
998
999  /**
1000   * Write out current snapshot. Presumes {@link #snapshot()} has been called previously.
1001   * @param logCacheFlushId flush sequence number
1002   * @param snapshot
1003   * @param status
1004   * @param throughputController
1005   * @return The path name of the tmp file to which the store was flushed
1006   * @throws IOException if exception occurs during process
1007   */
1008  protected List<Path> flushCache(final long logCacheFlushId, MemStoreSnapshot snapshot,
1009      MonitoredTask status, ThroughputController throughputController,
1010      FlushLifeCycleTracker tracker) throws IOException {
1011    // If an exception happens flushing, we let it out without clearing
1012    // the memstore snapshot.  The old snapshot will be returned when we say
1013    // 'snapshot', the next time flush comes around.
1014    // Retry after catching exception when flushing, otherwise server will abort
1015    // itself
1016    StoreFlusher flusher = storeEngine.getStoreFlusher();
1017    IOException lastException = null;
1018    for (int i = 0; i < flushRetriesNumber; i++) {
1019      try {
1020        List<Path> pathNames =
1021            flusher.flushSnapshot(snapshot, logCacheFlushId, status, throughputController, tracker);
1022        Path lastPathName = null;
1023        try {
1024          for (Path pathName : pathNames) {
1025            lastPathName = pathName;
1026            validateStoreFile(pathName);
1027          }
1028          return pathNames;
1029        } catch (Exception e) {
1030          LOG.warn("Failed validating store file {}, retrying num={}", lastPathName, i, e);
1031          if (e instanceof IOException) {
1032            lastException = (IOException) e;
1033          } else {
1034            lastException = new IOException(e);
1035          }
1036        }
1037      } catch (IOException e) {
1038        LOG.warn("Failed flushing store file, retrying num={}", i, e);
1039        lastException = e;
1040      }
1041      if (lastException != null && i < (flushRetriesNumber - 1)) {
1042        try {
1043          Thread.sleep(pauseTime);
1044        } catch (InterruptedException e) {
1045          IOException iie = new InterruptedIOException();
1046          iie.initCause(e);
1047          throw iie;
1048        }
1049      }
1050    }
1051    throw lastException;
1052  }
1053
1054  /**
1055   * @param path The pathname of the tmp file into which the store was flushed
1056   * @param logCacheFlushId
1057   * @param status
1058   * @return store file created.
1059   * @throws IOException
1060   */
1061  private HStoreFile commitFile(Path path, long logCacheFlushId, MonitoredTask status)
1062      throws IOException {
1063    // Write-out finished successfully, move into the right spot
1064    Path dstPath = fs.commitStoreFile(getColumnFamilyName(), path);
1065
1066    status.setStatus("Flushing " + this + ": reopening flushed file");
1067    HStoreFile sf = createStoreFileAndReader(dstPath);
1068
1069    StoreFileReader r = sf.getReader();
1070    this.storeSize.addAndGet(r.length());
1071    this.totalUncompressedBytes.addAndGet(r.getTotalUncompressedBytes());
1072
1073    if (LOG.isInfoEnabled()) {
1074      LOG.info("Added " + sf + ", entries=" + r.getEntries() +
1075        ", sequenceid=" + logCacheFlushId +
1076        ", filesize=" + TraditionalBinaryPrefix.long2String(r.length(), "", 1));
1077    }
1078    return sf;
1079  }
1080
1081  /**
1082   * @param maxKeyCount
1083   * @param compression Compression algorithm to use
1084   * @param isCompaction whether we are creating a new file in a compaction
1085   * @param includeMVCCReadpoint - whether to include MVCC or not
1086   * @param includesTag - includesTag or not
1087   * @return Writer for a new StoreFile in the tmp dir.
1088   */
1089  // TODO : allow the Writer factory to create Writers of ShipperListener type only in case of
1090  // compaction
1091  public StoreFileWriter createWriterInTmp(long maxKeyCount, Compression.Algorithm compression,
1092      boolean isCompaction, boolean includeMVCCReadpoint, boolean includesTag,
1093      boolean shouldDropBehind) throws IOException {
1094    final CacheConfig writerCacheConf;
1095    if (isCompaction) {
1096      // Don't cache data on write on compactions.
1097      writerCacheConf = new CacheConfig(cacheConf);
1098      writerCacheConf.setCacheDataOnWrite(false);
1099    } else {
1100      writerCacheConf = cacheConf;
1101    }
1102    InetSocketAddress[] favoredNodes = null;
1103    if (region.getRegionServerServices() != null) {
1104      favoredNodes = region.getRegionServerServices().getFavoredNodesForRegion(
1105          region.getRegionInfo().getEncodedName());
1106    }
1107    HFileContext hFileContext = createFileContext(compression, includeMVCCReadpoint, includesTag,
1108      cryptoContext);
1109    Path familyTempDir = new Path(fs.getTempDir(), family.getNameAsString());
1110    StoreFileWriter.Builder builder = new StoreFileWriter.Builder(conf, writerCacheConf,
1111        this.getFileSystem())
1112            .withOutputDir(familyTempDir)
1113            .withComparator(comparator)
1114            .withBloomType(family.getBloomFilterType())
1115            .withMaxKeyCount(maxKeyCount)
1116            .withFavoredNodes(favoredNodes)
1117            .withFileContext(hFileContext)
1118            .withShouldDropCacheBehind(shouldDropBehind);
1119    return builder.build();
1120  }
1121
1122  private HFileContext createFileContext(Compression.Algorithm compression,
1123      boolean includeMVCCReadpoint, boolean includesTag, Encryption.Context cryptoContext) {
1124    if (compression == null) {
1125      compression = HFile.DEFAULT_COMPRESSION_ALGORITHM;
1126    }
1127    HFileContext hFileContext = new HFileContextBuilder()
1128                                .withIncludesMvcc(includeMVCCReadpoint)
1129                                .withIncludesTags(includesTag)
1130                                .withCompression(compression)
1131                                .withCompressTags(family.isCompressTags())
1132                                .withChecksumType(checksumType)
1133                                .withBytesPerCheckSum(bytesPerChecksum)
1134                                .withBlockSize(blocksize)
1135                                .withHBaseCheckSum(true)
1136                                .withDataBlockEncoding(family.getDataBlockEncoding())
1137                                .withEncryptionContext(cryptoContext)
1138                                .withCreateTime(EnvironmentEdgeManager.currentTime())
1139                                .build();
1140    return hFileContext;
1141  }
1142
1143
1144  private long getTotalSize(Collection<HStoreFile> sfs) {
1145    return sfs.stream().mapToLong(sf -> sf.getReader().length()).sum();
1146  }
1147
1148  /**
1149   * Change storeFiles adding into place the Reader produced by this new flush.
1150   * @param sfs Store files
1151   * @param snapshotId
1152   * @throws IOException
1153   * @return Whether compaction is required.
1154   */
1155  private boolean updateStorefiles(List<HStoreFile> sfs, long snapshotId) throws IOException {
1156    this.lock.writeLock().lock();
1157    try {
1158      this.storeEngine.getStoreFileManager().insertNewFiles(sfs);
1159      if (snapshotId > 0) {
1160        this.memstore.clearSnapshot(snapshotId);
1161      }
1162    } finally {
1163      // We need the lock, as long as we are updating the storeFiles
1164      // or changing the memstore. Let us release it before calling
1165      // notifyChangeReadersObservers. See HBASE-4485 for a possible
1166      // deadlock scenario that could have happened if continue to hold
1167      // the lock.
1168      this.lock.writeLock().unlock();
1169    }
1170    // notify to be called here - only in case of flushes
1171    notifyChangedReadersObservers(sfs);
1172    if (LOG.isTraceEnabled()) {
1173      long totalSize = getTotalSize(sfs);
1174      String traceMessage = "FLUSH time,count,size,store size,store files ["
1175          + EnvironmentEdgeManager.currentTime() + "," + sfs.size() + "," + totalSize
1176          + "," + storeSize + "," + storeEngine.getStoreFileManager().getStorefileCount() + "]";
1177      LOG.trace(traceMessage);
1178    }
1179    return needsCompaction();
1180  }
1181
1182  /**
1183   * Notify all observers that set of Readers has changed.
1184   * @throws IOException
1185   */
1186  private void notifyChangedReadersObservers(List<HStoreFile> sfs) throws IOException {
1187    for (ChangedReadersObserver o : this.changedReaderObservers) {
1188      List<KeyValueScanner> memStoreScanners;
1189      this.lock.readLock().lock();
1190      try {
1191        memStoreScanners = this.memstore.getScanners(o.getReadPoint());
1192      } finally {
1193        this.lock.readLock().unlock();
1194      }
1195      o.updateReaders(sfs, memStoreScanners);
1196    }
1197  }
1198
1199  /**
1200   * Get all scanners with no filtering based on TTL (that happens further down the line).
1201   * @param cacheBlocks cache the blocks or not
1202   * @param usePread true to use pread, false if not
1203   * @param isCompaction true if the scanner is created for compaction
1204   * @param matcher the scan query matcher
1205   * @param startRow the start row
1206   * @param stopRow the stop row
1207   * @param readPt the read point of the current scan
1208   * @return all scanners for this store
1209   */
1210  public List<KeyValueScanner> getScanners(boolean cacheBlocks, boolean isGet, boolean usePread,
1211      boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, byte[] stopRow, long readPt)
1212      throws IOException {
1213    return getScanners(cacheBlocks, usePread, isCompaction, matcher, startRow, true, stopRow, false,
1214      readPt);
1215  }
1216
1217  /**
1218   * Get all scanners with no filtering based on TTL (that happens further down the line).
1219   * @param cacheBlocks cache the blocks or not
1220   * @param usePread true to use pread, false if not
1221   * @param isCompaction true if the scanner is created for compaction
1222   * @param matcher the scan query matcher
1223   * @param startRow the start row
1224   * @param includeStartRow true to include start row, false if not
1225   * @param stopRow the stop row
1226   * @param includeStopRow true to include stop row, false if not
1227   * @param readPt the read point of the current scan
1228   * @return all scanners for this store
1229   */
1230  public List<KeyValueScanner> getScanners(boolean cacheBlocks, boolean usePread,
1231      boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, boolean includeStartRow,
1232      byte[] stopRow, boolean includeStopRow, long readPt) throws IOException {
1233    Collection<HStoreFile> storeFilesToScan;
1234    List<KeyValueScanner> memStoreScanners;
1235    this.lock.readLock().lock();
1236    try {
1237      storeFilesToScan = this.storeEngine.getStoreFileManager().getFilesForScan(startRow,
1238        includeStartRow, stopRow, includeStopRow);
1239      memStoreScanners = this.memstore.getScanners(readPt);
1240    } finally {
1241      this.lock.readLock().unlock();
1242    }
1243
1244    // First the store file scanners
1245
1246    // TODO this used to get the store files in descending order,
1247    // but now we get them in ascending order, which I think is
1248    // actually more correct, since memstore get put at the end.
1249    List<StoreFileScanner> sfScanners = StoreFileScanner.getScannersForStoreFiles(storeFilesToScan,
1250      cacheBlocks, usePread, isCompaction, false, matcher, readPt);
1251    List<KeyValueScanner> scanners = new ArrayList<>(sfScanners.size() + 1);
1252    scanners.addAll(sfScanners);
1253    // Then the memstore scanners
1254    scanners.addAll(memStoreScanners);
1255    return scanners;
1256  }
1257
1258  /**
1259   * Create scanners on the given files and if needed on the memstore with no filtering based on TTL
1260   * (that happens further down the line).
1261   * @param files the list of files on which the scanners has to be created
1262   * @param cacheBlocks cache the blocks or not
1263   * @param usePread true to use pread, false if not
1264   * @param isCompaction true if the scanner is created for compaction
1265   * @param matcher the scan query matcher
1266   * @param startRow the start row
1267   * @param stopRow the stop row
1268   * @param readPt the read point of the current scan
1269   * @param includeMemstoreScanner true if memstore has to be included
1270   * @return scanners on the given files and on the memstore if specified
1271   */
1272  public List<KeyValueScanner> getScanners(List<HStoreFile> files, boolean cacheBlocks,
1273      boolean isGet, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher,
1274      byte[] startRow, byte[] stopRow, long readPt, boolean includeMemstoreScanner)
1275      throws IOException {
1276    return getScanners(files, cacheBlocks, usePread, isCompaction, matcher, startRow, true, stopRow,
1277      false, readPt, includeMemstoreScanner);
1278  }
1279
1280  /**
1281   * Create scanners on the given files and if needed on the memstore with no filtering based on TTL
1282   * (that happens further down the line).
1283   * @param files the list of files on which the scanners has to be created
1284   * @param cacheBlocks ache the blocks or not
1285   * @param usePread true to use pread, false if not
1286   * @param isCompaction true if the scanner is created for compaction
1287   * @param matcher the scan query matcher
1288   * @param startRow the start row
1289   * @param includeStartRow true to include start row, false if not
1290   * @param stopRow the stop row
1291   * @param includeStopRow true to include stop row, false if not
1292   * @param readPt the read point of the current scan
1293   * @param includeMemstoreScanner true if memstore has to be included
1294   * @return scanners on the given files and on the memstore if specified
1295   */
1296  public List<KeyValueScanner> getScanners(List<HStoreFile> files, boolean cacheBlocks,
1297      boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow,
1298      boolean includeStartRow, byte[] stopRow, boolean includeStopRow, long readPt,
1299      boolean includeMemstoreScanner) throws IOException {
1300    List<KeyValueScanner> memStoreScanners = null;
1301    if (includeMemstoreScanner) {
1302      this.lock.readLock().lock();
1303      try {
1304        memStoreScanners = this.memstore.getScanners(readPt);
1305      } finally {
1306        this.lock.readLock().unlock();
1307      }
1308    }
1309    List<StoreFileScanner> sfScanners = StoreFileScanner.getScannersForStoreFiles(files,
1310      cacheBlocks, usePread, isCompaction, false, matcher, readPt);
1311    List<KeyValueScanner> scanners = new ArrayList<>(sfScanners.size() + 1);
1312    scanners.addAll(sfScanners);
1313    // Then the memstore scanners
1314    if (memStoreScanners != null) {
1315      scanners.addAll(memStoreScanners);
1316    }
1317    return scanners;
1318  }
1319
1320  /**
1321   * @param o Observer who wants to know about changes in set of Readers
1322   */
1323  public void addChangedReaderObserver(ChangedReadersObserver o) {
1324    this.changedReaderObservers.add(o);
1325  }
1326
1327  /**
1328   * @param o Observer no longer interested in changes in set of Readers.
1329   */
1330  public void deleteChangedReaderObserver(ChangedReadersObserver o) {
1331    // We don't check if observer present; it may not be (legitimately)
1332    this.changedReaderObservers.remove(o);
1333  }
1334
1335  //////////////////////////////////////////////////////////////////////////////
1336  // Compaction
1337  //////////////////////////////////////////////////////////////////////////////
1338
1339  /**
1340   * Compact the StoreFiles.  This method may take some time, so the calling
1341   * thread must be able to block for long periods.
1342   *
1343   * <p>During this time, the Store can work as usual, getting values from
1344   * StoreFiles and writing new StoreFiles from the memstore.
1345   *
1346   * Existing StoreFiles are not destroyed until the new compacted StoreFile is
1347   * completely written-out to disk.
1348   *
1349   * <p>The compactLock prevents multiple simultaneous compactions.
1350   * The structureLock prevents us from interfering with other write operations.
1351   *
1352   * <p>We don't want to hold the structureLock for the whole time, as a compact()
1353   * can be lengthy and we want to allow cache-flushes during this period.
1354   *
1355   * <p> Compaction event should be idempotent, since there is no IO Fencing for
1356   * the region directory in hdfs. A region server might still try to complete the
1357   * compaction after it lost the region. That is why the following events are carefully
1358   * ordered for a compaction:
1359   *  1. Compaction writes new files under region/.tmp directory (compaction output)
1360   *  2. Compaction atomically moves the temporary file under region directory
1361   *  3. Compaction appends a WAL edit containing the compaction input and output files.
1362   *  Forces sync on WAL.
1363   *  4. Compaction deletes the input files from the region directory.
1364   *
1365   * Failure conditions are handled like this:
1366   *  - If RS fails before 2, compaction wont complete. Even if RS lives on and finishes
1367   *  the compaction later, it will only write the new data file to the region directory.
1368   *  Since we already have this data, this will be idempotent but we will have a redundant
1369   *  copy of the data.
1370   *  - If RS fails between 2 and 3, the region will have a redundant copy of the data. The
1371   *  RS that failed won't be able to finish snyc() for WAL because of lease recovery in WAL.
1372   *  - If RS fails after 3, the region region server who opens the region will pick up the
1373   *  the compaction marker from the WAL and replay it by removing the compaction input files.
1374   *  Failed RS can also attempt to delete those files, but the operation will be idempotent
1375   *
1376   * See HBASE-2231 for details.
1377   *
1378   * @param compaction compaction details obtained from requestCompaction()
1379   * @throws IOException
1380   * @return Storefile we compacted into or null if we failed or opted out early.
1381   */
1382  public List<HStoreFile> compact(CompactionContext compaction,
1383    ThroughputController throughputController, User user) throws IOException {
1384    assert compaction != null;
1385    List<HStoreFile> sfs = null;
1386    CompactionRequestImpl cr = compaction.getRequest();
1387    try {
1388      // Do all sanity checking in here if we have a valid CompactionRequestImpl
1389      // because we need to clean up after it on the way out in a finally
1390      // block below
1391      long compactionStartTime = EnvironmentEdgeManager.currentTime();
1392      assert compaction.hasSelection();
1393      Collection<HStoreFile> filesToCompact = cr.getFiles();
1394      assert !filesToCompact.isEmpty();
1395      synchronized (filesCompacting) {
1396        // sanity check: we're compacting files that this store knows about
1397        // TODO: change this to LOG.error() after more debugging
1398        Preconditions.checkArgument(filesCompacting.containsAll(filesToCompact));
1399      }
1400
1401      // Ready to go. Have list of files to compact.
1402      LOG.info("Starting compaction of " + filesToCompact +
1403        " into tmpdir=" + fs.getTempDir() + ", totalSize=" +
1404          TraditionalBinaryPrefix.long2String(cr.getSize(), "", 1));
1405
1406      return doCompaction(cr, filesToCompact, user, compactionStartTime,
1407          compaction.compact(throughputController, user));
1408    } finally {
1409      finishCompactionRequest(cr);
1410    }
1411  }
1412
1413  @VisibleForTesting
1414  protected List<HStoreFile> doCompaction(CompactionRequestImpl cr,
1415      Collection<HStoreFile> filesToCompact, User user, long compactionStartTime,
1416      List<Path> newFiles) throws IOException {
1417    // Do the steps necessary to complete the compaction.
1418    List<HStoreFile> sfs = moveCompactedFilesIntoPlace(cr, newFiles, user);
1419    writeCompactionWalRecord(filesToCompact, sfs);
1420    replaceStoreFiles(filesToCompact, sfs);
1421    if (cr.isMajor()) {
1422      majorCompactedCellsCount.addAndGet(getCompactionProgress().getTotalCompactingKVs());
1423      majorCompactedCellsSize.addAndGet(getCompactionProgress().totalCompactedSize);
1424    } else {
1425      compactedCellsCount.addAndGet(getCompactionProgress().getTotalCompactingKVs());
1426      compactedCellsSize.addAndGet(getCompactionProgress().totalCompactedSize);
1427    }
1428    long outputBytes = getTotalSize(sfs);
1429
1430    // At this point the store will use new files for all new scanners.
1431    completeCompaction(filesToCompact); // update store size.
1432
1433    long now = EnvironmentEdgeManager.currentTime();
1434    if (region.getRegionServerServices() != null
1435        && region.getRegionServerServices().getMetrics() != null) {
1436      region.getRegionServerServices().getMetrics().updateCompaction(cr.isMajor(),
1437          now - compactionStartTime, cr.getFiles().size(), newFiles.size(), cr.getSize(),
1438          outputBytes);
1439    }
1440
1441    logCompactionEndMessage(cr, sfs, now, compactionStartTime);
1442    return sfs;
1443  }
1444
1445  private List<HStoreFile> moveCompactedFilesIntoPlace(CompactionRequestImpl cr, List<Path> newFiles,
1446      User user) throws IOException {
1447    List<HStoreFile> sfs = new ArrayList<>(newFiles.size());
1448    for (Path newFile : newFiles) {
1449      assert newFile != null;
1450      HStoreFile sf = moveFileIntoPlace(newFile);
1451      if (this.getCoprocessorHost() != null) {
1452        getCoprocessorHost().postCompact(this, sf, cr.getTracker(), cr, user);
1453      }
1454      assert sf != null;
1455      sfs.add(sf);
1456    }
1457    return sfs;
1458  }
1459
1460  // Package-visible for tests
1461  HStoreFile moveFileIntoPlace(Path newFile) throws IOException {
1462    validateStoreFile(newFile);
1463    // Move the file into the right spot
1464    Path destPath = fs.commitStoreFile(getColumnFamilyName(), newFile);
1465    return createStoreFileAndReader(destPath);
1466  }
1467
1468  /**
1469   * Writes the compaction WAL record.
1470   * @param filesCompacted Files compacted (input).
1471   * @param newFiles Files from compaction.
1472   */
1473  private void writeCompactionWalRecord(Collection<HStoreFile> filesCompacted,
1474      Collection<HStoreFile> newFiles) throws IOException {
1475    if (region.getWAL() == null) {
1476      return;
1477    }
1478    List<Path> inputPaths =
1479        filesCompacted.stream().map(HStoreFile::getPath).collect(Collectors.toList());
1480    List<Path> outputPaths =
1481        newFiles.stream().map(HStoreFile::getPath).collect(Collectors.toList());
1482    RegionInfo info = this.region.getRegionInfo();
1483    CompactionDescriptor compactionDescriptor = ProtobufUtil.toCompactionDescriptor(info,
1484        family.getName(), inputPaths, outputPaths, fs.getStoreDir(getColumnFamilyDescriptor().getNameAsString()));
1485    // Fix reaching into Region to get the maxWaitForSeqId.
1486    // Does this method belong in Region altogether given it is making so many references up there?
1487    // Could be Region#writeCompactionMarker(compactionDescriptor);
1488    WALUtil.writeCompactionMarker(this.region.getWAL(), this.region.getReplicationScope(),
1489        this.region.getRegionInfo(), compactionDescriptor, this.region.getMVCC());
1490  }
1491
1492  @VisibleForTesting
1493  void replaceStoreFiles(Collection<HStoreFile> compactedFiles, Collection<HStoreFile> result)
1494      throws IOException {
1495    this.lock.writeLock().lock();
1496    try {
1497      this.storeEngine.getStoreFileManager().addCompactionResults(compactedFiles, result);
1498      synchronized (filesCompacting) {
1499        filesCompacting.removeAll(compactedFiles);
1500      }
1501    } finally {
1502      this.lock.writeLock().unlock();
1503    }
1504  }
1505
1506  /**
1507   * Log a very elaborate compaction completion message.
1508   * @param cr Request.
1509   * @param sfs Resulting files.
1510   * @param compactionStartTime Start time.
1511   */
1512  private void logCompactionEndMessage(
1513      CompactionRequestImpl cr, List<HStoreFile> sfs, long now, long compactionStartTime) {
1514    StringBuilder message = new StringBuilder(
1515      "Completed" + (cr.isMajor() ? " major" : "") + " compaction of "
1516      + cr.getFiles().size() + (cr.isAllFiles() ? " (all)" : "") + " file(s) in "
1517      + this + " of " + this.getRegionInfo().getShortNameToLog() + " into ");
1518    if (sfs.isEmpty()) {
1519      message.append("none, ");
1520    } else {
1521      for (HStoreFile sf: sfs) {
1522        message.append(sf.getPath().getName());
1523        message.append("(size=");
1524        message.append(TraditionalBinaryPrefix.long2String(sf.getReader().length(), "", 1));
1525        message.append("), ");
1526      }
1527    }
1528    message.append("total size for store is ")
1529      .append(StringUtils.TraditionalBinaryPrefix.long2String(storeSize.get(), "", 1))
1530      .append(". This selection was in queue for ")
1531      .append(StringUtils.formatTimeDiff(compactionStartTime, cr.getSelectionTime()))
1532      .append(", and took ").append(StringUtils.formatTimeDiff(now, compactionStartTime))
1533      .append(" to execute.");
1534    LOG.info(message.toString());
1535    if (LOG.isTraceEnabled()) {
1536      int fileCount = storeEngine.getStoreFileManager().getStorefileCount();
1537      long resultSize = getTotalSize(sfs);
1538      String traceMessage = "COMPACTION start,end,size out,files in,files out,store size,"
1539        + "store files [" + compactionStartTime + "," + now + "," + resultSize + ","
1540          + cr.getFiles().size() + "," + sfs.size() + "," +  storeSize + "," + fileCount + "]";
1541      LOG.trace(traceMessage);
1542    }
1543  }
1544
1545  /**
1546   * Call to complete a compaction. Its for the case where we find in the WAL a compaction
1547   * that was not finished.  We could find one recovering a WAL after a regionserver crash.
1548   * See HBASE-2231.
1549   * @param compaction
1550   */
1551  public void replayCompactionMarker(CompactionDescriptor compaction, boolean pickCompactionFiles,
1552      boolean removeFiles) throws IOException {
1553    LOG.debug("Completing compaction from the WAL marker");
1554    List<String> compactionInputs = compaction.getCompactionInputList();
1555    List<String> compactionOutputs = Lists.newArrayList(compaction.getCompactionOutputList());
1556
1557    // The Compaction Marker is written after the compaction is completed,
1558    // and the files moved into the region/family folder.
1559    //
1560    // If we crash after the entry is written, we may not have removed the
1561    // input files, but the output file is present.
1562    // (The unremoved input files will be removed by this function)
1563    //
1564    // If we scan the directory and the file is not present, it can mean that:
1565    //   - The file was manually removed by the user
1566    //   - The file was removed as consequence of subsequent compaction
1567    // so, we can't do anything with the "compaction output list" because those
1568    // files have already been loaded when opening the region (by virtue of
1569    // being in the store's folder) or they may be missing due to a compaction.
1570
1571    String familyName = this.getColumnFamilyName();
1572    Set<String> inputFiles = new HashSet<>();
1573    for (String compactionInput : compactionInputs) {
1574      Path inputPath = fs.getStoreFilePath(familyName, compactionInput);
1575      inputFiles.add(inputPath.getName());
1576    }
1577
1578    //some of the input files might already be deleted
1579    List<HStoreFile> inputStoreFiles = new ArrayList<>(compactionInputs.size());
1580    for (HStoreFile sf : this.getStorefiles()) {
1581      if (inputFiles.contains(sf.getPath().getName())) {
1582        inputStoreFiles.add(sf);
1583      }
1584    }
1585
1586    // check whether we need to pick up the new files
1587    List<HStoreFile> outputStoreFiles = new ArrayList<>(compactionOutputs.size());
1588
1589    if (pickCompactionFiles) {
1590      for (HStoreFile sf : this.getStorefiles()) {
1591        compactionOutputs.remove(sf.getPath().getName());
1592      }
1593      for (String compactionOutput : compactionOutputs) {
1594        StoreFileInfo storeFileInfo = fs.getStoreFileInfo(getColumnFamilyName(), compactionOutput);
1595        HStoreFile storeFile = createStoreFileAndReader(storeFileInfo);
1596        outputStoreFiles.add(storeFile);
1597      }
1598    }
1599
1600    if (!inputStoreFiles.isEmpty() || !outputStoreFiles.isEmpty()) {
1601      LOG.info("Replaying compaction marker, replacing input files: " +
1602          inputStoreFiles + " with output files : " + outputStoreFiles);
1603      this.replaceStoreFiles(inputStoreFiles, outputStoreFiles);
1604      this.completeCompaction(inputStoreFiles);
1605    }
1606  }
1607
1608  /**
1609   * This method tries to compact N recent files for testing.
1610   * Note that because compacting "recent" files only makes sense for some policies,
1611   * e.g. the default one, it assumes default policy is used. It doesn't use policy,
1612   * but instead makes a compaction candidate list by itself.
1613   * @param N Number of files.
1614   */
1615  @VisibleForTesting
1616  public void compactRecentForTestingAssumingDefaultPolicy(int N) throws IOException {
1617    List<HStoreFile> filesToCompact;
1618    boolean isMajor;
1619
1620    this.lock.readLock().lock();
1621    try {
1622      synchronized (filesCompacting) {
1623        filesToCompact = Lists.newArrayList(storeEngine.getStoreFileManager().getStorefiles());
1624        if (!filesCompacting.isEmpty()) {
1625          // exclude all files older than the newest file we're currently
1626          // compacting. this allows us to preserve contiguity (HBASE-2856)
1627          HStoreFile last = filesCompacting.get(filesCompacting.size() - 1);
1628          int idx = filesToCompact.indexOf(last);
1629          Preconditions.checkArgument(idx != -1);
1630          filesToCompact.subList(0, idx + 1).clear();
1631        }
1632        int count = filesToCompact.size();
1633        if (N > count) {
1634          throw new RuntimeException("Not enough files");
1635        }
1636
1637        filesToCompact = filesToCompact.subList(count - N, count);
1638        isMajor = (filesToCompact.size() == storeEngine.getStoreFileManager().getStorefileCount());
1639        filesCompacting.addAll(filesToCompact);
1640        Collections.sort(filesCompacting, storeEngine.getStoreFileManager()
1641            .getStoreFileComparator());
1642      }
1643    } finally {
1644      this.lock.readLock().unlock();
1645    }
1646
1647    try {
1648      // Ready to go. Have list of files to compact.
1649      List<Path> newFiles = ((DefaultCompactor)this.storeEngine.getCompactor())
1650          .compactForTesting(filesToCompact, isMajor);
1651      for (Path newFile: newFiles) {
1652        // Move the compaction into place.
1653        HStoreFile sf = moveFileIntoPlace(newFile);
1654        if (this.getCoprocessorHost() != null) {
1655          this.getCoprocessorHost().postCompact(this, sf, null, null, null);
1656        }
1657        replaceStoreFiles(filesToCompact, Collections.singletonList(sf));
1658        completeCompaction(filesToCompact);
1659      }
1660    } finally {
1661      synchronized (filesCompacting) {
1662        filesCompacting.removeAll(filesToCompact);
1663      }
1664    }
1665  }
1666
1667  @Override
1668  public boolean hasReferences() {
1669    // Grab the read lock here, because we need to ensure that: only when the atomic
1670    // replaceStoreFiles(..) finished, we can get all the complete store file list.
1671    this.lock.readLock().lock();
1672    try {
1673      // Merge the current store files with compacted files here due to HBASE-20940.
1674      Collection<HStoreFile> allStoreFiles = new ArrayList<>(getStorefiles());
1675      allStoreFiles.addAll(getCompactedFiles());
1676      return StoreUtils.hasReferences(allStoreFiles);
1677    } finally {
1678      this.lock.readLock().unlock();
1679    }
1680  }
1681
1682  /**
1683   * getter for CompactionProgress object
1684   * @return CompactionProgress object; can be null
1685   */
1686  public CompactionProgress getCompactionProgress() {
1687    return this.storeEngine.getCompactor().getProgress();
1688  }
1689
1690  @Override
1691  public boolean shouldPerformMajorCompaction() throws IOException {
1692    for (HStoreFile sf : this.storeEngine.getStoreFileManager().getStorefiles()) {
1693      // TODO: what are these reader checks all over the place?
1694      if (sf.getReader() == null) {
1695        LOG.debug("StoreFile {} has null Reader", sf);
1696        return false;
1697      }
1698    }
1699    return storeEngine.getCompactionPolicy().shouldPerformMajorCompaction(
1700        this.storeEngine.getStoreFileManager().getStorefiles());
1701  }
1702
1703  public Optional<CompactionContext> requestCompaction() throws IOException {
1704    return requestCompaction(NO_PRIORITY, CompactionLifeCycleTracker.DUMMY, null);
1705  }
1706
1707  public Optional<CompactionContext> requestCompaction(int priority,
1708      CompactionLifeCycleTracker tracker, User user) throws IOException {
1709    // don't even select for compaction if writes are disabled
1710    if (!this.areWritesEnabled()) {
1711      return Optional.empty();
1712    }
1713    // Before we do compaction, try to get rid of unneeded files to simplify things.
1714    removeUnneededFiles();
1715
1716    final CompactionContext compaction = storeEngine.createCompaction();
1717    CompactionRequestImpl request = null;
1718    this.lock.readLock().lock();
1719    try {
1720      synchronized (filesCompacting) {
1721        // First, see if coprocessor would want to override selection.
1722        if (this.getCoprocessorHost() != null) {
1723          final List<HStoreFile> candidatesForCoproc = compaction.preSelect(this.filesCompacting);
1724          boolean override = getCoprocessorHost().preCompactSelection(this,
1725              candidatesForCoproc, tracker, user);
1726          if (override) {
1727            // Coprocessor is overriding normal file selection.
1728            compaction.forceSelect(new CompactionRequestImpl(candidatesForCoproc));
1729          }
1730        }
1731
1732        // Normal case - coprocessor is not overriding file selection.
1733        if (!compaction.hasSelection()) {
1734          boolean isUserCompaction = priority == Store.PRIORITY_USER;
1735          boolean mayUseOffPeak = offPeakHours.isOffPeakHour() &&
1736              offPeakCompactionTracker.compareAndSet(false, true);
1737          try {
1738            compaction.select(this.filesCompacting, isUserCompaction,
1739              mayUseOffPeak, forceMajor && filesCompacting.isEmpty());
1740          } catch (IOException e) {
1741            if (mayUseOffPeak) {
1742              offPeakCompactionTracker.set(false);
1743            }
1744            throw e;
1745          }
1746          assert compaction.hasSelection();
1747          if (mayUseOffPeak && !compaction.getRequest().isOffPeak()) {
1748            // Compaction policy doesn't want to take advantage of off-peak.
1749            offPeakCompactionTracker.set(false);
1750          }
1751        }
1752        if (this.getCoprocessorHost() != null) {
1753          this.getCoprocessorHost().postCompactSelection(
1754              this, ImmutableList.copyOf(compaction.getRequest().getFiles()), tracker,
1755              compaction.getRequest(), user);
1756        }
1757        // Finally, we have the resulting files list. Check if we have any files at all.
1758        request = compaction.getRequest();
1759        Collection<HStoreFile> selectedFiles = request.getFiles();
1760        if (selectedFiles.isEmpty()) {
1761          return Optional.empty();
1762        }
1763
1764        addToCompactingFiles(selectedFiles);
1765
1766        // If we're enqueuing a major, clear the force flag.
1767        this.forceMajor = this.forceMajor && !request.isMajor();
1768
1769        // Set common request properties.
1770        // Set priority, either override value supplied by caller or from store.
1771        request.setPriority((priority != Store.NO_PRIORITY) ? priority : getCompactPriority());
1772        request.setDescription(getRegionInfo().getRegionNameAsString(), getColumnFamilyName());
1773        request.setTracker(tracker);
1774      }
1775    } finally {
1776      this.lock.readLock().unlock();
1777    }
1778
1779    if (LOG.isDebugEnabled()) {
1780      LOG.debug(getRegionInfo().getEncodedName() + " - " + getColumnFamilyName()
1781          + ": Initiating " + (request.isMajor() ? "major" : "minor") + " compaction"
1782          + (request.isAllFiles() ? " (all files)" : ""));
1783    }
1784    this.region.reportCompactionRequestStart(request.isMajor());
1785    return Optional.of(compaction);
1786  }
1787
1788  /** Adds the files to compacting files. filesCompacting must be locked. */
1789  private void addToCompactingFiles(Collection<HStoreFile> filesToAdd) {
1790    if (CollectionUtils.isEmpty(filesToAdd)) {
1791      return;
1792    }
1793    // Check that we do not try to compact the same StoreFile twice.
1794    if (!Collections.disjoint(filesCompacting, filesToAdd)) {
1795      Preconditions.checkArgument(false, "%s overlaps with %s", filesToAdd, filesCompacting);
1796    }
1797    filesCompacting.addAll(filesToAdd);
1798    Collections.sort(filesCompacting, storeEngine.getStoreFileManager().getStoreFileComparator());
1799  }
1800
1801  private void removeUnneededFiles() throws IOException {
1802    if (!conf.getBoolean("hbase.store.delete.expired.storefile", true)) return;
1803    if (getColumnFamilyDescriptor().getMinVersions() > 0) {
1804      LOG.debug("Skipping expired store file removal due to min version being {}",
1805          getColumnFamilyDescriptor().getMinVersions());
1806      return;
1807    }
1808    this.lock.readLock().lock();
1809    Collection<HStoreFile> delSfs = null;
1810    try {
1811      synchronized (filesCompacting) {
1812        long cfTtl = getStoreFileTtl();
1813        if (cfTtl != Long.MAX_VALUE) {
1814          delSfs = storeEngine.getStoreFileManager().getUnneededFiles(
1815              EnvironmentEdgeManager.currentTime() - cfTtl, filesCompacting);
1816          addToCompactingFiles(delSfs);
1817        }
1818      }
1819    } finally {
1820      this.lock.readLock().unlock();
1821    }
1822
1823    if (CollectionUtils.isEmpty(delSfs)) {
1824      return;
1825    }
1826
1827    Collection<HStoreFile> newFiles = Collections.emptyList(); // No new files.
1828    writeCompactionWalRecord(delSfs, newFiles);
1829    replaceStoreFiles(delSfs, newFiles);
1830    completeCompaction(delSfs);
1831    LOG.info("Completed removal of " + delSfs.size() + " unnecessary (expired) file(s) in "
1832        + this + " of " + this.getRegionInfo().getRegionNameAsString()
1833        + "; total size for store is "
1834        + TraditionalBinaryPrefix.long2String(storeSize.get(), "", 1));
1835  }
1836
1837  public void cancelRequestedCompaction(CompactionContext compaction) {
1838    finishCompactionRequest(compaction.getRequest());
1839  }
1840
1841  private void finishCompactionRequest(CompactionRequestImpl cr) {
1842    this.region.reportCompactionRequestEnd(cr.isMajor(), cr.getFiles().size(), cr.getSize());
1843    if (cr.isOffPeak()) {
1844      offPeakCompactionTracker.set(false);
1845      cr.setOffPeak(false);
1846    }
1847    synchronized (filesCompacting) {
1848      filesCompacting.removeAll(cr.getFiles());
1849    }
1850  }
1851
1852  /**
1853   * Validates a store file by opening and closing it. In HFileV2 this should not be an expensive
1854   * operation.
1855   * @param path the path to the store file
1856   */
1857  private void validateStoreFile(Path path) throws IOException {
1858    HStoreFile storeFile = null;
1859    try {
1860      storeFile = createStoreFileAndReader(path);
1861    } catch (IOException e) {
1862      LOG.error("Failed to open store file : {}, keeping it in tmp location", path, e);
1863      throw e;
1864    } finally {
1865      if (storeFile != null) {
1866        storeFile.closeStoreFile(false);
1867      }
1868    }
1869  }
1870
1871  /**
1872   * <p>It works by processing a compaction that's been written to disk.
1873   *
1874   * <p>It is usually invoked at the end of a compaction, but might also be
1875   * invoked at HStore startup, if the prior execution died midway through.
1876   *
1877   * <p>Moving the compacted TreeMap into place means:
1878   * <pre>
1879   * 1) Unload all replaced StoreFile, close and collect list to delete.
1880   * 2) Compute new store size
1881   * </pre>
1882   *
1883   * @param compactedFiles list of files that were compacted
1884   */
1885  @VisibleForTesting
1886  protected void completeCompaction(Collection<HStoreFile> compactedFiles)
1887    throws IOException {
1888    this.storeSize.set(0L);
1889    this.totalUncompressedBytes.set(0L);
1890    for (HStoreFile hsf : this.storeEngine.getStoreFileManager().getStorefiles()) {
1891      StoreFileReader r = hsf.getReader();
1892      if (r == null) {
1893        LOG.warn("StoreFile {} has a null Reader", hsf);
1894        continue;
1895      }
1896      this.storeSize.addAndGet(r.length());
1897      this.totalUncompressedBytes.addAndGet(r.getTotalUncompressedBytes());
1898    }
1899  }
1900
1901  /*
1902   * @param wantedVersions How many versions were asked for.
1903   * @return wantedVersions or this families' {@link HConstants#VERSIONS}.
1904   */
1905  int versionsToReturn(final int wantedVersions) {
1906    if (wantedVersions <= 0) {
1907      throw new IllegalArgumentException("Number of versions must be > 0");
1908    }
1909    // Make sure we do not return more than maximum versions for this store.
1910    int maxVersions = this.family.getMaxVersions();
1911    return wantedVersions > maxVersions ? maxVersions: wantedVersions;
1912  }
1913
1914  @Override
1915  public boolean canSplit() {
1916    this.lock.readLock().lock();
1917    try {
1918      // Not split-able if we find a reference store file present in the store.
1919      boolean result = !hasReferences();
1920      if (!result) {
1921        LOG.trace("Not splittable; has references: {}", this);
1922      }
1923      return result;
1924    } finally {
1925      this.lock.readLock().unlock();
1926    }
1927  }
1928
1929  /**
1930   * Determines if Store should be split.
1931   */
1932  public Optional<byte[]> getSplitPoint() {
1933    this.lock.readLock().lock();
1934    try {
1935      // Should already be enforced by the split policy!
1936      assert !this.getRegionInfo().isMetaRegion();
1937      // Not split-able if we find a reference store file present in the store.
1938      if (hasReferences()) {
1939        LOG.trace("Not splittable; has references: {}", this);
1940        return Optional.empty();
1941      }
1942      return this.storeEngine.getStoreFileManager().getSplitPoint();
1943    } catch(IOException e) {
1944      LOG.warn("Failed getting store size for {}", this, e);
1945    } finally {
1946      this.lock.readLock().unlock();
1947    }
1948    return Optional.empty();
1949  }
1950
1951  @Override
1952  public long getLastCompactSize() {
1953    return this.lastCompactSize;
1954  }
1955
1956  @Override
1957  public long getSize() {
1958    return storeSize.get();
1959  }
1960
1961  public void triggerMajorCompaction() {
1962    this.forceMajor = true;
1963  }
1964
1965  //////////////////////////////////////////////////////////////////////////////
1966  // File administration
1967  //////////////////////////////////////////////////////////////////////////////
1968
1969  /**
1970   * Return a scanner for both the memstore and the HStore files. Assumes we are not in a
1971   * compaction.
1972   * @param scan Scan to apply when scanning the stores
1973   * @param targetCols columns to scan
1974   * @return a scanner over the current key values
1975   * @throws IOException on failure
1976   */
1977  public KeyValueScanner getScanner(Scan scan, final NavigableSet<byte[]> targetCols, long readPt)
1978      throws IOException {
1979    lock.readLock().lock();
1980    try {
1981      ScanInfo scanInfo;
1982      if (this.getCoprocessorHost() != null) {
1983        scanInfo = this.getCoprocessorHost().preStoreScannerOpen(this);
1984      } else {
1985        scanInfo = getScanInfo();
1986      }
1987      return createScanner(scan, scanInfo, targetCols, readPt);
1988    } finally {
1989      lock.readLock().unlock();
1990    }
1991  }
1992
1993  // HMobStore will override this method to return its own implementation.
1994  protected KeyValueScanner createScanner(Scan scan, ScanInfo scanInfo,
1995      NavigableSet<byte[]> targetCols, long readPt) throws IOException {
1996    return scan.isReversed() ? new ReversedStoreScanner(this, scanInfo, scan, targetCols, readPt)
1997        : new StoreScanner(this, scanInfo, scan, targetCols, readPt);
1998  }
1999
2000  /**
2001   * Recreates the scanners on the current list of active store file scanners
2002   * @param currentFileScanners the current set of active store file scanners
2003   * @param cacheBlocks cache the blocks or not
2004   * @param usePread use pread or not
2005   * @param isCompaction is the scanner for compaction
2006   * @param matcher the scan query matcher
2007   * @param startRow the scan's start row
2008   * @param includeStartRow should the scan include the start row
2009   * @param stopRow the scan's stop row
2010   * @param includeStopRow should the scan include the stop row
2011   * @param readPt the read point of the current scane
2012   * @param includeMemstoreScanner whether the current scanner should include memstorescanner
2013   * @return list of scanners recreated on the current Scanners
2014   * @throws IOException
2015   */
2016  public List<KeyValueScanner> recreateScanners(List<KeyValueScanner> currentFileScanners,
2017      boolean cacheBlocks, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher,
2018      byte[] startRow, boolean includeStartRow, byte[] stopRow, boolean includeStopRow, long readPt,
2019      boolean includeMemstoreScanner) throws IOException {
2020    this.lock.readLock().lock();
2021    try {
2022      Map<String, HStoreFile> name2File =
2023          new HashMap<>(getStorefilesCount() + getCompactedFilesCount());
2024      for (HStoreFile file : getStorefiles()) {
2025        name2File.put(file.getFileInfo().getActiveFileName(), file);
2026      }
2027      Collection<HStoreFile> compactedFiles = getCompactedFiles();
2028      for (HStoreFile file : IterableUtils.emptyIfNull(compactedFiles)) {
2029        name2File.put(file.getFileInfo().getActiveFileName(), file);
2030      }
2031      List<HStoreFile> filesToReopen = new ArrayList<>();
2032      for (KeyValueScanner kvs : currentFileScanners) {
2033        assert kvs.isFileScanner();
2034        if (kvs.peek() == null) {
2035          continue;
2036        }
2037        filesToReopen.add(name2File.get(kvs.getFilePath().getName()));
2038      }
2039      if (filesToReopen.isEmpty()) {
2040        return null;
2041      }
2042      return getScanners(filesToReopen, cacheBlocks, false, false, matcher, startRow,
2043        includeStartRow, stopRow, includeStopRow, readPt, false);
2044    } finally {
2045      this.lock.readLock().unlock();
2046    }
2047  }
2048
2049  @Override
2050  public String toString() {
2051    return this.getColumnFamilyName();
2052  }
2053
2054  @Override
2055  public int getStorefilesCount() {
2056    return this.storeEngine.getStoreFileManager().getStorefileCount();
2057  }
2058
2059  @Override
2060  public int getCompactedFilesCount() {
2061    return this.storeEngine.getStoreFileManager().getCompactedFilesCount();
2062  }
2063
2064  private LongStream getStoreFileAgeStream() {
2065    return this.storeEngine.getStoreFileManager().getStorefiles().stream().filter(sf -> {
2066      if (sf.getReader() == null) {
2067        LOG.warn("StoreFile {} has a null Reader", sf);
2068        return false;
2069      } else {
2070        return true;
2071      }
2072    }).filter(HStoreFile::isHFile).mapToLong(sf -> sf.getFileInfo().getCreatedTimestamp())
2073        .map(t -> EnvironmentEdgeManager.currentTime() - t);
2074  }
2075
2076  @Override
2077  public OptionalLong getMaxStoreFileAge() {
2078    return getStoreFileAgeStream().max();
2079  }
2080
2081  @Override
2082  public OptionalLong getMinStoreFileAge() {
2083    return getStoreFileAgeStream().min();
2084  }
2085
2086  @Override
2087  public OptionalDouble getAvgStoreFileAge() {
2088    return getStoreFileAgeStream().average();
2089  }
2090
2091  @Override
2092  public long getNumReferenceFiles() {
2093    return this.storeEngine.getStoreFileManager().getStorefiles().stream()
2094        .filter(HStoreFile::isReference).count();
2095  }
2096
2097  @Override
2098  public long getNumHFiles() {
2099    return this.storeEngine.getStoreFileManager().getStorefiles().stream()
2100        .filter(HStoreFile::isHFile).count();
2101  }
2102
2103  @Override
2104  public long getStoreSizeUncompressed() {
2105    return this.totalUncompressedBytes.get();
2106  }
2107
2108  @Override
2109  public long getStorefilesSize() {
2110    // Include all StoreFiles
2111    return getStorefilesSize(this.storeEngine.getStoreFileManager().getStorefiles(), sf -> true);
2112  }
2113
2114  @Override
2115  public long getHFilesSize() {
2116    // Include only StoreFiles which are HFiles
2117    return getStorefilesSize(this.storeEngine.getStoreFileManager().getStorefiles(),
2118      HStoreFile::isHFile);
2119  }
2120
2121  private long getTotalUmcompressedBytes(List<HStoreFile> files) {
2122    return files.stream().filter(f -> f != null && f.getReader() != null)
2123        .mapToLong(f -> f.getReader().getTotalUncompressedBytes()).sum();
2124  }
2125
2126  private long getStorefilesSize(Collection<HStoreFile> files, Predicate<HStoreFile> predicate) {
2127    return files.stream().filter(f -> f != null && f.getReader() != null).filter(predicate)
2128        .mapToLong(f -> f.getReader().length()).sum();
2129  }
2130
2131  private long getStoreFileFieldSize(ToLongFunction<StoreFileReader> f) {
2132    return this.storeEngine.getStoreFileManager().getStorefiles().stream().filter(sf -> {
2133      if (sf.getReader() == null) {
2134        LOG.warn("StoreFile {} has a null Reader", sf);
2135        return false;
2136      } else {
2137        return true;
2138      }
2139    }).map(HStoreFile::getReader).mapToLong(f).sum();
2140  }
2141
2142  @Override
2143  public long getStorefilesRootLevelIndexSize() {
2144    return getStoreFileFieldSize(StoreFileReader::indexSize);
2145  }
2146
2147  @Override
2148  public long getTotalStaticIndexSize() {
2149    return getStoreFileFieldSize(StoreFileReader::getUncompressedDataIndexSize);
2150  }
2151
2152  @Override
2153  public long getTotalStaticBloomSize() {
2154    return getStoreFileFieldSize(StoreFileReader::getTotalBloomSize);
2155  }
2156
2157  @Override
2158  public MemStoreSize getMemStoreSize() {
2159    return this.memstore.size();
2160  }
2161
2162  @Override
2163  public int getCompactPriority() {
2164    int priority = this.storeEngine.getStoreFileManager().getStoreCompactionPriority();
2165    if (priority == PRIORITY_USER) {
2166      LOG.warn("Compaction priority is USER despite there being no user compaction");
2167    }
2168    return priority;
2169  }
2170
2171  public boolean throttleCompaction(long compactionSize) {
2172    return storeEngine.getCompactionPolicy().throttleCompaction(compactionSize);
2173  }
2174
2175  public HRegion getHRegion() {
2176    return this.region;
2177  }
2178
2179  public RegionCoprocessorHost getCoprocessorHost() {
2180    return this.region.getCoprocessorHost();
2181  }
2182
2183  @Override
2184  public RegionInfo getRegionInfo() {
2185    return this.fs.getRegionInfo();
2186  }
2187
2188  @Override
2189  public boolean areWritesEnabled() {
2190    return this.region.areWritesEnabled();
2191  }
2192
2193  @Override
2194  public long getSmallestReadPoint() {
2195    return this.region.getSmallestReadPoint();
2196  }
2197
2198  /**
2199   * Adds or replaces the specified KeyValues.
2200   * <p>
2201   * For each KeyValue specified, if a cell with the same row, family, and qualifier exists in
2202   * MemStore, it will be replaced. Otherwise, it will just be inserted to MemStore.
2203   * <p>
2204   * This operation is atomic on each KeyValue (row/family/qualifier) but not necessarily atomic
2205   * across all of them.
2206   * @param readpoint readpoint below which we can safely remove duplicate KVs
2207   * @throws IOException
2208   */
2209  public void upsert(Iterable<Cell> cells, long readpoint, MemStoreSizing memstoreSizing)
2210      throws IOException {
2211    this.lock.readLock().lock();
2212    try {
2213      this.memstore.upsert(cells, readpoint, memstoreSizing);
2214    } finally {
2215      this.lock.readLock().unlock();
2216    }
2217  }
2218
2219  public StoreFlushContext createFlushContext(long cacheFlushId, FlushLifeCycleTracker tracker) {
2220    return new StoreFlusherImpl(cacheFlushId, tracker);
2221  }
2222
2223  private final class StoreFlusherImpl implements StoreFlushContext {
2224
2225    private final FlushLifeCycleTracker tracker;
2226    private final long cacheFlushSeqNum;
2227    private MemStoreSnapshot snapshot;
2228    private List<Path> tempFiles;
2229    private List<Path> committedFiles;
2230    private long cacheFlushCount;
2231    private long cacheFlushSize;
2232    private long outputFileSize;
2233
2234    private StoreFlusherImpl(long cacheFlushSeqNum, FlushLifeCycleTracker tracker) {
2235      this.cacheFlushSeqNum = cacheFlushSeqNum;
2236      this.tracker = tracker;
2237    }
2238
2239    /**
2240     * This is not thread safe. The caller should have a lock on the region or the store.
2241     * If necessary, the lock can be added with the patch provided in HBASE-10087
2242     */
2243    @Override
2244    public MemStoreSize prepare() {
2245      // passing the current sequence number of the wal - to allow bookkeeping in the memstore
2246      this.snapshot = memstore.snapshot();
2247      this.cacheFlushCount = snapshot.getCellsCount();
2248      this.cacheFlushSize = snapshot.getDataSize();
2249      committedFiles = new ArrayList<>(1);
2250      return snapshot.getMemStoreSize();
2251    }
2252
2253    @Override
2254    public void flushCache(MonitoredTask status) throws IOException {
2255      RegionServerServices rsService = region.getRegionServerServices();
2256      ThroughputController throughputController =
2257          rsService == null ? null : rsService.getFlushThroughputController();
2258      tempFiles =
2259          HStore.this.flushCache(cacheFlushSeqNum, snapshot, status, throughputController, tracker);
2260    }
2261
2262    @Override
2263    public boolean commit(MonitoredTask status) throws IOException {
2264      if (CollectionUtils.isEmpty(this.tempFiles)) {
2265        return false;
2266      }
2267      List<HStoreFile> storeFiles = new ArrayList<>(this.tempFiles.size());
2268      for (Path storeFilePath : tempFiles) {
2269        try {
2270          HStoreFile sf = HStore.this.commitFile(storeFilePath, cacheFlushSeqNum, status);
2271          outputFileSize += sf.getReader().length();
2272          storeFiles.add(sf);
2273        } catch (IOException ex) {
2274          LOG.error("Failed to commit store file {}", storeFilePath, ex);
2275          // Try to delete the files we have committed before.
2276          for (HStoreFile sf : storeFiles) {
2277            Path pathToDelete = sf.getPath();
2278            try {
2279              sf.deleteStoreFile();
2280            } catch (IOException deleteEx) {
2281              LOG.error(HBaseMarkers.FATAL, "Failed to delete store file we committed, "
2282                  + "halting {}", pathToDelete, ex);
2283              Runtime.getRuntime().halt(1);
2284            }
2285          }
2286          throw new IOException("Failed to commit the flush", ex);
2287        }
2288      }
2289
2290      for (HStoreFile sf : storeFiles) {
2291        if (HStore.this.getCoprocessorHost() != null) {
2292          HStore.this.getCoprocessorHost().postFlush(HStore.this, sf, tracker);
2293        }
2294        committedFiles.add(sf.getPath());
2295      }
2296
2297      HStore.this.flushedCellsCount.addAndGet(cacheFlushCount);
2298      HStore.this.flushedCellsSize.addAndGet(cacheFlushSize);
2299      HStore.this.flushedOutputFileSize.addAndGet(outputFileSize);
2300
2301      // Add new file to store files.  Clear snapshot too while we have the Store write lock.
2302      return HStore.this.updateStorefiles(storeFiles, snapshot.getId());
2303    }
2304
2305    @Override
2306    public long getOutputFileSize() {
2307      return outputFileSize;
2308    }
2309
2310    @Override
2311    public List<Path> getCommittedFiles() {
2312      return committedFiles;
2313    }
2314
2315    /**
2316     * Similar to commit, but called in secondary region replicas for replaying the
2317     * flush cache from primary region. Adds the new files to the store, and drops the
2318     * snapshot depending on dropMemstoreSnapshot argument.
2319     * @param fileNames names of the flushed files
2320     * @param dropMemstoreSnapshot whether to drop the prepared memstore snapshot
2321     * @throws IOException
2322     */
2323    @Override
2324    public void replayFlush(List<String> fileNames, boolean dropMemstoreSnapshot)
2325        throws IOException {
2326      List<HStoreFile> storeFiles = new ArrayList<>(fileNames.size());
2327      for (String file : fileNames) {
2328        // open the file as a store file (hfile link, etc)
2329        StoreFileInfo storeFileInfo = fs.getStoreFileInfo(getColumnFamilyName(), file);
2330        HStoreFile storeFile = createStoreFileAndReader(storeFileInfo);
2331        storeFiles.add(storeFile);
2332        HStore.this.storeSize.addAndGet(storeFile.getReader().length());
2333        HStore.this.totalUncompressedBytes
2334            .addAndGet(storeFile.getReader().getTotalUncompressedBytes());
2335        if (LOG.isInfoEnabled()) {
2336          LOG.info("Region: " + HStore.this.getRegionInfo().getEncodedName() +
2337            " added " + storeFile + ", entries=" + storeFile.getReader().getEntries() +
2338              ", sequenceid=" + +storeFile.getReader().getSequenceID() + ", filesize="
2339              + TraditionalBinaryPrefix.long2String(storeFile.getReader().length(), "", 1));
2340        }
2341      }
2342
2343      long snapshotId = -1; // -1 means do not drop
2344      if (dropMemstoreSnapshot && snapshot != null) {
2345        snapshotId = snapshot.getId();
2346        snapshot.close();
2347      }
2348      HStore.this.updateStorefiles(storeFiles, snapshotId);
2349    }
2350
2351    /**
2352     * Abort the snapshot preparation. Drops the snapshot if any.
2353     * @throws IOException
2354     */
2355    @Override
2356    public void abort() throws IOException {
2357      if (snapshot != null) {
2358        //We need to close the snapshot when aborting, otherwise, the segment scanner
2359        //won't be closed. If we are using MSLAB, the chunk referenced by those scanners
2360        //can't be released, thus memory leak
2361        snapshot.close();
2362        HStore.this.updateStorefiles(Collections.emptyList(), snapshot.getId());
2363      }
2364    }
2365  }
2366
2367  @Override
2368  public boolean needsCompaction() {
2369    List<HStoreFile> filesCompactingClone = null;
2370    synchronized (filesCompacting) {
2371      filesCompactingClone = Lists.newArrayList(filesCompacting);
2372    }
2373    return this.storeEngine.needsCompaction(filesCompactingClone);
2374  }
2375
2376  /**
2377   * Used for tests.
2378   * @return cache configuration for this Store.
2379   */
2380  @VisibleForTesting
2381  public CacheConfig getCacheConfig() {
2382    return this.cacheConf;
2383  }
2384
2385  public static final long FIXED_OVERHEAD =
2386      ClassSize.align(ClassSize.OBJECT + (27 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_LONG)
2387              + (6 * Bytes.SIZEOF_INT) + (2 * Bytes.SIZEOF_BOOLEAN));
2388
2389  public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD
2390      + ClassSize.OBJECT + ClassSize.REENTRANT_LOCK
2391      + ClassSize.CONCURRENT_SKIPLISTMAP
2392      + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + ClassSize.OBJECT
2393      + ScanInfo.FIXED_OVERHEAD);
2394
2395  @Override
2396  public long heapSize() {
2397    MemStoreSize memstoreSize = this.memstore.size();
2398    return DEEP_OVERHEAD + memstoreSize.getHeapSize();
2399  }
2400
2401  @Override
2402  public CellComparator getComparator() {
2403    return comparator;
2404  }
2405
2406  public ScanInfo getScanInfo() {
2407    return scanInfo;
2408  }
2409
2410  /**
2411   * Set scan info, used by test
2412   * @param scanInfo new scan info to use for test
2413   */
2414  void setScanInfo(ScanInfo scanInfo) {
2415    this.scanInfo = scanInfo;
2416  }
2417
2418  @Override
2419  public boolean hasTooManyStoreFiles() {
2420    return getStorefilesCount() > this.blockingFileCount;
2421  }
2422
2423  @Override
2424  public long getFlushedCellsCount() {
2425    return flushedCellsCount.get();
2426  }
2427
2428  @Override
2429  public long getFlushedCellsSize() {
2430    return flushedCellsSize.get();
2431  }
2432
2433  @Override
2434  public long getFlushedOutputFileSize() {
2435    return flushedOutputFileSize.get();
2436  }
2437
2438  @Override
2439  public long getCompactedCellsCount() {
2440    return compactedCellsCount.get();
2441  }
2442
2443  @Override
2444  public long getCompactedCellsSize() {
2445    return compactedCellsSize.get();
2446  }
2447
2448  @Override
2449  public long getMajorCompactedCellsCount() {
2450    return majorCompactedCellsCount.get();
2451  }
2452
2453  @Override
2454  public long getMajorCompactedCellsSize() {
2455    return majorCompactedCellsSize.get();
2456  }
2457
2458  /**
2459   * Returns the StoreEngine that is backing this concrete implementation of Store.
2460   * @return Returns the {@link StoreEngine} object used internally inside this HStore object.
2461   */
2462  @VisibleForTesting
2463  public StoreEngine<?, ?, ?, ?> getStoreEngine() {
2464    return this.storeEngine;
2465  }
2466
2467  protected OffPeakHours getOffPeakHours() {
2468    return this.offPeakHours;
2469  }
2470
2471  /**
2472   * {@inheritDoc}
2473   */
2474  @Override
2475  public void onConfigurationChange(Configuration conf) {
2476    this.conf = new CompoundConfiguration()
2477            .add(conf)
2478            .addBytesMap(family.getValues());
2479    this.storeEngine.compactionPolicy.setConf(conf);
2480    this.offPeakHours = OffPeakHours.getInstance(conf);
2481  }
2482
2483  /**
2484   * {@inheritDoc}
2485   */
2486  @Override
2487  public void registerChildren(ConfigurationManager manager) {
2488    // No children to register
2489  }
2490
2491  /**
2492   * {@inheritDoc}
2493   */
2494  @Override
2495  public void deregisterChildren(ConfigurationManager manager) {
2496    // No children to deregister
2497  }
2498
2499  @Override
2500  public double getCompactionPressure() {
2501    return storeEngine.getStoreFileManager().getCompactionPressure();
2502  }
2503
2504  @Override
2505  public boolean isPrimaryReplicaStore() {
2506    return getRegionInfo().getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID;
2507  }
2508
2509  /**
2510   * Sets the store up for a region level snapshot operation.
2511   * @see #postSnapshotOperation()
2512   */
2513  public void preSnapshotOperation() {
2514    archiveLock.lock();
2515  }
2516
2517  /**
2518   * Perform tasks needed after the completion of snapshot operation.
2519   * @see #preSnapshotOperation()
2520   */
2521  public void postSnapshotOperation() {
2522    archiveLock.unlock();
2523  }
2524
2525  /**
2526   * Closes and archives the compacted files under this store
2527   */
2528  public synchronized void closeAndArchiveCompactedFiles() throws IOException {
2529    closeAndArchiveCompactedFiles(false);
2530  }
2531
2532  @VisibleForTesting
2533  public synchronized void closeAndArchiveCompactedFiles(boolean storeClosing) throws IOException {
2534    // ensure other threads do not attempt to archive the same files on close()
2535    archiveLock.lock();
2536    try {
2537      lock.readLock().lock();
2538      Collection<HStoreFile> copyCompactedfiles = null;
2539      try {
2540        Collection<HStoreFile> compactedfiles =
2541            this.getStoreEngine().getStoreFileManager().getCompactedfiles();
2542        if (CollectionUtils.isNotEmpty(compactedfiles)) {
2543          // Do a copy under read lock
2544          copyCompactedfiles = new ArrayList<>(compactedfiles);
2545        } else {
2546          LOG.trace("No compacted files to archive");
2547        }
2548      } finally {
2549        lock.readLock().unlock();
2550      }
2551      if (CollectionUtils.isNotEmpty(copyCompactedfiles)) {
2552        removeCompactedfiles(copyCompactedfiles, storeClosing);
2553      }
2554    } finally {
2555      archiveLock.unlock();
2556    }
2557  }
2558
2559  /**
2560   * Archives and removes the compacted files
2561   * @param compactedfiles The compacted files in this store that are not active in reads
2562   * @throws IOException
2563   */
2564  private void removeCompactedfiles(Collection<HStoreFile> compactedfiles, boolean storeClosing)
2565      throws IOException {
2566    final List<HStoreFile> filesToRemove = new ArrayList<>(compactedfiles.size());
2567    for (final HStoreFile file : compactedfiles) {
2568      synchronized (file) {
2569        try {
2570          StoreFileReader r = file.getReader();
2571          if (r == null) {
2572            LOG.debug("The file {} was closed but still not archived", file);
2573            filesToRemove.add(file);
2574            continue;
2575          }
2576
2577          //Compacted files in the list should always be marked compacted away. In the event
2578          //they're contradicting in order to guarantee data consistency
2579          //should we choose one and ignore the other?
2580          if (storeClosing && !file.isCompactedAway()) {
2581            String msg =
2582                "Region closing but StoreFile is in compacted list but not compacted away: " +
2583                file.getPath();
2584            throw new IllegalStateException(msg);
2585          }
2586
2587          //If store is closing we're ignoring any references to keep things consistent
2588          //and remove compacted storefiles from the region directory
2589          if (file.isCompactedAway() && (!file.isReferencedInReads() || storeClosing)) {
2590            if (storeClosing && file.isReferencedInReads()) {
2591              LOG.warn("Region closing but StoreFile still has references: file={}, refCount={}",
2592                  file.getPath(), r.getRefCount());
2593            }
2594            // Even if deleting fails we need not bother as any new scanners won't be
2595            // able to use the compacted file as the status is already compactedAway
2596            LOG.trace("Closing and archiving the file {}", file);
2597            r.close(true);
2598            file.closeStreamReaders(true);
2599            // Just close and return
2600            filesToRemove.add(file);
2601          } else {
2602            LOG.info("Can't archive compacted file " + file.getPath()
2603                + " because of either isCompactedAway=" + file.isCompactedAway()
2604                + " or file has reference, isReferencedInReads=" + file.isReferencedInReads()
2605                + ", refCount=" + r.getRefCount() + ", skipping for now.");
2606          }
2607        } catch (Exception e) {
2608          String msg = "Exception while trying to close the compacted store file " +
2609              file.getPath();
2610          if (storeClosing) {
2611            msg = "Store is closing. " + msg;
2612          }
2613          LOG.error(msg, e);
2614          //if we get an exception let caller know so it can abort the server
2615          if (storeClosing) {
2616            throw new IOException(msg, e);
2617          }
2618        }
2619      }
2620    }
2621    if (this.isPrimaryReplicaStore()) {
2622      // Only the primary region is allowed to move the file to archive.
2623      // The secondary region does not move the files to archive. Any active reads from
2624      // the secondary region will still work because the file as such has active readers on it.
2625      if (!filesToRemove.isEmpty()) {
2626        LOG.debug("Moving the files {} to archive", filesToRemove);
2627        // Only if this is successful it has to be removed
2628        try {
2629          this.fs.removeStoreFiles(this.getColumnFamilyDescriptor().getNameAsString(), filesToRemove);
2630        } catch (FailedArchiveException fae) {
2631          // Even if archiving some files failed, we still need to clear out any of the
2632          // files which were successfully archived.  Otherwise we will receive a
2633          // FileNotFoundException when we attempt to re-archive them in the next go around.
2634          Collection<Path> failedFiles = fae.getFailedFiles();
2635          Iterator<HStoreFile> iter = filesToRemove.iterator();
2636          while (iter.hasNext()) {
2637            if (failedFiles.contains(iter.next().getPath())) {
2638              iter.remove();
2639            }
2640          }
2641          if (!filesToRemove.isEmpty()) {
2642            clearCompactedfiles(filesToRemove);
2643          }
2644          throw fae;
2645        }
2646      }
2647    }
2648    if (!filesToRemove.isEmpty()) {
2649      // Clear the compactedfiles from the store file manager
2650      clearCompactedfiles(filesToRemove);
2651    }
2652  }
2653
2654  public Long preFlushSeqIDEstimation() {
2655    return memstore.preFlushSeqIDEstimation();
2656  }
2657
2658  @Override
2659  public boolean isSloppyMemStore() {
2660    return this.memstore.isSloppy();
2661  }
2662
2663  private void clearCompactedfiles(List<HStoreFile> filesToRemove) throws IOException {
2664    LOG.trace("Clearing the compacted file {} from this store", filesToRemove);
2665    try {
2666      lock.writeLock().lock();
2667      this.getStoreEngine().getStoreFileManager().removeCompactedFiles(filesToRemove);
2668    } finally {
2669      lock.writeLock().unlock();
2670    }
2671  }
2672
2673  public int getCurrentParallelPutCount() {
2674    return currentParallelPutCount.get();
2675  }
2676}