You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by GitBox <gi...@apache.org> on 2022/06/01 22:12:48 UTC

[GitHub] [accumulo] ctubbsii commented on a diff in pull request #2751: Add ZooKeeper info viewer utility

ctubbsii commented on code in PR #2751:
URL: https://github.com/apache/accumulo/pull/2751#discussion_r887332791


##########
server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooInfoViewer.java:
##########
@@ -0,0 +1,540 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.server.conf.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.io.BufferedWriter;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.time.Instant;
+import java.time.ZoneId;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+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.conf.SiteConfiguration;
+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.fate.zookeeper.ZooReader;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.fate.zookeeper.ZooUtil;
+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.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.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;
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+
+@AutoService(KeywordExecutable.class)
+@SuppressFBWarnings(value = "PATH_TRAVERSAL_OUT",
+    justification = "app is run in same security context as user providing the filename")
+public class ZooInfoViewer implements KeywordExecutable {
+  public static final DateTimeFormatter tsFormat =
+      DateTimeFormatter.ISO_OFFSET_DATE_TIME.withZone(ZoneId.from(ZoneOffset.UTC));
+  private static final Logger log = LoggerFactory.getLogger(ZooInfoViewer.class);
+  private final NullWatcher nullWatcher =
+      new NullWatcher(new ReadyMonitor(ZooInfoViewer.class.getSimpleName(), 20_000L));
+
+  private static final String INDENT = "  ";
+
+  /**
+   * No-op constructor - provided so ServiceLoader autoload does not consume resources.
+   */
+  public ZooInfoViewer() {}
+
+  public static void main(String[] args) throws Exception {
+    new ZooInfoViewer().execute(args);
+  }
+
+  @Override
+  public String keyword() {
+    return "zoo-info-viewer";
+  }
+
+  @Override
+  public String description() {
+    return "view Accumulo instance and property information stored in ZooKeeper";
+  }
+
+  @Override
+  public void execute(String[] args) {
+
+    ZooInfoViewer.Opts opts = new ZooInfoViewer.Opts();
+    opts.parseArgs(ZooInfoViewer.class.getName(), args);
+
+    log.info("print ids map: {}", opts.printIdMap);
+    log.info("print properties: {}", opts.printProps);
+    log.info("print instances: {}", opts.printInstanceIds);
+
+    ZooReader zooReader = new ZooReaderWriter(opts.getSiteConfiguration());
+
+    InstanceId iid = getInstanceId(zooReader, opts);
+    try {
+      generateReport(iid, opts, zooReader);
+    } catch (FileNotFoundException ex) {
+      throw new IllegalStateException("Failed to generate ZooKeeper info report", ex);

Review Comment:
   UncheckedIOException would be more appropriate here. However, there's no point in catching it at all. You can just throw it. The KeywordExecutable interface has a `throws Exception` in the interface API. All this does is add extra `caused by` clauses, making it harder for the user to parse the stack trace.
   



##########
server/base/src/test/java/org/apache/accumulo/server/conf/util/ZooInfoViewerTest.java:
##########
@@ -0,0 +1,437 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.server.conf.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.easymock.EasyMock.anyObject;
+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 static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.io.BufferedReader;
+import java.io.FileReader;
+import java.time.Instant;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+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.fate.zookeeper.ZooReader;
+import org.apache.accumulo.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.server.conf.codec.VersionedPropCodec;
+import org.apache.accumulo.server.conf.codec.VersionedProperties;
+import org.apache.accumulo.server.conf.store.NamespacePropKey;
+import org.apache.accumulo.server.conf.store.SystemPropKey;
+import org.apache.accumulo.server.conf.store.TablePropKey;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.Stat;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+
+public class ZooInfoViewerTest {
+
+  private final Logger log = LoggerFactory.getLogger(ZooInfoViewerTest.class);
+
+  private final VersionedPropCodec propCodec = VersionedPropCodec.getDefault();
+
+  @Test
+  public void simpleOutput() {
+    // StringWriter writer = new StringWriter();
+  }
+
+  @Test
+  public void optionsAllDefault() {
+    ZooInfoViewer.Opts opts = new ZooInfoViewer.Opts();
+    assertTrue(opts.printAllProps());
+    assertTrue(opts.printSysProps());
+    assertTrue(opts.printNamespaceProps());
+    assertTrue(opts.printTableProps());
+  }
+
+  @Test
+  public void onlySys() {
+    ZooInfoViewer.Opts opts = new ZooInfoViewer.Opts();
+    opts.parseArgs(ZooInfoViewer.class.getName(), new String[] {"--system"});
+
+    assertFalse(opts.printAllProps());
+    assertTrue(opts.printSysProps());
+    assertFalse(opts.printNamespaceProps());
+    assertFalse(opts.printTableProps());
+  }
+
+  @Test
+  public void onlyNamespaces() {
+    ZooInfoViewer.Opts opts = new ZooInfoViewer.Opts();
+    opts.parseArgs(ZooInfoViewer.class.getName(), new String[] {"-ns", "ns1", "ns2"});
+
+    assertFalse(opts.printAllProps());
+    assertFalse(opts.printSysProps());
+    assertTrue(opts.printNamespaceProps());
+    assertEquals(2, opts.getNamespaces().size());
+    assertFalse(opts.printTableProps());
+    assertEquals(0, opts.getTables().size());
+  }
+
+  @Test
+  public void allLongOpts() {
+    ZooInfoViewer.Opts opts = new ZooInfoViewer.Opts();
+    opts.parseArgs(ZooInfoViewer.class.getName(),
+        new String[] {"--system", "--namespaces", "ns1", "ns2", "--tables", "tb1", "tbl2"});
+
+    log.debug("namespaces: {}", opts.getNamespaces());
+    log.debug("tables: {}", opts.getTables());
+
+    assertFalse(opts.printAllProps());
+    assertTrue(opts.printSysProps());
+    assertTrue(opts.printNamespaceProps());
+    assertTrue(opts.printTableProps());
+    assertEquals(2, opts.getNamespaces().size());
+    assertEquals(2, opts.getTables().size());
+  }
+
+  @Test
+  public void allOpts() {
+    ZooInfoViewer.Opts opts = new ZooInfoViewer.Opts();
+    opts.parseArgs(ZooInfoViewer.class.getName(), new String[] {"-t", "tb1", "tbl2"});
+
+    assertFalse(opts.printAllProps());
+    assertFalse(opts.printSysProps());
+    assertFalse(opts.printNamespaceProps());
+    assertEquals(0, opts.getNamespaces().size());
+    assertTrue(opts.printTableProps());
+    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(instA, instanceMap.get(instAName));
+    assertEquals(instB, instanceMap.get(instBName));
+    verify(zooReader);
+  }
+
+  /**
+   * Expect that instance id passed is returned, instance name and zooReader are ignored.
+   */
+  @Test
+  public void instanceIdOption() 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.Opts opts = new ZooInfoViewer.Opts();
+    opts.parseArgs(ZooInfoViewer.class.getName(), new String[] {"--instanceName", instBName});
+
+    ZooInfoViewer viewer = new ZooInfoViewer();
+    InstanceId found = viewer.getInstanceId(zooReader, opts);
+
+    assertEquals(instB, found);
+
+    verify(zooReader);
+  }
+
+  /**
+   *
+   */
+  @Test
+  public void instanceNameTest() {
+    String uuid = UUID.randomUUID().toString();
+    ZooReader zooReader = createMock(ZooReader.class);
+    ZooInfoViewer.Opts opts = new ZooInfoViewer.Opts();
+    opts.parseArgs(ZooInfoViewer.class.getName(),
+        new String[] {"--instanceId", uuid, "--instanceName", "foo"});
+    replay(zooReader);
+
+    ZooInfoViewer viewer = new ZooInfoViewer();
+    InstanceId found = viewer.getInstanceId(zooReader, opts);
+
+    assertEquals(InstanceId.of(uuid), found);
+
+    verify(zooReader);
+  }
+
+  @Test
+  public void instanceIdOutputTest() throws Exception {
+    String uuid = UUID.randomUUID().toString();
+
+    ZooReader zooReader = createMock(ZooReader.class);
+    var instanceName = "test";
+    expect(zooReader.getChildren(eq(ZROOT + ZINSTANCES))).andReturn(List.of(instanceName)).once();
+    expect(zooReader.getData(eq(ZROOT + ZINSTANCES + "/" + instanceName)))
+        .andReturn(uuid.getBytes(UTF_8)).once();
+    replay(zooReader);
+
+    String testFileName = "./target/zoo-info-viewer-" + System.currentTimeMillis() + ".txt";
+
+    ZooInfoViewer.Opts opts = new ZooInfoViewer.Opts();
+    opts.parseArgs(ZooInfoViewer.class.getName(),
+        new String[] {"--instanceId", uuid, "--print-instances", "--outfile", testFileName});
+
+    ZooInfoViewer viewer = new ZooInfoViewer();
+    // InstanceId found = viewer.getInstanceId(zooReader, opts);
+    viewer.generateReport(InstanceId.of(uuid), opts, zooReader);
+    // assertEquals(InstanceId.of(uuid), found);
+
+    verify(zooReader);
+
+    String line;
+    try (BufferedReader in = new BufferedReader(new FileReader(testFileName, UTF_8))) {
+      boolean found = false;
+      while ((line = in.readLine()) != null) {
+        if (line.contains("=")) {
+          String trimmed = line.trim();
+          found = trimmed.startsWith(instanceName) && trimmed.endsWith(uuid);
+          break;
+        }
+      }
+      assertTrue(found, "expected instance name, instance id not found");
+
+    }
+  }
+
+  @Test
+  public void instanceNameOutputTest() throws Exception {
+    String uuid = UUID.randomUUID().toString();
+
+    ZooReader zooReader = createMock(ZooReader.class);
+    var instanceName = "test";
+    expect(zooReader.getChildren(eq(ZROOT + ZINSTANCES))).andReturn(List.of(instanceName)).once();
+    expect(zooReader.getData(eq(ZROOT + ZINSTANCES + "/" + instanceName)))
+        .andReturn(uuid.getBytes(UTF_8)).once();
+    replay(zooReader);
+
+    String testFileName = "./target/zoo-info-viewer-" + System.currentTimeMillis() + ".txt";
+
+    ZooInfoViewer.Opts opts = new ZooInfoViewer.Opts();
+    opts.parseArgs(ZooInfoViewer.class.getName(), new String[] {"--instanceName", instanceName,
+        "--print-instances", "--outfile", testFileName});
+
+    ZooInfoViewer viewer = new ZooInfoViewer();
+    // InstanceId found = viewer.getInstanceId(zooReader, opts);
+    viewer.generateReport(InstanceId.of(uuid), opts, zooReader);
+    // assertEquals(InstanceId.of(uuid), found);
+
+    verify(zooReader);
+
+    String line;
+    try (BufferedReader in = new BufferedReader(new FileReader(testFileName, UTF_8))) {
+      boolean found = false;
+      while ((line = in.readLine()) != null) {
+        if (line.contains("=")) {
+          String trimmed = line.trim();
+          found = trimmed.startsWith(instanceName) && trimmed.endsWith(uuid);
+          break;
+        }
+      }
+      assertTrue(found, "expected instance name, instance id not found");
+    }
+  }
+
+  @SuppressFBWarnings(value = "CRLF_INJECTION_LOGS",
+      justification = "test output of generated output")
+  @Test
+  public void propTest() throws Exception {
+    String uuid = UUID.randomUUID().toString();
+    InstanceId iid = InstanceId.of(uuid);
+
+    ZooReader zooReader = createMock(ZooReader.class);
+    var instanceName = "test";
+    expect(zooReader.getChildren(eq(ZROOT + ZINSTANCES))).andReturn(List.of(instanceName))
+        .anyTimes();
+    expect(zooReader.getData(eq(ZROOT + ZINSTANCES + "/" + instanceName)))
+        .andReturn(uuid.getBytes(UTF_8)).anyTimes();
+
+    var sysPropBytes = propCodec
+        .toBytes(new VersionedProperties(123, Instant.now(), Map.of("s1", "sv1", "s2", "sv2")));
+    expect(zooReader.getData(eq(SystemPropKey.of(iid).getNodePath()), anyObject(Watcher.class),
+        anyObject(Stat.class))).andReturn(sysPropBytes).anyTimes();
+
+    var nsBasePath = ZooUtil.getRoot(iid) + ZNAMESPACES;
+    expect(zooReader.getChildren(nsBasePath)).andReturn(List.of("a")).anyTimes();
+    expect(zooReader.getData(eq(nsBasePath + "/a" + ZNAMESPACE_NAME)))
+        .andReturn("a_name".getBytes(UTF_8)).anyTimes();
+    var nsPropBytes =
+        propCodec.toBytes(new VersionedProperties(123, Instant.now(), Map.of("n1", "nv1")));
+    NamespaceId nsId = NamespaceId.of("a");
+    expect(zooReader.getData(eq(NamespacePropKey.of(iid, nsId).getNodePath()),
+        anyObject(Watcher.class), anyObject(Stat.class))).andReturn(nsPropBytes).anyTimes();
+
+    var tBasePath = ZooUtil.getRoot(iid) + ZTABLES;
+    expect(zooReader.getChildren(tBasePath)).andReturn(List.of("t")).anyTimes();
+    expect(zooReader.getData(eq(tBasePath + "/t" + ZTABLE_NAME)))
+        .andReturn("t_table".getBytes(UTF_8)).anyTimes();
+    var tPropBytes =
+        propCodec.toBytes(new VersionedProperties(123, Instant.now(), Map.of("t1", "tv1")));
+    TableId tid = TableId.of("t");
+    expect(zooReader.getData(eq(TablePropKey.of(iid, tid).getNodePath()), anyObject(Watcher.class),
+        anyObject(Stat.class))).andReturn(tPropBytes).anyTimes();
+    expect(zooReader.getData(tBasePath + "/t" + ZTABLE_NAMESPACE))
+        .andReturn("+default".getBytes(UTF_8)).anyTimes();
+
+    replay(zooReader);
+
+    NamespacePropKey nsKey = NamespacePropKey.of(iid, nsId);
+    log.trace("namespace base path: {}", nsKey.getBasePath());
+
+    String testFileName = "./target/zoo-info-viewer-" + System.currentTimeMillis() + ".txt";
+
+    ZooInfoViewer.Opts opts = new ZooInfoViewer.Opts();
+    opts.parseArgs(ZooInfoViewer.class.getName(),
+        new String[] {"--instanceId", uuid, "--print-props", "--outfile", testFileName});
+
+    ZooInfoViewer viewer = new ZooInfoViewer();
+    // InstanceId found = viewer.getInstanceId(zooReader, opts);
+    viewer.generateReport(InstanceId.of(uuid), opts, zooReader);
+    // assertEquals(InstanceId.of(uuid), found);
+
+    verify(zooReader);
+
+    String line;
+    try (BufferedReader in = new BufferedReader(new FileReader(testFileName, UTF_8))) {
+      Map<String,String> props = new HashMap<>();
+      while ((line = in.readLine()) != null) {
+        if (line.contains("=")) {
+          log.trace("matched line: {}", line);
+          String trimmed = line.trim();
+          String[] kv = trimmed.split("=");
+          props.put(kv[0], kv[1]);
+        }
+      }
+      assertEquals(4, props.size());
+      assertEquals("sv1", props.get("s1"));
+      assertEquals("sv2", props.get("s2"));
+      assertEquals("nv1", props.get("n1"));
+      assertEquals("tv1", props.get("t1"));

Review Comment:
   You could probably just do `assertEquals(Map.of(...), props)`



##########
server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooInfoViewer.java:
##########
@@ -0,0 +1,540 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.server.conf.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.io.BufferedWriter;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.time.Instant;
+import java.time.ZoneId;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+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.conf.SiteConfiguration;
+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.fate.zookeeper.ZooReader;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.fate.zookeeper.ZooUtil;
+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.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.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;
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+
+@AutoService(KeywordExecutable.class)
+@SuppressFBWarnings(value = "PATH_TRAVERSAL_OUT",
+    justification = "app is run in same security context as user providing the filename")
+public class ZooInfoViewer implements KeywordExecutable {
+  public static final DateTimeFormatter tsFormat =
+      DateTimeFormatter.ISO_OFFSET_DATE_TIME.withZone(ZoneId.from(ZoneOffset.UTC));
+  private static final Logger log = LoggerFactory.getLogger(ZooInfoViewer.class);
+  private final NullWatcher nullWatcher =
+      new NullWatcher(new ReadyMonitor(ZooInfoViewer.class.getSimpleName(), 20_000L));
+
+  private static final String INDENT = "  ";
+
+  /**
+   * No-op constructor - provided so ServiceLoader autoload does not consume resources.
+   */
+  public ZooInfoViewer() {}
+
+  public static void main(String[] args) throws Exception {
+    new ZooInfoViewer().execute(args);
+  }
+
+  @Override
+  public String keyword() {
+    return "zoo-info-viewer";
+  }
+
+  @Override
+  public String description() {
+    return "view Accumulo instance and property information stored in ZooKeeper";
+  }
+
+  @Override
+  public void execute(String[] args) {
+
+    ZooInfoViewer.Opts opts = new ZooInfoViewer.Opts();
+    opts.parseArgs(ZooInfoViewer.class.getName(), args);
+
+    log.info("print ids map: {}", opts.printIdMap);
+    log.info("print properties: {}", opts.printProps);
+    log.info("print instances: {}", opts.printInstanceIds);
+
+    ZooReader zooReader = new ZooReaderWriter(opts.getSiteConfiguration());
+
+    InstanceId iid = getInstanceId(zooReader, opts);
+    try {
+      generateReport(iid, opts, zooReader);
+    } catch (FileNotFoundException ex) {
+      throw new IllegalStateException("Failed to generate ZooKeeper info report", ex);
+    }
+  }
+
+  void generateReport(final InstanceId iid, final ZooInfoViewer.Opts opts,
+      final ZooReader zooReader) throws FileNotFoundException {
+
+    OutputStream outStream;
+
+    String outfile = opts.getOutfile();
+    if (outfile == null || outfile.isEmpty()) {
+      log.trace("No output file, using stdout.");
+      outStream = System.out;
+    } else {
+      outStream = new FileOutputStream(outfile);
+    }
+
+    try (PrintWriter writer =
+        new PrintWriter(new BufferedWriter(new OutputStreamWriter(outStream, UTF_8)))) {
+
+      writer.println("-----------------------------------------------");
+      writer.println("Report Time: " + tsFormat.format(Instant.now()));
+      writer.println("-----------------------------------------------");
+      if (opts.printInstanceIds) {
+        Map<String,InstanceId> instanceMap = readInstancesFromZk(zooReader);
+        printInstanceIds(instanceMap, writer);
+      }
+
+      if (opts.printIdMap) {
+        printIdMapping(iid, zooReader, writer);
+      }
+
+      if (opts.printProps) {
+        printProps(iid, zooReader, opts, writer);
+      }
+
+      writer.println("-----------------------------------------------");
+    }
+  }
+
+  /**
+   * Get the instanceID from the command line options, or from value stored in HDFS. The search
+   * order is:
+   * <ol>
+   * <li>command line: --instanceId option</li>
+   * <li>command line: --instanceName option</li>
+   * <li>HDFS</li>
+   * </ol>
+   *
+   * @param zooReader
+   *          a ZooReader
+   * @param opts
+   *          the parsed command line options.
+   * @return an instance id
+   */
+  InstanceId getInstanceId(final ZooReader zooReader, final ZooInfoViewer.Opts opts) {
+
+    if (!opts.instanceId.isEmpty()) {
+      return InstanceId.of(opts.instanceId);
+    }
+    if (!opts.instanceName.isEmpty()) {
+      Map<String,InstanceId> instanceNameToIdMap = readInstancesFromZk(zooReader);
+      String instanceName = opts.instanceName;
+      for (Map.Entry<String,InstanceId> e : instanceNameToIdMap.entrySet()) {
+        if (e.getKey().equals(instanceName)) {
+          return e.getValue();
+        }
+      }
+      throw new IllegalArgumentException(
+          "Specified instance name '" + instanceName + "' not found in ZooKeeper");
+    }
+
+    try (ServerContext context = new ServerContext(SiteConfiguration.auto())) {
+      return context.getInstanceID();
+    } catch (Exception ex) {
+      throw new IllegalArgumentException(
+          "Failed to read instance id from HDFS. Instances can be specified on the command line",
+          ex);
+    }
+  }
+
+  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) {
+
+    if (opts.printAllProps()) {
+      log.info("all: {}", opts.printAllProps());
+    } else {
+      log.info("Filters:");
+      log.info("system: {}", opts.printSysProps());
+      log.info("namespaces: {} {}", opts.printNamespaceProps(),
+          opts.getNamespaces().size() > 0 ? opts.getNamespaces() : "");
+      log.info("tables: {} {}", opts.printTableProps(),
+          opts.getTables().size() > 0 ? opts.getTables() : "");

Review Comment:
   The default log configuration will print these to the console. These log messages will be interspersed with the non-log STDOUT. By default, though, they will go to STDERR. That may be what you want (so you can pipe them to another command without the extra info)... but if you intend them to always be included in the output, they should use the same console writer as the printing of the values.
   
   The other thing here is that you have multiple log statements in a row. This seems a bit unnecessary... like you're using logging for a non-logging purpose.



##########
server/base/src/test/java/org/apache/accumulo/server/conf/util/ZooInfoViewerTest.java:
##########
@@ -0,0 +1,437 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.server.conf.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.easymock.EasyMock.anyObject;
+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 static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.io.BufferedReader;
+import java.io.FileReader;
+import java.time.Instant;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+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.fate.zookeeper.ZooReader;
+import org.apache.accumulo.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.server.conf.codec.VersionedPropCodec;
+import org.apache.accumulo.server.conf.codec.VersionedProperties;
+import org.apache.accumulo.server.conf.store.NamespacePropKey;
+import org.apache.accumulo.server.conf.store.SystemPropKey;
+import org.apache.accumulo.server.conf.store.TablePropKey;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.Stat;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+
+public class ZooInfoViewerTest {
+
+  private final Logger log = LoggerFactory.getLogger(ZooInfoViewerTest.class);
+
+  private final VersionedPropCodec propCodec = VersionedPropCodec.getDefault();
+
+  @Test
+  public void simpleOutput() {
+    // StringWriter writer = new StringWriter();
+  }
+
+  @Test
+  public void optionsAllDefault() {
+    ZooInfoViewer.Opts opts = new ZooInfoViewer.Opts();
+    assertTrue(opts.printAllProps());
+    assertTrue(opts.printSysProps());
+    assertTrue(opts.printNamespaceProps());
+    assertTrue(opts.printTableProps());
+  }
+
+  @Test
+  public void onlySys() {
+    ZooInfoViewer.Opts opts = new ZooInfoViewer.Opts();
+    opts.parseArgs(ZooInfoViewer.class.getName(), new String[] {"--system"});
+
+    assertFalse(opts.printAllProps());
+    assertTrue(opts.printSysProps());
+    assertFalse(opts.printNamespaceProps());
+    assertFalse(opts.printTableProps());
+  }
+
+  @Test
+  public void onlyNamespaces() {
+    ZooInfoViewer.Opts opts = new ZooInfoViewer.Opts();
+    opts.parseArgs(ZooInfoViewer.class.getName(), new String[] {"-ns", "ns1", "ns2"});
+
+    assertFalse(opts.printAllProps());
+    assertFalse(opts.printSysProps());
+    assertTrue(opts.printNamespaceProps());
+    assertEquals(2, opts.getNamespaces().size());
+    assertFalse(opts.printTableProps());
+    assertEquals(0, opts.getTables().size());
+  }
+
+  @Test
+  public void allLongOpts() {
+    ZooInfoViewer.Opts opts = new ZooInfoViewer.Opts();
+    opts.parseArgs(ZooInfoViewer.class.getName(),
+        new String[] {"--system", "--namespaces", "ns1", "ns2", "--tables", "tb1", "tbl2"});
+
+    log.debug("namespaces: {}", opts.getNamespaces());
+    log.debug("tables: {}", opts.getTables());
+
+    assertFalse(opts.printAllProps());
+    assertTrue(opts.printSysProps());
+    assertTrue(opts.printNamespaceProps());
+    assertTrue(opts.printTableProps());
+    assertEquals(2, opts.getNamespaces().size());
+    assertEquals(2, opts.getTables().size());
+  }
+
+  @Test
+  public void allOpts() {
+    ZooInfoViewer.Opts opts = new ZooInfoViewer.Opts();
+    opts.parseArgs(ZooInfoViewer.class.getName(), new String[] {"-t", "tb1", "tbl2"});
+
+    assertFalse(opts.printAllProps());
+    assertFalse(opts.printSysProps());
+    assertFalse(opts.printNamespaceProps());
+    assertEquals(0, opts.getNamespaces().size());
+    assertTrue(opts.printTableProps());
+    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(instA, instanceMap.get(instAName));
+    assertEquals(instB, instanceMap.get(instBName));
+    verify(zooReader);
+  }
+
+  /**
+   * Expect that instance id passed is returned, instance name and zooReader are ignored.
+   */
+  @Test
+  public void instanceIdOption() 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.Opts opts = new ZooInfoViewer.Opts();
+    opts.parseArgs(ZooInfoViewer.class.getName(), new String[] {"--instanceName", instBName});
+
+    ZooInfoViewer viewer = new ZooInfoViewer();
+    InstanceId found = viewer.getInstanceId(zooReader, opts);
+
+    assertEquals(instB, found);
+
+    verify(zooReader);
+  }
+
+  /**
+   *
+   */
+  @Test
+  public void instanceNameTest() {
+    String uuid = UUID.randomUUID().toString();
+    ZooReader zooReader = createMock(ZooReader.class);
+    ZooInfoViewer.Opts opts = new ZooInfoViewer.Opts();
+    opts.parseArgs(ZooInfoViewer.class.getName(),
+        new String[] {"--instanceId", uuid, "--instanceName", "foo"});
+    replay(zooReader);
+
+    ZooInfoViewer viewer = new ZooInfoViewer();
+    InstanceId found = viewer.getInstanceId(zooReader, opts);
+
+    assertEquals(InstanceId.of(uuid), found);
+
+    verify(zooReader);
+  }
+
+  @Test
+  public void instanceIdOutputTest() throws Exception {
+    String uuid = UUID.randomUUID().toString();
+
+    ZooReader zooReader = createMock(ZooReader.class);
+    var instanceName = "test";
+    expect(zooReader.getChildren(eq(ZROOT + ZINSTANCES))).andReturn(List.of(instanceName)).once();
+    expect(zooReader.getData(eq(ZROOT + ZINSTANCES + "/" + instanceName)))
+        .andReturn(uuid.getBytes(UTF_8)).once();
+    replay(zooReader);
+
+    String testFileName = "./target/zoo-info-viewer-" + System.currentTimeMillis() + ".txt";
+
+    ZooInfoViewer.Opts opts = new ZooInfoViewer.Opts();
+    opts.parseArgs(ZooInfoViewer.class.getName(),
+        new String[] {"--instanceId", uuid, "--print-instances", "--outfile", testFileName});
+
+    ZooInfoViewer viewer = new ZooInfoViewer();
+    // InstanceId found = viewer.getInstanceId(zooReader, opts);
+    viewer.generateReport(InstanceId.of(uuid), opts, zooReader);
+    // assertEquals(InstanceId.of(uuid), found);
+
+    verify(zooReader);
+
+    String line;
+    try (BufferedReader in = new BufferedReader(new FileReader(testFileName, UTF_8))) {
+      boolean found = false;
+      while ((line = in.readLine()) != null) {
+        if (line.contains("=")) {
+          String trimmed = line.trim();
+          found = trimmed.startsWith(instanceName) && trimmed.endsWith(uuid);
+          break;
+        }
+      }
+      assertTrue(found, "expected instance name, instance id not found");
+
+    }
+  }
+
+  @Test
+  public void instanceNameOutputTest() throws Exception {
+    String uuid = UUID.randomUUID().toString();
+
+    ZooReader zooReader = createMock(ZooReader.class);
+    var instanceName = "test";
+    expect(zooReader.getChildren(eq(ZROOT + ZINSTANCES))).andReturn(List.of(instanceName)).once();
+    expect(zooReader.getData(eq(ZROOT + ZINSTANCES + "/" + instanceName)))
+        .andReturn(uuid.getBytes(UTF_8)).once();
+    replay(zooReader);
+
+    String testFileName = "./target/zoo-info-viewer-" + System.currentTimeMillis() + ".txt";
+
+    ZooInfoViewer.Opts opts = new ZooInfoViewer.Opts();
+    opts.parseArgs(ZooInfoViewer.class.getName(), new String[] {"--instanceName", instanceName,
+        "--print-instances", "--outfile", testFileName});
+
+    ZooInfoViewer viewer = new ZooInfoViewer();
+    // InstanceId found = viewer.getInstanceId(zooReader, opts);
+    viewer.generateReport(InstanceId.of(uuid), opts, zooReader);
+    // assertEquals(InstanceId.of(uuid), found);
+
+    verify(zooReader);
+
+    String line;
+    try (BufferedReader in = new BufferedReader(new FileReader(testFileName, UTF_8))) {
+      boolean found = false;
+      while ((line = in.readLine()) != null) {
+        if (line.contains("=")) {
+          String trimmed = line.trim();
+          found = trimmed.startsWith(instanceName) && trimmed.endsWith(uuid);
+          break;
+        }
+      }
+      assertTrue(found, "expected instance name, instance id not found");
+    }
+  }
+
+  @SuppressFBWarnings(value = "CRLF_INJECTION_LOGS",
+      justification = "test output of generated output")
+  @Test
+  public void propTest() throws Exception {
+    String uuid = UUID.randomUUID().toString();
+    InstanceId iid = InstanceId.of(uuid);
+
+    ZooReader zooReader = createMock(ZooReader.class);
+    var instanceName = "test";
+    expect(zooReader.getChildren(eq(ZROOT + ZINSTANCES))).andReturn(List.of(instanceName))
+        .anyTimes();
+    expect(zooReader.getData(eq(ZROOT + ZINSTANCES + "/" + instanceName)))
+        .andReturn(uuid.getBytes(UTF_8)).anyTimes();
+
+    var sysPropBytes = propCodec
+        .toBytes(new VersionedProperties(123, Instant.now(), Map.of("s1", "sv1", "s2", "sv2")));
+    expect(zooReader.getData(eq(SystemPropKey.of(iid).getNodePath()), anyObject(Watcher.class),
+        anyObject(Stat.class))).andReturn(sysPropBytes).anyTimes();
+
+    var nsBasePath = ZooUtil.getRoot(iid) + ZNAMESPACES;
+    expect(zooReader.getChildren(nsBasePath)).andReturn(List.of("a")).anyTimes();
+    expect(zooReader.getData(eq(nsBasePath + "/a" + ZNAMESPACE_NAME)))
+        .andReturn("a_name".getBytes(UTF_8)).anyTimes();
+    var nsPropBytes =
+        propCodec.toBytes(new VersionedProperties(123, Instant.now(), Map.of("n1", "nv1")));
+    NamespaceId nsId = NamespaceId.of("a");
+    expect(zooReader.getData(eq(NamespacePropKey.of(iid, nsId).getNodePath()),
+        anyObject(Watcher.class), anyObject(Stat.class))).andReturn(nsPropBytes).anyTimes();
+
+    var tBasePath = ZooUtil.getRoot(iid) + ZTABLES;
+    expect(zooReader.getChildren(tBasePath)).andReturn(List.of("t")).anyTimes();
+    expect(zooReader.getData(eq(tBasePath + "/t" + ZTABLE_NAME)))
+        .andReturn("t_table".getBytes(UTF_8)).anyTimes();
+    var tPropBytes =
+        propCodec.toBytes(new VersionedProperties(123, Instant.now(), Map.of("t1", "tv1")));
+    TableId tid = TableId.of("t");
+    expect(zooReader.getData(eq(TablePropKey.of(iid, tid).getNodePath()), anyObject(Watcher.class),
+        anyObject(Stat.class))).andReturn(tPropBytes).anyTimes();
+    expect(zooReader.getData(tBasePath + "/t" + ZTABLE_NAMESPACE))
+        .andReturn("+default".getBytes(UTF_8)).anyTimes();
+
+    replay(zooReader);
+
+    NamespacePropKey nsKey = NamespacePropKey.of(iid, nsId);
+    log.trace("namespace base path: {}", nsKey.getBasePath());
+
+    String testFileName = "./target/zoo-info-viewer-" + System.currentTimeMillis() + ".txt";
+
+    ZooInfoViewer.Opts opts = new ZooInfoViewer.Opts();
+    opts.parseArgs(ZooInfoViewer.class.getName(),
+        new String[] {"--instanceId", uuid, "--print-props", "--outfile", testFileName});
+
+    ZooInfoViewer viewer = new ZooInfoViewer();
+    // InstanceId found = viewer.getInstanceId(zooReader, opts);
+    viewer.generateReport(InstanceId.of(uuid), opts, zooReader);
+    // assertEquals(InstanceId.of(uuid), found);
+
+    verify(zooReader);
+
+    String line;
+    try (BufferedReader in = new BufferedReader(new FileReader(testFileName, UTF_8))) {

Review Comment:
   BufferedReader to read input like this is very old school style. You could use java.util.Scanner, or since the file is very small, just read all the lines with `Files.readAllLines(path)` and loop over that result.



##########
server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooInfoViewer.java:
##########
@@ -0,0 +1,540 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.server.conf.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.io.BufferedWriter;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.time.Instant;
+import java.time.ZoneId;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+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.conf.SiteConfiguration;
+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.fate.zookeeper.ZooReader;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.fate.zookeeper.ZooUtil;
+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.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.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;
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+
+@AutoService(KeywordExecutable.class)
+@SuppressFBWarnings(value = "PATH_TRAVERSAL_OUT",
+    justification = "app is run in same security context as user providing the filename")
+public class ZooInfoViewer implements KeywordExecutable {
+  public static final DateTimeFormatter tsFormat =

Review Comment:
   does this constant need to be public?



##########
server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooInfoViewer.java:
##########
@@ -0,0 +1,540 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.server.conf.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.io.BufferedWriter;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.time.Instant;
+import java.time.ZoneId;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+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.conf.SiteConfiguration;
+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.fate.zookeeper.ZooReader;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.fate.zookeeper.ZooUtil;
+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.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.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;
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+
+@AutoService(KeywordExecutable.class)
+@SuppressFBWarnings(value = "PATH_TRAVERSAL_OUT",
+    justification = "app is run in same security context as user providing the filename")
+public class ZooInfoViewer implements KeywordExecutable {
+  public static final DateTimeFormatter tsFormat =
+      DateTimeFormatter.ISO_OFFSET_DATE_TIME.withZone(ZoneId.from(ZoneOffset.UTC));
+  private static final Logger log = LoggerFactory.getLogger(ZooInfoViewer.class);
+  private final NullWatcher nullWatcher =
+      new NullWatcher(new ReadyMonitor(ZooInfoViewer.class.getSimpleName(), 20_000L));
+
+  private static final String INDENT = "  ";
+
+  /**
+   * No-op constructor - provided so ServiceLoader autoload does not consume resources.
+   */
+  public ZooInfoViewer() {}
+
+  public static void main(String[] args) throws Exception {
+    new ZooInfoViewer().execute(args);
+  }
+
+  @Override
+  public String keyword() {
+    return "zoo-info-viewer";
+  }
+
+  @Override
+  public String description() {
+    return "view Accumulo instance and property information stored in ZooKeeper";
+  }
+
+  @Override
+  public void execute(String[] args) {
+
+    ZooInfoViewer.Opts opts = new ZooInfoViewer.Opts();
+    opts.parseArgs(ZooInfoViewer.class.getName(), args);
+
+    log.info("print ids map: {}", opts.printIdMap);
+    log.info("print properties: {}", opts.printProps);
+    log.info("print instances: {}", opts.printInstanceIds);
+
+    ZooReader zooReader = new ZooReaderWriter(opts.getSiteConfiguration());
+
+    InstanceId iid = getInstanceId(zooReader, opts);
+    try {
+      generateReport(iid, opts, zooReader);
+    } catch (FileNotFoundException ex) {
+      throw new IllegalStateException("Failed to generate ZooKeeper info report", ex);
+    }
+  }
+
+  void generateReport(final InstanceId iid, final ZooInfoViewer.Opts opts,
+      final ZooReader zooReader) throws FileNotFoundException {
+
+    OutputStream outStream;
+
+    String outfile = opts.getOutfile();
+    if (outfile == null || outfile.isEmpty()) {
+      log.trace("No output file, using stdout.");
+      outStream = System.out;
+    } else {
+      outStream = new FileOutputStream(outfile);
+    }
+
+    try (PrintWriter writer =
+        new PrintWriter(new BufferedWriter(new OutputStreamWriter(outStream, UTF_8)))) {
+
+      writer.println("-----------------------------------------------");
+      writer.println("Report Time: " + tsFormat.format(Instant.now()));
+      writer.println("-----------------------------------------------");
+      if (opts.printInstanceIds) {
+        Map<String,InstanceId> instanceMap = readInstancesFromZk(zooReader);
+        printInstanceIds(instanceMap, writer);
+      }
+
+      if (opts.printIdMap) {
+        printIdMapping(iid, zooReader, writer);
+      }
+
+      if (opts.printProps) {
+        printProps(iid, zooReader, opts, writer);
+      }
+
+      writer.println("-----------------------------------------------");
+    }
+  }
+
+  /**
+   * Get the instanceID from the command line options, or from value stored in HDFS. The search
+   * order is:
+   * <ol>
+   * <li>command line: --instanceId option</li>
+   * <li>command line: --instanceName option</li>
+   * <li>HDFS</li>
+   * </ol>
+   *
+   * @param zooReader
+   *          a ZooReader
+   * @param opts
+   *          the parsed command line options.
+   * @return an instance id
+   */
+  InstanceId getInstanceId(final ZooReader zooReader, final ZooInfoViewer.Opts opts) {
+
+    if (!opts.instanceId.isEmpty()) {
+      return InstanceId.of(opts.instanceId);
+    }
+    if (!opts.instanceName.isEmpty()) {
+      Map<String,InstanceId> instanceNameToIdMap = readInstancesFromZk(zooReader);
+      String instanceName = opts.instanceName;
+      for (Map.Entry<String,InstanceId> e : instanceNameToIdMap.entrySet()) {
+        if (e.getKey().equals(instanceName)) {
+          return e.getValue();
+        }
+      }
+      throw new IllegalArgumentException(
+          "Specified instance name '" + instanceName + "' not found in ZooKeeper");
+    }
+
+    try (ServerContext context = new ServerContext(SiteConfiguration.auto())) {
+      return context.getInstanceID();
+    } catch (Exception ex) {
+      throw new IllegalArgumentException(
+          "Failed to read instance id from HDFS. Instances can be specified on the command line",
+          ex);
+    }
+  }
+
+  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) {
+
+    if (opts.printAllProps()) {
+      log.info("all: {}", opts.printAllProps());
+    } else {
+      log.info("Filters:");
+      log.info("system: {}", opts.printSysProps());
+      log.info("namespaces: {} {}", opts.printNamespaceProps(),
+          opts.getNamespaces().size() > 0 ? opts.getNamespaces() : "");
+      log.info("tables: {} {}", opts.printTableProps(),
+          opts.getTables().size() > 0 ? opts.getTables() : "");
+    }
+
+    writer.printf("ZooKeeper properties for instance ID: %s\n\n", iid.canonical());
+    if (opts.printSysProps()) {
+      printSortedProps(writer, Map.of("System", fetchSystemProp(iid, zooReader)));
+    }
+
+    if (opts.printNamespaceProps()) {
+      Map<NamespaceId,String> id2NamespaceMap = getNamespaceIdToNameMap(iid, zooReader);
+
+      Map<String,VersionedProperties> nsProps =
+          fetchNamespaceProps(iid, zooReader, id2NamespaceMap, opts.getNamespaces());
+
+      writer.println("Namespace: ");
+      printSortedProps(writer, nsProps);
+      writer.flush();
+    }
+
+    if (opts.printTableProps()) {
+      Map<String,VersionedProperties> tProps = fetchTableProps(iid, opts.getTables(), zooReader);
+      writer.println("Tables: ");
+      printSortedProps(writer, tProps);
+    }
+    writer.println();
+  }
+
+  private void printIdMapping(InstanceId iid, ZooReader zooReader, PrintWriter writer) {
+    // namespaces
+    Map<NamespaceId,String> id2NamespaceMap = getNamespaceIdToNameMap(iid, zooReader);
+    writer.println("ID Mapping (id => name) for instance: " + iid);
+    writer.println("Namespace ids:");
+    for (Map.Entry<NamespaceId,String> e : id2NamespaceMap.entrySet()) {
+      String v = e.getValue().isEmpty() ? "\"\"" : e.getValue();
+      writer.printf("%s%-9s => %24s\n", INDENT, e.getKey(), v);
+    }
+    writer.println();
+    // tables
+    Map<TableId,String> id2TableMap = getTableIdToName(iid, id2NamespaceMap, zooReader);
+    writer.println("Table ids:");
+    for (Map.Entry<TableId,String> e : id2TableMap.entrySet()) {
+      writer.printf("%s%-9s => %24s\n", INDENT, e.getKey(), e.getValue());
+    }
+    writer.println();
+  }
+
+  /**
+   * 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));
+    writer.println();
+  }
+
+  private Map<String,VersionedProperties> fetchNamespaceProps(InstanceId iid, ZooReader zooReader,
+      Map<NamespaceId,String> id2NamespaceMap, List<String> namespaces) {
+
+    Set<String> cmdOptNamespaces = new TreeSet<>(namespaces);
+
+    Map<NamespaceId,String> filteredIds;
+    if (cmdOptNamespaces.isEmpty()) {
+      filteredIds = id2NamespaceMap;
+    } else {
+      filteredIds =
+          id2NamespaceMap.entrySet().stream().filter(e -> cmdOptNamespaces.contains(e.getValue()))
+              .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+    }
+    log.trace("ns filter: {}", filteredIds);
+    Map<String,VersionedProperties> results = new TreeMap<>();
+
+    filteredIds.forEach((nid, name) -> {
+      try {
+        var key = NamespacePropKey.of(iid, nid);
+        log.trace("fetch props from path: {}", key.getNodePath());
+        var props = ZooPropStore.readFromZk(key, nullWatcher, zooReader);
+        results.put(name, props);
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        throw new IllegalStateException("Interrupted reading table properties from ZooKeeper", ex);
+      } catch (IOException | KeeperException ex) {
+        throw new IllegalStateException("Failed to read table properties from ZooKeeper", ex);
+      }
+    });
+
+    return results;
+  }
+
+  private Map<String,VersionedProperties> fetchTableProps(final InstanceId iid,
+      final List<String> tables, final ZooReader zooReader) {
+
+    Set<String> cmdOptTables = new TreeSet<>(tables);
+
+    Map<NamespaceId,String> id2NamespaceMap = getNamespaceIdToNameMap(iid, zooReader);
+    Map<TableId,String> allIds = getTableIdToName(iid, id2NamespaceMap, zooReader);
+
+    Map<TableId,String> filteredIds;
+    if (cmdOptTables.isEmpty()) {
+      filteredIds = allIds;
+    } else {
+      filteredIds = allIds.entrySet().stream().filter(e -> cmdOptTables.contains(e.getValue()))
+          .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+    }
+
+    log.trace("Looking for: {}", filteredIds);
+
+    Map<String,VersionedProperties> results = new TreeMap<>();
+
+    filteredIds.forEach((tid, name) -> {
+      try {
+        var key = TablePropKey.of(iid, tid);
+        log.trace("fetch props from path: {}", key.getNodePath());
+        var props = ZooPropStore.readFromZk(key, nullWatcher, zooReader);
+        results.put(name, props);
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        throw new IllegalStateException("Interrupted reading table properties from ZooKeeper", ex);
+      } catch (IOException | KeeperException ex) {
+        throw new IllegalStateException("Failed to read table properties from ZooKeeper", ex);
+      }
+    });
+
+    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);
+    props.forEach((n, p) -> {
+      writer.printf("Name: %s, Data Version:%s, Data Timestamp: %s:\n", n, p.getDataVersion(),
+          tsFormat.format(p.getTimestamp()));
+      Map<String,String> pMap = p.asMap();
+      if (pMap.isEmpty()) {
+        writer.println("-- none --");
+      } else {
+        TreeMap<String,String> sorted = new TreeMap<>(pMap);
+        sorted.forEach((name, value) -> writer.printf("%s%s=%s\n", INDENT, name, value));
+      }
+      writer.println();
+    });
+  }
+
+  private VersionedProperties fetchSystemProp(final InstanceId iid, final ZooReader zooReader) {
+
+    try {
+      SystemPropKey propKey = SystemPropKey.of(iid);
+      return ZooPropStore.readFromZk(propKey, nullWatcher, zooReader);
+    } catch (IOException | KeeperException | InterruptedException ex) {
+      throw new IllegalStateException("Failed to read system properties from ZooKeeper", ex);
+    }
+  }
+
+  static class Opts extends ConfigOpts {
+    @Parameter(names = {"--outfile"},
+        description = "Write the output to a file, if the file exists will not be overwritten.")
+    public String outfile = "";
+
+    @Parameter(names = {"--print-id-map"},
+        description = "print the namespace and table id, name mappings stored in ZooKeeper")
+    public boolean printIdMap = false;
+
+    @Parameter(names = {"--print-props"},
+        description = "print the property values stored in ZooKeeper, can be filtered with --namespaces and --tables options")

Review Comment:
   Should mention `--system` is also a valid flag to combine with this



##########
server/base/src/main/java/org/apache/accumulo/server/conf/util/ZooInfoViewer.java:
##########
@@ -0,0 +1,540 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.server.conf.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.io.BufferedWriter;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.time.Instant;
+import java.time.ZoneId;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+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.conf.SiteConfiguration;
+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.fate.zookeeper.ZooReader;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.fate.zookeeper.ZooUtil;
+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.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.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;
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+
+@AutoService(KeywordExecutable.class)
+@SuppressFBWarnings(value = "PATH_TRAVERSAL_OUT",
+    justification = "app is run in same security context as user providing the filename")
+public class ZooInfoViewer implements KeywordExecutable {
+  public static final DateTimeFormatter tsFormat =
+      DateTimeFormatter.ISO_OFFSET_DATE_TIME.withZone(ZoneId.from(ZoneOffset.UTC));
+  private static final Logger log = LoggerFactory.getLogger(ZooInfoViewer.class);
+  private final NullWatcher nullWatcher =
+      new NullWatcher(new ReadyMonitor(ZooInfoViewer.class.getSimpleName(), 20_000L));
+
+  private static final String INDENT = "  ";
+
+  /**
+   * No-op constructor - provided so ServiceLoader autoload does not consume resources.
+   */
+  public ZooInfoViewer() {}
+
+  public static void main(String[] args) throws Exception {
+    new ZooInfoViewer().execute(args);
+  }
+
+  @Override
+  public String keyword() {
+    return "zoo-info-viewer";
+  }
+
+  @Override
+  public String description() {
+    return "view Accumulo instance and property information stored in ZooKeeper";
+  }
+
+  @Override
+  public void execute(String[] args) {
+
+    ZooInfoViewer.Opts opts = new ZooInfoViewer.Opts();
+    opts.parseArgs(ZooInfoViewer.class.getName(), args);
+
+    log.info("print ids map: {}", opts.printIdMap);
+    log.info("print properties: {}", opts.printProps);
+    log.info("print instances: {}", opts.printInstanceIds);
+
+    ZooReader zooReader = new ZooReaderWriter(opts.getSiteConfiguration());
+
+    InstanceId iid = getInstanceId(zooReader, opts);
+    try {
+      generateReport(iid, opts, zooReader);
+    } catch (FileNotFoundException ex) {
+      throw new IllegalStateException("Failed to generate ZooKeeper info report", ex);
+    }
+  }
+
+  void generateReport(final InstanceId iid, final ZooInfoViewer.Opts opts,
+      final ZooReader zooReader) throws FileNotFoundException {
+
+    OutputStream outStream;
+
+    String outfile = opts.getOutfile();
+    if (outfile == null || outfile.isEmpty()) {
+      log.trace("No output file, using stdout.");
+      outStream = System.out;
+    } else {
+      outStream = new FileOutputStream(outfile);
+    }
+
+    try (PrintWriter writer =
+        new PrintWriter(new BufferedWriter(new OutputStreamWriter(outStream, UTF_8)))) {
+
+      writer.println("-----------------------------------------------");
+      writer.println("Report Time: " + tsFormat.format(Instant.now()));
+      writer.println("-----------------------------------------------");
+      if (opts.printInstanceIds) {
+        Map<String,InstanceId> instanceMap = readInstancesFromZk(zooReader);
+        printInstanceIds(instanceMap, writer);
+      }
+
+      if (opts.printIdMap) {
+        printIdMapping(iid, zooReader, writer);
+      }
+
+      if (opts.printProps) {
+        printProps(iid, zooReader, opts, writer);
+      }
+
+      writer.println("-----------------------------------------------");
+    }
+  }
+
+  /**
+   * Get the instanceID from the command line options, or from value stored in HDFS. The search
+   * order is:
+   * <ol>
+   * <li>command line: --instanceId option</li>
+   * <li>command line: --instanceName option</li>
+   * <li>HDFS</li>
+   * </ol>
+   *
+   * @param zooReader
+   *          a ZooReader
+   * @param opts
+   *          the parsed command line options.
+   * @return an instance id
+   */
+  InstanceId getInstanceId(final ZooReader zooReader, final ZooInfoViewer.Opts opts) {
+
+    if (!opts.instanceId.isEmpty()) {
+      return InstanceId.of(opts.instanceId);
+    }
+    if (!opts.instanceName.isEmpty()) {
+      Map<String,InstanceId> instanceNameToIdMap = readInstancesFromZk(zooReader);
+      String instanceName = opts.instanceName;
+      for (Map.Entry<String,InstanceId> e : instanceNameToIdMap.entrySet()) {
+        if (e.getKey().equals(instanceName)) {
+          return e.getValue();
+        }
+      }
+      throw new IllegalArgumentException(
+          "Specified instance name '" + instanceName + "' not found in ZooKeeper");
+    }
+
+    try (ServerContext context = new ServerContext(SiteConfiguration.auto())) {
+      return context.getInstanceID();
+    } catch (Exception ex) {
+      throw new IllegalArgumentException(
+          "Failed to read instance id from HDFS. Instances can be specified on the command line",
+          ex);
+    }
+  }
+
+  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) {
+
+    if (opts.printAllProps()) {
+      log.info("all: {}", opts.printAllProps());
+    } else {
+      log.info("Filters:");
+      log.info("system: {}", opts.printSysProps());
+      log.info("namespaces: {} {}", opts.printNamespaceProps(),
+          opts.getNamespaces().size() > 0 ? opts.getNamespaces() : "");
+      log.info("tables: {} {}", opts.printTableProps(),
+          opts.getTables().size() > 0 ? opts.getTables() : "");
+    }
+
+    writer.printf("ZooKeeper properties for instance ID: %s\n\n", iid.canonical());
+    if (opts.printSysProps()) {
+      printSortedProps(writer, Map.of("System", fetchSystemProp(iid, zooReader)));
+    }
+
+    if (opts.printNamespaceProps()) {
+      Map<NamespaceId,String> id2NamespaceMap = getNamespaceIdToNameMap(iid, zooReader);
+
+      Map<String,VersionedProperties> nsProps =
+          fetchNamespaceProps(iid, zooReader, id2NamespaceMap, opts.getNamespaces());
+
+      writer.println("Namespace: ");
+      printSortedProps(writer, nsProps);
+      writer.flush();
+    }
+
+    if (opts.printTableProps()) {
+      Map<String,VersionedProperties> tProps = fetchTableProps(iid, opts.getTables(), zooReader);
+      writer.println("Tables: ");
+      printSortedProps(writer, tProps);
+    }
+    writer.println();
+  }
+
+  private void printIdMapping(InstanceId iid, ZooReader zooReader, PrintWriter writer) {
+    // namespaces
+    Map<NamespaceId,String> id2NamespaceMap = getNamespaceIdToNameMap(iid, zooReader);
+    writer.println("ID Mapping (id => name) for instance: " + iid);
+    writer.println("Namespace ids:");
+    for (Map.Entry<NamespaceId,String> e : id2NamespaceMap.entrySet()) {
+      String v = e.getValue().isEmpty() ? "\"\"" : e.getValue();
+      writer.printf("%s%-9s => %24s\n", INDENT, e.getKey(), v);
+    }
+    writer.println();
+    // tables
+    Map<TableId,String> id2TableMap = getTableIdToName(iid, id2NamespaceMap, zooReader);
+    writer.println("Table ids:");
+    for (Map.Entry<TableId,String> e : id2TableMap.entrySet()) {
+      writer.printf("%s%-9s => %24s\n", INDENT, e.getKey(), e.getValue());
+    }
+    writer.println();
+  }
+
+  /**
+   * 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));
+    writer.println();
+  }
+
+  private Map<String,VersionedProperties> fetchNamespaceProps(InstanceId iid, ZooReader zooReader,
+      Map<NamespaceId,String> id2NamespaceMap, List<String> namespaces) {
+
+    Set<String> cmdOptNamespaces = new TreeSet<>(namespaces);
+
+    Map<NamespaceId,String> filteredIds;
+    if (cmdOptNamespaces.isEmpty()) {
+      filteredIds = id2NamespaceMap;
+    } else {
+      filteredIds =
+          id2NamespaceMap.entrySet().stream().filter(e -> cmdOptNamespaces.contains(e.getValue()))
+              .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+    }
+    log.trace("ns filter: {}", filteredIds);
+    Map<String,VersionedProperties> results = new TreeMap<>();
+
+    filteredIds.forEach((nid, name) -> {
+      try {
+        var key = NamespacePropKey.of(iid, nid);
+        log.trace("fetch props from path: {}", key.getNodePath());
+        var props = ZooPropStore.readFromZk(key, nullWatcher, zooReader);
+        results.put(name, props);
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        throw new IllegalStateException("Interrupted reading table properties from ZooKeeper", ex);
+      } catch (IOException | KeeperException ex) {
+        throw new IllegalStateException("Failed to read table properties from ZooKeeper", ex);
+      }
+    });
+
+    return results;
+  }
+
+  private Map<String,VersionedProperties> fetchTableProps(final InstanceId iid,
+      final List<String> tables, final ZooReader zooReader) {
+
+    Set<String> cmdOptTables = new TreeSet<>(tables);
+
+    Map<NamespaceId,String> id2NamespaceMap = getNamespaceIdToNameMap(iid, zooReader);
+    Map<TableId,String> allIds = getTableIdToName(iid, id2NamespaceMap, zooReader);
+
+    Map<TableId,String> filteredIds;
+    if (cmdOptTables.isEmpty()) {
+      filteredIds = allIds;
+    } else {
+      filteredIds = allIds.entrySet().stream().filter(e -> cmdOptTables.contains(e.getValue()))
+          .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+    }
+
+    log.trace("Looking for: {}", filteredIds);
+
+    Map<String,VersionedProperties> results = new TreeMap<>();
+
+    filteredIds.forEach((tid, name) -> {
+      try {
+        var key = TablePropKey.of(iid, tid);
+        log.trace("fetch props from path: {}", key.getNodePath());
+        var props = ZooPropStore.readFromZk(key, nullWatcher, zooReader);
+        results.put(name, props);
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        throw new IllegalStateException("Interrupted reading table properties from ZooKeeper", ex);
+      } catch (IOException | KeeperException ex) {
+        throw new IllegalStateException("Failed to read table properties from ZooKeeper", ex);
+      }
+    });
+
+    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);
+    props.forEach((n, p) -> {
+      writer.printf("Name: %s, Data Version:%s, Data Timestamp: %s:\n", n, p.getDataVersion(),
+          tsFormat.format(p.getTimestamp()));
+      Map<String,String> pMap = p.asMap();
+      if (pMap.isEmpty()) {
+        writer.println("-- none --");
+      } else {
+        TreeMap<String,String> sorted = new TreeMap<>(pMap);
+        sorted.forEach((name, value) -> writer.printf("%s%s=%s\n", INDENT, name, value));
+      }
+      writer.println();
+    });
+  }
+
+  private VersionedProperties fetchSystemProp(final InstanceId iid, final ZooReader zooReader) {
+
+    try {
+      SystemPropKey propKey = SystemPropKey.of(iid);
+      return ZooPropStore.readFromZk(propKey, nullWatcher, zooReader);
+    } catch (IOException | KeeperException | InterruptedException ex) {
+      throw new IllegalStateException("Failed to read system properties from ZooKeeper", ex);

Review Comment:
   This is a CLI utility, it's better to just throw it so it reaches through main.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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