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

commit 6dc1bcfa559be2fe8cab1b8e28552b44f7008957
Merge: 5583129435 08628c016b
Author: Keith Turner <ktur...@apache.org>
AuthorDate: Thu Jan 4 19:55:52 2024 -0500

    Merge branch '2.1'

 .../org/apache/accumulo/core/conf/Property.java    |   8 +-
 .../spi/compaction/DefaultCompactionPlanner.java   | 106 ++++++++++-
 .../compaction/DefaultCompactionPlannerTest.java   | 194 ++++++++++++++++++++-
 3 files changed, 300 insertions(+), 8 deletions(-)

diff --cc 
core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java
index 8f7969519a,aae0591567..eba25df062
--- 
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
@@@ -115,31 -105,26 +116,46 @@@ import edu.umd.cs.findbugs.annotations.
   *
   * Note that the use of 'external' requires that the CompactionCoordinator 
and at least one
   * Compactor for Queue1 is running.
 - * <li>{@code tserver.compaction.major.service.<service>.opts.maxOpen} This 
determines the maximum
 - * number of files that will be included in a single compaction.
 + * <li>{@code compaction.service.<service>.opts.maxOpen} This determines the 
maximum number of files
 + * that will be included in a single compaction.
 + * <li>{@code compaction.service.<service>.opts.queues} This is a json array 
of queue objects which
 + * have the following fields:
 + * <table>
 + * <caption>Default Compaction Planner Queue options</caption>
 + * <tr>
 + * <th>Field Name</th>
 + * <th>Description</th>
 + * </tr>
 + * <tr>
 + * <td>name</td>
 + * <td>name or alias of the queue (required)</td>
 + * </tr>
 + * <tr>
 + * <td>maxSize</td>
 + * <td>threshold sum of the input files (required for all but one of the 
configs)</td>
 + * </tr>
 + * </table>
 + * <br>
 + * This 'queues' object is used for defining external compaction queues 
without needing to use the
 + * thread-based 'executors' property.
   * </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
 + * @since 3.1.0
   * @see org.apache.accumulo.core.spi.compaction
   */
  
