You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by "ddanielr (via GitHub)" <gi...@apache.org> on 2023/12/16 00:54:41 UTC

[PR] Remove old compaction code [accumulo]

ddanielr opened a new pull request, #4083:
URL: https://github.com/apache/accumulo/pull/4083

   This PR removes the deprecated executors property and replaces it with groups.
   This also remove many references to "External" compactions but does not yet modify any "external" naming that may be tied to persisted data.
   
   Once this PR is merged, any specific changes in main will be addressed to allow for a clean deprecation path.
   
   Waiting for #4080 to be merged before moving to ready status. 
   
   #### Thoughts:
   
   The concept of a Compaction Group has been introduced in main via https://github.com/apache/accumulo/pull/4020, but it doesn't currently mesh well with the internal and external compaction types still co-existing. 
   
   For deprecation patterns, the COMPACTION_SERVICE_<service>_QUEUES property is going to be switched in main to COMPACTION_SERVICE_<service>_GROUPS.
   
   #### Additional Improvements:
   
   We may need to add some specific validation around the group name, (allowed characters, length, etc).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Remove old compaction code [accumulo]

Posted by "ddanielr (via GitHub)" <gi...@apache.org>.
ddanielr merged PR #4083:
URL: https://github.com/apache/accumulo/pull/4083


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Remove old compaction code [accumulo]

Posted by "ddanielr (via GitHub)" <gi...@apache.org>.
ddanielr commented on code in PR #4083:
URL: https://github.com/apache/accumulo/pull/4083#discussion_r1430839870


##########
core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java:
##########
@@ -57,85 +57,47 @@
  * compaction service you are configuring.
  *
  * <ul>
- * <li>{@code compaction.service.<service>.opts.executors} This is a json array of objects where
- * each object has the fields:
+ * <li>Note that the CompactionCoordinator and at least one Compactor for "Large" must be running.
+ * <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.groups} This is a json array of group objects which
+ * have the following fields:
  * <table>
- * <caption>Default Compaction Planner Executor options</caption>
+ * <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 executor (required)</td>
- * </tr>
- * <tr>
- * <td>type</td>
- * <td>valid values 'internal' or 'external' (required)</td>
+ * <td>name or alias of the group (required)</td>

Review Comment:
   Yeah, we don't have any of that support. I'll remove this. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Remove old compaction code [accumulo]

Posted by "keith-turner (via GitHub)" <gi...@apache.org>.
keith-turner commented on code in PR #4083:
URL: https://github.com/apache/accumulo/pull/4083#discussion_r1430721101


##########
core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java:
##########
@@ -57,85 +57,47 @@
  * compaction service you are configuring.
  *
  * <ul>
- * <li>{@code compaction.service.<service>.opts.executors} This is a json array of objects where
- * each object has the fields:
+ * <li>Note that the CompactionCoordinator and at least one Compactor for "Large" must be running.
+ * <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.groups} This is a json array of group objects which
+ * have the following fields:
  * <table>
- * <caption>Default Compaction Planner Executor options</caption>
+ * <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 executor (required)</td>
- * </tr>
- * <tr>
- * <td>type</td>
- * <td>valid values 'internal' or 'external' (required)</td>
+ * <td>name or alias of the group (required)</td>
  * </tr>
  * <tr>
  * <td>maxSize</td>
  * <td>threshold sum of the input files (required for all but one of the configs)</td>
  * </tr>
- * <tr>
- * <td>numThreads</td>
- * <td>number of threads for this executor configuration (required for 'internal', cannot be
- * specified for 'external')</td>
- * </tr>
- * <tr>
- * <td>group</td>
- * <td>name of the external compaction group (required for 'external', cannot be specified for
- * 'internal')</td>
- * </tr>
  * </table>
  * <br>
