You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by GitBox <gi...@apache.org> on 2021/12/27 15:45:21 UTC

[GitHub] [accumulo] milleruntime commented on a change in pull request #2345: Create tool to validate Compaction configuration

milleruntime commented on a change in pull request #2345:
URL: https://github.com/apache/accumulo/pull/2345#discussion_r775546788



##########
File path: core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionServicesConfig.java
##########
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.util.compaction;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Consumer;
+
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.spi.compaction.CompactionServiceId;
+import org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner;
+
+import com.google.common.collect.Sets;
+
+public class CompactionServicesConfig {
+
+  private final Map<String,String> planners = new HashMap<>();
+  private final Map<String,Long> rateLimits = new HashMap<>();
+  private final Map<String,Map<String,String>> options = new HashMap<>();
+  long defaultRateLimit;
+  private final Consumer<String> deprecationWarningConsumer;
+
+  public static final CompactionServiceId DEFAULT_SERVICE = CompactionServiceId.of("default");
+
+  @SuppressWarnings("removal")
+  private long getDefaultThroughput(AccumuloConfiguration aconf) {
+    if (aconf.isPropertySet(Property.TSERV_MAJC_THROUGHPUT, true)) {
+      return aconf.getAsBytes(Property.TSERV_MAJC_THROUGHPUT);
+    }
+
+    return ConfigurationTypeHelper
+        .getMemoryAsBytes(Property.TSERV_COMPACTION_SERVICE_DEFAULT_RATE_LIMIT.getDefaultValue());
+  }
+
+  @SuppressWarnings("removal")
+  private Map<String,String> getConfiguration(AccumuloConfiguration aconf) {
+
+    Map<String,String> configs =
+        aconf.getAllPropertiesWithPrefix(Property.TSERV_COMPACTION_SERVICE_PREFIX);
+
+    // check if deprecated properties for compaction executor are set
+    if (aconf.isPropertySet(Property.TSERV_MAJC_MAXCONCURRENT, true)) {
+
+      String defaultServicePrefix =
+          Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + DEFAULT_SERVICE.canonical() + ".";
+
+      // check if any properties for the default compaction service are set
+      boolean defaultServicePropsSet = configs.keySet().stream()
+          .filter(key -> key.startsWith(defaultServicePrefix)).map(Property::getPropertyByKey)
+          .anyMatch(prop -> prop == null || aconf.isPropertySet(prop, true));
+
+      if (defaultServicePropsSet) {
+
+        String warning = String.format(
+            "The deprecated property %s was set. Properties with the prefix %s "
+                + "were also set, which replace the deprecated properties. The deprecated "
+                + "property was therefore ignored.",
+            Property.TSERV_MAJC_MAXCONCURRENT.getKey(), defaultServicePrefix);
+
+        deprecationWarningConsumer.accept(warning);
+
+      } else {
+        String numThreads = aconf.get(Property.TSERV_MAJC_MAXCONCURRENT);
+
+        // Its possible a user has configured the other compaction services, but not the default
+        // service. In this case want to produce a config with the default service configs
+        // overridden using deprecated configs.
+
+        HashMap<String,String> configsCopy = new HashMap<>(configs);
+
+        Map<String,String> defaultServiceConfigs =
+            Map.of(defaultServicePrefix + "planner", DefaultCompactionPlanner.class.getName(),
+                defaultServicePrefix + "planner.opts.executors",
+                "[{'name':'deprecated', 'numThreads':" + numThreads + "}]");
+
+        configsCopy.putAll(defaultServiceConfigs);
+
+        String warning = String.format(
+            "The deprecated property %s was set. Properties with the prefix %s "
+                + "were not set, these should replace the deprecated properties. The old "
+                + "properties were automatically mapped to the new properties in process "
+                + "creating : %s.",
+            Property.TSERV_MAJC_MAXCONCURRENT.getKey(), defaultServicePrefix,
+            defaultServiceConfigs);
+
+        deprecationWarningConsumer.accept(warning);
+
+        configs = Map.copyOf(configsCopy);
+      }
+    }
+
+    return configs;
+
+  }
+
+  public CompactionServicesConfig(AccumuloConfiguration aconf,
+      Consumer<String> deprecationWarningConsumer) {
+    this.deprecationWarningConsumer = deprecationWarningConsumer;
+    Map<String,String> configs = getConfiguration(aconf);
+
+    configs.forEach((prop, val) -> {
+
+      var suffix = prop.substring(Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey().length());
+      String[] tokens = suffix.split("\\.");
+      if (tokens.length == 4 && tokens[1].equals("planner") && tokens[2].equals("opts")) {
+        getOptions().computeIfAbsent(tokens[0], k -> new HashMap<>()).put(tokens[3], val);
+      } else if (tokens.length == 2 && tokens[1].equals("planner")) {
+        getPlanners().put(tokens[0], val);
+      } else if (tokens.length == 3 && tokens[1].equals("rate") && tokens[2].equals("limit")) {
+        var eprop = Property.getPropertyByKey(prop);
+        if (eprop == null || aconf.isPropertySet(eprop, true)
+            || !isDeprecatedThroughputSet(aconf)) {
+          getRateLimits().put(tokens[0], ConfigurationTypeHelper.getFixedMemoryAsBytes(val));

Review comment:
       Should access the private maps (planners, rateLimits and options) directly here instead of the public methods (getOptions(), getplanners() and getRateLimits()). The methods are public so could be overridden and could change the behavior of the constructor. I don't think that is what you want since this code seems very specific to that one property.

##########
File path: core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionServicesConfig.java
##########
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.util.compaction;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Consumer;
+
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.spi.compaction.CompactionServiceId;
+import org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner;
+
+import com.google.common.collect.Sets;
+
+public class CompactionServicesConfig {
+
+  private final Map<String,String> planners = new HashMap<>();
+  private final Map<String,Long> rateLimits = new HashMap<>();
+  private final Map<String,Map<String,String>> options = new HashMap<>();
+  long defaultRateLimit;
+  private final Consumer<String> deprecationWarningConsumer;
+
+  public static final CompactionServiceId DEFAULT_SERVICE = CompactionServiceId.of("default");
+
+  @SuppressWarnings("removal")
+  private long getDefaultThroughput(AccumuloConfiguration aconf) {
+    if (aconf.isPropertySet(Property.TSERV_MAJC_THROUGHPUT, true)) {
+      return aconf.getAsBytes(Property.TSERV_MAJC_THROUGHPUT);
+    }
+
+    return ConfigurationTypeHelper
+        .getMemoryAsBytes(Property.TSERV_COMPACTION_SERVICE_DEFAULT_RATE_LIMIT.getDefaultValue());
+  }
+
+  @SuppressWarnings("removal")
+  private Map<String,String> getConfiguration(AccumuloConfiguration aconf) {
+
+    Map<String,String> configs =
+        aconf.getAllPropertiesWithPrefix(Property.TSERV_COMPACTION_SERVICE_PREFIX);
+
+    // check if deprecated properties for compaction executor are set
+    if (aconf.isPropertySet(Property.TSERV_MAJC_MAXCONCURRENT, true)) {
+
+      String defaultServicePrefix =
+          Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + DEFAULT_SERVICE.canonical() + ".";
+
+      // check if any properties for the default compaction service are set
+      boolean defaultServicePropsSet = configs.keySet().stream()
+          .filter(key -> key.startsWith(defaultServicePrefix)).map(Property::getPropertyByKey)
+          .anyMatch(prop -> prop == null || aconf.isPropertySet(prop, true));
+
+      if (defaultServicePropsSet) {
+
+        String warning = String.format(
+            "The deprecated property %s was set. Properties with the prefix %s "
+                + "were also set, which replace the deprecated properties. The deprecated "
+                + "property was therefore ignored.",
+            Property.TSERV_MAJC_MAXCONCURRENT.getKey(), defaultServicePrefix);
+
+        deprecationWarningConsumer.accept(warning);
+
+      } else {
+        String numThreads = aconf.get(Property.TSERV_MAJC_MAXCONCURRENT);
+
+        // Its possible a user has configured the other compaction services, but not the default
+        // service. In this case want to produce a config with the default service configs
+        // overridden using deprecated configs.
+
+        HashMap<String,String> configsCopy = new HashMap<>(configs);
+
+        Map<String,String> defaultServiceConfigs =
+            Map.of(defaultServicePrefix + "planner", DefaultCompactionPlanner.class.getName(),
+                defaultServicePrefix + "planner.opts.executors",
+                "[{'name':'deprecated', 'numThreads':" + numThreads + "}]");
+
+        configsCopy.putAll(defaultServiceConfigs);
+
+        String warning = String.format(
+            "The deprecated property %s was set. Properties with the prefix %s "
+                + "were not set, these should replace the deprecated properties. The old "
+                + "properties were automatically mapped to the new properties in process "
+                + "creating : %s.",
+            Property.TSERV_MAJC_MAXCONCURRENT.getKey(), defaultServicePrefix,
+            defaultServiceConfigs);
+
+        deprecationWarningConsumer.accept(warning);
+
+        configs = Map.copyOf(configsCopy);
+      }
+    }
+
+    return configs;
+
+  }
+
+  public CompactionServicesConfig(AccumuloConfiguration aconf,
+      Consumer<String> deprecationWarningConsumer) {

Review comment:
       Passing a consumer for the logger is nifty but I think this could be confusing for someone debugging looking at line numbers for a logger. Also, if you just passed the logger, you could use it for debug logging too.

##########
File path: server/base/src/main/java/org/apache/accumulo/server/conf/CheckCompactionConfig.java
##########
@@ -0,0 +1,147 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.server.conf;
+
+import java.io.FileNotFoundException;
+import java.nio.file.Path;
+import java.util.Set;
+
+import org.apache.accumulo.core.cli.Help;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.SiteConfiguration;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.spi.common.ServiceEnvironment;
+import org.apache.accumulo.core.spi.compaction.CompactionPlanner;
+import org.apache.accumulo.core.spi.compaction.CompactionServiceId;
+import org.apache.accumulo.core.util.ConfigurationImpl;
+import org.apache.accumulo.core.util.compaction.CompactionPlannerInitParams;
+import org.apache.accumulo.core.util.compaction.CompactionServicesConfig;
+import org.apache.accumulo.start.spi.KeywordExecutable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.beust.jcommander.Parameter;
+import com.google.auto.service.AutoService;
+
+@AutoService(KeywordExecutable.class)
+public class CheckCompactionConfig implements KeywordExecutable {
+
+  private final static Logger log = LoggerFactory.getLogger(CheckCompactionConfig.class);
+
+  static class Opts extends Help {
+    @Parameter(description = "<path/to/props/file>", required = true)
+    String filePath;
+  }
+
+  @Override
+  public String keyword() {
+    return "check-compaction-config";
+  }
+
+  @Override
+  public String description() {
+    return "Verifies compaction config within a given file";
+  }
+
+  public static void main(String[] args) throws Exception {
+    new CheckCompactionConfig().execute(args);
+  }
+
+  @Override
+  public void execute(String[] args) throws Exception {
+    Opts opts = new Opts();
+    opts.parseArgs("accumulo check-compaction-config", args);
+
+    if (opts.filePath == null) {
+      throw new IllegalArgumentException("No properties file was given");
+    }
+
+    Path path = Path.of(opts.filePath);
+    if (!path.toFile().exists())
+      throw new FileNotFoundException("File at given path could not be found");
+
+    AccumuloConfiguration config = SiteConfiguration.fromFile(path.toFile()).build();
+    var servicesConfig = new CompactionServicesConfig(config, log::warn);
+    ServiceEnvironment senv = createServiceEnvironment(config);
+
+    Set<String> defaultServices = Set.of("default", "meta", "root");
+    if (servicesConfig.getPlanners().keySet().equals(defaultServices)) {
+      throw new IllegalArgumentException("Only the default compaction services were created");

Review comment:
       Could make this a warning and return. Throwing an error is fine, I am just not sure how useful showing the user a stack trace would be for this situation.

##########
File path: server/tserver/src/main/java/org/apache/accumulo/tserver/compactions/CompactionManager.java
##########
@@ -91,144 +89,10 @@ public ExtCompInfo(KeyExtent extent, CompactionExecutorId executor) {
     }
   }
 
-  private class Config {

Review comment:
       This is a nice change, breaking out this class and giving it a more proper class name.

##########
File path: core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionServicesConfig.java
##########
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.util.compaction;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Consumer;
+
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.spi.compaction.CompactionServiceId;
+import org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner;
+
+import com.google.common.collect.Sets;
+
+public class CompactionServicesConfig {

Review comment:
       Since this class is specific to certain properties, it would be nice to note them in the javadoc. Examples of properties is always helpful too.

##########
File path: server/base/src/main/java/org/apache/accumulo/server/conf/CheckCompactionConfig.java
##########
@@ -0,0 +1,147 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.server.conf;
+
+import java.io.FileNotFoundException;
+import java.nio.file.Path;
+import java.util.Set;
+
+import org.apache.accumulo.core.cli.Help;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.SiteConfiguration;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.spi.common.ServiceEnvironment;
+import org.apache.accumulo.core.spi.compaction.CompactionPlanner;
+import org.apache.accumulo.core.spi.compaction.CompactionServiceId;
+import org.apache.accumulo.core.util.ConfigurationImpl;
+import org.apache.accumulo.core.util.compaction.CompactionPlannerInitParams;
+import org.apache.accumulo.core.util.compaction.CompactionServicesConfig;
+import org.apache.accumulo.start.spi.KeywordExecutable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.beust.jcommander.Parameter;
+import com.google.auto.service.AutoService;
+
+@AutoService(KeywordExecutable.class)
+public class CheckCompactionConfig implements KeywordExecutable {
+
+  private final static Logger log = LoggerFactory.getLogger(CheckCompactionConfig.class);
+
+  static class Opts extends Help {
+    @Parameter(description = "<path/to/props/file>", required = true)
+    String filePath;
+  }
+
+  @Override
+  public String keyword() {
+    return "check-compaction-config";
+  }
+
+  @Override
+  public String description() {
+    return "Verifies compaction config within a given file";
+  }
+
+  public static void main(String[] args) throws Exception {
+    new CheckCompactionConfig().execute(args);
+  }
+
+  @Override
+  public void execute(String[] args) throws Exception {
+    Opts opts = new Opts();
+    opts.parseArgs("accumulo check-compaction-config", args);
+
+    if (opts.filePath == null) {
+      throw new IllegalArgumentException("No properties file was given");
+    }
+
+    Path path = Path.of(opts.filePath);
+    if (!path.toFile().exists())
+      throw new FileNotFoundException("File at given path could not be found");
+
+    AccumuloConfiguration config = SiteConfiguration.fromFile(path.toFile()).build();
+    var servicesConfig = new CompactionServicesConfig(config, log::warn);
+    ServiceEnvironment senv = createServiceEnvironment(config);
+
+    Set<String> defaultServices = Set.of("default", "meta", "root");

Review comment:
       Could make these static final at the top of the class. Sometimes this makes hard coded strings easier to find.

##########
File path: core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionServicesConfig.java
##########
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.util.compaction;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Consumer;
+
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.spi.compaction.CompactionServiceId;
+import org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner;
+
+import com.google.common.collect.Sets;
+
+public class CompactionServicesConfig {
+
+  private final Map<String,String> planners = new HashMap<>();
+  private final Map<String,Long> rateLimits = new HashMap<>();
+  private final Map<String,Map<String,String>> options = new HashMap<>();
+  long defaultRateLimit;
+  private final Consumer<String> deprecationWarningConsumer;
+
+  public static final CompactionServiceId DEFAULT_SERVICE = CompactionServiceId.of("default");
+
+  @SuppressWarnings("removal")
+  private long getDefaultThroughput(AccumuloConfiguration aconf) {
+    if (aconf.isPropertySet(Property.TSERV_MAJC_THROUGHPUT, true)) {
+      return aconf.getAsBytes(Property.TSERV_MAJC_THROUGHPUT);
+    }
+
+    return ConfigurationTypeHelper
+        .getMemoryAsBytes(Property.TSERV_COMPACTION_SERVICE_DEFAULT_RATE_LIMIT.getDefaultValue());
+  }
+
+  @SuppressWarnings("removal")
+  private Map<String,String> getConfiguration(AccumuloConfiguration aconf) {
+
+    Map<String,String> configs =
+        aconf.getAllPropertiesWithPrefix(Property.TSERV_COMPACTION_SERVICE_PREFIX);
+
+    // check if deprecated properties for compaction executor are set
+    if (aconf.isPropertySet(Property.TSERV_MAJC_MAXCONCURRENT, true)) {
+
+      String defaultServicePrefix =
+          Property.TSERV_COMPACTION_SERVICE_PREFIX.getKey() + DEFAULT_SERVICE.canonical() + ".";
+
+      // check if any properties for the default compaction service are set
+      boolean defaultServicePropsSet = configs.keySet().stream()
+          .filter(key -> key.startsWith(defaultServicePrefix)).map(Property::getPropertyByKey)
+          .anyMatch(prop -> prop == null || aconf.isPropertySet(prop, true));
+
+      if (defaultServicePropsSet) {
+
+        String warning = String.format(
+            "The deprecated property %s was set. Properties with the prefix %s "
+                + "were also set, which replace the deprecated properties. The deprecated "
+                + "property was therefore ignored.",
+            Property.TSERV_MAJC_MAXCONCURRENT.getKey(), defaultServicePrefix);

Review comment:
       The performance of `String.format()` is not great. It seems like this could be called a lot in the case where a user stays with the default properties. I am not sure how often the CompactionManager code calls this though but either way, it doesn't seem like you really need the format since you can just concatenate the strings.




-- 
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