You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by dl...@apache.org on 2023/02/11 13:38:27 UTC

[accumulo] branch main updated: Removed arguments from server processes (#3192)

This is an automated email from the ASF dual-hosted git repository.

dlmarion pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/main by this push:
     new b74c97c260 Removed arguments from server processes (#3192)
b74c97c260 is described below

commit b74c97c260d6867f242945ff65e718e5819be229
Author: Dave Marion <dl...@apache.org>
AuthorDate: Sat Feb 11 08:38:21 2023 -0500

    Removed arguments from server processes (#3192)
    
    Removed ServerOpts and its subclasses. Replaced
    options with new Property definitions which can be
    passed to the server processes using the existing -o
    argument processing or set in the properties files.
    For example, instead of using "-a 127.0.0.1" to set
    the bind address for the server, the user will now use
    "-o general.process.bind.addr=127.0.0.1"
---
 .../org/apache/accumulo/core/conf/Property.java    | 14 +++++++++
 .../apache/accumulo/core/cli/ConfigOptsTest.java   | 20 ++++++------
 .../MiniAccumuloClusterControl.java                |  5 ++-
 .../org/apache/accumulo/server/AbstractServer.java |  7 +++--
 .../org/apache/accumulo/server/ServerOpts.java     | 36 ----------------------
 .../coordinator/CompactionCoordinator.java         |  8 ++---
 .../coordinator/CompactionCoordinatorTest.java     |  4 +--
 server/compactor/pom.xml                           |  4 ---
 .../org/apache/accumulo/compactor/Compactor.java   | 20 +++---------
 .../apache/accumulo/compactor/CompactorTest.java   |  5 ++-
 .../apache/accumulo/gc/SimpleGarbageCollector.java |  6 ++--
 .../java/org/apache/accumulo/manager/Manager.java  |  6 ++--
 .../java/org/apache/accumulo/monitor/Monitor.java  |  6 ++--
 .../org/apache/accumulo/tserver/ScanServer.java    | 22 +++----------
 .../org/apache/accumulo/tserver/TabletServer.java  |  6 ++--
 .../apache/accumulo/tserver/ScanServerTest.java    |  3 +-
 .../test/compaction/ExternalCompactionTServer.java |  6 ++--
 .../compaction/ExternalDoNothingCompactor.java     |  5 +--
 .../TestCompactionCoordinatorForOfflineTable.java  |  6 ++--
 19 files changed, 75 insertions(+), 114 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index 32e1b8714d..93fad5e98e 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -36,6 +36,7 @@ import org.apache.accumulo.core.spi.compaction.SimpleCompactionDispatcher;
 import org.apache.accumulo.core.spi.fs.RandomVolumeChooser;
 import org.apache.accumulo.core.spi.scan.ScanDispatcher;
 import org.apache.accumulo.core.spi.scan.ScanPrioritizer;
+import org.apache.accumulo.core.spi.scan.ScanServerSelector;
 import org.apache.accumulo.core.spi.scan.SimpleScanDispatcher;
 import org.apache.accumulo.core.util.format.DefaultFormatter;
 import org.slf4j.LoggerFactory;
@@ -284,6 +285,9 @@ public enum Property {
       PropertyType.BOOLEAN, "Enables JVM metrics functionality using Micrometer", "2.1.0"),
   GENERAL_MICROMETER_FACTORY("general.micrometer.factory", "", PropertyType.CLASSNAME,
       "Name of class that implements MeterRegistryFactory", "2.1.0"),
+  GENERAL_PROCESS_BIND_ADDRESS("general.process.bind.addr", "0.0.0.0", PropertyType.STRING,
+      "The local IP address to which this server should bind for sending and receiving network traffic",
+      "3.0.0"),
   // properties that are specific to manager server behavior
   MANAGER_PREFIX("manager.", null, PropertyType.PREFIX,
       "Properties in this category affect the behavior of the manager server. "
@@ -375,6 +379,13 @@ public enum Property {
   SSERV_DEFAULT_BLOCKSIZE("sserver.default.blocksize", "1M", PropertyType.BYTES,
       "Specifies a default blocksize for the scan server caches", "2.1.0"),
   @Experimental
+  SSERV_GROUP_NAME("sserver.group", ScanServerSelector.DEFAULT_SCAN_SERVER_GROUP_NAME,
+      PropertyType.STRING,
+      "Optional group name that will be made available to the "
+          + "ScanServerSelector client plugin. Groups support at least two use cases:"
+          + " dedicating resources to scans and/or using different hardware for scans.",
+      "3.0.0"),
+  @Experimental
   SSERV_CACHED_TABLET_METADATA_EXPIRATION("sserver.cache.metadata.expiration", "5m",
       PropertyType.TIMEDURATION, "The time after which cached tablet metadata will be refreshed.",
       "2.1.0"),
@@ -1141,6 +1152,9 @@ public enum Property {
   @Experimental
   COMPACTOR_MAX_MESSAGE_SIZE("compactor.message.size.max", "10M", PropertyType.BYTES,
       "The maximum size of a message that can be sent to a tablet server.", "2.1.0"),
+  @Experimental
+  COMPACTOR_QUEUE_NAME("compactor.queue", "", PropertyType.STRING,
+      "The queue for which this Compactor will perform compactions", "3.0.0"),
   // CompactionCoordinator properties
   @Experimental
   COMPACTION_COORDINATOR_PREFIX("compaction.coordinator.", null, PropertyType.PREFIX,
diff --git a/server/base/src/test/java/org/apache/accumulo/server/ServerOptsTest.java b/core/src/test/java/org/apache/accumulo/core/cli/ConfigOptsTest.java
similarity index 73%
rename from server/base/src/test/java/org/apache/accumulo/server/ServerOptsTest.java
rename to core/src/test/java/org/apache/accumulo/core/cli/ConfigOptsTest.java
index 1cde30e02e..b5a38dec3b 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/ServerOptsTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/cli/ConfigOptsTest.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.server;
+package org.apache.accumulo.core.cli;
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
@@ -26,32 +26,34 @@ import org.apache.accumulo.core.conf.Property;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 
-public class ServerOptsTest {
-  private ServerOpts opts;
+public class ConfigOptsTest {
+  private ConfigOpts opts;
 
   @BeforeEach
   public void setUp() {
-    opts = new ServerOpts();
+    opts = new ConfigOpts();
   }
 
   @Test
   public void testGetAddress() {
-    opts.parseArgs(ServerOptsTest.class.getName(), new String[] {"-a", "1.2.3.4"});
-    assertEquals("1.2.3.4", opts.getAddress());
+    opts.parseArgs(ConfigOptsTest.class.getName(),
+        new String[] {"-o", Property.GENERAL_PROCESS_BIND_ADDRESS.getKey() + "=1.2.3.4"});
+    assertEquals("1.2.3.4", opts.getSiteConfiguration().get(Property.GENERAL_PROCESS_BIND_ADDRESS));
   }
 
   @Test
   public void testGetAddress_NOne() {
-    opts.parseArgs(ServerOptsTest.class.getName(), new String[] {});
-    assertEquals("0.0.0.0", opts.getAddress());
+    opts.parseArgs(ConfigOptsTest.class.getName(), new String[] {});
+    assertEquals("0.0.0.0", opts.getSiteConfiguration().get(Property.GENERAL_PROCESS_BIND_ADDRESS));
   }
 
   @Test
   public void testOverrideConfig() {
     AccumuloConfiguration defaults = DefaultConfiguration.getInstance();
     assertEquals("localhost:2181", defaults.get(Property.INSTANCE_ZK_HOST));
-    opts.parseArgs(ServerOptsTest.class.getName(),
+    opts.parseArgs(ConfigOptsTest.class.getName(),
         new String[] {"-o", "instance.zookeeper.host=test:123"});
     assertEquals("test:123", opts.getSiteConfiguration().get(Property.INSTANCE_ZK_HOST));
   }
+
 }
diff --git a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java
index ca3480cfeb..a25e1e211e 100644
--- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java
@@ -38,6 +38,7 @@ import org.apache.accumulo.coordinator.CompactionCoordinator;
 import org.apache.accumulo.core.clientImpl.ClientContext;
 import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService;
 import org.apache.accumulo.core.compaction.thrift.TExternalCompactionList;
+import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
 import org.apache.accumulo.core.trace.TraceUtil;
@@ -173,7 +174,9 @@ public class MiniAccumuloClusterControl implements ClusterControl {
       int count =
           Math.min(limit, cluster.getConfig().getNumCompactors() - compactorProcesses.size());
       for (int i = 0; i < count; i++) {
-        compactorProcesses.add(cluster.exec(compactor, "-q", queueName).getProcess());
+        compactorProcesses.add(
+            cluster.exec(compactor, "-o", Property.COMPACTOR_QUEUE_NAME.getKey() + "=" + queueName)
+                .getProcess());
       }
     }
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/AbstractServer.java b/server/base/src/main/java/org/apache/accumulo/server/AbstractServer.java
index a23ba911db..9d1d4c4a0d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/AbstractServer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/AbstractServer.java
@@ -18,12 +18,13 @@
  */
 package org.apache.accumulo.server;
 
-import java.util.Objects;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.classloader.ClassLoaderUtil;
+import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.metrics.MetricsUtil;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.server.security.SecurityUtil;
@@ -37,12 +38,12 @@ public abstract class AbstractServer implements AutoCloseable, Runnable {
   private final String hostname;
   private final Logger log;
 
-  protected AbstractServer(String appName, ServerOpts opts, String[] args) {
+  protected AbstractServer(String appName, ConfigOpts opts, String[] args) {
     this.log = LoggerFactory.getLogger(getClass().getName());
     this.applicationName = appName;
     opts.parseArgs(appName, args);
-    this.hostname = Objects.requireNonNull(opts.getAddress());
     var siteConfig = opts.getSiteConfiguration();
+    this.hostname = siteConfig.get(Property.GENERAL_PROCESS_BIND_ADDRESS);
     SecurityUtil.serverLogin(siteConfig);
     context = new ServerContext(siteConfig);
     log.info("Version " + Constants.VERSION);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/ServerOpts.java b/server/base/src/main/java/org/apache/accumulo/server/ServerOpts.java
deleted file mode 100644
index 3bde48f436..0000000000
--- a/server/base/src/main/java/org/apache/accumulo/server/ServerOpts.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * 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;
-
-import org.apache.accumulo.core.cli.ConfigOpts;
-
-import com.beust.jcommander.Parameter;
-
-public class ServerOpts extends ConfigOpts {
-
-  @Parameter(names = {"-a", "--address"}, description = "address to bind to")
-  private String address = null;
-
-  public String getAddress() {
-    if (address != null) {
-      return address;
-    }
-    return "0.0.0.0";
-  }
-}
diff --git a/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java b/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
index dea9421ded..964aef284e 100644
--- a/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
+++ b/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
@@ -36,6 +36,7 @@ import java.util.stream.Collectors;
 
 import org.apache.accumulo.coordinator.QueueSummaries.PrioTserver;
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
@@ -76,7 +77,6 @@ import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.server.AbstractServer;
 import org.apache.accumulo.server.GarbageCollectionLogger;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.ServerOpts;
 import org.apache.accumulo.server.manager.LiveTServerSet;
 import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
 import org.apache.accumulo.server.rpc.ServerAddress;
@@ -133,11 +133,11 @@ public class CompactionCoordinator extends AbstractServer
 
   private ScheduledThreadPoolExecutor schedExecutor;
 
-  protected CompactionCoordinator(ServerOpts opts, String[] args) {
+  protected CompactionCoordinator(ConfigOpts opts, String[] args) {
     this(opts, args, null);
   }
 
-  protected CompactionCoordinator(ServerOpts opts, String[] args, AccumuloConfiguration conf) {
+  protected CompactionCoordinator(ConfigOpts opts, String[] args, AccumuloConfiguration conf) {
     super("compaction-coordinator", opts, args);
     aconf = conf == null ? super.getConfiguration() : conf;
     schedExecutor = ThreadPools.getServerThreadPools().createGeneralScheduledExecutorService(aconf);
@@ -732,7 +732,7 @@ public class CompactionCoordinator extends AbstractServer
   }
 
   public static void main(String[] args) throws Exception {
-    try (CompactionCoordinator compactor = new CompactionCoordinator(new ServerOpts(), args)) {
+    try (CompactionCoordinator compactor = new CompactionCoordinator(new ConfigOpts(), args)) {
       compactor.runServer();
     }
   }
diff --git a/server/compaction-coordinator/src/test/java/org/apache/accumulo/coordinator/CompactionCoordinatorTest.java b/server/compaction-coordinator/src/test/java/org/apache/accumulo/coordinator/CompactionCoordinatorTest.java
index 17b0ae60d2..7225960a53 100644
--- a/server/compaction-coordinator/src/test/java/org/apache/accumulo/coordinator/CompactionCoordinatorTest.java
+++ b/server/compaction-coordinator/src/test/java/org/apache/accumulo/coordinator/CompactionCoordinatorTest.java
@@ -37,6 +37,7 @@ import java.util.TreeSet;
 import java.util.UUID;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 
+import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.core.clientImpl.thrift.TInfo;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.compaction.thrift.TExternalCompaction;
@@ -56,7 +57,6 @@ import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
 import org.apache.accumulo.core.util.compaction.RunningCompaction;
 import org.apache.accumulo.server.AbstractServer;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.ServerOpts;
 import org.apache.accumulo.server.manager.LiveTServerSet;
 import org.apache.accumulo.server.rpc.ServerAddress;
 import org.apache.accumulo.server.security.AuditedSecurityOperation;
@@ -93,7 +93,7 @@ public class CompactionCoordinatorTest {
     protected TestCoordinator(CompactionFinalizer finalizer, LiveTServerSet tservers,
         ServerAddress client, Client tabletServerClient, ServerContext context,
         AuditedSecurityOperation security) {
-      super(new ServerOpts(), new String[] {}, context.getConfiguration());
+      super(new ConfigOpts(), new String[] {}, context.getConfiguration());
       this.compactionFinalizer = finalizer;
       this.tserverSet = tservers;
       this.client = client;
diff --git a/server/compactor/pom.xml b/server/compactor/pom.xml
index a6e303b142..4cfb414424 100644
--- a/server/compactor/pom.xml
+++ b/server/compactor/pom.xml
@@ -30,10 +30,6 @@
   <artifactId>accumulo-compactor</artifactId>
   <name>Apache Accumulo Compactor</name>
   <dependencies>
-    <dependency>
-      <groupId>com.beust</groupId>
-      <artifactId>jcommander</artifactId>
-    </dependency>
     <dependency>
       <groupId>com.google.auto.service</groupId>
       <artifactId>auto-service</artifactId>
diff --git a/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java b/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
index 45135e7466..1151c5178e 100644
--- a/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
+++ b/server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
@@ -41,6 +41,7 @@ import java.util.concurrent.atomic.LongAdder;
 import java.util.function.Supplier;
 
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.TableNotFoundException;
@@ -93,7 +94,6 @@ import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.core.util.threads.Threads;
 import org.apache.accumulo.server.AbstractServer;
 import org.apache.accumulo.server.GarbageCollectionLogger;
-import org.apache.accumulo.server.ServerOpts;
 import org.apache.accumulo.server.compaction.CompactionInfo;
 import org.apache.accumulo.server.compaction.CompactionWatcher;
 import org.apache.accumulo.server.compaction.FileCompactor;
@@ -112,7 +112,6 @@ import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.beust.jcommander.Parameter;
 import com.google.common.base.Preconditions;
 import com.google.common.net.HostAndPort;
 
@@ -123,15 +122,6 @@ public class Compactor extends AbstractServer implements MetricsProducer, Compac
 
   private static final SecureRandom random = new SecureRandom();
 
-  public static class CompactorServerOpts extends ServerOpts {
-    @Parameter(required = true, names = {"-q", "--queue"}, description = "compaction queue name")
-    private String queueName = null;
-
-    public String getQueueName() {
-      return queueName;
-    }
-  }
-
   private static final Logger LOG = LoggerFactory.getLogger(Compactor.class);
   private static final long TIME_BETWEEN_GC_CHECKS = 5000;
   private static final long TIME_BETWEEN_CANCEL_CHECKS = MINUTES.toMillis(5);
@@ -157,14 +147,14 @@ public class Compactor extends AbstractServer implements MetricsProducer, Compac
 
   private final AtomicBoolean compactionRunning = new AtomicBoolean(false);
 
-  protected Compactor(CompactorServerOpts opts, String[] args) {
+  protected Compactor(ConfigOpts opts, String[] args) {
     this(opts, args, null);
   }
 
-  protected Compactor(CompactorServerOpts opts, String[] args, AccumuloConfiguration conf) {
+  protected Compactor(ConfigOpts opts, String[] args, AccumuloConfiguration conf) {
     super("compactor", opts, args);
-    queueName = opts.getQueueName();
     aconf = conf == null ? super.getConfiguration() : conf;
+    queueName = aconf.get(Property.COMPACTOR_QUEUE_NAME);
     setupSecurity();
     watcher = new CompactionWatcher(aconf);
     var schedExecutor =
@@ -810,7 +800,7 @@ public class Compactor extends AbstractServer implements MetricsProducer, Compac
   }
 
   public static void main(String[] args) throws Exception {
-    try (Compactor compactor = new Compactor(new CompactorServerOpts(), args)) {
+    try (Compactor compactor = new Compactor(new ConfigOpts(), args)) {
       compactor.runServer();
     }
   }
diff --git a/server/compactor/src/test/java/org/apache/accumulo/compactor/CompactorTest.java b/server/compactor/src/test/java/org/apache/accumulo/compactor/CompactorTest.java
index 83ca676294..11b68e007a 100644
--- a/server/compactor/src/test/java/org/apache/accumulo/compactor/CompactorTest.java
+++ b/server/compactor/src/test/java/org/apache/accumulo/compactor/CompactorTest.java
@@ -33,6 +33,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.atomic.LongAdder;
 import java.util.function.Supplier;
 
+import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.core.compaction.thrift.TCompactionState;
 import org.apache.accumulo.core.compaction.thrift.TCompactionStatusUpdate;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
@@ -173,7 +174,9 @@ public class CompactorTest {
 
     SuccessfulCompactor(Supplier<UUID> uuid, ServerAddress address, TExternalCompactionJob job,
         ServerContext context, ExternalCompactionId eci) {
-      super(new CompactorServerOpts(), new String[] {"-q", "testQ"}, context.getConfiguration());
+      super(new ConfigOpts(),
+          new String[] {"-o", Property.COMPACTOR_QUEUE_NAME.getKey() + "=testQ"},
+          context.getConfiguration());
       this.uuid = uuid;
       this.address = address;
       this.job = job;
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
index 8f13fb3e0c..c2bdf6c6ac 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
@@ -28,6 +28,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.stream.IntStream;
 
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.clientImpl.thrift.TInfo;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
@@ -51,7 +52,6 @@ import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.gc.metrics.GcCycleMetrics;
 import org.apache.accumulo.gc.metrics.GcMetrics;
 import org.apache.accumulo.server.AbstractServer;
-import org.apache.accumulo.server.ServerOpts;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.manager.LiveTServerSet;
 import org.apache.accumulo.server.rpc.ServerAddress;
@@ -80,7 +80,7 @@ public class SimpleGarbageCollector extends AbstractServer implements Iface {
 
   private final GcCycleMetrics gcCycleMetrics = new GcCycleMetrics();
 
-  SimpleGarbageCollector(ServerOpts opts, String[] args) {
+  SimpleGarbageCollector(ConfigOpts opts, String[] args) {
     super("gc", opts, args);
 
     final AccumuloConfiguration conf = getConfiguration();
@@ -97,7 +97,7 @@ public class SimpleGarbageCollector extends AbstractServer implements Iface {
   }
 
   public static void main(String[] args) throws Exception {
-    try (SimpleGarbageCollector gc = new SimpleGarbageCollector(new ServerOpts(), args)) {
+    try (SimpleGarbageCollector gc = new SimpleGarbageCollector(new ConfigOpts(), args)) {
       gc.runServer();
     }
   }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java
index 9177f7fa1a..5469b0c3d4 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java
@@ -50,6 +50,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.stream.Collectors;
 
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
@@ -114,7 +115,6 @@ import org.apache.accumulo.manager.upgrade.UpgradeCoordinator;
 import org.apache.accumulo.server.AbstractServer;
 import org.apache.accumulo.server.HighlyAvailableService;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.ServerOpts;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.manager.LiveTServerSet;
 import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
@@ -403,12 +403,12 @@ public class Manager extends AbstractServer
   }
 
   public static void main(String[] args) throws Exception {
-    try (Manager manager = new Manager(new ServerOpts(), args)) {
+    try (Manager manager = new Manager(new ConfigOpts(), args)) {
       manager.runServer();
     }
   }
 
-  Manager(ServerOpts opts, String[] args) throws IOException {
+  Manager(ConfigOpts opts, String[] args) throws IOException {
     super("manager", opts, args);
     ServerContext context = super.getContext();
     balancerEnvironment = new BalancerEnvironmentImpl(context);
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
index 40d4a907e4..e8fe48e4a5 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
@@ -46,6 +46,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import jakarta.inject.Singleton;
 
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService;
 import org.apache.accumulo.core.compaction.thrift.TExternalCompaction;
 import org.apache.accumulo.core.compaction.thrift.TExternalCompactionList;
@@ -80,7 +81,6 @@ import org.apache.accumulo.monitor.util.logging.RecentLogs;
 import org.apache.accumulo.server.AbstractServer;
 import org.apache.accumulo.server.HighlyAvailableService;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.ServerOpts;
 import org.apache.accumulo.server.problems.ProblemReports;
 import org.apache.accumulo.server.problems.ProblemType;
 import org.apache.accumulo.server.util.TableInfoUtil;
@@ -112,12 +112,12 @@ public class Monitor extends AbstractServer implements HighlyAvailableService {
   private final long START_TIME;
 
   public static void main(String[] args) throws Exception {
-    try (Monitor monitor = new Monitor(new ServerOpts(), args)) {
+    try (Monitor monitor = new Monitor(new ConfigOpts(), args)) {
       monitor.runServer();
     }
   }
 
-  Monitor(ServerOpts opts, String[] args) {
+  Monitor(ConfigOpts opts, String[] args) {
     super("monitor", opts, args);
     START_TIME = System.currentTimeMillis();
   }
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java
index 1e447d6ed1..39cf7a6010 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java
@@ -47,6 +47,7 @@ import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.clientImpl.thrift.TInfo;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
@@ -74,7 +75,6 @@ import org.apache.accumulo.core.metadata.schema.Ample;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.metrics.MetricsUtil;
 import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
-import org.apache.accumulo.core.spi.scan.ScanServerSelector;
 import org.apache.accumulo.core.tabletscan.thrift.ActiveScan;
 import org.apache.accumulo.core.tabletscan.thrift.TSampleNotPresentException;
 import org.apache.accumulo.core.tabletscan.thrift.TSamplerConfiguration;
@@ -88,7 +88,6 @@ import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.server.AbstractServer;
 import org.apache.accumulo.server.GarbageCollectionLogger;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.ServerOpts;
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.rpc.ServerAddress;
@@ -113,7 +112,6 @@ import org.checkerframework.checker.nullness.qual.Nullable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.beust.jcommander.Parameter;
 import com.github.benmanes.caffeine.cache.CacheLoader;
 import com.github.benmanes.caffeine.cache.Caffeine;
 import com.github.benmanes.caffeine.cache.LoadingCache;
@@ -126,18 +124,6 @@ import com.google.common.net.HostAndPort;
 public class ScanServer extends AbstractServer
     implements TabletScanClientService.Iface, TabletHostingServer {
 
-  public static class ScanServerOpts extends ServerOpts {
-    @Parameter(required = false, names = {"-g", "--group"},
-        description = "Optional group name that will be made available to the ScanServerSelector client plugin.  If not specified will be set to '"
-            + ScanServerSelector.DEFAULT_SCAN_SERVER_GROUP_NAME
-            + "'.  Groups support at least two use cases : dedicating resources to scans and/or using different hardware for scans.")
-    private String groupName = ScanServerSelector.DEFAULT_SCAN_SERVER_GROUP_NAME;
-
-    public String getGroupName() {
-      return groupName;
-    }
-  }
-
   private static final Logger log = LoggerFactory.getLogger(ScanServer.class);
 
   private static class TabletMetadataLoader implements CacheLoader<KeyExtent,TabletMetadata> {
@@ -205,7 +191,7 @@ public class ScanServer extends AbstractServer
 
   private final String groupName;
 
-  public ScanServer(ScanServerOpts opts, String[] args) {
+  public ScanServer(ConfigOpts opts, String[] args) {
     super("sserver", opts, args);
 
     context = super.getContext();
@@ -249,7 +235,7 @@ public class ScanServer extends AbstractServer
 
     delegate = newThriftScanClientHandler(new WriteTracker());
 
-    this.groupName = Objects.requireNonNull(opts.getGroupName());
+    this.groupName = getConfiguration().get(Property.SSERV_GROUP_NAME);
 
     ThreadPools.watchCriticalScheduledTask(getContext().getScheduledExecutor()
         .scheduleWithFixedDelay(() -> cleanUpReservedFiles(scanServerReservationExpiration),
@@ -1038,7 +1024,7 @@ public class ScanServer extends AbstractServer
   }
 
   public static void main(String[] args) throws Exception {
-    try (ScanServer tserver = new ScanServer(new ScanServerOpts(), args)) {
+    try (ScanServer tserver = new ScanServer(new ConfigOpts(), args)) {
       tserver.runServer();
     }
   }
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index 379434aa4a..89e8045964 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -65,6 +65,7 @@ import java.util.concurrent.locks.ReentrantLock;
 import java.util.function.Consumer;
 
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.core.client.Durability;
 import org.apache.accumulo.core.clientImpl.DurabilityImpl;
 import org.apache.accumulo.core.clientImpl.TabletLocator;
@@ -109,7 +110,6 @@ import org.apache.accumulo.core.util.threads.Threads;
 import org.apache.accumulo.server.AbstractServer;
 import org.apache.accumulo.server.GarbageCollectionLogger;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.ServerOpts;
 import org.apache.accumulo.server.TabletLevel;
 import org.apache.accumulo.server.client.ClientServiceHandler;
 import org.apache.accumulo.server.compaction.CompactionWatcher;
@@ -236,12 +236,12 @@ public class TabletServer extends AbstractServer implements TabletHostingServer
   private final ServerContext context;
 
   public static void main(String[] args) throws Exception {
-    try (TabletServer tserver = new TabletServer(new ServerOpts(), args)) {
+    try (TabletServer tserver = new TabletServer(new ConfigOpts(), args)) {
       tserver.runServer();
     }
   }
 
-  protected TabletServer(ServerOpts opts, String[] args) {
+  protected TabletServer(ConfigOpts opts, String[] args) {
     super("tserver", opts, args);
     context = super.getContext();
     this.managerLockCache = new ZooCache(context.getZooReader(), null);
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/ScanServerTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/ScanServerTest.java
index 8d6434b7f8..ee04f0c4d6 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/ScanServerTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/ScanServerTest.java
@@ -33,6 +33,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.clientImpl.thrift.TInfo;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
@@ -66,7 +67,7 @@ public class ScanServerTest {
     private TabletResolver resolver;
     private ScanReservation reservation;
 
-    protected TestScanServer(ScanServerOpts opts, String[] args) {
+    protected TestScanServer(ConfigOpts opts, String[] args) {
       super(opts, args);
     }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTServer.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTServer.java
index 8719c8860e..63fa29e156 100644
--- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTServer.java
+++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionTServer.java
@@ -18,7 +18,7 @@
  */
 package org.apache.accumulo.test.compaction;
 
-import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.server.zookeeper.TransactionWatcher;
 import org.apache.accumulo.tserver.TabletClientHandler;
 import org.apache.accumulo.tserver.TabletServer;
@@ -26,7 +26,7 @@ import org.apache.accumulo.tserver.WriteTracker;
 
 public class ExternalCompactionTServer extends TabletServer {
 
-  ExternalCompactionTServer(ServerOpts opts, String[] args) {
+  ExternalCompactionTServer(ConfigOpts opts, String[] args) {
     super(opts, args);
   }
 
@@ -38,7 +38,7 @@ public class ExternalCompactionTServer extends TabletServer {
 
   public static void main(String[] args) throws Exception {
     try (
-        ExternalCompactionTServer tserver = new ExternalCompactionTServer(new ServerOpts(), args)) {
+        ExternalCompactionTServer tserver = new ExternalCompactionTServer(new ConfigOpts(), args)) {
       tserver.runServer();
     }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalDoNothingCompactor.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalDoNothingCompactor.java
index b03e181d40..14b3329faa 100644
--- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalDoNothingCompactor.java
+++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalDoNothingCompactor.java
@@ -26,6 +26,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.atomic.LongAdder;
 
 import org.apache.accumulo.compactor.Compactor;
+import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.core.compaction.thrift.CompactorService.Iface;
 import org.apache.accumulo.core.compaction.thrift.TCompactionState;
 import org.apache.accumulo.core.compaction.thrift.TCompactionStatusUpdate;
@@ -39,7 +40,7 @@ public class ExternalDoNothingCompactor extends Compactor implements Iface {
 
   private static final Logger LOG = LoggerFactory.getLogger(ExternalDoNothingCompactor.class);
 
-  ExternalDoNothingCompactor(CompactorServerOpts opts, String[] args) {
+  ExternalDoNothingCompactor(ConfigOpts opts, String[] args) {
     super(opts, args);
   }
 
@@ -87,7 +88,7 @@ public class ExternalDoNothingCompactor extends Compactor implements Iface {
   }
 
   public static void main(String[] args) throws Exception {
-    try (var compactor = new ExternalDoNothingCompactor(new CompactorServerOpts(), args)) {
+    try (var compactor = new ExternalDoNothingCompactor(new ConfigOpts(), args)) {
       compactor.runServer();
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/TestCompactionCoordinatorForOfflineTable.java b/test/src/main/java/org/apache/accumulo/test/compaction/TestCompactionCoordinatorForOfflineTable.java
index 7e5242b6fa..8788d43236 100644
--- a/test/src/main/java/org/apache/accumulo/test/compaction/TestCompactionCoordinatorForOfflineTable.java
+++ b/test/src/main/java/org/apache/accumulo/test/compaction/TestCompactionCoordinatorForOfflineTable.java
@@ -23,13 +23,13 @@ import java.util.concurrent.ScheduledThreadPoolExecutor;
 
 import org.apache.accumulo.coordinator.CompactionCoordinator;
 import org.apache.accumulo.coordinator.CompactionFinalizer;
+import org.apache.accumulo.core.cli.ConfigOpts;
 import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionFinalState;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionFinalState.FinalState;
 import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.ServerOpts;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -62,7 +62,7 @@ public class TestCompactionCoordinatorForOfflineTable extends CompactionCoordina
 
   }
 
-  protected TestCompactionCoordinatorForOfflineTable(ServerOpts opts, String[] args) {
+  protected TestCompactionCoordinatorForOfflineTable(ConfigOpts opts, String[] args) {
     super(opts, args);
   }
 
@@ -73,7 +73,7 @@ public class TestCompactionCoordinatorForOfflineTable extends CompactionCoordina
 
   public static void main(String[] args) throws Exception {
     try (TestCompactionCoordinatorForOfflineTable coordinator =
-        new TestCompactionCoordinatorForOfflineTable(new ServerOpts(), args)) {
+        new TestCompactionCoordinatorForOfflineTable(new ConfigOpts(), args)) {
       coordinator.runServer();
     }
   }