- * Note: The "executors" option has been deprecated in 3.1 and will be removed in a future release.
- * This example uses the new `compaction.service` prefix. The property prefix
- * "tserver.compaction.major.service" has also been deprecated in 3.1 and will be removed in a
- * future release. The maxSize field determines the maximum size of compaction that will run on an
- * executor. The maxSize field can have a suffix of K,M,G for kilobytes, megabytes, or gigabytes and
- * represents the sum of the input files for a given compaction. One executor can have no max size
- * and it will run everything that is too large for the other executors. If all executors have a max
- * size, then system compactions will only run for compactions smaller than the largest max size.
- * User, chop, and selector compactions will always run, even if there is no executor for their
- * size. These compactions will run on the executor with the largest max size. The following example
- * value for this property will create 3 threads to run compactions of files whose file size sum is
- * less than 100M, 3 threads to run compactions of files whose file size sum is less than 500M, and
- * run all other compactions on Compactors configured to run compactions for Queue1:
+ * This 'groups' object is used for defining compaction groups. The maxSize field determines the
+ * maximum size of compaction that will run in a group. The maxSize field can have a suffix of K,M,G
+ * for kilobytes, megabytes, or gigabytes and represents the sum of the input files for a given
+ * compaction. One group can have no max size and it will run everything that is too large for the
+ * other groups. If all groups have a max size, then system compactions will only run for
+ * compactions smaller than the largest max size. User, chop, and selector compactions will always

Review Comment:
   This should have already been removed, but it looks like we missed it.  Chop compactions are gone, so can drop chop.



##########
core/src/main/java/org/apache/accumulo/core/conf/Property.java:
##########
@@ -53,12 +53,40 @@ public enum Property {
           + "A new external compaction service would be defined like the following:\n"
           + "`compaction.service.newService.planner="
           + "\"org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner\".`\n"
-          + "`compaction.service.newService.opts.queues=\""
+          + "`compaction.service.newService.opts.groups=\""
           + "[{\"name\": \"small\", \"maxSize\":\"32M\"},"
           + "{ \"name\":\"medium\", \"maxSize\":\"512M\"},{\"name\":\"large\"}]`\n"
           + "`compaction.service.newService.opts.maxOpen=50`.\n"
           + "Additional options can be defined using the `compaction.service.<service>.opts.<option>` property.",
       "3.1.0"),
+  COMPACTION_SERVICE_ROOT_PLANNER(COMPACTION_SERVICE_PREFIX + "root.planner",

Review Comment:
   Are you adding these as a temporary step to get things working?  Asking because of #3981



##########
core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java:
##########
@@ -57,85 +57,47 @@
  * compaction service you are configuring.
  *
  * <ul>
- * <li>{@code compaction.service.<service>.opts.executors} This is a json array of objects where
- * each object has the fields:
+ * <li>Note that the CompactionCoordinator and at least one Compactor for "Large" must be running.
+ * <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.groups} This is a json array of group objects which
+ * have the following fields:
  * <table>
- * <caption>Default Compaction Planner Executor options</caption>
+ * <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 executor (required)</td>
- * </tr>
- * <tr>
- * <td>type</td>
- * <td>valid values 'internal' or 'external' (required)</td>
+ * <td>name or alias of the group (required)</td>

Review Comment:
   Why mention alias here?  The feels like there is some support for group name indirection elsewhere.



##########
core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java:
##########
@@ -57,85 +57,47 @@
  * compaction service you are configuring.
  *
  * <ul>
- * <li>{@code compaction.service.<service>.opts.executors} This is a json array of objects where
- * each object has the fields:
+ * <li>Note that the CompactionCoordinator and at least one Compactor for "Large" must be running.

Review Comment:
   What is `"Large"`?



##########
core/src/test/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlannerTest.java:
##########
@@ -586,73 +501,14 @@ public void testErrorOnlyOneMaxSize() {
     EasyMock.expect(senv.getConfiguration()).andReturn(conf).anyTimes();
     EasyMock.replay(conf, senv);
 
-    String executors = getExecutors("'type': 'internal','maxSize':'32M','numThreads':1",
-        "'type': 'internal','numThreads':2", "'type': 'external','group':'q1'");
+    String groups =
+        "[{\"name\":\"small\", \"maxSize\":\"32M\"}, {\"name\":\"medium\"}, {\"name\":\"large\"}]";
     var e = assertThrows(IllegalArgumentException.class,
-        () -> planner.init(getInitParams(senv, executors)), "Failed to throw error");
+        () -> planner.init(getInitParams(senv, groups)), "Failed to throw error");
     assertTrue(e.getMessage().contains("maxSize"), "Error message didn't contain maxSize");
   }
 
