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 e4f4727e73 Considers projected compactions when searching for files to 
compact (#5675)
e4f4727e73 is described below

commit e4f4727e735ce90a4e72999006c38e92d8b82cdb
Author: Keith Turner <ktur...@apache.org>
AuthorDate: Thu Aug 7 16:20:54 2025 -0400

    Considers projected compactions when searching for files to compact (#5675)
    
    When attempting to merge #5588 to main, found a bug with the change.
    The bug was that the code did not consider transitive compactions when
    searching past currently running compactions.  This bug would not cause
    any errors, it was just suboptimal. After these changes no compactions
    that would transitively include any smaller files will be planned.
    
    For a tablet that has 6 1M files compacting, 5 7M files and 4 40M files
    the code without this fix would find the 4 40M files to compact.
    However if we wait for the compaction of the 5 7M files (which is
    waiting on the compaction of the 6 1M files), then the 4 40M files would
    be compacted with the 41M file that would be produced. These changes
    would wait for that making the three compactions run sequentially so they
    can include each others output.
    
    A second test added in this change gives examples cases of compactions
    that can be found when there are running compaction.
---
 .../spi/compaction/DefaultCompactionPlanner.java   |  6 ++
 .../compaction/DefaultCompactionPlannerTest.java   | 94 +++++++++++++++++++---
 2 files changed, 90 insertions(+), 10 deletions(-)

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 38af298b6e..99bfd8472c 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
@@ -324,6 +324,12 @@ public class DefaultCompactionPlanner implements 
CompactionPlanner {
           var futureFile = getExpectedFile(group, nextExpected);
           Preconditions.checkState(expectedFiles.add(futureFile), "Unexpected 
duplicate %s in %s",
               futureFile, expectedFiles);
+          // Include this expected file in the set of files used for planning 
future compactions.
+          // This will cause any compaction that would include this file to be 
ignored. If a
+          // compaction would include this file, then it is best if the 
compactions run
+          // sequentially.
+          Preconditions.checkState(filesCopy.add(futureFile), "Unexpected 
duplicate %s in %s",
+              futureFile, filesCopy);
           // look for any compaction work in the remaining set of files
           group = findDataFilesToCompact(filesCopy, params.getRatio(), 
maxFilesToCompact,
               maxSizeToCompact);
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 04bd7baa09..45d6d09373 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
@@ -222,11 +222,72 @@ public class DefaultCompactionPlannerTest {
     var params = createPlanningParams(all, candidates, jobs, 2, 
CompactionKind.SYSTEM);
     var plan = planner.makePlan(params);
 
-    // the size 100 files should be excluded because the job running over size 
10 files will produce
-    // a file in their size range, so should see the 1000 size files planned 
for compaction
+    // 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 aconf = SiteConfiguration.empty()
+        .withOverrides(Map.of(
+            Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + 
"cs1.planner.opts.maxOpen", "10"))
+        .build();
+    ConfigurationImpl config = new ConfigurationImpl(aconf);
+
+    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},"
+        + "{'name':'huge','type': 'internal','numThreads':4}]";
+
+    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 10 files of size 
11, therefore its ok to
+    // compact the 10 files of size 11 and they should be found.
     var job = getOnlyElement(plan.getJobs());
-    assertEquals(4, job.getFiles().size());
-    assertTrue(job.getFiles().stream().allMatch(f -> f.getEstimatedSize() == 
1_000));
+    assertEquals(10, job.getFiles().size());
+    assertTrue(job.getFiles().stream().allMatch(f -> f.getEstimatedSize() == 
11));
 
     // try planning again incorporating the job returned from previous plan
     var jobs2 = Sets.union(jobs, Set.copyOf(plan.getJobs()));
@@ -235,12 +296,12 @@ public class DefaultCompactionPlannerTest {
     params = createPlanningParams(all, candidates2, jobs2, 2, 
CompactionKind.SYSTEM);
     plan = planner.makePlan(params);
 
-    // The two running jobs are over 10 and 1000 sized files. The jobs should 
exclude 100 and 10_000
-    // sized files because they would produce a file in those size ranges. 
This leaves the 100_000
-    // sized files available to compact.
+    // All 20 of the size 11 files are compacting, which would produce 2 files 
of size 110. There
+    // are 17 files of size 100, compacting 10 files of size 100 would not 
include the two projected
+    // files of size 110. Should find 10 files of size 100 to compact.
     job = getOnlyElement(plan.getJobs());
-    assertEquals(4, job.getFiles().size());
-    assertTrue(job.getFiles().stream().allMatch(f -> f.getEstimatedSize() == 
100_000));
+    assertEquals(10, job.getFiles().size());
+    assertTrue(job.getFiles().stream().allMatch(f -> f.getEstimatedSize() == 
100));
 
     // try planning again incorporating the job returned from previous plan
     var jobs3 = Sets.union(jobs2, Set.copyOf(plan.getJobs()));
@@ -249,7 +310,20 @@ public class DefaultCompactionPlannerTest {
     params = createPlanningParams(all, candidates3, jobs3, 2, 
CompactionKind.SYSTEM);
     plan = planner.makePlan(params);
 
-    // should find nothing to compact at this point
+    // 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.
+    job = getOnlyElement(plan.getJobs());
+    assertEquals(9, job.getFiles().size());
+    assertTrue(job.getFiles().stream().allMatch(f -> f.getEstimatedSize() == 
10_000));
+
+    var jobs4 = Sets.union(jobs3, Set.copyOf(plan.getJobs()));
+    var candidates4 = new HashSet<>(candidates3);
+    candidates4.removeAll(job.getFiles());
+    params = createPlanningParams(all, candidates4, jobs4, 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());
   }
 

Reply via email to