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

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


The following commit(s) were added to refs/heads/main by this push:
     new 1501b02ff6 Looks ahead more when planning compactions
1501b02ff6 is described below

commit 1501b02ff6b8261ecab48ec5e07edca8d4eb5fe9
Author: Keith Turner <[email protected]>
AuthorDate: Thu Aug 7 22:49:14 2025 +0000

    Looks ahead more when planning compactions
    
    Adapted the 2.1 changes in #5588 and #5675 for main.
    
    Fixes #5634
---
 .../compaction/RatioBasedCompactionPlanner.java    |  15 +--
 .../RatioBasedCompactionPlannerTest.java           | 131 +++++++++++++++++++++
 2 files changed, 136 insertions(+), 10 deletions(-)

diff --git 
a/core/src/main/java/org/apache/accumulo/core/spi/compaction/RatioBasedCompactionPlanner.java
 
b/core/src/main/java/org/apache/accumulo/core/spi/compaction/RatioBasedCompactionPlanner.java
index 354e35f2d1..fb44d038e6 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/spi/compaction/RatioBasedCompactionPlanner.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/spi/compaction/RatioBasedCompactionPlanner.java
@@ -272,8 +272,7 @@ public class RatioBasedCompactionPlanner implements 
CompactionPlanner {
 
     // This set represents future files that will be produced by running 
compactions. If the optimal
     // set of files to compact is computed and contains one of these files, 
then it's optimal to
-    // wait
-    // for this compaction to finish.
+    // wait for this compaction to finish.
     Set<CompactableFile> expectedFiles = new HashSet<>();
     params.getRunningCompactions().stream().filter(job -> job.getKind() == 
params.getKind())
         .map(job -> getExpected(job.getFiles(), fakeFileGenerator))
@@ -286,14 +285,12 @@ public class RatioBasedCompactionPlanner implements 
CompactionPlanner {
     while (true) {
       var filesToCompact =
           findDataFilesToCompact(filesCopy, params.getRatio(), 
maxFilesToCompact, maxSizeToCompact);
-      if (!Collections.disjoint(filesToCompact, expectedFiles)) {
-        // the optimal set of files to compact includes the output of a 
running compaction, so lets
-        // wait for that running compaction to finish.
+      if (filesToCompact.isEmpty()) {
         break;
       }
 
-      if (filesToCompact.isEmpty()) {
-        break;
+      if (Collections.disjoint(filesToCompact, expectedFiles)) {
+        compactionJobs.add(filesToCompact);
       }
 
       filesCopy.removeAll(filesToCompact);
@@ -305,9 +302,7 @@ public class RatioBasedCompactionPlanner implements 
CompactionPlanner {
       Preconditions.checkState(expectedFiles.add(expectedFile));
       Preconditions.checkState(filesCopy.add(expectedFile));
 
-      compactionJobs.add(filesToCompact);
-
-      if (filesToCompact.size() < maxFilesToCompact) {
+      if (filesToCompact.size() < maxFilesToCompact && 
!compactionJobs.isEmpty()) {
         // Only continue looking for more compaction jobs when a set of files 
is found equals
         // maxFilesToCompact in size. When the files found is less than the 
max size its an
         // indication that the compaction ratio was no longer met and 
therefore it would be
diff --git 
a/core/src/test/java/org/apache/accumulo/core/spi/compaction/RatioBasedCompactionPlannerTest.java
 
b/core/src/test/java/org/apache/accumulo/core/spi/compaction/RatioBasedCompactionPlannerTest.java
index 69ac0918b0..bfed12f6de 100644
--- 
a/core/src/test/java/org/apache/accumulo/core/spi/compaction/RatioBasedCompactionPlannerTest.java
+++ 
b/core/src/test/java/org/apache/accumulo/core/spi/compaction/RatioBasedCompactionPlannerTest.java
@@ -62,6 +62,7 @@ import org.easymock.EasyMock;
 import org.junit.jupiter.api.Test;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Sets;
 import com.google.gson.JsonParseException;
 
 public class RatioBasedCompactionPlannerTest {
@@ -184,6 +185,136 @@ public class RatioBasedCompactionPlannerTest {
     assertEquals(ResourceGroupId.of("medium"), job.getGroup());
   }
 
+  @Test
+  public void testRunningCompactionLookAhead() {
+    String executors = 
"[{'group':'small','maxSize':'32M'},{'group':'medium','maxSize':'128M'},"
+        + "{'group':'large','maxSize':'512M'},{'group':'huge'}]";
+
+    var planner = createPlanner(defaultConf, executors);
+
+    int count = 0;
+
+    // create 4 files of size 10 as compacting
+    List<String> compactingString = new ArrayList<>();
+    for (int i = 0; i < 4; i++) {
+      compactingString.add("F" + count++);
+      compactingString.add(10 + "");
+    }
+
+    // create 4 files of size 100,1000,10_000, and 100_000 as the tablets files
+    List<String> candidateStrings = new ArrayList<>();
+    for (int size = 100; size < 1_000_000; size *= 10) {
+      for (int i = 0; i < 4; i++) {
+        candidateStrings.add("F" + count++);
+        candidateStrings.add(size + "");
+      }
+    }
+
+    var compacting = createCFs(compactingString.toArray(new String[0]));
+    var candidates = createCFs(candidateStrings.toArray(new String[0]));
+    var all = Sets.union(compacting, candidates);
+    var jobs = Set.of(createJob(CompactionKind.SYSTEM, all, compacting));
+    var params = createPlanningParams(all, candidates, jobs, 2, 
CompactionKind.SYSTEM);
+    var plan = planner.makePlan(params);
+
+    // The compaction running over the size 10 files would produce a file that 
would be used by a
+    // compaction over the size 100 files. A compaction over the size 100 
files would produce a file
+    // that would be used by a compaction over the size 1000 files. This 
should continue up the
+    // chain disqualifying all sets of files for compaction.
+    assertEquals(List.of(), plan.getJobs());
+  }
+
+  @Test
+  public void testRunningCompactionLookAhead2() {
+
+    var config = ServiceEnvironment.Configuration
+        .from(Map.of(prefix + "cs1.planner.opts.maxOpen", "10"), true);
+
+    String executors = 
"[{'group':'small','maxSize':'32M'},{'group':'medium','maxSize':'128M'},"
+        + "{'group':'large','maxSize':'512M'},{'group':'huge'}]";
+
+    var planner = createPlanner(config, executors);
+
+    int count = 0;
+
+    // create 10 files of size 11 as compacting
+    List<String> compactingString = new ArrayList<>();
+    for (int i = 0; i < 10; i++) {
+      compactingString.add("F" + count++);
+      compactingString.add(11 + "");
+    }
+
+    // create 10 files of size 11 as the tablets files
+    List<String> candidateStrings = new ArrayList<>();
+    for (int i = 0; i < 10; i++) {
+      candidateStrings.add("F" + count++);
+      candidateStrings.add(11 + "");
+    }
+
+    // create 17 files of size 100,1000, and 10_000 as the tablets files
+    for (int size = 100; size < 100_000; size *= 10) {
+      for (int i = 0; i < 17; i++) {
+        candidateStrings.add("F" + count++);
+        candidateStrings.add(size + "");
+      }
+    }
+
+    // create 5 files of 100_000 as the tablets files
+    for (int i = 0; i < 5; i++) {
+      candidateStrings.add("F" + count++);
+      candidateStrings.add(100_000 + "");
+    }
+
+    var compacting = createCFs(compactingString.toArray(new String[0]));
+    var candidates = createCFs(candidateStrings.toArray(new String[0]));
+    var all = Sets.union(compacting, candidates);
+    var jobs = Set.of(createJob(CompactionKind.SYSTEM, all, compacting));
+    var params = createPlanningParams(all, candidates, jobs, 2, 
CompactionKind.SYSTEM);
+    var plan = planner.makePlan(params);
+
+    // There are currently 20 files of size 11 of which 10 are compacting. The 
10 files that are
+    // compacting would produce a file with a projected size of 110. The file 
with a projected size
+    // of 110 would not be included in a compaction of the other 10 files of 
size 11, therefore its
+    // ok to compact the 10 files of size 11 and they should be found. 
Additionally, there are 10
+    // files of size 100 that can be compacted w/o including any of the files 
produced by compacting
+    // the files of size 11. So these file 10 files of size 100 should also be 
found.
+    assertEquals(2, plan.getJobs().size());
+    boolean saw11 = false;
+    boolean saw100 = false;
+    for (var job : plan.getJobs()) {
+      assertEquals(10, job.getFiles().size());
+      saw11 |= job.getFiles().stream().allMatch(f -> f.getEstimatedSize() == 
11);
+      saw100 |= job.getFiles().stream().allMatch(f -> f.getEstimatedSize() == 
100);
+    }
+    assertTrue(saw11 && saw100);
+
+    // try planning again incorporating the jobs returned from previous plan
+    var jobs2 = Sets.union(jobs, Set.copyOf(plan.getJobs()));
+    var candidates2 = new HashSet<>(candidates);
+    for (var job : plan.getJobs()) {
+      candidates2.removeAll(job.getFiles());
+    }
+    params = createPlanningParams(all, candidates2, jobs2, 2, 
CompactionKind.SYSTEM);
+    plan = planner.makePlan(params);
+
+    // Simulating multiple compactions forward, the next set of files that 
would not include files
+    // from any other projected or running compactions are 9 files of size 
10_000.
+    var job = getOnlyElement(plan.getJobs());
+    assertEquals(9, job.getFiles().size());
+    assertTrue(job.getFiles().stream().allMatch(f -> f.getEstimatedSize() == 
10_000));
+
+    // try planning again incorporating the job returned from previous plan
+    var jobs3 = Sets.union(jobs2, Set.copyOf(plan.getJobs()));
+    var candidates3 = new HashSet<>(candidates2);
+    candidates3.removeAll(job.getFiles());
+    params = createPlanningParams(all, candidates3, jobs3, 2, 
CompactionKind.SYSTEM);
+    plan = planner.makePlan(params);
+
+    // The 5 files of size 100_000 should not be found because it would be 
most optimal to compact
+    // those 5 files with the output of the compactions over the files of size 
10_000.
+    assertEquals(0, plan.getJobs().size());
+  }
+
   @Test
   public void testUserCompaction() {
     var config = ServiceEnvironment.Configuration

Reply via email to