This is an automated email from the ASF dual-hosted git repository.

kturner pushed a commit to branch 2.1
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/2.1 by this push:
     new 08628c016b Updates DefaultCompactionPlanner to honor table.file.max 
prop (#4127)
08628c016b is described below

commit 08628c016bdc8597de973cf3f4fe04907a43fc7d
Author: Keith Turner <ktur...@apache.org>
AuthorDate: Thu Jan 4 16:38:09 2024 -0800

    Updates DefaultCompactionPlanner to honor table.file.max prop (#4127)
    
    This change updates the DefaultCompactionPlanner to honor the
    table.file.max property.  When a tablet has more files than is
    configured in table.file.max and its not compacting, it will try to
    find a compaction ratio that will cause a compaction.
    
    Using a lower compaction ratio will find files of a similar size to
    compact. This avoids the problem with deprecated DefautlCompactionStrategy
    where it may select a 1K and 1G file for compaction because it selected
    the N smallest files that would bring the tablet under the
    table.file.max limit.
---
 .../org/apache/accumulo/core/conf/Property.java    |   8 +-
 .../spi/compaction/DefaultCompactionPlanner.java   | 109 +++++++++++-
 .../compaction/DefaultCompactionPlannerTest.java   | 196 ++++++++++++++++++++-
 3 files changed, 304 insertions(+), 9 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java 
b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index c65545dc77..f594984a77 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -1130,8 +1130,12 @@ public enum Property {
           + " adjusting this property you may want to consider adjusting"
           + " table.compaction.major.ratio also. Setting this property to 0 
will make"
           + " it default to tserver.scan.files.open.max-1, this will prevent a 
tablet"
-          + " from having more RFiles than can be opened. Setting this 
property low may"
-          + " throttle ingest and increase query performance.",
+          + " from having more RFiles than can be opened. Prior to 2.1.0 this 
property"
+          + " was used to trigger merging minor compactions, but merging minor 
compactions"
+          + " were removed in 2.1.0. Now this property is only used by the"
+          + " DefaultCompactionStrategy and the DefaultCompactionPlanner."
+          + " The DefaultCompactionPlanner started using this property in 
2.1.3, before"
+          + " that it did not use the property.",
       "1.4.0"),
   TABLE_FILE_SUMMARY_MAX_SIZE("table.file.summary.maxSize", "256k", 
PropertyType.BYTES,
       "The maximum size summary that will be stored. The number of RFiles that"
diff --git 
a/core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java
 
b/core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java
index 9385806831..aae0591567 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java
@@ -32,6 +32,7 @@ import java.util.Set;
 import org.apache.accumulo.core.client.admin.compaction.CompactableFile;
 import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.spi.common.ServiceEnvironment;
 import org.apache.accumulo.core.util.compaction.CompactionJobPrioritizer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -108,6 +109,21 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
  * number of files that will be included in a single compaction.
  * </ul>
  *
+ * <p>
+ * Starting with Accumulo 2.1.3, this plugin will use the table config option
+ * {@code "table.file.max"}. When the following four conditions are met, then 
this plugin will try
+ * to find a lower compaction ratio that will result in a compaction:
+ * <ol>
+ * <li>When a tablet has no compactions running</li>
+ * <li>Its number of files exceeds table.file.max</li>
+ * <li>System compactions are not finding anything to compact</li>
+ * <li>No files are selected for user compaction</li>
+ * </ol>
+ * For example, given a tablet with 20 files, and table.file.max is 15 and no 
compactions are
+ * planned. If the compaction ratio is set to 3, then this plugin will find 
the largest compaction
+ * ratio less than 3 that results in a compaction.
+ *
+ *
  * @since 2.1.0
  * @see org.apache.accumulo.core.spi.compaction
  */
@@ -290,12 +306,25 @@ public class DefaultCompactionPlanner implements 
CompactionPlanner {
         group = Set.of();
       }
 
-      if (group.isEmpty()
-          && (params.getKind() == CompactionKind.USER || params.getKind() == 
CompactionKind.SELECTOR
-              || params.getKind() == CompactionKind.CHOP)
-          && params.getRunningCompactions().stream()
-              .noneMatch(job -> job.getKind() == params.getKind())) {
-        group = findMaximalRequiredSetToCompact(params.getCandidates(), 
maxFilesToCompact);
+      if (group.isEmpty()) {
+
+        if ((params.getKind() == CompactionKind.USER || params.getKind() == 
CompactionKind.SELECTOR
+            || params.getKind() == CompactionKind.CHOP)
+            && params.getRunningCompactions().stream()
+                .noneMatch(job -> job.getKind() == params.getKind())) {
+          group = findMaximalRequiredSetToCompact(params.getCandidates(), 
maxFilesToCompact);
+        } else if (params.getKind() == CompactionKind.SYSTEM
+            && params.getRunningCompactions().isEmpty()
+            && params.getAll().size() == params.getCandidates().size()) {
+          int maxTabletFiles = getMaxTabletFiles(
+              
params.getServiceEnvironment().getConfiguration(params.getTableId()));
+          if (params.getAll().size() > maxTabletFiles) {
+            // The tablet is above its max files, there are no compactions 
running, all files are
+            // candidates for a system compaction, and no files were found to 
compact. Attempt to
+            // find a set of files to compact by lowering the compaction ratio.
+            group = findFilesToCompactWithLowerRatio(params, maxSizeToCompact, 
maxTabletFiles);
+          }
+        }
       }
 
       if (group.isEmpty()) {
@@ -312,6 +341,74 @@ public class DefaultCompactionPlanner implements 
CompactionPlanner {
     }
   }
 
+  static int getMaxTabletFiles(ServiceEnvironment.Configuration configuration) 
{
+    int maxTabletFiles = 
Integer.parseInt(configuration.get(Property.TABLE_FILE_MAX.getKey()));
+    if (maxTabletFiles <= 0) {
+      maxTabletFiles =
+          
Integer.parseInt(configuration.get(Property.TSERV_SCAN_MAX_OPENFILES.getKey())) 
- 1;
+    }
+    return maxTabletFiles;
+  }
+
+  /**
+   * Searches for the highest compaction ratio that is less than the 
configured ratio that will
+   * lower the number of files.
+   */
+  private Collection<CompactableFile> 
findFilesToCompactWithLowerRatio(PlanningParameters params,
+      long maxSizeToCompact, int maxTabletFiles) {
+    double lowRatio = 1.0;
+    double highRatio = params.getRatio();
+
+    Preconditions.checkArgument(highRatio >= lowRatio);
+
+    var candidates = Set.copyOf(params.getCandidates());
+    Collection<CompactableFile> found = Set.of();
+
+    int goalCompactionSize = candidates.size() - maxTabletFiles + 1;
+    if (goalCompactionSize > maxFilesToCompact) {
+      // The tablet is way over max tablet files, so multiple compactions will 
be needed. Therefore,
+      // do not set a goal size for this compaction and find the largest 
compaction ratio that will
+      // compact some set of files.
+      goalCompactionSize = 0;
+    }
+
+    // Do a binary search of the compaction ratios.
+    while (highRatio - lowRatio > .1) {
+      double ratioToCheck = (highRatio - lowRatio) / 2 + lowRatio;
+
+      // This is continually resorting the list of files in the following 
call, could optimize this
+      var filesToCompact =
+          findDataFilesToCompact(candidates, ratioToCheck, maxFilesToCompact, 
maxSizeToCompact);
+
+      log.trace("Tried ratio {} and found {} {} {}", ratioToCheck, 
filesToCompact,
+          filesToCompact.size() >= goalCompactionSize, goalCompactionSize);
+
+      if (filesToCompact.isEmpty() || filesToCompact.size() < 
goalCompactionSize) {
+        highRatio = ratioToCheck;
+      } else {
+        lowRatio = ratioToCheck;
+        found = filesToCompact;
+      }
+    }
+
+    if (found.isEmpty() && lowRatio == 1.0) {
+      // in this case the data must be really skewed, operator intervention 
may be needed.
+      log.warn(
+          "Attempted to lower compaction ration from {} to {} for {} because 
there are {} files "
+              + "and the max tablet files is {}, however no set of files to 
compact were found.",
+          params.getRatio(), highRatio, params.getTableId(), 
params.getCandidates().size(),
+          maxTabletFiles);
+    }
+
+    log.info(
+        "For {} found {} files to compact lowering compaction ratio from {} to 
{} because the tablet "
+            + "exceeded {} files, it had {}",
+        params.getTableId(), found.size(), params.getRatio(), lowRatio, 
maxTabletFiles,
+        params.getCandidates().size());
+
+    return found;
+  }
+
   private static short createPriority(PlanningParameters params,
       Collection<CompactableFile> group) {
     return CompactionJobPrioritizer.createPriority(params.getKind(), 
params.getAll().size(),
diff --git 
a/core/src/test/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlannerTest.java
 
b/core/src/test/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlannerTest.java
index ab2003841b..302106fc95 100644
--- 
a/core/src/test/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlannerTest.java
+++ 
b/core/src/test/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlannerTest.java
@@ -25,10 +25,13 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.net.URI;
 import java.net.URISyntaxException;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
 import java.util.stream.Collectors;
 
@@ -44,6 +47,7 @@ import 
org.apache.accumulo.core.spi.common.ServiceEnvironment.Configuration;
 import org.apache.accumulo.core.spi.compaction.CompactionPlan.Builder;
 import org.apache.accumulo.core.util.ConfigurationImpl;
 import org.apache.accumulo.core.util.compaction.CompactionExecutorIdImpl;
+import org.apache.accumulo.core.util.compaction.CompactionJobImpl;
 import org.apache.accumulo.core.util.compaction.CompactionPlanImpl;
 import org.apache.accumulo.core.util.compaction.CompactionPlannerInitParams;
 import org.apache.accumulo.core.util.compaction.CompactionServicesConfig;
@@ -52,6 +56,8 @@ import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Preconditions;
+
 public class DefaultCompactionPlannerTest {
 
   private static <T> T getOnlyElement(Collection<T> c) {
@@ -464,6 +470,147 @@ public class DefaultCompactionPlannerTest {
     assertTrue(e.getMessage().contains("maxSize"), "Error message didn't 
contain maxSize");
   }
 
+  // Test cases where a tablet has more than table.file.max files, but no 
files were found using the
+  // compaction ratio. The planner should try to find the highest ratio that 
will result in a
+  // compaction.
+  @Test
+  public void testMaxTabletFiles() {
+    String executors = "[{'name':'small','type': 
'internal','maxSize':'32M','numThreads':1},"
+        + "{'name':'medium','type': 
'internal','maxSize':'128M','numThreads':2},"
+        + "{'name':'large','type': 'internal','numThreads':3}]";
+
+    Map<String,String> overrides = new HashMap<>();
+    overrides.put(Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + 
"cs1.planner.opts.maxOpen",
+        "10");
+    overrides.put(Property.TABLE_FILE_MAX.getKey(), "7");
+    var conf = new 
ConfigurationImpl(SiteConfiguration.empty().withOverrides(overrides).build());
+
+    // For this case need to compact three files and the highest ratio that 
achieves that is 1.8
+    var planner = createPlanner(conf, executors);
+    var all = createCFs(1000, 1.1, 1.9, 1.8, 1.6, 1.3, 1.4, 1.3, 1.2, 1.1);
+    var params = createPlanningParams(all, all, Set.of(), 3, 
CompactionKind.SYSTEM, conf);
+    var plan = planner.makePlan(params);
+    var job = getOnlyElement(plan.getJobs());
+    assertEquals(createCFs(1000, 1.1, 1.9, 1.8), job.getFiles());
+
+    // For this case need to compact two files and the highest ratio that 
achieves that is 2.9
+    all = createCFs(1000, 2, 2.9, 2.8, 2.7, 2.6, 2.5, 2.4, 2.3);
+    params = createPlanningParams(all, all, Set.of(), 3, 
CompactionKind.SYSTEM, conf);
+    plan = planner.makePlan(params);
+    job = getOnlyElement(plan.getJobs());
+    assertEquals(createCFs(1000, 2, 2.9), job.getFiles());
+
+    all =
+        createCFs(1000, 1.1, 2.89, 2.85, 2.7, 2.3, 2.9, 2.8, 2, 2, 2, 2, 2, 2, 
2, 2, 2, 2, 2, 2, 2);
+    params = createPlanningParams(all, all, Set.of(), 3, 
CompactionKind.SYSTEM, conf);
+    plan = planner.makePlan(params);
+    job = getOnlyElement(plan.getJobs());
+    assertEquals(createCFs(1000, 1.1, 2.89, 2.85, 2.7, 2.3, 2.9), 
job.getFiles());
+
+    all = createCFs(1000, 1.1, 1.2, 1.3, 1.4, 1.5, 1.6, 1.7, 1.8, 1.9, 1.1);
+    params = createPlanningParams(all, all, Set.of(), 3, 
CompactionKind.SYSTEM, conf);
+    plan = planner.makePlan(params);
+    job = getOnlyElement(plan.getJobs());
+    assertEquals(createCFs(1000, 1.1, 1.2, 1.3, 1.4, 1.5, 1.6, 1.7, 1.8, 1.9), 
job.getFiles());
+
+    // In this case the tablet can not be brought below the max files limit in 
a single compaction,
+    // so it should find the highest ratio to compact
+    for (var ratio : List.of(1.9, 2.0, 3.0, 4.0)) {
+      all = createCFs(1000, 1.9, 1.8, 1.7, 1.6, 1.5, 1.4, 1.5, 1.2, 1.1, 1.1, 
1.1, 1.1, 1.1, 1.1,
+          1.1, 1.1);
+      params = createPlanningParams(all, all, Set.of(), ratio, 
CompactionKind.SYSTEM, conf);
+      plan = planner.makePlan(params);
+      job = getOnlyElement(plan.getJobs());
+      assertEquals(createCFs(1000, 1.9), job.getFiles());
+    }
+
+    // In this case the tablet can be brought below the max limit in single 
compaction, so it should
+    // find this
+    all =
+        createCFs(1000, 1.9, 1.8, 1.7, 1.6, 1.5, 1.4, 1.5, 1.2, 1.1, 1.1, 1.1, 
1.1, 1.1, 1.1, 1.1);
+    params = createPlanningParams(all, all, Set.of(), 3, 
CompactionKind.SYSTEM, conf);
+    plan = planner.makePlan(params);
+    job = getOnlyElement(plan.getJobs());
+    assertEquals(createCFs(1000, 1.9, 1.8, 1.7, 1.6, 1.5, 1.4, 1.5, 1.2, 1.1), 
job.getFiles());
+
+    // each file is 10x the size of the file smaller than it
+    all = createCFs(10, 1.1, 1.1, 1.1, 1.1, 1.1, 1.1, 1.1, 1.1);
+    params = createPlanningParams(all, all, Set.of(), 3, 
CompactionKind.SYSTEM, conf);
+    plan = planner.makePlan(params);
+    job = getOnlyElement(plan.getJobs());
+    assertEquals(createCFs(10, 1.1, 1.1, 1.1, 1.1, 1.1, 1.1, 1.1, 1.1), 
job.getFiles());
+
+    // test with some files growing 20x, ensure those are not included
+    for (var ratio : List.of(1.9, 2.0, 3.0, 4.0)) {
+      all = createCFs(10, 1.05, 1.05, 1.25, 1.75, 1.25, 1.05, 1.05, 1.05);
+      params = createPlanningParams(all, all, Set.of(), ratio, 
CompactionKind.SYSTEM, conf);
+      plan = planner.makePlan(params);
+      job = getOnlyElement(plan.getJobs());
+      assertEquals(createCFs(10, 1.05, 1.05, 1.25, 1.75), job.getFiles());
+    }
+
+  }
+
+  @Test
+  public void testMaxTabletFilesNoCompaction() {
+    String executors = "[{'name':'small','type': 
'internal','maxSize':'32M','numThreads':1},"
+        + "{'name':'medium','type': 
'internal','maxSize':'128M','numThreads':2},"
+        + "{'name':'large','type': 
'internal','maxSize':'512M','numThreads':3}]";
+
+    Map<String,String> overrides = new HashMap<>();
+    overrides.put(Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + 
"cs1.planner.opts.maxOpen",
+        "10");
+    overrides.put(Property.TABLE_FILE_MAX.getKey(), "7");
+    var conf = new 
ConfigurationImpl(SiteConfiguration.empty().withOverrides(overrides).build());
+
+    // ensure that when a compaction would be over the max size limit that it 
is not planned
+    var planner = createPlanner(conf, executors);
+    var all = createCFs(1_000_000_000, 2, 2, 2, 2, 2, 2, 2);
+    var params = createPlanningParams(all, all, Set.of(), 3, 
CompactionKind.SYSTEM, conf);
+    var plan = planner.makePlan(params);
+
+    assertTrue(plan.getJobs().isEmpty());
+
+    // ensure when a compaction is running and we are over files max but below 
the compaction ratio
+    // that a compaction is not planned
+    all = createCFs(1_000, 2, 2, 2, 2, 2, 2, 2);
+    var job = new CompactionJobImpl((short) 1, 
CompactionExecutorIdImpl.externalId("ee1"),
+        createCFs("F1", "1000"), CompactionKind.SYSTEM, Optional.of(false));
+    params = createPlanningParams(all, all, Set.of(job), 3, 
CompactionKind.SYSTEM, conf);
+    plan = planner.makePlan(params);
+
+    assertTrue(plan.getJobs().isEmpty());
+
+    // a really bad situation, each file is 20 times the size of its smaller 
file. The algorithm
+    // does not search that for ratios that low.
+    all = createCFs(10, 1.05, 1.05, 1.05, 1.05, 1.05, 1.05, 1.05, 1.05);
+    params = createPlanningParams(all, all, Set.of(), 3, 
CompactionKind.SYSTEM, conf);
+    plan = planner.makePlan(params);
+    assertTrue(plan.getJobs().isEmpty());
+  }
+
+  // Test to ensure that plugin falls back from TABLE_FILE_MAX to 
TSERV_SCAN_MAX_OPENFILES
+  @Test
+  public void testMaxTableFilesFallback() {
+    String executors = "[{'name':'small','type': 
'internal','maxSize':'32M','numThreads':1},"
+        + "{'name':'medium','type': 
'internal','maxSize':'128M','numThreads':2},"
+        + "{'name':'large','type': 'internal','numThreads':3}]";
+
+    Map<String,String> overrides = new HashMap<>();
+    overrides.put(Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + 
"cs1.planner.opts.maxOpen",
+        "10");
+    overrides.put(Property.TABLE_FILE_MAX.getKey(), "0");
+    overrides.put(Property.TSERV_SCAN_MAX_OPENFILES.getKey(), "5");
+    var conf = new 
ConfigurationImpl(SiteConfiguration.empty().withOverrides(overrides).build());
+
+    var planner = createPlanner(conf, executors);
+    var all = createCFs(1000, 1.9, 1.8, 1.7, 1.6, 1.5, 1.4, 1.3, 1.2, 1.1);
+    var params = createPlanningParams(all, all, Set.of(), 3, 
CompactionKind.SYSTEM, conf);
+    var plan = planner.makePlan(params);
+    var job = getOnlyElement(plan.getJobs());
+    assertEquals(createCFs(1000, 1.9, 1.8, 1.7, 1.6, 1.5, 1.4), 
job.getFiles());
+  }
+
   private CompactionJob createJob(CompactionKind kind, Set<CompactableFile> 
all,
       Set<CompactableFile> files) {
     return new CompactionPlanImpl.BuilderImpl(kind, all, all)
@@ -471,6 +618,43 @@ public class DefaultCompactionPlannerTest {
         .build().getJobs().iterator().next();
   }
 
+  // Create a set of files whose sizes would require certain compaction ratios 
to compact
+  private Set<CompactableFile> createCFs(int initialSize, double... 
desiredRatios) {
+    List<String> pairs = new ArrayList<>();
+    pairs.add("F1");
+    pairs.add(initialSize + "");
+
+    double previousFileSizes = initialSize;
+
+    int i = 2;
+    for (double desiredRatio : desiredRatios) {
+      Preconditions.checkArgument(desiredRatio > 1.0);
+      Preconditions.checkArgument(desiredRatio <= i);
+
+      /*
+       * The compaction ratio formula is fileSize * ratio < fileSize + 
previousFileSizes. Solved the
+       * following equation to compute a file size given a desired ratio.
+       *
+       * fileSize * ratio = fileSize + previousFileSizes
+       *
+       * fileSize * ratio - fileSize = previousFileSizes
+       *
+       * fileSize * (ratio - 1) = previousFileSizes
+       *
+       * fileSize = previousFileSizes / (ratio - 1)
+       */
+
+      double fileSize = previousFileSizes / (desiredRatio - 1);
+      pairs.add("F" + i + "_" + desiredRatio);
+      pairs.add(Math.round(fileSize) + "");
+
+      previousFileSizes += fileSize;
+      i++;
+    }
+
+    return createCFs(pairs.toArray(new String[0]));
+  }
+
   private static Set<CompactableFile> createCFs(String... namesSizePairs) {
     Set<CompactableFile> files = new HashSet<>();
 
@@ -515,6 +699,12 @@ public class DefaultCompactionPlannerTest {
   private static CompactionPlanner.PlanningParameters 
createPlanningParams(Set<CompactableFile> all,
       Set<CompactableFile> candidates, Set<CompactionJob> compacting, double 
ratio,
       CompactionKind kind) {
+    return createPlanningParams(all, candidates, compacting, ratio, kind, 
defaultConf);
+  }
+
+  private static CompactionPlanner.PlanningParameters 
createPlanningParams(Set<CompactableFile> all,
+      Set<CompactableFile> candidates, Set<CompactionJob> compacting, double 
ratio,
+      CompactionKind kind, Configuration conf) {
     return new CompactionPlanner.PlanningParameters() {
 
       @Override
@@ -524,7 +714,11 @@ public class DefaultCompactionPlannerTest {
 
       @Override
       public ServiceEnvironment getServiceEnvironment() {
-        throw new UnsupportedOperationException();
+        ServiceEnvironment senv = 
EasyMock.createMock(ServiceEnvironment.class);
+        EasyMock.expect(senv.getConfiguration()).andReturn(conf).anyTimes();
+        
EasyMock.expect(senv.getConfiguration(TableId.of("42"))).andReturn(conf).anyTimes();
+        EasyMock.replay(senv);
+        return senv;
       }
 
       @Override

Reply via email to