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;
019
020import org.apache.hadoop.conf.Configuration;
021import org.apache.hadoop.hbase.regionserver.HStoreFile;
022import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
023import org.apache.hadoop.hbase.regionserver.StoreUtils;
024import org.apache.yetus.audience.InterfaceAudience;
025import org.slf4j.Logger;
026import org.slf4j.LoggerFactory;
027import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
028import org.apache.hbase.thirdparty.com.google.common.base.Predicate;
029import org.apache.hbase.thirdparty.com.google.common.collect.Collections2;
030import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
031
032/**
033 * An abstract compaction policy that select files on seq id order.
034 */
035@InterfaceAudience.Private
036public abstract class SortedCompactionPolicy extends CompactionPolicy {
037
038  private static final Logger LOG = LoggerFactory.getLogger(SortedCompactionPolicy.class);
039
040  public SortedCompactionPolicy(Configuration conf, StoreConfigInformation storeConfigInfo) {
041    super(conf, storeConfigInfo);
042  }
043
044  public List<HStoreFile> preSelectCompactionForCoprocessor(Collection<HStoreFile> candidates,
045      List<HStoreFile> filesCompacting) {
046    return getCurrentEligibleFiles(new ArrayList<>(candidates), filesCompacting);
047  }
048
049  /**
050   * @param candidateFiles candidate files, ordered from oldest to newest by seqId. We rely on
051   *   DefaultStoreFileManager to sort the files by seqId to guarantee contiguous compaction based
052   *   on seqId for data consistency.
053   * @return subset copy of candidate list that meets compaction criteria
054   */
055  public CompactionRequestImpl selectCompaction(Collection<HStoreFile> candidateFiles,
056      List<HStoreFile> filesCompacting, boolean isUserCompaction, boolean mayUseOffPeak,
057      boolean forceMajor) throws IOException {
058    // Preliminary compaction subject to filters
059    ArrayList<HStoreFile> candidateSelection = new ArrayList<>(candidateFiles);
060    // Stuck and not compacting enough (estimate). It is not guaranteed that we will be
061    // able to compact more if stuck and compacting, because ratio policy excludes some
062    // non-compacting files from consideration during compaction (see getCurrentEligibleFiles).
063    int futureFiles = filesCompacting.isEmpty() ? 0 : 1;
064    boolean mayBeStuck = (candidateFiles.size() - filesCompacting.size() + futureFiles)
065        >= storeConfigInfo.getBlockingFileCount();
066
067    candidateSelection = getCurrentEligibleFiles(candidateSelection, filesCompacting);
068    LOG.debug("Selecting compaction from " + candidateFiles.size() + " store files, " +
069        filesCompacting.size() + " compacting, " + candidateSelection.size() +
070        " eligible, " + storeConfigInfo.getBlockingFileCount() + " blocking");
071
072    // If we can't have all files, we cannot do major anyway
073    boolean isAllFiles = candidateFiles.size() == candidateSelection.size();
074    if (!(forceMajor && isAllFiles)) {
075      candidateSelection = skipLargeFiles(candidateSelection, mayUseOffPeak);
076      isAllFiles = candidateFiles.size() == candidateSelection.size();
077    }
078
079    // Try a major compaction if this is a user-requested major compaction,
080    // or if we do not have too many files to compact and this was requested as a major compaction
081    boolean isTryingMajor = (forceMajor && isAllFiles && isUserCompaction)
082        || (((forceMajor && isAllFiles) || shouldPerformMajorCompaction(candidateSelection))
083          && (candidateSelection.size() < comConf.getMaxFilesToCompact()));
084    // Or, if there are any references among the candidates.
085    boolean isAfterSplit = StoreUtils.hasReferences(candidateSelection);
086
087    CompactionRequestImpl result = createCompactionRequest(candidateSelection,
088      isTryingMajor || isAfterSplit, mayUseOffPeak, mayBeStuck);
089
090    ArrayList<HStoreFile> filesToCompact = Lists.newArrayList(result.getFiles());
091    removeExcessFiles(filesToCompact, isUserCompaction, isTryingMajor);
092    result.updateFiles(filesToCompact);
093
094    isAllFiles = (candidateFiles.size() == filesToCompact.size());
095    result.setOffPeak(!filesToCompact.isEmpty() && !isAllFiles && mayUseOffPeak);
096    result.setIsMajor(isTryingMajor && isAllFiles, isAllFiles);
097
098    return result;
099  }
100
101  protected abstract CompactionRequestImpl createCompactionRequest(
102      ArrayList<HStoreFile> candidateSelection, boolean tryingMajor, boolean mayUseOffPeak,
103      boolean mayBeStuck) throws IOException;
104
105  /**
106   * @param filesToCompact Files to compact. Can be null.
107   * @return True if we should run a major compaction.
108   */
109  @Override
110  public abstract boolean shouldPerformMajorCompaction(Collection<HStoreFile> filesToCompact)
111      throws IOException;
112
113  /**
114   * Used calculation jitter
115   */
116  private final Random random = new Random();
117
118  /**
119   * @param filesToCompact
120   * @return When to run next major compaction
121   */
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},
131     * that 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      // Synchronized to ensure one user of random instance at a time.
142      double rnd;
143      synchronized (this) {
144        this.random.setSeed(seed.getAsInt());
145        rnd = this.random.nextDouble();
146      }
147      long jitter = Math.round(period * jitterPct);
148      return period + jitter - Math.round(2L * jitter * rnd);
149    } else {
150      return 0L;
151    }
152  }
153
154  /**
155   * @param compactionSize Total size of some compaction
156   * @return whether this should be a large or small compaction
157   */
158  @Override
159  public boolean throttleCompaction(long compactionSize) {
160    return compactionSize > comConf.getThrottlePoint();
161  }
162
163  public abstract boolean needsCompaction(Collection<HStoreFile> storeFiles,
164      List<HStoreFile> filesCompacting);
165
166  protected ArrayList<HStoreFile> getCurrentEligibleFiles(ArrayList<HStoreFile> candidateFiles,
167      final List<HStoreFile> filesCompacting) {
168    // candidates = all storefiles not already in compaction queue
169    if (!filesCompacting.isEmpty()) {
170      // exclude all files older than the newest file we're currently
171      // compacting. this allows us to preserve contiguity (HBASE-2856)
172      HStoreFile last = filesCompacting.get(filesCompacting.size() - 1);
173      int idx = candidateFiles.indexOf(last);
174      Preconditions.checkArgument(idx != -1);
175      candidateFiles.subList(0, idx + 1).clear();
176    }
177    return candidateFiles;
178  }
179
180  /**
181   * @param candidates pre-filtrate
182   * @return filtered subset exclude all files above maxCompactSize
183   *   Also save all references. We MUST compact them
184   */
185  protected ArrayList<HStoreFile> skipLargeFiles(ArrayList<HStoreFile> candidates,
186    boolean mayUseOffpeak) {
187    int pos = 0;
188    while (pos < candidates.size() && !candidates.get(pos).isReference()
189      && (candidates.get(pos).getReader().length() > comConf.getMaxCompactSize(mayUseOffpeak))) {
190      ++pos;
191    }
192    if (pos > 0) {
193      LOG.debug("Some files are too large. Excluding " + pos
194          + " files from compaction candidates");
195      candidates.subList(0, pos).clear();
196    }
197    return candidates;
198  }
199
200  /**
201   * @param candidates pre-filtrate
202   * @return filtered subset exclude all bulk load files if configured
203   */
204  protected ArrayList<HStoreFile> filterBulk(ArrayList<HStoreFile> candidates) {
205    candidates.removeAll(Collections2.filter(candidates, new Predicate<HStoreFile>() {
206      @Override
207      public boolean apply(HStoreFile input) {
208        return input.excludeFromMinorCompaction();
209      }
210    }));
211    return candidates;
212  }
213
214  /**
215   * @param candidates pre-filtrate
216   */
217  protected void removeExcessFiles(ArrayList<HStoreFile> candidates,
218      boolean isUserCompaction, boolean isMajorCompaction) {
219    int excess = candidates.size() - comConf.getMaxFilesToCompact();
220    if (excess > 0) {
221      if (isMajorCompaction && isUserCompaction) {
222        LOG.debug("Warning, compacting more than " + comConf.getMaxFilesToCompact()
223            + " files because of a user-requested major compaction");
224      } else {
225        LOG.debug("Too many admissible files. Excluding " + excess
226            + " files from compaction candidates");
227        candidates.subList(comConf.getMaxFilesToCompact(), candidates.size()).clear();
228      }
229    }
230  }
231
232  /**
233   * @param candidates pre-filtrate
234   * @return filtered subset forget the compactionSelection if we don't have enough files
235   */
236  protected ArrayList<HStoreFile> checkMinFilesCriteria(ArrayList<HStoreFile> candidates,
237      int minFiles) {
238    if (candidates.size() < minFiles) {
239      if (LOG.isDebugEnabled()) {
240        LOG.debug("Not compacting files because we only have " + candidates.size() +
241            " files ready for compaction. Need " + minFiles + " to initiate.");
242      }
243      candidates.clear();
244    }
245    return candidates;
246  }
247}