-  /**
-   * Tests executors can only have one without a max size.
-   */
-  @Test
-  public void testErrorDuplicateMaxSize() {

Review Comment:
   Maybe the test moved and I did not see it. Seems like it would still be useful to have a test for groups with duplicate max sizes.



##########
core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionServicesConfig.java:
##########
@@ -97,54 +70,21 @@ public CompactionServicesConfig(AccumuloConfiguration aconf) {
     this(getConfiguration(aconf::getAllPropertiesWithPrefixStripped), aconf::isPropertySet);
   }
 
-  @SuppressWarnings("removal")
   private CompactionServicesConfig(Map<String,Map<String,String>> configs,
       Predicate<Property> isSetPredicate) {
     configs.forEach((prefix, props) -> {
       props.forEach((prop, val) -> {
         String[] tokens = prop.split("\\.");
         if (tokens.length == 2 && tokens[1].equals("planner")) {
-          if (prefix.equals(oldPrefix.getKey())) {
-            // Log a warning if the old prefix planner is defined by a user.
-            Property userDefined = null;
-            try {
-              userDefined = Property.valueOf(prefix + prop);
-            } catch (IllegalArgumentException e) {
-              log.trace("Property: {} is not set by default configuration", prefix + prop);
-            }
-            boolean isPropSet = true;
-            if (userDefined != null) {
-              isPropSet = isSetPredicate.test(userDefined);
-            }
-            if (isPropSet) {
-              log.warn(
-                  "Found compaction planner '{}' using a deprecated prefix. Please update property to use the '{}' prefix",
-                  tokens[0], newPrefix);
-            }
-          }
           plannerPrefixes.put(tokens[0], prefix);
           planners.put(tokens[0], val);
-        }
-      });
-    });
-
-    // Now find all compaction planner options.
-    configs.forEach((prefix, props) -> {
-      props.forEach((prop, val) -> {
-        String[] tokens = prop.split("\\.");
-        if (!plannerPrefixes.containsKey(tokens[0])) {
-          throw new IllegalArgumentException(
-              "Incomplete compaction service definition, missing planner class: " + prop);
-        }
-        if (tokens.length == 4 && tokens[1].equals("planner") && tokens[2].equals("opts")) {
+        } else if (tokens.length == 4 && tokens[1].equals("planner") && tokens[2].equals("opts")) {

Review Comment:
   Getting a bit lost in these diff.  Will this code still fail if a service is configured with planner options and no planner class?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Remove old compaction code [accumulo]

Posted by "keith-turner (via GitHub)" <gi...@apache.org>.
keith-turner commented on PR #4083:
URL: https://github.com/apache/accumulo/pull/4083#issuecomment-1878761621

   @ddanielr in the merge of I pulled in some new test into DefaultCompactionPlannerTest 6f908863f571bcb76dd77e0576ec1bd135cd0284 that caused conflicts with this PR.  I need to make another commit for those new test to make them use the new config instead of the old executors config.  Also in that commit I readded some code related to maxOpen in the test class that was removed in elasticity.  Was not sure exactly what was going on there, was just trying to get the new test running while working through the merge conflicts.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Remove old compaction code [accumulo]

Posted by "ddanielr (via GitHub)" <gi...@apache.org>.
ddanielr commented on code in PR #4083:
URL: https://github.com/apache/accumulo/pull/4083#discussion_r1430848137


##########
core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionServicesConfig.java:
##########
@@ -97,54 +70,21 @@ public CompactionServicesConfig(AccumuloConfiguration aconf) {
     this(getConfiguration(aconf::getAllPropertiesWithPrefixStripped), aconf::isPropertySet);
   }
 
-  @SuppressWarnings("removal")
   private CompactionServicesConfig(Map<String,Map<String,String>> configs,
       Predicate<Property> isSetPredicate) {
     configs.forEach((prefix, props) -> {
       props.forEach((prop, val) -> {
         String[] tokens = prop.split("\\.");
         if (tokens.length == 2 && tokens[1].equals("planner")) {
-          if (prefix.equals(oldPrefix.getKey())) {
-            // Log a warning if the old prefix planner is defined by a user.
-            Property userDefined = null;
-            try {
-              userDefined = Property.valueOf(prefix + prop);
-            } catch (IllegalArgumentException e) {
-              log.trace("Property: {} is not set by default configuration", prefix + prop);
-            }
-            boolean isPropSet = true;
-            if (userDefined != null) {
-              isPropSet = isSetPredicate.test(userDefined);
-            }
-            if (isPropSet) {
-              log.warn(
-                  "Found compaction planner '{}' using a deprecated prefix. Please update property to use the '{}' prefix",
-                  tokens[0], newPrefix);
-            }
-          }
           plannerPrefixes.put(tokens[0], prefix);
           planners.put(tokens[0], val);
-        }
-      });
-    });
-
-    // Now find all compaction planner options.
-    configs.forEach((prefix, props) -> {
-      props.forEach((prop, val) -> {
-        String[] tokens = prop.split("\\.");
-        if (!plannerPrefixes.containsKey(tokens[0])) {
-          throw new IllegalArgumentException(
-              "Incomplete compaction service definition, missing planner class: " + prop);
-        }
-        if (tokens.length == 4 && tokens[1].equals("planner") && tokens[2].equals("opts")) {
+        } else if (tokens.length == 4 && tokens[1].equals("planner") && tokens[2].equals("opts")) {

Review Comment:
   Yes, that happens on lines 94-99. I removed all the dual prefix validation bits.
   
   https://github.com/apache/accumulo/pull/4083/files#diff-ba8ebe6daae5454d3fd92a0f328ed7a6b40a5796994831c314f5ad0a6dd5aa8aR94-R99



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Remove old compaction code [accumulo]

Posted by "keith-turner (via GitHub)" <gi...@apache.org>.
keith-turner commented on code in PR #4083:
URL: https://github.com/apache/accumulo/pull/4083#discussion_r1441163615


##########
core/src/test/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlannerTest.java:
##########
@@ -433,114 +424,57 @@ public void testUserCompactionDoesNotWaitOnSystemCompaction() {
   }
 
   @Test
-  public void testQueueCreation() throws Exception {
+  public void testQueueCreation() {
     DefaultCompactionPlanner planner = new DefaultCompactionPlanner();
 
-    String queues = "[{\"name\": \"small\", \"maxSize\":\"32M\"},{\"name\":\"midsize\"}]";
-    planner.init(getInitParamQueues(defaultConf, queues));
+    String groups = "[{\"name\": \"small\", \"maxSize\":\"32M\"},{\"name\":\"midsize\"}]";
+    planner.init(getInitParams(defaultConf, groups));
 
     var all = createCFs("F1", "1M", "F2", "1M", "F3", "1M", "F4", "1M");
     var params = createPlanningParams(all, all, Set.of(), 2, CompactionKind.SYSTEM);
     var plan = planner.makePlan(params);
 
     var job = getOnlyElement(plan.getJobs());
     assertEquals(all, job.getFiles());
-    assertEquals(CompactionExecutorIdImpl.externalId("small"), job.getExecutor());
+    assertEquals(CompactionGroupIdImpl.groupId("small"), job.getGroup());
 
     all = createCFs("F1", "100M", "F2", "100M", "F3", "100M", "F4", "100M");
     params = createPlanningParams(all, all, Set.of(), 2, CompactionKind.SYSTEM);
     plan = planner.makePlan(params);
 
     job = getOnlyElement(plan.getJobs());
     assertEquals(all, job.getFiles());
-    assertEquals(CompactionExecutorIdImpl.externalId("midsize"), job.getExecutor());
+    assertEquals(CompactionGroupIdImpl.groupId("midsize"), job.getGroup());
   }
 
   /**
    * Tests that additional fields in the JSON objects cause errors to be thrown.
    */
   @Test
   public void testErrorAdditionalConfigFields() {
-    DefaultCompactionPlanner QueuePlanner = new DefaultCompactionPlanner();
+    DefaultCompactionPlanner planner = new DefaultCompactionPlanner();
 
-    String queues =
+    String groups =
         "[{\"name\":\"smallQueue\", \"maxSize\":\"32M\"}, {\"name\":\"largeQueue\", \"type\":\"internal\", \"foo\":\"bar\", \"queue\":\"broken\"}]";
 
-    final InitParameters queueParams = getInitParamQueues(defaultConf, queues);
-    assertNotNull(queueParams);
-    var e = assertThrows(JsonParseException.class, () -> QueuePlanner.init(queueParams),
-        "Failed to throw error");
+    final InitParameters params = getInitParams(defaultConf, groups);

Review Comment:
   This test is much simpler now.



##########
core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java:
##########
@@ -58,85 +58,48 @@
  * compaction service you are configuring.
  *
  * <ul>
- * <li>{@code compaction.service.<service>.opts.executors} This is a json array of objects where
- * each object has the fields:
+ * <li>Note that the CompactionCoordinator and at least one Compactor for the "large" compaction

Review Comment:
   Thinking we may want to use the terminology "compactor group" when expanding "group".  If that sounds good there are a few other places that could change.
   
   ```suggestion
    * <li>Note that the CompactionCoordinator and at least one Compactor for the "large" compactor
   ```



##########
core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java:
##########
@@ -58,85 +58,48 @@
  * compaction service you are configuring.
  *
  * <ul>
- * <li>{@code compaction.service.<service>.opts.executors} This is a json array of objects where
- * each object has the fields:
+ * <li>Note that the CompactionCoordinator and at least one Compactor for the "large" compaction
+ * group must be running.
+ * <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.groups} This is a json array of group objects which
+ * have the following fields:
  * <table>
- * <caption>Default Compaction Planner Executor options</caption>
+ * <caption>Default Compaction Planner Group options</caption>
  * <tr>
  * <th>Field Name</th>
  * <th>Description</th>
  * </tr>
  * <tr>
  * <td>name</td>
- * <td>name or alias of the executor (required)</td>
- * </tr>
- * <tr>
- * <td>type</td>
- * <td>valid values 'internal' or 'external' (required)</td>
+ * <td>name of the group (required)</td>
  * </tr>
  * <tr>
  * <td>maxSize</td>
  * <td>threshold sum of the input files (required for all but one of the configs)</td>
  * </tr>
- * <tr>
- * <td>numThreads</td>
- * <td>number of threads for this executor configuration (required for 'internal', cannot be
- * specified for 'external')</td>
- * </tr>
- * <tr>
- * <td>group</td>
- * <td>name of the external compaction group (required for 'external', cannot be specified for
- * 'internal')</td>
- * </tr>
  * </table>
  * <br>
- * Note: The "executors" option has been deprecated in 3.1 and will be removed in a future release.
- * This example uses the new `compaction.service` prefix. The property prefix
- * "tserver.compaction.major.service" has also been deprecated in 3.1 and will be removed in a
- * future release. The maxSize field determines the maximum size of compaction that will run on an
- * executor. The maxSize field can have a suffix of K,M,G for kilobytes, megabytes, or gigabytes and
- * represents the sum of the input files for a given compaction. One executor can have no max size
- * and it will run everything that is too large for the other executors. If all executors have a max
- * size, then system compactions will only run for compactions smaller than the largest max size.
- * User, chop, and selector compactions will always run, even if there is no executor for their
- * size. These compactions will run on the executor with the largest max size. The following example
- * value for this property will create 3 threads to run compactions of files whose file size sum is
- * less than 100M, 3 threads to run compactions of files whose file size sum is less than 500M, and
- * run all other compactions on Compactors configured to run compactions for Queue1:
+ * This 'groups' object is used for defining compaction groups. The maxSize field determines the

Review Comment:
   ```suggestion
    * This 'groups' object provides information that is used for mapping a compaction job to a compactor group. The maxSize field determines the
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Remove old compaction code [accumulo]

Posted by "ddanielr (via GitHub)" <gi...@apache.org>.
ddanielr commented on code in PR #4083:
URL: https://github.com/apache/accumulo/pull/4083#discussion_r1430753203


##########
core/src/main/java/org/apache/accumulo/core/conf/Property.java:
##########
@@ -53,12 +53,40 @@ public enum Property {
           + "A new external compaction service would be defined like the following:\n"
           + "`compaction.service.newService.planner="
           + "\"org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner\".`\n"
-          + "`compaction.service.newService.opts.queues=\""
+          + "`compaction.service.newService.opts.groups=\""
           + "[{\"name\": \"small\", \"maxSize\":\"32M\"},"
           + "{ \"name\":\"medium\", \"maxSize\":\"512M\"},{\"name\":\"large\"}]`\n"
           + "`compaction.service.newService.opts.maxOpen=50`.\n"
           + "Additional options can be defined using the `compaction.service.<service>.opts.<option>` property.",
       "3.1.0"),
+  COMPACTION_SERVICE_ROOT_PLANNER(COMPACTION_SERVICE_PREFIX + "root.planner",

Review Comment:
   Yes, I wanted to keep this PR smaller and focus on the compaction code logic changing. 
   Then modify the property default values in a later PR.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Remove old compaction code [accumulo]

Posted by "ddanielr (via GitHub)" <gi...@apache.org>.
ddanielr commented on code in PR #4083:
URL: https://github.com/apache/accumulo/pull/4083#discussion_r1430839499


##########
core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java:
##########
@@ -57,85 +57,47 @@
  * compaction service you are configuring.
  *
  * <ul>
- * <li>{@code compaction.service.<service>.opts.executors} This is a json array of objects where
- * each object has the fields:
+ * <li>Note that the CompactionCoordinator and at least one Compactor for "Large" must be running.

Review Comment:
   `"Large"` is the compaction group that does not have a defined size.
   
   It's defined down at the bottom of the javadoc. I'll update the wording so it's a bit clearer.
   ```
     [    {"name":"small", "maxSize":"100M"},    {"name":"medium", "maxSize":"500M"},    {"name": "large"}   ]
   ```
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] Remove old compaction code [accumulo]

Posted by "ddanielr (via GitHub)" <gi...@apache.org>.
ddanielr commented on code in PR #4083:
URL: https://github.com/apache/accumulo/pull/4083#discussion_r1430892726


##########
core/src/test/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlannerTest.java:
##########
@@ -586,73 +501,14 @@ public void testErrorOnlyOneMaxSize() {
     EasyMock.expect(senv.getConfiguration()).andReturn(conf).anyTimes();
     EasyMock.replay(conf, senv);
 
-    String executors = getExecutors("'type': 'internal','maxSize':'32M','numThreads':1",
-        "'type': 'internal','numThreads':2", "'type': 'external','group':'q1'");
+    String groups =
+        "[{\"name\":\"small\", \"maxSize\":\"32M\"}, {\"name\":\"medium\"}, {\"name\":\"large\"}]";
     var e = assertThrows(IllegalArgumentException.class,
-        () -> planner.init(getInitParams(senv, executors)), "Failed to throw error");
+        () -> planner.init(getInitParams(senv, groups)), "Failed to throw error");
     assertTrue(e.getMessage().contains("maxSize"), "Error message didn't contain maxSize");
   }
 
-  /**
-   * Tests executors can only have one without a max size.
-   */
-  @Test
-  public void testErrorDuplicateMaxSize() {

Review Comment:
   This test was removed because the test comment and exception check statement looked identical to `testErrorOnlyOneMaxSize()`
   
   I'll add the test back and change the check statements to be unique. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org