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