@@@ -280,107 -226,118 +296,119 @@@ public class DefaultCompactionPlanner i
      determineMaxFilesToCompact(params);
    }
  
 -  @SuppressWarnings("removal")
 +  @SuppressWarnings("deprecation")
    private void determineMaxFilesToCompact(InitParameters params) {
 -    String fqo = params.getFullyQualifiedOption("maxOpen");
 -    if (!params.getServiceEnvironment().getConfiguration().isSet(fqo)
 -        && params.getServiceEnvironment().getConfiguration()
 -            .isSet(Property.TSERV_MAJC_THREAD_MAXOPEN.getKey())) {
 -      log.warn("The property " + Property.TSERV_MAJC_THREAD_MAXOPEN.getKey()
 -          + " was set, it is deprecated.  Set the " + fqo + " option 
instead.");
 -      this.maxFilesToCompact = 
Integer.parseInt(params.getServiceEnvironment().getConfiguration()
 -          .get(Property.TSERV_MAJC_THREAD_MAXOPEN.getKey()));
 -    } else {
 -      this.maxFilesToCompact = 
Integer.parseInt(params.getOptions().getOrDefault("maxOpen",
 -          
Property.TSERV_COMPACTION_SERVICE_DEFAULT_MAX_OPEN.getDefaultValue()));
 +
 +    String maxOpen = params.getOptions().get("maxOpen");
 +    if (maxOpen == null) {
 +      maxOpen = 
Property.TSERV_COMPACTION_SERVICE_DEFAULT_MAX_OPEN.getDefaultValue();
 +      log.trace("default maxOpen not set, defaulting to {}", maxOpen);
      }
 +    this.maxFilesToCompact = Integer.parseInt(maxOpen);
    }
  
 -  @Override
 -  public CompactionPlan makePlan(PlanningParameters params) {
 -    try {
 +  private void validateConfig(JsonElement json, List<String> fields, String 
className) {
  
 -      if (params.getCandidates().isEmpty()) {
 -        return params.createPlanBuilder().build();
 -      }
 +    JsonObject jsonObject = GSON.get().fromJson(json, JsonObject.class);
  
 -      Set<CompactableFile> filesCopy = new HashSet<>(params.getCandidates());
 +    List<String> objectProperties = new ArrayList<>(jsonObject.keySet());
 +    HashSet<String> classFieldNames = new HashSet<>(fields);
 +
 +    if (!classFieldNames.containsAll(objectProperties)) {
 +      objectProperties.removeAll(classFieldNames);
 +      throw new JsonParseException(
 +          "Invalid fields: " + objectProperties + " provided for class: " + 
className);
 +    }
 +  }
 +
 +  @Override
 +  public CompactionPlan makePlan(PlanningParameters params) {
 +    if (params.getCandidates().isEmpty()) {
 +      return params.createPlanBuilder().build();
 +    }
  
 -      long maxSizeToCompact = getMaxSizeToCompact(params.getKind());
 +    Set<CompactableFile> filesCopy = new HashSet<>(params.getCandidates());
  
 -      Collection<CompactableFile> group;
 -      if (params.getRunningCompactions().isEmpty()) {
 -        group = findDataFilesToCompact(filesCopy, params.getRatio(), 
maxFilesToCompact,
 -            maxSizeToCompact);
 +    long maxSizeToCompact = getMaxSizeToCompact(params.getKind());
  
 -        if (!group.isEmpty() && group.size() < params.getCandidates().size()
 -            && params.getCandidates().size() <= maxFilesToCompact
 -            && (params.getKind() == CompactionKind.USER
 -                || params.getKind() == CompactionKind.SELECTOR)) {
 -          // USER and SELECTOR compactions must eventually compact all files. 
When a subset of files
 -          // that meets the compaction ratio is selected, look ahead and see 
if the next compaction
 -          // would also meet the compaction ratio. If not then compact 
everything to avoid doing
 -          // more than logarithmic work across multiple comapctions.
 +    Collection<CompactableFile> group;
 +    if (params.getRunningCompactions().isEmpty()) {
 +      group =
 +          findDataFilesToCompact(filesCopy, params.getRatio(), 
maxFilesToCompact, maxSizeToCompact);
  
 -          filesCopy.removeAll(group);
 -          filesCopy.add(getExpected(group, 0));
 +      if (!group.isEmpty() && group.size() < params.getCandidates().size()
 +          && params.getCandidates().size() <= maxFilesToCompact
 +          && (params.getKind() == CompactionKind.USER
 +              || params.getKind() == CompactionKind.SELECTOR)) {
 +        // USER and SELECTOR compactions must eventually compact all files. 
When a subset of files
 +        // that meets the compaction ratio is selected, look ahead and see if 
the next compaction
 +        // would also meet the compaction ratio. If not then compact 
everything to avoid doing
 +        // more than logarithmic work across multiple comapctions.
  
 -          if (findDataFilesToCompact(filesCopy, params.getRatio(), 
maxFilesToCompact,
 -              maxSizeToCompact).isEmpty()) {
 -            // The next possible compaction does not meet the compaction 
ratio, so compact
 -            // everything.
 -            group = Set.copyOf(params.getCandidates());
 -          }
 +        filesCopy.removeAll(group);
 +        filesCopy.add(getExpected(group, 0));
  
 +        if (findDataFilesToCompact(filesCopy, params.getRatio(), 
maxFilesToCompact,
 +            maxSizeToCompact).isEmpty()) {
 +          // The next possible compaction does not meet the compaction ratio, 
so compact
 +          // everything.
 +          group = Set.copyOf(params.getCandidates());
          }
  
 -      } else if (params.getKind() == CompactionKind.SYSTEM) {
 -        // This code determines if once the files compacting finish would 
they be included in a
 -        // compaction with the files smaller than them? If so, then wait for 
the running compaction
 -        // to complete.
 +      }
  
 -        // The set of files running compactions may produce
 -        var expectedFiles = getExpected(params.getRunningCompactions());
 +    } else if (params.getKind() == CompactionKind.SYSTEM) {
 +      // This code determines if once the files compacting finish would they 
be included in a
 +      // compaction with the files smaller than them? If so, then wait for 
the running compaction
 +      // to complete.
  
 -        if (!Collections.disjoint(filesCopy, expectedFiles)) {
 -          throw new AssertionError();
 -        }
 +      // The set of files running compactions may produce
 +      var expectedFiles = getExpected(params.getRunningCompactions());
  
 -        filesCopy.addAll(expectedFiles);
 +      if (!Collections.disjoint(filesCopy, expectedFiles)) {
 +        throw new AssertionError();
 +      }
  
 -        group = findDataFilesToCompact(filesCopy, params.getRatio(), 
maxFilesToCompact,
 -            maxSizeToCompact);
 +      filesCopy.addAll(expectedFiles);
  
 -        if (!Collections.disjoint(group, expectedFiles)) {
 -          // file produced by running compaction will eventually compact with 
existing files, so
 -          // wait.
 -          group = Set.of();
 -        }
 -      } else {
 +      group =
 +          findDataFilesToCompact(filesCopy, params.getRatio(), 
maxFilesToCompact, maxSizeToCompact);
 +
 +      if (!Collections.disjoint(group, expectedFiles)) {
 +        // file produced by running compaction will eventually compact with 
existing files, so
 +        // wait.
          group = Set.of();
        }
 +    } else {
 +      group = Set.of();
 +    }
  
-     if (group.isEmpty()
-         && (params.getKind() == CompactionKind.USER || params.getKind() == 
CompactionKind.SELECTOR)
-         && 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()) {
++      if ((params.getKind() == CompactionKind.USER || params.getKind() == 
CompactionKind.SELECTOR)
++          && 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()) {
 -        return params.createPlanBuilder().build();
 -      } else {
 -        // determine which executor to use based on the size of the files
 -        var ceid = getExecutor(group);
 +    if (group.isEmpty()) {
 +      return params.createPlanBuilder().build();
 +    } else {
 +      // determine which executor to use based on the size of the files
 +      var ceid = getExecutor(group);
  
 -        return params.createPlanBuilder().addJob(createPriority(params, 
group), ceid, group)
 -            .build();
 -      }
 -    } catch (RuntimeException e) {
 -      throw e;
 +      return params.createPlanBuilder().addJob(createPriority(params, group), 
ceid, group).build();
      }
    }
  
diff --cc 
core/src/test/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlannerTest.java
index bafda93e3e,302106fc95..9f4f9d315c
--- 
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
@@@ -42,15 -45,18 +46,17 @@@ import org.apache.accumulo.core.data.Ta
  import org.apache.accumulo.core.spi.common.ServiceEnvironment;
  import org.apache.accumulo.core.spi.common.ServiceEnvironment.Configuration;
  import org.apache.accumulo.core.spi.compaction.CompactionPlan.Builder;
 +import 
org.apache.accumulo.core.spi.compaction.CompactionPlanner.InitParameters;
  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;
  import org.easymock.EasyMock;
  import org.junit.jupiter.api.Test;
 -import org.slf4j.Logger;
 -import org.slf4j.LoggerFactory;
  
+ import com.google.common.base.Preconditions;
 +import com.google.gson.JsonParseException;
  
  public class DefaultCompactionPlannerTest {
  
@@@ -440,6 -470,147 +446,144 @@@
      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() {
++  public void testMaxTabletFiles() throws Exception {
+     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.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() {
++  public void testMaxTabletFilesNoCompaction() throws Exception {
+     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.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() {
++  public void testMaxTableFilesFallback() throws Exception {
+     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.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)
@@@ -447,8 -618,44 +591,46 @@@
          .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) {
++  private Set<CompactableFile> createCFs(int initialSize, double... 
desiredRatios)
++      throws URISyntaxException {
+     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) {
 +  private static Set<CompactableFile> createCFs(String... namesSizePairs)
 +      throws URISyntaxException {
      Set<CompactableFile> files = new HashSet<>();
  
      for (int i = 0; i < namesSizePairs.length; i += 2) {

Reply via email to