001/**
002 * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
003 * agreements. See the NOTICE file distributed with this work for additional information regarding
004 * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
005 * "License"); you may not use this file except in compliance with the License. You may obtain a
006 * copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable
007 * law or agreed to in writing, software distributed under the License is distributed on an "AS IS"
008 * BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License
009 * for the specific language governing permissions and limitations under the License.
010 */
011package org.apache.hadoop.hbase.regionserver.compactions;
012
013import java.io.IOException;
014import java.util.ArrayList;
015import java.util.Collection;
016import java.util.List;
017import java.util.OptionalInt;
018import java.util.Random;
019import org.apache.hadoop.conf.Configuration;
020import org.apache.hadoop.hbase.regionserver.HStoreFile;
021import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
022import org.apache.hadoop.hbase.regionserver.StoreUtils;
023import org.apache.yetus.audience.InterfaceAudience;
024import org.slf4j.Logger;
025import org.slf4j.LoggerFactory;
026
027import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
028import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
029
030/**
031 * An abstract compaction policy that select files on seq id order.
032 */
033@InterfaceAudience.Private
034public abstract class SortedCompactionPolicy extends CompactionPolicy {
035
036  private static final Logger LOG = LoggerFactory.getLogger(SortedCompactionPolicy.class);
037
038  public SortedCompactionPolicy(Configuration conf, StoreConfigInformation storeConfigInfo) {
039    super(conf, storeConfigInfo);
040  }
041
042  public List<HStoreFile> preSelectCompactionForCoprocessor(Collection<HStoreFile> candidates,
043      List<HStoreFile> filesCompacting) {
044    return getCurrentEligibleFiles(new ArrayList<>(candidates), filesCompacting);
045  }
046
047  /**
048   * @param candidateFiles candidate files, ordered from oldest to newest by seqId. We rely on
049   *   DefaultStoreFileManager to sort the files by seqId to guarantee contiguous compaction based
050   *   on seqId for data consistency.
051   * @return subset copy of candidate list that meets compaction criteria
052   */
053  public CompactionRequestImpl selectCompaction(Collection<HStoreFile> candidateFiles,
054      List<HStoreFile> filesCompacting, boolean isUserCompaction, boolean mayUseOffPeak,
055      boolean forceMajor) throws IOException {
056    // Preliminary compaction subject to filters
057    ArrayList<HStoreFile> candidateSelection = new ArrayList<>(candidateFiles);
058    // Stuck and not compacting enough (estimate). It is not guaranteed that we will be
059    // able to compact more if stuck and compacting, because ratio policy excludes some
060    // non-compacting files from consideration during compaction (see getCurrentEligibleFiles).
061    int futureFiles = filesCompacting.isEmpty() ? 0 : 1;
062    boolean mayBeStuck = (candidateFiles.size() - filesCompacting.size() + futureFiles)
063        >= storeConfigInfo.getBlockingFileCount();
064
065    candidateSelection = getCurrentEligibleFiles(candidateSelection, filesCompacting);
066    LOG.debug("Selecting compaction from " + candidateFiles.size() + " store files, " +
067        filesCompacting.size() + " compacting, " + candidateSelection.size() +
068        " eligible, " + storeConfigInfo.getBlockingFileCount() + " blocking");
069
070    // If we can't have all files, we cannot do major anyway
071    boolean isAllFiles = candidateFiles.size() == candidateSelection.size();
072    if (!(forceMajor && isAllFiles)) {
073      candidateSelection = skipLargeFiles(candidateSelection, mayUseOffPeak);
074      isAllFiles = candidateFiles.size() == candidateSelection.size();
075    }
076
077    // Try a major compaction if this is a user-requested major compaction,
078    // or if we do not have too many files to compact and this was requested as a major compaction
079    boolean isTryingMajor = (forceMajor && isAllFiles && isUserCompaction)
080        || (((forceMajor && isAllFiles) || shouldPerformMajorCompaction(candidateSelection))
081          && (candidateSelection.size() < comConf.getMaxFilesToCompact()));
082    // Or, if there are any references among the candidates.
083    boolean isAfterSplit = StoreUtils.hasReferences(candidateSelection);
084
085    CompactionRequestImpl result = createCompactionRequest(candidateSelection,
086      isTryingMajor || isAfterSplit, mayUseOffPeak, mayBeStuck);
087
088    ArrayList<HStoreFile> filesToCompact = Lists.newArrayList(result.getFiles());
089    removeExcessFiles(filesToCompact, isUserCompaction, isTryingMajor);
090    result.updateFiles(filesToCompact);
091
092    isAllFiles = (candidateFiles.size() == filesToCompact.size());
093    result.setOffPeak(!filesToCompact.isEmpty() && !isAllFiles && mayUseOffPeak);
094    result.setIsMajor(isTryingMajor && isAllFiles, isAllFiles);
095
096    return result;
097  }
098
099  protected abstract CompactionRequestImpl createCompactionRequest(
100      ArrayList<HStoreFile> candidateSelection, boolean tryingMajor, boolean mayUseOffPeak,
101      boolean mayBeStuck) throws IOException;
102
103  /**
104   * @param filesToCompact Files to compact. Can be null.
105   * @return True if we should run a major compaction.
106   */
107  @Override
108  public abstract boolean shouldPerformMajorCompaction(Collection<HStoreFile> filesToCompact)
109      throws IOException;
110
111  /**
112   * Used calculation jitter
113   */
114  private final Random random = new Random();
115
116  /**
117   * @param filesToCompact
118   * @return When to run next major compaction
119   */
120  public long getNextMajorCompactTime(Collection<HStoreFile> filesToCompact) {
121    /** Default to {@link org.apache.hadoop.hbase.HConstants#DEFAULT_MAJOR_COMPACTION_PERIOD}. */
122    long period = comConf.getMajorCompactionPeriod();
123    if (period <= 0) {
124      return period;
125    }
126
127    /**
128     * Default to {@link org.apache.hadoop.hbase.HConstants#DEFAULT_MAJOR_COMPACTION_JITTER},
129     * that is, +/- 3.5 days (7 days * 0.5).
130     */
131    double jitterPct = comConf.getMajorCompactionJitter();
132    if (jitterPct <= 0) {
133      return period;
134    }
135
136    // deterministic jitter avoids a major compaction storm on restart
137    OptionalInt seed = StoreUtils.getDeterministicRandomSeed(filesToCompact);
138    if (seed.isPresent()) {
139      // Synchronized to ensure one user of random instance at a time.
140      double rnd;
141      synchronized (this) {
142        this.random.setSeed(seed.getAsInt());
143        rnd = this.random.nextDouble();
144      }
145      long jitter = Math.round(period * jitterPct);
146      return period + jitter - Math.round(2L * jitter * rnd);
147    } else {
148      return 0L;
149    }
150  }
151
152  /**
153   * @param compactionSize Total size of some compaction
154   * @return whether this should be a large or small compaction
155   */
156  @Override
157  public boolean throttleCompaction(long compactionSize) {
158    return compactionSize > comConf.getThrottlePoint();
159  }
160
161  public abstract boolean needsCompaction(Collection<HStoreFile> storeFiles,
162      List<HStoreFile> filesCompacting);
163
164  protected ArrayList<HStoreFile> getCurrentEligibleFiles(ArrayList<HStoreFile> candidateFiles,
165      final List<HStoreFile> filesCompacting) {
166    // candidates = all storefiles not already in compaction queue
167    if (!filesCompacting.isEmpty()) {
168      // exclude all files older than the newest file we're currently
169      // compacting. this allows us to preserve contiguity (HBASE-2856)
170      HStoreFile last = filesCompacting.get(filesCompacting.size() - 1);
171      int idx = candidateFiles.indexOf(last);
172      Preconditions.checkArgument(idx != -1);
173      candidateFiles.subList(0, idx + 1).clear();
174    }
175    return candidateFiles;
176  }
177
178  /**
179   * @param candidates pre-filtrate
180   * @return filtered subset exclude all files above maxCompactSize
181   *   Also save all references. We MUST compact them
182   */
183  protected ArrayList<HStoreFile> skipLargeFiles(ArrayList<HStoreFile> candidates,
184    boolean mayUseOffpeak) {
185    int pos = 0;
186    while (pos < candidates.size() && !candidates.get(pos).isReference()
187      && (candidates.get(pos).getReader().length() > comConf.getMaxCompactSize(mayUseOffpeak))) {
188      ++pos;
189    }
190    if (pos > 0) {
191      LOG.debug("Some files are too large. Excluding " + pos
192          + " files from compaction candidates");
193      candidates.subList(0, pos).clear();
194    }
195    return candidates;
196  }
197
198  /**
199   * @param candidates pre-filtrate
200   */
201  protected void filterBulk(ArrayList<HStoreFile> candidates) {
202    candidates.removeIf(HStoreFile::excludeFromMinorCompaction);
203  }
204
205  /**
206   * @param candidates pre-filtrate
207   */
208  protected void removeExcessFiles(ArrayList<HStoreFile> candidates,
209      boolean isUserCompaction, boolean isMajorCompaction) {
210    int excess = candidates.size() - comConf.getMaxFilesToCompact();
211    if (excess > 0) {
212      if (isMajorCompaction && isUserCompaction) {
213        LOG.debug("Warning, compacting more than " + comConf.getMaxFilesToCompact()
214            + " files because of a user-requested major compaction");
215      } else {
216        LOG.debug("Too many admissible files. Excluding " + excess
217            + " files from compaction candidates");
218        candidates.subList(comConf.getMaxFilesToCompact(), candidates.size()).clear();
219      }
220    }
221  }
222
223  /**
224   * @param candidates pre-filtrate
225   * @return filtered subset forget the compactionSelection if we don't have enough files
226   */
227  protected ArrayList<HStoreFile> checkMinFilesCriteria(ArrayList<HStoreFile> candidates,
228      int minFiles) {
229    if (candidates.size() < minFiles) {
230      if (LOG.isDebugEnabled()) {
231        LOG.debug("Not compacting files because we only have " + candidates.size() +
232            " files ready for compaction. Need " + minFiles + " to initiate.");
233      }
234      candidates.clear();
235    }
236    return candidates;
237  }
238}