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/14 13:49:18 UTC

[accumulo] branch main updated: Use serialized object instead of byte[] for service locks (#3189)

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 476ebf2001 Use serialized object instead of byte[] for service locks (#3189)
476ebf2001 is described below

commit 476ebf2001df4d92f54bca9c5c98691746b6df6a
Author: Dave Marion <dl...@apache.org>
AuthorDate: Tue Feb 14 08:49:12 2023 -0500

    Use serialized object instead of byte[] for service locks (#3189)
    
    Renamed ServerServices class to ServiceLockData and modified
    ServiceLock to use and return ServiceLockData objects. Prior
    to this change the various server processes were creating
    their own byte arrays for the service lock data in ZooKeeper.
    With this change, the server processes create ServiceLockData
    objects and pass that to ServiceLock, which performs the
    serialization into ZooKeeeper.
---
 .../accumulo/core/clientImpl/ClientContext.java    |  28 ++-
 .../accumulo/core/fate/zookeeper/ServiceLock.java  |  43 ++--
 .../accumulo/core/fate/zookeeper/ZooCache.java     |  16 +-
 .../core/metadata/schema/TabletMetadata.java       |  16 +-
 .../accumulo/core/rpc/clients/TServerClient.java   |  19 +-
 .../accumulo/core/spi/scan/ScanServerSelector.java |   3 +-
 .../apache/accumulo/core/util/ServerServices.java  |  93 --------
 .../apache/accumulo/core/util/ServiceLockData.java | 233 +++++++++++++++++++++
 .../util/compaction/ExternalCompactionUtil.java    |   8 +-
 .../core/metadata/schema/TabletMetadataTest.java   |   2 +-
 .../accumulo/core/util/ServiceLockDataTest.java    | 123 +++++++++++
 .../accumulo/server/manager/LiveTServerSet.java    |  11 +-
 .../apache/accumulo/server/util/ListInstances.java |   9 +-
 .../accumulo/server/util/TabletServerLocks.java    |  11 +-
 .../coordinator/CompactionCoordinator.java         |   5 +-
 .../org/apache/accumulo/compactor/Compactor.java   |   9 +-
 .../apache/accumulo/gc/SimpleGarbageCollector.java |   6 +-
 .../java/org/apache/accumulo/manager/Manager.java  |  18 +-
 .../java/org/apache/accumulo/monitor/Monitor.java  |  14 +-
 .../org/apache/accumulo/tserver/ScanServer.java    |   9 +-
 .../org/apache/accumulo/tserver/TabletServer.java  |  22 +-
 .../accumulo/test/BadDeleteMarkersCreatedIT.java   |  12 +-
 .../test/fate/zookeeper/ServiceLockIT.java         |  53 +++--
 .../test/functional/GarbageCollectorIT.java        |  24 +--
 .../accumulo/test/functional/ReadWriteIT.java      |   8 +-
 .../apache/accumulo/test/functional/RestartIT.java |  20 +-
 .../accumulo/test/functional/SplitRecoveryIT.java  |   6 +-
 .../accumulo/test/functional/ZombieTServer.java    |  10 +-
 28 files changed, 587 insertions(+), 244 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
index 25eba7ecc4..dbd6a90848 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
@@ -35,6 +35,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.Optional;
 import java.util.Properties;
 import java.util.UUID;
 import java.util.concurrent.Callable;
@@ -98,6 +99,8 @@ import org.apache.accumulo.core.spi.scan.ScanServerInfo;
 import org.apache.accumulo.core.spi.scan.ScanServerSelector;
 import org.apache.accumulo.core.util.OpTimer;
 import org.apache.accumulo.core.util.Pair;
+import org.apache.accumulo.core.util.ServiceLockData;
+import org.apache.accumulo.core.util.ServiceLockData.ThriftService;
 import org.apache.accumulo.core.util.tables.TableZooHelper;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.core.util.threads.Threads;
@@ -402,11 +405,10 @@ public class ClientContext implements AccumuloClient {
       try {
         final var zLockPath = ServiceLock.path(root + "/" + addr);
         ZcStat stat = new ZcStat();
-        byte[] lockData = ServiceLock.getLockData(getZooCache(), zLockPath, stat);
-        if (lockData != null) {
-          String[] fields = new String(lockData, UTF_8).split(",", 2);
-          UUID uuid = UUID.fromString(fields[0]);
-          String group = fields[1];
+        Optional<ServiceLockData> sld = ServiceLock.getLockData(getZooCache(), zLockPath, stat);
+        if (sld.isPresent()) {
+          UUID uuid = sld.get().getServerUUID(ThriftService.TABLET_SCAN);
+          String group = sld.get().getGroup(ThriftService.TABLET_SCAN);
           liveScanServers.put(addr, new Pair<>(uuid, group));
         }
       } catch (IllegalArgumentException e) {
@@ -512,24 +514,28 @@ public class ClientContext implements AccumuloClient {
     OpTimer timer = null;
 
     if (log.isTraceEnabled()) {
-      log.trace("tid={} Looking up manager location in zookeeper.", Thread.currentThread().getId());
+      log.trace("tid={} Looking up manager location in zookeeper at {}.",
+          Thread.currentThread().getId(), zLockManagerPath);
       timer = new OpTimer().start();
     }
 
-    byte[] loc = zooCache.getLockData(zLockManagerPath);
+    Optional<ServiceLockData> sld = zooCache.getLockData(zLockManagerPath);
+    String location = null;
+    if (sld.isPresent()) {
+      location = sld.get().getAddressString(ThriftService.MANAGER);
+    }
 
     if (timer != null) {
       timer.stop();
       log.trace("tid={} Found manager at {} in {}", Thread.currentThread().getId(),
-          (loc == null ? "null" : new String(loc, UTF_8)),
-          String.format("%.3f secs", timer.scale(SECONDS)));
+          (location == null ? "null" : location), String.format("%.3f secs", timer.scale(SECONDS)));
     }
 
-    if (loc == null) {
+    if (location == null) {
       return Collections.emptyList();
     }
 
-    return Collections.singletonList(new String(loc, UTF_8));
+    return Collections.singletonList(location);
   }
 
   /**
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ServiceLock.java b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ServiceLock.java
index 3e8c444691..597102712c 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ServiceLock.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ServiceLock.java
@@ -23,11 +23,13 @@ import static java.util.Objects.requireNonNull;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Optional;
 import java.util.UUID;
 
 import org.apache.accumulo.core.fate.zookeeper.ZooCache.ZcStat;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil.LockID;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy;
+import org.apache.accumulo.core.util.ServiceLockData;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.Code;
@@ -151,12 +153,12 @@ public class ServiceLock implements Watcher {
 
   }
 
-  public synchronized boolean tryLock(LockWatcher lw, byte[] data)
+  public synchronized boolean tryLock(LockWatcher lw, ServiceLockData lockData)
       throws KeeperException, InterruptedException {
 
     LockWatcherWrapper lww = new LockWatcherWrapper(lw);
 
-    lock(lww, data);
+    lock(lww, lockData);
 
     if (lww.acquiredLock) {
       return true;
@@ -383,7 +385,7 @@ public class ServiceLock implements Watcher {
     localLw.lostLock(reason);
   }
 
-  public synchronized void lock(final AccumuloLockWatcher lw, byte[] data) {
+  public synchronized void lock(final AccumuloLockWatcher lw, ServiceLockData lockData) {
 
     if (lockWatcher != null || lockNodeName != null || createdNodeName != null) {
       throw new IllegalStateException();
@@ -397,9 +399,9 @@ public class ServiceLock implements Watcher {
       // except that instead of the ephemeral lock node being of the form guid-lock- use lock-guid-.
       // Another deviation from the recipe is that we cleanup any extraneous ephemeral nodes that
       // were created.
-      final String createPath =
-          zooKeeper.create(lockPathPrefix, data, ZooUtil.PUBLIC, CreateMode.EPHEMERAL_SEQUENTIAL);
-      LOG.debug("[{}] Ephemeral node {} created", vmLockPrefix, createPath);
+      final String createPath = zooKeeper.create(lockPathPrefix, lockData.serialize(),
+          ZooUtil.PUBLIC, CreateMode.EPHEMERAL_SEQUENTIAL);
+      LOG.debug("[{}] Ephemeral node {} created with data: {}", vmLockPrefix, createPath, lockData);
 
       // It's possible that the call above was retried several times and multiple ephemeral nodes
       // were created but the client missed the response for some reason. Find the ephemeral nodes
@@ -592,9 +594,12 @@ public class ServiceLock implements Watcher {
     return lockNodeName != null;
   }
 
-  public synchronized void replaceLockData(byte[] b) throws KeeperException, InterruptedException {
+  public synchronized void replaceLockData(ServiceLockData lockData)
+      throws KeeperException, InterruptedException {
     if (getLockPath() != null) {
-      zooKeeper.setData(getLockPath(), b, -1);
+      zooKeeper.setData(getLockPath(), lockData.serialize(), -1);
+      LOG.debug("[{}] Lock data replaced at path {} with data: {}", vmLockPrefix, getLockPath(),
+          lockData);
     }
   }
 
@@ -646,27 +651,31 @@ public class ServiceLock implements Watcher {
     return zc.get(lid.path + "/" + lid.node, stat) != null && stat.getEphemeralOwner() == lid.eid;
   }
 
-  public static byte[] getLockData(ZooKeeper zk, ServiceLockPath path)
+  public static Optional<ServiceLockData> getLockData(ZooKeeper zk, ServiceLockPath path)
       throws KeeperException, InterruptedException {
 
     List<String> children = validateAndSort(path, zk.getChildren(path.toString(), null));
 
     if (children == null || children.isEmpty()) {
-      return null;
+      return Optional.empty();
     }
 
     String lockNode = children.get(0);
 
-    return zk.getData(path + "/" + lockNode, false, null);
+    byte[] data = zk.getData(path + "/" + lockNode, false, null);
+    if (data == null) {
+      data = new byte[0];
+    }
+    return ServiceLockData.parse(data);
   }
 
-  public static byte[] getLockData(org.apache.accumulo.core.fate.zookeeper.ZooCache zc,
-      ServiceLockPath path, ZcStat stat) {
+  public static Optional<ServiceLockData> getLockData(
+      org.apache.accumulo.core.fate.zookeeper.ZooCache zc, ServiceLockPath path, ZcStat stat) {
 
     List<String> children = validateAndSort(path, zc.getChildren(path.toString()));
 
     if (children == null || children.isEmpty()) {
-      return null;
+      return Optional.empty();
     }
 
     String lockNode = children.get(0);
@@ -675,7 +684,11 @@ public class ServiceLock implements Watcher {
       throw new RuntimeException("Node " + lockNode + " at " + path + " is not a lock node");
     }
 
-    return zc.get(path + "/" + lockNode, stat);
+    byte[] data = zc.get(path + "/" + lockNode, stat);
+    if (data == null) {
+      data = new byte[0];
+    }
+    return ServiceLockData.parse(data);
   }
 
   public static long getSessionId(ZooCache zc, ServiceLockPath path) {
diff --git a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCache.java b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCache.java
index fe8807c570..6c013773ee 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCache.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCache.java
@@ -26,12 +26,14 @@ import java.util.ConcurrentModificationException;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.LockSupport;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.accumulo.core.fate.zookeeper.ServiceLock.ServiceLockPath;
+import org.apache.accumulo.core.util.ServiceLockData;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.Code;
 import org.apache.zookeeper.WatchedEvent;
@@ -525,13 +527,21 @@ public class ZooCache {
     }
   }
 
-  public byte[] getLockData(ServiceLockPath path) {
+  public Optional<ServiceLockData> getLockData(ServiceLockPath path) {
     List<String> children = ServiceLock.validateAndSort(path, getChildren(path.toString()));
     if (children == null || children.isEmpty()) {
-      return null;
+      return Optional.empty();
     }
     String lockNode = children.get(0);
-    return get(path + "/" + lockNode);
+
+    byte[] lockData = get(path + "/" + lockNode);
+    if (log.isTraceEnabled()) {
+      log.trace("Data from lockNode {} is {}", lockNode, new String(lockData, UTF_8));
+    }
+    if (lockData == null) {
+      lockData = new byte[0];
+    }
+    return ServiceLockData.parse(lockData);
   }
 
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
index fd89e874aa..42221e8f98 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
@@ -18,7 +18,6 @@
  */
 package org.apache.accumulo.core.metadata.schema;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.COMPACT_QUAL;
 import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.DIRECTORY_QUAL;
 import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.FLUSH_QUAL;
@@ -69,7 +68,7 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.Se
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.SuspendLocationColumn;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
-import org.apache.accumulo.core.util.ServerServices;
+import org.apache.accumulo.core.util.ServiceLockData;
 import org.apache.hadoop.io.Text;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -472,13 +471,14 @@ public class TabletMetadata {
     Optional<TServerInstance> server = Optional.empty();
     final var lockPath = ServiceLock.path(path + "/" + zPath);
     ZooCache.ZcStat stat = new ZooCache.ZcStat();
-    byte[] lockData = ServiceLock.getLockData(context.getZooCache(), lockPath, stat);
+    Optional<ServiceLockData> sld = ServiceLock.getLockData(context.getZooCache(), lockPath, stat);
 
-    log.trace("Checking server at ZK path = " + lockPath);
-    if (lockData != null) {
-      ServerServices services = new ServerServices(new String(lockData, UTF_8));
-      HostAndPort client = services.getAddress(ServerServices.Service.TSERV_CLIENT);
-      server = Optional.of(new TServerInstance(client, stat.getEphemeralOwner()));
+    if (sld.isPresent()) {
+      log.trace("Checking server at ZK path = " + lockPath);
+      HostAndPort client = sld.get().getAddress(ServiceLockData.ThriftService.TSERV);
+      if (client != null) {
+        server = Optional.of(new TServerInstance(client, stat.getEphemeralOwner()));
+      }
     }
     return server;
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/clients/TServerClient.java b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TServerClient.java
index 4be93e8444..0e98b45cc9 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/clients/TServerClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/clients/TServerClient.java
@@ -20,10 +20,10 @@ package org.apache.accumulo.core.rpc.clients;
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
-import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 
 import java.util.ArrayList;
+import java.util.Optional;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.accumulo.core.Constants;
@@ -39,8 +39,8 @@ import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes.Exec;
 import org.apache.accumulo.core.rpc.clients.ThriftClientTypes.ExecVoid;
 import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.core.util.ServerServices;
-import org.apache.accumulo.core.util.ServerServices.Service;
+import org.apache.accumulo.core.util.ServiceLockData;
+import org.apache.accumulo.core.util.ServiceLockData.ThriftService;
 import org.apache.thrift.TApplicationException;
 import org.apache.thrift.TException;
 import org.apache.thrift.TServiceClient;
@@ -48,6 +48,8 @@ import org.apache.thrift.transport.TTransport;
 import org.apache.thrift.transport.TTransportException;
 import org.slf4j.Logger;
 
+import com.google.common.net.HostAndPort;
+
 public interface TServerClient<C extends TServiceClient> {
 
   Pair<String,C> getTabletServerConnection(ClientContext context, boolean preferCachedConnections)
@@ -66,12 +68,11 @@ public interface TServerClient<C extends TServiceClient> {
     for (String tserver : zc.getChildren(context.getZooKeeperRoot() + Constants.ZTSERVERS)) {
       var zLocPath =
           ServiceLock.path(context.getZooKeeperRoot() + Constants.ZTSERVERS + "/" + tserver);
-      byte[] data = zc.getLockData(zLocPath);
-      if (data != null) {
-        String strData = new String(data, UTF_8);
-        if (!strData.equals("manager")) {
-          servers.add(new ThriftTransportKey(
-              new ServerServices(strData).getAddress(Service.TSERV_CLIENT), rpcTimeout, context));
+      Optional<ServiceLockData> sld = zc.getLockData(zLocPath);
+      if (sld.isPresent()) {
+        HostAndPort address = sld.get().getAddress(ThriftService.TSERV);
+        if (address != null) {
+          servers.add(new ThriftTransportKey(address, rpcTimeout, context));
         }
       }
     }
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerSelector.java b/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerSelector.java
index 541ff7a858..c07667cfc8 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerSelector.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerSelector.java
@@ -25,6 +25,7 @@ import java.util.function.Supplier;
 import org.apache.accumulo.core.client.ScannerBase;
 import org.apache.accumulo.core.data.TabletId;
 import org.apache.accumulo.core.spi.common.ServiceEnvironment;
+import org.apache.accumulo.core.util.ServiceLockData;
 
 import com.google.common.base.Preconditions;
 
@@ -43,7 +44,7 @@ public interface ScanServerSelector {
   /**
    * The scan server group name that will be used when one is not specified.
    */
-  String DEFAULT_SCAN_SERVER_GROUP_NAME = "default";
+  String DEFAULT_SCAN_SERVER_GROUP_NAME = ServiceLockData.ServiceDescriptor.DEFAULT_GROUP_NAME;
 
   /**
    * This method is called once after a {@link ScanServerSelector} is instantiated.
diff --git a/core/src/main/java/org/apache/accumulo/core/util/ServerServices.java b/core/src/main/java/org/apache/accumulo/core/util/ServerServices.java
deleted file mode 100644
index 039608c1c5..0000000000
--- a/core/src/main/java/org/apache/accumulo/core/util/ServerServices.java
+++ /dev/null
@@ -1,93 +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.core.util;
-
-import java.util.EnumMap;
-
-import com.google.common.net.HostAndPort;
-
-public class ServerServices implements Comparable<ServerServices> {
-  public static enum Service {
-    TSERV_CLIENT, GC_CLIENT, COMPACTOR_CLIENT, SSERV_CLIENT;
-  }
-
-  public static final String SERVICE_SEPARATOR = ";";
-  public static final String SEPARATOR_CHAR = "=";
-
-  private EnumMap<Service,String> services;
-  private String stringForm = null;
-
-  public ServerServices(String services) {
-    this.services = new EnumMap<>(Service.class);
-
-    String[] addresses = services.split(SERVICE_SEPARATOR);
-    for (String address : addresses) {
-      String[] sa = address.split(SEPARATOR_CHAR, 2);
-      this.services.put(Service.valueOf(sa[0]), sa[1]);
-    }
-  }
-
-  public ServerServices(String address, Service service) {
-    this(service.name() + SEPARATOR_CHAR + address);
-  }
-
-  public String getAddressString(Service service) {
-    return services.get(service);
-  }
-
-  public HostAndPort getAddress(Service service) {
-    return AddressUtil.parseAddress(getAddressString(service), false);
-  }
-
-  // DON'T CHANGE THIS; WE'RE USING IT FOR SERIALIZATION!!!
-  @Override
-  public String toString() {
-    if (stringForm == null) {
-      StringBuilder sb = new StringBuilder();
-      String prefix = "";
-      for (Service service : new Service[] {Service.TSERV_CLIENT, Service.GC_CLIENT}) {
-        if (services.containsKey(service)) {
-          sb.append(prefix).append(service.name()).append(SEPARATOR_CHAR)
-              .append(services.get(service));
-          prefix = SERVICE_SEPARATOR;
-        }
-      }
-      stringForm = sb.toString();
-    }
-    return stringForm;
-  }
-
-  @Override
-  public int hashCode() {
-    return toString().hashCode();
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (o instanceof ServerServices) {
-      return toString().equals(o.toString());
-    }
-    return false;
-  }
-
-  @Override
-  public int compareTo(ServerServices other) {
-    return toString().compareTo(other.toString());
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/ServiceLockData.java b/core/src/main/java/org/apache/accumulo/core/util/ServiceLockData.java
new file mode 100644
index 0000000000..c73dadd8b6
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/util/ServiceLockData.java
@@ -0,0 +1,233 @@
+/*
+ * 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.util;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static java.util.Objects.requireNonNull;
+
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+
+import com.google.common.net.HostAndPort;
+import com.google.gson.Gson;
+
+public class ServiceLockData implements Comparable<ServiceLockData> {
+
+  private static final Gson gson = new Gson();
+
+  /**
+   * Thrift Service list
+   */
+  public static enum ThriftService {
+    CLIENT,
+    COORDINATOR,
+    COMPACTOR,
+    FATE,
+    GC,
+    MANAGER,
+    NONE,
+    TABLET_INGEST,
+    TABLET_MANAGEMENT,
+    TABLET_SCAN,
+    TSERV
+  }
+
+  /**
+   * An object that describes a process, the group assigned to that process, the Thrift service and
+   * the address to use to communicate with that service.
+   */
+  public static class ServiceDescriptor {
+
+    /**
+     * The group name that will be used when one is not specified.
+     */
+    public static final String DEFAULT_GROUP_NAME = "default";
+
+    private final UUID uuid;
+    private final ThriftService service;
+    private final String address;
+    private final String group;
+
+    public ServiceDescriptor(UUID uuid, ThriftService service, String address) {
+      this(uuid, service, address, DEFAULT_GROUP_NAME);
+    }
+
+    public ServiceDescriptor(UUID uuid, ThriftService service, String address, String group) {
+      this.uuid = requireNonNull(uuid);
+      this.service = requireNonNull(service);
+      this.address = requireNonNull(address);
+      this.group = requireNonNull(group);
+    }
+
+    public UUID getUUID() {
+      return uuid;
+    }
+
+    public ThriftService getService() {
+      return service;
+    }
+
+    public String getAddress() {
+      return address;
+    }
+
+    public String getGroup() {
+      return group;
+    }
+
+    @Override
+    public int hashCode() {
+      return toString().hashCode();
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj) {
+        return true;
+      }
+      if (obj == null) {
+        return false;
+      }
+      if (getClass() != obj.getClass()) {
+        return false;
+      }
+      ServiceDescriptor other = (ServiceDescriptor) obj;
+      return toString().equals(other.toString());
+    }
+
+    @Override
+    public String toString() {
+      return gson.toJson(this);
+    }
+
+  }
+
+  /**
+   * A set of ServiceDescriptor's
+   */
+  public static class ServiceDescriptors {
+    private final Set<ServiceDescriptor> descriptors;
+
+    public ServiceDescriptors() {
+      descriptors = new HashSet<>();
+    }
+
+    public ServiceDescriptors(HashSet<ServiceDescriptor> descriptors) {
+      this.descriptors = descriptors;
+    }
+
+    public void addService(ServiceDescriptor sd) {
+      this.descriptors.add(sd);
+    }
+
+    public Set<ServiceDescriptor> getServices() {
+      return descriptors;
+    }
+  }
+
+  private EnumMap<ThriftService,ServiceDescriptor> services;
+
+  public ServiceLockData(ServiceDescriptors sds) {
+    this.services = new EnumMap<>(ThriftService.class);
+    sds.getServices().forEach(sd -> this.services.put(sd.getService(), sd));
+  }
+
+  public ServiceLockData(UUID uuid, String address, ThriftService service, String group) {
+    this(new ServiceDescriptors(new HashSet<>(
+        Collections.singleton(new ServiceDescriptor(uuid, service, address, group)))));
+  }
+
+  public ServiceLockData(UUID uuid, String address, ThriftService service) {
+    this(new ServiceDescriptors(
+        new HashSet<>(Collections.singleton(new ServiceDescriptor(uuid, service, address)))));
+  }
+
+  public String getAddressString(ThriftService service) {
+    ServiceDescriptor sd = services.get(service);
+    if (sd == null) {
+      return null;
+    }
+    return sd.getAddress();
+  }
+
+  public HostAndPort getAddress(ThriftService service) {
+    return AddressUtil.parseAddress(getAddressString(service), false);
+  }
+
+  public String getGroup(ThriftService service) {
+    ServiceDescriptor sd = services.get(service);
+    if (sd == null) {
+      return null;
+    }
+    return sd.getGroup();
+  }
+
+  public UUID getServerUUID(ThriftService service) {
+    ServiceDescriptor sd = services.get(service);
+    if (sd == null) {
+      return null;
+    }
+    return sd.getUUID();
+  }
+
+  public byte[] serialize() {
+    ServiceDescriptors sd = new ServiceDescriptors();
+    services.values().forEach(s -> sd.addService(s));
+    return gson.toJson(sd).getBytes(UTF_8);
+  }
+
+  @Override
+  public String toString() {
+    return new String(serialize(), UTF_8);
+  }
+
+  @Override
+  public int hashCode() {
+    return toString().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o instanceof ServiceLockData) {
+      return toString().equals(o.toString());
+    }
+    return false;
+  }
+
+  @Override
+  public int compareTo(ServiceLockData other) {
+    return toString().compareTo(other.toString());
+  }
+
+  public static Optional<ServiceLockData> parse(byte[] lockData) {
+    if (lockData == null) {
+      return Optional.empty();
+    }
+    String data = new String(lockData, UTF_8);
+    if (data.isBlank()) {
+      return Optional.empty();
+    }
+    return Optional.of(new ServiceLockData(gson.fromJson(data, ServiceDescriptors.class)));
+  }
+
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java b/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java
index fc3d2e0040..e0201e8dd6 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java
@@ -42,6 +42,8 @@ import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
 import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction;
 import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
 import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.util.ServiceLockData;
+import org.apache.accumulo.core.util.ServiceLockData.ThriftService;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.thrift.TException;
 import org.apache.zookeeper.KeeperException;
@@ -102,11 +104,11 @@ public class ExternalCompactionUtil {
     try {
       var zk = ZooSession.getAnonymousSession(context.getZooKeepers(),
           context.getZooKeepersSessionTimeOut());
-      byte[] address = ServiceLock.getLockData(zk, ServiceLock.path(lockPath));
-      if (null == address) {
+      Optional<ServiceLockData> sld = ServiceLock.getLockData(zk, ServiceLock.path(lockPath));
+      if (sld.isEmpty()) {
         return Optional.empty();
       }
-      return Optional.of(HostAndPort.fromString(new String(address)));
+      return Optional.ofNullable(sld.get().getAddress(ThriftService.COORDINATOR));
     } catch (KeeperException | InterruptedException e) {
       throw new RuntimeException(e);
     }
diff --git a/core/src/test/java/org/apache/accumulo/core/metadata/schema/TabletMetadataTest.java b/core/src/test/java/org/apache/accumulo/core/metadata/schema/TabletMetadataTest.java
index 7cad30694d..fadc54fa70 100644
--- a/core/src/test/java/org/apache/accumulo/core/metadata/schema/TabletMetadataTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/metadata/schema/TabletMetadataTest.java
@@ -19,7 +19,6 @@
 package org.apache.accumulo.core.metadata.schema;
 
 import static java.util.stream.Collectors.toSet;
-import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.SuspendLocationColumn;
 import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.COMPACT_COLUMN;
 import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN;
 import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.FLUSH_COLUMN;
@@ -57,6 +56,7 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.Da
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.FutureLocationColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LastLocationColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ScanFileColumnFamily;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.SuspendLocationColumn;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
diff --git a/core/src/test/java/org/apache/accumulo/core/util/ServiceLockDataTest.java b/core/src/test/java/org/apache/accumulo/core/util/ServiceLockDataTest.java
new file mode 100644
index 0000000000..ae2a75b091
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/util/ServiceLockDataTest.java
@@ -0,0 +1,123 @@
+/*
+ * 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.util;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.Optional;
+import java.util.UUID;
+
+import org.apache.accumulo.core.util.ServiceLockData.ServiceDescriptor;
+import org.apache.accumulo.core.util.ServiceLockData.ServiceDescriptors;
+import org.apache.accumulo.core.util.ServiceLockData.ThriftService;
+import org.junit.jupiter.api.Test;
+
+import com.google.common.net.HostAndPort;
+
+public class ServiceLockDataTest {
+
+  private final UUID serverUUID = UUID.randomUUID();
+
+  @Test
+  public void testSingleServiceConstructor() throws Exception {
+    ServiceLockData ss = new ServiceLockData(serverUUID, "127.0.0.1", ThriftService.TSERV);
+    assertEquals(serverUUID, ss.getServerUUID(ThriftService.TSERV));
+    assertEquals("127.0.0.1", ss.getAddressString(ThriftService.TSERV));
+    assertThrows(IllegalArgumentException.class, () -> ss.getAddress(ThriftService.TSERV));
+    assertEquals(ServiceDescriptor.DEFAULT_GROUP_NAME, ss.getGroup(ThriftService.TSERV));
+    assertNull(ss.getServerUUID(ThriftService.TABLET_SCAN));
+    assertNull(ss.getAddressString(ThriftService.TABLET_SCAN));
+    assertThrows(NullPointerException.class, () -> ss.getAddress(ThriftService.TABLET_SCAN));
+    assertNull(ss.getGroup(ThriftService.TABLET_SCAN));
+  }
+
+  @Test
+  public void testMultipleServiceConstructor() throws Exception {
+    ServiceDescriptors sds = new ServiceDescriptors();
+    sds.addService(new ServiceDescriptor(serverUUID, ThriftService.TSERV, "127.0.0.1:9997"));
+    sds.addService(new ServiceDescriptor(serverUUID, ThriftService.TABLET_SCAN, "127.0.0.1:9998"));
+    ServiceLockData ss = new ServiceLockData(sds);
+    assertEquals(serverUUID, ss.getServerUUID(ThriftService.TSERV));
+    assertEquals("127.0.0.1:9997", ss.getAddressString(ThriftService.TSERV));
+    assertEquals(HostAndPort.fromString("127.0.0.1:9997"), ss.getAddress(ThriftService.TSERV));
+    assertEquals(ServiceDescriptor.DEFAULT_GROUP_NAME, ss.getGroup(ThriftService.TSERV));
+    assertEquals(serverUUID, ss.getServerUUID(ThriftService.TABLET_SCAN));
+    assertEquals("127.0.0.1:9998", ss.getAddressString(ThriftService.TABLET_SCAN));
+    assertEquals(HostAndPort.fromString("127.0.0.1:9998"),
+        ss.getAddress(ThriftService.TABLET_SCAN));
+    assertEquals(ServiceDescriptor.DEFAULT_GROUP_NAME, ss.getGroup(ThriftService.TSERV));
+  }
+
+  @Test
+  public void testSingleServiceConstructorWithGroup() throws Exception {
+    ServiceLockData ss = new ServiceLockData(serverUUID, "127.0.0.1", ThriftService.TSERV, "meta");
+    assertEquals(serverUUID, ss.getServerUUID(ThriftService.TSERV));
+    assertEquals("127.0.0.1", ss.getAddressString(ThriftService.TSERV));
+    assertThrows(IllegalArgumentException.class, () -> ss.getAddress(ThriftService.TSERV));
+    assertEquals("meta", ss.getGroup(ThriftService.TSERV));
+    assertNull(ss.getServerUUID(ThriftService.TABLET_SCAN));
+    assertNull(ss.getAddressString(ThriftService.TABLET_SCAN));
+    assertThrows(NullPointerException.class, () -> ss.getAddress(ThriftService.TABLET_SCAN));
+    assertNull(ss.getGroup(ThriftService.TABLET_SCAN));
+  }
+
+  @Test
+  public void testSingleServiceConstructor2WithGroup() throws Exception {
+    ServiceLockData ss = new ServiceLockData(serverUUID, "127.0.0.1", ThriftService.TSERV, "meta");
+    assertEquals(serverUUID, ss.getServerUUID(ThriftService.TSERV));
+    assertEquals("127.0.0.1", ss.getAddressString(ThriftService.TSERV));
+    assertThrows(IllegalArgumentException.class, () -> ss.getAddress(ThriftService.TSERV));
+    assertEquals("meta", ss.getGroup(ThriftService.TSERV));
+    assertEquals(serverUUID, ss.getServerUUID(ThriftService.TSERV));
+    assertNull(ss.getAddressString(ThriftService.TABLET_SCAN));
+    assertThrows(NullPointerException.class, () -> ss.getAddress(ThriftService.TABLET_SCAN));
+    assertNull(ss.getGroup(ThriftService.TABLET_SCAN));
+  }
+
+  @Test
+  public void testMultipleServiceConstructorWithGroup() throws Exception {
+    ServiceDescriptors sds = new ServiceDescriptors();
+    sds.addService(
+        new ServiceDescriptor(serverUUID, ThriftService.TSERV, "127.0.0.1:9997", "meta"));
+    sds.addService(
+        new ServiceDescriptor(serverUUID, ThriftService.TABLET_SCAN, "127.0.0.1:9998", "ns1"));
+    ServiceLockData ss = new ServiceLockData(sds);
+    assertEquals(serverUUID, ss.getServerUUID(ThriftService.TSERV));
+    assertEquals("127.0.0.1:9997", ss.getAddressString(ThriftService.TSERV));
+    assertEquals(HostAndPort.fromString("127.0.0.1:9997"), ss.getAddress(ThriftService.TSERV));
+    assertEquals("meta", ss.getGroup(ThriftService.TSERV));
+    assertEquals(serverUUID, ss.getServerUUID(ThriftService.TABLET_SCAN));
+    assertEquals("127.0.0.1:9998", ss.getAddressString(ThriftService.TABLET_SCAN));
+    assertEquals(HostAndPort.fromString("127.0.0.1:9998"),
+        ss.getAddress(ThriftService.TABLET_SCAN));
+    assertEquals("ns1", ss.getGroup(ThriftService.TABLET_SCAN));
+  }
+
+  @Test
+  public void testParseEmpty() {
+    Optional<ServiceLockData> sld = ServiceLockData.parse(new byte[0]);
+    assertTrue(sld.isEmpty());
+    assertFalse(sld.isPresent());
+  }
+
+}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/LiveTServerSet.java b/server/base/src/main/java/org/apache/accumulo/server/manager/LiveTServerSet.java
index 890e2a9cd3..fbc4ace944 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/manager/LiveTServerSet.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/manager/LiveTServerSet.java
@@ -18,7 +18,6 @@
  */
 package org.apache.accumulo.server.manager;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.concurrent.TimeUnit.MINUTES;
 import static org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy.SKIP;
 
@@ -27,6 +26,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
@@ -49,7 +49,7 @@ import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.AddressUtil;
 import org.apache.accumulo.core.util.Halt;
-import org.apache.accumulo.core.util.ServerServices;
+import org.apache.accumulo.core.util.ServiceLockData;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.hadoop.io.Text;
@@ -307,9 +307,9 @@ public class LiveTServerSet implements Watcher {
 
     final var zLockPath = ServiceLock.path(path + "/" + zPath);
     ZcStat stat = new ZcStat();
-    byte[] lockData = ServiceLock.getLockData(getZooCache(), zLockPath, stat);
+    Optional<ServiceLockData> sld = ServiceLock.getLockData(getZooCache(), zLockPath, stat);
 
-    if (lockData == null) {
+    if (sld.isEmpty()) {
       if (info != null) {
         doomed.add(info.instance);
         current.remove(zPath);
@@ -325,8 +325,7 @@ public class LiveTServerSet implements Watcher {
       }
     } else {
       locklessServers.remove(zPath);
-      ServerServices services = new ServerServices(new String(lockData, UTF_8));
-      HostAndPort client = services.getAddress(ServerServices.Service.TSERV_CLIENT);
+      HostAndPort client = sld.get().getAddress(ServiceLockData.ThriftService.TSERV);
       TServerInstance instance = new TServerInstance(client, stat.getEphemeralOwner());
 
       if (info == null) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ListInstances.java b/server/base/src/main/java/org/apache/accumulo/server/util/ListInstances.java
index 131ded3d57..4c87adee6a 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ListInstances.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ListInstances.java
@@ -24,6 +24,7 @@ import java.util.Formattable;
 import java.util.Formatter;
 import java.util.List;
 import java.util.Map.Entry;
+import java.util.Optional;
 import java.util.TreeMap;
 import java.util.TreeSet;
 
@@ -35,6 +36,8 @@ import org.apache.accumulo.core.data.InstanceId;
 import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache;
 import org.apache.accumulo.core.fate.zookeeper.ZooReader;
+import org.apache.accumulo.core.util.ServiceLockData;
+import org.apache.accumulo.core.util.ServiceLockData.ThriftService;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -164,11 +167,11 @@ public class ListInstances {
     try {
       var zLockManagerPath =
           ServiceLock.path(Constants.ZROOT + "/" + iid + Constants.ZMANAGER_LOCK);
-      byte[] manager = ServiceLock.getLockData(cache, zLockManagerPath, null);
-      if (manager == null) {
+      Optional<ServiceLockData> sld = ServiceLock.getLockData(cache, zLockManagerPath, null);
+      if (sld.isEmpty()) {
         return null;
       }
-      return new String(manager, UTF_8);
+      return sld.get().getAddressString(ThriftService.MANAGER);
     } catch (Exception e) {
       handleException(e, printErrors);
       return null;
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java b/server/base/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java
index e55237d327..1d4e2146c1 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java
@@ -18,14 +18,15 @@
  */
 package org.apache.accumulo.server.util;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
-
 import java.util.List;
+import java.util.Optional;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.core.util.ServiceLockData;
+import org.apache.accumulo.core.util.ServiceLockData.ThriftService;
 import org.apache.accumulo.server.ServerContext;
 
 public class TabletServerLocks {
@@ -45,10 +46,10 @@ public class TabletServerLocks {
 
       for (String tabletServer : tabletServers) {
         var zLockPath = ServiceLock.path(tserverPath + "/" + tabletServer);
-        byte[] lockData = ServiceLock.getLockData(cache, zLockPath, null);
+        Optional<ServiceLockData> lockData = ServiceLock.getLockData(cache, zLockPath, null);
         final String holder;
-        if (lockData != null) {
-          holder = new String(lockData, UTF_8);
+        if (lockData.isPresent()) {
+          holder = lockData.get().getAddressString(ThriftService.TSERV);
         } else {
           holder = "<none>";
         }
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 964aef284e..5c73ed4b46 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
@@ -70,6 +70,8 @@ import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
 import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
 import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService;
 import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.util.ServiceLockData;
+import org.apache.accumulo.core.util.ServiceLockData.ThriftService;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
 import org.apache.accumulo.core.util.compaction.RunningCompaction;
@@ -212,7 +214,8 @@ public class CompactionCoordinator extends AbstractServer
       CoordinatorLockWatcher coordinatorLockWatcher = new CoordinatorLockWatcher();
       coordinatorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(),
           ServiceLock.path(lockPath), zooLockUUID);
-      coordinatorLock.lock(coordinatorLockWatcher, coordinatorClientAddress.getBytes());
+      coordinatorLock.lock(coordinatorLockWatcher,
+          new ServiceLockData(zooLockUUID, coordinatorClientAddress, ThriftService.COORDINATOR));
 
       coordinatorLockWatcher.waitForChange();
       if (coordinatorLockWatcher.isAcquiredLock()) {
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 1151c5178e..a9f0bfbf40 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
@@ -86,8 +86,8 @@ import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
 import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.Halt;
-import org.apache.accumulo.core.util.ServerServices;
-import org.apache.accumulo.core.util.ServerServices.Service;
+import org.apache.accumulo.core.util.ServiceLockData;
+import org.apache.accumulo.core.util.ServiceLockData.ThriftService;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
 import org.apache.accumulo.core.util.threads.ThreadPools;
@@ -286,12 +286,11 @@ public class Compactor extends AbstractServer implements MetricsProducer, Compac
     };
 
     try {
-      byte[] lockContent =
-          new ServerServices(hostPort, Service.COMPACTOR_CLIENT).toString().getBytes(UTF_8);
       for (int i = 0; i < 25; i++) {
         zoo.putPersistentData(zPath, new byte[0], NodeExistsPolicy.SKIP);
 
-        if (compactorLock.tryLock(lw, lockContent)) {
+        if (compactorLock.tryLock(lw,
+            new ServiceLockData(compactorId, hostPort, ThriftService.COMPACTOR, this.queueName))) {
           LOG.debug("Obtained Compactor lock {}", compactorLock.getLockPath());
           return;
         }
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 c2bdf6c6ac..555c0fc775 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
@@ -46,8 +46,8 @@ import org.apache.accumulo.core.metrics.MetricsUtil;
 import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.Halt;
-import org.apache.accumulo.core.util.ServerServices;
-import org.apache.accumulo.core.util.ServerServices.Service;
+import org.apache.accumulo.core.util.ServiceLockData;
+import org.apache.accumulo.core.util.ServiceLockData.ThriftService;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.gc.metrics.GcCycleMetrics;
 import org.apache.accumulo.gc.metrics.GcMetrics;
@@ -368,7 +368,7 @@ public class SimpleGarbageCollector extends AbstractServer implements Iface {
       ServiceLock lock =
           new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(), path, zooLockUUID);
       if (lock.tryLock(lockWatcher,
-          new ServerServices(addr.toString(), Service.GC_CLIENT).toString().getBytes())) {
+          new ServiceLockData(zooLockUUID, addr.toString(), ThriftService.GC))) {
         log.debug("Got GC ZooKeeper lock");
         return;
       }
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 5469b0c3d4..22afda649c 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
@@ -104,6 +104,8 @@ import org.apache.accumulo.core.tablet.thrift.TUnloadTabletGoal;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.Halt;
 import org.apache.accumulo.core.util.Retry;
+import org.apache.accumulo.core.util.ServiceLockData;
+import org.apache.accumulo.core.util.ServiceLockData.ThriftService;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.core.util.threads.Threads;
 import org.apache.accumulo.manager.metrics.ManagerMetrics;
@@ -1072,8 +1074,9 @@ public class Manager extends AbstractServer
     log.info("Started Manager client service at {}", sa.address);
 
     // block until we can obtain the ZK lock for the manager
+    ServiceLockData sld = null;
     try {
-      getManagerLock(ServiceLock.path(zroot + Constants.ZMANAGER_LOCK));
+      sld = getManagerLock(ServiceLock.path(zroot + Constants.ZMANAGER_LOCK));
     } catch (KeeperException | InterruptedException e) {
       throw new IllegalStateException("Exception getting manager lock", e);
     }
@@ -1223,9 +1226,11 @@ public class Manager extends AbstractServer
     }
 
     String address = sa.address.toString();
-    log.info("Setting manager lock data to {}", address);
+    sld = new ServiceLockData(sld.getServerUUID(ThriftService.MANAGER), address,
+        ThriftService.MANAGER);
+    log.info("Setting manager lock data to {}", sld.toString());
     try {
-      managerLock.replaceLockData(address.getBytes());
+      managerLock.replaceLockData(sld);
     } catch (KeeperException | InterruptedException e) {
       throw new IllegalStateException("Exception updating manager lock", e);
     }
@@ -1425,7 +1430,7 @@ public class Manager extends AbstractServer
     }
   }
 
-  private void getManagerLock(final ServiceLockPath zManagerLoc)
+  private ServiceLockData getManagerLock(final ServiceLockPath zManagerLoc)
       throws KeeperException, InterruptedException {
     var zooKeeper = getContext().getZooReaderWriter().getZooKeeper();
     log.info("trying to get manager lock");
@@ -1434,11 +1439,13 @@ public class Manager extends AbstractServer
         getHostname() + ":" + getConfiguration().getPort(Property.MANAGER_CLIENTPORT)[0];
 
     UUID zooLockUUID = UUID.randomUUID();
+    ServiceLockData sld =
+        new ServiceLockData(zooLockUUID, managerClientAddress, ThriftService.MANAGER);
     while (true) {
 
       ManagerLockWatcher managerLockWatcher = new ManagerLockWatcher();
       managerLock = new ServiceLock(zooKeeper, zManagerLoc, zooLockUUID);
-      managerLock.lock(managerLockWatcher, managerClientAddress.getBytes());
+      managerLock.lock(managerLockWatcher, sld);
 
       managerLockWatcher.waitForChange();
 
@@ -1456,6 +1463,7 @@ public class Manager extends AbstractServer
     }
 
     setManagerState(ManagerState.HAVE_LOCK);
+    return sld;
   }
 
   @Override
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 e8fe48e4a5..70da3d5ebe 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
@@ -72,8 +72,8 @@ import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService.Cl
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.Halt;
 import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.core.util.ServerServices;
-import org.apache.accumulo.core.util.ServerServices.Service;
+import org.apache.accumulo.core.util.ServiceLockData;
+import org.apache.accumulo.core.util.ServiceLockData.ThriftService;
 import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
 import org.apache.accumulo.core.util.threads.Threads;
 import org.apache.accumulo.monitor.rest.compactions.external.ExternalCompactionInfo;
@@ -428,8 +428,11 @@ public class Monitor extends AbstractServer implements HighlyAvailableService {
       var path = ServiceLock.path(context.getZooKeeperRoot() + Constants.ZGC_LOCK);
       List<String> locks = ServiceLock.validateAndSort(path, zk.getChildren(path.toString()));
       if (locks != null && !locks.isEmpty()) {
-        address = new ServerServices(new String(zk.getData(path + "/" + locks.get(0)), UTF_8))
-            .getAddress(Service.GC_CLIENT);
+        Optional<ServiceLockData> sld =
+            ServiceLockData.parse(zk.getData(path + "/" + locks.get(0)));
+        if (sld.isPresent()) {
+          address = sld.get().getAddress(ThriftService.GC);
+        }
         GCMonitorService.Client client =
             ThriftUtil.getClient(ThriftClientTypes.GC, address, context);
         try {
@@ -822,7 +825,8 @@ public class Monitor extends AbstractServer implements HighlyAvailableService {
     while (true) {
       MoniterLockWatcher monitorLockWatcher = new MoniterLockWatcher();
       monitorLock = new ServiceLock(zoo.getZooKeeper(), monitorLockPath, zooLockUUID);
-      monitorLock.lock(monitorLockWatcher, new byte[0]);
+      monitorLock.lock(monitorLockWatcher,
+          new ServiceLockData(zooLockUUID, getHostname(), ThriftService.NONE));
 
       monitorLockWatcher.waitForChange();
 
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 39cf7a6010..5a2608b1b1 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
@@ -19,7 +19,6 @@
 package org.apache.accumulo.tserver;
 
 import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
-import static java.nio.charset.StandardCharsets.UTF_8;
 
 import java.io.IOException;
 import java.io.UncheckedIOException;
@@ -83,6 +82,8 @@ import org.apache.accumulo.core.tabletscan.thrift.TooManyFilesException;
 import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
 import org.apache.accumulo.core.util.Halt;
+import org.apache.accumulo.core.util.ServiceLockData;
+import org.apache.accumulo.core.util.ServiceLockData.ThriftService;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.server.AbstractServer;
@@ -322,13 +323,11 @@ public class ScanServer extends AbstractServer
         }
       };
 
-      // Don't use the normal ServerServices lock content, instead put the server UUID here.
-      byte[] lockContent = (serverLockUUID.toString() + "," + groupName).getBytes(UTF_8);
-
       for (int i = 0; i < 120 / 5; i++) {
         zoo.putPersistentData(zLockPath.toString(), new byte[0], NodeExistsPolicy.SKIP);
 
-        if (scanServerLock.tryLock(lw, lockContent)) {
+        if (scanServerLock.tryLock(lw, new ServiceLockData(serverLockUUID, getClientAddressString(),
+            ThriftService.TABLET_SCAN, this.groupName))) {
           LOG.debug("Obtained scan server lock {}", scanServerLock.getLockPath());
           return scanServerLock;
         }
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 89e8045964..1f614035f1 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
@@ -19,7 +19,6 @@
 package org.apache.accumulo.tserver;
 
 import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
-import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.ECOMP;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.FILES;
 import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOGS;
@@ -102,8 +101,10 @@ import org.apache.accumulo.core.util.MapCounter;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.Retry;
 import org.apache.accumulo.core.util.Retry.RetryFactory;
-import org.apache.accumulo.core.util.ServerServices;
-import org.apache.accumulo.core.util.ServerServices.Service;
+import org.apache.accumulo.core.util.ServiceLockData;
+import org.apache.accumulo.core.util.ServiceLockData.ServiceDescriptor;
+import org.apache.accumulo.core.util.ServiceLockData.ServiceDescriptors;
+import org.apache.accumulo.core.util.ServiceLockData.ThriftService;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.core.util.threads.Threads;
@@ -632,7 +633,8 @@ public class TabletServer extends AbstractServer implements TabletHostingServer
         throw e;
       }
 
-      tabletServerLock = new ServiceLock(zoo.getZooKeeper(), zLockPath, UUID.randomUUID());
+      UUID tabletServerUUID = UUID.randomUUID();
+      tabletServerLock = new ServiceLock(zoo.getZooKeeper(), zLockPath, tabletServerUUID);
 
       LockWatcher lw = new LockWatcher() {
 
@@ -653,12 +655,18 @@ public class TabletServer extends AbstractServer implements TabletHostingServer
         }
       };
 
-      byte[] lockContent = new ServerServices(getClientAddressString(), Service.TSERV_CLIENT)
-          .toString().getBytes(UTF_8);
       for (int i = 0; i < 120 / 5; i++) {
         zoo.putPersistentData(zLockPath.toString(), new byte[0], NodeExistsPolicy.SKIP);
 
-        if (tabletServerLock.tryLock(lw, lockContent)) {
+        ServiceDescriptors descriptors = new ServiceDescriptors();
+        for (ThriftService svc : new ThriftService[] {ThriftService.CLIENT,
+            ThriftService.TABLET_INGEST, ThriftService.TABLET_MANAGEMENT, ThriftService.TABLET_SCAN,
+            ThriftService.TSERV}) {
+          descriptors
+              .addService(new ServiceDescriptor(tabletServerUUID, svc, getClientAddressString()));
+        }
+
+        if (tabletServerLock.tryLock(lw, new ServiceLockData(descriptors))) {
           log.debug("Obtained tablet server lock {}", tabletServerLock.getLockPath());
           lockID = tabletServerLock.getLockID()
               .serialize(getContext().getZooKeeperRoot() + Constants.ZTSERVERS + "/");
diff --git a/test/src/main/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java b/test/src/main/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
index 1902fe1b4d..92f6df2f0c 100644
--- a/test/src/main/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
@@ -26,6 +26,7 @@ import static org.junit.jupiter.api.Assertions.fail;
 import java.time.Duration;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Optional;
 import java.util.SortedSet;
 import java.util.TreeSet;
 import java.util.concurrent.TimeUnit;
@@ -45,6 +46,7 @@ import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.DeletesSection;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.ServiceLockData;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
@@ -106,14 +108,14 @@ public class BadDeleteMarkersCreatedIT extends AccumuloClusterHarness {
       zcache.clear();
       var path = ServiceLock
           .path(ZooUtil.getRoot(client.instanceOperations().getInstanceId()) + Constants.ZGC_LOCK);
-      byte[] gcLockData;
+      Optional<ServiceLockData> gcLockData;
       do {
         gcLockData = ServiceLock.getLockData(zcache, path, null);
-        if (gcLockData != null) {
+        if (gcLockData.isPresent()) {
           log.info("Waiting for GC ZooKeeper lock to expire");
           Thread.sleep(2000);
         }
-      } while (gcLockData != null);
+      } while (gcLockData.isPresent());
 
       log.info("GC lock was lost");
 
@@ -122,11 +124,11 @@ public class BadDeleteMarkersCreatedIT extends AccumuloClusterHarness {
 
       do {
         gcLockData = ServiceLock.getLockData(zcache, path, null);
-        if (gcLockData == null) {
+        if (gcLockData.isEmpty()) {
           log.info("Waiting for GC ZooKeeper lock to be acquired");
           Thread.sleep(2000);
         }
-      } while (gcLockData == null);
+      } while (gcLockData.isEmpty());
 
       log.info("GC lock was acquired");
     }
diff --git a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ServiceLockIT.java b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ServiceLockIT.java
index fd71842f78..b139be28b0 100644
--- a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ServiceLockIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ServiceLockIT.java
@@ -31,6 +31,7 @@ import java.io.IOException;
 import java.lang.reflect.Field;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Optional;
 import java.util.UUID;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -44,6 +45,9 @@ import org.apache.accumulo.core.fate.zookeeper.ServiceLock.ServiceLockPath;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.fate.zookeeper.ZooSession;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.core.util.ServiceLockData;
+import org.apache.accumulo.core.util.ServiceLockData.ServiceDescriptor;
+import org.apache.accumulo.core.util.ServiceLockData.ThriftService;
 import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
@@ -228,7 +232,8 @@ public class ServiceLockIT {
 
     TestALW lw = new TestALW();
 
-    zl.lock(lw, "test1".getBytes(UTF_8));
+    zl.lock(lw, new ServiceLockData(UUID.randomUUID(), "test1", ThriftService.TSERV,
+        ServiceDescriptor.DEFAULT_GROUP_NAME));
 
     lw.waitForChanges(1);
 
@@ -252,7 +257,8 @@ public class ServiceLockIT {
 
     TestALW lw = new TestALW();
 
-    zl.lock(lw, "test1".getBytes(UTF_8));
+    zl.lock(lw, new ServiceLockData(UUID.randomUUID(), "test1", ThriftService.TSERV,
+        ServiceDescriptor.DEFAULT_GROUP_NAME));
 
     lw.waitForChanges(1);
 
@@ -277,7 +283,8 @@ public class ServiceLockIT {
 
     TestALW lw = new TestALW();
 
-    zl.lock(lw, "test1".getBytes(UTF_8));
+    zl.lock(lw, new ServiceLockData(UUID.randomUUID(), "test1", ThriftService.TSERV,
+        ServiceDescriptor.DEFAULT_GROUP_NAME));
 
     lw.waitForChanges(1);
 
@@ -310,7 +317,8 @@ public class ServiceLockIT {
 
     TestALW lw = new TestALW();
 
-    zl.lock(lw, "test1".getBytes(UTF_8));
+    zl.lock(lw, new ServiceLockData(UUID.randomUUID(), "test1", ThriftService.TSERV,
+        ServiceDescriptor.DEFAULT_GROUP_NAME));
 
     lw.waitForChanges(1);
 
@@ -323,7 +331,8 @@ public class ServiceLockIT {
 
     TestALW lw2 = new TestALW();
 
-    zl2.lock(lw2, "test2".getBytes(UTF_8));
+    zl2.lock(lw2, new ServiceLockData(UUID.randomUUID(), "test2", ThriftService.TSERV,
+        ServiceDescriptor.DEFAULT_GROUP_NAME));
 
     assertFalse(lw2.locked);
     assertFalse(zl2.isLocked());
@@ -332,7 +341,8 @@ public class ServiceLockIT {
 
     TestALW lw3 = new TestALW();
 
-    zl3.lock(lw3, "test3".getBytes(UTF_8));
+    zl3.lock(lw3, new ServiceLockData(UUID.randomUUID(), "test3", ThriftService.TSERV,
+        ServiceDescriptor.DEFAULT_GROUP_NAME));
 
     List<String> children = ServiceLock.validateAndSort(parent, zk.getChildren(parent.toString()));
 
@@ -387,7 +397,8 @@ public class ServiceLockIT {
 
       TestALW lw = new TestALW();
 
-      zl.lock(lw, "test1".getBytes(UTF_8));
+      zl.lock(lw, new ServiceLockData(UUID.randomUUID(), "test1", ThriftService.TSERV,
+          ServiceDescriptor.DEFAULT_GROUP_NAME));
 
       lw.waitForChanges(1);
 
@@ -437,7 +448,8 @@ public class ServiceLockIT {
       final RetryLockWatcher zlw1 = new RetryLockWatcher();
       ServiceLock zl1 =
           getZooLock(zk1, parent, UUID.fromString("00000000-0000-0000-0000-aaaaaaaaaaaa"));
-      zl1.lock(zlw1, "test1".getBytes(UTF_8));
+      zl1.lock(zlw1, new ServiceLockData(UUID.randomUUID(), "test1", ThriftService.TSERV,
+          ServiceDescriptor.DEFAULT_GROUP_NAME));
       // The call above creates two nodes in ZK because of the overridden create method in
       // ZooKeeperWrapper.
       // The nodes created are:
@@ -452,7 +464,8 @@ public class ServiceLockIT {
       final RetryLockWatcher zlw2 = new RetryLockWatcher();
       ServiceLock zl2 =
           getZooLock(zk2, parent, UUID.fromString("00000000-0000-0000-0000-bbbbbbbbbbbb"));
-      zl2.lock(zlw2, "test1".getBytes(UTF_8));
+      zl2.lock(zlw2, new ServiceLockData(UUID.randomUUID(), "test2", ThriftService.TSERV,
+          ServiceDescriptor.DEFAULT_GROUP_NAME));
       // The call above creates two nodes in ZK because of the overridden create method in
       // ZooKeeperWrapper.
       // The nodes created are:
@@ -539,7 +552,8 @@ public class ServiceLockIT {
           ServiceLock zl = getZooLock(zk, parent, uuid);
           getLockLatch.countDown(); // signal we are done
           getLockLatch.await(); // wait for others to finish
-          zl.lock(lockWatcher, "test1".getBytes(UTF_8)); // race to the lock
+          zl.lock(lockWatcher, new ServiceLockData(UUID.randomUUID(), "test1", ThriftService.TSERV,
+              ServiceDescriptor.DEFAULT_GROUP_NAME)); // race to the lock
           lockCompletedLatch.countDown();
           unlockLatch.await();
           zl.unlock();
@@ -668,7 +682,8 @@ public class ServiceLockIT {
 
       TestALW lw = new TestALW();
 
-      boolean ret = zl.tryLock(lw, "test1".getBytes(UTF_8));
+      boolean ret = zl.tryLock(lw, new ServiceLockData(UUID.randomUUID(), "test1",
+          ThriftService.TSERV, ServiceDescriptor.DEFAULT_GROUP_NAME));
 
       assertTrue(ret);
 
@@ -702,11 +717,17 @@ public class ServiceLockIT {
 
       TestALW lw = new TestALW();
 
-      zl.lock(lw, "test1".getBytes(UTF_8));
-      assertEquals("test1", new String(zk.getData(zl.getLockPath(), null, null)));
-
-      zl.replaceLockData("test2".getBytes(UTF_8));
-      assertEquals("test2", new String(zk.getData(zl.getLockPath(), null, null)));
+      ServiceLockData sld1 = new ServiceLockData(UUID.randomUUID(), "test1", ThriftService.TSERV,
+          ServiceDescriptor.DEFAULT_GROUP_NAME);
+      zl.lock(lw, sld1);
+      assertEquals(Optional.of(sld1),
+          ServiceLockData.parse(zk.getData(zl.getLockPath(), null, null)));
+
+      ServiceLockData sld2 = new ServiceLockData(UUID.randomUUID(), "test2", ThriftService.TSERV,
+          ServiceDescriptor.DEFAULT_GROUP_NAME);
+      zl.replaceLockData(sld2);
+      assertEquals(Optional.of(sld2),
+          ServiceLockData.parse(zk.getData(zl.getLockPath(), null, null)));
     }
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
index bcd1e8f018..2b13807ab7 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java
@@ -21,6 +21,7 @@ package org.apache.accumulo.test.functional;
 import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.junit.jupiter.api.Assertions.fail;
@@ -30,6 +31,7 @@ import java.time.Duration;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Map.Entry;
+import java.util.Optional;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.Constants;
@@ -52,8 +54,8 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.DeletesSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.DeletesSection.SkewedKeyValue;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.core.util.ServerServices;
-import org.apache.accumulo.core.util.ServerServices.Service;
+import org.apache.accumulo.core.util.ServiceLockData;
+import org.apache.accumulo.core.util.ServiceLockData.ThriftService;
 import org.apache.accumulo.gc.SimpleGarbageCollector;
 import org.apache.accumulo.minicluster.MemoryUnit;
 import org.apache.accumulo.minicluster.ServerType;
@@ -73,6 +75,7 @@ import org.apache.zookeeper.KeeperException.NoNodeException;
 import org.junit.jupiter.api.Test;
 
 import com.google.common.collect.Iterators;
+import com.google.common.net.HostAndPort;
 
 public class GarbageCollectorIT extends ConfigurableMacBase {
   private static final String OUR_SECRET = "itsreallysecret";
@@ -284,22 +287,15 @@ public class GarbageCollectorIT extends ConfigurableMacBase {
         if (locks != null && !locks.isEmpty()) {
           String lockPath = path + "/" + locks.get(0);
 
-          String gcLoc = new String(zk.getData(lockPath));
+          Optional<ServiceLockData> sld = ServiceLockData.parse(zk.getData(lockPath));
 
-          assertTrue(gcLoc.startsWith(Service.GC_CLIENT.name()),
-              "Found unexpected data in zookeeper for GC location: " + gcLoc);
-          int loc = gcLoc.indexOf(ServerServices.SEPARATOR_CHAR);
-          assertNotEquals(-1, loc, "Could not find split point of GC location for: " + gcLoc);
-          String addr = gcLoc.substring(loc + 1);
+          assertNotNull(sld.get());
+          HostAndPort hostAndPort = sld.get().getAddress(ThriftService.GC);
 
-          int addrSplit = addr.indexOf(':');
-          assertNotEquals(-1, addrSplit, "Could not find split of GC host:port for: " + addr);
-
-          String host = addr.substring(0, addrSplit), port = addr.substring(addrSplit + 1);
           // We shouldn't have the "bindall" address in zk
-          assertNotEquals("0.0.0.0", host);
+          assertNotEquals("0.0.0.0", hostAndPort.getHost());
           // Nor should we have the "random port" in zk
-          assertNotEquals(0, Integer.parseInt(port));
+          assertNotEquals(0, hostAndPort.getPort());
           return;
         }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java
index e2693edf90..b01b35aefc 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java
@@ -38,6 +38,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Optional;
 import java.util.Set;
 import java.util.TreeMap;
 import java.util.concurrent.ExecutorService;
@@ -78,6 +79,7 @@ import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.MonitorUtil;
+import org.apache.accumulo.core.util.ServiceLockData;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
@@ -169,14 +171,14 @@ public class ReadWriteIT extends AccumuloClusterHarness {
       var zLockPath =
           ServiceLock.path(ZooUtil.getRoot(accumuloClient.instanceOperations().getInstanceId())
               + Constants.ZMANAGER_LOCK);
-      byte[] managerLockData;
+      Optional<ServiceLockData> managerLockData;
       do {
         managerLockData = ServiceLock.getLockData(zcache, zLockPath, null);
-        if (managerLockData != null) {
+        if (managerLockData.isPresent()) {
           log.info("Manager lock is still held");
           Thread.sleep(1000);
         }
-      } while (managerLockData != null);
+      } while (managerLockData.isPresent());
       control.stopAllServers(ServerType.MANAGER);
       control.stopAllServers(ServerType.TABLET_SERVER);
       control.stopAllServers(ServerType.GARBAGE_COLLECTOR);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/RestartIT.java b/test/src/main/java/org/apache/accumulo/test/functional/RestartIT.java
index 4fe658c330..f67b99e680 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/RestartIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/RestartIT.java
@@ -24,6 +24,7 @@ import static org.junit.jupiter.api.Assertions.assertNotNull;
 
 import java.io.IOException;
 import java.util.Map.Entry;
+import java.util.Optional;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
@@ -39,6 +40,7 @@ import org.apache.accumulo.core.fate.zookeeper.ServiceLock;
 import org.apache.accumulo.core.fate.zookeeper.ZooCache;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
 import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.util.ServiceLockData;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
@@ -133,27 +135,27 @@ public class RestartIT extends AccumuloClusterHarness {
       ZooCache zcache = cluster.getServerContext().getZooCache();
       var zLockPath = ServiceLock
           .path(ZooUtil.getRoot(c.instanceOperations().getInstanceId()) + Constants.ZMANAGER_LOCK);
-      byte[] managerLockData;
+      Optional<ServiceLockData> managerLockData;
       do {
         managerLockData = ServiceLock.getLockData(zcache, zLockPath, null);
-        if (managerLockData != null) {
+        if (managerLockData.isPresent()) {
           log.info("Manager lock is still held");
           Thread.sleep(1000);
         }
-      } while (managerLockData != null);
+      } while (managerLockData.isPresent());
 
       cluster.start();
       sleepUninterruptibly(5, TimeUnit.MILLISECONDS);
       control.stopAllServers(ServerType.MANAGER);
 
-      managerLockData = new byte[0];
+      managerLockData = null;
       do {
         managerLockData = ServiceLock.getLockData(zcache, zLockPath, null);
-        if (managerLockData != null) {
+        if (managerLockData.isPresent()) {
           log.info("Manager lock is still held");
           Thread.sleep(1000);
         }
-      } while (managerLockData != null);
+      } while (managerLockData.isPresent());
       cluster.start();
       VerifyIngest.verifyIngest(c, params);
     }
@@ -184,14 +186,14 @@ public class RestartIT extends AccumuloClusterHarness {
       ZooCache zcache = cluster.getServerContext().getZooCache();
       var zLockPath = ServiceLock
           .path(ZooUtil.getRoot(c.instanceOperations().getInstanceId()) + Constants.ZMANAGER_LOCK);
-      byte[] managerLockData;
+      Optional<ServiceLockData> managerLockData;
       do {
         managerLockData = ServiceLock.getLockData(zcache, zLockPath, null);
-        if (managerLockData != null) {
+        if (managerLockData.isPresent()) {
           log.info("Manager lock is still held");
           Thread.sleep(1000);
         }
-      } while (managerLockData != null);
+      } while (managerLockData.isPresent());
 
       cluster.start();
       assertEquals(0, ret.get().intValue());
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java
index d4332c34cc..9c6eee4d3e 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java
@@ -18,7 +18,6 @@
  */
 package org.apache.accumulo.test.functional;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
@@ -69,6 +68,8 @@ import org.apache.accumulo.core.metadata.schema.TabletMetadata;
 import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.ColumnFQ;
+import org.apache.accumulo.core.util.ServiceLockData;
+import org.apache.accumulo.core.util.ServiceLockData.ThriftService;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.manager.state.Assignment;
 import org.apache.accumulo.server.util.ManagerMetadataUtil;
@@ -113,7 +114,8 @@ public class SplitRecoveryIT extends ConfigurableMacBase {
       public void unableToMonitorLockNode(Exception e) {
         System.exit(-1);
       }
-    }, "foo".getBytes(UTF_8));
+    }, new ServiceLockData(UUID.randomUUID(), "foo", ThriftService.TSERV,
+        ServiceLockData.ServiceDescriptor.DEFAULT_GROUP_NAME));
 
     if (!gotLock) {
       System.err.println("Failed to get lock " + zPath);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java b/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
index 0305251acd..e80f24b15d 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
@@ -19,7 +19,6 @@
 package org.apache.accumulo.test.functional;
 
 import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
-import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.accumulo.harness.AccumuloITBase.random;
 
 import java.util.HashMap;
@@ -41,8 +40,8 @@ import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
 import org.apache.accumulo.core.tabletscan.thrift.TabletScanClientService;
 import org.apache.accumulo.core.tabletserver.thrift.TabletServerClientService;
 import org.apache.accumulo.core.trace.TraceUtil;
-import org.apache.accumulo.core.util.ServerServices;
-import org.apache.accumulo.core.util.ServerServices.Service;
+import org.apache.accumulo.core.util.ServiceLockData;
+import org.apache.accumulo.core.util.ServiceLockData.ThriftService;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.client.ClientServiceHandler;
@@ -160,9 +159,8 @@ public class ZombieTServer {
       }
     };
 
-    byte[] lockContent =
-        new ServerServices(addressString, Service.TSERV_CLIENT).toString().getBytes(UTF_8);
-    if (zlock.tryLock(lw, lockContent)) {
+    if (zlock.tryLock(lw, new ServiceLockData(UUID.randomUUID(), addressString, ThriftService.TSERV,
+        ServiceLockData.ServiceDescriptor.DEFAULT_GROUP_NAME))) {
       log.debug("Obtained tablet server lock {}", zlock.getLockPath());
     }
     // modify metadata