You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ct...@apache.org on 2023/06/12 23:24:53 UTC

[accumulo] branch 2.1 updated: Add tool to edit props in ZooKeeper with Accumulo offline (#3445)

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

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


The following commit(s) were added to refs/heads/2.1 by this push:
     new c6d47d032e Add tool to edit props in ZooKeeper with Accumulo offline (#3445)
c6d47d032e is described below

commit c6d47d032ef59c7c9dbe5e604bb49eb3970176da
Author: EdColeman <de...@etcoleman.com>
AuthorDate: Mon Jun 12 19:24:47 2023 -0400

    Add tool to edit props in ZooKeeper with Accumulo offline (#3445)
    
    Add tool to modify properties in ZooKeeper without a running cluster
---
 .../accumulo/server/conf/util/ZooInfoViewer.java   | 100 +------
 .../accumulo/server/conf/util/ZooPropEditor.java   | 305 +++++++++++++++++++++
 .../accumulo/server/conf/util/ZooPropUtils.java    | 140 ++++++++++
 .../server/conf/util/ZooInfoViewerTest.java        |  25 --
 .../server/conf/util/ZooPropEditorTest.java        |  42 +++
 .../server/conf/util/ZooPropUtilsTest.java         |  68 +++++
 .../accumulo/test/conf/util/ZooPropEditorIT.java   | 138 ++++++++++
 .../apache/accumulo/test/start/KeywordStartIT.java |   2 +
 8 files changed, 698 insertions(+), 122 deletions(-)

diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooInfoViewer.java b/server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooInfoViewer.java
index 43aae8c8d5..c45b0b8832 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooInfoViewer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooInfoViewer.java
@@ -20,12 +20,10 @@ package org.apache.accumulo.server.conf.util;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.accumulo.core.Constants.ZINSTANCES;
-import static org.apache.accumulo.core.Constants.ZNAMESPACES;
-import static org.apache.accumulo.core.Constants.ZNAMESPACE_NAME;
 import static org.apache.accumulo.core.Constants.ZROOT;
-import static org.apache.accumulo.core.Constants.ZTABLES;
-import static org.apache.accumulo.core.Constants.ZTABLE_NAME;
-import static org.apache.accumulo.core.Constants.ZTABLE_NAMESPACE;
+import static org.apache.accumulo.server.conf.util.ZooPropUtils.getNamespaceIdToNameMap;
+import static org.apache.accumulo.server.conf.util.ZooPropUtils.getTableIdToName;
+import static org.apache.accumulo.server.conf.util.ZooPropUtils.readInstancesFromZk;
 import static org.apache.accumulo.server.zookeeper.ZooAclUtil.checkWritableAuth;
 import static org.apache.accumulo.server.zookeeper.ZooAclUtil.extractAuthName;
 import static org.apache.accumulo.server.zookeeper.ZooAclUtil.translateZooPerm;
@@ -45,14 +43,11 @@ import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.TreeSet;
-import java.util.UUID;
 import java.util.stream.Collectors;
 
 import org.apache.accumulo.core.cli.ConfigOpts;
-import org.apache.accumulo.core.clientImpl.Namespace;
 import org.apache.accumulo.core.data.InstanceId;
 import org.apache.accumulo.core.data.NamespaceId;
 import org.apache.accumulo.core.data.TableId;
@@ -169,25 +164,6 @@ public class ZooInfoViewer implements KeywordExecutable {
     }
   }
 
-  Map<NamespaceId,String> getNamespaceIdToNameMap(InstanceId iid, final ZooReader zooReader) {
-    SortedMap<NamespaceId,String> namespaceToName = new TreeMap<>();
-    String zooNsRoot = ZooUtil.getRoot(iid) + ZNAMESPACES;
-    try {
-      List<String> nsids = zooReader.getChildren(zooNsRoot);
-      for (String id : nsids) {
-        String path = zooNsRoot + "/" + id + ZNAMESPACE_NAME;
-        String name = new String(zooReader.getData(path), UTF_8);
-        namespaceToName.put(NamespaceId.of(id), name);
-      }
-    } catch (InterruptedException ex) {
-      Thread.currentThread().interrupt();
-      throw new IllegalStateException("Interrupted reading namespace ids from ZooKeeper", ex);
-    } catch (KeeperException ex) {
-      throw new IllegalStateException("Failed to read namespace ids from ZooKeeper", ex);
-    }
-    return namespaceToName;
-  }
-
   private void printProps(final InstanceId iid, final ZooReader zooReader, final Opts opts,
       final PrintWriter writer) throws Exception {
 
@@ -329,49 +305,6 @@ public class ZooInfoViewer implements KeywordExecutable {
 
   }
 
-  /**
-   * Read the instance names and instance ids from ZooKeeper. The storage structure in ZooKeeper is:
-   *
-   * <pre>
-   *   /accumulo/instances/instance_name  - with the instance id stored as data.
-   * </pre>
-   *
-   * @return a map of (instance name, instance id) entries
-   */
-  Map<String,InstanceId> readInstancesFromZk(final ZooReader zooReader) {
-    String instanceRoot = ZROOT + ZINSTANCES;
-    Map<String,InstanceId> idMap = new TreeMap<>();
-    try {
-      List<String> names = zooReader.getChildren(instanceRoot);
-      names.forEach(name -> {
-        InstanceId iid = getInstanceIdForName(zooReader, name);
-        idMap.put(name, iid);
-      });
-    } catch (InterruptedException ex) {
-      Thread.currentThread().interrupt();
-      throw new IllegalStateException("Interrupted reading instance name info from ZooKeeper", ex);
-    } catch (KeeperException ex) {
-      throw new IllegalStateException("Failed to read instance name info from ZooKeeper", ex);
-    }
-    return idMap;
-  }
-
-  private InstanceId getInstanceIdForName(ZooReader zooReader, String name) {
-    String instanceRoot = ZROOT + ZINSTANCES;
-    String path = "";
-    try {
-      path = instanceRoot + "/" + name;
-      byte[] uuid = zooReader.getData(path);
-      return InstanceId.of(UUID.fromString(new String(uuid, UTF_8)));
-    } catch (InterruptedException ex) {
-      Thread.currentThread().interrupt();
-      throw new IllegalStateException("Interrupted reading instance id from ZooKeeper", ex);
-    } catch (KeeperException ex) {
-      log.warn("Failed to read instance id for " + path);
-      return null;
-    }
-  }
-
   private void printInstanceIds(final Map<String,InstanceId> instanceIdMap, PrintWriter writer) {
     writer.println("Instances (Instance Name, Instance ID)");
     instanceIdMap.forEach((name, iid) -> writer.println(name + "=" + iid));
@@ -448,33 +381,6 @@ public class ZooInfoViewer implements KeywordExecutable {
     return results;
   }
 
-  private Map<TableId,String> getTableIdToName(InstanceId iid,
-      Map<NamespaceId,String> id2NamespaceMap, ZooReader zooReader) {
-    SortedMap<TableId,String> idToName = new TreeMap<>();
-
-    String zooTables = ZooUtil.getRoot(iid) + ZTABLES;
-    try {
-      List<String> tids = zooReader.getChildren(zooTables);
-      for (String t : tids) {
-        String path = zooTables + "/" + t;
-        String tname = new String(zooReader.getData(path + ZTABLE_NAME), UTF_8);
-        NamespaceId tNsId =
-            NamespaceId.of(new String(zooReader.getData(path + ZTABLE_NAMESPACE), UTF_8));
-        if (tNsId.equals(Namespace.DEFAULT.id())) {
-          idToName.put(TableId.of(t), tname);
-        } else {
-          idToName.put(TableId.of(t), id2NamespaceMap.get(tNsId) + "." + tname);
-        }
-      }
-    } catch (InterruptedException ex) {
-      Thread.currentThread().interrupt();
-      throw new IllegalStateException("Interrupted reading table ids from ZooKeeper", ex);
-    } catch (KeeperException ex) {
-      throw new IllegalStateException("Failed reading table id info from ZooKeeper");
-    }
-    return idToName;
-  }
-
   private void printSortedProps(final PrintWriter writer,
       final Map<String,VersionedProperties> props) {
     log.trace("Printing: {}", props);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooPropEditor.java b/server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooPropEditor.java
new file mode 100644
index 0000000000..b629e261ff
--- /dev/null
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooPropEditor.java
@@ -0,0 +1,305 @@
+/*
+ * 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.conf.util;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.server.conf.util.ZooPropUtils.getNamespaceIdToNameMap;
+import static org.apache.accumulo.server.conf.util.ZooPropUtils.getTableIdToName;
+
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.accumulo.core.cli.ConfigOpts;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.InstanceId;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.fate.zookeeper.ZooReader;
+import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.conf.codec.VersionedProperties;
+import org.apache.accumulo.server.conf.store.NamespacePropKey;
+import org.apache.accumulo.server.conf.store.PropStoreKey;
+import org.apache.accumulo.server.conf.store.SystemPropKey;
+import org.apache.accumulo.server.conf.store.TablePropKey;
+import org.apache.accumulo.server.conf.store.impl.PropStoreWatcher;
+import org.apache.accumulo.server.conf.store.impl.ReadyMonitor;
+import org.apache.accumulo.server.conf.store.impl.ZooPropStore;
+import org.apache.accumulo.server.util.PropUtil;
+import org.apache.accumulo.start.spi.KeywordExecutable;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.beust.jcommander.Parameter;
+import com.google.auto.service.AutoService;
+
+@AutoService(KeywordExecutable.class)
+public class ZooPropEditor implements KeywordExecutable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ZooPropEditor.class);
+  private final NullWatcher nullWatcher =
+      new NullWatcher(new ReadyMonitor(ZooInfoViewer.class.getSimpleName(), 20_000L));
+
+  /**
+   * No-op constructor - provided so ServiceLoader autoload does not consume resources.
+   */
+  public ZooPropEditor() {}
+
+  @Override
+  public String keyword() {
+    return "zoo-prop-editor";
+  }
+
+  @Override
+  public String description() {
+    return "Emergency tool to modify properties stored in ZooKeeper without a cluster."
+        + " Prefer using the shell if it is available";
+  }
+
+  @Override
+  public void execute(String[] args) throws Exception {
+    ZooPropEditor.Opts opts = new ZooPropEditor.Opts();
+    opts.parseArgs(ZooPropEditor.class.getName(), args);
+
+    ZooReaderWriter zrw = new ZooReaderWriter(opts.getSiteConfiguration());
+
+    var siteConfig = opts.getSiteConfiguration();
+    try (ServerContext context = new ServerContext(siteConfig)) {
+
+      InstanceId iid = context.getInstanceID();
+
+      PropStoreKey<?> propKey = getPropKey(iid, opts, zrw);
+      switch (opts.getCmdMode()) {
+        case SET:
+          setProperty(context, propKey, opts);
+          break;
+        case DELETE:
+          deleteProperty(context, propKey, readPropNode(propKey, zrw), opts);
+          break;
+        case PRINT:
+          printProperties(context, propKey, readPropNode(propKey, zrw));
+          break;
+        case ERROR:
+        default:
+          throw new IllegalArgumentException("Invalid operation requested");
+      }
+    }
+  }
+
+  private void setProperty(final ServerContext context, final PropStoreKey<?> propKey,
+      final Opts opts) {
+    LOG.trace("set {}", propKey);
+
+    if (!opts.setOpt.contains("=")) {
+      throw new IllegalArgumentException(
+          "Invalid set property format. Requires name=value, received " + opts.setOpt);
+    }
+    String[] tokens = opts.setOpt.split("=");
+    Map<String,String> propValue = Map.of(tokens[0].trim(), tokens[1].trim());
+    PropUtil.setProperties(context, propKey, propValue);
+  }
+
+  private void deleteProperty(final ServerContext context, final PropStoreKey<?> propKey,
+      VersionedProperties versionedProperties, final Opts opts) {
+    LOG.trace("delete {} - {}", propKey, opts.deleteOpt);
+    String p = opts.deleteOpt.trim();
+    if (p.isEmpty() || !Property.isValidPropertyKey(p)) {
+      throw new IllegalArgumentException("Invalid property name, Received: '" + p + "'");
+    }
+    // warn, but not throwing an error. If this was run in a script, allow the script to continue.
+    if (!versionedProperties.asMap().containsKey(p)) {
+      LOG.warn("skipping delete: property '{}' is not set for: {}- delete would have no effect", p,
+          propKey);
+      return;
+    }
+    PropUtil.removeProperties(context, propKey, List.of(p));
+  }
+
+  private void printProperties(final ServerContext context, final PropStoreKey<?> propKey,
+      final VersionedProperties props) {
+    LOG.trace("print {}", propKey);
+
+    OutputStream outStream = System.out;
+
+    String scope;
+    if (propKey instanceof SystemPropKey) {
+      scope = "SYSTEM";
+    } else if (propKey instanceof NamespacePropKey) {
+      scope = "NAMESPACE";
+    } else if (propKey instanceof TablePropKey) {
+      scope = "TABLE";
+    } else {
+      scope = "unknown";
+    }
+
+    try (PrintWriter writer =
+        new PrintWriter(new BufferedWriter(new OutputStreamWriter(outStream, UTF_8)))) {
+      // header
+      writer.printf(": Instance name: %s\n", context.getInstanceName());
+      writer.printf(": Instance Id: %s\n", context.getInstanceID());
+      writer.printf(": Property scope: %s\n", scope);
+      writer.printf(": ZooKeeper path: %s\n", propKey.getPath());
+      writer.printf(": Name: %s\n",
+          getDisplayName(propKey, context.getInstanceID(), context.getZooReader()));
+      writer.printf(": Id: %s\n", propKey.getId());
+      writer.printf(": Data version: %d\n", props.getDataVersion());
+      writer.printf(": Timestamp: %s\n", props.getTimestampISO());
+
+      // skip filtering if no props
+      if (props.asMap().isEmpty()) {
+        return;
+      }
+
+      SortedMap<String,String> sortedMap = new TreeMap<>(props.asMap());
+      sortedMap.forEach((name, value) -> writer.printf("%s=%s\n", name, value));
+    }
+  }
+
+  private VersionedProperties readPropNode(final PropStoreKey<?> propKey,
+      final ZooReader zooReader) {
+    try {
+      return ZooPropStore.readFromZk(propKey, nullWatcher, zooReader);
+    } catch (IOException | KeeperException | InterruptedException ex) {
+      throw new IllegalStateException(ex);
+    }
+  }
+
+  private PropStoreKey<?> getPropKey(final InstanceId iid, final ZooPropEditor.Opts opts,
+      final ZooReader zooReader) {
+
+    // either tid or table name option provided, get the table id
+    if (!opts.tableOpt.isEmpty() || !opts.tableIdOpt.isEmpty()) {
+      TableId tid = getTableId(iid, opts, zooReader);
+      return TablePropKey.of(iid, tid);
+    }
+
+    // either nid of namespace name provided, get the namespace id.
+    if (!opts.namespaceOpt.isEmpty() || !opts.namespaceIdOpt.isEmpty()) {
+      NamespaceId nid = getNamespaceId(iid, opts, zooReader);
+      return NamespacePropKey.of(iid, nid);
+    }
+
+    // no table or namespace, assume system.
+    return SystemPropKey.of(iid);
+  }
+
+  private TableId getTableId(final InstanceId iid, final ZooPropEditor.Opts opts,
+      final ZooReader zooReader) {
+    if (!opts.tableIdOpt.isEmpty()) {
+      return TableId.of(opts.tableIdOpt);
+    }
+    Map<NamespaceId,String> nids = getNamespaceIdToNameMap(iid, zooReader);
+
+    Map<TableId,String> tids = getTableIdToName(iid, nids, zooReader);
+    return tids.entrySet().stream().filter(entry -> opts.tableOpt.equals(entry.getValue()))
+        .map(Map.Entry::getKey).findAny()
+        .orElseThrow(() -> new IllegalArgumentException("Could not find table " + opts.tableOpt));
+  }
+
+  private NamespaceId getNamespaceId(final InstanceId iid, final ZooPropEditor.Opts opts,
+      final ZooReader zooReader) {
+    if (!opts.namespaceIdOpt.isEmpty()) {
+      return NamespaceId.of(opts.namespaceIdOpt);
+    }
+    Map<NamespaceId,String> nids = getNamespaceIdToNameMap(iid, zooReader);
+    return nids.entrySet().stream().filter(entry -> opts.namespaceOpt.equals(entry.getValue()))
+        .map(Map.Entry::getKey).findAny().orElseThrow(
+            () -> new IllegalArgumentException("Could not find namespace " + opts.namespaceOpt));
+  }
+
+  private String getDisplayName(final PropStoreKey<?> propStoreKey, final InstanceId iid,
+      final ZooReader zooReader) {
+
+    if (propStoreKey instanceof TablePropKey) {
+      Map<NamespaceId,String> nids = getNamespaceIdToNameMap(iid, zooReader);
+      return getTableIdToName(iid, nids, zooReader).getOrDefault((TableId) propStoreKey.getId(),
+          "unknown");
+    }
+    if (propStoreKey instanceof NamespacePropKey) {
+      return getNamespaceIdToNameMap(iid, zooReader)
+          .getOrDefault((NamespaceId) propStoreKey.getId(), "unknown");
+    }
+    if (propStoreKey instanceof SystemPropKey) {
+      return "system";
+    }
+    LOG.info("Undefined PropStoreKey type provided, cannot decode name for classname: {}",
+        propStoreKey.getClass().getName());
+    return "unknown";
+  }
+
+  static class Opts extends ConfigOpts {
+
+    @Parameter(names = {"-d", "--delete"}, description = "delete a property")
+    public String deleteOpt = "";
+    @Parameter(names = {"-ns", "--namespace"},
+        description = "namespace to display/set/delete properties for")
+    public String namespaceOpt = "";
+    @Parameter(names = {"-nid", "--namespace-id"},
+        description = "namespace id to display/set/delete properties for")
+    public String namespaceIdOpt = "";
+    @Parameter(names = {"-s", "--set"}, description = "set a property")
+    public String setOpt = "";
+    @Parameter(names = {"-t", "--table"},
+        description = "table to display/set/delete properties for")
+    public String tableOpt = "";
+    @Parameter(names = {"-tid", "--table-id"},
+        description = "table id to display/set/delete properties for")
+    public String tableIdOpt = "";
+
+    @Override
+    public void parseArgs(String programName, String[] args, Object... others) {
+      super.parseArgs(programName, args, others);
+      var cmdMode = getCmdMode();
+      if (cmdMode == Opts.CmdMode.ERROR) {
+        throw new IllegalArgumentException("Cannot use set and delete in one command");
+      }
+    }
+
+    CmdMode getCmdMode() {
+      if (!deleteOpt.isEmpty() && !setOpt.isEmpty()) {
+        return CmdMode.ERROR;
+      }
+      if (!deleteOpt.isEmpty()) {
+        return CmdMode.DELETE;
+      }
+      if (!setOpt.isEmpty()) {
+        return CmdMode.SET;
+      }
+      return CmdMode.PRINT;
+    }
+
+    enum CmdMode {
+      ERROR, PRINT, SET, DELETE
+    }
+  }
+
+  private static class NullWatcher extends PropStoreWatcher {
+    public NullWatcher(ReadyMonitor zkReadyMonitor) {
+      super(zkReadyMonitor);
+    }
+  }
+}
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooPropUtils.java b/server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooPropUtils.java
new file mode 100644
index 0000000000..f403a12fa1
--- /dev/null
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooPropUtils.java
@@ -0,0 +1,140 @@
+/*
+ * 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.conf.util;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.Constants.ZINSTANCES;
+import static org.apache.accumulo.core.Constants.ZNAMESPACES;
+import static org.apache.accumulo.core.Constants.ZNAMESPACE_NAME;
+import static org.apache.accumulo.core.Constants.ZROOT;
+import static org.apache.accumulo.core.Constants.ZTABLES;
+import static org.apache.accumulo.core.Constants.ZTABLE_NAME;
+import static org.apache.accumulo.core.Constants.ZTABLE_NAMESPACE;
+
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.UUID;
+
+import org.apache.accumulo.core.clientImpl.Namespace;
+import org.apache.accumulo.core.data.InstanceId;
+import org.apache.accumulo.core.data.NamespaceId;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.fate.zookeeper.ZooReader;
+import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ZooPropUtils {
+  private static final Logger LOG = LoggerFactory.getLogger(ZooPropUtils.class);
+
+  private ZooPropUtils() {}
+
+  /**
+   * Read the instance names and instance ids from ZooKeeper. The storage structure in ZooKeeper is:
+   *
+   * <pre>
+   *   /accumulo/instances/instance_name  - with the instance id stored as data.
+   * </pre>
+   *
+   * @return a map of (instance name, instance id) entries
+   */
+  public static Map<String,InstanceId> readInstancesFromZk(final ZooReader zooReader) {
+    String instanceRoot = ZROOT + ZINSTANCES;
+    Map<String,InstanceId> idMap = new TreeMap<>();
+    try {
+      List<String> names = zooReader.getChildren(instanceRoot);
+      names.forEach(name -> {
+        InstanceId iid = getInstanceIdForName(zooReader, name);
+        idMap.put(name, iid);
+      });
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      throw new IllegalStateException("Interrupted reading instance name info from ZooKeeper", ex);
+    } catch (KeeperException ex) {
+      throw new IllegalStateException("Failed to read instance name info from ZooKeeper", ex);
+    }
+    return idMap;
+  }
+
+  private static InstanceId getInstanceIdForName(ZooReader zooReader, String name) {
+    String instanceRoot = ZROOT + ZINSTANCES;
+    String path = "";
+    try {
+      path = instanceRoot + "/" + name;
+      byte[] uuid = zooReader.getData(path);
+      return InstanceId.of(UUID.fromString(new String(uuid, UTF_8)));
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      throw new IllegalStateException("Interrupted reading instance id from ZooKeeper", ex);
+    } catch (KeeperException ex) {
+      LOG.warn("Failed to read instance id for " + path);
+      return null;
+    }
+  }
+
+  public static Map<NamespaceId,String> getNamespaceIdToNameMap(final InstanceId iid,
+      final ZooReader zooReader) {
+    SortedMap<NamespaceId,String> namespaceToName = new TreeMap<>();
+    String zooNsRoot = ZooUtil.getRoot(iid) + ZNAMESPACES;
+    try {
+      List<String> nsids = zooReader.getChildren(zooNsRoot);
+      for (String id : nsids) {
+        String path = zooNsRoot + "/" + id + ZNAMESPACE_NAME;
+        String name = new String(zooReader.getData(path), UTF_8);
+        namespaceToName.put(NamespaceId.of(id), name);
+      }
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      throw new IllegalStateException("Interrupted reading namespace ids from ZooKeeper", ex);
+    } catch (KeeperException ex) {
+      throw new IllegalStateException("Failed to read namespace ids from ZooKeeper", ex);
+    }
+    return namespaceToName;
+  }
+
+  public static Map<TableId,String> getTableIdToName(InstanceId iid,
+      Map<NamespaceId,String> id2NamespaceMap, ZooReader zooReader) {
+    SortedMap<TableId,String> idToName = new TreeMap<>();
+
+    String zooTables = ZooUtil.getRoot(iid) + ZTABLES;
+    try {
+      List<String> tids = zooReader.getChildren(zooTables);
+      for (String t : tids) {
+        String path = zooTables + "/" + t;
+        String tname = new String(zooReader.getData(path + ZTABLE_NAME), UTF_8);
+        NamespaceId tNsId =
+            NamespaceId.of(new String(zooReader.getData(path + ZTABLE_NAMESPACE), UTF_8));
+        if (tNsId.equals(Namespace.DEFAULT.id())) {
+          idToName.put(TableId.of(t), tname);
+        } else {
+          idToName.put(TableId.of(t), id2NamespaceMap.get(tNsId) + "." + tname);
+        }
+      }
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      throw new IllegalStateException("Interrupted reading table ids from ZooKeeper", ex);
+    } catch (KeeperException ex) {
+      throw new IllegalStateException("Failed reading table id info from ZooKeeper");
+    }
+    return idToName;
+  }
+}
diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/util/ZooInfoViewerTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/util/ZooInfoViewerTest.java
index 7cd89f75f0..9ea587a320 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/conf/util/ZooInfoViewerTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/conf/util/ZooInfoViewerTest.java
@@ -134,31 +134,6 @@ public class ZooInfoViewerTest {
     assertEquals(2, opts.getTables().size());
   }
 
-  @Test
-  public void fetchInstancesFromZk() throws Exception {
-
-    String instAName = "INST_A";
-    InstanceId instA = InstanceId.of(UUID.randomUUID());
-    String instBName = "INST_B";
-    InstanceId instB = InstanceId.of(UUID.randomUUID());
-
-    ZooReader zooReader = createMock(ZooReader.class);
-    String namePath = ZROOT + ZINSTANCES;
-    expect(zooReader.getChildren(eq(namePath))).andReturn(List.of(instAName, instBName)).once();
-    expect(zooReader.getData(eq(namePath + "/" + instAName)))
-        .andReturn(instA.canonical().getBytes(UTF_8)).once();
-    expect(zooReader.getData(eq(namePath + "/" + instBName)))
-        .andReturn(instB.canonical().getBytes(UTF_8)).once();
-    replay(zooReader);
-
-    ZooInfoViewer viewer = new ZooInfoViewer();
-    Map<String,InstanceId> instanceMap = viewer.readInstancesFromZk(zooReader);
-
-    log.trace("id map returned: {}", instanceMap);
-    assertEquals(Map.of(instAName, instA, instBName, instB), instanceMap);
-    verify(zooReader);
-  }
-
   @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN", justification = "test generated output")
   @Test
   public void instanceIdOutputTest() throws Exception {
diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/util/ZooPropEditorTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/util/ZooPropEditorTest.java
new file mode 100644
index 0000000000..ab3268c36b
--- /dev/null
+++ b/server/base/src/test/java/org/apache/accumulo/server/conf/util/ZooPropEditorTest.java
@@ -0,0 +1,42 @@
+/*
+ * 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.conf.util;
+
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import org.junit.jupiter.api.Test;
+
+public class ZooPropEditorTest {
+
+  @Test
+  public void optionsAllDefault() {
+    ZooPropEditor.Opts opts = new ZooPropEditor.Opts();
+    assertTrue(opts.setOpt.isEmpty());
+    assertTrue(opts.deleteOpt.isEmpty());
+  }
+
+  @Test
+  public void invalidSetAndDelete() {
+    ZooPropEditor.Opts opts = new ZooPropEditor.Opts();
+    assertThrows(IllegalArgumentException.class, () -> opts.parseArgs(ZooInfoViewer.class.getName(),
+        new String[] {"-s", "foo=1", "-d", "bar=2"}));
+  }
+}
diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/util/ZooPropUtilsTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/util/ZooPropUtilsTest.java
new file mode 100644
index 0000000000..e77832ceab
--- /dev/null
+++ b/server/base/src/test/java/org/apache/accumulo/server/conf/util/ZooPropUtilsTest.java
@@ -0,0 +1,68 @@
+/*
+ * 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.conf.util;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.core.Constants.ZINSTANCES;
+import static org.apache.accumulo.core.Constants.ZROOT;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.eq;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import org.apache.accumulo.core.data.InstanceId;
+import org.apache.accumulo.core.fate.zookeeper.ZooReader;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class ZooPropUtilsTest {
+  private static final Logger LOG = LoggerFactory.getLogger(ZooPropUtilsTest.class);
+
+  @Test
+  public void fetchInstancesFromZk() throws Exception {
+
+    String instAName = "INST_A";
+    InstanceId instA = InstanceId.of(UUID.randomUUID());
+    String instBName = "INST_B";
+    InstanceId instB = InstanceId.of(UUID.randomUUID());
+
+    ZooReader zooReader = createMock(ZooReader.class);
+    String namePath = ZROOT + ZINSTANCES;
+    expect(zooReader.getChildren(eq(namePath))).andReturn(List.of(instAName, instBName)).once();
+    expect(zooReader.getData(eq(namePath + "/" + instAName)))
+        .andReturn(instA.canonical().getBytes(UTF_8)).once();
+    expect(zooReader.getData(eq(namePath + "/" + instBName)))
+        .andReturn(instB.canonical().getBytes(UTF_8)).once();
+    replay(zooReader);
+
+    Map<String,InstanceId> instanceMap = ZooPropUtils.readInstancesFromZk(zooReader);
+
+    LOG.trace("id map returned: {}", instanceMap);
+    assertEquals(Map.of(instAName, instA, instBName, instB), instanceMap);
+    verify(zooReader);
+  }
+
+}
diff --git a/test/src/main/java/org/apache/accumulo/test/conf/util/ZooPropEditorIT.java b/test/src/main/java/org/apache/accumulo/test/conf/util/ZooPropEditorIT.java
new file mode 100644
index 0000000000..caea14064d
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/conf/util/ZooPropEditorIT.java
@@ -0,0 +1,138 @@
+/*
+ * 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.test.conf.util;
+
+import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
+import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.time.Duration;
+
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.server.conf.util.ZooPropEditor;
+import org.apache.accumulo.test.util.Wait;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Tag(MINI_CLUSTER_ONLY)
+@Tag(SUNNY_DAY)
+public class ZooPropEditorIT extends SharedMiniClusterBase {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ZooPropEditorIT.class);
+
+  @Override
+  protected Duration defaultTimeout() {
+    return Duration.ofMinutes(1);
+  }
+
+  @BeforeAll
+  public static void setup() throws Exception {
+    SharedMiniClusterBase.startMiniCluster();
+  }
+
+  @AfterAll
+  public static void teardown() {
+    SharedMiniClusterBase.stopMiniCluster();
+  }
+
+  @Test
+  public void modifyPropTest() throws Exception {
+    String[] names = getUniqueNames(2);
+    String namespace = names[0];
+    String table = namespace + "." + names[1];
+
+    try (var client = Accumulo.newClient().from(getClientProps()).build()) {
+
+      client.namespaceOperations().create(namespace);
+      client.tableOperations().create(table);
+
+      LOG.debug("Tables: {}", client.tableOperations().list());
+
+      // override default in sys, and then over-ride that for table prop
+      client.instanceOperations().setProperty(Property.TABLE_BLOOM_ENABLED.getKey(), "true");
+      client.namespaceOperations().setProperty(namespace, Property.TABLE_BLOOM_ENABLED.getKey(),
+          "true");
+      client.tableOperations().setProperty(table, Property.TABLE_BLOOM_ENABLED.getKey(), "false");
+
+      assertTrue(Wait.waitFor(() -> client.instanceOperations().getSystemConfiguration()
+          .get(Property.TABLE_BLOOM_ENABLED.getKey()).equals("true"), 5000, 500));
+
+      ZooPropEditor tool = new ZooPropEditor();
+      // before - check setup correct
+      assertTrue(Wait.waitFor(() -> client.tableOperations().getTableProperties(table)
+          .get(Property.TABLE_BLOOM_ENABLED.getKey()).equals("false"), 5000, 500));
+
+      // set table property (table.bloom.enabled=true)
+      String[] setTablePropArgs = {"-p", getCluster().getAccumuloPropertiesPath(), "-t", table,
+          "-s", Property.TABLE_BLOOM_ENABLED.getKey() + "=true"};
+      tool.execute(setTablePropArgs);
+
+      // after set - check prop changed in ZooKeeper
+      assertTrue(Wait.waitFor(() -> client.tableOperations().getTableProperties(table)
+          .get(Property.TABLE_BLOOM_ENABLED.getKey()).equals("true"), 5000, 500));
+
+      String[] deleteTablePropArgs = {"-p", getCluster().getAccumuloPropertiesPath(), "-t", table,
+          "-d", Property.TABLE_BLOOM_ENABLED.getKey()};
+      tool.execute(deleteTablePropArgs);
+
+      // after delete - check map entry is null (removed from ZooKeeper)
+      assertTrue(Wait.waitFor(() -> client.tableOperations().getTableProperties(table)
+          .get(Property.TABLE_BLOOM_ENABLED.getKey()) == null, 5000, 500));
+
+      // set system property (changed from setup)
+      assertTrue(Wait.waitFor(() -> client.instanceOperations().getSystemConfiguration()
+          .get(Property.TABLE_BLOOM_ENABLED.getKey()).equals("true"), 5000, 500));
+
+      String[] setSystemPropArgs = {"-p", getCluster().getAccumuloPropertiesPath(), "-s",
+          Property.TABLE_BLOOM_ENABLED.getKey() + "=false"};
+      tool.execute(setSystemPropArgs);
+
+      // after set - check map entry is false
+      assertTrue(Wait.waitFor(() -> client.instanceOperations().getSystemConfiguration()
+          .get(Property.TABLE_BLOOM_ENABLED.getKey()).equals("false"), 5000, 500));
+
+      // set namespace property (changed from setup)
+      assertTrue(Wait.waitFor(() -> client.namespaceOperations().getNamespaceProperties(namespace)
+          .get(Property.TABLE_BLOOM_ENABLED.getKey()).equals("true"), 5000, 500));
+
+      String[] setNamespacePropArgs = {"-p", getCluster().getAccumuloPropertiesPath(), "-ns",
+          namespace, "-s", Property.TABLE_BLOOM_ENABLED.getKey() + "=false"};
+      tool.execute(setNamespacePropArgs);
+
+      // after set - check map entry is false
+      assertTrue(Wait.waitFor(() -> client.namespaceOperations().getNamespaceProperties(namespace)
+          .get(Property.TABLE_BLOOM_ENABLED.getKey()).equals("false"), 5000, 500));
+
+      String[] deleteNamespacePropArgs = {"-p", getCluster().getAccumuloPropertiesPath(), "-ns",
+          namespace, "-d", Property.TABLE_BLOOM_ENABLED.getKey()};
+      tool.execute(deleteNamespacePropArgs);
+
+      // after set - check map entry is false
+      assertTrue(Wait.waitFor(() -> client.namespaceOperations().getNamespaceProperties(namespace)
+          .get(Property.TABLE_BLOOM_ENABLED.getKey()) == null, 5000, 500));
+
+    }
+  }
+}
diff --git a/test/src/main/java/org/apache/accumulo/test/start/KeywordStartIT.java b/test/src/main/java/org/apache/accumulo/test/start/KeywordStartIT.java
index 2887d9e846..97b75467e5 100644
--- a/test/src/main/java/org/apache/accumulo/test/start/KeywordStartIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/start/KeywordStartIT.java
@@ -57,6 +57,7 @@ import org.apache.accumulo.server.conf.CheckCompactionConfig;
 import org.apache.accumulo.server.conf.CheckServerConfig;
 import org.apache.accumulo.server.conf.util.ConfigPropertyUpgrader;
 import org.apache.accumulo.server.conf.util.ZooInfoViewer;
+import org.apache.accumulo.server.conf.util.ZooPropEditor;
 import org.apache.accumulo.server.init.Initialize;
 import org.apache.accumulo.server.util.Admin;
 import org.apache.accumulo.server.util.ConvertConfig;
@@ -155,6 +156,7 @@ public class KeywordStartIT {
     expectSet.put("version", Version.class);
     expectSet.put("wal-info", LogReader.class);
     expectSet.put("zoo-info-viewer", ZooInfoViewer.class);
+    expectSet.put("zoo-prop-editor", ZooPropEditor.class);
     expectSet.put("zoo-zap", ZooZap.class);
     expectSet.put("zookeeper", ZooKeeperMain.class);