dlmarion commented on code in PR #5749:
URL: https://github.com/apache/accumulo/pull/5749#discussion_r2226560807


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ResourceGroupOperationsImpl.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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
+ *
+ *   https://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.clientImpl;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import java.time.Duration;
+import java.util.ConcurrentModificationException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Consumer;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.ResourceGroupNotFoundException;
+import org.apache.accumulo.core.client.admin.ResourceGroupOperations;
+import org.apache.accumulo.core.clientImpl.thrift.ConfigurationType;
+import org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties;
+import 
org.apache.accumulo.core.clientImpl.thrift.ThriftResourceGroupNotExistsException;
+import org.apache.accumulo.core.conf.DeprecatedPropertyUtil;
+import org.apache.accumulo.core.data.ResourceGroupId;
+import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.util.LocalityGroupUtil;
+import 
org.apache.accumulo.core.util.LocalityGroupUtil.LocalityGroupConfigurationError;
+import org.apache.accumulo.core.util.Retry;
+import org.slf4j.LoggerFactory;
+
+public class ResourceGroupOperationsImpl implements ResourceGroupOperations {
+
+  private final ClientContext context;
+
+  public ResourceGroupOperationsImpl(ClientContext context) {
+    checkArgument(context != null, "context is null");
+    this.context = context;
+  }
+
+  @Override
+  public boolean exists(String group) {
+    Objects.requireNonNull(group, "group parameter must be supplied");

Review Comment:
   This can be removed, ResourceGroupId constructor handles this



##########
core/src/main/java/org/apache/accumulo/core/client/admin/ResourceGroupOperations.java:
##########
@@ -0,0 +1,232 @@
+/*
+ * 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
+ *
+ *   https://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.client.admin;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Consumer;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.ResourceGroupNotFoundException;
+import org.apache.accumulo.core.data.ResourceGroupId;
+
+/**
+ * A ResourceGroup is a grouping of Accumulo server processes that have some 
shared characteristic
+ * that is different than server processes in other resource groups. Examples 
could be homogeneous
+ * hardware configurations for the server processes in one resource group but 
different than other
+ * resource groups, or a resource group could be created for physical 
separation of processing for
+ * table(s).
+ *
+ * A default resource group exists in which all server processes are assigned. 
The Manager, Monitor,
+ * and GarbageCollector are assigned to the default resource group. Compactor, 
ScanServer and
+ * TabletServer processes are also assigned to the default resource group, 
unless their respective
+ * properties are set.
+ *
+ * This object is for defining, interacting, and removing resource group 
configurations. When the
+ * Accumulo server processes get the system configuration, they will receive a 
merged view of the
+ * system configuration and applicable resource group configuration, with any 
property defined in
+ * the resource group configuration given higher priority.
+ *
+ * @since 4.0.0
+ */
+public interface ResourceGroupOperations {
+
+  /**
+   * A method to check if a resource group configuration exists in Accumulo.
+   *
+   * @param group the name of the resource group
+   * @return true if the group exists
+   * @since 4.0.0
+   */
+  boolean exists(String group);
+
+  /**
+   * Retrieve a list of resource groups in Accumulo.
+   *
+   * @return Set of resource groups in accumulo
+   * @since 4.0.0
+   */
+  Set<ResourceGroupId> list();
+
+  /**
+   * Create a configuration node in zookeeper for a resource group. If not 
defined, then processes
+   * running in the resource group will use the values of the properties 
defined in the system
+   * configuration.

Review Comment:
   This is currently incorrect. If not defined, processes assigned to run in 
the resource group will fail to start. This somewhat presents a problem for 
users that define their cluster deployment using the cluster.yaml in that they 
can't start that deployment until they first start the instance and define 
create the resource groups. Other options include:
   
     1. having the process *not* fail if the resource group configuration node 
is not defined.
     2. Modify accumulo-cluster to create the resource group configuration 
nodes.
     3. Modify AbstractServer to create the resource group configuration node 
if it doesn't exist.
   



##########
minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java:
##########
@@ -552,7 +552,11 @@ public synchronized void start() throws IOException, 
InterruptedException {
 
       if (!config.useExistingZooKeepers()) {
         log.warn("Starting ZooKeeper");
-        control.start(ServerType.ZOOKEEPER);
+        try {
+          control.start(ServerType.ZOOKEEPER);
+        } catch (IOException | KeeperException | InterruptedException e) {
+          throw new IOException("Error starting zookeeper", e);

Review Comment:
   These new try/catch blocks are here because adding KeeperException to the 
method causes apilyzer violations.



##########
server/base/src/main/java/org/apache/accumulo/server/conf/ServerConfigurationFactory.java:
##########
@@ -75,13 +77,14 @@ public class ServerConfigurationFactory extends 
ServerConfiguration {
 
   private final ConfigRefreshRunner refresher;
 
-  public ServerConfigurationFactory(ServerContext context, SiteConfiguration 
siteConfig) {
+  public ServerConfigurationFactory(ServerContext context, SiteConfiguration 
siteConfig,
+      ResourceGroupId rgid) {
     this.context = context;
     this.siteConfig = siteConfig;
     this.systemConfig = memoize(() -> {
       var sysConf = new SystemConfiguration(context, SystemPropKey.of(), 
siteConfig);
       ConfigCheckUtil.validate(sysConf, "system config");
-      return sysConf;
+      return new ResourceGroupConfiguration(context, 
ResourceGroupPropKey.of(rgid), sysConf);

Review Comment:
   This is where the server processes fail if the resource group is not defined 
before the process is started.



##########
shell/src/main/java/org/apache/accumulo/shell/commands/ResourceGroupCommand.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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
+ *
+ *   https://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.shell.commands;
+
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.accumulo.core.client.admin.ResourceGroupOperations;
+import org.apache.accumulo.core.data.ResourceGroupId;
+import org.apache.accumulo.shell.Shell;
+import org.apache.accumulo.shell.Shell.Command;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+
+public class ResourceGroupCommand extends Command {
+
+  private Option createOpt;
+  private Option listOpt;
+  private Option deleteOpt;
+
+  @Override
+  public int execute(String fullCommand, CommandLine cl, Shell shellState) 
throws Exception {
+
+    final ResourceGroupOperations ops = 
shellState.getAccumuloClient().resourceGroupOperations();
+
+    if (cl.hasOption(createOpt.getOpt())) {
+      ops.create(ResourceGroupId.of(cl.getOptionValue(createOpt)));
+    } else if (cl.hasOption(deleteOpt.getOpt())) {
+      ops.remove(ResourceGroupId.of(cl.getOptionValue(deleteOpt)));
+    } else if (cl.hasOption(listOpt.getOpt())) {
+      Set<String> sorted = new TreeSet<>();
+      ops.list().forEach(rg -> sorted.add(rg.canonical()));
+      shellState.printLines(sorted.iterator(), false);
+    }
+    return 0;
+  }
+
+  @Override
+  public Options getOptions() {
+    final Options o = new Options();
+
+    createOpt = new Option("c", "create", true, "create a resource group");
+    createOpt.setArgName("group");
+    o.addOption(createOpt);
+
+    listOpt = new Option("l", "list", false, "display resource group names");
+    o.addOption(listOpt);
+
+    deleteOpt = new Option("d", "delete", true, "delete a resource group");
+    deleteOpt.setArgName("group");
+    o.addOption(deleteOpt);
+
+    return o;
+  }
+
+  @Override
+  public String description() {
+    return "create, list, or remove resource groups";

Review Comment:
   do we need separate `createresourcegroup` and `deleteresourcegroup` commands?



##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ResourceGroupOperationsImpl.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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
+ *
+ *   https://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.clientImpl;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import java.time.Duration;
+import java.util.ConcurrentModificationException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Consumer;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.ResourceGroupNotFoundException;
+import org.apache.accumulo.core.client.admin.ResourceGroupOperations;
+import org.apache.accumulo.core.clientImpl.thrift.ConfigurationType;
+import org.apache.accumulo.core.clientImpl.thrift.TVersionedProperties;
+import 
org.apache.accumulo.core.clientImpl.thrift.ThriftResourceGroupNotExistsException;
+import org.apache.accumulo.core.conf.DeprecatedPropertyUtil;
+import org.apache.accumulo.core.data.ResourceGroupId;
+import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.util.LocalityGroupUtil;
+import 
org.apache.accumulo.core.util.LocalityGroupUtil.LocalityGroupConfigurationError;
+import org.apache.accumulo.core.util.Retry;
+import org.slf4j.LoggerFactory;
+
+public class ResourceGroupOperationsImpl implements ResourceGroupOperations {
+
+  private final ClientContext context;
+
+  public ResourceGroupOperationsImpl(ClientContext context) {
+    checkArgument(context != null, "context is null");
+    this.context = context;
+  }
+
+  @Override
+  public boolean exists(String group) {
+    Objects.requireNonNull(group, "group parameter must be supplied");
+    return list().contains(ResourceGroupId.of(group));
+  }
+
+  @Override
+  public Set<ResourceGroupId> list() {
+    Set<ResourceGroupId> groups = new HashSet<>();
+    context.getZooCache().getChildren(Constants.ZRESOURCEGROUPS)
+        .forEach(c -> groups.add(ResourceGroupId.of(c)));
+    return Set.copyOf(groups);
+  }
+
+  @Override
+  public void create(ResourceGroupId group) throws AccumuloException, 
AccumuloSecurityException {
+    ThriftClientTypes.MANAGER.executeVoid(context, client -> client
+        .createResourceGroupNode(TraceUtil.traceInfo(), context.rpcCreds(), 
group.canonical()));
+  }
+
+  @Override
+  public Map<String,String> getConfiguration(ResourceGroupId group)
+      throws AccumuloException, AccumuloSecurityException, 
ResourceGroupNotFoundException {
+    Map<String,String> config = new HashMap<>();
+    config.putAll(ThriftClientTypes.CLIENT.execute(context, client -> client
+        .getConfiguration(TraceUtil.traceInfo(), context.rpcCreds(), 
ConfigurationType.CURRENT)));
+    config.putAll(getProperties(group));
+    return Map.copyOf(config);
+  }
+
+  @Override
+  public Map<String,String> getProperties(ResourceGroupId group)
+      throws AccumuloException, AccumuloSecurityException, 
ResourceGroupNotFoundException {
+    try {
+      TVersionedProperties rgProps = ThriftClientTypes.CLIENT.execute(context,
+          client -> 
client.getVersionedResourceGroupProperties(TraceUtil.traceInfo(),
+              context.rpcCreds(), group.canonical()));
+      if (rgProps != null && rgProps.getPropertiesSize() > 0) {
+        return Map.copyOf(rgProps.getProperties());
+      } else {
+        return Map.of();
+      }
+    } catch (AccumuloException | AccumuloSecurityException e) {
+      Throwable t = e.getCause();
+      if (t instanceof ThriftResourceGroupNotExistsException) {
+        ThriftResourceGroupNotExistsException te = 
(ThriftResourceGroupNotExistsException) t;
+        throw new ResourceGroupNotFoundException(te.getResourceGroupName());
+      }
+      throw e;
+    }
+  }
+
+  @Override
+  public void setProperty(ResourceGroupId group, String property, String value)
+      throws AccumuloException, AccumuloSecurityException, 
ResourceGroupNotFoundException {
+    checkArgument(property != null, "property is null");
+    checkArgument(value != null, "value is null");
+    DeprecatedPropertyUtil.getReplacementName(property, (log, replacement) -> {
+      // force a warning on the client side, but send the name the user used 
to the server-side
+      // to trigger a warning in the server logs, and to handle it there
+      log.warn("{} was deprecated and will be removed in a future release;"
+          + " setting its replacement {} instead", property, replacement);
+    });
+    try {
+      ThriftClientTypes.MANAGER.executeVoid(context,
+          client -> client.setResourceGroupProperty(TraceUtil.traceInfo(), 
context.rpcCreds(),
+              group.canonical(), property, value));
+    } catch (AccumuloException | AccumuloSecurityException e) {
+      Throwable t = e.getCause();
+      if (t instanceof ThriftResourceGroupNotExistsException) {
+        ThriftResourceGroupNotExistsException te = 
(ThriftResourceGroupNotExistsException) t;
+        throw new ResourceGroupNotFoundException(te.getResourceGroupName());
+      }
+      throw e;
+    }
+    checkLocalityGroups(property);
+  }
+
+  private Map<String,String> tryToModifyProperties(final ResourceGroupId group,
+      final Consumer<Map<String,String>> mapMutator) throws AccumuloException,
+      AccumuloSecurityException, IllegalArgumentException, 
ResourceGroupNotFoundException {
+    checkArgument(mapMutator != null, "mapMutator is null");
+
+    TVersionedProperties vProperties;
+    try {
+      vProperties = ThriftClientTypes.CLIENT.execute(context,
+          client -> 
client.getVersionedResourceGroupProperties(TraceUtil.traceInfo(),
+              context.rpcCreds(), group.canonical()));
+    } catch (AccumuloException | AccumuloSecurityException e) {
+      Throwable t = e.getCause();
+      if (t instanceof ThriftResourceGroupNotExistsException) {
+        ThriftResourceGroupNotExistsException te = 
(ThriftResourceGroupNotExistsException) t;
+        throw new ResourceGroupNotFoundException(te.getResourceGroupName());
+      }
+      throw e;
+    }
+    mapMutator.accept(vProperties.getProperties());
+
+    // A reference to the map was passed to the user, maybe they still have 
the reference and are
+    // modifying it. Buggy Accumulo code could attempt to make modifications 
to the map after this
+    // point. Because of these potential issues, create an immutable snapshot 
of the map so that
+    // from here on the code is assured to always be dealing with the same map.
+    vProperties.setProperties(Map.copyOf(vProperties.getProperties()));
+
+    for (Map.Entry<String,String> entry : 
vProperties.getProperties().entrySet()) {
+      final String property = Objects.requireNonNull(entry.getKey(), "property 
key is null");
+
+      DeprecatedPropertyUtil.getReplacementName(property, (log, replacement) 
-> {
+        // force a warning on the client side, but send the name the user used 
to the
+        // server-side
+        // to trigger a warning in the server logs, and to handle it there
+        log.warn("{} was deprecated and will be removed in a future release;"
+            + " setting its replacement {} instead", property, replacement);
+      });
+      checkLocalityGroups(property);
+    }
+
+    // Send to server
+    try {
+      ThriftClientTypes.MANAGER.executeVoid(context,
+          client -> 
client.modifyResourceGroupProperties(TraceUtil.traceInfo(), context.rpcCreds(),
+              group.canonical(), vProperties));
+    } catch (AccumuloException | AccumuloSecurityException e) {
+      Throwable t = e.getCause();
+      if (t instanceof ThriftResourceGroupNotExistsException) {
+        ThriftResourceGroupNotExistsException te = 
(ThriftResourceGroupNotExistsException) t;
+        throw new ResourceGroupNotFoundException(te.getResourceGroupName());
+      }
+      throw e;
+    }
+
+    return vProperties.getProperties();
+  }
+
+  @Override
+  public Map<String,String> modifyProperties(ResourceGroupId group,
+      Consumer<Map<String,String>> mapMutator) throws AccumuloException, 
AccumuloSecurityException,
+      IllegalArgumentException, ResourceGroupNotFoundException {
+
+    var log = LoggerFactory.getLogger(InstanceOperationsImpl.class);
+
+    Retry retry = 
Retry.builder().infiniteRetries().retryAfter(Duration.ofMillis(25))
+        
.incrementBy(Duration.ofMillis(25)).maxWait(Duration.ofSeconds(30)).backOffFactor(1.5)
+        .logInterval(Duration.ofMinutes(3)).createRetry();
+
+    while (true) {
+      try {
+        var props = tryToModifyProperties(group, mapMutator);
+        retry.logCompletion(log, "Modifying resource group properties");
+        return props;
+      } catch (ConcurrentModificationException cme) {
+        try {
+          retry.logRetry(log,
+              "Unable to modify resource group properties for because of 
concurrent modification");
+          retry.waitForNextAttempt(log, "Modify resource group properties");
+        } catch (InterruptedException e) {
+          throw new RuntimeException(e);
+        }
+      } finally {
+        retry.useRetry();
+      }
+    }
+  }
+
+  @Override
+  public void removeProperty(ResourceGroupId group, String property)
+      throws AccumuloException, AccumuloSecurityException, 
ResourceGroupNotFoundException {
+    checkArgument(property != null, "property is null");
+    DeprecatedPropertyUtil.getReplacementName(property, (log, replacement) -> {
+      // force a warning on the client side, but send the name the user used 
to the server-side
+      // to trigger a warning in the server logs, and to handle it there
+      log.warn("{} was deprecated and will be removed in a future release; 
assuming user meant"
+          + " its replacement {} and will remove that instead", property, 
replacement);
+    });
+    try {
+      ThriftClientTypes.MANAGER.executeVoid(context,
+          client -> client.removeResourceGroupProperty(TraceUtil.traceInfo(), 
context.rpcCreds(),
+              group.canonical(), property));
+    } catch (AccumuloException | AccumuloSecurityException e) {
+      Throwable t = e.getCause();
+      if (t instanceof ThriftResourceGroupNotExistsException) {
+        ThriftResourceGroupNotExistsException te = 
(ThriftResourceGroupNotExistsException) t;
+        throw new ResourceGroupNotFoundException(te.getResourceGroupName());
+      }
+      throw e;
+    }
+    checkLocalityGroups(property);
+  }
+
+  @Override
+  public void remove(ResourceGroupId group)
+      throws AccumuloException, AccumuloSecurityException, 
ResourceGroupNotFoundException {
+    try {
+      ThriftClientTypes.MANAGER.executeVoid(context, client -> client
+          .removeResourceGroupNode(TraceUtil.traceInfo(), context.rpcCreds(), 
group.canonical()));
+    } catch (AccumuloException | AccumuloSecurityException e) {
+      Throwable t = e.getCause();
+      if (t instanceof ThriftResourceGroupNotExistsException) {
+        ThriftResourceGroupNotExistsException te = 
(ThriftResourceGroupNotExistsException) t;
+        throw new ResourceGroupNotFoundException(te.getResourceGroupName());
+      }
+      throw e;
+    }
+  }
+
+  private void checkLocalityGroups(String propChanged)

Review Comment:
   Copied this from a different class, not sure if it's needed here.



##########
server/base/src/main/java/org/apache/accumulo/server/util/ResourceGroupPropUtil.java:
##########
@@ -0,0 +1,87 @@
+/*
+ * 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
+ *
+ *   https://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.util;
+
+import static org.apache.accumulo.core.conf.Property.COMPACTION_PREFIX;
+import static org.apache.accumulo.core.conf.Property.COMPACTOR_PREFIX;
+import static org.apache.accumulo.core.conf.Property.SSERV_PREFIX;
+import static org.apache.accumulo.core.conf.Property.TSERV_PREFIX;
+
+import org.apache.accumulo.core.conf.DeprecatedPropertyUtil;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.conf.PropertyType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ResourceGroupPropUtil {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ResourceGroupPropUtil.class);
+
+  public static String validateResourceGroupProperty(String property, String 
value) {
+    // Retrieve the replacement name for this property, if there is one.
+    // Do this before we check if the name is a valid zookeeper name.
+    final var original = property;
+    property = DeprecatedPropertyUtil.getReplacementName(property,
+        (log, replacement) -> log.warn("{} was deprecated and will be removed 
in a future release;"
+            + " setting its replacement {} instead", original, replacement));
+
+    if (property.startsWith(COMPACTION_PREFIX.getKey())
+        || property.startsWith(COMPACTOR_PREFIX.getKey())
+        || property.startsWith(SSERV_PREFIX.getKey())
+        || property.startsWith(TSERV_PREFIX.getKey())) {

Review Comment:
   Allowed resource group properties. Put in Property.java?



-- 
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-unsubscr...@accumulo.apache.org

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

Reply via email to