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.List;
023import org.apache.hadoop.conf.Configuration;
024import org.apache.hadoop.hbase.regionserver.HStoreFile;
025import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
026import org.apache.yetus.audience.InterfaceAudience;
027import org.slf4j.Logger;
028import org.slf4j.LoggerFactory;
029
030/**
031 * Class to pick which files if any to compact together. This class will search all possibilities
032 * for different and if it gets stuck it will choose the smallest set of files to compact.
033 */
034@InterfaceAudience.Private
035public class ExploringCompactionPolicy extends RatioBasedCompactionPolicy {
036  private static final Logger LOG = LoggerFactory.getLogger(ExploringCompactionPolicy.class);
037
038  /**
039   * Constructor for ExploringCompactionPolicy.
040   * @param conf            The configuration object
041   * @param storeConfigInfo An object to provide info about the store.
042   */
043  public ExploringCompactionPolicy(final Configuration conf,
044    final StoreConfigInformation storeConfigInfo) {
045    super(conf, storeConfigInfo);
046  }
047
048  @Override
049  protected final ArrayList<HStoreFile> applyCompactionPolicy(ArrayList<HStoreFile> candidates,
050    boolean mayUseOffPeak, boolean mightBeStuck) throws IOException {
051    return new ArrayList<>(applyCompactionPolicy(candidates, mightBeStuck, mayUseOffPeak,
052      comConf.getMinFilesToCompact(), comConf.getMaxFilesToCompact()));
053  }
054
055  public List<HStoreFile> applyCompactionPolicy(List<HStoreFile> candidates, boolean mightBeStuck,
056    boolean mayUseOffPeak, int minFiles, int maxFiles) {
057    final double currentRatio =
058      mayUseOffPeak ? comConf.getCompactionRatioOffPeak() : comConf.getCompactionRatio();
059
060    // Start off choosing nothing.
061    List<HStoreFile> bestSelection = new ArrayList<>(0);
062    List<HStoreFile> smallest = mightBeStuck ? new ArrayList<>(0) : null;
063    long bestSize = 0;
064    long smallestSize = Long.MAX_VALUE;
065
066    int opts = 0, optsInRatio = 0, bestStart = -1; // for debug logging
067    // Consider every starting place.
068    for (int start = 0; start < candidates.size(); start++) {
069      // Consider every different sub list permutation in between start and end with min files.
070      for (int currentEnd = start + minFiles - 1; currentEnd < candidates.size(); currentEnd++) {
071        List<HStoreFile> potentialMatchFiles = candidates.subList(start, currentEnd + 1);
072
073        // Sanity checks
074        if (potentialMatchFiles.size() < minFiles) {
075          continue;
076        }
077        if (potentialMatchFiles.size() > maxFiles) {
078          continue;
079        }
080
081        // Compute the total size of files that will
082        // have to be read if this set of files is compacted.
083        long size = getTotalStoreSize(potentialMatchFiles);
084
085        // Store the smallest set of files. This stored set of files will be used
086        // if it looks like the algorithm is stuck.
087        if (mightBeStuck && size < smallestSize) {
088          smallest = potentialMatchFiles;
089          smallestSize = size;
090        }
091
092        if (size > comConf.getMaxCompactSize(mayUseOffPeak)) {
093          continue;
094        }
095
096        ++opts;
097        if (
098          size >= comConf.getMinCompactSize() && !filesInRatio(potentialMatchFiles, currentRatio)
099        ) {
100          continue;
101        }
102
103        ++optsInRatio;
104        if (isBetterSelection(bestSelection, bestSize, potentialMatchFiles, size, mightBeStuck)) {
105          bestSelection = potentialMatchFiles;
106          bestSize = size;
107          bestStart = start;
108        }
109      }
110    }
111    if (bestSelection.isEmpty() && mightBeStuck) {
112      LOG.debug("Exploring compaction algorithm has selected " + smallest.size() + " files of size "
113        + smallestSize + " because the store might be stuck");
114      return new ArrayList<>(smallest);
115    }
116    LOG.debug(
117      "Exploring compaction algorithm has selected {}  files of size {} starting at "
118        + "candidate #{} after considering {} permutations with {} in ratio",
119      bestSelection.size(), bestSize, bestStart, opts, optsInRatio);
120    return new ArrayList<>(bestSelection);
121  }
122
123  /**
124   * Select at least one file in the candidates list to compact, through choosing files from the
125   * head to the index that the accumulation length larger the max compaction size. This method is a
126   * supplementary of the selectSimpleCompaction() method, aims to make sure at least one file can
127   * be selected to compact, for compactions like L0 files, which need to compact all files and as
128   * soon as possible.
129   */
130  public List<HStoreFile> selectCompactFiles(final List<HStoreFile> candidates, int maxFiles,
131    boolean isOffpeak) {
132    long selectedSize = 0L;
133    for (int end = 0; end < Math.min(candidates.size(), maxFiles); end++) {
134      selectedSize += candidates.get(end).getReader().length();
135      if (selectedSize >= comConf.getMaxCompactSize(isOffpeak)) {
136        return candidates.subList(0, end + 1);
137      }
138    }
139    return candidates;
140  }
141
142  private boolean isBetterSelection(List<HStoreFile> bestSelection, long bestSize,
143    List<HStoreFile> selection, long size, boolean mightBeStuck) {
144    if (mightBeStuck && bestSize > 0 && size > 0) {
145      // Keep the selection that removes most files for least size. That penaltizes adding
146      // large files to compaction, but not small files, so we don't become totally inefficient
147      // (might want to tweak that in future). Also, given the current order of looking at
148      // permutations, prefer earlier files and smaller selection if the difference is small.
149      final double REPLACE_IF_BETTER_BY = 1.05;
150      double thresholdQuality = ((double) bestSelection.size() / bestSize) * REPLACE_IF_BETTER_BY;
151      return thresholdQuality < ((double) selection.size() / size);
152    }
153    // Keep if this gets rid of more files. Or the same number of files for less io.
154    return selection.size() > bestSelection.size()
155      || (selection.size() == bestSelection.size() && size < bestSize);
156  }
157
158  /**
159   * Find the total size of a list of store files.
160   * @param potentialMatchFiles StoreFile list.
161   * @return Sum of StoreFile.getReader().length();
162   */
163  private long getTotalStoreSize(List<HStoreFile> potentialMatchFiles) {
164    return potentialMatchFiles.stream().mapToLong(sf -> sf.getReader().length()).sum();
165  }
166
167  /**
168   * Check that all files satisfy the constraint
169   *
170   * <pre>
171   * FileSize(i) <= ( Sum(0,N,FileSize(_)) - FileSize(i)) * Ratio.
172   * </pre>
173   *
174   * @param files        List of store files to consider as a compaction candidate.
175   * @param currentRatio The ratio to use.
176   * @return a boolean if these files satisfy the ratio constraints.
177   */
178  private boolean filesInRatio(List<HStoreFile> files, double currentRatio) {
179    if (files.size() < 2) {
180      return true;
181    }
182
183    long totalFileSize = getTotalStoreSize(files);
184
185    for (HStoreFile file : files) {
186      long singleFileSize = file.getReader().length();
187      long sumAllOtherFileSizes = totalFileSize - singleFileSize;
188
189      if (singleFileSize > sumAllOtherFileSizes * currentRatio) {
190        return false;
191      }
192    }
193    return true;
194  }
195}