Rename master comment/constant/variable references. (#1912)

This change renames reference to master in comments, variable names,
constants, etc. to manager. In some deprecated classes whose purpose is
to maintain compatibility until the next major release, the references
were not renamed.

* Change the constant defining the zookeeper dir where manager lock,
  tick, and goal state info are stored from masters to managers.
* Add a utility to rename the existing masters dir in ZK from masters
  to managers.
* Attempt renaming from SetGoalState, but also allow an admin to invoke
  directly if desired.

fixes #1642
diff --git a/assemble/bin/accumulo-cluster b/assemble/bin/accumulo-cluster
index e5ebffa..4856bfb 100755
--- a/assemble/bin/accumulo-cluster
+++ b/assemble/bin/accumulo-cluster
@@ -75,7 +75,7 @@
   fi
   if [[ ! -f "${conf}/tracers" ]]; then
     if [[ -z "${manager1}" ]] ; then
-      echo "Could not find a master node to use as a default for the tracer role."
+      echo "Could not find a manager node to use as a default for the tracer role."
       echo "Either set up \"${conf}/tracers\" or make sure \"${conf}/$manager_file\" is non-empty."
       exit 1
     else
@@ -254,7 +254,7 @@
   done
 
   echo "Cleaning all server entries in ZooKeeper"
-  ${accumulo_cmd} org.apache.accumulo.server.util.ZooZap -master -tservers -tracers
+  ${accumulo_cmd} org.apache.accumulo.server.util.ZooZap -manager -tservers -tracers
 }
 
 
@@ -262,7 +262,7 @@
   echo "Stopping Accumulo cluster..."
   if ! ${accumulo_cmd} admin stopAll
   then
-    echo "Invalid password or unable to connect to the master"
+    echo "Invalid password or unable to connect to the manager"
     echo "Initiating forced shutdown in 15 seconds (Ctrl-C to abort)"
     sleep 10
     echo "Initiating forced shutdown in  5 seconds (Ctrl-C to abort)"
@@ -294,7 +294,7 @@
   stop_tservers
 
   echo "Cleaning all server entries in ZooKeeper"
-  ${accumulo_cmd} org.apache.accumulo.server.util.ZooZap -master -tservers -tracers
+  ${accumulo_cmd} org.apache.accumulo.server.util.ZooZap -manager -tservers -tracers
 }
 
 function stop_here() {
diff --git a/assemble/conf/templates/hadoop-metrics2-accumulo.properties b/assemble/conf/templates/hadoop-metrics2-accumulo.properties
index 9f2e65c..98ecfd9 100644
--- a/assemble/conf/templates/hadoop-metrics2-accumulo.properties
+++ b/assemble/conf/templates/hadoop-metrics2-accumulo.properties
@@ -40,10 +40,10 @@
 # accumulo.sink.file-tserver.context=tserver
 # accumulo.sink.file-tserver.filename=tserver.metrics
 
-# File sink for master metrics
-# accumulo.sink.file-master.class=org.apache.hadoop.metrics2.sink.FileSink
-# accumulo.sink.file-master.context=master
-# accumulo.sink.file-master.filename=master.metrics
+# File sink for manager metrics
+# accumulo.sink.file-manager.class=org.apache.hadoop.metrics2.sink.FileSink
+# accumulo.sink.file-manager.context=manager
+# accumulo.sink.file-manager.filename=manager.metrics
 
 # File sink for thrift server metrics
 # accumulo.sink.file-thrift.class=org.apache.hadoop.metrics2.sink.FileSink
@@ -88,7 +88,7 @@
 #
 # accumulo.sink.ganglia.tagsForPrefix.jvm=ProcesName
 # accumulo.sink.ganglia.tagsForPrefix.tserver=
-# accumulo.sink.ganglia.tagsForPrefix.master=
+# accumulo.sink.ganglia.tagsForPrefix.manager=
 # accumulo.sink.ganglia.tagsForPrefix.thrift=
 
 # Ganglia host(s)
diff --git a/core/src/main/java/org/apache/accumulo/core/Constants.java b/core/src/main/java/org/apache/accumulo/core/Constants.java
index d30e9b4..96b8b43 100644
--- a/core/src/main/java/org/apache/accumulo/core/Constants.java
+++ b/core/src/main/java/org/apache/accumulo/core/Constants.java
@@ -42,11 +42,11 @@
   public static final String ZNAMESPACE_NAME = "/name";
   public static final String ZNAMESPACE_CONF = "/conf";
 
-  public static final String ZMASTERS = "/masters";
-  public static final String ZMASTER_LOCK = ZMASTERS + "/lock";
-  public static final String ZMASTER_GOAL_STATE = ZMASTERS + "/goal_state";
-  public static final String ZMASTER_REPLICATION_COORDINATOR_ADDR = ZMASTERS + "/repl_coord_addr";
-  public static final String ZMASTER_TICK = ZMASTERS + "/tick";
+  public static final String ZMANAGERS = "/managers";
+  public static final String ZMANAGER_LOCK = ZMANAGERS + "/lock";
+  public static final String ZMANAGER_GOAL_STATE = ZMANAGERS + "/goal_state";
+  public static final String ZMANAGER_REPLICATION_COORDINATOR_ADDR = ZMANAGERS + "/repl_coord_addr";
+  public static final String ZMANAGER_TICK = ZMANAGERS + "/tick";
 
   public static final String ZGC = "/gc";
   public static final String ZGC_LOCK = ZGC + "/lock";
diff --git a/core/src/main/java/org/apache/accumulo/core/client/Instance.java b/core/src/main/java/org/apache/accumulo/core/client/Instance.java
index fe8c0df..8a60068 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/Instance.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/Instance.java
@@ -41,7 +41,7 @@
   String getRootTabletLocation();
 
   /**
-   * Returns the location(s) of the accumulo master and any redundant servers.
+   * Returns the location(s) of the accumulo manager and any redundant servers.
    *
    * @return a list of locations in "hostname:port" form
    */
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
index bcf0358..16b3ad2 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
@@ -135,7 +135,7 @@
 
   @Override
   public List<String> getMasterLocations() {
-    return ClientContext.getMasterLocations(zooCache, getInstanceID());
+    return ClientContext.getManagerLocations(zooCache, getInstanceID());
   }
 
   @Override
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
index cf1933b..f20ec82 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
@@ -168,7 +168,7 @@
 
       @Override
       public List<String> getMasterLocations() {
-        return context.getMasterLocations();
+        return context.getManagerLocations();
       }
 
       @Override
@@ -355,31 +355,31 @@
   }
 
   /**
-   * Returns the location(s) of the accumulo master and any redundant servers.
+   * Returns the location(s) of the accumulo manager and any redundant servers.
    *
    * @return a list of locations in "hostname:port" form
    */
-  public List<String> getMasterLocations() {
+  public List<String> getManagerLocations() {
     ensureOpen();
-    return getMasterLocations(zooCache, getInstanceID());
+    return getManagerLocations(zooCache, getInstanceID());
   }
 
   // available only for sharing code with old ZooKeeperInstance
-  public static List<String> getMasterLocations(ZooCache zooCache, String instanceId) {
-    String masterLocPath = ZooUtil.getRoot(instanceId) + Constants.ZMASTER_LOCK;
+  public static List<String> getManagerLocations(ZooCache zooCache, String instanceId) {
+    String managerLocPath = ZooUtil.getRoot(instanceId) + Constants.ZMANAGER_LOCK;
 
     OpTimer timer = null;
 
     if (log.isTraceEnabled()) {
-      log.trace("tid={} Looking up master location in zookeeper.", Thread.currentThread().getId());
+      log.trace("tid={} Looking up manager location in zookeeper.", Thread.currentThread().getId());
       timer = new OpTimer().start();
     }
 
-    byte[] loc = zooCache.getLockData(masterLocPath);
+    byte[] loc = zooCache.getLockData(managerLocPath);
 
     if (timer != null) {
       timer.stop();
-      log.trace("tid={} Found master at {} in {}", Thread.currentThread().getId(),
+      log.trace("tid={} Found manager at {} in {}", Thread.currentThread().getId(),
           (loc == null ? "null" : new String(loc, UTF_8)),
           String.format("%.3f secs", timer.scale(TimeUnit.SECONDS)));
     }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java
index 8dea8ff..1ec0005 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java
@@ -125,7 +125,7 @@
 
   @Override
   public List<String> getManagerLocations() {
-    return context.getMasterLocations();
+    return context.getManagerLocations();
   }
 
   @Override
@@ -139,7 +139,7 @@
         var copy = new ArrayList<>(children);
         Collections.sort(copy);
         var data = cache.get(path + "/" + candidate + "/" + copy.get(0));
-        if (data != null && !"master".equals(new String(data, UTF_8))) {
+        if (data != null && !"manager".equals(new String(data, UTF_8))) {
           results.add(candidate);
         }
       }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ManagerClient.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ManagerClient.java
index 2c9c24f..ce356f6 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ManagerClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ManagerClient.java
@@ -56,20 +56,20 @@
   public static ManagerClientService.Client getConnection(ClientContext context) {
     checkArgument(context != null, "context is null");
 
-    List<String> locations = context.getMasterLocations();
+    List<String> locations = context.getManagerLocations();
 
     if (locations.isEmpty()) {
-      log.debug("No masters...");
+      log.debug("No managers...");
       return null;
     }
 
-    HostAndPort master = HostAndPort.fromString(locations.get(0));
-    if (master.getPort() == 0)
+    HostAndPort manager = HostAndPort.fromString(locations.get(0));
+    if (manager.getPort() == 0)
       return null;
 
     try {
-      // Master requests can take a long time: don't ever time out
-      return ThriftUtil.getClientNoTimeout(new ManagerClientService.Client.Factory(), master,
+      // Manager requests can take a long time: don't ever time out
+      return ThriftUtil.getClientNoTimeout(new ManagerClientService.Client.Factory(), manager,
           context);
     } catch (TTransportException tte) {
       Throwable cause = tte.getCause();
@@ -77,7 +77,7 @@
         // do not expect to recover from this
         throw new RuntimeException(tte);
       }
-      log.debug("Failed to connect to master=" + master + ", will retry... ", tte);
+      log.debug("Failed to connect to manager=" + manager + ", will retry... ", tte);
       return null;
     }
   }
@@ -88,7 +88,7 @@
         && client.getInputProtocol().getTransport() != null) {
       ThriftTransportPool.getInstance().returnTransport(client.getInputProtocol().getTransport());
     } else {
-      log.debug("Attempt to close null connection to the master", new Exception());
+      log.debug("Attempt to close null connection to the manager", new Exception());
     }
   }
 
@@ -118,7 +118,7 @@
         }
       } catch (ThriftNotActiveServiceException e) {
         // Let it loop, fetching a new location
-        log.debug("Contacted a Master which is no longer active, retrying");
+        log.debug("Contacted a Manager which is no longer active, retrying");
         sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
       } catch (Exception e) {
         throw new AccumuloException(e);
@@ -156,8 +156,8 @@
         }
       } catch (ThriftNotActiveServiceException e) {
         // Let it loop, fetching a new location
-        log.debug("Contacted a Master which is no longer active, re-creating"
-            + " the connection to the active Master");
+        log.debug("Contacted a Manager which is no longer active, re-creating"
+            + " the connection to the active Manager");
       } catch (Exception e) {
         throw new AccumuloException(e);
       } finally {
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ReplicationClient.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ReplicationClient.java
index 89466b5..8f50a6c 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ReplicationClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ReplicationClient.java
@@ -65,32 +65,32 @@
     }
 
     throw new AccumuloException(
-        "Timed out trying to communicate with master from " + context.getInstanceName());
+        "Timed out trying to communicate with manager from " + context.getInstanceName());
   }
 
   public static ReplicationCoordinator.Client getCoordinatorConnection(ClientContext context) {
-    List<String> locations = context.getMasterLocations();
+    List<String> locations = context.getManagerLocations();
 
     if (locations.isEmpty()) {
-      log.debug("No masters for replication to instance {}", context.getInstanceName());
+      log.debug("No managers for replication to instance {}", context.getInstanceName());
       return null;
     }
 
-    // This is the master thrift service, we just want the hostname, not the port
-    String masterThriftService = locations.get(0);
-    if (masterThriftService.endsWith(":0")) {
-      log.warn("Master found for {} did not have real location {}", context.getInstanceName(),
-          masterThriftService);
+    // This is the manager thrift service, we just want the hostname, not the port
+    String managerThriftService = locations.get(0);
+    if (managerThriftService.endsWith(":0")) {
+      log.warn("Manager found for {} did not have real location {}", context.getInstanceName(),
+          managerThriftService);
       return null;
     }
 
-    String zkPath = context.getZooKeeperRoot() + Constants.ZMASTER_REPLICATION_COORDINATOR_ADDR;
+    String zkPath = context.getZooKeeperRoot() + Constants.ZMANAGER_REPLICATION_COORDINATOR_ADDR;
     String replCoordinatorAddr;
 
-    log.debug("Using ZooKeeper quorum at {} with path {} to find peer Master information",
+    log.debug("Using ZooKeeper quorum at {} with path {} to find peer Manager information",
         context.getZooKeepers(), zkPath);
 
-    // Get the coordinator port for the master we're trying to connect to
+    // Get the coordinator port for the manager we're trying to connect to
     try {
       ZooReader reader =
           new ZooReader(context.getZooKeepers(), context.getZooKeepersSessionTimeOut());
@@ -103,14 +103,14 @@
     // Throw the hostname and port through HostAndPort to get some normalization
     HostAndPort coordinatorAddr = HostAndPort.fromString(replCoordinatorAddr);
 
-    log.debug("Connecting to master at {}", coordinatorAddr);
+    log.debug("Connecting to manager at {}", coordinatorAddr);
 
     try {
-      // Master requests can take a long time: don't ever time out
+      // Manager requests can take a long time: don't ever time out
       return ThriftUtil.getClientNoTimeout(new ReplicationCoordinator.Client.Factory(),
           coordinatorAddr, context);
     } catch (TTransportException tte) {
-      log.debug("Failed to connect to master coordinator service ({})", coordinatorAddr, tte);
+      log.debug("Failed to connect to manager coordinator service ({})", coordinatorAddr, tte);
       return null;
     }
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ReplicationOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ReplicationOperationsImpl.java
index b2a111d..56851af 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ReplicationOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ReplicationOperationsImpl.java
@@ -95,11 +95,11 @@
     final TInfo tinfo = TraceUtil.traceInfo();
     final TCredentials rpcCreds = context.rpcCreds();
 
-    // Ask the master if the table is fully replicated given these WALs, but don't poll inside the
-    // master
+    // Ask the manager if the table is fully replicated given these WALs, but don't poll inside the
+    // manager
     boolean drained = false;
     while (!drained) {
-      drained = getMasterDrain(tinfo, rpcCreds, tableName, wals);
+      drained = getManagerDrain(tinfo, rpcCreds, tableName, wals);
 
       if (!drained) {
         try {
@@ -112,7 +112,7 @@
     }
   }
 
-  protected boolean getMasterDrain(final TInfo tinfo, final TCredentials rpcCreds,
+  protected boolean getManagerDrain(final TInfo tinfo, final TCredentials rpcCreds,
       final String tableName, final Set<String> wals)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     return ManagerClient.execute(context,
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ServerClient.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ServerClient.java
index 197ebad..470ad7f 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ServerClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ServerClient.java
@@ -147,7 +147,7 @@
       byte[] data = zc.getLockData(path);
       if (data != null) {
         String strData = new String(data, UTF_8);
-        if (!strData.equals("master"))
+        if (!strData.equals("manager"))
           servers.add(new ThriftTransportKey(
               new ServerServices(strData).getAddress(Service.TSERV_CLIENT), rpcTimeout, context));
       }
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
index e81bafd..72b5a1e 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/TableOperationsImpl.java
@@ -261,7 +261,7 @@
         sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
       } catch (ThriftNotActiveServiceException e) {
         // Let it loop, fetching a new location
-        log.debug("Contacted a Master which is no longer active, retrying");
+        log.debug("Contacted a Manager which is no longer active, retrying");
         sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
       } finally {
         ManagerClient.close(client);
@@ -286,7 +286,7 @@
         sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
       } catch (ThriftNotActiveServiceException e) {
         // Let it loop, fetching a new location
-        log.debug("Contacted a Master which is no longer active, retrying");
+        log.debug("Contacted a Manager which is no longer active, retrying");
         sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
       } finally {
         ManagerClient.close(client);
@@ -306,7 +306,7 @@
         sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
       } catch (ThriftNotActiveServiceException e) {
         // Let it loop, fetching a new location
-        log.debug("Contacted a Master which is no longer active, retrying");
+        log.debug("Contacted a Manager which is no longer active, retrying");
         sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
       } finally {
         ManagerClient.close(client);
@@ -326,7 +326,7 @@
         sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
       } catch (ThriftNotActiveServiceException e) {
         // Let it loop, fetching a new location
-        log.debug("Contacted a Master which is no longer active, retrying");
+        log.debug("Contacted a Manager which is no longer active, retrying");
         sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
       } finally {
         ManagerClient.close(client);
@@ -942,7 +942,7 @@
           sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
         } catch (ThriftNotActiveServiceException e) {
           // Let it loop, fetching a new location
-          log.debug("Contacted a Master which is no longer active, retrying");
+          log.debug("Contacted a Manager which is no longer active, retrying");
           sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
         } finally {
           ManagerClient.close(client);
@@ -962,7 +962,7 @@
           sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
         } catch (ThriftNotActiveServiceException e) {
           // Let it loop, fetching a new location
-          log.debug("Contacted a Master which is no longer active, retrying");
+          log.debug("Contacted a Manager which is no longer active, retrying");
           sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
         } finally {
           ManagerClient.close(client);
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationDocGen.java b/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationDocGen.java
index 5fd3ec0..ba60da2 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationDocGen.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/ConfigurationDocGen.java
@@ -70,7 +70,7 @@
         + " that is copied from Accumulo build (from core/target/generated-docs)" + " -->\n");
     doc.println("Below are properties set in `accumulo.properties` or the"
         + " Accumulo shell that configure Accumulo servers (i.e tablet server,"
-        + " master, etc). Properties labeled 'Experimental' could be part of an incomplete"
+        + " manager, etc). Properties labeled 'Experimental' could be part of an incomplete"
         + " feature or have a higher risk of changing in the future.\n");
   }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/logging/TabletLogger.java b/core/src/main/java/org/apache/accumulo/core/logging/TabletLogger.java
index 333b8bf..ab8808a 100644
--- a/core/src/main/java/org/apache/accumulo/core/logging/TabletLogger.java
+++ b/core/src/main/java/org/apache/accumulo/core/logging/TabletLogger.java
@@ -63,7 +63,7 @@
   }
 
   /**
-   * A tablet server has received an assignment message from master and queued the tablet for
+   * A tablet server has received an assignment message from manager and queued the tablet for
    * loading.
    */
   public static void loading(KeyExtent extent, TServerInstance server) {
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/ThriftUtil.java b/core/src/main/java/org/apache/accumulo/core/rpc/ThriftUtil.java
index 5ebeb06..6cb6ef6 100644
--- a/core/src/main/java/org/apache/accumulo/core/rpc/ThriftUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/ThriftUtil.java
@@ -393,7 +393,7 @@
   /**
    * Some wonderful snippets of documentation from HBase on performing the re-login client-side (as
    * well as server-side) in the following paragraph. We want to attempt a re-login to automatically
-   * refresh the client's Krb "credentials" (remember, a server might also be a client, master
+   * refresh the client's Krb "credentials" (remember, a server might also be a client, manager
    * sending RPC to tserver), but we have to take care to avoid Kerberos' replay attack protection.
    * <p>
    * If multiple clients with the same principal try to connect to the same server at the same time,
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancer.java b/core/src/main/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancer.java
index 9038912..46157cb 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancer.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/balancer/HostRegexTableLoadBalancer.java
@@ -76,7 +76,7 @@
  * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets
  * that it should not be according to the regex configuration. If this occurs then the offending
  * tablets will be reassigned. This would cover the case where the configuration is changed and the
- * master is restarted while the tablet servers are up. To change the out of bounds check time
+ * manager is restarted while the tablet servers are up. To change the out of bounds check time
  * period, set the following property:<br>
  * <b>table.custom.balancer.host.regex.oob.period=5m</b><br>
  * Regex matching can be based on either the host name (default) or host ip address. To set this
diff --git a/core/src/main/java/org/apache/accumulo/core/spi/balancer/TabletBalancer.java b/core/src/main/java/org/apache/accumulo/core/spi/balancer/TabletBalancer.java
index 0a4ef69..b120ae7 100644
--- a/core/src/main/java/org/apache/accumulo/core/spi/balancer/TabletBalancer.java
+++ b/core/src/main/java/org/apache/accumulo/core/spi/balancer/TabletBalancer.java
@@ -102,7 +102,7 @@
   void init(BalancerEnvironment balancerEnvironment);
 
   /**
-   * Assign tablets to tablet servers. This method is called whenever the master finds tablets that
+   * Assign tablets to tablet servers. This method is called whenever the manager finds tablets that
    * are unassigned.
    */
   void getAssignments(AssignmentParameters params);
diff --git a/core/src/main/java/org/apache/accumulo/fate/AdminUtil.java b/core/src/main/java/org/apache/accumulo/fate/AdminUtil.java
index fe01d98..9c97598 100644
--- a/core/src/main/java/org/apache/accumulo/fate/AdminUtil.java
+++ b/core/src/main/java/org/apache/accumulo/fate/AdminUtil.java
@@ -515,20 +515,20 @@
   public boolean checkGlobalLock(ZooReaderWriter zk, String path) {
     try {
       if (ZooLock.getLockData(zk.getZooKeeper(), path) != null) {
-        System.err.println("ERROR: Master lock is held, not running");
+        System.err.println("ERROR: Manager lock is held, not running");
         if (this.exitOnError)
           System.exit(1);
         else
           return false;
       }
     } catch (KeeperException e) {
-      System.err.println("ERROR: Could not read master lock, not running " + e.getMessage());
+      System.err.println("ERROR: Could not read manager lock, not running " + e.getMessage());
       if (this.exitOnError)
         System.exit(1);
       else
         return false;
     } catch (InterruptedException e) {
-      System.err.println("ERROR: Could not read master lock, not running" + e.getMessage());
+      System.err.println("ERROR: Could not read manager lock, not running" + e.getMessage());
       if (this.exitOnError)
         System.exit(1);
       else
diff --git a/core/src/main/java/org/apache/accumulo/fate/Fate.java b/core/src/main/java/org/apache/accumulo/fate/Fate.java
index 85bec1e..a13b825 100644
--- a/core/src/main/java/org/apache/accumulo/fate/Fate.java
+++ b/core/src/main/java/org/apache/accumulo/fate/Fate.java
@@ -134,7 +134,7 @@
     /**
      * The Hadoop Filesystem registers a java shutdown hook that closes the file system. This can
      * cause threads to get spurious IOException. If this happens, instead of failing a FATE
-     * transaction just wait for process to die. When the master start elsewhere the FATE
+     * transaction just wait for process to die. When the manager start elsewhere the FATE
      * transaction can resume.
      */
     private void blockIfHadoopShutdown(long tid, Exception e) {
diff --git a/core/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java b/core/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java
index 1c3f984..f5e476e 100644
--- a/core/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java
+++ b/core/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java
@@ -354,7 +354,7 @@
                 determineLockOwnership(createdEphemeralNode, lw);
               }
             } catch (KeeperException | InterruptedException e) {
-              lw.failedToAcquireLock(new Exception("Failed to renew watch on other master node"));
+              lw.failedToAcquireLock(new Exception("Failed to renew watch on other manager node"));
             }
           }
         }
diff --git a/minicluster/minicluster.properties b/minicluster/minicluster.properties
index 5fbad01..4cbb624 100644
--- a/minicluster/minicluster.properties
+++ b/minicluster/minicluster.properties
@@ -25,7 +25,7 @@
 jdwpEnabled=true
 zooKeeperMemory=128M
 tserverMemory=256M
-masterMemory=128M
+managerMemory=128M
 defaultMemory=256M
 shutdownPort=4446
-site.instance.secret=HUSH
\ No newline at end of file
+site.instance.secret=HUSH
diff --git a/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneClusterControl.java b/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneClusterControl.java
index be531e0..f472bfe 100644
--- a/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneClusterControl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneClusterControl.java
@@ -54,7 +54,7 @@
 
   private static final String ACCUMULO_SERVICE_SCRIPT = "accumulo-service",
       ACCUMULO_SCRIPT = "accumulo";
-  private static final String MASTER_HOSTS_FILE = "masters", GC_HOSTS_FILE = "gc",
+  private static final String MANAGER_HOSTS_FILE = "managers", GC_HOSTS_FILE = "gc",
       TSERVER_HOSTS_FILE = "tservers", TRACER_HOSTS_FILE = "tracers",
       MONITOR_HOSTS_FILE = "monitor";
 
@@ -107,7 +107,7 @@
 
   @Override
   public Entry<Integer,String> execWithStdout(Class<?> clz, String[] args) throws IOException {
-    String master = getHosts(MASTER_HOSTS_FILE).get(0);
+    String manager = getHosts(MANAGER_HOSTS_FILE).get(0);
     List<String> cmd = new ArrayList<>();
     cmd.add(clientCmdPrefix);
     cmd.add(accumuloPath);
@@ -116,8 +116,8 @@
     for (String arg : args) {
       cmd.add("'" + arg + "'");
     }
-    log.info("Running: '{}' on {}", sanitize(String.join(" ", cmd)), sanitize(master));
-    return exec(master, cmd.toArray(new String[cmd.size()]));
+    log.info("Running: '{}' on {}", sanitize(String.join(" ", cmd)), sanitize(manager));
+    return exec(manager, cmd.toArray(new String[cmd.size()]));
   }
 
   /**
@@ -130,10 +130,10 @@
 
   @Override
   public void adminStopAll() throws IOException {
-    String master = getHosts(MASTER_HOSTS_FILE).get(0);
+    String manager = getHosts(MANAGER_HOSTS_FILE).get(0);
     String[] cmd = {serverCmdPrefix, accumuloPath, Admin.class.getName(), "stopAll"};
     // Directly invoke the RemoteShell
-    Entry<Integer,String> pair = exec(master, cmd);
+    Entry<Integer,String> pair = exec(manager, cmd);
     if (pair.getKey() != 0) {
       throw new IOException("stopAll did not finish successfully, retcode=" + pair.getKey()
           + ", stdout=" + pair.getValue());
@@ -151,9 +151,9 @@
   public void setGoalState(String goalState) throws IOException {
     requireNonNull(goalState, "Goal state must not be null");
     checkArgument(ManagerGoalState.valueOf(goalState) != null, "Unknown goal state: " + goalState);
-    String master = getHosts(MASTER_HOSTS_FILE).get(0);
+    String manager = getHosts(MANAGER_HOSTS_FILE).get(0);
     String[] cmd = {serverCmdPrefix, accumuloPath, SetGoalState.class.getName(), goalState};
-    Entry<Integer,String> pair = exec(master, cmd);
+    Entry<Integer,String> pair = exec(manager, cmd);
     if (pair.getKey() != 0) {
       throw new IOException("SetGoalState did not finish successfully, retcode=" + pair.getKey()
           + ", stdout=" + pair.getValue());
@@ -171,14 +171,14 @@
         break;
       case MASTER:
       case MANAGER:
-        for (String master : getHosts(MASTER_HOSTS_FILE)) {
-          start(server, master);
+        for (String manager : getHosts(MANAGER_HOSTS_FILE)) {
+          start(server, manager);
         }
         break;
       case GARBAGE_COLLECTOR:
         List<String> hosts = getHosts(GC_HOSTS_FILE);
         if (hosts.isEmpty()) {
-          hosts = getHosts(MASTER_HOSTS_FILE);
+          hosts = getHosts(MANAGER_HOSTS_FILE);
           if (hosts.isEmpty()) {
             throw new IOException("Found hosts to run garbage collector on");
           }
@@ -225,8 +225,8 @@
         break;
       case MASTER:
       case MANAGER:
-        for (String master : getHosts(MASTER_HOSTS_FILE)) {
-          stop(server, master);
+        for (String manager : getHosts(MANAGER_HOSTS_FILE)) {
+          stop(server, manager);
         }
         break;
       case GARBAGE_COLLECTOR:
@@ -331,7 +331,7 @@
         return "gc";
       case MASTER:
       case MANAGER:
-        return "master";
+        return "manager";
       case TRACER:
         return "tracer";
       case MONITOR:
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java
index 1b8ce44..7eccfed 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java
@@ -71,7 +71,9 @@
   private static final String ROOT_PASSWORD_PROP = "rootPassword";
   private static final String SHUTDOWN_PORT_PROP = "shutdownPort";
   private static final String DEFAULT_MEMORY_PROP = "defaultMemory";
+  @Deprecated(since = "2.1.0")
   private static final String MASTER_MEMORY_PROP = "masterMemory";
+  private static final String MANAGER_MEMORY_PROP = "managerMemory";
   private static final String TSERVER_MEMORY_PROP = "tserverMemory";
   private static final String ZOO_KEEPER_MEMORY_PROP = "zooKeeperMemory";
   private static final String JDWP_ENABLED_PROP = "jdwpEnabled";
@@ -96,7 +98,7 @@
     System.out.println("#" + ZOO_KEEPER_STARTUP_TIME_PROP + "=39000");
     System.out.println("#" + SHUTDOWN_PORT_PROP + "=41414");
     System.out.println("#" + DEFAULT_MEMORY_PROP + "=128M");
-    System.out.println("#" + MASTER_MEMORY_PROP + "=128M");
+    System.out.println("#" + MANAGER_MEMORY_PROP + "=128M");
     System.out.println("#" + TSERVER_MEMORY_PROP + "=128M");
     System.out.println("#" + ZOO_KEEPER_MEMORY_PROP + "=128M");
     System.out.println("#" + JDWP_ENABLED_PROP + "=false");
@@ -197,8 +199,12 @@
     if (opts.prop.containsKey(TSERVER_MEMORY_PROP))
       setMemoryOnConfig(config, opts.prop.getProperty(TSERVER_MEMORY_PROP),
           ServerType.TABLET_SERVER);
-    if (opts.prop.containsKey(MASTER_MEMORY_PROP))
+    if (opts.prop.containsKey(MASTER_MEMORY_PROP)) {
+      log.warn("{} is deprecated. Use {} instead.", MASTER_MEMORY_PROP, MANAGER_MEMORY_PROP);
       setMemoryOnConfig(config, opts.prop.getProperty(MASTER_MEMORY_PROP), ServerType.MANAGER);
+    }
+    if (opts.prop.containsKey(MANAGER_MEMORY_PROP))
+      setMemoryOnConfig(config, opts.prop.getProperty(MANAGER_MEMORY_PROP), ServerType.MANAGER);
     if (opts.prop.containsKey(DEFAULT_MEMORY_PROP))
       setMemoryOnConfig(config, opts.prop.getProperty(DEFAULT_MEMORY_PROP));
     if (opts.prop.containsKey(SHUTDOWN_PORT_PROP))
diff --git a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java
index 8e3277f..0e36d51 100644
--- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterControl.java
@@ -53,7 +53,7 @@
   protected MiniAccumuloClusterImpl cluster;
 
   Process zooKeeperProcess = null;
-  Process masterProcess = null;
+  Process managerProcess = null;
   Process gcProcess = null;
   Process monitor = null;
   Process tracer = null;
@@ -143,8 +143,8 @@
         break;
       case MASTER:
       case MANAGER:
-        if (masterProcess == null) {
-          masterProcess = cluster._exec(Manager.class, server, configOverrides).getProcess();
+        if (managerProcess == null) {
+          managerProcess = cluster._exec(Manager.class, server, configOverrides).getProcess();
         }
         break;
       case ZOOKEEPER:
@@ -189,15 +189,15 @@
     switch (server) {
       case MASTER:
       case MANAGER:
-        if (masterProcess != null) {
+        if (managerProcess != null) {
           try {
-            cluster.stopProcessWithTimeout(masterProcess, 30, TimeUnit.SECONDS);
+            cluster.stopProcessWithTimeout(managerProcess, 30, TimeUnit.SECONDS);
           } catch (ExecutionException | TimeoutException e) {
-            log.warn("Master did not fully stop after 30 seconds", e);
+            log.warn("Manager did not fully stop after 30 seconds", e);
           } catch (InterruptedException e) {
             Thread.currentThread().interrupt();
           } finally {
-            masterProcess = null;
+            managerProcess = null;
           }
         }
         break;
@@ -298,13 +298,13 @@
     switch (type) {
       case MASTER:
       case MANAGER:
-        if (procRef.getProcess().equals(masterProcess)) {
+        if (procRef.getProcess().equals(managerProcess)) {
           try {
-            cluster.stopProcessWithTimeout(masterProcess, 30, TimeUnit.SECONDS);
+            cluster.stopProcessWithTimeout(managerProcess, 30, TimeUnit.SECONDS);
           } catch (ExecutionException | TimeoutException e) {
-            log.warn("Master did not fully stop after 30 seconds", e);
+            log.warn("Manager did not fully stop after 30 seconds", e);
           }
-          masterProcess = null;
+          managerProcess = null;
           found = true;
         }
         break;
diff --git a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
index 0c605a6..61dbd5a 100644
--- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
@@ -590,7 +590,7 @@
       sleepUninterruptibly(1, TimeUnit.SECONDS);
     }
     if (ret != 0) {
-      throw new RuntimeException("Could not set master goal state, process returned " + ret
+      throw new RuntimeException("Could not set manager goal state, process returned " + ret
           + ". Check the logs in " + config.getLogDir() + " for errors.");
     }
 
@@ -626,7 +626,7 @@
   public Map<ServerType,Collection<ProcessReference>> getProcesses() {
     Map<ServerType,Collection<ProcessReference>> result = new HashMap<>();
     MiniAccumuloClusterControl control = getClusterControl();
-    result.put(ServerType.MANAGER, references(control.masterProcess));
+    result.put(ServerType.MANAGER, references(control.managerProcess));
     result.put(ServerType.TABLET_SERVER,
         references(control.tabletServerProcesses.toArray(new Process[0])));
     if (control.zooKeeperProcess != null) {
@@ -770,7 +770,7 @@
    *
    * @since 1.6.1
    */
-  public ManagerMonitorInfo getMasterMonitorInfo()
+  public ManagerMonitorInfo getManagerMonitorInfo()
       throws AccumuloException, AccumuloSecurityException {
     ManagerClientService.Iface client = null;
     while (true) {
@@ -781,7 +781,7 @@
         throw new AccumuloSecurityException(exception);
       } catch (ThriftNotActiveServiceException e) {
         // Let it loop, fetching a new location
-        log.debug("Contacted a Master which is no longer active, retrying");
+        log.debug("Contacted a Manager which is no longer active, retrying");
         sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
       } catch (TException exception) {
         throw new AccumuloException(exception);
diff --git a/minicluster/src/test/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImplTest.java b/minicluster/src/test/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImplTest.java
index 9243908..ba170ef 100644
--- a/minicluster/src/test/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImplTest.java
+++ b/minicluster/src/test/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImplTest.java
@@ -105,7 +105,7 @@
   public void saneMonitorInfo() throws Exception {
     ManagerMonitorInfo stats;
     while (true) {
-      stats = accumulo.getMasterMonitorInfo();
+      stats = accumulo.getManagerMonitorInfo();
       if (stats.tableMap.size() <= 2) {
         continue;
       }
@@ -116,8 +116,8 @@
     }
     List<ManagerState> validStates = Arrays.asList(ManagerState.values());
     List<ManagerGoalState> validGoals = Arrays.asList(ManagerGoalState.values());
-    assertTrue("master state should be valid.", validStates.contains(stats.state));
-    assertTrue("master goal state should be in " + validGoals + ". is " + stats.goalState,
+    assertTrue("manager state should be valid.", validStates.contains(stats.state));
+    assertTrue("manager goal state should be in " + validGoals + ". is " + stats.goalState,
         validGoals.contains(stats.goalState));
     assertNotNull("should have a table map.", stats.tableMap);
     assertTrue("root table should exist in " + stats.tableMap.keySet(),
diff --git a/server/base/src/main/java/org/apache/accumulo/server/ServerUtil.java b/server/base/src/main/java/org/apache/accumulo/server/ServerUtil.java
index 00c53c4..3708b71 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/ServerUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/ServerUtil.java
@@ -249,7 +249,7 @@
    * upgrade. The status of the operations is irrelevant; those in SUCCESSFUL status cause the same
    * problem as those just queued.
    *
-   * Note that the Master should not allow write access to Fate until after all upgrade steps are
+   * Note that the Manager should not allow write access to Fate until after all upgrade steps are
    * complete.
    *
    * Should be called as a guard before performing any upgrade steps, after determining that an
diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
index 8ac9f48..ad5a584 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
@@ -129,7 +129,7 @@
   @Override
   public void ping(TCredentials credentials) {
     // anybody can call this; no authentication check
-    log.info("Master reports: I just got pinged!");
+    log.info("Manager reports: I just got pinged!");
   }
 
   @Override
diff --git a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
index bc725d8..226532a 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
@@ -631,11 +631,11 @@
         NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + RootTable.ZROOT_TABLET_GC_CANDIDATES,
         new RootGcCandidates().toJson().getBytes(UTF_8), NodeExistsPolicy.FAIL);
-    zoo.putPersistentData(zkInstanceRoot + Constants.ZMASTERS, EMPTY_BYTE_ARRAY,
+    zoo.putPersistentData(zkInstanceRoot + Constants.ZMANAGERS, EMPTY_BYTE_ARRAY,
         NodeExistsPolicy.FAIL);
-    zoo.putPersistentData(zkInstanceRoot + Constants.ZMASTER_LOCK, EMPTY_BYTE_ARRAY,
+    zoo.putPersistentData(zkInstanceRoot + Constants.ZMANAGER_LOCK, EMPTY_BYTE_ARRAY,
         NodeExistsPolicy.FAIL);
-    zoo.putPersistentData(zkInstanceRoot + Constants.ZMASTER_GOAL_STATE,
+    zoo.putPersistentData(zkInstanceRoot + Constants.ZMANAGER_GOAL_STATE,
         ManagerGoalState.NORMAL.toString().getBytes(UTF_8), NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + Constants.ZGC, EMPTY_BYTE_ARRAY, NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + Constants.ZGC_LOCK, EMPTY_BYTE_ARRAY,
diff --git a/server/base/src/main/java/org/apache/accumulo/server/log/WalStateManager.java b/server/base/src/main/java/org/apache/accumulo/server/log/WalStateManager.java
index 2f1ae74..81aa047 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/log/WalStateManager.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/log/WalStateManager.java
@@ -44,9 +44,9 @@
  * by tablet servers and the replication machinery.
  *
  * <p>
- * The Master needs to know the state of the WALs to mark tablets during recovery. The GC needs to
- * know when a log is no longer needed so it can be removed. The replication mechanism needs to know
- * when a log is closed and can be forwarded to the destination table.
+ * The Accumulo Manager needs to know the state of the WALs to mark tablets during recovery. The GC
+ * needs to know when a log is no longer needed so it can be removed. The replication mechanism
+ * needs to know when a log is closed and can be forwarded to the destination table.
  *
  * <p>
  * The state of the WALs is kept in Zookeeper under /accumulo/&lt;instanceid&gt;/wals. For each
@@ -59,7 +59,7 @@
  * file.
  *
  * <p>
- * In the event of a recovery, the log is identified as belonging to a dead server. The master will
+ * In the event of a recovery, the log is identified as belonging to a dead server. The manager will
  * update the tablets assigned to that server with log references. Once all tablets have been
  * reassigned and the log references are removed, the log will be eligible for deletion.
  *
@@ -157,7 +157,7 @@
     return new Pair<>(WalState.valueOf(parts[0]), new Path(parts[1]));
   }
 
-  // Master needs to know the logs for the given instance
+  // Manager needs to know the logs for the given instance
   public List<Path> getWalsInUse(TServerInstance tsi) throws WalMarkerException {
     List<Path> result = new ArrayList<>();
     try {
@@ -166,13 +166,13 @@
       for (String child : zoo.getChildren(zpath)) {
         byte[] zdata = null;
         try {
-          // This function is called by the Master. Its possible that Accumulo GC deletes an
+          // This function is called by the Manager. Its possible that Accumulo GC deletes an
           // unreferenced WAL in ZK after the call to getChildren above. Catch this exception inside
           // the loop so that not all children are ignored.
           zdata = zoo.getData(zpath + "/" + child);
         } catch (KeeperException.NoNodeException e) {
           log.debug("WAL state removed {} {} during getWalsInUse.  Likely a race condition between "
-              + "master and GC.", tsi, child);
+              + "manager and GC.", tsi, child);
         }
 
         if (zdata != null) {
@@ -257,7 +257,7 @@
   }
 
   // tablet server can mark the log as closed (but still needed), for replication to begin
-  // master can mark a log as unreferenced after it has made log recovery markers on the tablets
+  // manager can mark a log as unreferenced after it has made log recovery markers on the tablets
   // that need to be recovered
   public void closeWal(TServerInstance instance, Path path) throws WalMarkerException {
     updateState(instance, path, WalState.CLOSED);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/LiveTServerSet.java b/server/base/src/main/java/org/apache/accumulo/server/manager/LiveTServerSet.java
index b103328..8e85250 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/manager/LiveTServerSet.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/manager/LiveTServerSet.java
@@ -79,7 +79,7 @@
     }
 
     private String lockString(ZooLock mlock) {
-      return mlock.getLockID().serialize(context.getZooKeeperRoot() + Constants.ZMASTER_LOCK);
+      return mlock.getLockID().serialize(context.getZooKeeperRoot() + Constants.ZMANAGER_LOCK);
     }
 
     private void loadTablet(TabletClientService.Client client, ZooLock lock, KeyExtent extent)
diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/state/CurrentState.java b/server/base/src/main/java/org/apache/accumulo/server/manager/state/CurrentState.java
index ea5c8a7..f9c8524 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/manager/state/CurrentState.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/manager/state/CurrentState.java
@@ -42,5 +42,5 @@
    */
   Set<KeyExtent> migrationsSnapshot();
 
-  ManagerState getMasterState();
+  ManagerState getManagerState();
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/state/MetaDataTableScanner.java b/server/base/src/main/java/org/apache/accumulo/server/manager/state/MetaDataTableScanner.java
index 70f29c9..2b955f1 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/manager/state/MetaDataTableScanner.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/manager/state/MetaDataTableScanner.java
@@ -96,7 +96,7 @@
       TabletStateChangeIterator.setOnlineTables(tabletChange, state.onlineTables());
       TabletStateChangeIterator.setMerges(tabletChange, state.merges());
       TabletStateChangeIterator.setMigrations(tabletChange, state.migrationsSnapshot());
-      TabletStateChangeIterator.setMasterState(tabletChange, state.getMasterState());
+      TabletStateChangeIterator.setManagerState(tabletChange, state.getManagerState());
       TabletStateChangeIterator.setShuttingDown(tabletChange, state.shutdownServers());
     }
     scanner.addScanIterator(tabletChange);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletServerState.java b/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletServerState.java
index e75948c..33538fa 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletServerState.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletServerState.java
@@ -31,7 +31,7 @@
   UNRESPONSIVE((byte) 2),
   DOWN((byte) 3),
 
-  // the following are bad states and cause tservers to be ignored by the master
+  // the following are bad states and cause tservers to be ignored by the manager
   BAD_SYSTEM_PASSWORD((byte) 101),
   BAD_VERSION((byte) 102),
   BAD_INSTANCE((byte) 103),
diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletStateChangeIterator.java b/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletStateChangeIterator.java
index a56bed1..cf665a7 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletStateChangeIterator.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/manager/state/TabletStateChangeIterator.java
@@ -57,7 +57,7 @@
   private static final String MERGES_OPTION = "merges";
   private static final String DEBUG_OPTION = "debug";
   private static final String MIGRATIONS_OPTION = "migrations";
-  private static final String MASTER_STATE_OPTION = "managerState";
+  private static final String MANAGER_STATE_OPTION = "managerState";
   private static final String SHUTTING_DOWN_OPTION = "shuttingDown";
   private static final Logger log = LoggerFactory.getLogger(TabletStateChangeIterator.class);
 
@@ -78,10 +78,10 @@
     debug = options.containsKey(DEBUG_OPTION);
     migrations = parseMigrations(options.get(MIGRATIONS_OPTION));
     try {
-      managerState = ManagerState.valueOf(options.get(MASTER_STATE_OPTION));
+      managerState = ManagerState.valueOf(options.get(MANAGER_STATE_OPTION));
     } catch (Exception ex) {
-      if (options.get(MASTER_STATE_OPTION) != null) {
-        log.error("Unable to decode managerState {}", options.get(MASTER_STATE_OPTION));
+      if (options.get(MANAGER_STATE_OPTION) != null) {
+        log.error("Unable to decode managerState {}", options.get(MANAGER_STATE_OPTION));
       }
     }
     Set<TServerInstance> shuttingDown = parseServers(options.get(SHUTTING_DOWN_OPTION));
@@ -168,7 +168,7 @@
         if (tls == null)
           return;
       } catch (BadLocationStateException e) {
-        // maybe the master can do something with a tablet with bad/inconsistent state
+        // maybe the manager can do something with a tablet with bad/inconsistent state
         return;
       }
       // we always want data about merges
@@ -263,8 +263,8 @@
     cfg.addOption(MIGRATIONS_OPTION, encoded);
   }
 
-  public static void setMasterState(IteratorSetting cfg, ManagerState state) {
-    cfg.addOption(MASTER_STATE_OPTION, state.toString());
+  public static void setManagerState(IteratorSetting cfg, ManagerState state) {
+    cfg.addOption(MANAGER_STATE_OPTION, state.toString());
   }
 
   public static void setShuttingDown(IteratorSetting cfg, Set<TServerInstance> servers) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java
index 2083e25..5312f88 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java
@@ -72,7 +72,7 @@
  * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets
  * that it should not be according to the regex configuration. If this occurs then the offending
  * tablets will be reassigned. This would cover the case where the configuration is changed and the
- * master is restarted while the tablet servers are up. To change the out of bounds check time
+ * manager is restarted while the tablet servers are up. To change the out of bounds check time
  * period, set the following property:<br>
  * <b>table.custom.balancer.host.regex.oob.period=5m</b><br>
  * Regex matching can be based on either the host name (default) or host ip address. To set this
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
index a1a7a7c..60d3722 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
@@ -119,7 +119,7 @@
   }
 
   /**
-   * Assign tablets to tablet servers. This method is called whenever the master finds tablets that
+   * Assign tablets to tablet servers. This method is called whenever the manager finds tablets that
    * are unassigned.
    *
    * @param current
diff --git a/server/base/src/main/java/org/apache/accumulo/server/rpc/TServerUtils.java b/server/base/src/main/java/org/apache/accumulo/server/rpc/TServerUtils.java
index 6545af1..9cc2899 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/rpc/TServerUtils.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/rpc/TServerUtils.java
@@ -523,7 +523,7 @@
     // their configuration.
     if (!hostname.equals(fqdn)) {
       log.error("Expected hostname of '{}' but got '{}'. Ensure the entries in"
-          + " the Accumulo hosts files (e.g. masters, tservers) are the FQDN for"
+          + " the Accumulo hosts files (e.g. managers, tservers) are the FQDN for"
           + " each host when using SASL.", fqdn, hostname);
       transport.close();
       throw new RuntimeException("SASL requires that the address the thrift"
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/delegation/AuthenticationTokenKeyManager.java b/server/base/src/main/java/org/apache/accumulo/server/security/delegation/AuthenticationTokenKeyManager.java
index 2c77e3b..0018449 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/delegation/AuthenticationTokenKeyManager.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/delegation/AuthenticationTokenKeyManager.java
@@ -113,7 +113,7 @@
 
         // Try to use the last key instead of creating a new one right away. This will present more
         // expected
-        // functionality if the active master happens to die for some reason
+        // functionality if the active manager happens to die for some reason
         AuthenticationKey currentKey = secretManager.getCurrentKey();
         if (currentKey != null) {
           log.info("Updating last key update to {} from current secret manager key",
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/delegation/AuthenticationTokenSecretManager.java b/server/base/src/main/java/org/apache/accumulo/server/security/delegation/AuthenticationTokenSecretManager.java
index 08f94cd..f700158 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/delegation/AuthenticationTokenSecretManager.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/delegation/AuthenticationTokenSecretManager.java
@@ -47,8 +47,8 @@
  * Manages an internal list of secret keys used to sign new authentication tokens as they are
  * generated, and to validate existing tokens used for authentication.
  *
- * Each TabletServer, in addition to the Master, has an instance of this {@link SecretManager} so
- * that each can authenticate requests from clients presenting delegation tokens. The Master will
+ * Each TabletServer, in addition to the Manager, has an instance of this {@link SecretManager} so
+ * that each can authenticate requests from clients presenting delegation tokens. The Manager will
  * also run an instance of {@link AuthenticationTokenKeyManager} which handles generation of new
  * keys and removal of old keys. That class will call the methods here to ensure the in-memory cache
  * is consistent with what is advertised in ZooKeeper.
@@ -127,12 +127,12 @@
     if (identifier.getIssueDate() > now) {
       throw new InvalidToken("Token issued in the future");
     }
-    AuthenticationKey masterKey = allKeys.get(identifier.getKeyId());
-    if (masterKey == null) {
-      throw new InvalidToken("Unknown master key for token (id=" + identifier.getKeyId() + ")");
+    AuthenticationKey managerKey = allKeys.get(identifier.getKeyId());
+    if (managerKey == null) {
+      throw new InvalidToken("Unknown manager key for token (id=" + identifier.getKeyId() + ")");
     }
     // regenerate the password
-    return createPassword(identifier.getBytes(), masterKey.getKey());
+    return createPassword(identifier.getBytes(), managerKey.getKey());
   }
 
   @Override
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/delegation/ZooAuthenticationKeyWatcher.java b/server/base/src/main/java/org/apache/accumulo/server/security/delegation/ZooAuthenticationKeyWatcher.java
index 66575e7..1cf140e 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/delegation/ZooAuthenticationKeyWatcher.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/delegation/ZooAuthenticationKeyWatcher.java
@@ -150,7 +150,7 @@
         secretManager.addKey(key);
         keysAdded++;
       } catch (NoNodeException e) {
-        // The master expired(deleted) the key between when we saw it in getChildren() and when we
+        // The manager expired(deleted) the key between when we saw it in getChildren() and when we
         // went to add it to our secret manager.
         log.trace("{} was deleted when we tried to access it", childPath);
       }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/AccumuloStatus.java b/server/base/src/main/java/org/apache/accumulo/server/util/AccumuloStatus.java
index 37eeb2e..224f6f1 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/AccumuloStatus.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/AccumuloStatus.java
@@ -39,7 +39,7 @@
         if (!reader.getChildren(rootPath + Constants.ZTSERVERS + "/" + child).isEmpty())
           return false;
       }
-      if (!reader.getChildren(rootPath + Constants.ZMASTER_LOCK).isEmpty())
+      if (!reader.getChildren(rootPath + Constants.ZMANAGER_LOCK).isEmpty())
         return false;
       if (!reader.getChildren(rootPath + Constants.ZMONITOR_LOCK).isEmpty())
         return false;
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java
index 54ae75a..333cbce 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/Admin.java
@@ -374,8 +374,8 @@
 
   private static void stopTabletServer(final ClientContext context, List<String> servers,
       final boolean force) throws AccumuloException, AccumuloSecurityException {
-    if (context.getMasterLocations().isEmpty()) {
-      log.info("No masters running. Not attempting safe unload of tserver.");
+    if (context.getManagerLocations().isEmpty()) {
+      log.info("No managers running. Not attempting safe unload of tserver.");
       return;
     }
     final String zTServerRoot = getTServersZkPath(context);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/Info.java b/server/base/src/main/java/org/apache/accumulo/server/util/Info.java
index 0a12843..faac6bd 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/Info.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/Info.java
@@ -48,7 +48,7 @@
   public void execute(final String[] args) throws KeeperException, InterruptedException {
     var context = new ServerContext(SiteConfiguration.auto());
     System.out.println("monitor: " + MonitorUtil.getLocation(context));
-    System.out.println("masters: " + context.getMasterLocations());
+    System.out.println("managers: " + context.getManagerLocations());
     System.out.println("zookeepers: " + context.getZooKeepers());
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ListInstances.java b/server/base/src/main/java/org/apache/accumulo/server/util/ListInstances.java
index 2ab465e..14ba4cd 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ListInstances.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ListInstances.java
@@ -46,7 +46,7 @@
 
   private static final int NAME_WIDTH = 20;
   private static final int UUID_WIDTH = 37;
-  private static final int MASTER_WIDTH = 30;
+  private static final int MANAGER_WIDTH = 30;
 
   private static final int ZOOKEEPER_TIMER_MILLIS = 30 * 1000;
 
@@ -137,42 +137,42 @@
   }
 
   private static void printHeader() {
-    System.out.printf(" %-" + NAME_WIDTH + "s| %-" + UUID_WIDTH + "s| %-" + MASTER_WIDTH + "s%n",
-        "Instance Name", "Instance ID", "Master");
+    System.out.printf(" %-" + NAME_WIDTH + "s| %-" + UUID_WIDTH + "s| %-" + MANAGER_WIDTH + "s%n",
+        "Instance Name", "Instance ID", "Manager");
     System.out.printf(
-        "%" + (NAME_WIDTH + 1) + "s+%" + (UUID_WIDTH + 1) + "s+%" + (MASTER_WIDTH + 1) + "s%n",
+        "%" + (NAME_WIDTH + 1) + "s+%" + (UUID_WIDTH + 1) + "s+%" + (MANAGER_WIDTH + 1) + "s%n",
         new CharFiller('-'), new CharFiller('-'), new CharFiller('-'));
 
   }
 
   private static void printInstanceInfo(ZooCache cache, String instanceName, UUID iid,
       boolean printErrors) {
-    String master = getMaster(cache, iid, printErrors);
+    String manager = getManager(cache, iid, printErrors);
     if (instanceName == null) {
       instanceName = "";
     }
 
-    if (master == null) {
-      master = "";
+    if (manager == null) {
+      manager = "";
     }
 
-    System.out.printf("%" + NAME_WIDTH + "s |%" + UUID_WIDTH + "s |%" + MASTER_WIDTH + "s%n",
-        "\"" + instanceName + "\"", iid, master);
+    System.out.printf("%" + NAME_WIDTH + "s |%" + UUID_WIDTH + "s |%" + MANAGER_WIDTH + "s%n",
+        "\"" + instanceName + "\"", iid, manager);
   }
 
-  private static String getMaster(ZooCache cache, UUID iid, boolean printErrors) {
+  private static String getManager(ZooCache cache, UUID iid, boolean printErrors) {
 
     if (iid == null) {
       return null;
     }
 
     try {
-      String masterLocPath = Constants.ZROOT + "/" + iid + Constants.ZMASTER_LOCK;
-      byte[] master = ZooLock.getLockData(cache, masterLocPath, null);
-      if (master == null) {
+      String managerLocPath = Constants.ZROOT + "/" + iid + Constants.ZMANAGER_LOCK;
+      byte[] manager = ZooLock.getLockData(cache, managerLocPath, null);
+      if (manager == null) {
         return null;
       }
-      return new String(master, UTF_8);
+      return new String(manager, UTF_8);
     } catch (Exception e) {
       handleException(e, printErrors);
       return null;
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ServerBulkImportStatus.java b/server/base/src/main/java/org/apache/accumulo/server/util/ServerBulkImportStatus.java
index 1bf6822..0aadcb8 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ServerBulkImportStatus.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ServerBulkImportStatus.java
@@ -26,7 +26,7 @@
 import org.apache.accumulo.core.master.thrift.BulkImportState;
 import org.apache.accumulo.core.master.thrift.BulkImportStatus;
 
-// A little class to hold bulk import status information in the Master
+// A little class to hold bulk import status information in the Manager
 // and two places in the tablet server.
 public class ServerBulkImportStatus {
   private final ConcurrentMap<String,BulkImportStatus> status = new ConcurrentHashMap<>();
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ZooZap.java b/server/base/src/main/java/org/apache/accumulo/server/util/ZooZap.java
index 0603cef..d88c85d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ZooZap.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ZooZap.java
@@ -51,8 +51,12 @@
   }
 
   static class Opts extends Help {
-    @Parameter(names = "-master", description = "remove master locks")
+    @Deprecated(since = "2.1.0")
+    @Parameter(names = "-master",
+        description = "remove master locks (deprecated -- user -manager instead")
     boolean zapMaster = false;
+    @Parameter(names = "-manager", description = "remove manager locks")
+    boolean zapManager = false;
     @Parameter(names = "-tservers", description = "remove tablet server locks")
     boolean zapTservers = false;
     @Parameter(names = "-tracers", description = "remove tracer locks")
@@ -65,7 +69,7 @@
     Opts opts = new Opts();
     opts.parseArgs(ZooZap.class.getName(), args);
 
-    if (!opts.zapMaster && !opts.zapTservers && !opts.zapTracers) {
+    if (!opts.zapMaster && !opts.zapManager && !opts.zapTservers && !opts.zapTracers) {
       new JCommander(opts).usage();
       return;
     }
@@ -83,10 +87,13 @@
       ZooReaderWriter zoo = new ZooReaderWriter(siteConf);
 
       if (opts.zapMaster) {
-        String masterLockPath = Constants.ZROOT + "/" + iid + Constants.ZMASTER_LOCK;
+        log.warn("The -master option is deprecated. Please use -manager instead.");
+      }
+      if (opts.zapManager || opts.zapMaster) {
+        String managerLockPath = Constants.ZROOT + "/" + iid + Constants.ZMANAGER_LOCK;
 
         try {
-          zapDirectory(zoo, masterLockPath, opts);
+          zapDirectory(zoo, managerLockPath, opts);
         } catch (Exception e) {
           e.printStackTrace();
         }
@@ -99,7 +106,7 @@
           for (String child : children) {
             message("Deleting " + tserversPath + "/" + child + " from zookeeper", opts);
 
-            if (opts.zapMaster) {
+            if (opts.zapManager || opts.zapMaster) {
               zoo.recursiveDelete(tserversPath + "/" + child, NodeMissingPolicy.SKIP);
             } else {
               String path = tserversPath + "/" + child;
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java
index cd88059..d998321 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java
@@ -109,7 +109,7 @@
     }
   }
 
-  // ugh... so wish I had provided mock objects to the LoadBalancer in the master
+  // ugh... so wish I had provided mock objects to the LoadBalancer in the manager
   class TableLoadBalancer extends org.apache.accumulo.server.master.balancer.TableLoadBalancer {
 
     // use our new classname to test class loading
diff --git a/server/base/src/test/java/org/apache/accumulo/server/security/delegation/AuthenticationTokenSecretManagerTest.java b/server/base/src/test/java/org/apache/accumulo/server/security/delegation/AuthenticationTokenSecretManagerTest.java
index 3df7128..ab8d9de 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/security/delegation/AuthenticationTokenSecretManagerTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/security/delegation/AuthenticationTokenSecretManagerTest.java
@@ -269,7 +269,7 @@
   }
 
   @Test(expected = InvalidToken.class)
-  public void testRolledMasterKey() throws Exception {
+  public void testRolledManagerKey() throws Exception {
     // start of the test
     long then = System.currentTimeMillis();
 
@@ -304,7 +304,7 @@
   }
 
   @Test(timeout = 20 * 1000)
-  public void testMasterKeyExpiration() throws Exception {
+  public void testManagerKeyExpiration() throws Exception {
     ZooAuthenticationKeyDistributor keyDistributor =
         createMock(ZooAuthenticationKeyDistributor.class);
     // start of the test
diff --git a/server/base/src/test/java/org/apache/accumulo/server/security/delegation/ZooAuthenticationKeyWatcherTest.java b/server/base/src/test/java/org/apache/accumulo/server/security/delegation/ZooAuthenticationKeyWatcherTest.java
index 3f3bd0e..1050a1e 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/security/delegation/ZooAuthenticationKeyWatcherTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/security/delegation/ZooAuthenticationKeyWatcherTest.java
@@ -326,7 +326,7 @@
     expect(zk.exists(baseNode, keyWatcher)).andReturn(true);
     // We saw key1
     expect(zk.getChildren(baseNode, keyWatcher)).andReturn(children);
-    // but it was gone when we tried to access it (master deleted it)
+    // but it was gone when we tried to access it (manager deleted it)
     expect(zk.getData(baseNode + "/" + key1.getKeyId(), keyWatcher))
         .andThrow(new NoNodeException());
 
diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/AdminCommandsTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/AdminCommandsTest.java
index c85b160..514afc5 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/util/AdminCommandsTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/util/AdminCommandsTest.java
@@ -45,6 +45,11 @@
   }
 
   @Test
+  public void testStopManagerCommand() {
+    new Admin.StopManagerCommand();
+  }
+
+  @Test
   @SuppressWarnings("deprecation")
   public void testStopMasterCommand() {
     new Admin.StopMasterCommand();
diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/TServerUtilsTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/TServerUtilsTest.java
index 6c69a9f..287b906 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/util/TServerUtilsTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/util/TServerUtilsTest.java
@@ -219,9 +219,9 @@
     // uses those port numbers to Accumulo services in the below (ascending) sequence
     // 0. TServer default client port (this test binds to this port to force a port search)
     // 1. GC
-    // 2. Master
+    // 2. Manager
     // 3. Monitor
-    // 4. Master Replication Coordinator
+    // 4. Manager Replication Coordinator
     // 5. One free port - this is the one that we expect the TServer to finally use
     int[] ports = findTwoFreeSequentialPorts(1024);
     int tserverDefaultPort = ports[0];
@@ -230,14 +230,14 @@
     conf.set(Property.GC_PORT, Integer.toString(gcPort));
 
     ports = findTwoFreeSequentialPorts(gcPort + 1);
-    int masterPort = ports[0];
-    conf.set(Property.MANAGER_CLIENTPORT, Integer.toString(masterPort));
+    int managerPort = ports[0];
+    conf.set(Property.MANAGER_CLIENTPORT, Integer.toString(managerPort));
     int monitorPort = ports[1];
     conf.set(Property.MONITOR_PORT, Integer.toString(monitorPort));
 
     ports = findTwoFreeSequentialPorts(monitorPort + 1);
-    int masterReplCoordPort = ports[0];
-    conf.set(Property.MANAGER_REPLICATION_COORDINATOR_PORT, Integer.toString(masterReplCoordPort));
+    int managerReplCoordPort = ports[0];
+    conf.set(Property.MANAGER_REPLICATION_COORDINATOR_PORT, Integer.toString(managerReplCoordPort));
     int tserverFinalPort = ports[1];
 
     conf.set(Property.TSERV_PORTSEARCH, "true");
@@ -246,12 +246,12 @@
     Map<Integer,Property> reservedPorts = TServerUtils.getReservedPorts(conf);
     assertFalse(reservedPorts.containsKey(tserverDefaultPort));
 
-    // Ensure that all the ports we assigned (GC, Master, Monitor) are included in the reserved
+    // Ensure that all the ports we assigned (GC, Manager, Monitor) are included in the reserved
     // ports as returned by TServerUtils
     assertTrue(reservedPorts.containsKey(gcPort));
-    assertTrue(reservedPorts.containsKey(masterPort));
+    assertTrue(reservedPorts.containsKey(managerPort));
     assertTrue(reservedPorts.containsKey(monitorPort));
-    assertTrue(reservedPorts.containsKey(masterReplCoordPort));
+    assertTrue(reservedPorts.containsKey(managerReplCoordPort));
 
     InetAddress addr = InetAddress.getByName("localhost");
     try (ServerSocket s = new ServerSocket(tserverDefaultPort, 50, addr)) {
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
index 698768a..72f47e9 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
@@ -285,7 +285,7 @@
 
     // remove any entries if there's a log reference (recovery hasn't finished)
     for (TabletLocationState state : store) {
-      // Tablet is still assigned to a dead server. Master has moved markers and reassigned it
+      // Tablet is still assigned to a dead server. Manager has moved markers and reassigned it
       // Easiest to just ignore all the WALs for the dead server.
       if (state.getState(liveServers) == TabletState.ASSIGNED_TO_DEAD_SERVER) {
         Set<UUID> idsToIgnore = candidates.remove(state.current);
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
index b940ef1..ee71706 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
@@ -460,7 +460,7 @@
   public void run() {
     final VolumeManager fs = getContext().getVolumeManager();
 
-    // Sleep for an initial period, giving the master time to start up and
+    // Sleep for an initial period, giving the manager time to start up and
     // old data files to be unused
     log.info("Trying to acquire ZooKeeper lock for garbage collector");
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java b/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java
index 4e63870..0db5622 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/FateServiceHandler.java
@@ -91,18 +91,18 @@
 
 class FateServiceHandler implements FateService.Iface {
 
-  protected final Manager master;
+  protected final Manager manager;
   protected static final Logger log = Manager.log;
 
-  public FateServiceHandler(Manager master) {
-    this.master = master;
+  public FateServiceHandler(Manager manager) {
+    this.manager = manager;
   }
 
   @Override
   public long beginFateOperation(TInfo tinfo, TCredentials credentials)
       throws ThriftSecurityException {
     authenticate(credentials);
-    return master.fate.startTransaction();
+    return manager.fate.startTransaction();
   }
 
   @Override
@@ -117,10 +117,10 @@
         validateArgumentCount(arguments, tableOp, 1);
         String namespace = validateNewNamespaceArgument(arguments.get(0), tableOp, null);
 
-        if (!master.security.canCreateNamespace(c))
+        if (!manager.security.canCreateNamespace(c))
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
-        master.fate.seedTransaction(opid,
+        manager.fate.seedTransaction(opid,
             new TraceRepo<>(new CreateNamespace(c.getPrincipal(), namespace, options)),
             autoCleanup);
         break;
@@ -133,11 +133,11 @@
         String newName = validateNewNamespaceArgument(arguments.get(1), tableOp, null);
 
         NamespaceId namespaceId =
-            ClientServiceHandler.checkNamespaceId(master.getContext(), oldName, tableOp);
-        if (!master.security.canRenameNamespace(c, namespaceId))
+            ClientServiceHandler.checkNamespaceId(manager.getContext(), oldName, tableOp);
+        if (!manager.security.canRenameNamespace(c, namespaceId))
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
-        master.fate.seedTransaction(opid,
+        manager.fate.seedTransaction(opid,
             new TraceRepo<>(new RenameNamespace(namespaceId, oldName, newName)), autoCleanup);
         break;
       }
@@ -148,11 +148,11 @@
             Namespaces.NOT_DEFAULT.and(Namespaces.NOT_ACCUMULO));
 
         NamespaceId namespaceId =
-            ClientServiceHandler.checkNamespaceId(master.getContext(), namespace, tableOp);
-        if (!master.security.canDeleteNamespace(c, namespaceId))
+            ClientServiceHandler.checkNamespaceId(manager.getContext(), namespace, tableOp);
+        if (!manager.security.canDeleteNamespace(c, namespaceId))
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
-        master.fate.seedTransaction(opid, new TraceRepo<>(new DeleteNamespace(namespaceId)),
+        manager.fate.seedTransaction(opid, new TraceRepo<>(new DeleteNamespace(namespaceId)),
             autoCleanup);
         break;
       }
@@ -188,16 +188,16 @@
 
         try {
           namespaceId =
-              Namespaces.getNamespaceId(master.getContext(), Tables.qualify(tableName).getFirst());
+              Namespaces.getNamespaceId(manager.getContext(), Tables.qualify(tableName).getFirst());
         } catch (NamespaceNotFoundException e) {
           throw new ThriftTableOperationException(null, tableName, tableOp,
               TableOperationExceptionType.NAMESPACE_NOTFOUND, "");
         }
 
-        if (!master.security.canCreateTable(c, tableName, namespaceId))
+        if (!manager.security.canCreateTable(c, tableName, namespaceId))
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
-        master.fate.seedTransaction(opid,
+        manager.fate.seedTransaction(opid,
             new TraceRepo<>(new CreateTable(c.getPrincipal(), tableName, timeType, options,
                 splitsPath, splitCount, splitsDirsPath, initialTableState, namespaceId)),
             autoCleanup);
@@ -229,13 +229,13 @@
             });
 
         TableId tableId =
-            ClientServiceHandler.checkTableId(master.getContext(), oldTableName, tableOp);
+            ClientServiceHandler.checkTableId(manager.getContext(), oldTableName, tableOp);
         NamespaceId namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
 
         final boolean canRename;
         try {
           canRename =
-              master.security.canRenameTable(c, tableId, oldTableName, newTableName, namespaceId);
+              manager.security.canRenameTable(c, tableId, oldTableName, newTableName, namespaceId);
         } catch (ThriftSecurityException e) {
           throwIfTableMissingSecurityException(e, tableId, oldTableName, TableOperation.RENAME);
           throw e;
@@ -245,7 +245,7 @@
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
         try {
-          master.fate.seedTransaction(opid,
+          manager.fate.seedTransaction(opid,
               new TraceRepo<>(new RenameTable(namespaceId, tableId, oldTableName, newTableName)),
               autoCleanup);
         } catch (NamespaceNotFoundException e) {
@@ -268,7 +268,7 @@
         NamespaceId namespaceId;
         try {
           namespaceId =
-              Namespaces.getNamespaceId(master.getContext(), Tables.qualify(tableName).getFirst());
+              Namespaces.getNamespaceId(manager.getContext(), Tables.qualify(tableName).getFirst());
         } catch (NamespaceNotFoundException e) {
           // shouldn't happen, but possible once cloning between namespaces is supported
           throw new ThriftTableOperationException(null, tableName, tableOp,
@@ -278,7 +278,7 @@
         final boolean canCloneTable;
         try {
           canCloneTable =
-              master.security.canCloneTable(c, srcTableId, tableName, namespaceId, namespaceId);
+              manager.security.canCloneTable(c, srcTableId, tableName, namespaceId, namespaceId);
         } catch (ThriftSecurityException e) {
           throwIfTableMissingSecurityException(e, srcTableId, null, TableOperation.CLONE);
           throw e;
@@ -306,7 +306,7 @@
           propertiesToSet.put(entry.getKey(), entry.getValue());
         }
 
-        master.fate.seedTransaction(opid, new TraceRepo<>(new CloneTable(c.getPrincipal(),
+        manager.fate.seedTransaction(opid, new TraceRepo<>(new CloneTable(c.getPrincipal(),
             namespaceId, srcTableId, tableName, propertiesToSet, propertiesToExclude, keepOffline)),
             autoCleanup);
 
@@ -318,12 +318,12 @@
         String tableName = validateTableNameArgument(arguments.get(0), tableOp, NOT_SYSTEM);
 
         final TableId tableId =
-            ClientServiceHandler.checkTableId(master.getContext(), tableName, tableOp);
+            ClientServiceHandler.checkTableId(manager.getContext(), tableName, tableOp);
         NamespaceId namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
 
         final boolean canDeleteTable;
         try {
-          canDeleteTable = master.security.canDeleteTable(c, tableId, namespaceId);
+          canDeleteTable = manager.security.canDeleteTable(c, tableId, namespaceId);
         } catch (ThriftSecurityException e) {
           throwIfTableMissingSecurityException(e, tableId, tableName, TableOperation.DELETE);
           throw e;
@@ -331,7 +331,7 @@
 
         if (!canDeleteTable)
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-        master.fate.seedTransaction(opid, new TraceRepo<>(new DeleteTable(namespaceId, tableId)),
+        manager.fate.seedTransaction(opid, new TraceRepo<>(new DeleteTable(namespaceId, tableId)),
             autoCleanup);
         break;
       }
@@ -344,7 +344,7 @@
         final boolean canOnlineOfflineTable;
         try {
           canOnlineOfflineTable =
-              master.security.canOnlineOfflineTable(c, tableId, op, namespaceId);
+              manager.security.canOnlineOfflineTable(c, tableId, op, namespaceId);
         } catch (ThriftSecurityException e) {
           throwIfTableMissingSecurityException(e, tableId, null, TableOperation.ONLINE);
           throw e;
@@ -353,7 +353,7 @@
         if (!canOnlineOfflineTable)
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
-        master.fate.seedTransaction(opid,
+        manager.fate.seedTransaction(opid,
             new TraceRepo<>(new ChangeTableState(namespaceId, tableId, tableOp)), autoCleanup);
         break;
       }
@@ -366,7 +366,7 @@
         final boolean canOnlineOfflineTable;
         try {
           canOnlineOfflineTable =
-              master.security.canOnlineOfflineTable(c, tableId, op, namespaceId);
+              manager.security.canOnlineOfflineTable(c, tableId, op, namespaceId);
         } catch (ThriftSecurityException e) {
           throwIfTableMissingSecurityException(e, tableId, null, TableOperation.OFFLINE);
           throw e;
@@ -375,7 +375,7 @@
         if (!canOnlineOfflineTable)
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
-        master.fate.seedTransaction(opid,
+        manager.fate.seedTransaction(opid,
             new TraceRepo<>(new ChangeTableState(namespaceId, tableId, tableOp)), autoCleanup);
         break;
       }
@@ -387,12 +387,12 @@
         Text endRow = ByteBufferUtil.toText(arguments.get(2));
 
         final TableId tableId =
-            ClientServiceHandler.checkTableId(master.getContext(), tableName, tableOp);
+            ClientServiceHandler.checkTableId(manager.getContext(), tableName, tableOp);
         NamespaceId namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
 
         final boolean canMerge;
         try {
-          canMerge = master.security.canMerge(c, tableId, namespaceId);
+          canMerge = manager.security.canMerge(c, tableId, namespaceId);
         } catch (ThriftSecurityException e) {
           throwIfTableMissingSecurityException(e, tableId, tableName, TableOperation.MERGE);
           throw e;
@@ -402,7 +402,7 @@
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
         Manager.log.debug("Creating merge op: {} {} {}", tableId, startRow, endRow);
-        master.fate.seedTransaction(opid, new TraceRepo<>(
+        manager.fate.seedTransaction(opid, new TraceRepo<>(
             new TableRangeOp(MergeInfo.Operation.MERGE, namespaceId, tableId, startRow, endRow)),
             autoCleanup);
         break;
@@ -415,13 +415,13 @@
         Text endRow = ByteBufferUtil.toText(arguments.get(2));
 
         final TableId tableId =
-            ClientServiceHandler.checkTableId(master.getContext(), tableName, tableOp);
+            ClientServiceHandler.checkTableId(manager.getContext(), tableName, tableOp);
         NamespaceId namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
 
         final boolean canDeleteRange;
         try {
           canDeleteRange =
-              master.security.canDeleteRange(c, tableId, tableName, startRow, endRow, namespaceId);
+              manager.security.canDeleteRange(c, tableId, tableName, startRow, endRow, namespaceId);
         } catch (ThriftSecurityException e) {
           throwIfTableMissingSecurityException(e, tableId, tableName, TableOperation.DELETE_RANGE);
           throw e;
@@ -430,7 +430,7 @@
         if (!canDeleteRange)
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
-        master.fate.seedTransaction(opid, new TraceRepo<>(
+        manager.fate.seedTransaction(opid, new TraceRepo<>(
             new TableRangeOp(MergeInfo.Operation.DELETE, namespaceId, tableId, startRow, endRow)),
             autoCleanup);
         break;
@@ -444,13 +444,13 @@
         boolean setTime = Boolean.parseBoolean(ByteBufferUtil.toString(arguments.get(3)));
 
         final TableId tableId =
-            ClientServiceHandler.checkTableId(master.getContext(), tableName, tableOp);
+            ClientServiceHandler.checkTableId(manager.getContext(), tableName, tableOp);
         NamespaceId namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
 
         final boolean canBulkImport;
         try {
           canBulkImport =
-              master.security.canBulkImport(c, tableId, tableName, dir, failDir, namespaceId);
+              manager.security.canBulkImport(c, tableId, tableName, dir, failDir, namespaceId);
         } catch (ThriftSecurityException e) {
           throwIfTableMissingSecurityException(e, tableId, tableName, TableOperation.BULK_IMPORT);
           throw e;
@@ -459,8 +459,8 @@
         if (!canBulkImport)
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
-        master.updateBulkImportStatus(dir, BulkImportState.INITIAL);
-        master.fate.seedTransaction(opid,
+        manager.updateBulkImportStatus(dir, BulkImportState.INITIAL);
+        manager.fate.seedTransaction(opid,
             new TraceRepo<>(new org.apache.accumulo.manager.tableOps.bulkVer1.BulkImport(tableId,
                 dir, failDir, setTime)),
             autoCleanup);
@@ -476,7 +476,7 @@
 
         final boolean canCompact;
         try {
-          canCompact = master.security.canCompact(c, tableId, namespaceId);
+          canCompact = manager.security.canCompact(c, tableId, namespaceId);
         } catch (ThriftSecurityException e) {
           throwIfTableMissingSecurityException(e, tableId, null, TableOperation.COMPACT);
           throw e;
@@ -485,7 +485,7 @@
         if (!canCompact)
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
-        master.fate.seedTransaction(opid,
+        manager.fate.seedTransaction(opid,
             new TraceRepo<>(new CompactRange(namespaceId, tableId, compactionConfig)), autoCleanup);
         break;
       }
@@ -497,7 +497,7 @@
 
         final boolean canCancelCompact;
         try {
-          canCancelCompact = master.security.canCompact(c, tableId, namespaceId);
+          canCancelCompact = manager.security.canCompact(c, tableId, namespaceId);
         } catch (ThriftSecurityException e) {
           throwIfTableMissingSecurityException(e, tableId, null, TableOperation.COMPACT_CANCEL);
           throw e;
@@ -506,7 +506,7 @@
         if (!canCancelCompact)
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
-        master.fate.seedTransaction(opid,
+        manager.fate.seedTransaction(opid,
             new TraceRepo<>(new CancelCompactions(namespaceId, tableId)), autoCleanup);
         break;
       }
@@ -524,7 +524,7 @@
         NamespaceId namespaceId;
         try {
           namespaceId =
-              Namespaces.getNamespaceId(master.getContext(), Tables.qualify(tableName).getFirst());
+              Namespaces.getNamespaceId(manager.getContext(), Tables.qualify(tableName).getFirst());
         } catch (NamespaceNotFoundException e) {
           throw new ThriftTableOperationException(null, tableName, tableOp,
               TableOperationExceptionType.NAMESPACE_NOTFOUND, "");
@@ -532,7 +532,7 @@
 
         final boolean canImport;
         try {
-          canImport = master.security.canImport(c, tableName, exportDirs, namespaceId);
+          canImport = manager.security.canImport(c, tableName, exportDirs, namespaceId);
         } catch (ThriftSecurityException e) {
           throwIfTableMissingSecurityException(e, null, tableName, TableOperation.IMPORT);
           throw e;
@@ -541,7 +541,7 @@
         if (!canImport)
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
-        master.fate.seedTransaction(opid,
+        manager.fate.seedTransaction(opid,
             new TraceRepo<>(new ImportTable(c.getPrincipal(), tableName, exportDirs, namespaceId)),
             autoCleanup);
         break;
@@ -553,12 +553,12 @@
         String exportDir = ByteBufferUtil.toString(arguments.get(1));
 
         TableId tableId =
-            ClientServiceHandler.checkTableId(master.getContext(), tableName, tableOp);
+            ClientServiceHandler.checkTableId(manager.getContext(), tableName, tableOp);
         NamespaceId namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
 
         final boolean canExport;
         try {
-          canExport = master.security.canExport(c, tableId, tableName, exportDir, namespaceId);
+          canExport = manager.security.canExport(c, tableId, tableName, exportDir, namespaceId);
         } catch (ThriftSecurityException e) {
           throwIfTableMissingSecurityException(e, tableId, tableName, TableOperation.EXPORT);
           throw e;
@@ -567,7 +567,7 @@
         if (!canExport)
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
-        master.fate.seedTransaction(opid,
+        manager.fate.seedTransaction(opid,
             new TraceRepo<>(new ExportTable(namespaceId, tableName, tableId, exportDir)),
             autoCleanup);
         break;
@@ -584,9 +584,9 @@
 
         final boolean canBulkImport;
         try {
-          String tableName = Tables.getTableName(master.getContext(), tableId);
+          String tableName = Tables.getTableName(manager.getContext(), tableId);
           canBulkImport =
-              master.security.canBulkImport(c, tableId, tableName, dir, null, namespaceId);
+              manager.security.canBulkImport(c, tableId, tableName, dir, null, namespaceId);
         } catch (ThriftSecurityException e) {
           throwIfTableMissingSecurityException(e, tableId, "", TableOperation.BULK_IMPORT);
           throw e;
@@ -599,7 +599,7 @@
         if (!canBulkImport)
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
-        master.fate.seedTransaction(opid,
+        manager.fate.seedTransaction(opid,
             new TraceRepo<>(new PrepBulkImport(tableId, dir, setTime)), autoCleanup);
         break;
       default:
@@ -611,7 +611,7 @@
       throws ThriftTableOperationException {
     NamespaceId namespaceId;
     try {
-      namespaceId = Tables.getNamespaceId(master.getContext(), tableId);
+      namespaceId = Tables.getNamespaceId(manager.getContext(), tableId);
     } catch (TableNotFoundException e) {
       throw new ThriftTableOperationException(tableId.canonical(), null, tableOp,
           TableOperationExceptionType.NOTFOUND, e.getMessage());
@@ -635,7 +635,7 @@
    * @param tableName
    *          Table name being operated on, or null
    * @param op
-   *          The TableOperation the Master was attempting to perform
+   *          The TableOperation the Manager was attempting to perform
    * @throws ThriftTableOperationException
    *           Thrown if {@code e} was thrown because {@link SecurityErrorCode#TABLE_DOESNT_EXIST}
    */
@@ -653,9 +653,9 @@
       throws ThriftSecurityException, ThriftTableOperationException {
     authenticate(credentials);
 
-    TStatus status = master.fate.waitForCompletion(opid);
+    TStatus status = manager.fate.waitForCompletion(opid);
     if (status == TStatus.FAILED) {
-      Exception e = master.fate.getException(opid);
+      Exception e = manager.fate.getException(opid);
       if (e instanceof ThriftTableOperationException)
         throw (ThriftTableOperationException) e;
       else if (e instanceof ThriftSecurityException)
@@ -666,7 +666,7 @@
         throw new RuntimeException(e);
     }
 
-    String ret = master.fate.getReturn(opid);
+    String ret = manager.fate.getReturn(opid);
     if (ret == null)
       ret = ""; // thrift does not like returning null
     return ret;
@@ -676,14 +676,14 @@
   public void finishFateOperation(TInfo tinfo, TCredentials credentials, long opid)
       throws ThriftSecurityException {
     authenticate(credentials);
-    master.fate.delete(opid);
+    manager.fate.delete(opid);
   }
 
   protected void authenticate(TCredentials credentials) throws ThriftSecurityException {
     // this is a bit redundant, the credentials of the caller (the first arg) will throw an
     // exception if it fails to authenticate
     // before the second arg is checked (which would return true or false)
-    if (!master.security.authenticateUser(credentials, credentials))
+    if (!manager.security.authenticateUser(credentials, credentials))
       throw new ThriftSecurityException(credentials.getPrincipal(),
           SecurityErrorCode.BAD_CREDENTIALS);
   }
@@ -784,7 +784,7 @@
    */
   private void writeSplitsToFile(Path splitsPath, final List<ByteBuffer> arguments,
       final int splitCount, final int splitOffset) throws IOException {
-    FileSystem fs = splitsPath.getFileSystem(master.getContext().getHadoopConf());
+    FileSystem fs = splitsPath.getFileSystem(manager.getContext().getHadoopConf());
     try (FSDataOutputStream stream = fs.create(splitsPath)) {
       // base64 encode because splits can contain binary
       for (int i = splitOffset; i < splitCount + splitOffset; i++) {
@@ -806,7 +806,7 @@
    * @return the path of the created directory
    */
   public Path mkTempDir(long opid) throws IOException {
-    Volume vol = master.getVolumeManager().getFirst();
+    Volume vol = manager.getVolumeManager().getFirst();
     Path p = vol.prefixChild("/tmp/fate-" + String.format("%016x", opid));
     FileSystem fs = vol.getFileSystem();
     if (fs.exists(p))
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java
index ed069d0..08c546b 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java
@@ -158,9 +158,9 @@
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 
 /**
- * The Master is responsible for assigning and balancing tablets to tablet servers.
+ * The Manager is responsible for assigning and balancing tablets to tablet servers.
  * <p>
- * The master will also coordinate log recoveries and reports general status.
+ * The manager will also coordinate log recoveries and reports general status.
  */
 public class Manager extends AbstractServer
     implements LiveTServerSet.Listener, TableObserver, CurrentState, HighlyAvailableService {
@@ -201,7 +201,7 @@
   private ZooAuthenticationKeyDistributor keyDistributor;
   private AuthenticationTokenKeyManager authenticationTokenKeyManager;
 
-  ZooLock masterLock = null;
+  ZooLock managerLock = null;
   private TServer clientService = null;
   private volatile TabletBalancer tabletBalancer;
   private final BalancerEnvironment balancerEnvironment;
@@ -216,15 +216,15 @@
       Collections.unmodifiableSortedMap(new TreeMap<>());
   final ServerBulkImportStatus bulkImportStatus = new ServerBulkImportStatus();
 
-  private final AtomicBoolean masterInitialized = new AtomicBoolean(false);
+  private final AtomicBoolean managerInitialized = new AtomicBoolean(false);
 
   @Override
-  public synchronized ManagerState getMasterState() {
+  public synchronized ManagerState getManagerState() {
     return state;
   }
 
-  public boolean stillMaster() {
-    return getMasterState() != ManagerState.STOP;
+  public boolean stillManager() {
+    return getManagerState() != ManagerState.STOP;
   }
 
   static final boolean X = true;
@@ -240,12 +240,12 @@
       /* UNLOAD_ROOT_TABLET */      {O, O, O, X, X, X, X},
       /* STOP */                    {O, O, O, O, O, X, X}};
   //@formatter:on
-  synchronized void setMasterState(ManagerState newState) {
+  synchronized void setManagerState(ManagerState newState) {
     if (state.equals(newState)) {
       return;
     }
     if (!transitionOK[state.ordinal()][newState.ordinal()]) {
-      log.error("Programmer error: master should not transition from {} to {}", state, newState);
+      log.error("Programmer error: manager should not transition from {} to {}", state, newState);
     }
     ManagerState oldState = state;
     state = newState;
@@ -255,7 +255,7 @@
       // thread requesting the stop can return
       ThreadPools.createGeneralScheduledExecutorService(getConfiguration())
           .scheduleWithFixedDelay(() -> {
-            // This frees the main thread and will cause the master to exit
+            // This frees the main thread and will cause the manager to exit
             clientService.stop();
             Manager.this.nextEvent.event("stopped event loop");
           }, 100L, 1000L, TimeUnit.MILLISECONDS);
@@ -268,7 +268,7 @@
     if (oldState != newState && (newState == ManagerState.NORMAL)) {
       if (fate != null) {
         throw new IllegalStateException("Access to Fate should not have been"
-            + " initialized prior to the Master finishing upgrades. Please save"
+            + " initialized prior to the Manager finishing upgrades. Please save"
             + " all logs and file a bug.");
       }
       upgradeMetadataFuture = upgradeCoordinator.upgradeMetadata(getContext(), nextEvent);
@@ -318,7 +318,7 @@
   // The number of unassigned tablets that should be assigned: displayed on the monitor page
   int displayUnassigned() {
     int result = 0;
-    switch (getMasterState()) {
+    switch (getManagerState()) {
       case NORMAL:
         // Count offline tablets for online tables
         for (TabletGroupWatcher watcher : watchers) {
@@ -374,7 +374,7 @@
   }
 
   Manager(ServerOpts opts, String[] args) throws IOException {
-    super("master", opts, args);
+    super("manager", opts, args);
     ServerContext context = super.getContext();
     balancerEnvironment = new BalancerEnvironmentImpl(context);
 
@@ -483,21 +483,21 @@
     nextEvent.event("Merge state of %s cleared", tableId);
   }
 
-  void setMasterGoalState(ManagerGoalState state) {
+  void setManagerGoalState(ManagerGoalState state) {
     try {
       getContext().getZooReaderWriter().putPersistentData(
-          getZooKeeperRoot() + Constants.ZMASTER_GOAL_STATE, state.name().getBytes(),
+          getZooKeeperRoot() + Constants.ZMANAGER_GOAL_STATE, state.name().getBytes(),
           NodeExistsPolicy.OVERWRITE);
     } catch (Exception ex) {
-      log.error("Unable to set master goal state in zookeeper");
+      log.error("Unable to set manager goal state in zookeeper");
     }
   }
 
-  ManagerGoalState getMasterGoalState() {
+  ManagerGoalState getManagerGoalState() {
     while (true) {
       try {
         byte[] data = getContext().getZooReaderWriter()
-            .getData(getZooKeeperRoot() + Constants.ZMASTER_GOAL_STATE);
+            .getData(getZooKeeperRoot() + Constants.ZMANAGER_GOAL_STATE);
         return ManagerGoalState.valueOf(new String(data));
       } catch (Exception e) {
         log.error("Problem getting real goal state from zookeeper: ", e);
@@ -541,7 +541,7 @@
   }
 
   TabletGoalState getSystemGoalState(TabletLocationState tls) {
-    switch (getMasterState()) {
+    switch (getManagerState()) {
       case NORMAL:
         return TabletGoalState.HOSTED;
       case HAVE_LOCK: // fall-through intended
@@ -561,7 +561,7 @@
       case STOP:
         return TabletGoalState.UNASSIGNED;
       default:
-        throw new IllegalStateException("Unknown Master State");
+        throw new IllegalStateException("Unknown Manager State");
     }
   }
 
@@ -647,7 +647,7 @@
 
     @Override
     public void run() {
-      while (stillMaster()) {
+      while (stillManager()) {
         if (!migrations.isEmpty()) {
           try {
             cleanupOfflineMigrations();
@@ -661,7 +661,7 @@
     }
 
     /**
-     * If a migrating tablet splits, and the tablet dies before sending the master a message, the
+     * If a migrating tablet splits, and the tablet dies before sending the manager a message, the
      * migration will refer to a non-existing tablet, so it can never complete. Periodically scan
      * the metadata table and remove any migrating tablets that no longer exist.
      */
@@ -699,7 +699,7 @@
 
     private boolean goodStats() {
       int start;
-      switch (getMasterState()) {
+      switch (getManagerState()) {
         case UNLOAD_METADATA_TABLETS:
           start = 1;
           break;
@@ -711,9 +711,9 @@
       }
       for (int i = start; i < watchers.size(); i++) {
         TabletGroupWatcher watcher = watchers.get(i);
-        if (watcher.stats.getLastMasterState() != getMasterState()) {
-          log.debug("{}: {} != {}", watcher.getName(), watcher.stats.getLastMasterState(),
-              getMasterState());
+        if (watcher.stats.getLastManagerState() != getManagerState()) {
+          log.debug("{}: {} != {}", watcher.getName(), watcher.stats.getLastManagerState(),
+              getManagerState());
           return false;
         }
       }
@@ -723,32 +723,32 @@
     @Override
     public void run() {
       EventCoordinator.Listener eventListener = nextEvent.getListener();
-      while (stillMaster()) {
+      while (stillManager()) {
         long wait = DEFAULT_WAIT_FOR_WATCHER;
         try {
-          switch (getMasterGoalState()) {
+          switch (getManagerGoalState()) {
             case NORMAL:
-              setMasterState(ManagerState.NORMAL);
+              setManagerState(ManagerState.NORMAL);
               break;
             case SAFE_MODE:
-              if (getMasterState() == ManagerState.NORMAL) {
-                setMasterState(ManagerState.SAFE_MODE);
+              if (getManagerState() == ManagerState.NORMAL) {
+                setManagerState(ManagerState.SAFE_MODE);
               }
-              if (getMasterState() == ManagerState.HAVE_LOCK) {
-                setMasterState(ManagerState.SAFE_MODE);
+              if (getManagerState() == ManagerState.HAVE_LOCK) {
+                setManagerState(ManagerState.SAFE_MODE);
               }
               break;
             case CLEAN_STOP:
-              switch (getMasterState()) {
+              switch (getManagerState()) {
                 case NORMAL:
-                  setMasterState(ManagerState.SAFE_MODE);
+                  setManagerState(ManagerState.SAFE_MODE);
                   break;
                 case SAFE_MODE: {
                   int count = nonMetaDataTabletsAssignedOrHosted();
                   log.debug(
                       String.format("There are %d non-metadata tablets assigned or hosted", count));
                   if (count == 0 && goodStats()) {
-                    setMasterState(ManagerState.UNLOAD_METADATA_TABLETS);
+                    setManagerState(ManagerState.UNLOAD_METADATA_TABLETS);
                   }
                 }
                   break;
@@ -757,7 +757,7 @@
                   log.debug(
                       String.format("There are %d metadata tablets assigned or hosted", count));
                   if (count == 0 && goodStats()) {
-                    setMasterState(ManagerState.UNLOAD_ROOT_TABLET);
+                    setManagerState(ManagerState.UNLOAD_ROOT_TABLET);
                   }
                 }
                   break;
@@ -765,7 +765,7 @@
                   int count = assignedOrHosted(MetadataTable.ID);
                   if (count > 0 && goodStats()) {
                     log.debug(String.format("%d metadata tablets online", count));
-                    setMasterState(ManagerState.UNLOAD_ROOT_TABLET);
+                    setManagerState(ManagerState.UNLOAD_ROOT_TABLET);
                   }
                   int root_count = assignedOrHosted(RootTable.ID);
                   if (root_count > 0 && goodStats()) {
@@ -777,7 +777,7 @@
                     for (TServerInstance server : currentServers) {
                       try {
                         serversToShutdown.add(server);
-                        tserverSet.getConnection(server).fastHalt(masterLock);
+                        tserverSet.getConnection(server).fastHalt(managerLock);
                       } catch (TException e) {
                         // its probably down, and we don't care
                       } finally {
@@ -785,7 +785,7 @@
                       }
                     }
                     if (currentServers.isEmpty()) {
-                      setMasterState(ManagerState.STOP);
+                      setManagerState(ManagerState.STOP);
                     }
                   }
                   break;
@@ -794,7 +794,7 @@
               }
           }
         } catch (Exception t) {
-          log.error("Error occurred reading / switching master goal state. Will"
+          log.error("Error occurred reading / switching manager goal state. Will"
               + " continue with attempt to update status", t);
         }
 
@@ -821,8 +821,8 @@
             badServers.keySet());
       } else if (notHosted() > 0) {
         log.debug("not balancing because there are unhosted tablets: {}", notHosted());
-      } else if (getMasterGoalState() == ManagerGoalState.CLEAN_STOP) {
-        log.debug("not balancing because the master is attempting to stop cleanly");
+      } else if (getManagerGoalState() == ManagerGoalState.CLEAN_STOP) {
+        log.debug("not balancing because the manager is attempting to stop cleanly");
       } else if (!serversToShutdown.isEmpty()) {
         log.debug("not balancing while shutting down servers {}", serversToShutdown);
       } else {
@@ -856,7 +856,7 @@
         try {
           TServerConnection connection = tserverSet.getConnection(instance);
           if (connection != null) {
-            connection.fastHalt(masterLock);
+            connection.fastHalt(managerLock);
           }
         } catch (TException e) {
           log.error("{}", e.getMessage(), e);
@@ -967,7 +967,7 @@
               try {
                 TServerConnection connection2 = tserverSet.getConnection(server);
                 if (connection2 != null) {
-                  connection2.halt(masterLock);
+                  connection2.halt(managerLock);
                 }
               } catch (TTransportException e1) {
                 // ignore: it's probably down
@@ -1015,9 +1015,9 @@
     // ACCUMULO-4424 Put up the Thrift servers before getting the lock as a sign of process health
     // when a hot-standby
     //
-    // Start the Master's Client service
+    // Start the Manager's Client service
     clientHandler = new ManagerClientServiceHandler(this);
-    // Ensure that calls before the master gets the lock fail
+    // Ensure that calls before the manager gets the lock fail
     Iface haProxy = HighlyAvailableServiceWrapper.service(clientHandler, this);
     Iface rpcProxy = TraceUtil.wrapService(haProxy);
     final Processor<Iface> processor;
@@ -1031,20 +1031,20 @@
     ServerAddress sa;
     try {
       sa = TServerUtils.startServer(getMetricsSystem(), context, getHostname(),
-          Property.MANAGER_CLIENTPORT, processor, "Master", "Master Client Service Handler", null,
+          Property.MANAGER_CLIENTPORT, processor, "Manager", "Manager Client Service Handler", null,
           Property.MANAGER_MINTHREADS, Property.MANAGER_MINTHREADS_TIMEOUT,
           Property.MANAGER_THREADCHECK, Property.GENERAL_MAX_MESSAGE_SIZE);
     } catch (UnknownHostException e) {
       throw new IllegalStateException("Unable to start server on host " + getHostname(), e);
     }
     clientService = sa.server;
-    log.info("Started Master client service at {}", sa.address);
+    log.info("Started Manager client service at {}", sa.address);
 
-    // block until we can obtain the ZK lock for the master
+    // block until we can obtain the ZK lock for the manager
     try {
-      getMasterLock(zroot + Constants.ZMASTER_LOCK);
+      getManagerLock(zroot + Constants.ZMANAGER_LOCK);
     } catch (KeeperException | InterruptedException e) {
-      throw new IllegalStateException("Exception getting master lock", e);
+      throw new IllegalStateException("Exception getting manager lock", e);
     }
 
     recoveryManager = new RecoveryManager(this, TIME_TO_CACHE_RECOVERY_WAL_EXISTENCE);
@@ -1146,7 +1146,7 @@
     }
 
     // Make sure that we have a secret key (either a new one or an old one from ZK) before we start
-    // the master client service.
+    // the manager client service.
     Thread authenticationTokenKeyManagerThread = null;
     if (authenticationTokenKeyManager != null && keyDistributor != null) {
       log.info("Starting delegation-token key manager");
@@ -1172,11 +1172,11 @@
     }
 
     String address = sa.address.toString();
-    log.info("Setting master lock data to {}", address);
+    log.info("Setting manager lock data to {}", address);
     try {
-      masterLock.replaceLockData(address.getBytes());
+      managerLock.replaceLockData(address.getBytes());
     } catch (KeeperException | InterruptedException e) {
-      throw new IllegalStateException("Exception updating master lock", e);
+      throw new IllegalStateException("Exception updating manager lock", e);
     }
 
     while (!clientService.isServing()) {
@@ -1211,8 +1211,8 @@
     // checking stored user hashes if any of them uses an outdated algorithm
     security.validateStoredUserCreditentials();
 
-    // The master is fully initialized. Clients are allowed to connect now.
-    masterInitialized.set(true);
+    // The manager is fully initialized. Clients are allowed to connect now.
+    managerInitialized.set(true);
 
     while (clientService.isServing()) {
       sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
@@ -1259,15 +1259,15 @@
   }
 
   /**
-   * Allows property configuration to block master start-up waiting for a minimum number of tservers
-   * to register in zookeeper. It also accepts a maximum time to wait - if the time expires, the
-   * start-up will continue with any tservers available. This check is only performed at master
-   * initialization, when the master acquires the lock. The following properties are used to control
-   * the behaviour:
+   * Allows property configuration to block manager start-up waiting for a minimum number of
+   * tservers to register in zookeeper. It also accepts a maximum time to wait - if the time
+   * expires, the start-up will continue with any tservers available. This check is only performed
+   * at manager initialization, when the manager acquires the lock. The following properties are
+   * used to control the behaviour:
    * <ul>
-   * <li>MASTER_STARTUP_TSERVER_AVAIL_MIN_COUNT - when set to 0 or less, no blocking occurs (default
-   * behaviour) otherwise will block until the number of tservers are available.</li>
-   * <li>MASTER_STARTUP_TSERVER_AVAIL_MAX_WAIT - time to wait in milliseconds. When set to 0 or
+   * <li>MANAGER_STARTUP_TSERVER_AVAIL_MIN_COUNT - when set to 0 or less, no blocking occurs
+   * (default behaviour) otherwise will block until the number of tservers are available.</li>
+   * <li>MANAGER_STARTUP_TSERVER_AVAIL_MAX_WAIT - time to wait in milliseconds. When set to 0 or
    * less, will block indefinitely.</li>
    * </ul>
    *
@@ -1353,7 +1353,7 @@
         new ReplicationCoordinator.Processor<>(TraceUtil.wrapService(haReplicationProxy));
     ServerAddress replAddress = TServerUtils.startServer(getMetricsSystem(), context, getHostname(),
         Property.MANAGER_REPLICATION_COORDINATOR_PORT, replicationCoordinatorProcessor,
-        "Master Replication Coordinator", "Replication Coordinator", null,
+        "Manager Replication Coordinator", "Replication Coordinator", null,
         Property.MANAGER_REPLICATION_COORDINATOR_MINTHREADS, null,
         Property.MANAGER_REPLICATION_COORDINATOR_THREADCHECK, Property.GENERAL_MAX_MESSAGE_SIZE);
 
@@ -1369,7 +1369,7 @@
 
     // Advertise that port we used so peers don't have to be told what it is
     context.getZooReaderWriter().putPersistentData(
-        getZooKeeperRoot() + Constants.ZMASTER_REPLICATION_COORDINATOR_ADDR,
+        getZooKeeperRoot() + Constants.ZMANAGER_REPLICATION_COORDINATOR_ADDR,
         replAddress.address.toString().getBytes(UTF_8), NodeExistsPolicy.OVERWRITE);
     return replAddress.server;
   }
@@ -1378,8 +1378,8 @@
     return Math.max(1, deadline - System.currentTimeMillis());
   }
 
-  public ZooLock getMasterLock() {
-    return masterLock;
+  public ZooLock getManagerLock() {
+    return managerLock;
   }
 
   private static class ManagerLockWatcher implements ZooLock.AccumuloLockWatcher {
@@ -1389,19 +1389,19 @@
 
     @Override
     public void lostLock(LockLossReason reason) {
-      Halt.halt("Master lock in zookeeper lost (reason = " + reason + "), exiting!", -1);
+      Halt.halt("Manager lock in zookeeper lost (reason = " + reason + "), exiting!", -1);
     }
 
     @Override
     public void unableToMonitorLockNode(final Exception e) {
       // ACCUMULO-3651 Changed level to error and added FATAL to message for slf4j compatibility
-      Halt.halt(-1, () -> log.error("FATAL: No longer able to monitor master lock node", e));
+      Halt.halt(-1, () -> log.error("FATAL: No longer able to monitor manager lock node", e));
 
     }
 
     @Override
     public synchronized void acquiredLock() {
-      log.debug("Acquired master lock");
+      log.debug("Acquired manager lock");
 
       if (acquiredLock || failedToAcquireLock) {
         Halt.halt("Zoolock in unexpected state AL " + acquiredLock + " " + failedToAcquireLock, -1);
@@ -1413,10 +1413,10 @@
 
     @Override
     public synchronized void failedToAcquireLock(Exception e) {
-      log.warn("Failed to get master lock", e);
+      log.warn("Failed to get manager lock", e);
 
       if (e instanceof NoAuthException) {
-        String msg = "Failed to acquire master lock due to incorrect ZooKeeper authentication.";
+        String msg = "Failed to acquire manager lock due to incorrect ZooKeeper authentication.";
         log.error("{} Ensure instance.secret is consistent across Accumulo configuration", msg, e);
         Halt.halt(msg, -1);
       }
@@ -1439,19 +1439,20 @@
     }
   }
 
-  private void getMasterLock(final String zMasterLoc) throws KeeperException, InterruptedException {
+  private void getManagerLock(final String zManagerLoc)
+      throws KeeperException, InterruptedException {
     ServerContext context = getContext();
-    log.info("trying to get master lock");
+    log.info("trying to get manager lock");
 
-    final String masterClientAddress =
+    final String managerClientAddress =
         getHostname() + ":" + getConfiguration().getPort(Property.MANAGER_CLIENTPORT)[0];
 
     UUID zooLockUUID = UUID.randomUUID();
     while (true) {
 
       ManagerLockWatcher managerLockWatcher = new ManagerLockWatcher();
-      masterLock = new ZooLock(context.getSiteConfiguration(), zMasterLoc, zooLockUUID);
-      masterLock.lock(managerLockWatcher, masterClientAddress.getBytes());
+      managerLock = new ZooLock(context.getSiteConfiguration(), zManagerLoc, zooLockUUID);
+      managerLock.lock(managerLockWatcher, managerClientAddress.getBytes());
 
       managerLockWatcher.waitForChange();
 
@@ -1460,15 +1461,15 @@
       }
 
       if (!managerLockWatcher.failedToAcquireLock) {
-        throw new IllegalStateException("master lock in unknown state");
+        throw new IllegalStateException("manager lock in unknown state");
       }
 
-      masterLock.tryToCancelAsyncLockOrUnlock();
+      managerLock.tryToCancelAsyncLockOrUnlock();
 
       sleepUninterruptibly(TIME_TO_WAIT_BETWEEN_LOCK_CHECKS, TimeUnit.MILLISECONDS);
     }
 
-    setMasterState(ManagerState.HAVE_LOCK);
+    setManagerState(ManagerState.HAVE_LOCK);
   }
 
   @Override
@@ -1488,7 +1489,7 @@
         if (serversToShutdown.contains(dead)) {
           cause = "clean shutdown"; // maybe an incorrect assumption
         }
-        if (!getMasterGoalState().equals(ManagerGoalState.CLEAN_STOP)) {
+        if (!getManagerGoalState().equals(ManagerGoalState.CLEAN_STOP)) {
           obit.post(dead.getHostPort(), cause);
         }
       }
@@ -1496,7 +1497,7 @@
       Set<TServerInstance> unexpected = new HashSet<>(deleted);
       unexpected.removeAll(this.serversToShutdown);
       if (!unexpected.isEmpty()) {
-        if (stillMaster() && !getMasterGoalState().equals(ManagerGoalState.CLEAN_STOP)) {
+        if (stillManager() && !getManagerGoalState().equals(ManagerGoalState.CLEAN_STOP)) {
           log.warn("Lost servers {}", unexpected);
         }
       }
@@ -1566,11 +1567,11 @@
   @Override
   public Set<TableId> onlineTables() {
     Set<TableId> result = new HashSet<>();
-    if (getMasterState() != ManagerState.NORMAL) {
-      if (getMasterState() != ManagerState.UNLOAD_METADATA_TABLETS) {
+    if (getManagerState() != ManagerState.NORMAL) {
+      if (getManagerState() != ManagerState.UNLOAD_METADATA_TABLETS) {
         result.add(MetadataTable.ID);
       }
-      if (getMasterState() != ManagerState.UNLOAD_ROOT_TABLET) {
+      if (getManagerState() != ManagerState.UNLOAD_ROOT_TABLET) {
         result.add(RootTable.ID);
       }
       return result;
@@ -1619,14 +1620,14 @@
 
   public void assignedTablet(KeyExtent extent) {
     if (extent.isMeta()) {
-      if (getMasterState().equals(ManagerState.UNLOAD_ROOT_TABLET)) {
-        setMasterState(ManagerState.UNLOAD_METADATA_TABLETS);
+      if (getManagerState().equals(ManagerState.UNLOAD_ROOT_TABLET)) {
+        setManagerState(ManagerState.UNLOAD_METADATA_TABLETS);
       }
     }
     if (extent.isRootTablet()) {
       // probably too late, but try anyhow
-      if (getMasterState().equals(ManagerState.STOP)) {
-        setMasterState(ManagerState.UNLOAD_ROOT_TABLET);
+      if (getManagerState().equals(ManagerState.STOP)) {
+        setManagerState(ManagerState.UNLOAD_ROOT_TABLET);
       }
     }
   }
@@ -1647,7 +1648,7 @@
     }
   }
 
-  public ManagerMonitorInfo getMasterMonitorInfo() {
+  public ManagerMonitorInfo getManagerMonitorInfo() {
     final ManagerMonitorInfo result = new ManagerMonitorInfo();
 
     result.tServerInfo = new ArrayList<>();
@@ -1666,8 +1667,8 @@
         result.badTServers.put(bad.getHostPort(), TabletServerState.UNRESPONSIVE.getId());
       }
     }
-    result.state = getMasterState();
-    result.goalState = getMasterGoalState();
+    result.state = getManagerState();
+    result.goalState = getManagerGoalState();
     result.unassignedTablets = displayUnassigned();
     result.serversShuttingDown = new HashSet<>();
     synchronized (serversToShutdown) {
@@ -1713,9 +1714,9 @@
   }
 
   /**
-   * Return how long (in milliseconds) there has been a master overseeing this cluster. This is an
-   * approximately monotonic clock, which will be approximately consistent between different masters
-   * or different runs of the same master.
+   * Return how long (in milliseconds) there has been a manager overseeing this cluster. This is an
+   * approximately monotonic clock, which will be approximately consistent between different
+   * managers or different runs of the same manager.
    */
   public Long getSteadyTime() {
     return timeKeeper.getTime();
@@ -1723,7 +1724,7 @@
 
   @Override
   public boolean isActiveService() {
-    return masterInitialized.get();
+    return managerInitialized.get();
   }
 
   void initializeBalancer() {
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java b/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java
index 694ac03..96919cb 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java
@@ -105,10 +105,10 @@
 
   private static final Logger log = Manager.log;
   private static final Logger drainLog =
-      LoggerFactory.getLogger("org.apache.accumulo.master.MasterDrainImpl");
+      LoggerFactory.getLogger("org.apache.accumulo.manager.ManagerDrainImpl");
 
-  protected ManagerClientServiceHandler(Manager master) {
-    super(master);
+  protected ManagerClientServiceHandler(Manager manager) {
+    super(manager);
   }
 
   @Override
@@ -116,13 +116,13 @@
       throws ThriftSecurityException, ThriftTableOperationException {
     TableId tableId = TableId.of(tableIdStr);
     NamespaceId namespaceId = getNamespaceIdFromTableId(TableOperation.FLUSH, tableId);
-    if (!master.security.canFlush(c, tableId, namespaceId))
+    if (!manager.security.canFlush(c, tableId, namespaceId))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
-    String zTablePath = Constants.ZROOT + "/" + master.getInstanceID() + Constants.ZTABLES + "/"
+    String zTablePath = Constants.ZROOT + "/" + manager.getInstanceID() + Constants.ZTABLES + "/"
         + tableId + Constants.ZTABLE_FLUSH_ID;
 
-    ZooReaderWriter zoo = master.getContext().getZooReaderWriter();
+    ZooReaderWriter zoo = manager.getContext().getZooReaderWriter();
     byte[] fid;
     try {
       fid = zoo.mutateExisting(zTablePath, currentValue -> {
@@ -146,7 +146,7 @@
       throws ThriftSecurityException, ThriftTableOperationException {
     TableId tableId = TableId.of(tableIdStr);
     NamespaceId namespaceId = getNamespaceIdFromTableId(TableOperation.FLUSH, tableId);
-    if (!master.security.canFlush(c, tableId, namespaceId))
+    if (!manager.security.canFlush(c, tableId, namespaceId))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
     Text startRow = ByteBufferUtil.toText(startRowBB);
@@ -156,15 +156,15 @@
       throw new ThriftTableOperationException(tableId.canonical(), null, TableOperation.FLUSH,
           TableOperationExceptionType.BAD_RANGE, "start row must be less than end row");
 
-    Set<TServerInstance> serversToFlush = new HashSet<>(master.tserverSet.getCurrentServers());
+    Set<TServerInstance> serversToFlush = new HashSet<>(manager.tserverSet.getCurrentServers());
 
     for (long l = 0; l < maxLoops; l++) {
 
       for (TServerInstance instance : serversToFlush) {
         try {
-          final TServerConnection server = master.tserverSet.getConnection(instance);
+          final TServerConnection server = manager.tserverSet.getConnection(instance);
           if (server != null)
-            server.flush(master.masterLock, tableId, ByteBufferUtil.toBytes(startRowBB),
+            server.flush(manager.managerLock, tableId, ByteBufferUtil.toBytes(startRowBB),
                 ByteBufferUtil.toBytes(endRowBB));
         } catch (TException ex) {
           Manager.log.error(ex.toString());
@@ -183,7 +183,7 @@
 
       try (TabletsMetadata tablets =
           TabletsMetadata.builder().forTable(tableId).overlapping(startRow, endRow)
-              .fetch(FLUSH_ID, LOCATION, LOGS, PREV_ROW).build(master.getContext())) {
+              .fetch(FLUSH_ID, LOCATION, LOGS, PREV_ROW).build(manager.getContext())) {
         int tabletsToWaitFor = 0;
         int tabletCount = 0;
 
@@ -205,7 +205,7 @@
 
         // TODO detect case of table offline AND tablets w/ logs? - ACCUMULO-1296
 
-        if (tabletCount == 0 && !Tables.exists(master.getContext(), tableId))
+        if (tabletCount == 0 && !Tables.exists(manager.getContext(), tableId))
           throw new ThriftTableOperationException(tableId.canonical(), null, TableOperation.FLUSH,
               TableOperationExceptionType.NOTFOUND, null);
 
@@ -221,7 +221,7 @@
       throws ThriftTableOperationException {
     NamespaceId namespaceId;
     try {
-      namespaceId = Tables.getNamespaceId(master.getContext(), tableId);
+      namespaceId = Tables.getNamespaceId(manager.getContext(), tableId);
     } catch (TableNotFoundException e) {
       throw new ThriftTableOperationException(tableId.canonical(), null, tableOp,
           TableOperationExceptionType.NOTFOUND, e.getMessage());
@@ -231,7 +231,7 @@
 
   @Override
   public ManagerMonitorInfo getManagerStats(TInfo info, TCredentials credentials) {
-    return master.getMasterMonitorInfo();
+    return manager.getManagerMonitorInfo();
   }
 
   @Override
@@ -249,40 +249,40 @@
   @Override
   public void shutdown(TInfo info, TCredentials c, boolean stopTabletServers)
       throws ThriftSecurityException {
-    if (!master.security.canPerformSystemActions(c))
+    if (!manager.security.canPerformSystemActions(c))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
     if (stopTabletServers) {
-      master.setMasterGoalState(ManagerGoalState.CLEAN_STOP);
-      EventCoordinator.Listener eventListener = master.nextEvent.getListener();
+      manager.setManagerGoalState(ManagerGoalState.CLEAN_STOP);
+      EventCoordinator.Listener eventListener = manager.nextEvent.getListener();
       do {
         eventListener.waitForEvents(Manager.ONE_SECOND);
-      } while (master.tserverSet.size() > 0);
+      } while (manager.tserverSet.size() > 0);
     }
-    master.setMasterState(ManagerState.STOP);
+    manager.setManagerState(ManagerState.STOP);
   }
 
   @Override
   public void shutdownTabletServer(TInfo info, TCredentials c, String tabletServer, boolean force)
       throws ThriftSecurityException {
-    if (!master.security.canPerformSystemActions(c))
+    if (!manager.security.canPerformSystemActions(c))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
-    final TServerInstance doomed = master.tserverSet.find(tabletServer);
+    final TServerInstance doomed = manager.tserverSet.find(tabletServer);
     if (!force) {
-      final TServerConnection server = master.tserverSet.getConnection(doomed);
+      final TServerConnection server = manager.tserverSet.getConnection(doomed);
       if (server == null) {
         Manager.log.warn("No server found for name {}", tabletServer);
         return;
       }
     }
 
-    long tid = master.fate.startTransaction();
+    long tid = manager.fate.startTransaction();
 
     log.debug("Seeding FATE op to shutdown " + tabletServer + " with tid " + tid);
 
-    master.fate.seedTransaction(tid, new TraceRepo<>(new ShutdownTServer(doomed, force)), false);
-    master.fate.waitForCompletion(tid);
-    master.fate.delete(tid);
+    manager.fate.seedTransaction(tid, new TraceRepo<>(new ShutdownTServer(doomed, force)), false);
+    manager.fate.waitForCompletion(tid);
+    manager.fate.delete(tid);
 
     log.debug("FATE op shutting down " + tabletServer + " finished");
   }
@@ -291,12 +291,12 @@
   public void reportSplitExtent(TInfo info, TCredentials credentials, String serverName,
       TabletSplit split) {
     KeyExtent oldTablet = KeyExtent.fromThrift(split.oldTablet);
-    if (master.migrations.remove(oldTablet) != null) {
+    if (manager.migrations.remove(oldTablet) != null) {
       Manager.log.info("Canceled migration of {}", split.oldTablet);
     }
-    for (TServerInstance instance : master.tserverSet.getCurrentServers()) {
+    for (TServerInstance instance : manager.tserverSet.getCurrentServers()) {
       if (serverName.equals(instance.getHostPort())) {
-        master.nextEvent.event("%s reported split %s, %s", serverName,
+        manager.nextEvent.event("%s reported split %s, %s", serverName,
             KeyExtent.fromThrift(split.newTablets.get(0)),
             KeyExtent.fromThrift(split.newTablets.get(1)));
         return;
@@ -315,10 +315,10 @@
         Manager.log.error("{} reports assignment failed for tablet {}", serverName, tablet);
         break;
       case LOADED:
-        master.nextEvent.event("tablet %s was loaded on %s", tablet, serverName);
+        manager.nextEvent.event("tablet %s was loaded on %s", tablet, serverName);
         break;
       case UNLOADED:
-        master.nextEvent.event("tablet %s was unloaded from %s", tablet, serverName);
+        manager.nextEvent.event("tablet %s was unloaded from %s", tablet, serverName);
         break;
       case UNLOAD_ERROR:
         Manager.log.error("{} reports unload failed for tablet {}", serverName, tablet);
@@ -330,7 +330,7 @@
         }
         break;
       case CHOPPED:
-        master.nextEvent.event("tablet %s chopped", tablet);
+        manager.nextEvent.event("tablet %s chopped", tablet);
         break;
     }
   }
@@ -338,20 +338,20 @@
   @Override
   public void setManagerGoalState(TInfo info, TCredentials c, ManagerGoalState state)
       throws ThriftSecurityException {
-    if (!master.security.canPerformSystemActions(c))
+    if (!manager.security.canPerformSystemActions(c))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
-    master.setMasterGoalState(state);
+    manager.setManagerGoalState(state);
   }
 
   @Override
   public void removeSystemProperty(TInfo info, TCredentials c, String property)
       throws ThriftSecurityException {
-    if (!master.security.canPerformSystemActions(c))
+    if (!manager.security.canPerformSystemActions(c))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
     try {
-      SystemPropUtil.removeSystemProperty(master.getContext(), property);
+      SystemPropUtil.removeSystemProperty(manager.getContext(), property);
       updatePlugins(property);
     } catch (Exception e) {
       Manager.log.error("Problem removing config property in zookeeper", e);
@@ -362,11 +362,11 @@
   @Override
   public void setSystemProperty(TInfo info, TCredentials c, String property, String value)
       throws TException {
-    if (!master.security.canPerformSystemActions(c))
+    if (!manager.security.canPerformSystemActions(c))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
     try {
-      SystemPropUtil.setSystemProperty(master.getContext(), property, value);
+      SystemPropUtil.setSystemProperty(manager.getContext(), property, value);
       updatePlugins(property);
     } catch (IllegalArgumentException iae) {
       // throw the exception here so it is not caught and converted to a generic TException
@@ -394,21 +394,21 @@
       throws ThriftSecurityException, ThriftTableOperationException {
 
     NamespaceId namespaceId =
-        ClientServiceHandler.checkNamespaceId(master.getContext(), namespace, op);
+        ClientServiceHandler.checkNamespaceId(manager.getContext(), namespace, op);
 
-    if (!master.security.canAlterNamespace(c, namespaceId))
+    if (!manager.security.canAlterNamespace(c, namespaceId))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
     try {
       if (value == null) {
-        NamespacePropUtil.removeNamespaceProperty(master.getContext(), namespaceId, property);
+        NamespacePropUtil.removeNamespaceProperty(manager.getContext(), namespaceId, property);
       } else {
-        NamespacePropUtil.setNamespaceProperty(master.getContext(), namespaceId, property, value);
+        NamespacePropUtil.setNamespaceProperty(manager.getContext(), namespaceId, property, value);
       }
     } catch (KeeperException.NoNodeException e) {
       // race condition... namespace no longer exists? This call will throw an exception if the
       // namespace was deleted:
-      ClientServiceHandler.checkNamespaceId(master.getContext(), namespace, op);
+      ClientServiceHandler.checkNamespaceId(manager.getContext(), namespace, op);
       log.info("Error altering namespace property", e);
       throw new ThriftTableOperationException(namespaceId.canonical(), namespace, op,
           TableOperationExceptionType.OTHER, "Problem altering namespaceproperty");
@@ -421,21 +421,21 @@
 
   private void alterTableProperty(TCredentials c, String tableName, String property, String value,
       TableOperation op) throws ThriftSecurityException, ThriftTableOperationException {
-    final TableId tableId = ClientServiceHandler.checkTableId(master.getContext(), tableName, op);
+    final TableId tableId = ClientServiceHandler.checkTableId(manager.getContext(), tableName, op);
     NamespaceId namespaceId = getNamespaceIdFromTableId(op, tableId);
-    if (!master.security.canAlterTable(c, tableId, namespaceId))
+    if (!manager.security.canAlterTable(c, tableId, namespaceId))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
     try {
       if (value == null || value.isEmpty()) {
-        TablePropUtil.removeTableProperty(master.getContext(), tableId, property);
-      } else if (!TablePropUtil.setTableProperty(master.getContext(), tableId, property, value)) {
+        TablePropUtil.removeTableProperty(manager.getContext(), tableId, property);
+      } else if (!TablePropUtil.setTableProperty(manager.getContext(), tableId, property, value)) {
         throw new Exception("Invalid table property.");
       }
     } catch (KeeperException.NoNodeException e) {
       // race condition... table no longer exists? This call will throw an exception if the table
       // was deleted:
-      ClientServiceHandler.checkTableId(master.getContext(), tableName, op);
+      ClientServiceHandler.checkTableId(manager.getContext(), tableName, op);
       log.info("Error altering table property", e);
       throw new ThriftTableOperationException(tableId.canonical(), tableName, op,
           TableOperationExceptionType.OTHER, "Problem altering table property");
@@ -450,19 +450,19 @@
     // resolve without warning; any warnings should have already occurred
     String resolved = DeprecatedPropertyUtil.getReplacementName(property, (log, replacement) -> {});
     if (resolved.equals(Property.MANAGER_TABLET_BALANCER.getKey())) {
-      master.initializeBalancer();
-      log.info("tablet balancer changed to {}", master.getBalancerClass().getName());
+      manager.initializeBalancer();
+      log.info("tablet balancer changed to {}", manager.getBalancerClass().getName());
     }
   }
 
   @Override
   public void waitForBalance(TInfo tinfo) {
-    master.waitForBalance();
+    manager.waitForBalance();
   }
 
   @Override
   public List<String> getActiveTservers(TInfo tinfo, TCredentials credentials) {
-    Set<TServerInstance> tserverInstances = master.onlineTabletServers();
+    Set<TServerInstance> tserverInstances = manager.onlineTabletServers();
     List<String> servers = new ArrayList<>();
     for (TServerInstance tserverInstance : tserverInstances) {
       servers.add(tserverInstance.getHostPort());
@@ -474,19 +474,19 @@
   @Override
   public TDelegationToken getDelegationToken(TInfo tinfo, TCredentials credentials,
       TDelegationTokenConfig tConfig) throws ThriftSecurityException, TException {
-    if (!master.security.canObtainDelegationToken(credentials)) {
+    if (!manager.security.canObtainDelegationToken(credentials)) {
       throw new ThriftSecurityException(credentials.getPrincipal(),
           SecurityErrorCode.PERMISSION_DENIED);
     }
 
     // Make sure we're actually generating the secrets to make delegation tokens
     // Round-about way to verify that SASL is also enabled.
-    if (!master.delegationTokensAvailable()) {
+    if (!manager.delegationTokensAvailable()) {
       throw new TException("Delegation tokens are not available for use");
     }
 
     final DelegationTokenConfig config = DelegationTokenConfigSerializer.deserialize(tConfig);
-    final AuthenticationTokenSecretManager secretManager = master.getContext().getSecretManager();
+    final AuthenticationTokenSecretManager secretManager = manager.getContext().getSecretManager();
     try {
       Entry<Token<AuthenticationTokenIdentifier>,AuthenticationTokenIdentifier> pair =
           secretManager.generateToken(credentials.principal, config);
@@ -501,9 +501,9 @@
   @Override
   public boolean drainReplicationTable(TInfo tfino, TCredentials credentials, String tableName,
       Set<String> logsToWatch) throws TException {
-    AccumuloClient client = master.getContext();
+    AccumuloClient client = manager.getContext();
 
-    final Text tableId = new Text(getTableId(master.getContext(), tableName).canonical());
+    final Text tableId = new Text(getTableId(manager.getContext(), tableName).canonical());
 
     drainLog.trace("Waiting for {} to be replicated for {}", logsToWatch, tableId);
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/ManagerTime.java b/server/manager/src/main/java/org/apache/accumulo/manager/ManagerTime.java
index a99f110..c168d7e 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/ManagerTime.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/ManagerTime.java
@@ -36,14 +36,14 @@
 import org.slf4j.LoggerFactory;
 
 /**
- * Keep a persistent roughly monotone view of how long a master has been overseeing this cluster.
+ * Keep a persistent roughly monotone view of how long a manager has been overseeing this cluster.
  */
 public class ManagerTime {
   private static final Logger log = LoggerFactory.getLogger(ManagerTime.class);
 
   private final String zPath;
   private final ZooReaderWriter zk;
-  private final Manager master;
+  private final Manager manager;
 
   /**
    * Difference between time stored in ZooKeeper and System.nanoTime() when we last read from
@@ -51,37 +51,37 @@
    */
   private final AtomicLong skewAmount;
 
-  public ManagerTime(Manager master, AccumuloConfiguration conf) throws IOException {
-    this.zPath = master.getZooKeeperRoot() + Constants.ZMASTER_TICK;
-    this.zk = master.getContext().getZooReaderWriter();
-    this.master = master;
+  public ManagerTime(Manager manager, AccumuloConfiguration conf) throws IOException {
+    this.zPath = manager.getZooKeeperRoot() + Constants.ZMANAGER_TICK;
+    this.zk = manager.getContext().getZooReaderWriter();
+    this.manager = manager;
 
     try {
       zk.putPersistentData(zPath, "0".getBytes(UTF_8), NodeExistsPolicy.SKIP);
       skewAmount =
           new AtomicLong(Long.parseLong(new String(zk.getData(zPath), UTF_8)) - System.nanoTime());
     } catch (Exception ex) {
-      throw new IOException("Error updating master time", ex);
+      throw new IOException("Error updating manager time", ex);
     }
 
     ThreadPools.createGeneralScheduledExecutorService(conf).scheduleWithFixedDelay(
-        Threads.createNamedRunnable("Master time keeper", () -> run()), 0,
+        Threads.createNamedRunnable("Manager time keeper", () -> run()), 0,
         MILLISECONDS.convert(10, SECONDS), MILLISECONDS);
   }
 
   /**
-   * How long has this cluster had a Master?
+   * How long has this cluster had a Manager?
    *
-   * @return Approximate total duration this cluster has had a Master, in milliseconds.
+   * @return Approximate total duration this cluster has had a Manager, in milliseconds.
    */
   public long getTime() {
     return MILLISECONDS.convert(System.nanoTime() + skewAmount.get(), NANOSECONDS);
   }
 
   public void run() {
-    switch (master.getMasterState()) {
+    switch (manager.getManagerState()) {
       // If we don't have the lock, periodically re-read the value in ZooKeeper, in case there's
-      // another master we're
+      // another manager we're
       // shadowing for.
       case INITIAL:
       case STOP:
@@ -90,7 +90,7 @@
           skewAmount.set(zkTime - System.nanoTime());
         } catch (Exception ex) {
           if (log.isDebugEnabled()) {
-            log.debug("Failed to retrieve master tick time", ex);
+            log.debug("Failed to retrieve manager tick time", ex);
           }
         }
         break;
@@ -106,7 +106,7 @@
               NodeExistsPolicy.OVERWRITE);
         } catch (Exception ex) {
           if (log.isDebugEnabled()) {
-            log.debug("Failed to update master tick time", ex);
+            log.debug("Failed to update manager tick time", ex);
           }
         }
     }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java b/server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java
index cc968b7..65ffbe1 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java
@@ -102,14 +102,14 @@
 abstract class TabletGroupWatcher extends Thread {
   // Constants used to make sure assignment logging isn't excessive in quantity or size
 
-  private final Manager master;
+  private final Manager manager;
   private final TabletStateStore store;
   private final TabletGroupWatcher dependentWatcher;
   final TableStats stats = new TableStats();
   private SortedSet<TServerInstance> lastScanServers = ImmutableSortedSet.of();
 
-  TabletGroupWatcher(Manager master, TabletStateStore store, TabletGroupWatcher dependentWatcher) {
-    this.master = master;
+  TabletGroupWatcher(Manager manager, TabletStateStore store, TabletGroupWatcher dependentWatcher) {
+    this.manager = manager;
     this.store = store;
     this.dependentWatcher = dependentWatcher;
     setName("Watching " + store.name());
@@ -169,11 +169,11 @@
   @Override
   public void run() {
     int[] oldCounts = new int[TabletState.values().length];
-    EventCoordinator.Listener eventListener = this.master.nextEvent.getListener();
+    EventCoordinator.Listener eventListener = this.manager.nextEvent.getListener();
 
-    WalStateManager wals = new WalStateManager(master.getContext());
+    WalStateManager wals = new WalStateManager(manager.getContext());
 
-    while (master.stillMaster()) {
+    while (manager.stillManager()) {
       // slow things down a little, otherwise we spam the logs when there are many wake-up events
       sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
 
@@ -183,7 +183,7 @@
       try {
         Map<TableId,MergeStats> mergeStatsCache = new HashMap<>();
         Map<TableId,MergeStats> currentMerges = new HashMap<>();
-        for (MergeInfo merge : master.merges()) {
+        for (MergeInfo merge : manager.merges()) {
           if (merge.getExtent() != null) {
             currentMerges.put(merge.getExtent().tableId(), new MergeStats(merge));
           }
@@ -191,8 +191,8 @@
 
         // Get the current status for the current list of tservers
         SortedMap<TServerInstance,TabletServerStatus> currentTServers = new TreeMap<>();
-        for (TServerInstance entry : master.tserverSet.getCurrentServers()) {
-          currentTServers.put(entry, master.tserverStatus.get(entry));
+        for (TServerInstance entry : manager.tserverSet.getCurrentServers()) {
+          currentTServers.put(entry, manager.tserverStatus.get(entry));
         }
 
         if (currentTServers.isEmpty()) {
@@ -203,9 +203,9 @@
           continue;
         }
 
-        TabletLists tLists = new TabletLists(master, currentTServers);
+        TabletLists tLists = new TabletLists(manager, currentTServers);
 
-        ManagerState managerState = master.getMasterState();
+        ManagerState managerState = manager.getManagerState();
         int[] counts = new int[TabletState.values().length];
         stats.begin();
         // Walk through the tablets in our store, and work tablets
@@ -218,7 +218,7 @@
           }
 
           // ignore entries for tables that do not exist in zookeeper
-          if (master.getTableManager().getTableState(tls.extent.tableId()) == null)
+          if (manager.getTableManager().getTableState(tls.extent.tableId()) == null)
             continue;
 
           // Don't overwhelm the tablet servers with work
@@ -230,13 +230,13 @@
             eventListener.waitForEvents(Manager.TIME_TO_WAIT_BETWEEN_SCANS);
           }
           TableId tableId = tls.extent.tableId();
-          TableConfiguration tableConf = master.getContext().getTableConfiguration(tableId);
+          TableConfiguration tableConf = manager.getContext().getTableConfiguration(tableId);
 
           MergeStats mergeStats = mergeStatsCache.computeIfAbsent(tableId, k -> {
             var mStats = currentMerges.get(k);
             return mStats != null ? mStats : new MergeStats(new MergeInfo());
           });
-          TabletGoalState goal = master.getGoalState(tls, mergeStats.getMergeInfo());
+          TabletGoalState goal = manager.getGoalState(tls, mergeStats.getMergeInfo());
           TServerInstance location = tls.getLocation();
           TabletState state = tls.getState(currentTServers.keySet());
 
@@ -255,7 +255,7 @@
 
           // if we are shutting down all the tabletservers, we have to do it in order
           if (goal == TabletGoalState.SUSPENDED && state == TabletState.HOSTED) {
-            if (master.serversToShutdown.equals(currentTServers.keySet())) {
+            if (manager.serversToShutdown.equals(currentTServers.keySet())) {
               if (dependentWatcher != null && dependentWatcher.assignedOrHosted() > 0) {
                 goal = TabletGoalState.HOSTED;
               }
@@ -264,13 +264,13 @@
 
           if (goal == TabletGoalState.HOSTED) {
             if (state != TabletState.HOSTED && !tls.walogs.isEmpty()) {
-              if (master.recoveryManager.recoverLogs(tls.extent, tls.walogs))
+              if (manager.recoveryManager.recoverLogs(tls.extent, tls.walogs))
                 continue;
             }
             switch (state) {
               case HOSTED:
-                if (location.equals(master.migrations.get(tls.extent)))
-                  master.migrations.remove(tls.extent);
+                if (location.equals(manager.migrations.get(tls.extent)))
+                  manager.migrations.remove(tls.extent);
                 break;
               case ASSIGNED_TO_DEAD_SERVER:
                 hostDeadTablet(tLists, tls, location, wals);
@@ -300,10 +300,10 @@
                 unassignDeadTablet(tLists, tls, wals);
                 break;
               case HOSTED:
-                TServerConnection client = master.tserverSet.getConnection(location);
+                TServerConnection client = manager.tserverSet.getConnection(location);
                 if (client != null) {
-                  client.unloadTablet(master.masterLock, tls.extent, goal.howUnload(),
-                      master.getSteadyTime());
+                  client.unloadTablet(manager.managerLock, tls.extent, goal.howUnload(),
+                      manager.getSteadyTime());
                   unloaded++;
                   totalUnloaded++;
                 } else {
@@ -326,7 +326,7 @@
         for (TabletState state : TabletState.values()) {
           int i = state.ordinal();
           if (counts[i] > 0 && counts[i] != oldCounts[i]) {
-            master.nextEvent.event("[%s]: %d tablets are %s", store.name(), counts[i],
+            manager.nextEvent.event("[%s]: %d tablets are %s", store.name(), counts[i],
                 state.name());
           }
         }
@@ -334,7 +334,7 @@
             stats.getScanTime() / 1000.));
         oldCounts = counts;
         if (totalUnloaded > 0) {
-          master.nextEvent.event("[%s]: %d tablets unloaded", store.name(), totalUnloaded);
+          manager.nextEvent.event("[%s]: %d tablets unloaded", store.name(), totalUnloaded);
         }
 
         updateMergeState(mergeStatsCache);
@@ -342,7 +342,7 @@
         synchronized (this) {
           lastScanServers = ImmutableSortedSet.copyOf(currentTServers.keySet());
         }
-        if (master.tserverSet.getCurrentServers().equals(currentTServers.keySet())) {
+        if (manager.tserverSet.getCurrentServers().equals(currentTServers.keySet())) {
           Manager.log.debug(String.format("[%s] sleeping for %.2f seconds", store.name(),
               Manager.TIME_TO_WAIT_BETWEEN_SCANS / 1000.));
           eventListener.waitForEvents(Manager.TIME_TO_WAIT_BETWEEN_SCANS);
@@ -380,14 +380,14 @@
   private void hostUnassignedTablet(TabletLists tLists, KeyExtent tablet,
       TServerInstance location) {
     // maybe it's a finishing migration
-    TServerInstance dest = master.migrations.get(tablet);
+    TServerInstance dest = manager.migrations.get(tablet);
     if (dest != null) {
       // if destination is still good, assign it
       if (tLists.destinations.containsKey(dest)) {
         tLists.assignments.add(new Assignment(tablet, dest));
       } else {
         // get rid of this migration
-        master.migrations.remove(tablet);
+        manager.migrations.remove(tablet);
         tLists.unassigned.put(tablet, location);
       }
     } else {
@@ -397,7 +397,7 @@
 
   private void hostSuspendedTablet(TabletLists tLists, TabletLocationState tls,
       TServerInstance location, TableConfiguration tableConf) {
-    if (master.getSteadyTime() - tls.suspend.suspensionTime
+    if (manager.getSteadyTime() - tls.suspend.suspensionTime
         < tableConf.getTimeInMillis(Property.TABLE_SUSPEND_DURATION)) {
       // Tablet is suspended. See if its tablet server is back.
       TServerInstance returnInstance = null;
@@ -425,8 +425,8 @@
   private void hostDeadTablet(TabletLists tLists, TabletLocationState tls, TServerInstance location,
       WalStateManager wals) throws WalMarkerException {
     tLists.assignedToDeadServers.add(tls);
-    if (location.equals(master.migrations.get(tls.extent)))
-      master.migrations.remove(tls.extent);
+    if (location.equals(manager.migrations.get(tls.extent)))
+      manager.migrations.remove(tls.extent);
     TServerInstance tserver = tls.futureOrCurrent();
     if (!tLists.logsForDeadServers.containsKey(tserver)) {
       tLists.logsForDeadServers.put(tserver, wals.getWalsInUse(tserver));
@@ -434,10 +434,10 @@
   }
 
   private void cancelOfflineTableMigrations(KeyExtent extent) {
-    TServerInstance dest = master.migrations.get(extent);
-    TableState tableState = master.getTableManager().getTableState(extent.tableId());
+    TServerInstance dest = manager.migrations.get(extent);
+    TableState tableState = manager.getTableManager().getTableState(extent.tableId());
     if (dest != null && tableState == TableState.OFFLINE) {
-      master.migrations.remove(extent);
+      manager.migrations.remove(extent);
     }
   }
 
@@ -453,7 +453,7 @@
       String table = MetadataTable.NAME;
       if (extent.isMeta())
         table = RootTable.NAME;
-      Scanner scanner = master.getContext().createScanner(table, Authorizations.EMPTY);
+      Scanner scanner = manager.getContext().createScanner(table, Authorizations.EMPTY);
       scanner.fetchColumnFamily(CurrentLocationColumnFamily.NAME);
       scanner.fetchColumnFamily(FutureLocationColumnFamily.NAME);
       scanner.setRange(new Range(row));
@@ -479,10 +479,10 @@
           Iterators.concat(future.entrySet().iterator(), assigned.entrySet().iterator());
       while (iter.hasNext()) {
         Entry<Key,Value> entry = iter.next();
-        TServerInstance alive = master.tserverSet.find(entry.getValue().toString());
+        TServerInstance alive = manager.tserverSet.find(entry.getValue().toString());
         if (alive == null) {
           Manager.log.info("Removing entry  {}", entry);
-          BatchWriter bw = master.getContext().createBatchWriter(table, new BatchWriterConfig());
+          BatchWriter bw = manager.getContext().createBatchWriter(table, new BatchWriterConfig());
           Mutation m = new Mutation(entry.getKey().getRow());
           m.putDelete(entry.getKey().getColumnFamily(), entry.getKey().getColumnQualifier());
           bw.addMutation(m);
@@ -530,7 +530,7 @@
           continue;
         try {
           TServerConnection conn;
-          conn = master.tserverSet.getConnection(tls.current);
+          conn = manager.tserverSet.getConnection(tls.current);
           if (conn != null) {
             Manager.log.info("Asking {} to split {} at {}", tls.current, tls.extent, splitPoint);
             conn.splitTablet(tls.extent, splitPoint);
@@ -560,10 +560,10 @@
     if (info.needsToBeChopped(tls.extent)) {
       TServerConnection conn;
       try {
-        conn = master.tserverSet.getConnection(tls.current);
+        conn = manager.tserverSet.getConnection(tls.current);
         if (conn != null) {
           Manager.log.info("Asking {} to chop {}", tls.current, tls.extent);
-          conn.chop(master.masterLock, tls.extent);
+          conn.chop(manager.managerLock, tls.extent);
         } else {
           Manager.log.warn("Could not connect to server {}", tls.current);
         }
@@ -576,7 +576,7 @@
   private void updateMergeState(Map<TableId,MergeStats> mergeStatsCache) {
     for (MergeStats stats : mergeStatsCache.values()) {
       try {
-        MergeState update = stats.nextMergeState(master.getContext(), master);
+        MergeState update = stats.nextMergeState(manager.getContext(), manager);
         // when next state is MERGING, its important to persist this before
         // starting the merge... the verification check that is done before
         // moving into the merging state could fail if merge starts but does
@@ -584,7 +584,7 @@
         if (update == MergeState.COMPLETE)
           update = MergeState.NONE;
         if (update != stats.getMergeInfo().getState()) {
-          master.setMergeState(stats.getMergeInfo(), update);
+          manager.setMergeState(stats.getMergeInfo(), update);
         }
 
         if (update == MergeState.MERGING) {
@@ -595,7 +595,7 @@
               mergeMetadataRecords(stats.getMergeInfo());
             }
             update = MergeState.COMPLETE;
-            master.setMergeState(stats.getMergeInfo(), update);
+            manager.setMergeState(stats.getMergeInfo(), update);
           } catch (Exception ex) {
             Manager.log.error("Unable merge metadata table records", ex);
           }
@@ -620,8 +620,8 @@
       Manager.log.debug("Found following tablet {}", followingTablet);
     }
     try {
-      AccumuloClient client = master.getContext();
-      ServerContext context = master.getContext();
+      AccumuloClient client = manager.getContext();
+      ServerContext context = manager.getContext();
       Ample ample = context.getAmple();
       Text start = extent.prevEndRow();
       if (start == null) {
@@ -684,8 +684,8 @@
       } else {
         // Recreate the default tablet to hold the end of the table
         MetadataTableUtil.addTablet(new KeyExtent(extent.tableId(), null, extent.prevEndRow()),
-            ServerColumnFamily.DEFAULT_TABLET_DIR_NAME, master.getContext(), metadataTime.getType(),
-            master.masterLock);
+            ServerColumnFamily.DEFAULT_TABLET_DIR_NAME, manager.getContext(),
+            metadataTime.getType(), manager.managerLock);
       }
     } catch (RuntimeException | TableNotFoundException ex) {
       throw new AccumuloException(ex);
@@ -710,7 +710,7 @@
       targetSystemTable = RootTable.NAME;
     }
 
-    AccumuloClient client = master.getContext();
+    AccumuloClient client = manager.getContext();
 
     try (BatchWriter bw = client.createBatchWriter(targetSystemTable, new BatchWriterConfig())) {
       long fileCount = 0;
@@ -739,7 +739,7 @@
         } else if (ServerColumnFamily.DIRECTORY_COLUMN.hasColumns(key)) {
           String uri =
               GcVolumeUtil.getDeleteTabletOnAllVolumesUri(range.tableId(), value.toString());
-          bw.addMutation(master.getContext().getAmple().createDeleteMutation(uri));
+          bw.addMutation(manager.getContext().getAmple().createDeleteMutation(uri));
         }
       }
 
@@ -825,7 +825,7 @@
 
   private KeyExtent getHighTablet(KeyExtent range) throws AccumuloException {
     try {
-      AccumuloClient client = master.getContext();
+      AccumuloClient client = manager.getContext();
       Scanner scanner = client.createScanner(range.isMeta() ? RootTable.NAME : MetadataTable.NAME,
           Authorizations.EMPTY);
       TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
@@ -858,12 +858,12 @@
           deadTablets.subList(0, maxServersToShow));
       Manager.log.debug("logs for dead servers: {}", deadLogs);
       if (canSuspendTablets()) {
-        store.suspend(deadTablets, deadLogs, master.getSteadyTime());
+        store.suspend(deadTablets, deadLogs, manager.getSteadyTime());
       } else {
         store.unassign(deadTablets, deadLogs);
       }
       markDeadServerLogsAsClosed(wals, deadLogs);
-      master.nextEvent.event(
+      manager.nextEvent.event(
           "Marked %d tablets as suspended because they don't have current servers",
           deadTablets.size());
     }
@@ -879,7 +879,7 @@
       Map<KeyExtent,TServerInstance> unassigned) {
     if (!tLists.currentTServers.isEmpty()) {
       Map<KeyExtent,TServerInstance> assignedOut = new HashMap<>();
-      master.getAssignments(tLists.currentTServers, unassigned, assignedOut);
+      manager.getAssignments(tLists.currentTServers, unassigned, assignedOut);
       for (Entry<KeyExtent,TServerInstance> assignment : assignedOut.entrySet()) {
         if (unassigned.containsKey(assignment.getKey())) {
           if (assignment.getValue() != null) {
@@ -918,13 +918,13 @@
     }
     tLists.assignments.addAll(tLists.assigned);
     for (Assignment a : tLists.assignments) {
-      TServerConnection client = master.tserverSet.getConnection(a.server);
+      TServerConnection client = manager.tserverSet.getConnection(a.server);
       if (client != null) {
-        client.assignTablet(master.masterLock, a.tablet);
+        client.assignTablet(manager.managerLock, a.tablet);
       } else {
         Manager.log.warn("Could not connect to server {}", a.server);
       }
-      master.assignedTablet(a.tablet);
+      manager.assignedTablet(a.tablet);
     }
   }
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ManagerMetrics.java b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ManagerMetrics.java
index d8749d4b..1430386 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ManagerMetrics.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ManagerMetrics.java
@@ -23,7 +23,7 @@
 public abstract class ManagerMetrics extends Metrics {
 
   protected ManagerMetrics(String subName, String description, String record) {
-    super("Master,sub=" + subName, description, "master", record);
+    super("Manager,sub=" + subName, description, "manager", record);
   }
 
 }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ManagerMetricsFactory.java b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ManagerMetricsFactory.java
index d66e672..8861f47 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ManagerMetricsFactory.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ManagerMetricsFactory.java
@@ -29,9 +29,9 @@
 import org.slf4j.LoggerFactory;
 
 /**
- * Provide master metrics configuration. Currently this is replication and FATE metrics. Metrics can
- * be configured using hadoop metrics2 Fate metrics must be enabled via configuration file (default
- * is disabled)
+ * Provide manager metrics configuration. Currently this is replication and FATE metrics. Metrics
+ * can be configured using hadoop metrics2 Fate metrics must be enabled via configuration file
+ * (default is disabled)
  */
 public class ManagerMetricsFactory {
 
@@ -46,13 +46,13 @@
     fateMinUpdateInterval = conf.getTimeInMillis(Property.MANAGER_FATE_METRICS_MIN_UPDATE_INTERVAL);
   }
 
-  public int register(Manager master) {
-    MetricsSystem metricsSystem = master.getMetricsSystem();
+  public int register(Manager manager) {
+    MetricsSystem metricsSystem = manager.getMetricsSystem();
 
     int failureCount = 0;
 
     try {
-      new ReplicationMetrics(master).register(metricsSystem);
+      new ReplicationMetrics(manager).register(metricsSystem);
       log.info("Registered replication metrics module");
     } catch (Exception ex) {
       failureCount++;
@@ -61,7 +61,7 @@
 
     try {
       if (enableFateMetrics) {
-        new FateMetrics(master.getContext(), fateMinUpdateInterval).register(metricsSystem);
+        new FateMetrics(manager.getContext(), fateMinUpdateInterval).register(metricsSystem);
         log.info("Registered FATE metrics module");
       }
     } catch (Exception ex) {
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ReplicationMetrics.java b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ReplicationMetrics.java
index db5c673..2778daf 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ReplicationMetrics.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/ReplicationMetrics.java
@@ -41,19 +41,19 @@
 
   private static final Logger log = LoggerFactory.getLogger(ReplicationMetrics.class);
 
-  private final Manager master;
+  private final Manager manager;
   private final ReplicationUtil replicationUtil;
   private final MutableQuantiles replicationQueueTimeQuantiles;
   private final MutableStat replicationQueueTimeStat;
   private final Map<Path,Long> pathModTimes;
 
-  ReplicationMetrics(Manager master) {
-    super("Replication", "Data-Center Replication Metrics", "MasterReplication");
-    this.master = master;
+  ReplicationMetrics(Manager manager) {
+    super("Replication", "Data-Center Replication Metrics", "ManagerReplication");
+    this.manager = manager;
 
     pathModTimes = new HashMap<>();
 
-    replicationUtil = new ReplicationUtil(master.getContext());
+    replicationUtil = new ReplicationUtil(manager.getContext());
     MetricsRegistry registry = super.getRegistry();
     replicationQueueTimeQuantiles = registry.newQuantiles("replicationQueue10m",
         "Replication queue time quantiles in milliseconds", "ops", "latency", 600);
@@ -65,7 +65,7 @@
   protected void prepareMetrics() {
     final String PENDING_FILES = "filesPendingReplication";
     // Only add these metrics if the replication table is online and there are peers
-    if (TableState.ONLINE == Tables.getTableState(master.getContext(), ReplicationTable.ID)
+    if (TableState.ONLINE == Tables.getTableState(manager.getContext(), ReplicationTable.ID)
         && !replicationUtil.getPeers().isEmpty()) {
       getRegistry().add(PENDING_FILES, getNumFilesPendingReplication());
       addReplicationQueueTimeMetrics();
@@ -103,7 +103,7 @@
   }
 
   protected int getMaxReplicationThreads() {
-    return replicationUtil.getMaxReplicationThreads(master.getMasterMonitorInfo());
+    return replicationUtil.getMaxReplicationThreads(manager.getManagerMonitorInfo());
   }
 
   protected void addReplicationQueueTimeMetrics() {
@@ -119,7 +119,7 @@
       if (!pathModTimes.containsKey(path)) {
         try {
           pathModTimes.put(path,
-              master.getVolumeManager().getFileStatus(path).getModificationTime());
+              manager.getVolumeManager().getFileStatus(path).getModificationTime());
         } catch (IOException e) {
           // Ignore all IOExceptions
           // Either the system is unavailable or the file was deleted
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/recovery/RecoveryManager.java b/server/manager/src/main/java/org/apache/accumulo/manager/recovery/RecoveryManager.java
index 419808a..4406f5a 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/recovery/RecoveryManager.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/recovery/RecoveryManager.java
@@ -64,21 +64,21 @@
   private Set<String> sortsQueued = new HashSet<>();
   private Cache<Path,Boolean> existenceCache;
   private ScheduledExecutorService executor;
-  private Manager master;
+  private Manager manager;
   private ZooCache zooCache;
 
-  public RecoveryManager(Manager master, long timeToCacheExistsInMillis) {
-    this.master = master;
+  public RecoveryManager(Manager manager, long timeToCacheExistsInMillis) {
+    this.manager = manager;
     existenceCache =
         CacheBuilder.newBuilder().expireAfterWrite(timeToCacheExistsInMillis, TimeUnit.MILLISECONDS)
             .maximumWeight(10_000_000).weigher((path, exist) -> path.toString().length()).build();
 
     executor = ThreadPools.createScheduledExecutorService(4, "Walog sort starter ", false);
-    zooCache = new ZooCache(master.getContext().getZooReaderWriter(), null);
+    zooCache = new ZooCache(manager.getContext().getZooReaderWriter(), null);
     try {
       List<String> workIDs =
-          new DistributedWorkQueue(master.getZooKeeperRoot() + Constants.ZRECOVERY,
-              master.getConfiguration()).getWorkQueued();
+          new DistributedWorkQueue(manager.getZooKeeperRoot() + Constants.ZRECOVERY,
+              manager.getConfiguration()).getWorkQueued();
       sortsQueued.addAll(workIDs);
     } catch (Exception e) {
       log.warn("{}", e.getMessage(), e);
@@ -102,8 +102,8 @@
     public void run() {
       boolean rescheduled = false;
       try {
-        long time = closer.close(master.getConfiguration(), master.getContext().getHadoopConf(),
-            master.getVolumeManager(), new Path(source));
+        long time = closer.close(manager.getConfiguration(), manager.getContext().getHadoopConf(),
+            manager.getVolumeManager(), new Path(source));
 
         if (time > 0) {
           executor.schedule(this, time, TimeUnit.MILLISECONDS);
@@ -129,20 +129,20 @@
   private void initiateSort(String sortId, String source, final String destination)
       throws KeeperException, InterruptedException {
     String work = source + "|" + destination;
-    new DistributedWorkQueue(master.getZooKeeperRoot() + Constants.ZRECOVERY,
-        master.getConfiguration()).addWork(sortId, work.getBytes(UTF_8));
+    new DistributedWorkQueue(manager.getZooKeeperRoot() + Constants.ZRECOVERY,
+        manager.getConfiguration()).addWork(sortId, work.getBytes(UTF_8));
 
     synchronized (this) {
       sortsQueued.add(sortId);
     }
 
-    final String path = master.getZooKeeperRoot() + Constants.ZRECOVERY + "/" + sortId;
+    final String path = manager.getZooKeeperRoot() + Constants.ZRECOVERY + "/" + sortId;
     log.info("Created zookeeper entry {} with data {}", path, work);
   }
 
   private boolean exists(final Path path) throws IOException {
     try {
-      return existenceCache.get(path, () -> master.getVolumeManager().exists(path));
+      return existenceCache.get(path, () -> manager.getVolumeManager().exists(path));
     } catch (ExecutionException e) {
       throw new IOException(e);
     }
@@ -155,8 +155,9 @@
     for (Collection<String> logs : walogs) {
       for (String walog : logs) {
 
-        Path switchedWalog = VolumeUtil.switchVolume(walog, FileType.WAL, ServerConstants
-            .getVolumeReplacements(master.getConfiguration(), master.getContext().getHadoopConf()));
+        Path switchedWalog =
+            VolumeUtil.switchVolume(walog, FileType.WAL, ServerConstants.getVolumeReplacements(
+                manager.getConfiguration(), manager.getContext().getHadoopConf()));
         if (switchedWalog != null) {
           // replaces the volume used for sorting, but do not change entry in metadata table. When
           // the tablet loads it will change the metadata table entry. If
@@ -176,7 +177,7 @@
         }
 
         if (sortQueued
-            && zooCache.get(master.getZooKeeperRoot() + Constants.ZRECOVERY + "/" + sortId)
+            && zooCache.get(manager.getZooKeeperRoot() + Constants.ZRECOVERY + "/" + sortId)
                 == null) {
           synchronized (this) {
             sortsQueued.remove(sortId);
@@ -195,7 +196,7 @@
         recoveryNeeded = true;
         synchronized (this) {
           if (!closeTasksQueued.contains(sortId) && !sortsQueued.contains(sortId)) {
-            AccumuloConfiguration aconf = master.getConfiguration();
+            AccumuloConfiguration aconf = manager.getConfiguration();
             LogCloser closer = Property.createInstanceFromPropertyName(aconf,
                 Property.MANAGER_WALOG_CLOSER_IMPLEMETATION, LogCloser.class,
                 new HadoopLogCloser());
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/replication/ManagerReplicationCoordinator.java b/server/manager/src/main/java/org/apache/accumulo/manager/replication/ManagerReplicationCoordinator.java
index 9c25f5e..c4ed070 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/replication/ManagerReplicationCoordinator.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/replication/ManagerReplicationCoordinator.java
@@ -48,29 +48,29 @@
 public class ManagerReplicationCoordinator implements ReplicationCoordinator.Iface {
   private static final Logger log = LoggerFactory.getLogger(ManagerReplicationCoordinator.class);
 
-  private final Manager master;
+  private final Manager manager;
   private final Random rand;
   private final ZooReader reader;
   private final SecurityOperation security;
 
-  public ManagerReplicationCoordinator(Manager master) {
-    this(master, new ZooReader(master.getContext().getZooKeepers(),
-        master.getContext().getZooKeepersSessionTimeOut()));
+  public ManagerReplicationCoordinator(Manager manager) {
+    this(manager, new ZooReader(manager.getContext().getZooKeepers(),
+        manager.getContext().getZooKeepersSessionTimeOut()));
   }
 
-  protected ManagerReplicationCoordinator(Manager master, ZooReader reader) {
-    this.master = master;
+  protected ManagerReplicationCoordinator(Manager manager, ZooReader reader) {
+    this.manager = manager;
     this.rand = new SecureRandom();
     this.rand.setSeed(358923462L);
     this.reader = reader;
-    this.security = AuditedSecurityOperation.getInstance(master.getContext());
+    this.security = AuditedSecurityOperation.getInstance(manager.getContext());
   }
 
   @Override
   public String getServicerAddress(String remoteTableId, TCredentials creds)
       throws ReplicationCoordinatorException, TException {
     try {
-      security.authenticateUser(master.getContext().rpcCreds(), creds);
+      security.authenticateUser(manager.getContext().rpcCreds(), creds);
     } catch (ThriftSecurityException e) {
       log.error("{} failed to authenticate for replication to {}", creds.getPrincipal(),
           remoteTableId);
@@ -78,7 +78,7 @@
           "Could not authenticate " + creds.getPrincipal());
     }
 
-    Set<TServerInstance> tservers = master.onlineTabletServers();
+    Set<TServerInstance> tservers = manager.onlineTabletServers();
     if (tservers.isEmpty()) {
       throw new ReplicationCoordinatorException(
           ReplicationCoordinatorErrorCode.NO_AVAILABLE_SERVERS,
@@ -88,7 +88,7 @@
     TServerInstance tserver = getRandomTServer(tservers, rand.nextInt(tservers.size()));
     String replServiceAddr;
     try {
-      replServiceAddr = new String(reader.getData(master.getZooKeeperRoot()
+      replServiceAddr = new String(reader.getData(manager.getZooKeeperRoot()
           + ReplicationConstants.ZOO_TSERVERS + "/" + tserver.getHostPort()), UTF_8);
     } catch (KeeperException | InterruptedException e) {
       log.error("Could not fetch repliation service port for tserver", e);
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/replication/ReplicationDriver.java b/server/manager/src/main/java/org/apache/accumulo/manager/replication/ReplicationDriver.java
index 6bac255..b653d52 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/replication/ReplicationDriver.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/replication/ReplicationDriver.java
@@ -31,12 +31,12 @@
 import org.slf4j.LoggerFactory;
 
 /**
- * Daemon wrapper around the {@link WorkMaker} that separates it from the Master
+ * Daemon wrapper around the {@link WorkMaker} that separates it from the Manager
  */
 public class ReplicationDriver implements Runnable {
   private static final Logger log = LoggerFactory.getLogger(ReplicationDriver.class);
 
-  private final Manager master;
+  private final Manager manager;
   private final AccumuloConfiguration conf;
 
   private WorkMaker workMaker;
@@ -45,9 +45,9 @@
   private RemoveCompleteReplicationRecords rcrr;
   private AccumuloClient client;
 
-  public ReplicationDriver(Manager master) {
-    this.master = master;
-    this.conf = master.getConfiguration();
+  public ReplicationDriver(Manager manager) {
+    this.manager = manager;
+    this.conf = manager.getConfiguration();
   }
 
   @Override
@@ -61,16 +61,16 @@
 
     log.debug("Starting replication loop");
 
-    while (master.stillMaster()) {
+    while (manager.stillManager()) {
       if (workMaker == null) {
-        client = master.getContext();
-        statusMaker = new StatusMaker(client, master.getVolumeManager());
-        workMaker = new WorkMaker(master.getContext(), client);
+        client = manager.getContext();
+        statusMaker = new StatusMaker(client, manager.getVolumeManager());
+        workMaker = new WorkMaker(manager.getContext(), client);
         finishedWorkUpdater = new FinishedWorkUpdater(client);
         rcrr = new RemoveCompleteReplicationRecords(client);
       }
 
-      try (TraceScope replicationDriver = Trace.startSpan("masterReplicationDriver", sampler)) {
+      try (TraceScope replicationDriver = Trace.startSpan("managerReplicationDriver", sampler)) {
 
         // Make status markers from replication records in metadata, removing entries in
         // metadata which are no longer needed (closed records)
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/replication/StatusMaker.java b/server/manager/src/main/java/org/apache/accumulo/manager/replication/StatusMaker.java
index 1d7c2a8..bddcfe4 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/replication/StatusMaker.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/replication/StatusMaker.java
@@ -233,7 +233,7 @@
   }
 
   /**
-   * Because there is only one active Master, and thus one active StatusMaker, the only safe time
+   * Because there is only one active Manager, and thus one active StatusMaker, the only safe time
    * that we can issue the delete for a Status which is closed is immediately after writing it to
    * the replication table.
    * <p>
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/replication/UnorderedWorkAssigner.java b/server/manager/src/main/java/org/apache/accumulo/manager/replication/UnorderedWorkAssigner.java
index a536e2d..f530832 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/replication/UnorderedWorkAssigner.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/replication/UnorderedWorkAssigner.java
@@ -45,7 +45,7 @@
  * <p>
  * The implementation allows for multiple tservers to concurrently replicate data to peer(s),
  * however it is possible that data for a table is replayed on the peer in a different order than
- * the master. The {@link SequentialWorkAssigner} should be used if this must be guaranteed at the
+ * the manager. The {@link SequentialWorkAssigner} should be used if this must be guaranteed at the
  * cost of replication throughput.
  */
 public class UnorderedWorkAssigner extends DistributedWorkQueueWorkAssigner {
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/replication/WorkDriver.java b/server/manager/src/main/java/org/apache/accumulo/manager/replication/WorkDriver.java
index ba0cbbb..756cd82 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/replication/WorkDriver.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/replication/WorkDriver.java
@@ -36,18 +36,18 @@
 public class WorkDriver implements Runnable {
   private static final Logger log = LoggerFactory.getLogger(WorkDriver.class);
 
-  private Manager master;
+  private Manager manager;
   private AccumuloClient client;
   private AccumuloConfiguration conf;
 
   private WorkAssigner assigner;
   private String assignerImplName;
 
-  public WorkDriver(Manager master) {
+  public WorkDriver(Manager manager) {
     super();
-    this.master = master;
-    this.client = master.getContext();
-    this.conf = master.getConfiguration();
+    this.manager = manager;
+    this.client = manager.getContext();
+    this.conf = manager.getConfiguration();
     configureWorkAssigner();
   }
 
@@ -79,7 +79,7 @@
   public void run() {
     log.info("Starting replication work assignment thread using {}", assignerImplName);
 
-    while (master.stillMaster()) {
+    while (manager.stillManager()) {
       // Assign the work using the configured implementation
       try {
         assigner.assignWork();
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/state/MergeStats.java b/server/manager/src/main/java/org/apache/accumulo/manager/state/MergeStats.java
index a5ac4cd..755f8dc 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/state/MergeStats.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/state/MergeStats.java
@@ -105,7 +105,7 @@
       this.unassigned++;
   }
 
-  public MergeState nextMergeState(AccumuloClient accumuloClient, CurrentState master)
+  public MergeState nextMergeState(AccumuloClient accumuloClient, CurrentState manager)
       throws Exception {
     MergeState state = info.getState();
     if (state == MergeState.NONE)
@@ -153,7 +153,7 @@
         log.info("{} tablets are chopped, {} are offline {}", chopped, unassigned,
             info.getExtent());
         if (unassigned == total) {
-          if (verifyMergeConsistency(accumuloClient, master))
+          if (verifyMergeConsistency(accumuloClient, manager))
             state = MergeState.MERGING;
           else
             log.info("Merge consistency check failed {}", info.getExtent());
@@ -186,13 +186,13 @@
     return state;
   }
 
-  private boolean verifyMergeConsistency(AccumuloClient accumuloClient, CurrentState master)
+  private boolean verifyMergeConsistency(AccumuloClient accumuloClient, CurrentState manager)
       throws TableNotFoundException, IOException {
     MergeStats verify = new MergeStats(info);
     KeyExtent extent = info.getExtent();
     Scanner scanner = accumuloClient
         .createScanner(extent.isMeta() ? RootTable.NAME : MetadataTable.NAME, Authorizations.EMPTY);
-    MetaDataTableScanner.configureScanner(scanner, master);
+    MetaDataTableScanner.configureScanner(scanner, manager);
     Text start = extent.prevEndRow();
     if (start == null) {
       start = new Text();
@@ -230,8 +230,8 @@
           return false;
         }
 
-        if (tls.getState(master.onlineTabletServers()) != TabletState.UNASSIGNED
-            && tls.getState(master.onlineTabletServers()) != TabletState.SUSPENDED) {
+        if (tls.getState(manager.onlineTabletServers()) != TabletState.UNASSIGNED
+            && tls.getState(manager.onlineTabletServers()) != TabletState.SUSPENDED) {
           log.debug("failing consistency: assigned or hosted {}", tls);
           return false;
         }
@@ -243,7 +243,7 @@
 
       prevExtent = tls.extent;
 
-      verify.update(tls.extent, tls.getState(master.onlineTabletServers()), tls.chopped,
+      verify.update(tls.extent, tls.getState(manager.onlineTabletServers()), tls.chopped,
           !tls.walogs.isEmpty());
       // stop when we've seen the tablet just beyond our range
       if (tls.extent.prevEndRow() != null && extent.endRow() != null
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/state/SetGoalState.java b/server/manager/src/main/java/org/apache/accumulo/manager/state/SetGoalState.java
index 71cb852..5fb8bdf 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/state/SetGoalState.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/state/SetGoalState.java
@@ -26,6 +26,7 @@
 import org.apache.accumulo.core.singletons.SingletonManager;
 import org.apache.accumulo.core.singletons.SingletonManager.Mode;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
+import org.apache.accumulo.manager.upgrade.RenameMasterDirInZK;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.ServerUtil;
 import org.apache.accumulo.server.security.SecurityUtil;
@@ -33,7 +34,7 @@
 public class SetGoalState {
 
   /**
-   * Utility program that will change the goal state for the master from the command line.
+   * Utility program that will change the goal state for the manager from the command line.
    */
   public static void main(String[] args) throws Exception {
     if (args.length != 1 || ManagerGoalState.valueOf(args[0]) == null) {
@@ -44,10 +45,11 @@
 
     try {
       var context = new ServerContext(SiteConfiguration.auto());
+      RenameMasterDirInZK.renameMasterDirInZK(context);
       SecurityUtil.serverLogin(context.getConfiguration());
       ServerUtil.waitForZookeeperAndHdfs(context);
       context.getZooReaderWriter().putPersistentData(
-          context.getZooKeeperRoot() + Constants.ZMASTER_GOAL_STATE, args[0].getBytes(UTF_8),
+          context.getZooKeeperRoot() + Constants.ZMANAGER_GOAL_STATE, args[0].getBytes(UTF_8),
           NodeExistsPolicy.OVERWRITE);
     } finally {
       SingletonManager.setMode(Mode.CLOSED);
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/state/TableStats.java b/server/manager/src/main/java/org/apache/accumulo/manager/state/TableStats.java
index 0e9628e..daccf25 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/state/TableStats.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/state/TableStats.java
@@ -57,7 +57,7 @@
     return last;
   }
 
-  public synchronized ManagerState getLastMasterState() {
+  public synchronized ManagerState getLastManagerState() {
     return state;
   }
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/Utils.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/Utils.java
index f9399cf..86dfbf1 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/Utils.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/Utils.java
@@ -200,9 +200,9 @@
    * @param path
    *          the fully-qualified path
    */
-  public static SortedSet<Text> getSortedSetFromFile(Manager master, Path path, boolean encoded)
+  public static SortedSet<Text> getSortedSetFromFile(Manager manager, Path path, boolean encoded)
       throws IOException {
-    FileSystem fs = path.getFileSystem(master.getContext().getHadoopConf());
+    FileSystem fs = path.getFileSystem(manager.getContext().getHadoopConf());
     var data = new TreeSet<Text>();
     try (var file = new java.util.Scanner(fs.open(path), UTF_8)) {
       while (file.hasNextLine()) {
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/BulkImport.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/BulkImport.java
index 8729de2..c710ea2 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/BulkImport.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/BulkImport.java
@@ -62,14 +62,14 @@
 /**
  * Bulk import makes requests of tablet servers, and those requests can take a long time. Our
  * communications to the tablet server may fail, so we won't know the status of the request. The
- * master will repeat failed requests so now there are multiple requests to the tablet server. The
+ * manager will repeat failed requests so now there are multiple requests to the tablet server. The
  * tablet server will not execute the request multiple times, so long as the marker it wrote in the
- * metadata table stays there. The master needs to know when all requests have finished so it can
+ * metadata table stays there. The manager needs to know when all requests have finished so it can
  * remove the markers. Did it start? Did it finish? We can see that *a* request completed by seeing
  * the flag written into the metadata table, but we won't know if some other rogue thread is still
  * waiting to start a thread and repeat the operation.
  *
- * The master can ask the tablet server if it has any requests still running. Except the tablet
+ * The manager can ask the tablet server if it has any requests still running. Except the tablet
  * server might have some thread about to start a request, but before it has made any bookkeeping
  * about the request. To prevent problems like this, an Arbitrator is used. Before starting any new
  * request, the tablet server checks the Arbitrator to see if the request is still valid.
@@ -94,16 +94,16 @@
   }
 
   @Override
-  public long isReady(long tid, Manager master) throws Exception {
-    if (!Utils.getReadLock(master, tableId, tid).tryLock())
+  public long isReady(long tid, Manager manager) throws Exception {
+    if (!Utils.getReadLock(manager, tableId, tid).tryLock())
       return 100;
 
-    Tables.clearCache(master.getContext());
-    if (Tables.getTableState(master.getContext(), tableId) == TableState.ONLINE) {
+    Tables.clearCache(manager.getContext());
+    if (Tables.getTableState(manager.getContext(), tableId) == TableState.ONLINE) {
       long reserve1, reserve2;
-      reserve1 = reserve2 = Utils.reserveHdfsDirectory(master, sourceDir, tid);
+      reserve1 = reserve2 = Utils.reserveHdfsDirectory(manager, sourceDir, tid);
       if (reserve1 == 0)
-        reserve2 = Utils.reserveHdfsDirectory(master, errorDir, tid);
+        reserve2 = Utils.reserveHdfsDirectory(manager, errorDir, tid);
       return reserve2;
     } else {
       throw new AcceptableThriftTableOperationException(tableId.canonical(), null,
@@ -112,15 +112,15 @@
   }
 
   @Override
-  public Repo<Manager> call(long tid, Manager master) throws Exception {
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
     String fmtTid = FateTxId.formatTid(tid);
 
     log.debug(" {} sourceDir {}", fmtTid, sourceDir);
 
-    Utils.getReadLock(master, tableId, tid).lock();
+    Utils.getReadLock(manager, tableId, tid).lock();
 
     // check that the error directory exists and is empty
-    VolumeManager fs = master.getVolumeManager();
+    VolumeManager fs = manager.getVolumeManager();
 
     Path errorPath = new Path(errorDir);
     FileStatus errorStatus = null;
@@ -142,11 +142,11 @@
           TableOperation.BULK_IMPORT, TableOperationExceptionType.BULK_BAD_ERROR_DIRECTORY,
           errorDir + " is not empty");
 
-    ZooArbitrator.start(master.getContext(), Constants.BULK_ARBITRATOR_TYPE, tid);
-    master.updateBulkImportStatus(sourceDir, BulkImportState.MOVING);
+    ZooArbitrator.start(manager.getContext(), Constants.BULK_ARBITRATOR_TYPE, tid);
+    manager.updateBulkImportStatus(sourceDir, BulkImportState.MOVING);
     // move the files into the directory
     try {
-      String bulkDir = prepareBulkImport(master.getContext(), fs, sourceDir, tableId, tid);
+      String bulkDir = prepareBulkImport(manager.getContext(), fs, sourceDir, tableId, tid);
       log.debug(" {} bulkDir {}", tid, bulkDir);
       return new LoadFiles(tableId, sourceDir, bulkDir, errorDir, setTime);
     } catch (IOException ex) {
@@ -189,19 +189,19 @@
   }
 
   @VisibleForTesting
-  public static String prepareBulkImport(ServerContext master, final VolumeManager fs, String dir,
+  public static String prepareBulkImport(ServerContext manager, final VolumeManager fs, String dir,
       TableId tableId, long tid) throws Exception {
-    final Path bulkDir = createNewBulkDir(master, fs, dir, tableId);
+    final Path bulkDir = createNewBulkDir(manager, fs, dir, tableId);
 
-    MetadataTableUtil.addBulkLoadInProgressFlag(master,
+    MetadataTableUtil.addBulkLoadInProgressFlag(manager,
         "/" + bulkDir.getParent().getName() + "/" + bulkDir.getName(), tid);
 
     Path dirPath = new Path(dir);
     FileStatus[] mapFiles = fs.listStatus(dirPath);
 
-    final UniqueNameAllocator namer = master.getUniqueNameAllocator();
+    final UniqueNameAllocator namer = manager.getUniqueNameAllocator();
 
-    AccumuloConfiguration serverConfig = master.getConfiguration();
+    AccumuloConfiguration serverConfig = manager.getConfiguration();
     @SuppressWarnings("deprecation")
     ExecutorService workers = ThreadPools.createExecutorService(serverConfig, serverConfig
         .resolve(Property.MANAGER_RENAME_THREADS, Property.MANAGER_BULK_RENAME_THREADS));
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CleanUpBulkImport.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CleanUpBulkImport.java
index 549f4fb..ddc0166 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CleanUpBulkImport.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CleanUpBulkImport.java
@@ -54,24 +54,24 @@
   }
 
   @Override
-  public Repo<Manager> call(long tid, Manager master) throws Exception {
-    master.updateBulkImportStatus(source, BulkImportState.CLEANUP);
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
+    manager.updateBulkImportStatus(source, BulkImportState.CLEANUP);
     log.debug("removing the bulkDir processing flag file in " + bulk);
     Path bulkDir = new Path(bulk);
-    MetadataTableUtil.removeBulkLoadInProgressFlag(master.getContext(),
+    MetadataTableUtil.removeBulkLoadInProgressFlag(manager.getContext(),
         "/" + bulkDir.getParent().getName() + "/" + bulkDir.getName());
-    master.getContext().getAmple().putGcFileAndDirCandidates(tableId,
+    manager.getContext().getAmple().putGcFileAndDirCandidates(tableId,
         Collections.singleton(bulkDir.toString()));
     log.debug("removing the metadata table markers for loaded files");
-    AccumuloClient client = master.getContext();
+    AccumuloClient client = manager.getContext();
     MetadataTableUtil.removeBulkLoadEntries(client, tableId, tid);
     log.debug("releasing HDFS reservations for " + source + " and " + error);
-    Utils.unreserveHdfsDirectory(master, source, tid);
-    Utils.unreserveHdfsDirectory(master, error, tid);
-    Utils.getReadLock(master, tableId, tid).unlock();
+    Utils.unreserveHdfsDirectory(manager, source, tid);
+    Utils.unreserveHdfsDirectory(manager, error, tid);
+    Utils.getReadLock(manager, tableId, tid).unlock();
     log.debug("completing bulkDir import transaction " + FateTxId.formatTid(tid));
-    ZooArbitrator.cleanup(master.getContext(), Constants.BULK_ARBITRATOR_TYPE, tid);
-    master.removeBulkImportStatus(source);
+    ZooArbitrator.cleanup(manager.getContext(), Constants.BULK_ARBITRATOR_TYPE, tid);
+    manager.removeBulkImportStatus(source);
     return null;
   }
 }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CompleteBulkImport.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CompleteBulkImport.java
index ab9377b..a315bbc 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CompleteBulkImport.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CompleteBulkImport.java
@@ -42,8 +42,8 @@
   }
 
   @Override
-  public Repo<Manager> call(long tid, Manager master) throws Exception {
-    ZooArbitrator.stop(master.getContext(), Constants.BULK_ARBITRATOR_TYPE, tid);
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
+    ZooArbitrator.stop(manager.getContext(), Constants.BULK_ARBITRATOR_TYPE, tid);
     return new CopyFailed(tableId, source, bulk, error);
   }
 }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CopyFailed.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CopyFailed.java
index 8f7fb61..e0b72cd 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CopyFailed.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/CopyFailed.java
@@ -70,12 +70,12 @@
   }
 
   @Override
-  public long isReady(long tid, Manager master) {
+  public long isReady(long tid, Manager manager) {
     Set<TServerInstance> finished = new HashSet<>();
-    Set<TServerInstance> running = master.onlineTabletServers();
+    Set<TServerInstance> running = manager.onlineTabletServers();
     for (TServerInstance server : running) {
       try {
-        TServerConnection client = master.getConnection(server);
+        TServerConnection client = manager.getConnection(server);
         if (client != null && !client.isActive(tid))
           finished.add(server);
       } catch (TException ex) {
@@ -89,10 +89,10 @@
   }
 
   @Override
-  public Repo<Manager> call(long tid, Manager master) throws Exception {
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
     // This needs to execute after the arbiter is stopped
-    master.updateBulkImportStatus(source, BulkImportState.COPY_FILES);
-    VolumeManager fs = master.getVolumeManager();
+    manager.updateBulkImportStatus(source, BulkImportState.COPY_FILES);
+    VolumeManager fs = manager.getVolumeManager();
 
     if (!fs.exists(new Path(error, BulkImport.FAILURES_TXT)))
       return new CleanUpBulkImport(tableId, source, bulk, error);
@@ -116,7 +116,7 @@
      */
 
     // determine which failed files were loaded
-    AccumuloClient client = master.getContext();
+    AccumuloClient client = manager.getContext();
     try (Scanner mscanner =
         new IsolatedScanner(client.createScanner(MetadataTable.NAME, Authorizations.EMPTY))) {
       mscanner.setRange(new KeyExtent(tableId, null, null).toMetaRange());
@@ -142,8 +142,8 @@
 
     if (!loadedFailures.isEmpty()) {
       DistributedWorkQueue bifCopyQueue = new DistributedWorkQueue(
-          Constants.ZROOT + "/" + master.getInstanceID() + Constants.ZBULK_FAILED_COPYQ,
-          master.getConfiguration());
+          Constants.ZROOT + "/" + manager.getInstanceID() + Constants.ZBULK_FAILED_COPYQ,
+          manager.getConfiguration());
 
       HashSet<String> workIds = new HashSet<>();
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/LoadFiles.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/LoadFiles.java
index ac252f8..b9e0cc8 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/LoadFiles.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer1/LoadFiles.java
@@ -82,26 +82,26 @@
   }
 
   @Override
-  public long isReady(long tid, Manager master) {
-    if (master.onlineTabletServers().isEmpty())
+  public long isReady(long tid, Manager manager) {
+    if (manager.onlineTabletServers().isEmpty())
       return 500;
     return 0;
   }
 
-  private static synchronized ExecutorService getThreadPool(Manager master) {
+  private static synchronized ExecutorService getThreadPool(Manager manager) {
     if (threadPool == null) {
-      threadPool = ThreadPools.createExecutorService(master.getConfiguration(),
+      threadPool = ThreadPools.createExecutorService(manager.getConfiguration(),
           Property.MANAGER_BULK_THREADPOOL_SIZE);
     }
     return threadPool;
   }
 
   @Override
-  public Repo<Manager> call(final long tid, final Manager master) throws Exception {
-    master.updateBulkImportStatus(source, BulkImportState.LOADING);
-    ExecutorService executor = getThreadPool(master);
-    final AccumuloConfiguration conf = master.getConfiguration();
-    VolumeManager fs = master.getVolumeManager();
+  public Repo<Manager> call(final long tid, final Manager manager) throws Exception {
+    manager.updateBulkImportStatus(source, BulkImportState.LOADING);
+    ExecutorService executor = getThreadPool(manager);
+    final AccumuloConfiguration conf = manager.getConfiguration();
+    VolumeManager fs = manager.getVolumeManager();
     List<FileStatus> files = new ArrayList<>();
     Collections.addAll(files, fs.listStatus(new Path(bulk)));
     log.debug(FateTxId.formatTid(tid) + " importing " + files.size() + " files");
@@ -125,11 +125,11 @@
     for (int attempt = 0; attempt < RETRIES && !filesToLoad.isEmpty(); attempt++) {
       List<Future<Void>> results = new ArrayList<>();
 
-      if (master.onlineTabletServers().isEmpty())
+      if (manager.onlineTabletServers().isEmpty())
         log.warn("There are no tablet server to process bulk import, waiting (tid = "
             + FateTxId.formatTid(tid) + ")");
 
-      while (master.onlineTabletServers().isEmpty()) {
+      while (manager.onlineTabletServers().isEmpty()) {
         sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
       }
 
@@ -139,11 +139,11 @@
       final TServerInstance[] servers;
       String prop = conf.get(Property.MANAGER_BULK_TSERVER_REGEX);
       if (prop == null || "".equals(prop)) {
-        servers = master.onlineTabletServers().toArray(new TServerInstance[0]);
+        servers = manager.onlineTabletServers().toArray(new TServerInstance[0]);
       } else {
         Pattern regex = Pattern.compile(prop);
         List<TServerInstance> subset = new ArrayList<>();
-        master.onlineTabletServers().forEach(t -> {
+        manager.onlineTabletServers().forEach(t -> {
           if (regex.matcher(t.getHost()).matches()) {
             subset.add(t);
           }
@@ -161,17 +161,17 @@
             HostAndPort server = null;
             try {
               // get a connection to a random tablet server, do not prefer cached connections
-              // because this is running on the master and there are lots of connections to tablet
+              // because this is running on the manager and there are lots of connections to tablet
               // servers serving the metadata tablets
               long timeInMillis =
-                  master.getConfiguration().getTimeInMillis(Property.MANAGER_BULK_TIMEOUT);
+                  manager.getConfiguration().getTimeInMillis(Property.MANAGER_BULK_TIMEOUT);
               server = servers[random.nextInt(servers.length)].getHostAndPort();
-              client = ThriftUtil.getTServerClient(server, master.getContext(), timeInMillis);
+              client = ThriftUtil.getTServerClient(server, manager.getContext(), timeInMillis);
               List<String> attempt1 = Collections.singletonList(file);
               log.debug("Asking " + server + " to bulk import " + file);
               List<String> fail =
-                  client.bulkImportFiles(TraceUtil.traceInfo(), master.getContext().rpcCreds(), tid,
-                      tableId.canonical(), attempt1, errorDir, setTime);
+                  client.bulkImportFiles(TraceUtil.traceInfo(), manager.getContext().rpcCreds(),
+                      tid, tableId.canonical(), attempt1, errorDir, setTime);
               if (fail.isEmpty()) {
                 loaded.add(file);
               }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/BulkImportMove.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/BulkImportMove.java
index 0be174c..4788afd 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/BulkImportMove.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/BulkImportMove.java
@@ -44,14 +44,14 @@
 /**
  * Bulk import makes requests of tablet servers, and those requests can take a long time. Our
  * communications to the tablet server may fail, so we won't know the status of the request. The
- * master will repeat failed requests so now there are multiple requests to the tablet server. The
+ * manager will repeat failed requests so now there are multiple requests to the tablet server. The
  * tablet server will not execute the request multiple times, so long as the marker it wrote in the
- * metadata table stays there. The master needs to know when all requests have finished so it can
+ * metadata table stays there. The manager needs to know when all requests have finished so it can
  * remove the markers. Did it start? Did it finish? We can see that *a* request completed by seeing
  * the flag written into the metadata table, but we won't know if some other rogue thread is still
  * waiting to start a thread and repeat the operation.
  *
- * The master can ask the tablet server if it has any requests still running. Except the tablet
+ * The manager can ask the tablet server if it has any requests still running. Except the tablet
  * server might have some thread about to start a request, but before it has made any bookkeeping
  * about the request. To prevent problems like this, an Arbitrator is used. Before starting any new
  * request, the tablet server checks the Arbitrator to see if the request is still valid.
@@ -69,7 +69,7 @@
   }
 
   @Override
-  public Repo<Manager> call(long tid, Manager master) throws Exception {
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
     final Path bulkDir = new Path(bulkInfo.bulkDir);
     final Path sourceDir = new Path(bulkInfo.sourceDir);
 
@@ -77,16 +77,16 @@
 
     log.debug("{} sourceDir {}", fmtTid, sourceDir);
 
-    VolumeManager fs = master.getVolumeManager();
+    VolumeManager fs = manager.getVolumeManager();
 
     if (bulkInfo.tableState == TableState.ONLINE) {
-      ZooArbitrator.start(master.getContext(), Constants.BULK_ARBITRATOR_TYPE, tid);
+      ZooArbitrator.start(manager.getContext(), Constants.BULK_ARBITRATOR_TYPE, tid);
     }
 
     try {
       Map<String,String> oldToNewNameMap =
           BulkSerialize.readRenameMap(bulkDir.toString(), fs::open);
-      moveFiles(tid, sourceDir, bulkDir, master, fs, oldToNewNameMap);
+      moveFiles(tid, sourceDir, bulkDir, manager, fs, oldToNewNameMap);
 
       return new LoadFiles(bulkInfo);
     } catch (Exception ex) {
@@ -99,12 +99,12 @@
   /**
    * For every entry in renames, move the file from the key path to the value path
    */
-  private void moveFiles(long tid, Path sourceDir, Path bulkDir, Manager master,
+  private void moveFiles(long tid, Path sourceDir, Path bulkDir, Manager manager,
       final VolumeManager fs, Map<String,String> renames) throws Exception {
-    MetadataTableUtil.addBulkLoadInProgressFlag(master.getContext(),
+    MetadataTableUtil.addBulkLoadInProgressFlag(manager.getContext(),
         "/" + bulkDir.getParent().getName() + "/" + bulkDir.getName(), tid);
 
-    AccumuloConfiguration aConf = master.getConfiguration();
+    AccumuloConfiguration aConf = manager.getConfiguration();
     @SuppressWarnings("deprecation")
     int workerCount = aConf.getCount(
         aConf.resolve(Property.MANAGER_RENAME_THREADS, Property.MANAGER_BULK_RENAME_THREADS));
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/CleanUpBulkImport.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/CleanUpBulkImport.java
index 7a3d4a9..0b3d3cb 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/CleanUpBulkImport.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/CleanUpBulkImport.java
@@ -48,33 +48,33 @@
   }
 
   @Override
-  public Repo<Manager> call(long tid, Manager master) throws Exception {
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
     log.debug("removing the bulkDir processing flag file in " + info.bulkDir);
     Path bulkDir = new Path(info.bulkDir);
-    MetadataTableUtil.removeBulkLoadInProgressFlag(master.getContext(),
+    MetadataTableUtil.removeBulkLoadInProgressFlag(manager.getContext(),
         "/" + bulkDir.getParent().getName() + "/" + bulkDir.getName());
-    master.getContext().getAmple().putGcFileAndDirCandidates(info.tableId,
+    manager.getContext().getAmple().putGcFileAndDirCandidates(info.tableId,
         Collections.singleton(bulkDir.toString()));
     if (info.tableState == TableState.ONLINE) {
       log.debug("removing the metadata table markers for loaded files");
-      AccumuloClient client = master.getContext();
+      AccumuloClient client = manager.getContext();
       MetadataTableUtil.removeBulkLoadEntries(client, info.tableId, tid);
     }
-    Utils.unreserveHdfsDirectory(master, info.sourceDir, tid);
-    Utils.getReadLock(master, info.tableId, tid).unlock();
+    Utils.unreserveHdfsDirectory(manager, info.sourceDir, tid);
+    Utils.getReadLock(manager, info.tableId, tid).unlock();
     // delete json renames and mapping files
     Path renamingFile = new Path(bulkDir, Constants.BULK_RENAME_FILE);
     Path mappingFile = new Path(bulkDir, Constants.BULK_LOAD_MAPPING);
     try {
-      master.getVolumeManager().delete(renamingFile);
-      master.getVolumeManager().delete(mappingFile);
+      manager.getVolumeManager().delete(renamingFile);
+      manager.getVolumeManager().delete(mappingFile);
     } catch (IOException ioe) {
       log.debug("Failed to delete renames and/or loadmap", ioe);
     }
 
     log.debug("completing bulkDir import transaction " + FateTxId.formatTid(tid));
     if (info.tableState == TableState.ONLINE) {
-      ZooArbitrator.cleanup(master.getContext(), Constants.BULK_ARBITRATOR_TYPE, tid);
+      ZooArbitrator.cleanup(manager.getContext(), Constants.BULK_ARBITRATOR_TYPE, tid);
     }
     return null;
   }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/CompleteBulkImport.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/CompleteBulkImport.java
index 374d691..2337393 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/CompleteBulkImport.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/CompleteBulkImport.java
@@ -35,8 +35,8 @@
   }
 
   @Override
-  public Repo<Manager> call(long tid, Manager master) throws Exception {
-    ZooArbitrator.stop(master.getContext(), Constants.BULK_ARBITRATOR_TYPE, tid);
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
+    ZooArbitrator.stop(manager.getContext(), Constants.BULK_ARBITRATOR_TYPE, tid);
     return new CleanUpBulkImport(info);
   }
 }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/LoadFiles.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/LoadFiles.java
index 3c4eb8b..c68bfd0 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/LoadFiles.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/LoadFiles.java
@@ -88,22 +88,22 @@
   }
 
   @Override
-  public long isReady(long tid, Manager master) throws Exception {
-    if (master.onlineTabletServers().isEmpty()) {
+  public long isReady(long tid, Manager manager) throws Exception {
+    if (manager.onlineTabletServers().isEmpty()) {
       log.warn("There are no tablet server to process bulkDir import, waiting (tid = "
           + FateTxId.formatTid(tid) + ")");
       return 100;
     }
-    VolumeManager fs = master.getVolumeManager();
+    VolumeManager fs = manager.getVolumeManager();
     final Path bulkDir = new Path(bulkInfo.bulkDir);
     try (LoadMappingIterator lmi =
         BulkSerialize.getUpdatedLoadMapping(bulkDir.toString(), bulkInfo.tableId, fs::open)) {
-      return loadFiles(bulkInfo.tableId, bulkDir, lmi, master, tid);
+      return loadFiles(bulkInfo.tableId, bulkDir, lmi, manager, tid);
     }
   }
 
   @Override
-  public Repo<Manager> call(final long tid, final Manager master) {
+  public Repo<Manager> call(final long tid, final Manager manager) {
     if (bulkInfo.tableState == TableState.ONLINE) {
       return new CompleteBulkImport(bulkInfo);
     } else {
@@ -113,13 +113,13 @@
 
   private abstract static class Loader {
     protected Path bulkDir;
-    protected Manager master;
+    protected Manager manager;
     protected long tid;
     protected boolean setTime;
 
-    void start(Path bulkDir, Manager master, long tid, boolean setTime) throws Exception {
+    void start(Path bulkDir, Manager manager, long tid, boolean setTime) throws Exception {
       this.bulkDir = bulkDir;
-      this.master = master;
+      this.manager = manager;
       this.tid = tid;
       this.setTime = setTime;
     }
@@ -145,10 +145,10 @@
     private int queuedDataSize = 0;
 
     @Override
-    void start(Path bulkDir, Manager master, long tid, boolean setTime) throws Exception {
-      super.start(bulkDir, master, tid, setTime);
+    void start(Path bulkDir, Manager manager, long tid, boolean setTime) throws Exception {
+      super.start(bulkDir, manager, tid, setTime);
 
-      timeInMillis = master.getConfiguration().getTimeInMillis(Property.MANAGER_BULK_TIMEOUT);
+      timeInMillis = manager.getConfiguration().getTimeInMillis(Property.MANAGER_BULK_TIMEOUT);
       fmtTid = FateTxId.formatTid(tid);
 
       loadMsgs = new MapCounter<>();
@@ -167,8 +167,8 @@
 
           TabletClientService.Client client = null;
           try {
-            client = ThriftUtil.getTServerClient(server, master.getContext(), timeInMillis);
-            client.loadFiles(TraceUtil.traceInfo(), master.getContext().rpcCreds(), tid,
+            client = ThriftUtil.getTServerClient(server, manager.getContext(), timeInMillis);
+            client.loadFiles(TraceUtil.traceInfo(), manager.getContext().rpcCreds(), tid,
                 bulkDir.toString(), tabletFiles, setTime);
           } catch (TException ex) {
             log.debug("rpc failed server: " + server + ", " + fmtTid + " " + ex.getMessage(), ex);
@@ -262,10 +262,10 @@
     MapCounter<HostAndPort> unloadingTablets;
 
     @Override
-    void start(Path bulkDir, Manager master, long tid, boolean setTime) throws Exception {
+    void start(Path bulkDir, Manager manager, long tid, boolean setTime) throws Exception {
       Preconditions.checkArgument(!setTime);
-      super.start(bulkDir, master, tid, setTime);
-      bw = master.getContext().createBatchWriter(MetadataTable.NAME);
+      super.start(bulkDir, manager, tid, setTime);
+      bw = manager.getContext().createBatchWriter(MetadataTable.NAME);
       unloadingTablets = new MapCounter<>();
     }
 
@@ -314,7 +314,7 @@
    * all files have been loaded.
    */
   private long loadFiles(TableId tableId, Path bulkDir, LoadMappingIterator loadMapIter,
-      Manager master, long tid) throws Exception {
+      Manager manager, long tid) throws Exception {
     PeekingIterator<Map.Entry<KeyExtent,Bulk.Files>> lmi = new PeekingIterator<>(loadMapIter);
     Map.Entry<KeyExtent,Bulk.Files> loadMapEntry = lmi.peek();
 
@@ -322,7 +322,7 @@
 
     Iterator<TabletMetadata> tabletIter =
         TabletsMetadata.builder().forTable(tableId).overlapping(startRow, null).checkConsistency()
-            .fetch(PREV_ROW, LOCATION, LOADED).build(master.getContext()).iterator();
+            .fetch(PREV_ROW, LOCATION, LOADED).build(manager.getContext()).iterator();
 
     Loader loader;
     if (bulkInfo.tableState == TableState.ONLINE) {
@@ -331,7 +331,7 @@
       loader = new OfflineLoader();
     }
 
-    loader.start(bulkDir, master, tid, bulkInfo.setTime);
+    loader.start(bulkDir, manager, tid, bulkInfo.setTime);
 
     long t1 = System.currentTimeMillis();
     while (lmi.hasNext()) {
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/PrepBulkImport.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/PrepBulkImport.java
index 823e222..05a6df6 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/PrepBulkImport.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/PrepBulkImport.java
@@ -88,15 +88,15 @@
   }
 
   @Override
-  public long isReady(long tid, Manager master) throws Exception {
-    if (!Utils.getReadLock(master, bulkInfo.tableId, tid).tryLock())
+  public long isReady(long tid, Manager manager) throws Exception {
+    if (!Utils.getReadLock(manager, bulkInfo.tableId, tid).tryLock())
       return 100;
 
-    if (master.onlineTabletServers().isEmpty())
+    if (manager.onlineTabletServers().isEmpty())
       return 500;
-    Tables.clearCache(master.getContext());
+    Tables.clearCache(manager.getContext());
 
-    return Utils.reserveHdfsDirectory(master, bulkInfo.sourceDir, tid);
+    return Utils.reserveHdfsDirectory(manager, bulkInfo.sourceDir, tid);
   }
 
   @VisibleForTesting
@@ -180,12 +180,12 @@
     }
   }
 
-  private void checkForMerge(final long tid, final Manager master) throws Exception {
+  private void checkForMerge(final long tid, final Manager manager) throws Exception {
 
-    VolumeManager fs = master.getVolumeManager();
+    VolumeManager fs = manager.getVolumeManager();
     final Path bulkDir = new Path(bulkInfo.sourceDir);
 
-    int maxTablets = Integer.parseInt(master.getContext().getTableConfiguration(bulkInfo.tableId)
+    int maxTablets = Integer.parseInt(manager.getContext().getTableConfiguration(bulkInfo.tableId)
         .get(Property.TABLE_BULK_MAX_TABLETS));
 
     try (LoadMappingIterator lmi =
@@ -193,25 +193,25 @@
 
       TabletIterFactory tabletIterFactory = startRow -> TabletsMetadata.builder()
           .forTable(bulkInfo.tableId).overlapping(startRow, null).checkConsistency().fetch(PREV_ROW)
-          .build(master.getContext()).stream().map(TabletMetadata::getExtent).iterator();
+          .build(manager.getContext()).stream().map(TabletMetadata::getExtent).iterator();
 
       sanityCheckLoadMapping(bulkInfo.tableId.canonical(), lmi, tabletIterFactory, maxTablets, tid);
     }
   }
 
   @Override
-  public Repo<Manager> call(final long tid, final Manager master) throws Exception {
+  public Repo<Manager> call(final long tid, final Manager manager) throws Exception {
     // now that table lock is acquired check that all splits in load mapping exists in table
-    checkForMerge(tid, master);
+    checkForMerge(tid, manager);
 
-    bulkInfo.tableState = Tables.getTableState(master.getContext(), bulkInfo.tableId);
+    bulkInfo.tableState = Tables.getTableState(manager.getContext(), bulkInfo.tableId);
 
-    VolumeManager fs = master.getVolumeManager();
-    final UniqueNameAllocator namer = master.getContext().getUniqueNameAllocator();
+    VolumeManager fs = manager.getVolumeManager();
+    final UniqueNameAllocator namer = manager.getContext().getUniqueNameAllocator();
     Path sourceDir = new Path(bulkInfo.sourceDir);
     List<FileStatus> files = BulkImport.filterInvalid(fs.listStatus(sourceDir));
 
-    Path bulkDir = createNewBulkDir(master.getContext(), fs, bulkInfo.tableId);
+    Path bulkDir = createNewBulkDir(manager.getContext(), fs, bulkInfo.tableId);
     Path mappingFile = new Path(sourceDir, Constants.BULK_LOAD_MAPPING);
 
     Map<String,String> oldToNewNameMap = new HashMap<>();
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/clone/CloneMetadata.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/clone/CloneMetadata.java
index f32c493..6cf0f26 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/clone/CloneMetadata.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/clone/CloneMetadata.java
@@ -46,7 +46,7 @@
     // need to clear out any metadata entries for tableId just in case this
     // died before and is executing again
     MetadataTableUtil.deleteTable(cloneInfo.tableId, false, environment.getContext(),
-        environment.getMasterLock());
+        environment.getManagerLock());
     MetadataTableUtil.cloneTable(environment.getContext(), cloneInfo.srcTableId, cloneInfo.tableId);
     return new FinishCloneTable(cloneInfo);
   }
@@ -54,7 +54,7 @@
   @Override
   public void undo(long tid, Manager environment) throws Exception {
     MetadataTableUtil.deleteTable(cloneInfo.tableId, false, environment.getContext(),
-        environment.getMasterLock());
+        environment.getManagerLock());
   }
 
 }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CompactionDriver.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CompactionDriver.java
index 1389f14..4ebaab1 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CompactionDriver.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CompactionDriver.java
@@ -64,17 +64,17 @@
   }
 
   @Override
-  public long isReady(long tid, Manager master) throws Exception {
+  public long isReady(long tid, Manager manager) throws Exception {
 
     if (tableId.equals(RootTable.ID)) {
       // this codes not properly handle the root table. See #798
       return 0;
     }
 
-    String zCancelID = Constants.ZROOT + "/" + master.getInstanceID() + Constants.ZTABLES + "/"
+    String zCancelID = Constants.ZROOT + "/" + manager.getInstanceID() + Constants.ZTABLES + "/"
         + tableId + Constants.ZTABLE_COMPACT_CANCEL_ID;
 
-    ZooReaderWriter zoo = master.getContext().getZooReaderWriter();
+    ZooReaderWriter zoo = manager.getContext().getZooReaderWriter();
 
     if (Long.parseLong(new String(zoo.getData(zCancelID))) >= compactId) {
       // compaction was canceled
@@ -90,7 +90,7 @@
 
     TabletsMetadata tablets =
         TabletsMetadata.builder().forTable(tableId).overlapping(startRow, endRow)
-            .fetch(LOCATION, PREV_ROW, COMPACT_ID).build(master.getContext());
+            .fetch(LOCATION, PREV_ROW, COMPACT_ID).build(manager.getContext());
 
     for (TabletMetadata tablet : tablets) {
       if (tablet.getCompactId().orElse(-1) < compactId) {
@@ -105,13 +105,13 @@
 
     long scanTime = System.currentTimeMillis() - t1;
 
-    Tables.clearCache(master.getContext());
-    if (tabletCount == 0 && !Tables.exists(master.getContext(), tableId))
+    Tables.clearCache(manager.getContext());
+    if (tabletCount == 0 && !Tables.exists(manager.getContext(), tableId))
       throw new AcceptableThriftTableOperationException(tableId.canonical(), null,
           TableOperation.COMPACT, TableOperationExceptionType.NOTFOUND, null);
 
     if (serversToFlush.size() == 0
-        && Tables.getTableState(master.getContext(), tableId) == TableState.OFFLINE)
+        && Tables.getTableState(manager.getContext(), tableId) == TableState.OFFLINE)
       throw new AcceptableThriftTableOperationException(tableId.canonical(), null,
           TableOperation.COMPACT, TableOperationExceptionType.OFFLINE, null);
 
@@ -120,9 +120,9 @@
 
     for (TServerInstance tsi : serversToFlush.keySet()) {
       try {
-        final TServerConnection server = master.getConnection(tsi);
+        final TServerConnection server = manager.getConnection(tsi);
         if (server != null)
-          server.compact(master.getMasterLock(), tableId.canonical(), startRow, endRow);
+          server.compact(manager.getManagerLock(), tableId.canonical(), startRow, endRow);
       } catch (TException ex) {
         LoggerFactory.getLogger(CompactionDriver.class).error(ex.toString());
       }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/ChooseDir.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/ChooseDir.java
index 3fd74f3..c1d73c0 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/ChooseDir.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/ChooseDir.java
@@ -54,21 +54,21 @@
   }
 
   @Override
-  public Repo<Manager> call(long tid, Manager master) throws Exception {
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
     if (tableInfo.getInitialSplitSize() > 0) {
-      createTableDirectoriesInfo(master);
+      createTableDirectoriesInfo(manager);
     }
     return new PopulateMetadata(tableInfo);
   }
 
   @Override
-  public void undo(long tid, Manager master) throws Exception {
+  public void undo(long tid, Manager manager) throws Exception {
     // Clean up split files if ChooseDir operation fails
     Path p = null;
     try {
       if (tableInfo.getInitialSplitSize() > 0) {
         p = tableInfo.getSplitDirsPath();
-        FileSystem fs = p.getFileSystem(master.getContext().getHadoopConf());
+        FileSystem fs = p.getFileSystem(manager.getContext().getHadoopConf());
         fs.delete(p, true);
       }
     } catch (IOException e) {
@@ -81,19 +81,19 @@
    * Create unique table directory names that will be associated with split values. Then write these
    * to the file system for later use during this FATE operation.
    */
-  private void createTableDirectoriesInfo(Manager master) throws IOException {
-    SortedSet<Text> splits = Utils.getSortedSetFromFile(master, tableInfo.getSplitPath(), true);
-    SortedSet<Text> tabletDirectoryInfo = createTabletDirectoriesSet(master, splits.size());
-    writeTabletDirectoriesToFileSystem(master, tabletDirectoryInfo);
+  private void createTableDirectoriesInfo(Manager manager) throws IOException {
+    SortedSet<Text> splits = Utils.getSortedSetFromFile(manager, tableInfo.getSplitPath(), true);
+    SortedSet<Text> tabletDirectoryInfo = createTabletDirectoriesSet(manager, splits.size());
+    writeTabletDirectoriesToFileSystem(manager, tabletDirectoryInfo);
   }
 
   /**
    * Create a set of unique table directories. These will be associated with splits in a follow-on
    * FATE step.
    */
-  private static SortedSet<Text> createTabletDirectoriesSet(Manager master, int num) {
+  private static SortedSet<Text> createTabletDirectoriesSet(Manager manager, int num) {
     String tabletDir;
-    UniqueNameAllocator namer = master.getContext().getUniqueNameAllocator();
+    UniqueNameAllocator namer = manager.getContext().getUniqueNameAllocator();
     SortedSet<Text> splitDirs = new TreeSet<>();
     for (int i = 0; i < num; i++) {
       tabletDir = Constants.GENERATED_TABLET_DIRECTORY_PREFIX + namer.getNextName();
@@ -106,10 +106,10 @@
    * Write the SortedSet of Tablet Directory names to the file system for use in the next phase of
    * the FATE operation.
    */
-  private void writeTabletDirectoriesToFileSystem(Manager master, SortedSet<Text> dirs)
+  private void writeTabletDirectoriesToFileSystem(Manager manager, SortedSet<Text> dirs)
       throws IOException {
     Path p = tableInfo.getSplitDirsPath();
-    FileSystem fs = p.getFileSystem(master.getContext().getHadoopConf());
+    FileSystem fs = p.getFileSystem(manager.getContext().getHadoopConf());
     if (fs.exists(p))
       fs.delete(p, true);
     try (FSDataOutputStream stream = fs.create(p)) {
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/CreateTable.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/CreateTable.java
index 0025cbd..583e189 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/CreateTable.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/CreateTable.java
@@ -65,17 +65,17 @@
   }
 
   @Override
-  public Repo<Manager> call(long tid, Manager master) throws Exception {
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
     // first step is to reserve a table id.. if the machine fails during this step
     // it is ok to retry... the only side effect is that a table id may not be used
     // or skipped
 
-    // assuming only the master process is creating tables
+    // assuming only the manager process is creating tables
 
     Utils.getIdLock().lock();
     try {
       String tName = tableInfo.getTableName();
-      tableInfo.setTableId(Utils.getNextId(tName, master.getContext(), TableId::of));
+      tableInfo.setTableId(Utils.getNextId(tName, manager.getContext(), TableId::of));
       return new SetupPermissions(tableInfo);
     } finally {
       Utils.getIdLock().unlock();
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/PopulateMetadata.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/PopulateMetadata.java
index d7c2653..b9398f3 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/PopulateMetadata.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/PopulateMetadata.java
@@ -67,7 +67,7 @@
   public Repo<Manager> call(long tid, Manager env) throws Exception {
     KeyExtent extent = new KeyExtent(tableInfo.getTableId(), null, null);
     MetadataTableUtil.addTablet(extent, ServerColumnFamily.DEFAULT_TABLET_DIR_NAME,
-        env.getContext(), tableInfo.getTimeType(), env.getMasterLock());
+        env.getContext(), tableInfo.getTimeType(), env.getManagerLock());
 
     if (tableInfo.getInitialSplitSize() > 0) {
       SortedSet<Text> splits = Utils.getSortedSetFromFile(env, tableInfo.getSplitPath(), true);
@@ -75,7 +75,7 @@
       Map<Text,Text> splitDirMap = createSplitDirectoryMap(splits, dirs);
       try (BatchWriter bw = env.getContext().createBatchWriter(MetadataTable.NAME)) {
         writeSplitsToMetadataTable(env.getContext(), tableInfo.getTableId(), splits, splitDirMap,
-            tableInfo.getTimeType(), env.getMasterLock(), bw);
+            tableInfo.getTimeType(), env.getManagerLock(), bw);
       }
     }
     return new FinishCreateTable(tableInfo);
@@ -102,7 +102,7 @@
   @Override
   public void undo(long tid, Manager environment) throws Exception {
     MetadataTableUtil.deleteTable(tableInfo.getTableId(), false, environment.getContext(),
-        environment.getMasterLock());
+        environment.getManagerLock());
   }
 
   /**
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/PopulateZookeeper.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/PopulateZookeeper.java
index 4ecd9f6..0554cea 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/PopulateZookeeper.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/create/PopulateZookeeper.java
@@ -46,23 +46,23 @@
   }
 
   @Override
-  public Repo<Manager> call(long tid, Manager master) throws Exception {
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
     // reserve the table name in zookeeper or fail
 
     Utils.getTableNameLock().lock();
     try {
       // write tableName & tableId to zookeeper
-      Utils.checkTableDoesNotExist(master.getContext(), tableInfo.getTableName(),
+      Utils.checkTableDoesNotExist(manager.getContext(), tableInfo.getTableName(),
           tableInfo.getTableId(), TableOperation.CREATE);
 
-      master.getTableManager().addTable(tableInfo.getTableId(), tableInfo.getNamespaceId(),
+      manager.getTableManager().addTable(tableInfo.getTableId(), tableInfo.getNamespaceId(),
           tableInfo.getTableName());
 
       for (Entry<String,String> entry : tableInfo.props.entrySet())
-        TablePropUtil.setTableProperty(master.getContext(), tableInfo.getTableId(), entry.getKey(),
+        TablePropUtil.setTableProperty(manager.getContext(), tableInfo.getTableId(), entry.getKey(),
             entry.getValue());
 
-      Tables.clearCache(master.getContext());
+      Tables.clearCache(manager.getContext());
       return new ChooseDir(tableInfo);
     } finally {
       Utils.getTableNameLock().unlock();
@@ -71,10 +71,10 @@
   }
 
   @Override
-  public void undo(long tid, Manager master) throws Exception {
-    master.getTableManager().removeTable(tableInfo.getTableId());
-    Utils.unreserveTable(master, tableInfo.getTableId(), tid, true);
-    Tables.clearCache(master.getContext());
+  public void undo(long tid, Manager manager) throws Exception {
+    manager.getTableManager().removeTable(tableInfo.getTableId());
+    Utils.unreserveTable(manager, tableInfo.getTableId(), tid, true);
+    Tables.clearCache(manager.getContext());
   }
 
 }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/CleanUp.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/CleanUp.java
index c3eb64e..d74b71a 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/CleanUp.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/CleanUp.java
@@ -86,24 +86,24 @@
   }
 
   @Override
-  public long isReady(long tid, Manager master) throws Exception {
-    if (!master.hasCycled(creationTime)) {
+  public long isReady(long tid, Manager manager) throws Exception {
+    if (!manager.hasCycled(creationTime)) {
       return 50;
     }
 
     boolean done = true;
     Range tableRange = new KeyExtent(tableId, null, null).toMetaRange();
-    Scanner scanner = master.getContext().createScanner(MetadataTable.NAME, Authorizations.EMPTY);
-    MetaDataTableScanner.configureScanner(scanner, master);
+    Scanner scanner = manager.getContext().createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    MetaDataTableScanner.configureScanner(scanner, manager);
     scanner.setRange(tableRange);
 
     for (Entry<Key,Value> entry : scanner) {
       TabletLocationState locationState =
           MetaDataTableScanner.createTabletLocationState(entry.getKey(), entry.getValue());
-      TabletState state = locationState.getState(master.onlineTabletServers());
+      TabletState state = locationState.getState(manager.onlineTabletServers());
       if (!state.equals(TabletState.UNASSIGNED)) {
         // This code will even wait on tablets that are assigned to dead tablets servers. This is
-        // intentional because the master may make metadata writes for these tablets. See #587
+        // intentional because the manager may make metadata writes for these tablets. See #587
         log.debug("Still waiting for table to be deleted: " + tableId + " locationState: "
             + locationState);
         done = false;
@@ -118,15 +118,15 @@
   }
 
   @Override
-  public Repo<Manager> call(long tid, Manager master) {
+  public Repo<Manager> call(long tid, Manager manager) {
 
-    master.clearMigrations(tableId);
+    manager.clearMigrations(tableId);
 
     int refCount = 0;
 
     try {
       // look for other tables that references this table's files
-      AccumuloClient client = master.getContext();
+      AccumuloClient client = manager.getContext();
       try (BatchScanner bs =
           client.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 8)) {
         Range allTables = TabletsSection.getRange();
@@ -155,19 +155,19 @@
 
     // remove metadata table entries
     try {
-      // Intentionally do not pass master lock. If master loses lock, this operation may complete
-      // before master can kill itself.
-      // If the master lock passed to deleteTable, it is possible that the delete mutations will be
+      // Intentionally do not pass manager lock. If manager loses lock, this operation may complete
+      // before manager can kill itself.
+      // If the manager lock passed to deleteTable, it is possible that the delete mutations will be
       // dropped. If the delete operations
       // are dropped and the operation completes, then the deletes will not be repeated.
-      MetadataTableUtil.deleteTable(tableId, refCount != 0, master.getContext(), null);
+      MetadataTableUtil.deleteTable(tableId, refCount != 0, manager.getContext(), null);
     } catch (Exception e) {
       log.error("error deleting " + tableId + " from metadata table", e);
     }
 
     // remove any problem reports the table may have
     try {
-      ProblemReports.getInstance(master.getContext()).deleteProblemReports(tableId);
+      ProblemReports.getInstance(manager.getContext()).deleteProblemReports(tableId);
     } catch (Exception e) {
       log.error("Failed to delete problem reports for table " + tableId, e);
     }
@@ -175,8 +175,8 @@
     if (refCount == 0) {
       // delete the map files
       try {
-        VolumeManager fs = master.getVolumeManager();
-        for (String dir : ServerConstants.getTablesDirs(master.getContext())) {
+        VolumeManager fs = manager.getVolumeManager();
+        for (String dir : ServerConstants.getTablesDirs(manager.getContext())) {
           fs.deleteRecursively(new Path(dir, tableId.canonical()));
         }
       } catch (IOException e) {
@@ -193,22 +193,22 @@
 
     // remove table from zookeeper
     try {
-      master.getTableManager().removeTable(tableId);
-      Tables.clearCache(master.getContext());
+      manager.getTableManager().removeTable(tableId);
+      Tables.clearCache(manager.getContext());
     } catch (Exception e) {
       log.error("Failed to find table id in zookeeper", e);
     }
 
     // remove any permissions associated with this table
     try {
-      AuditedSecurityOperation.getInstance(master.getContext())
-          .deleteTable(master.getContext().rpcCreds(), tableId, namespaceId);
+      AuditedSecurityOperation.getInstance(manager.getContext())
+          .deleteTable(manager.getContext().rpcCreds(), tableId, namespaceId);
     } catch (ThriftSecurityException e) {
       log.error("{}", e.getMessage(), e);
     }
 
-    Utils.unreserveTable(master, tableId, tid, true);
-    Utils.unreserveNamespace(master, namespaceId, tid, false);
+    Utils.unreserveTable(manager, tableId, tid, true);
+    Utils.unreserveNamespace(manager, namespaceId, tid, false);
 
     LoggerFactory.getLogger(CleanUp.class).debug("Deleted table " + tableId);
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/TableRangeOp.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/TableRangeOp.java
index d9a80ec..68f6fc8 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/TableRangeOp.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/TableRangeOp.java
@@ -94,7 +94,7 @@
 
   @Override
   public void undo(long tid, Manager env) throws Exception {
-    // Not sure this is a good thing to do. The Master state engine should be the one to remove it.
+    // Not sure this is a good thing to do. The Manager state engine should be the one to remove it.
     MergeInfo mergeInfo = env.getMergeInfo(tableId);
     if (mergeInfo.getState() != MergeState.NONE)
       log.info("removing merge information {}", mergeInfo);
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/TableRangeOpWait.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/TableRangeOpWait.java
index b4f354d..4bdadaa 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/TableRangeOpWait.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/TableRangeOpWait.java
@@ -67,12 +67,12 @@
   }
 
   @Override
-  public Repo<Manager> call(long tid, Manager master) throws Exception {
-    MergeInfo mergeInfo = master.getMergeInfo(tableId);
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
+    MergeInfo mergeInfo = manager.getMergeInfo(tableId);
     log.info("removing merge information " + mergeInfo);
-    master.clearMergeState(tableId);
-    Utils.unreserveTable(master, tableId, tid, true);
-    Utils.unreserveNamespace(master, namespaceId, tid, false);
+    manager.clearMergeState(tableId);
+    Utils.unreserveTable(manager, tableId, tid, true);
+    Utils.unreserveNamespace(manager, namespaceId, tid, false);
     return null;
   }
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/create/CreateNamespace.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/create/CreateNamespace.java
index b4e4c6a..6f31e2c 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/create/CreateNamespace.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/create/CreateNamespace.java
@@ -44,11 +44,11 @@
   }
 
   @Override
-  public Repo<Manager> call(long tid, Manager master) throws Exception {
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
     Utils.getIdLock().lock();
     try {
       namespaceInfo.namespaceId =
-          Utils.getNextId(namespaceInfo.namespaceName, master.getContext(), NamespaceId::of);
+          Utils.getNextId(namespaceInfo.namespaceName, manager.getContext(), NamespaceId::of);
       return new SetupNamespacePermissions(namespaceInfo);
     } finally {
       Utils.getIdLock().unlock();
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/create/PopulateZookeeperWithNamespace.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/create/PopulateZookeeperWithNamespace.java
index 3011c59..1f0c656 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/create/PopulateZookeeperWithNamespace.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/create/PopulateZookeeperWithNamespace.java
@@ -47,22 +47,22 @@
   }
 
   @Override
-  public Repo<Manager> call(long tid, Manager master) throws Exception {
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
 
     Utils.getTableNameLock().lock();
     try {
-      Utils.checkNamespaceDoesNotExist(master.getContext(), namespaceInfo.namespaceName,
+      Utils.checkNamespaceDoesNotExist(manager.getContext(), namespaceInfo.namespaceName,
           namespaceInfo.namespaceId, TableOperation.CREATE);
 
-      TableManager.prepareNewNamespaceState(master.getContext().getZooReaderWriter(),
-          master.getInstanceID(), namespaceInfo.namespaceId, namespaceInfo.namespaceName,
+      TableManager.prepareNewNamespaceState(manager.getContext().getZooReaderWriter(),
+          manager.getInstanceID(), namespaceInfo.namespaceId, namespaceInfo.namespaceName,
           NodeExistsPolicy.OVERWRITE);
 
       for (Entry<String,String> entry : namespaceInfo.props.entrySet())
-        NamespacePropUtil.setNamespaceProperty(master.getContext(), namespaceInfo.namespaceId,
+        NamespacePropUtil.setNamespaceProperty(manager.getContext(), namespaceInfo.namespaceId,
             entry.getKey(), entry.getValue());
 
-      Tables.clearCache(master.getContext());
+      Tables.clearCache(manager.getContext());
 
       return new FinishCreateNamespace(namespaceInfo);
     } finally {
@@ -71,10 +71,10 @@
   }
 
   @Override
-  public void undo(long tid, Manager master) throws Exception {
-    master.getTableManager().removeNamespace(namespaceInfo.namespaceId);
-    Tables.clearCache(master.getContext());
-    Utils.unreserveNamespace(master, namespaceInfo.namespaceId, tid, true);
+  public void undo(long tid, Manager manager) throws Exception {
+    manager.getTableManager().removeNamespace(namespaceInfo.namespaceId);
+    Tables.clearCache(manager.getContext());
+    Utils.unreserveNamespace(manager, namespaceInfo.namespaceId, tid, true);
   }
 
 }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/delete/NamespaceCleanUp.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/delete/NamespaceCleanUp.java
index 4bab467..db1ceb4 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/delete/NamespaceCleanUp.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/delete/NamespaceCleanUp.java
@@ -42,30 +42,30 @@
   }
 
   @Override
-  public long isReady(long tid, Manager master) {
+  public long isReady(long tid, Manager manager) {
     return 0;
   }
 
   @Override
-  public Repo<Manager> call(long id, Manager master) {
+  public Repo<Manager> call(long id, Manager manager) {
 
     // remove from zookeeper
     try {
-      master.getTableManager().removeNamespace(namespaceId);
+      manager.getTableManager().removeNamespace(namespaceId);
     } catch (Exception e) {
       log.error("Failed to find namespace in zookeeper", e);
     }
-    Tables.clearCache(master.getContext());
+    Tables.clearCache(manager.getContext());
 
     // remove any permissions associated with this namespace
     try {
-      AuditedSecurityOperation.getInstance(master.getContext())
-          .deleteNamespace(master.getContext().rpcCreds(), namespaceId);
+      AuditedSecurityOperation.getInstance(manager.getContext())
+          .deleteNamespace(manager.getContext().rpcCreds(), namespaceId);
     } catch (ThriftSecurityException e) {
       log.error("{}", e.getMessage(), e);
     }
 
-    Utils.unreserveNamespace(master, namespaceId, id, true);
+    Utils.unreserveNamespace(manager, namespaceId, id, true);
 
     log.debug("Deleted namespace " + namespaceId);
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/rename/RenameNamespace.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/rename/RenameNamespace.java
index e6ba512..38102fb 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/rename/RenameNamespace.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/namespace/rename/RenameNamespace.java
@@ -52,16 +52,16 @@
   }
 
   @Override
-  public Repo<Manager> call(long id, Manager master) throws Exception {
+  public Repo<Manager> call(long id, Manager manager) throws Exception {
 
-    ZooReaderWriter zoo = master.getContext().getZooReaderWriter();
+    ZooReaderWriter zoo = manager.getContext().getZooReaderWriter();
 
     Utils.getTableNameLock().lock();
     try {
-      Utils.checkNamespaceDoesNotExist(master.getContext(), newName, namespaceId,
+      Utils.checkNamespaceDoesNotExist(manager.getContext(), newName, namespaceId,
           TableOperation.RENAME);
 
-      final String tap = master.getZooKeeperRoot() + Constants.ZNAMESPACES + "/" + namespaceId
+      final String tap = manager.getZooKeeperRoot() + Constants.ZNAMESPACES + "/" + namespaceId
           + Constants.ZNAMESPACE_NAME;
 
       zoo.mutateExisting(tap, current -> {
@@ -74,10 +74,10 @@
         }
         return newName.getBytes(UTF_8);
       });
-      Tables.clearCache(master.getContext());
+      Tables.clearCache(manager.getContext());
     } finally {
       Utils.getTableNameLock().unlock();
-      Utils.unreserveNamespace(master, namespaceId, id, true);
+      Utils.unreserveNamespace(manager, namespaceId, id, true);
     }
 
     LoggerFactory.getLogger(RenameNamespace.class).debug("Renamed namespace {} {} {}", namespaceId,
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/rename/RenameTable.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/rename/RenameTable.java
index 9abc62f..a6380d6 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/rename/RenameTable.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/rename/RenameTable.java
@@ -60,29 +60,29 @@
   }
 
   @Override
-  public Repo<Manager> call(long tid, Manager master) throws Exception {
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
     Pair<String,String> qualifiedOldTableName = Tables.qualify(oldTableName);
     Pair<String,String> qualifiedNewTableName = Tables.qualify(newTableName);
 
     // ensure no attempt is made to rename across namespaces
     if (newTableName.contains(".") && !namespaceId
-        .equals(Namespaces.getNamespaceId(master.getContext(), qualifiedNewTableName.getFirst())))
+        .equals(Namespaces.getNamespaceId(manager.getContext(), qualifiedNewTableName.getFirst())))
       throw new AcceptableThriftTableOperationException(tableId.canonical(), oldTableName,
           TableOperation.RENAME, TableOperationExceptionType.INVALID_NAME,
           "Namespace in new table name does not match the old table name");
 
-    ZooReaderWriter zoo = master.getContext().getZooReaderWriter();
+    ZooReaderWriter zoo = manager.getContext().getZooReaderWriter();
 
     Utils.getTableNameLock().lock();
     try {
-      Utils.checkTableDoesNotExist(master.getContext(), newTableName, tableId,
+      Utils.checkTableDoesNotExist(manager.getContext(), newTableName, tableId,
           TableOperation.RENAME);
 
       final String newName = qualifiedNewTableName.getSecond();
       final String oldName = qualifiedOldTableName.getSecond();
 
       final String tap =
-          master.getZooKeeperRoot() + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_NAME;
+          manager.getZooKeeperRoot() + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_NAME;
 
       zoo.mutateExisting(tap, current -> {
         final String currentName = new String(current, UTF_8);
@@ -95,11 +95,11 @@
         }
         return newName.getBytes(UTF_8);
       });
-      Tables.clearCache(master.getContext());
+      Tables.clearCache(manager.getContext());
     } finally {
       Utils.getTableNameLock().unlock();
-      Utils.unreserveTable(master, tableId, tid, true);
-      Utils.unreserveNamespace(master, namespaceId, tid, false);
+      Utils.unreserveTable(manager, tableId, tid, true);
+      Utils.unreserveNamespace(manager, namespaceId, tid, false);
     }
 
     LoggerFactory.getLogger(RenameTable.class).debug("Renamed table {} {} {}", tableId,
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableExport/WriteExportFiles.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableExport/WriteExportFiles.java
index 94b1f69..bb604ae 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableExport/WriteExportFiles.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableExport/WriteExportFiles.java
@@ -90,17 +90,17 @@
   }
 
   @Override
-  public long isReady(long tid, Manager master) throws Exception {
+  public long isReady(long tid, Manager manager) throws Exception {
 
-    long reserved = Utils.reserveNamespace(master, tableInfo.namespaceID, tid, false, true,
+    long reserved = Utils.reserveNamespace(manager, tableInfo.namespaceID, tid, false, true,
         TableOperation.EXPORT)
-        + Utils.reserveTable(master, tableInfo.tableID, tid, false, true, TableOperation.EXPORT);
+        + Utils.reserveTable(manager, tableInfo.tableID, tid, false, true, TableOperation.EXPORT);
     if (reserved > 0)
       return reserved;
 
-    AccumuloClient client = master.getContext();
+    AccumuloClient client = manager.getContext();
 
-    checkOffline(master.getContext());
+    checkOffline(manager.getContext());
 
     Scanner metaScanner = client.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
     metaScanner.setRange(new KeyExtent(tableInfo.tableID, null, null).toMetaRange());
@@ -130,18 +130,18 @@
   }
 
   @Override
-  public Repo<Manager> call(long tid, Manager master) throws Exception {
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
     try {
-      exportTable(master.getVolumeManager(), master.getContext(), tableInfo.tableName,
+      exportTable(manager.getVolumeManager(), manager.getContext(), tableInfo.tableName,
           tableInfo.tableID, tableInfo.exportDir);
     } catch (IOException ioe) {
       throw new AcceptableThriftTableOperationException(tableInfo.tableID.canonical(),
           tableInfo.tableName, TableOperation.EXPORT, TableOperationExceptionType.OTHER,
           "Failed to create export files " + ioe.getMessage());
     }
-    Utils.unreserveNamespace(master, tableInfo.namespaceID, tid, false);
-    Utils.unreserveTable(master, tableInfo.tableID, tid, false);
-    Utils.unreserveHdfsDirectory(master, new Path(tableInfo.exportDir).toString(), tid);
+    Utils.unreserveNamespace(manager, tableInfo.namespaceID, tid, false);
+    Utils.unreserveTable(manager, tableInfo.tableID, tid, false);
+    Utils.unreserveHdfsDirectory(manager, new Path(tableInfo.exportDir).toString(), tid);
     return null;
   }
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/CreateImportDir.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/CreateImportDir.java
index b8d33e4..e427c52 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/CreateImportDir.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/CreateImportDir.java
@@ -42,11 +42,11 @@
   }
 
   @Override
-  public Repo<Manager> call(long tid, Manager master) throws Exception {
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
 
-    Set<String> tableDirs = ServerConstants.getTablesDirs(master.getContext());
+    Set<String> tableDirs = ServerConstants.getTablesDirs(manager.getContext());
 
-    create(tableDirs, master);
+    create(tableDirs, manager);
 
     return new MapImportFileNames(tableInfo);
   }
@@ -58,19 +58,19 @@
    * @param tableDirs
    *          the set of table directories on HDFS where files will be moved e.g:
    *          hdfs://volume1/accumulo/tables/
-   * @param master
-   *          the master instance performing the table import.
+   * @param manager
+   *          the manager instance performing the table import.
    * @throws IOException
    *           if any import directory does not reside on a volume configured for accumulo.
    */
-  void create(Set<String> tableDirs, Manager master) throws IOException {
-    UniqueNameAllocator namer = master.getContext().getUniqueNameAllocator();
+  void create(Set<String> tableDirs, Manager manager) throws IOException {
+    UniqueNameAllocator namer = manager.getContext().getUniqueNameAllocator();
 
     for (ImportedTableInfo.DirectoryMapping dm : tableInfo.directories) {
       Path exportDir = new Path(dm.exportDir);
 
       log.info("Looking for matching filesystem for {} from options {}", exportDir, tableDirs);
-      Path base = master.getVolumeManager().matchingFileSystem(exportDir, tableDirs);
+      Path base = manager.getVolumeManager().matchingFileSystem(exportDir, tableDirs);
       if (base == null) {
         throw new IOException(
             dm.exportDir + " is not in the same file system as any volume configured for Accumulo");
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/ImportTable.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/ImportTable.java
index 39eb0db..bd5e168 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/ImportTable.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/ImportTable.java
@@ -86,7 +86,7 @@
     // it is ok to retry... the only side effect is that a table id may not be used
     // or skipped
 
-    // assuming only the master process is creating tables
+    // assuming only the manager process is creating tables
 
     Utils.getIdLock().lock();
     try {
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/MoveExportedFiles.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/MoveExportedFiles.java
index 20bd0dd..3f2c2b0 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/MoveExportedFiles.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/MoveExportedFiles.java
@@ -55,11 +55,11 @@
   }
 
   @Override
-  public Repo<Manager> call(long tid, Manager master) throws Exception {
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
     String fmtTid = FateTxId.formatTid(tid);
 
-    int workerCount = master.getConfiguration().getCount(Property.MANAGER_RENAME_THREADS);
-    VolumeManager fs = master.getVolumeManager();
+    int workerCount = manager.getConfiguration().getCount(Property.MANAGER_RENAME_THREADS);
+    VolumeManager fs = manager.getVolumeManager();
     Map<Path,Path> oldToNewPaths = new HashMap<>();
 
     for (ImportedTableInfo.DirectoryMapping dm : tableInfo.directories) {
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/PopulateMetadataTable.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/PopulateMetadataTable.java
index 4868f1d..4a33ad2 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/PopulateMetadataTable.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/tableImport/PopulateMetadataTable.java
@@ -87,7 +87,7 @@
   }
 
   @Override
-  public Repo<Manager> call(long tid, Manager master) throws Exception {
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
 
     Path path = new Path(tableInfo.exportFile);
 
@@ -95,9 +95,9 @@
     ZipInputStream zis = null;
 
     try {
-      VolumeManager fs = master.getVolumeManager();
+      VolumeManager fs = manager.getVolumeManager();
 
-      mbw = master.getContext().createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
+      mbw = manager.getContext().createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
 
       zis = new ZipInputStream(fs.open(path));
 
@@ -206,6 +206,6 @@
   @Override
   public void undo(long tid, Manager environment) throws Exception {
     MetadataTableUtil.deleteTable(tableInfo.tableId, false, environment.getContext(),
-        environment.getMasterLock());
+        environment.getManagerLock());
   }
 }
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tserverOps/ShutdownTServer.java b/server/manager/src/main/java/org/apache/accumulo/manager/tserverOps/ShutdownTServer.java
index 314abd0..cb1a0c0 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/tserverOps/ShutdownTServer.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/tserverOps/ShutdownTServer.java
@@ -50,24 +50,24 @@
   }
 
   @Override
-  public long isReady(long tid, Manager master) {
+  public long isReady(long tid, Manager manager) {
     TServerInstance server = new TServerInstance(hostAndPort, serverSession);
     // suppress assignment of tablets to the server
     if (force) {
       return 0;
     }
 
-    // Inform the master that we want this server to shutdown
-    master.shutdownTServer(server);
+    // Inform the manager that we want this server to shutdown
+    manager.shutdownTServer(server);
 
-    if (master.onlineTabletServers().contains(server)) {
-      TServerConnection connection = master.getConnection(server);
+    if (manager.onlineTabletServers().contains(server)) {
+      TServerConnection connection = manager.getConnection(server);
       if (connection != null) {
         try {
           TabletServerStatus status = connection.getTableMap(false);
           if (status.tableMap != null && status.tableMap.isEmpty()) {
             log.info("tablet server hosts no tablets {}", server);
-            connection.halt(master.getMasterLock());
+            connection.halt(manager.getManagerLock());
             log.info("tablet server asked to halt {}", server);
             return 0;
           }
@@ -78,7 +78,7 @@
         }
 
         // If the connection was non-null and we could communicate with it
-        // give the master some more time to tell it to stop and for the
+        // give the manager some more time to tell it to stop and for the
         // tserver to ack the request and stop itself.
         return 1000;
       }
@@ -88,13 +88,13 @@
   }
 
   @Override
-  public Repo<Manager> call(long tid, Manager master) throws Exception {
+  public Repo<Manager> call(long tid, Manager manager) throws Exception {
     // suppress assignment of tablets to the server
     if (force) {
-      ZooReaderWriter zoo = master.getContext().getZooReaderWriter();
-      String path = master.getZooKeeperRoot() + Constants.ZTSERVERS + "/" + hostAndPort;
+      ZooReaderWriter zoo = manager.getContext().getZooReaderWriter();
+      String path = manager.getZooKeeperRoot() + Constants.ZTSERVERS + "/" + hostAndPort;
       ZooLock.deleteLock(zoo, path);
-      path = master.getZooKeeperRoot() + Constants.ZDEADTSERVERS + "/" + hostAndPort;
+      path = manager.getZooKeeperRoot() + Constants.ZDEADTSERVERS + "/" + hostAndPort;
       zoo.putPersistentData(path, "forced down".getBytes(UTF_8), NodeExistsPolicy.OVERWRITE);
     }
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/RenameMasterDirInZK.java b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/RenameMasterDirInZK.java
new file mode 100644
index 0000000..022108d
--- /dev/null
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/RenameMasterDirInZK.java
@@ -0,0 +1,71 @@
+/*
+ * 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.manager.upgrade;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.conf.SiteConfiguration;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A utility to handle the renaming of "/masters" to "/managers" in Zookeeper when upgrading from a
+ * 2.0 (or earlier) to 2.1 instance. This utility is invoked automatically by
+ * {@link org.apache.accumulo.manager.state.SetGoalState} (which normally runs first as a part of
+ * accumulo startup scripts). However, if a user is not using the standard scripts or wishes to
+ * perform the upgrade as a separate process, this utility can be invoked with:
+ *
+ * <pre>
+ * {@code
+ * bin/accumulo org.apache.accumulo.manager.upgrade.RenameMasterDirInZK
+ * }
+ * </pre>
+ */
+public class RenameMasterDirInZK {
+  private static final Logger LOG = LoggerFactory.getLogger(RenameMasterDirInZK.class);
+
+  public static void main(String[] args) {
+    var ctx = new ServerContext(SiteConfiguration.auto());
+    if (!renameMasterDirInZK(ctx)) {
+      LOG.info(
+          "Masters directory in ZooKeeper has already been renamed to managers. No action was taken.");
+    }
+  }
+
+  public static boolean renameMasterDirInZK(ServerContext ctx) {
+    final ZooReaderWriter zoo = ctx.getZooReaderWriter();
+    final String mastersZooDir = ctx.getZooKeeperRoot() + "/masters";
+    final String managersZooDir = ctx.getZooKeeperRoot() + Constants.ZMANAGERS;
+    try {
+      boolean mastersDirExists = zoo.exists(mastersZooDir);
+      if (mastersDirExists) {
+        LOG.info("Copying ZooKeeper directory {} to {}.", mastersZooDir, managersZooDir);
+        zoo.recursiveCopyPersistentOverwrite(mastersZooDir, managersZooDir);
+        LOG.info("Deleting ZooKeeper directory {}.", mastersZooDir);
+        zoo.recursiveDelete(mastersZooDir, ZooUtil.NodeMissingPolicy.SKIP);
+      }
+      return mastersDirExists;
+    } catch (KeeperException | InterruptedException e) {
+      throw new RuntimeException("Unable to rename " + mastersZooDir + " in ZooKeeper", e);
+    }
+  }
+}
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java
index e4c5f48..fb5f283 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/UpgradeCoordinator.java
@@ -116,7 +116,7 @@
   }
 
   @SuppressFBWarnings(value = "DM_EXIT",
-      justification = "Want to immediately stop all master threads on upgrade error")
+      justification = "Want to immediately stop all manager threads on upgrade error")
   private void handleFailure(Exception e) {
     log.error("FATAL: Error performing upgrade", e);
     // do not want to call setStatus and signal an event in this case
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader9to10.java b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader9to10.java
index 48c99b6..d88ebd2 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader9to10.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader9to10.java
@@ -246,7 +246,7 @@
               // Earlier, it was checked that root tablet metadata did not exists. However the
               // earlier check does handle race conditions. Race conditions are unexpected. This is
               // a sanity check when making the update in ZK using compare and set. If this fails
-              // and its not a bug, then its likely some concurrency issue. For example two masters
+              // and its not a bug, then its likely some concurrency issue. For example two managers
               // concurrently running upgrade could cause this to fail.
               Preconditions.checkState(currVal.length == 0,
                   "Expected root tablet metadata to be empty!");
diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/util/FateAdmin.java b/server/manager/src/main/java/org/apache/accumulo/manager/util/FateAdmin.java
index 2ea268b..3b620bc 100644
--- a/server/manager/src/main/java/org/apache/accumulo/manager/util/FateAdmin.java
+++ b/server/manager/src/main/java/org/apache/accumulo/manager/util/FateAdmin.java
@@ -82,19 +82,19 @@
     try (var context = new ServerContext(SiteConfiguration.auto())) {
       final String zkRoot = context.getZooKeeperRoot();
       String path = zkRoot + Constants.ZFATE;
-      String masterPath = zkRoot + Constants.ZMASTER_LOCK;
+      String managerPath = zkRoot + Constants.ZMANAGER_LOCK;
       ZooReaderWriter zk = context.getZooReaderWriter();
       ZooStore<Manager> zs = new ZooStore<>(path, zk);
 
       if (jc.getParsedCommand().equals("fail")) {
         for (String txid : txOpts.get(jc.getParsedCommand()).txids) {
-          if (!admin.prepFail(zs, zk, masterPath, txid)) {
+          if (!admin.prepFail(zs, zk, managerPath, txid)) {
             System.exit(1);
           }
         }
       } else if (jc.getParsedCommand().equals("delete")) {
         for (String txid : txOpts.get(jc.getParsedCommand()).txids) {
-          if (!admin.prepDelete(zs, zk, masterPath, txid)) {
+          if (!admin.prepDelete(zs, zk, managerPath, txid)) {
             System.exit(1);
           }
           admin.deleteLocks(zk, zkRoot + Constants.ZTABLE_LOCKS, txid);
diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/metrics/ReplicationMetricsTest.java b/server/manager/src/test/java/org/apache/accumulo/manager/metrics/ReplicationMetricsTest.java
index ba7cfa1..486bc30 100644
--- a/server/manager/src/test/java/org/apache/accumulo/manager/metrics/ReplicationMetricsTest.java
+++ b/server/manager/src/test/java/org/apache/accumulo/manager/metrics/ReplicationMetricsTest.java
@@ -39,8 +39,8 @@
    * Extend the class to override the current time for testing
    */
   public class ReplicationMetricsTestMetrics extends ReplicationMetrics {
-    ReplicationMetricsTestMetrics(Manager master) {
-      super(master);
+    ReplicationMetricsTestMetrics(Manager manager) {
+      super(manager);
     }
 
     @Override
@@ -51,7 +51,7 @@
 
   @Test
   public void testAddReplicationQueueTimeMetrics() throws Exception {
-    Manager master = EasyMock.createMock(Manager.class);
+    Manager manager = EasyMock.createMock(Manager.class);
     ServerContext context = EasyMock.createMock(ServerContext.class);
     VolumeManager fileSystem = EasyMock.createMock(VolumeManager.class);
     ReplicationUtil util = EasyMock.createMock(ReplicationUtil.class);
@@ -62,11 +62,11 @@
     Path path2 = new Path("hdfs://localhost:9000/accumulo/wal/file2");
 
     // First call will initialize the map of paths to modification time
-    EasyMock.expect(master.getContext()).andReturn(context).anyTimes();
+    EasyMock.expect(manager.getContext()).andReturn(context).anyTimes();
     EasyMock.expect(util.getPendingReplicationPaths()).andReturn(Set.of(path1, path2));
-    EasyMock.expect(master.getVolumeManager()).andReturn(fileSystem);
+    EasyMock.expect(manager.getVolumeManager()).andReturn(fileSystem);
     EasyMock.expect(fileSystem.getFileStatus(path1)).andReturn(createStatus(100));
-    EasyMock.expect(master.getVolumeManager()).andReturn(fileSystem);
+    EasyMock.expect(manager.getVolumeManager()).andReturn(fileSystem);
     EasyMock.expect(fileSystem.getFileStatus(path2)).andReturn(createStatus(200));
 
     // Second call will recognize the missing path1 and add the latency stat
@@ -83,9 +83,9 @@
     stat.add(currentTime - 100);
     EasyMock.expectLastCall();
 
-    EasyMock.replay(master, fileSystem, util, stat, quantiles);
+    EasyMock.replay(manager, fileSystem, util, stat, quantiles);
 
-    ReplicationMetrics metrics = new ReplicationMetricsTestMetrics(master);
+    ReplicationMetrics metrics = new ReplicationMetricsTestMetrics(manager);
 
     // Inject our mock objects
     replaceField(metrics, "replicationUtil", util);
@@ -96,7 +96,7 @@
     metrics.addReplicationQueueTimeMetrics();
     metrics.addReplicationQueueTimeMetrics();
 
-    EasyMock.verify(master, fileSystem, util, stat, quantiles);
+    EasyMock.verify(manager, fileSystem, util, stat, quantiles);
   }
 
   private void replaceField(Object instance, String fieldName, Object target)
diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/replication/ManagerReplicationCoordinatorTest.java b/server/manager/src/test/java/org/apache/accumulo/manager/replication/ManagerReplicationCoordinatorTest.java
index 32d1bb8..924d95b 100644
--- a/server/manager/src/test/java/org/apache/accumulo/manager/replication/ManagerReplicationCoordinatorTest.java
+++ b/server/manager/src/test/java/org/apache/accumulo/manager/replication/ManagerReplicationCoordinatorTest.java
@@ -39,15 +39,15 @@
 
   @Test
   public void randomServer() {
-    Manager master = EasyMock.createMock(Manager.class);
+    Manager manager = EasyMock.createMock(Manager.class);
     ZooReader reader = EasyMock.createMock(ZooReader.class);
     ServerContext context = EasyMock.createMock(ServerContext.class);
     EasyMock.expect(context.getConfiguration()).andReturn(config).anyTimes();
-    EasyMock.expect(master.getContext()).andReturn(context);
-    EasyMock.expect(master.getInstanceID()).andReturn("1234");
-    EasyMock.replay(master, reader);
+    EasyMock.expect(manager.getContext()).andReturn(context);
+    EasyMock.expect(manager.getInstanceID()).andReturn("1234");
+    EasyMock.replay(manager, reader);
 
-    ManagerReplicationCoordinator coordinator = new ManagerReplicationCoordinator(master, reader);
+    ManagerReplicationCoordinator coordinator = new ManagerReplicationCoordinator(manager, reader);
     TServerInstance inst1 = new TServerInstance(HostAndPort.fromParts("host1", 1234), "session");
 
     assertEquals(inst1, coordinator.getRandomTServer(Collections.singleton(inst1), 0));
@@ -55,15 +55,15 @@
 
   @Test(expected = IllegalArgumentException.class)
   public void invalidOffset() {
-    Manager master = EasyMock.createMock(Manager.class);
+    Manager manager = EasyMock.createMock(Manager.class);
     ServerContext context = EasyMock.createMock(ServerContext.class);
     EasyMock.expect(context.getConfiguration()).andReturn(config).anyTimes();
     ZooReader reader = EasyMock.createMock(ZooReader.class);
-    EasyMock.expect(master.getContext()).andReturn(context);
-    EasyMock.expect(master.getInstanceID()).andReturn("1234");
-    EasyMock.replay(master, reader);
+    EasyMock.expect(manager.getContext()).andReturn(context);
+    EasyMock.expect(manager.getInstanceID()).andReturn("1234");
+    EasyMock.replay(manager, reader);
 
-    ManagerReplicationCoordinator coordinator = new ManagerReplicationCoordinator(master, reader);
+    ManagerReplicationCoordinator coordinator = new ManagerReplicationCoordinator(manager, reader);
     TServerInstance inst1 = new TServerInstance(HostAndPort.fromParts("host1", 1234), "session");
 
     assertEquals(inst1, coordinator.getRandomTServer(Collections.singleton(inst1), 1));
@@ -71,19 +71,19 @@
 
   @Test
   public void randomServerFromMany() {
-    Manager master = EasyMock.createMock(Manager.class);
+    Manager manager = EasyMock.createMock(Manager.class);
     ZooReader reader = EasyMock.createMock(ZooReader.class);
     ServerContext context = EasyMock.createMock(ServerContext.class);
     EasyMock.expect(context.getConfiguration()).andReturn(config).anyTimes();
     EasyMock.expect(context.getInstanceID()).andReturn("1234").anyTimes();
     EasyMock.expect(context.getZooReaderWriter()).andReturn(null).anyTimes();
-    EasyMock.expect(master.getInstanceID()).andReturn("1234").anyTimes();
-    EasyMock.expect(master.getContext()).andReturn(context).anyTimes();
-    EasyMock.replay(master, context, reader);
+    EasyMock.expect(manager.getInstanceID()).andReturn("1234").anyTimes();
+    EasyMock.expect(manager.getContext()).andReturn(context).anyTimes();
+    EasyMock.replay(manager, context, reader);
 
-    ManagerReplicationCoordinator coordinator = new ManagerReplicationCoordinator(master, reader);
+    ManagerReplicationCoordinator coordinator = new ManagerReplicationCoordinator(manager, reader);
 
-    EasyMock.verify(master, reader);
+    EasyMock.verify(manager, reader);
 
     TreeSet<TServerInstance> instances = new TreeSet<>();
     TServerInstance inst1 = new TServerInstance(HostAndPort.fromParts("host1", 1234), "session");
diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/ShutdownTServerTest.java b/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/ShutdownTServerTest.java
index 6fd367f..3113b7f 100644
--- a/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/ShutdownTServerTest.java
+++ b/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/ShutdownTServerTest.java
@@ -45,7 +45,7 @@
 
     final ShutdownTServer op = new ShutdownTServer(tserver, force);
 
-    final Manager master = EasyMock.createMock(Manager.class);
+    final Manager manager = EasyMock.createMock(Manager.class);
     final long tid = 1L;
 
     final TServerConnection tserverCnxn = EasyMock.createMock(TServerConnection.class);
@@ -54,44 +54,44 @@
     // Put in a table info record, don't care what
     status.tableMap.put("a_table", new TableInfo());
 
-    master.shutdownTServer(tserver);
+    manager.shutdownTServer(tserver);
     EasyMock.expectLastCall().once();
-    EasyMock.expect(master.onlineTabletServers()).andReturn(Collections.singleton(tserver));
-    EasyMock.expect(master.getConnection(tserver)).andReturn(tserverCnxn);
+    EasyMock.expect(manager.onlineTabletServers()).andReturn(Collections.singleton(tserver));
+    EasyMock.expect(manager.getConnection(tserver)).andReturn(tserverCnxn);
     EasyMock.expect(tserverCnxn.getTableMap(false)).andReturn(status);
 
-    EasyMock.replay(tserverCnxn, master);
+    EasyMock.replay(tserverCnxn, manager);
 
     // FATE op is not ready
-    long wait = op.isReady(tid, master);
+    long wait = op.isReady(tid, manager);
     assertTrue("Expected wait to be greater than 0", wait > 0);
 
-    EasyMock.verify(tserverCnxn, master);
+    EasyMock.verify(tserverCnxn, manager);
 
     // Reset the mocks
-    EasyMock.reset(tserverCnxn, master);
+    EasyMock.reset(tserverCnxn, manager);
 
     // reset the table map to the empty set to simulate all tablets unloaded
     status.tableMap = new HashMap<>();
-    master.shutdownTServer(tserver);
+    manager.shutdownTServer(tserver);
     EasyMock.expectLastCall().once();
-    EasyMock.expect(master.onlineTabletServers()).andReturn(Collections.singleton(tserver));
-    EasyMock.expect(master.getConnection(tserver)).andReturn(tserverCnxn);
+    EasyMock.expect(manager.onlineTabletServers()).andReturn(Collections.singleton(tserver));
+    EasyMock.expect(manager.getConnection(tserver)).andReturn(tserverCnxn);
     EasyMock.expect(tserverCnxn.getTableMap(false)).andReturn(status);
-    EasyMock.expect(master.getMasterLock()).andReturn(null);
+    EasyMock.expect(manager.getManagerLock()).andReturn(null);
     tserverCnxn.halt(null);
     EasyMock.expectLastCall().once();
 
-    EasyMock.replay(tserverCnxn, master);
+    EasyMock.replay(tserverCnxn, manager);
 
     // FATE op is not ready
-    wait = op.isReady(tid, master);
+    wait = op.isReady(tid, manager);
     assertTrue("Expected wait to be 0", wait == 0);
 
-    Repo<Manager> op2 = op.call(tid, master);
+    Repo<Manager> op2 = op.call(tid, manager);
     assertNull("Expected no follow on step", op2);
 
-    EasyMock.verify(tserverCnxn, master);
+    EasyMock.verify(tserverCnxn, manager);
   }
 
 }
diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/tableImport/ImportTableTest.java b/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/tableImport/ImportTableTest.java
index 1f07a00..242d3e4 100644
--- a/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/tableImport/ImportTableTest.java
+++ b/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/tableImport/ImportTableTest.java
@@ -75,7 +75,7 @@
 
   @Test
   public void testCreateImportDir() throws Exception {
-    Manager master = EasyMock.createMock(Manager.class);
+    Manager manager = EasyMock.createMock(Manager.class);
     ServerContext context = EasyMock.createMock(ServerContext.class);
     VolumeManager volumeManager = EasyMock.createMock(VolumeManager.class);
     UniqueNameAllocator uniqueNameAllocator = EasyMock.createMock(UniqueNameAllocator.class);
@@ -90,8 +90,8 @@
 
     String dirName = "abcd";
 
-    EasyMock.expect(master.getContext()).andReturn(context);
-    EasyMock.expect(master.getVolumeManager()).andReturn(volumeManager).times(3);
+    EasyMock.expect(manager.getContext()).andReturn(context);
+    EasyMock.expect(manager.getVolumeManager()).andReturn(volumeManager).times(3);
     EasyMock.expect(context.getUniqueNameAllocator()).andReturn(uniqueNameAllocator);
     EasyMock.expect(volumeManager.matchingFileSystem(EasyMock.eq(new Path(expDirs[0])),
         EasyMock.eq(tableDirSet))).andReturn(new Path(tableDirs[0]));
@@ -106,10 +106,10 @@
     ti.directories = ImportTable.parseExportDir(Set.of(expDirs));
     assertEquals(3, ti.directories.size());
 
-    EasyMock.replay(master, context, volumeManager, uniqueNameAllocator);
+    EasyMock.replay(manager, context, volumeManager, uniqueNameAllocator);
 
     CreateImportDir ci = new CreateImportDir(ti);
-    ci.create(tableDirSet, master);
+    ci.create(tableDirSet, manager);
     assertEquals(3, ti.directories.size());
     for (ImportedTableInfo.DirectoryMapping dm : ti.directories) {
       assertNotNull(dm.exportDir);
@@ -119,7 +119,7 @@
       assertTrue(
           dm.importDir.contains(ti.tableId.canonical() + "/" + Constants.BULK_PREFIX + dirName));
     }
-    EasyMock.verify(master, context, volumeManager, uniqueNameAllocator);
+    EasyMock.verify(manager, context, volumeManager, uniqueNameAllocator);
   }
 
   private static void assertMatchingFilesystem(String expected, String target) {
diff --git a/server/manager/src/test/resources/hadoop-metrics2-accumulo.properties b/server/manager/src/test/resources/hadoop-metrics2-accumulo.properties
index 73ed2e3..9689807 100644
--- a/server/manager/src/test/resources/hadoop-metrics2-accumulo.properties
+++ b/server/manager/src/test/resources/hadoop-metrics2-accumulo.properties
@@ -45,13 +45,13 @@
 # accumulo.sink.file-tserver.context=tserver
 # accumulo.sink.file-tserver.filename=tserver.metrics
 
-# File sink for master metrics
-accumulo.sink.file-master.class=org.apache.hadoop.metrics2.sink.FileSink
-accumulo.sink.file-master.context=master
-accumulo.sink.file-master.filename=./target/master.metrics
-accumulo.sink.file-master.period=5
+# File sink for manager metrics
+accumulo.sink.file-manager.class=org.apache.hadoop.metrics2.sink.FileSink
+accumulo.sink.file-manager.context=manager
+accumulo.sink.file-manager.filename=./target/manager.metrics
+accumulo.sink.file-manager.period=5
 
-accumulo.jmx.master.context=master
+accumulo.jmx.manager.context=manager
 
 # File sink for thrift server metrics
 # accumulo.sink.file-thrift.class=org.apache.hadoop.metrics2.sink.FileSink
diff --git a/server/manager/src/test/resources/log4j2-test.properties b/server/manager/src/test/resources/log4j2-test.properties
index 06d4c39..b58c293 100644
--- a/server/manager/src/test/resources/log4j2-test.properties
+++ b/server/manager/src/test/resources/log4j2-test.properties
@@ -19,7 +19,7 @@
 
 status = info
 dest = err
-name = AccumuloMasterTestLoggingProperties
+name = AccumuloManagerTestLoggingProperties
 
 appender.console.type = Console
 appender.console.name = STDOUT
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
index a8a3c47..6438b8f 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
@@ -91,7 +91,7 @@
 import org.slf4j.LoggerFactory;
 
 /**
- * Serve master statistics with an embedded web server.
+ * Serve manager statistics with an embedded web server.
  */
 public class Monitor extends AbstractServer implements HighlyAvailableService {
 
@@ -259,7 +259,7 @@
             retry = false;
           } else {
             mmi = null;
-            log.error("Unable to get info from Master");
+            log.error("Unable to get info from Manager");
           }
           gcStatus = fetchGcStatus();
         } catch (Exception e) {
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/SummaryInformation.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/SummaryInformation.java
index cb6959d..05c9f85 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/SummaryInformation.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/SummaryInformation.java
@@ -42,7 +42,7 @@
   // Variable names become JSON keys
   public List<TabletServer> servers = new ArrayList<>();
 
-  public String masterGoalState, masterState;
+  public String managerGoalState, managerState;
 
   public BadTabletServers badTabletServers;
   public ServersShuttingDown tabletServersShuttingDown;
@@ -63,15 +63,15 @@
    * @param size
    *          Number of tservers
    * @param info
-   *          Master information
+   *          Manager information
    * @param tablesList
    *          Table list
    */
   public SummaryInformation(int size, ManagerInformation info, TableInformationList tablesList) {
     this.servers = new ArrayList<>(size);
 
-    this.masterGoalState = info.masterGoalState;
-    this.masterState = info.masterState;
+    this.managerGoalState = info.managerGoalState;
+    this.managerState = info.managerState;
 
     this.badTabletServers = info.badTabletServers;
     this.tabletServersShuttingDown = info.tabletServersShuttingDown;
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/manager/ManagerInformation.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/manager/ManagerInformation.java
index e425e9c..d35f40f 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/manager/ManagerInformation.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/manager/ManagerInformation.java
@@ -24,18 +24,18 @@
 import org.apache.accumulo.monitor.rest.tservers.ServersShuttingDown;
 
 /**
- * Responsible for storing master information as a JSON object
+ * Responsible for storing manager information as a JSON object
  *
  * @since 2.0.0
  */
 public class ManagerInformation {
 
   // Variable names become JSON keys
-  public String master = "No Masters running";
+  public String manager = "No Managers running";
   public String lastGC = "0";
   public String gcStatus;
-  public String masterGoalState;
-  public String masterState;
+  public String managerGoalState;
+  public String managerState;
 
   public Integer onlineTabletServers = 0;
   public Integer totalTabletServers = 0;
@@ -61,19 +61,19 @@
   public DeadLoggerList deadLoggers;
 
   /**
-   * Creates an empty master JSON object
+   * Creates an empty manager JSON object
    */
   public ManagerInformation() {}
 
-  public ManagerInformation(String master) {
-    this.master = master;
+  public ManagerInformation(String manager) {
+    this.manager = manager;
   }
 
   /**
-   * Stores a new master JSON object
+   * Stores a new manager JSON object
    *
-   * @param master
-   *          Master location
+   * @param manager
+   *          Manager location
    * @param onlineTabletServers
    *          Number of online tservers
    * @param totalTabletServers
@@ -106,10 +106,10 @@
    *          Time the Monitor has been running
    * @param gcStatus
    *          Status of the garbage collector
-   * @param masterGoalState
-   *          Goal state of the master
-   * @param masterState
-   *          Current state of the master
+   * @param managerGoalState
+   *          Goal state of the manager
+   * @param managerState
+   *          Current state of the manager
    * @param badTabletServers
    *          Number of bad tservers
    * @param tabletServersShuttingDown
@@ -119,15 +119,15 @@
    * @param deadLoggers
    *          Number of dead loggers
    */
-  public ManagerInformation(String master, int onlineTabletServers, int totalTabletServers,
+  public ManagerInformation(String manager, int onlineTabletServers, int totalTabletServers,
       String lastGC, int tablets, int unassignedTablets, long entries, double ingest,
       double entriesRead, double entriesReturned, long holdTime, double osLoad, int tables,
       int deadTabletServersCount, long lookups, long uptime, String gcStatus,
-      String masterGoalState, String masterState, BadTabletServers badTabletServers,
+      String managerGoalState, String managerState, BadTabletServers badTabletServers,
       ServersShuttingDown tabletServersShuttingDown, DeadServerList deadTabletServers,
       DeadLoggerList deadLoggers) {
 
-    this.master = master;
+    this.manager = manager;
     this.onlineTabletServers = onlineTabletServers;
     this.totalTabletServers = totalTabletServers;
     this.lastGC = lastGC;
@@ -144,8 +144,8 @@
     this.lookups = lookups;
     this.uptime = uptime;
     this.gcStatus = gcStatus;
-    this.masterGoalState = masterGoalState;
-    this.masterState = masterState;
+    this.managerGoalState = managerGoalState;
+    this.managerState = managerState;
     this.badTabletServers = badTabletServers;
     this.tabletServersShuttingDown = tabletServersShuttingDown;
     this.deadTabletServers = deadTabletServers;
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/manager/ManagerResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/manager/ManagerResource.java
index 409bdf4..fba45f7 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/manager/ManagerResource.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/manager/ManagerResource.java
@@ -47,22 +47,22 @@
 import org.apache.accumulo.server.manager.state.TabletServerState;
 
 /**
- * Responsible for generating a new Master information JSON object
+ * Responsible for generating a new Manager information JSON object
  *
  * @since 2.0.0
  */
-@Path("/master")
+@Path("/manager")
 @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
 public class ManagerResource {
-  public static final String NO_MASTERS = "No Masters running";
+  public static final String NO_MANAGERS = "No Managers running";
 
   @Inject
   private Monitor monitor;
 
   /**
-   * Generates a master information JSON object
+   * Generates a manager information JSON object
    *
-   * @return master JSON object
+   * @return manager JSON object
    */
   @GET
   public ManagerInformation getTables() {
@@ -99,10 +99,10 @@
       for (DeadServer down : mmi.deadTabletServers) {
         tservers.add(down.server);
       }
-      List<String> masters = monitor.getContext().getMasterLocations();
+      List<String> managers = monitor.getContext().getManagerLocations();
 
-      String master =
-          masters.isEmpty() ? "Down" : AddressUtil.parseAddress(masters.get(0), false).getHost();
+      String manager =
+          managers.isEmpty() ? "Down" : AddressUtil.parseAddress(managers.get(0), false).getHost();
       int onlineTabletServers = mmi.tServerInfo.size();
       int totalTabletServers = tservers.size();
       int tablets = monitor.getTotalTabletCount();
@@ -119,7 +119,7 @@
       long lookups = monitor.getTotalLookups();
       long uptime = System.currentTimeMillis() - monitor.getStartTime();
 
-      managerInformation = new ManagerInformation(master, onlineTabletServers, totalTabletServers,
+      managerInformation = new ManagerInformation(manager, onlineTabletServers, totalTabletServers,
           gcStatus, tablets, unassignedTablets, entries, ingest, entriesRead, entriesReturned,
           holdTime, osLoad, tables, deadTabletServers, lookups, uptime, label,
           getGoalState(monitor), getState(monitor), getNumBadTservers(monitor),
@@ -131,27 +131,27 @@
   }
 
   /**
-   * Returns the current state of the master
+   * Returns the current state of the manager
    *
-   * @return master state
+   * @return manager state
    */
   public static String getState(Monitor monitor) {
     ManagerMonitorInfo mmi = monitor.getMmi();
     if (mmi == null) {
-      return NO_MASTERS;
+      return NO_MANAGERS;
     }
     return mmi.state.toString();
   }
 
   /**
-   * Returns the goal state of the master
+   * Returns the goal state of the manager
    *
-   * @return master goal state
+   * @return manager goal state
    */
   public static String getGoalState(Monitor monitor) {
     ManagerMonitorInfo mmi = monitor.getMmi();
     if (mmi == null) {
-      return NO_MASTERS;
+      return NO_MANAGERS;
     }
     return mmi.goalState.name();
   }
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/status/StatusInformation.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/status/StatusInformation.java
index 2fc5b23..0c9e853 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/status/StatusInformation.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/status/StatusInformation.java
@@ -19,14 +19,14 @@
 package org.apache.accumulo.monitor.rest.status;
 
 /**
- * Generates status of master, gc, and tservers, as well as log and problem report
+ * Generates status of manager, gc, and tservers, as well as log and problem report
  *
  * @since 2.0.0
  */
 public class StatusInformation {
 
   // Variable names become JSON keys
-  public String masterStatus = null;
+  public String managerStatus = null;
   public String gcStatus = null;
   public String tServerStatus = null;
 
@@ -39,8 +39,8 @@
   /**
    * Generate the status report for the services
    *
-   * @param masterStatus
-   *          Status for the master
+   * @param managerStatus
+   *          Status for the manager
    * @param gcStatus
    *          Status for the GC
    * @param tServerStatus
@@ -52,9 +52,9 @@
    * @param problemNumber
    *          Number of problems per table
    */
-  public StatusInformation(String masterStatus, String gcStatus, String tServerStatus,
+  public StatusInformation(String managerStatus, String gcStatus, String tServerStatus,
       Integer logNumber, boolean logsHaveError, Integer problemNumber) {
-    this.masterStatus = masterStatus;
+    this.managerStatus = managerStatus;
     this.gcStatus = gcStatus;
     this.tServerStatus = tServerStatus;
     this.logNumber = logNumber;
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/status/StatusResource.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/status/StatusResource.java
index f9a8d35..4f88b56 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/status/StatusResource.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/status/StatusResource.java
@@ -30,7 +30,7 @@
 import org.apache.accumulo.monitor.Monitor;
 
 /**
- * Generates the status for master, gc, and tservers as well as log and problem reports
+ * Generates the status for manager, gc, and tservers as well as log and problem reports
  *
  * @since 2.0.0
  */
@@ -53,7 +53,7 @@
   @GET
   public StatusInformation getTables() {
 
-    Status masterStatus;
+    Status managerStatus;
     Status gcStatus;
     Status tServerStatus = Status.ERROR;
     ManagerMonitorInfo mmi = monitor.getMmi();
@@ -65,8 +65,8 @@
         gcStatus = Status.ERROR;
       }
 
-      List<String> masters = monitor.getContext().getMasterLocations();
-      masterStatus = masters.isEmpty() ? Status.ERROR : Status.OK;
+      List<String> managers = monitor.getContext().getManagerLocations();
+      managerStatus = managers.isEmpty() ? Status.ERROR : Status.OK;
 
       int tServerUp = mmi.getTServerInfoSize();
       int tServerDown = mmi.getDeadTabletServersSize();
@@ -85,7 +85,7 @@
         tServerStatus = Status.ERROR;
       }
     } else {
-      masterStatus = Status.ERROR;
+      managerStatus = Status.ERROR;
       if (monitor.getGcStatus() == null) {
         gcStatus = Status.ERROR;
       } else {
@@ -94,7 +94,7 @@
       tServerStatus = Status.ERROR;
     }
 
-    return new StatusInformation(masterStatus.toString(), gcStatus.toString(),
+    return new StatusInformation(managerStatus.toString(), gcStatus.toString(),
         tServerStatus.toString(), monitor.recentLogs().numEvents(),
         monitor.recentLogs().eventsIncludeErrors(), monitor.getProblemSummary().entrySet().size());
   }
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServers.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServers.java
index 5d82776..a6cee6d 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServers.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/tservers/TabletServers.java
@@ -45,7 +45,7 @@
    * Adds bad and dead servers to the list
    *
    * @param info
-   *          Master information to get bad and dead server information
+   *          Manager information to get bad and dead server information
    */
   public void addBadTabletServer(ManagerInformation info) {
     badServers = info.badTabletServers.badTabletServer;
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/view/WebViews.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/view/WebViews.java
index 3f1c95e..c59d0d4 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/view/WebViews.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/view/WebViews.java
@@ -127,19 +127,19 @@
   }
 
   /**
-   * Returns the master template
+   * Returns the manager template
    *
-   * @return Master model
+   * @return Manager model
    */
   @GET
-  @Path("{parameter: master|monitor}")
+  @Path("{parameter: manager|monitor}")
   @Template(name = "/default.ftl")
-  public Map<String,Object> getMaster() {
+  public Map<String,Object> getManager() {
 
     Map<String,Object> model = getModel();
-    model.put("title", "Master Server");
-    model.put("template", "master.ftl");
-    model.put("js", "master.js");
+    model.put("title", "Manager Server");
+    model.put("template", "manager.ftl");
+    model.put("js", "manager.js");
 
     model.put("tablesTitle", "Table Status");
     model.put("tablesTemplate", "tables.ftl");
diff --git a/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/bulkImport.js b/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/bulkImport.js
index 27587b2..544443e 100644
--- a/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/bulkImport.js
+++ b/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/bulkImport.js
@@ -42,11 +42,11 @@
 }
 
 /**
- * Generates the master bulk import status table
+ * Generates the manager bulk import status table
  */
 function refreshBulkImportTable() {
 
-  clearTableBody('masterBulkImportStatus');
+  clearTableBody('managerBulkImportStatus');
 
   /*
    * Get the bulk import value obtained earlier, if it doesn't exists,
@@ -61,7 +61,7 @@
   if (data.length === 0 || data.bulkImport.length === 0) {
     $('<tr/>', {
       html: createEmptyRow(3, 'Empty')
-    }).appendTo('#masterBulkImportStatus tbody');
+    }).appendTo('#managerBulkImportStatus tbody');
   } else {
     $.each(data.bulkImport, function(key, val) {
       var items = [];
@@ -70,7 +70,7 @@
       items.push(createRightCell(val.state, val.state));
       $('<tr/>', {
         html: items.join('')
-      }).appendTo('#masterBulkImportStatus tbody');
+      }).appendTo('#managerBulkImportStatus tbody');
     });
   }
 
diff --git a/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/functions.js b/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/functions.js
index 21a910d..9cc5ff9 100644
--- a/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/functions.js
+++ b/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/functions.js
@@ -258,12 +258,12 @@
 ///// REST Calls /////////////
 
 /**
- * REST GET call for the master information,
+ * REST GET call for the manager information,
  * stores it on a sessionStorage variable
  */
-function getMaster() {
-  return $.getJSON('/rest/master', function(data) {
-    sessionStorage.master = JSON.stringify(data);
+function getManager() {
+  return $.getJSON('/rest/manager', function(data) {
+    sessionStorage.manager = JSON.stringify(data);
   });
 }
 
diff --git a/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/master.js b/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/manager.js
similarity index 81%
rename from server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/master.js
rename to server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/manager.js
index caafa01..924b920 100644
--- a/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/master.js
+++ b/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/manager.js
@@ -18,18 +18,18 @@
  */
 
 /**
- * Creates master initial table
+ * Creates manager initial table
  */
 $(document).ready(function() {
-  refreshMaster();
+  refreshManager();
 });
 
 /**
  * Makes the REST calls, generates the tables with the new information
  */
-function refreshMaster() {
-  getMaster().then(function() {
-    refreshMasterTable();
+function refreshManager() {
+  getManager().then(function() {
+    refreshManagerTable();
   });
   getRecoveryList().then(function() {
     recoveryList();
@@ -39,13 +39,13 @@
 
 /*
  * The tables refresh function will do this functionality
- * If tables are removed from Master, uncomment this function
+ * If tables are removed from Manager, uncomment this function
  */
 /**
  * Used to redraw the page
  */
 /*function refresh() {
-  refreshMaster();
+  refreshManager();
 }*/
 
 /**
@@ -85,25 +85,25 @@
 }
 
 /**
- * Generates the master table
+ * Generates the manager table
  */
-function refreshMasterTable() {
-  // Gets the master status
-  var status = JSON.parse(sessionStorage.status).masterStatus;
+function refreshManagerTable() {
+  // Gets the manager status
+  var status = JSON.parse(sessionStorage.status).managerStatus;
 
-  // Hide the banner and the master table
-  $('#masterBanner').hide();
-  clearTableBody('masterStatus');
-  $('#masterStatus').hide();
+  // Hide the banner and the manager table
+  $('#managerBanner').hide();
+  clearTableBody('managerStatus');
+  $('#managerStatus').hide();
 
-  // If master status is error, show banner, otherwise, create master table
+  // If manager status is error, show banner, otherwise, create manager table
   if (status === 'ERROR') {
-    $('#masterBanner').show();
+    $('#managerBanner').show();
   } else {
-    $('#masterStatus').show();
-    var data = JSON.parse(sessionStorage.master);
+    $('#managerStatus').show();
+    var data = JSON.parse(sessionStorage.manager);
     var items = [];
-    items.push(createFirstCell(data.master, data.master));
+    items.push(createFirstCell(data.manager, data.manager));
 
     items.push(createRightCell(data.onlineTabletServers,
         data.onlineTabletServers));
@@ -141,6 +141,6 @@
 
     $('<tr/>', {
      html: items.join('')
-    }).appendTo('#masterStatus tbody');
+    }).appendTo('#managerStatus tbody');
   }
 }
diff --git a/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/navbar.js b/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/navbar.js
index fc46cb9..7053a54 100644
--- a/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/navbar.js
+++ b/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/navbar.js
@@ -49,10 +49,10 @@
       undefined : JSON.parse(sessionStorage.status);
 
   // Setting individual status notification
-  if (data.masterStatus === 'OK') {
-    $('#masterStatusNotification').removeClass('error').addClass('normal');
+  if (data.managerStatus === 'OK') {
+    $('#managerStatusNotification').removeClass('error').addClass('normal');
   } else {
-    $('#masterStatusNotification').removeClass('normal').addClass('error');
+    $('#managerStatusNotification').removeClass('normal').addClass('error');
   }
   if (data.tServerStatus === 'OK') {
     $('#serverStatusNotification').removeClass('error').removeClass('warning').
@@ -71,12 +71,12 @@
   }
 
   // Setting overall status notification
-  if (data.masterStatus === 'OK' &&
+  if (data.managerStatus === 'OK' &&
       data.tServerStatus === 'OK' &&
       data.gcStatus === 'OK') {
     $('#statusNotification').removeClass('error').removeClass('warning').
         addClass('normal');
-  } else if (data.masterStatus === 'ERROR' ||
+  } else if (data.managerStatus === 'ERROR' ||
       data.tServerStatus === 'ERROR' ||
       data.gcStatus === 'ERROR') {
     $('#statusNotification').removeClass('normal').removeClass('warning').
diff --git a/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/overview.js b/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/overview.js
index e6b33b1..57c8840 100644
--- a/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/overview.js
+++ b/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/overview.js
@@ -28,8 +28,8 @@
  * Makes the REST calls, generates the tables with the new information
  */
 function refreshOverview() {
-  getMaster().then(function() {
-    refreshMasterTable();
+  getManager().then(function() {
+    refreshManagerTable();
   });
   getZK().then(function() {
     refreshZKTable();
@@ -59,21 +59,21 @@
 }
 
 /**
- * Refreshes the master table
+ * Refreshes the manager table
  */
-function refreshMasterTable() {
-  var data = sessionStorage.master === undefined ?
-      [] : JSON.parse(sessionStorage.master);
+function refreshManagerTable() {
+  var data = sessionStorage.manager === undefined ?
+      [] : JSON.parse(sessionStorage.manager);
 
-  $('#master tr td:first').hide();
-  $('#master tr td').hide();
+  $('#manager tr td:first').hide();
+  $('#manager tr td').hide();
 
-  // If the master is down, show the first row, otherwise refresh old values
-  if (data.length === 0 || data.master === 'No Masters running') {
-    $('#master tr td:first').show();
+  // If the manager is down, show the first row, otherwise refresh old values
+  if (data.length === 0 || data.manager === 'No Managers running') {
+    $('#manager tr td:first').show();
   } else {
-    $('#master tr td:not(:first)').show();
-    var table = $('#master td.right');
+    $('#manager tr td:not(:first)').show();
+    var table = $('#manager td.right');
 
     table.eq(0).html(bigNumberForQuantity(data.tables));
     table.eq(1).html(bigNumberForQuantity(data.totalTabletServers));
diff --git a/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/bulkImport.ftl b/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/bulkImport.ftl
index 5af827f..9277d52 100644
--- a/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/bulkImport.ftl
+++ b/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/bulkImport.ftl
@@ -25,7 +25,7 @@
       </div>
       <div class="row">
         <div class="col-xs-12">
-          <table id="masterBulkImportStatus" class="table table-bordered table-striped table-condensed">
+          <table id="managerBulkImportStatus" class="table table-bordered table-striped table-condensed">
             <caption><span class="table-caption">Legacy Bulk Imports</span><br/></caption>
             <thead>
               <tr>
diff --git a/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/listType.ftl b/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/listType.ftl
index 5b49c47..9ecc5df 100644
--- a/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/listType.ftl
+++ b/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/listType.ftl
@@ -37,7 +37,7 @@
       <div class="row">
         <div class="col-xs-12">
           <table id="trace" class="table table-bordered table-striped table-condensed">
-            <caption><span class="table-caption">Traces for masterReplicationDriver</span><br/></caption>
+            <caption><span class="table-caption">Traces for managerReplicationDriver</span><br/></caption>
             <thead>
               <tr>
                 <th class="firstcell" title="Start Time of selected trace type">Start</th>
diff --git a/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/master.ftl b/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/manager.ftl
similarity index 91%
rename from server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/master.ftl
rename to server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/manager.ftl
index 19ef519..bde976e 100644
--- a/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/master.ftl
+++ b/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/manager.ftl
@@ -23,13 +23,13 @@
           <h3>${title}</h3>
         </div>
       </div>
-      <div id="masterBanner" style="display: none;"><div class="alert alert-danger" role="alert">Master Server Not Running</div></div>
+      <div id="managerBanner" style="display: none;"><div class="alert alert-danger" role="alert">Manager Server Not Running</div></div>
       <div class="row">
         <div class="col-xs-12">
-          <table id="masterStatus" class="table table-bordered table-striped table-condensed">
+          <table id="managerStatus" class="table table-bordered table-striped table-condensed">
             <thead>
               <tr>
-                <th class="firstcell" title="The hostname of the master server">Hostname</th>
+                <th class="firstcell" title="The hostname of the manager server">Hostname</th>
                 <th title="Number of tablet servers currently available">Online TServers&nbsp;</th>
                 <th title="The total number of tablet servers configured">TotalTServers&nbsp;</th>
                 <th title="The last time files were cleaned-up from HDFS.">Last&nbsp;GC</th>
diff --git a/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/navbar.ftl b/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/navbar.ftl
index d85f6b8..e6f3da0 100644
--- a/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/navbar.ftl
+++ b/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/navbar.ftl
@@ -40,7 +40,7 @@
               <a class="dropdown-toggle" data-toggle="dropdown" href="#" role="button" aria-haspopup="true" aria-expanded="false"><span id="statusNotification" class="icon-dot normal"></span>&nbsp;Servers&nbsp;<span class="caret"></span>
               </a>
               <ul class="dropdown-menu">
-                <li><a href="/master"><span id="masterStatusNotification" class="icon-dot normal"></span>&nbsp;Master&nbsp;Server&nbsp;</a></li>
+                <li><a href="/manager"><span id="managerStatusNotification" class="icon-dot normal"></span>&nbsp;Manager&nbsp;Server&nbsp;</a></li>
                 <li><a href="/tservers"><span id="serverStatusNotification" class="icon-dot normal"></span>&nbsp;Tablet&nbsp;Servers&nbsp;</a></li>
                 <li><a href="/gc"><span id="gcStatusNotification" class="icon-dot normal"></span>&nbsp;Garbage&nbsp;collector&nbsp;</a></li>
               </ul>
diff --git a/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/overview.ftl b/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/overview.ftl
index a75be63..7e92c37 100644
--- a/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/overview.ftl
+++ b/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/overview.ftl
@@ -24,11 +24,11 @@
         </div>
       </div>
       <div class="row">
-        <div class="col-sm-6" id="master">
+        <div class="col-sm-6" id="manager">
           <table class="table table-bordered table-striped table-condensed">
             <thead>
-              <tr><th colspan="2"><a href="/master">Accumulo Master</a></th></tr>
-              <tr><td colspan="2" class="center" style="display:none;"><span class="label label-danger nowrap">Master is Down</span></td></tr>
+              <tr><th colspan="2"><a href="/manager">Accumulo Manager</a></th></tr>
+              <tr><td colspan="2" class="center" style="display:none;"><span class="label label-danger nowrap">Manager is Down</span></td></tr>
               <tr><td class="left"><a href="/tables">Tables</a></td><td class="right"></td></tr>
               <tr><td class="left"><a href="/tservers">Total&nbsp;Known&nbsp;Tablet&nbsp;Servers</a></td><td class="right"></td></tr>
               <tr><td class="left"><a href="/tservers">Dead&nbsp;Tablet&nbsp;Servers</a></td><td class="right"></td></tr>
diff --git a/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/tables.ftl b/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/tables.ftl
index d4b232b..80c79ee 100644
--- a/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/tables.ftl
+++ b/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/tables.ftl
@@ -81,7 +81,7 @@
          */
         function refresh() {
           <#if js??>
-            refreshMaster();
+            refreshManager();
           </#if>
 
           tableList.ajax.reload(null, false ); // user paging is not reset on reload
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/AssignmentHandler.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/AssignmentHandler.java
index cf753a9..678f388 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/AssignmentHandler.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/AssignmentHandler.java
@@ -37,7 +37,7 @@
 import org.apache.accumulo.server.problems.ProblemReports;
 import org.apache.accumulo.server.util.ManagerMetadataUtil;
 import org.apache.accumulo.tserver.TabletServerResourceManager.TabletResourceManager;
-import org.apache.accumulo.tserver.mastermessage.TabletStatusMessage;
+import org.apache.accumulo.tserver.managermessage.TabletStatusMessage;
 import org.apache.accumulo.tserver.tablet.Tablet;
 import org.apache.accumulo.tserver.tablet.TabletData;
 import org.apache.hadoop.io.Text;
@@ -132,7 +132,7 @@
         server.openingTablets.notifyAll();
       }
       log.warn("Failed to verify tablet " + extent, e);
-      server.enqueueMasterMessage(new TabletStatusMessage(TabletLoadState.LOAD_FAILURE, extent));
+      server.enqueueManagerMessage(new TabletStatusMessage(TabletLoadState.LOAD_FAILURE, extent));
       throw new RuntimeException(e);
     }
 
@@ -143,7 +143,7 @@
         server.openingTablets.remove(extent);
         server.openingTablets.notifyAll();
       }
-      server.enqueueMasterMessage(new TabletStatusMessage(TabletLoadState.LOAD_FAILURE, extent));
+      server.enqueueManagerMessage(new TabletStatusMessage(TabletLoadState.LOAD_FAILURE, extent));
       return;
     }
 
@@ -203,7 +203,7 @@
     }
 
     if (successful) {
-      server.enqueueMasterMessage(new TabletStatusMessage(TabletLoadState.LOADED, extent));
+      server.enqueueManagerMessage(new TabletStatusMessage(TabletLoadState.LOADED, extent));
     } else {
       synchronized (server.unopenedTablets) {
         synchronized (server.openingTablets) {
@@ -212,8 +212,8 @@
           server.openingTablets.notifyAll();
         }
       }
-      log.warn("failed to open tablet {} reporting failure to master", extent);
-      server.enqueueMasterMessage(new TabletStatusMessage(TabletLoadState.LOAD_FAILURE, extent));
+      log.warn("failed to open tablet {} reporting failure to manager", extent);
+      server.enqueueManagerMessage(new TabletStatusMessage(TabletLoadState.LOAD_FAILURE, extent));
       long reschedule = Math.min((1L << Math.min(32, retryAttempt)) * 1000, 10 * 60 * 1000L);
       log.warn(String.format("rescheduling tablet load in %.2f seconds", reschedule / 1000.));
       ThreadPools.createGeneralScheduledExecutorService(server.getConfiguration())
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index 87c8116..e6b46b6 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -134,8 +134,8 @@
 import org.apache.accumulo.tserver.log.LogSorter;
 import org.apache.accumulo.tserver.log.MutationReceiver;
 import org.apache.accumulo.tserver.log.TabletServerLogger;
-import org.apache.accumulo.tserver.mastermessage.ManagerMessage;
-import org.apache.accumulo.tserver.mastermessage.SplitReportMessage;
+import org.apache.accumulo.tserver.managermessage.ManagerMessage;
+import org.apache.accumulo.tserver.managermessage.SplitReportMessage;
 import org.apache.accumulo.tserver.metrics.CompactionExecutorsMetrics;
 import org.apache.accumulo.tserver.metrics.TabletServerMetrics;
 import org.apache.accumulo.tserver.metrics.TabletServerMinCMetrics;
@@ -173,7 +173,7 @@
   private static final long TIME_BETWEEN_LOCATOR_CACHE_CLEARS = 60 * 60 * 1000;
 
   final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
-  final ZooCache masterLockCache;
+  final ZooCache managerLockCache;
 
   final TabletServerLogger logger;
 
@@ -239,7 +239,7 @@
     super("tserver", opts, args);
     ServerContext context = super.getContext();
     context.setupCrypto();
-    this.masterLockCache = new ZooCache(context.getZooReaderWriter(), null);
+    this.managerLockCache = new ZooCache(context.getZooReaderWriter(), null);
     final AccumuloConfiguration aconf = getConfiguration();
     log.info("Version " + Constants.VERSION);
     log.info("Instance " + getInstanceID());
@@ -498,8 +498,8 @@
     // lose the reference to the old tablet and open two new ones
     onlineTablets.split(tablet.getExtent(), newTablets[0], newTablets[1]);
 
-    // tell the master
-    enqueueMasterMessage(new SplitReportMessage(tablet.getExtent(), newTablets[0].getExtent(),
+    // tell the manager
+    enqueueManagerMessage(new SplitReportMessage(tablet.getExtent(), newTablets[0].getExtent(),
         new Text("/" + newTablets[0].getDirName()), newTablets[1].getExtent(),
         new Text("/" + newTablets[1].getDirName())));
 
@@ -511,8 +511,8 @@
     return tabletInfo;
   }
 
-  // add a message for the main thread to send back to the master
-  public void enqueueMasterMessage(ManagerMessage m) {
+  // add a message for the main thread to send back to the manager
+  public void enqueueManagerMessage(ManagerMessage m) {
     managerMessages.addLast(m);
   }
 
@@ -540,35 +540,35 @@
     return sp.address;
   }
 
-  private HostAndPort getMasterAddress() {
+  private HostAndPort getManagerAddress() {
     try {
-      List<String> locations = getContext().getMasterLocations();
+      List<String> locations = getContext().getManagerLocations();
       if (locations.isEmpty()) {
         return null;
       }
       return HostAndPort.fromString(locations.get(0));
     } catch (Exception e) {
-      log.warn("Failed to obtain master host " + e);
+      log.warn("Failed to obtain manager host " + e);
     }
 
     return null;
   }
 
-  // Connect to the master for posting asynchronous results
-  private ManagerClientService.Client masterConnection(HostAndPort address) {
+  // Connect to the manager for posting asynchronous results
+  private ManagerClientService.Client managerConnection(HostAndPort address) {
     try {
       if (address == null) {
         return null;
       }
-      // log.info("Listener API to master has been opened");
+      // log.info("Listener API to manager has been opened");
       return ThriftUtil.getClient(new ManagerClientService.Client.Factory(), address, getContext());
     } catch (Exception e) {
-      log.warn("Issue with masterConnection (" + address + ") " + e, e);
+      log.warn("Issue with managerConnection (" + address + ") " + e, e);
     }
     return null;
   }
 
-  private void returnMasterConnection(ManagerClientService.Client client) {
+  private void returnManagerConnection(ManagerClientService.Client client) {
     ThriftUtil.returnClient(client);
   }
 
@@ -780,7 +780,7 @@
         new BulkImportCacheCleaner(this), CLEANUP_BULK_LOADED_CACHE_MILLIS,
         CLEANUP_BULK_LOADED_CACHE_MILLIS, TimeUnit.MILLISECONDS);
 
-    HostAndPort masterHost;
+    HostAndPort managerHost;
     while (!serverStopRequested) {
       // send all of the pending messages
       try {
@@ -794,10 +794,10 @@
             mm = managerMessages.poll(1000, TimeUnit.MILLISECONDS);
           }
 
-          // have a message to send to the master, so grab a
+          // have a message to send to the manager, so grab a
           // connection
-          masterHost = getMasterAddress();
-          iface = masterConnection(masterHost);
+          managerHost = getManagerAddress();
+          iface = managerConnection(managerHost);
           TServiceClient client = iface;
 
           // if while loop does not execute at all and mm != null,
@@ -826,7 +826,7 @@
           if (mm != null) {
             managerMessages.putFirst(mm);
           }
-          returnMasterConnection(iface);
+          returnManagerConnection(iface);
 
           sleepUninterruptibly(1, TimeUnit.SECONDS);
         }
@@ -835,17 +835,17 @@
         serverStopRequested = true;
 
       } catch (Exception e) {
-        // may have lost connection with master
+        // may have lost connection with manager
         // loop back to the beginning and wait for a new one
-        // this way we survive master failures
-        log.error(getClientAddressString() + ": TServerInfo: Exception. Master down?", e);
+        // this way we survive manager failures
+        log.error(getClientAddressString() + ": TServerInfo: Exception. Manager down?", e);
       }
     }
 
     // wait for shutdown
-    // if the main thread exits oldServer the master listener, the JVM will
+    // if the main thread exits oldServer the manager listener, the JVM will
     // kill the other threads and finalize objects. We want the shutdown that is
-    // running in the master listener thread to complete oldServer this happens.
+    // running in the manager listener thread to complete oldServer this happens.
     // consider making other threads daemon threads so that objects don't
     // get prematurely finalized
     synchronized (this) {
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/ThriftClientHandler.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/ThriftClientHandler.java
index 228ec83..04d08b7 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/ThriftClientHandler.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/ThriftClientHandler.java
@@ -1354,21 +1354,21 @@
 
     if (lock != null) {
       ZooUtil.LockID lid =
-          new ZooUtil.LockID(context.getZooKeeperRoot() + Constants.ZMASTER_LOCK, lock);
+          new ZooUtil.LockID(context.getZooKeeperRoot() + Constants.ZMANAGER_LOCK, lock);
 
       try {
-        if (!ZooLock.isLockHeld(server.masterLockCache, lid)) {
-          // maybe the cache is out of date and a new master holds the
+        if (!ZooLock.isLockHeld(server.managerLockCache, lid)) {
+          // maybe the cache is out of date and a new manager holds the
           // lock?
-          server.masterLockCache.clear();
-          if (!ZooLock.isLockHeld(server.masterLockCache, lid)) {
-            log.warn("Got {} message from a master that does not hold the current lock {}", request,
-                lock);
-            throw new RuntimeException("bad master lock");
+          server.managerLockCache.clear();
+          if (!ZooLock.isLockHeld(server.managerLockCache, lid)) {
+            log.warn("Got {} message from a manager that does not hold the current lock {}",
+                request, lock);
+            throw new RuntimeException("bad manager lock");
           }
         }
       } catch (Exception e) {
-        throw new RuntimeException("bad master lock", e);
+        throw new RuntimeException("bad manager lock", e);
       }
     }
   }
@@ -1539,7 +1539,7 @@
     checkPermission(credentials, lock, "halt");
 
     Halt.halt(0, () -> {
-      log.info("Master requested tablet server halt");
+      log.info("Manager requested tablet server halt");
       server.gcLogger.logGCInfo(server.getConfiguration());
       server.requestStop();
       try {
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/UnloadTabletHandler.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/UnloadTabletHandler.java
index b044cea..c52e263 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/UnloadTabletHandler.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/UnloadTabletHandler.java
@@ -30,7 +30,7 @@
 import org.apache.accumulo.core.tabletserver.thrift.TUnloadTabletGoal;
 import org.apache.accumulo.server.manager.state.DistributedStoreException;
 import org.apache.accumulo.server.manager.state.TabletStateStore;
-import org.apache.accumulo.tserver.mastermessage.TabletStatusMessage;
+import org.apache.accumulo.tserver.managermessage.TabletStatusMessage;
 import org.apache.accumulo.tserver.tablet.Tablet;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
@@ -59,7 +59,7 @@
     synchronized (server.unopenedTablets) {
       if (server.unopenedTablets.contains(extent)) {
         server.unopenedTablets.remove(extent);
-        // enqueueMasterMessage(new TabletUnloadedMessage(extent));
+        // enqueueManagerMessage(new TabletUnloadedMessage(extent));
         return;
       }
     }
@@ -78,11 +78,11 @@
     }
 
     if (t == null) {
-      // Tablet has probably been recently unloaded: repeated master
+      // Tablet has probably been recently unloaded: repeated manager
       // unload request is crossing the successful unloaded message
       if (!server.recentlyUnloadedCache.containsKey(extent)) {
         log.info("told to unload tablet that was not being served {}", extent);
-        server.enqueueMasterMessage(
+        server.enqueueManagerMessage(
             new TabletStatusMessage(TabletLoadState.UNLOAD_FAILURE_NOT_SERVING, extent));
       }
       return;
@@ -97,7 +97,7 @@
             e.getMessage());
       } else {
         log.error("Failed to close tablet {}... Aborting migration", extent, e);
-        server.enqueueMasterMessage(new TabletStatusMessage(TabletLoadState.UNLOAD_ERROR, extent));
+        server.enqueueManagerMessage(new TabletStatusMessage(TabletLoadState.UNLOAD_ERROR, extent));
       }
       return;
     }
@@ -132,8 +132,8 @@
       log.warn("Interrupted while getting our zookeeper session information", e);
     }
 
-    // tell the master how it went
-    server.enqueueMasterMessage(new TabletStatusMessage(TabletLoadState.UNLOADED, extent));
+    // tell the manager how it went
+    server.enqueueManagerMessage(new TabletStatusMessage(TabletLoadState.UNLOADED, extent));
 
     // roll tablet stats over into tablet server's statsKeeper object as
     // historical data
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/mastermessage/ManagerMessage.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/managermessage/ManagerMessage.java
similarity index 95%
rename from server/tserver/src/main/java/org/apache/accumulo/tserver/mastermessage/ManagerMessage.java
rename to server/tserver/src/main/java/org/apache/accumulo/tserver/managermessage/ManagerMessage.java
index 59cc05a..63ee525 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/mastermessage/ManagerMessage.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/managermessage/ManagerMessage.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.tserver.mastermessage;
+package org.apache.accumulo.tserver.managermessage;
 
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.master.thrift.ManagerClientService;
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/mastermessage/SplitReportMessage.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/managermessage/SplitReportMessage.java
similarity index 97%
rename from server/tserver/src/main/java/org/apache/accumulo/tserver/mastermessage/SplitReportMessage.java
rename to server/tserver/src/main/java/org/apache/accumulo/tserver/managermessage/SplitReportMessage.java
index 7bca512..340cc90 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/mastermessage/SplitReportMessage.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/managermessage/SplitReportMessage.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.tserver.mastermessage;
+package org.apache.accumulo.tserver.managermessage;
 
 import java.util.Map;
 import java.util.TreeMap;
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/mastermessage/TabletStatusMessage.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/managermessage/TabletStatusMessage.java
similarity index 96%
rename from server/tserver/src/main/java/org/apache/accumulo/tserver/mastermessage/TabletStatusMessage.java
rename to server/tserver/src/main/java/org/apache/accumulo/tserver/managermessage/TabletStatusMessage.java
index 40e8410..b80f448 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/mastermessage/TabletStatusMessage.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/managermessage/TabletStatusMessage.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.tserver.mastermessage;
+package org.apache.accumulo.tserver.managermessage;
 
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java
index b661e8d..13e01c2 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java
@@ -216,20 +216,20 @@
         String peerTserverStr;
         log.debug("Fetching peer tserver address");
         try (TraceScope span = Trace.startSpan("Fetch peer tserver")) {
-          // Ask the master on the remote what TServer we should talk with to replicate the data
+          // Ask the manager on the remote what TServer we should talk with to replicate the data
           peerTserverStr = ReplicationClient.executeCoordinatorWithReturn(peerContext,
               client -> client.getServicerAddress(remoteTableId, peerContext.rpcCreds()));
         } catch (AccumuloException | AccumuloSecurityException e) {
           // No progress is made
           log.error(
-              "Could not connect to master at {}, cannot proceed with replication. Will retry",
+              "Could not connect to manager at {}, cannot proceed with replication. Will retry",
               target, e);
           continue;
         }
 
         if (peerTserverStr == null) {
           // Something went wrong, and we didn't get a valid tserver from the remote for some reason
-          log.warn("Did not receive tserver from master at {}, cannot proceed"
+          log.warn("Did not receive tserver from manager at {}, cannot proceed"
               + " with replication. Will retry.", target);
           continue;
         }
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java
index f7aa504..3b8b976 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java
@@ -57,7 +57,7 @@
 import org.apache.accumulo.server.util.MetadataTableUtil;
 import org.apache.accumulo.tserver.compactions.Compactable;
 import org.apache.accumulo.tserver.compactions.CompactionManager;
-import org.apache.accumulo.tserver.mastermessage.TabletStatusMessage;
+import org.apache.accumulo.tserver.managermessage.TabletStatusMessage;
 import org.apache.accumulo.tserver.tablet.Compactor.CompactionCanceledException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -201,7 +201,7 @@
     MetadataTableUtil.chopped(tablet.getTabletServer().getContext(), getExtent(),
         tablet.getTabletServer().getLock());
     tablet.getTabletServer()
-        .enqueueMasterMessage(new TabletStatusMessage(TabletLoadState.CHOPPED, getExtent()));
+        .enqueueManagerMessage(new TabletStatusMessage(TabletLoadState.CHOPPED, getExtent()));
   }
 
   private Set<StoredTabletFile> selectChopFiles(Set<StoredTabletFile> chopCandidates) {
diff --git a/shell/src/main/java/org/apache/accumulo/shell/commands/FateCommand.java b/shell/src/main/java/org/apache/accumulo/shell/commands/FateCommand.java
index 54931f0..0dbad53 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/commands/FateCommand.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/commands/FateCommand.java
@@ -130,7 +130,7 @@
     AdminUtil<FateCommand> admin = new AdminUtil<>(false);
 
     String path = context.getZooKeeperRoot() + Constants.ZFATE;
-    String masterPath = context.getZooKeeperRoot() + Constants.ZMASTER_LOCK;
+    String managerPath = context.getZooKeeperRoot() + Constants.ZMANAGER_LOCK;
     ZooReaderWriter zk =
         getZooReaderWriter(context, siteConfig, cl.getOptionValue(secretOption.getOpt()));
     ZooStore<FateCommand> zs = new ZooStore<>(path, zk);
@@ -140,7 +140,7 @@
         throw new ParseException("Must provide transaction ID");
       }
       for (int i = 1; i < args.length; i++) {
-        if (!admin.prepFail(zs, zk, masterPath, args[i])) {
+        if (!admin.prepFail(zs, zk, managerPath, args[i])) {
           System.out.printf("Could not fail transaction: %s%n", args[i]);
           failedCommand = true;
         }
@@ -150,7 +150,7 @@
         throw new ParseException("Must provide transaction ID");
       }
       for (int i = 1; i < args.length; i++) {
-        if (admin.prepDelete(zs, zk, masterPath, args[i])) {
+        if (admin.prepDelete(zs, zk, managerPath, args[i])) {
           admin.deleteLocks(zk, context.getZooKeeperRoot() + Constants.ZTABLE_LOCKS, args[i]);
         } else {
           System.out.printf("Could not delete transaction: %s%n", args[i]);
diff --git a/test/src/main/java/org/apache/accumulo/test/BulkImportMonitoringIT.java b/test/src/main/java/org/apache/accumulo/test/BulkImportMonitoringIT.java
index 322a5ee..ab108a5 100644
--- a/test/src/main/java/org/apache/accumulo/test/BulkImportMonitoringIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/BulkImportMonitoringIT.java
@@ -82,7 +82,7 @@
       var ntc = new NewTableConfiguration().setProperties(props).withSplits(splits);
       c.tableOperations().create(tableName, ntc);
 
-      ManagerMonitorInfo stats = getCluster().getMasterMonitorInfo();
+      ManagerMonitorInfo stats = getCluster().getManagerMonitorInfo();
       assertEquals(1, stats.tServerInfo.size());
       assertEquals(0, stats.bulkImports.size());
       assertEquals(0, stats.tServerInfo.get(0).bulkImports.size());
@@ -137,7 +137,7 @@
       while (!es.isTerminated()
           && stats.bulkImports.size() + stats.tServerInfo.get(0).bulkImports.size() == 0) {
         es.awaitTermination(10, TimeUnit.MILLISECONDS);
-        stats = getCluster().getMasterMonitorInfo();
+        stats = getCluster().getManagerMonitorInfo();
       }
       log.info(stats.bulkImports.toString());
       assertTrue(!stats.bulkImports.isEmpty());
diff --git a/test/src/main/java/org/apache/accumulo/test/CountNameNodeOpsBulkIT.java b/test/src/main/java/org/apache/accumulo/test/CountNameNodeOpsBulkIT.java
index 5818f09..09d8528 100644
--- a/test/src/main/java/org/apache/accumulo/test/CountNameNodeOpsBulkIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/CountNameNodeOpsBulkIT.java
@@ -113,7 +113,7 @@
       var ntc = new NewTableConfiguration().setProperties(props).withSplits(splits);
       c.tableOperations().create(tableName, ntc);
 
-      ManagerMonitorInfo stats = getCluster().getMasterMonitorInfo();
+      ManagerMonitorInfo stats = getCluster().getManagerMonitorInfo();
       assertEquals(1, stats.tServerInfo.size());
 
       log.info("Creating lots of bulk import files");
diff --git a/test/src/main/java/org/apache/accumulo/test/DetectDeadTabletServersIT.java b/test/src/main/java/org/apache/accumulo/test/DetectDeadTabletServersIT.java
index 3642d3e..bce130d 100644
--- a/test/src/main/java/org/apache/accumulo/test/DetectDeadTabletServersIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/DetectDeadTabletServersIT.java
@@ -94,7 +94,7 @@
     while (true) {
       try {
         client = ManagerClient.getConnectionWithRetry(context);
-        log.info("Fetching master stats");
+        log.info("Fetching manager stats");
         return client.getManagerStats(TraceUtil.traceInfo(), context.rpcCreds());
       } catch (ThriftNotActiveServiceException e) {
         // Let it loop, fetching a new location
diff --git a/test/src/main/java/org/apache/accumulo/test/ExistingMacIT.java b/test/src/main/java/org/apache/accumulo/test/ExistingMacIT.java
index bf32007..d26fa0c 100644
--- a/test/src/main/java/org/apache/accumulo/test/ExistingMacIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ExistingMacIT.java
@@ -159,7 +159,7 @@
   public void testExistingRunningInstance() throws Exception {
     final String table = getUniqueNames(1)[0];
     try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) {
-      // Ensure that a master and tserver are up so the existing instance check won't fail.
+      // Ensure that a manager and tserver are up so the existing instance check won't fail.
       client.tableOperations().create(table);
       try (BatchWriter bw = client.createBatchWriter(table)) {
         Mutation m = new Mutation("foo");
diff --git a/test/src/main/java/org/apache/accumulo/test/ManagerRepairsDualAssignmentIT.java b/test/src/main/java/org/apache/accumulo/test/ManagerRepairsDualAssignmentIT.java
index 1909a0f..c2219c0 100644
--- a/test/src/main/java/org/apache/accumulo/test/ManagerRepairsDualAssignmentIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ManagerRepairsDualAssignmentIT.java
@@ -139,7 +139,7 @@
       TabletMutator tabletMutator = serverContext.getAmple().mutateTablet(moved.extent);
       tabletMutator.putLocation(moved.current, LocationType.CURRENT);
       tabletMutator.mutate();
-      // wait for the master to fix the problem
+      // wait for the manager to fix the problem
       waitForCleanStore(store);
       // now jam up the metadata table
       tabletMutator =
diff --git a/test/src/main/java/org/apache/accumulo/test/ThriftServerBindsBeforeZooKeeperLockIT.java b/test/src/main/java/org/apache/accumulo/test/ThriftServerBindsBeforeZooKeeperLockIT.java
index 29e93a2..36eff22 100644
--- a/test/src/main/java/org/apache/accumulo/test/ThriftServerBindsBeforeZooKeeperLockIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ThriftServerBindsBeforeZooKeeperLockIT.java
@@ -131,33 +131,33 @@
   @SuppressFBWarnings(value = "UNENCRYPTED_SOCKET",
       justification = "unencrypted socket is okay for testing")
   @Test
-  public void testMasterService() throws Exception {
+  public void testManagerService() throws Exception {
     final MiniAccumuloClusterImpl cluster = (MiniAccumuloClusterImpl) getCluster();
     try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
       final String instanceID = client.instanceOperations().getInstanceID();
 
-      // Wait for the Master to grab its lock
+      // Wait for the Manager to grab its lock
       while (true) {
         final ZooReader reader = new ZooReader(cluster.getZooKeepers(), 30000);
         try {
           List<String> locks =
-              reader.getChildren(Constants.ZROOT + "/" + instanceID + Constants.ZMASTER_LOCK);
+              reader.getChildren(Constants.ZROOT + "/" + instanceID + Constants.ZMANAGER_LOCK);
           if (!locks.isEmpty()) {
             break;
           }
         } catch (Exception e) {
-          LOG.debug("Failed to find active master location, retrying", e);
+          LOG.debug("Failed to find active manager location, retrying", e);
           Thread.sleep(1000);
         }
       }
 
-      LOG.debug("Found active master");
+      LOG.debug("Found active manager");
 
       int freePort = PortUtils.getRandomFreePort();
-      Process master = null;
+      Process manager = null;
       try {
-        LOG.debug("Starting standby master on {}", freePort);
-        master = startProcess(cluster, ServerType.MANAGER, freePort);
+        LOG.debug("Starting standby manager on {}", freePort);
+        manager = startProcess(cluster, ServerType.MANAGER, freePort);
 
         while (true) {
           try (Socket s = new Socket("localhost", freePort)) {
@@ -166,7 +166,7 @@
               return;
             }
           } catch (Exception e) {
-            LOG.debug("Caught exception trying to connect to Master", e);
+            LOG.debug("Caught exception trying to connect to Manager", e);
           }
           // Wait before trying again
           Thread.sleep(1000);
@@ -174,15 +174,15 @@
           // actually
           // free and the process
           // died trying to bind it. Pick a new port and restart it in that case.
-          if (!master.isAlive()) {
+          if (!manager.isAlive()) {
             freePort = PortUtils.getRandomFreePort();
-            LOG.debug("Master died, restarting it listening on {}", freePort);
-            master = startProcess(cluster, ServerType.MANAGER, freePort);
+            LOG.debug("Manager died, restarting it listening on {}", freePort);
+            manager = startProcess(cluster, ServerType.MANAGER, freePort);
           }
         }
       } finally {
-        if (master != null) {
-          master.destroyForcibly();
+        if (manager != null) {
+          manager.destroyForcibly();
         }
       }
     }
@@ -196,7 +196,7 @@
     try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
       String instanceID = client.instanceOperations().getInstanceID();
 
-      // Wait for the Master to grab its lock
+      // Wait for the Manager to grab its lock
       while (true) {
         final ZooReader reader = new ZooReader(cluster.getZooKeepers(), 30000);
         try {
@@ -214,10 +214,10 @@
       LOG.debug("Found active gc");
 
       int freePort = PortUtils.getRandomFreePort();
-      Process master = null;
+      Process manager = null;
       try {
         LOG.debug("Starting standby gc on {}", freePort);
-        master = startProcess(cluster, ServerType.GARBAGE_COLLECTOR, freePort);
+        manager = startProcess(cluster, ServerType.GARBAGE_COLLECTOR, freePort);
 
         while (true) {
           try (Socket s = new Socket("localhost", freePort)) {
@@ -234,15 +234,15 @@
           // actually
           // free and the process
           // died trying to bind it. Pick a new port and restart it in that case.
-          if (!master.isAlive()) {
+          if (!manager.isAlive()) {
             freePort = PortUtils.getRandomFreePort();
             LOG.debug("GC died, restarting it listening on {}", freePort);
-            master = startProcess(cluster, ServerType.GARBAGE_COLLECTOR, freePort);
+            manager = startProcess(cluster, ServerType.GARBAGE_COLLECTOR, freePort);
           }
         }
       } finally {
-        if (master != null) {
-          master.destroyForcibly();
+        if (manager != null) {
+          manager.destroyForcibly();
         }
       }
     }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BackupManagerIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BackupManagerIT.java
index 6e7da59..12410fc 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BackupManagerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BackupManagerIT.java
@@ -21,6 +21,7 @@
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.conf.Property;
@@ -39,7 +40,7 @@
 
   @Test
   public void test() throws Exception {
-    // wait for master
+    // wait for manager
     UtilWaitThread.sleep(1000);
     // create a backup
     Process backup = exec(Manager.class);
@@ -47,24 +48,24 @@
       String secret = getCluster().getSiteConfiguration().get(Property.INSTANCE_SECRET);
       ZooReaderWriter writer = new ZooReaderWriter(cluster.getZooKeepers(), 30 * 1000, secret);
       String root = "/accumulo/" + client.instanceOperations().getInstanceID();
-      List<String> children = Collections.emptyList();
+      List<String> children;
       // wait for 2 lock entries
       do {
         UtilWaitThread.sleep(100);
-        children = writer.getChildren(root + "/masters/lock");
+        children = writer.getChildren(root + Constants.ZMANAGER_LOCK);
       } while (children.size() != 2);
       Collections.sort(children);
-      // wait for the backup master to learn to be the backup
+      // wait for the backup manager to learn to be the backup
       UtilWaitThread.sleep(1000);
       // generate a false zookeeper event
-      String lockPath = root + "/masters/lock/" + children.get(0);
+      String lockPath = root + Constants.ZMANAGER_LOCK + children.get(0);
       byte[] data = writer.getData(lockPath);
       writer.getZooKeeper().setData(lockPath, data, -1);
       // let it propagate
       UtilWaitThread.sleep(500);
-      // kill the master by removing its lock
+      // kill the manager by removing its lock
       writer.recursiveDelete(lockPath, NodeMissingPolicy.FAIL);
-      // ensure the backup becomes the master
+      // ensure the backup becomes the manager
       client.tableOperations().create(getUniqueNames(1)[0]);
     } finally {
       backup.destroy();
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BalanceAfterCommsFailureIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BalanceAfterCommsFailureIT.java
index d63af7c..496a0cb 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BalanceAfterCommsFailureIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BalanceAfterCommsFailureIT.java
@@ -118,7 +118,7 @@
           break;
         } catch (ThriftNotActiveServiceException e) {
           // Let it loop, fetching a new location
-          log.debug("Contacted a Master which is no longer active, retrying");
+          log.debug("Contacted a Manager which is no longer active, retrying");
           sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
         } finally {
           if (client != null)
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java
index d9a32f4..a23bda9 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java
@@ -167,7 +167,7 @@
           throw new AccumuloSecurityException(exception);
         } catch (ThriftNotActiveServiceException e) {
           // Let it loop, fetching a new location
-          log.debug("Contacted a Master which is no longer active, retrying");
+          log.debug("Contacted a Manager which is no longer active, retrying");
           sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
         } catch (TException exception) {
           throw new AccumuloException(exception);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/KerberosIT.java b/test/src/main/java/org/apache/accumulo/test/functional/KerberosIT.java
index 7d1391a..c991d6e 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/KerberosIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/KerberosIT.java
@@ -513,7 +513,7 @@
   }
 
   @Test
-  public void testRestartedMasterReusesSecretKey() throws Exception {
+  public void testRestartedManagerReusesSecretKey() throws Exception {
     // Login as the "root" user
     UserGroupInformation root = UserGroupInformation.loginUserFromKeytabAndReturnUGI(
         rootUser.getPrincipal(), rootUser.getKeytab().getAbsolutePath());
@@ -537,10 +537,10 @@
           return token;
         });
 
-    log.info("Stopping master");
+    log.info("Stopping manager");
     mac.getClusterControl().stop(ServerType.MANAGER);
     Thread.sleep(5000);
-    log.info("Restarting master");
+    log.info("Restarting manager");
     mac.getClusterControl().start(ServerType.MANAGER);
 
     // Make sure our original token is still good
@@ -571,7 +571,7 @@
           return token;
         });
 
-    // A restarted master should reuse the same secret key after a restart if the secret key hasn't
+    // A restarted manager should reuse the same secret key after a restart if the secret key hasn't
     // expired (1day by default)
     DelegationTokenImpl dt1 = (DelegationTokenImpl) delegationToken1;
     DelegationTokenImpl dt2 = (DelegationTokenImpl) delegationToken2;
@@ -649,7 +649,7 @@
 
   /**
    * Creates a table, adds a record to it, and then compacts the table. A simple way to make sure
-   * that the system user exists (since the master does an RPC to the tserver which will create the
+   * that the system user exists (since the manager does an RPC to the tserver which will create the
    * system user if it doesn't already exist).
    */
   private void createTableWithDataAndCompact(AccumuloClient client) throws TableNotFoundException,
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/KerberosRenewalIT.java b/test/src/main/java/org/apache/accumulo/test/functional/KerberosRenewalIT.java
index 13f4648..7cfaaf7 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/KerberosRenewalIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/KerberosRenewalIT.java
@@ -178,7 +178,7 @@
 
   /**
    * Creates a table, adds a record to it, and then compacts the table. A simple way to make sure
-   * that the system user exists (since the master does an RPC to the tserver which will create the
+   * that the system user exists (since the manager does an RPC to the tserver which will create the
    * system user if it doesn't already exist).
    */
   private void createReadWriteDrop(AccumuloClient client) throws TableNotFoundException,
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/LateLastContactIT.java b/test/src/main/java/org/apache/accumulo/test/functional/LateLastContactIT.java
index 2c90d24..66a3950 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/LateLastContactIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/LateLastContactIT.java
@@ -29,7 +29,7 @@
 
 /**
  * Fake the "tablet stops talking but holds its lock" problem we see when hard drives and NFS fail.
- * Start a ZombieTServer, and see that master stops it.
+ * Start a ZombieTServer, and see that manager stops it.
  */
 public class LateLastContactIT extends ConfigurableMacBase {
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ManagerApiIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ManagerApiIT.java
index c0f7b13..e8bd277 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ManagerApiIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ManagerApiIT.java
@@ -90,8 +90,8 @@
   private Function<TCredentials,ClientExec<ManagerClientService.Client>> op;
 
   @Test
-  public void testPermissions_setMasterGoalState() throws Exception {
-    // To setMasterGoalState, user needs SystemPermission.SYSTEM
+  public void testPermissions_setManagerGoalState() throws Exception {
+    // To setManagerGoalState, user needs SystemPermission.SYSTEM
     op = user -> client -> client.setManagerGoalState(null, user, ManagerGoalState.NORMAL);
     expectPermissionDenied(op, regularUser);
     expectPermissionSuccess(op, rootUser);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ManagerMetricsIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ManagerMetricsIT.java
index 085e775..69f5499 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ManagerMetricsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ManagerMetricsIT.java
@@ -77,7 +77,7 @@
   private static final Set<String> OPTIONAL_METRIC_KEYS =
       new HashSet<>(Collections.singletonList("FateTxOpType_CompactRange"));
 
-  private final MetricsFileTailer metricsTail = new MetricsFileTailer("accumulo.sink.file-master");
+  private final MetricsFileTailer metricsTail = new MetricsFileTailer("accumulo.sink.file-manager");
 
   @Before
   public void setup() {
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java
index 3f41ecd..c6fedc1 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java
@@ -170,17 +170,17 @@
       ClientInfo info = ClientInfo.from(accumuloClient.properties());
       ZooReader zreader = new ZooReader(info.getZooKeepers(), info.getZooKeepersSessionTimeOut());
       ZooCache zcache = new ZooCache(zreader, null);
-      byte[] masterLockData;
+      byte[] managerLockData;
       do {
-        masterLockData = ZooLock.getLockData(zcache,
+        managerLockData = ZooLock.getLockData(zcache,
             ZooUtil.getRoot(accumuloClient.instanceOperations().getInstanceID())
-                + Constants.ZMASTER_LOCK,
+                + Constants.ZMANAGER_LOCK,
             null);
-        if (masterLockData != null) {
-          log.info("Master lock is still held");
+        if (managerLockData != null) {
+          log.info("Manager lock is still held");
           Thread.sleep(1000);
         }
-      } while (masterLockData != null);
+      } while (managerLockData != null);
 
       control.stopAllServers(ServerType.GARBAGE_COLLECTOR);
       control.stopAllServers(ServerType.MONITOR);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/RestartIT.java b/test/src/main/java/org/apache/accumulo/test/functional/RestartIT.java
index 59ec1c2..198277a 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/RestartIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/RestartIT.java
@@ -96,7 +96,7 @@
   }
 
   @Test
-  public void restartMaster() throws Exception {
+  public void restartManager() throws Exception {
     try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
       final String tableName = getUniqueNames(1)[0];
       c.tableOperations().create(tableName);
@@ -122,7 +122,7 @@
   }
 
   @Test
-  public void restartMasterRecovery() throws Exception {
+  public void restartManagerRecovery() throws Exception {
     try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
       String tableName = getUniqueNames(1)[0];
       c.tableOperations().create(tableName);
@@ -141,36 +141,38 @@
       ClientInfo info = ClientInfo.from(c.properties());
       ZooReader zreader = new ZooReader(info.getZooKeepers(), info.getZooKeepersSessionTimeOut());
       ZooCache zcache = new ZooCache(zreader, null);
-      byte[] masterLockData;
+      byte[] managerLockData;
       do {
-        masterLockData = ZooLock.getLockData(zcache,
-            ZooUtil.getRoot(c.instanceOperations().getInstanceID()) + Constants.ZMASTER_LOCK, null);
-        if (masterLockData != null) {
-          log.info("Master lock is still held");
+        managerLockData = ZooLock.getLockData(zcache,
+            ZooUtil.getRoot(c.instanceOperations().getInstanceID()) + Constants.ZMANAGER_LOCK,
+            null);
+        if (managerLockData != null) {
+          log.info("Manager lock is still held");
           Thread.sleep(1000);
         }
-      } while (masterLockData != null);
+      } while (managerLockData != null);
 
       cluster.start();
       sleepUninterruptibly(5, TimeUnit.MILLISECONDS);
       control.stopAllServers(ServerType.MANAGER);
 
-      masterLockData = new byte[0];
+      managerLockData = new byte[0];
       do {
-        masterLockData = ZooLock.getLockData(zcache,
-            ZooUtil.getRoot(c.instanceOperations().getInstanceID()) + Constants.ZMASTER_LOCK, null);
-        if (masterLockData != null) {
-          log.info("Master lock is still held");
+        managerLockData = ZooLock.getLockData(zcache,
+            ZooUtil.getRoot(c.instanceOperations().getInstanceID()) + Constants.ZMANAGER_LOCK,
+            null);
+        if (managerLockData != null) {
+          log.info("Manager lock is still held");
           Thread.sleep(1000);
         }
-      } while (masterLockData != null);
+      } while (managerLockData != null);
       cluster.start();
       VerifyIngest.verifyIngest(c, params);
     }
   }
 
   @Test
-  public void restartMasterSplit() throws Exception {
+  public void restartManagerSplit() throws Exception {
     try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) {
       final String tableName = getUniqueNames(1)[0];
       final ClusterControl control = getCluster().getClusterControl();
@@ -194,15 +196,16 @@
       ClientInfo info = ClientInfo.from(c.properties());
       ZooReader zreader = new ZooReader(info.getZooKeepers(), info.getZooKeepersSessionTimeOut());
       ZooCache zcache = new ZooCache(zreader, null);
-      byte[] masterLockData;
+      byte[] managerLockData;
       do {
-        masterLockData = ZooLock.getLockData(zcache,
-            ZooUtil.getRoot(c.instanceOperations().getInstanceID()) + Constants.ZMASTER_LOCK, null);
-        if (masterLockData != null) {
-          log.info("Master lock is still held");
+        managerLockData = ZooLock.getLockData(zcache,
+            ZooUtil.getRoot(c.instanceOperations().getInstanceID()) + Constants.ZMANAGER_LOCK,
+            null);
+        if (managerLockData != null) {
+          log.info("Manager lock is still held");
           Thread.sleep(1000);
         }
-      } while (masterLockData != null);
+      } while (managerLockData != null);
 
       cluster.start();
       assertEquals(0, ret.get().intValue());
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java b/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
index 7e68534..0a66f3d 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/TabletStateChangeIteratorIT.java
@@ -312,7 +312,7 @@
     }
 
     @Override
-    public ManagerState getMasterState() {
+    public ManagerState getManagerState() {
       return ManagerState.NORMAL;
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java b/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
index c00fd57..8316c42 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
@@ -141,7 +141,7 @@
 
       // wait for the metadata table to go back online
       getRecoveryMarkers(c);
-      // allow a little time for the master to notice ASSIGNED_TO_DEAD_SERVER tablets
+      // allow a little time for the manager to notice ASSIGNED_TO_DEAD_SERVER tablets
       sleepUninterruptibly(5, TimeUnit.SECONDS);
       Map<KeyExtent,List<String>> markers = getRecoveryMarkers(c);
       // log.debug("markers " + markers);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ZookeeperRestartIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ZookeeperRestartIT.java
index 25c0de9..d96222f 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ZookeeperRestartIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ZookeeperRestartIT.java
@@ -86,7 +86,7 @@
         assertTrue(i.hasNext());
         assertEquals("row", i.next().getKey().getRow().toString());
         assertFalse(i.hasNext());
-        // use the master
+        // use the manager
         c.tableOperations().delete("test_ingest");
       }
     }
diff --git a/test/src/main/java/org/apache/accumulo/test/manager/MergeStateIT.java b/test/src/main/java/org/apache/accumulo/test/manager/MergeStateIT.java
index 2d15f57..495037e 100644
--- a/test/src/main/java/org/apache/accumulo/test/manager/MergeStateIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/manager/MergeStateIT.java
@@ -91,7 +91,7 @@
     }
 
     @Override
-    public ManagerState getMasterState() {
+    public ManagerState getManagerState() {
       return ManagerState.NORMAL;
     }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/manager/SuspendedTabletsIT.java b/test/src/main/java/org/apache/accumulo/test/manager/SuspendedTabletsIT.java
index c4b523a..92f6401 100644
--- a/test/src/main/java/org/apache/accumulo/test/manager/SuspendedTabletsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/manager/SuspendedTabletsIT.java
@@ -206,7 +206,7 @@
       // Eventually some tablets will be suspended.
       log.info("Waiting on suspended tablets");
       ds = TabletLocations.retrieve(ctx, tableName);
-      // Until we can scan the metadata table, the master probably can't either, so won't have been
+      // Until we can scan the metadata table, the manager probably can't either, so won't have been
       // able to suspend the tablets.
       // So we note the time that we were first able to successfully scan the metadata table.
       long killTime = System.nanoTime();
diff --git a/test/src/main/java/org/apache/accumulo/test/metrics/MetricsTestSinkProperties.java b/test/src/main/java/org/apache/accumulo/test/metrics/MetricsTestSinkProperties.java
index b203365..4e137f0 100644
--- a/test/src/main/java/org/apache/accumulo/test/metrics/MetricsTestSinkProperties.java
+++ b/test/src/main/java/org/apache/accumulo/test/metrics/MetricsTestSinkProperties.java
@@ -25,6 +25,6 @@
 
   public static final String METRICS_PROP_FILENAME = "hadoop-metrics2-accumulo.properties";
   public static final String ACC_GC_SINK_PREFIX = "accumulo.sink.file-gc";
-  public static final String ACC_MASTER_SINK_PREFIX = "accumulo.sink.file-master";
+  public static final String ACC_MANAGER_SINK_PREFIX = "accumulo.sink.file-manager";
 
 }
diff --git a/test/src/main/java/org/apache/accumulo/test/metrics/fate/FateMetricsTest.java b/test/src/main/java/org/apache/accumulo/test/metrics/fate/FateMetricsTest.java
index 054c8af..f6695c8 100644
--- a/test/src/main/java/org/apache/accumulo/test/metrics/fate/FateMetricsTest.java
+++ b/test/src/main/java/org/apache/accumulo/test/metrics/fate/FateMetricsTest.java
@@ -62,7 +62,7 @@
   private ZooStore<Manager> zooStore = null;
   private ZooKeeper zookeeper = null;
   private ServerContext context = null;
-  private Manager master;
+  private Manager manager;
 
   @BeforeClass
   public static void setupZk() {
@@ -77,8 +77,8 @@
   }
 
   /**
-   * Instantiate a test zookeeper and setup mocks for Master and Context. The test zookeeper is used
-   * create a ZooReaderWriter. The zookeeper used in tests needs to be the one from the
+   * Instantiate a test zookeeper and setup mocks for Manager and Context. The test zookeeper is
+   * used create a ZooReaderWriter. The zookeeper used in tests needs to be the one from the
    * zooReaderWriter, not the test server, because the zooReaderWriter sets up ACLs.
    *
    * @throws Exception
@@ -94,13 +94,13 @@
 
     zooStore = new ZooStore<>(MOCK_ZK_ROOT + Constants.ZFATE, zooReaderWriter);
 
-    master = EasyMock.createMock(Manager.class);
+    manager = EasyMock.createMock(Manager.class);
     context = EasyMock.createMock(ServerContext.class);
 
     EasyMock.expect(context.getZooReaderWriter()).andReturn(zooReaderWriter).anyTimes();
     EasyMock.expect(context.getZooKeeperRoot()).andReturn(MOCK_ZK_ROOT).anyTimes();
 
-    EasyMock.replay(master, context);
+    EasyMock.replay(manager, context);
   }
 
   @After
@@ -141,7 +141,7 @@
     assertEquals(0L, collector.getValue("FateTxState_IN_PROGRESS"));
     assertEquals(0L, collector.getValue("currentFateOps"));
 
-    EasyMock.verify(master);
+    EasyMock.verify(manager);
 
   }
 
@@ -172,7 +172,7 @@
     assertTrue(collector.contains("FateTxState_IN_PROGRESS"));
     assertEquals(0L, collector.getValue("FateTxState_IN_PROGRESS"));
 
-    EasyMock.verify(master);
+    EasyMock.verify(manager);
 
   }
 
@@ -205,7 +205,7 @@
     assertEquals(1L, collector.getValue("FateTxState_IN_PROGRESS"));
     assertEquals(1L, collector.getValue("FateTxOpType_FakeOp"));
 
-    EasyMock.verify(master);
+    EasyMock.verify(manager);
   }
 
   private long seedTransaction() throws Exception {
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java b/test/src/main/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java
index 736f6d3..4d32b08 100644
--- a/test/src/main/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java
@@ -162,133 +162,134 @@
 
   @Test
   public void dataIsNotOverReplicated() throws Exception {
-    File master1Dir = createTestDir("master1"), master2Dir = createTestDir("master2");
+    File manager1Dir = createTestDir("manager1"), manager2Dir = createTestDir("manager2");
     String password = "password";
 
-    MiniAccumuloConfigImpl master1Cfg;
-    MiniAccumuloClusterImpl master1Cluster;
+    MiniAccumuloConfigImpl manager1Cfg;
+    MiniAccumuloClusterImpl manager1Cluster;
     while (true) {
-      master1Cfg = new MiniAccumuloConfigImpl(master1Dir, password);
-      master1Cfg.setNumTservers(1);
-      master1Cfg.setInstanceName("master1");
+      manager1Cfg = new MiniAccumuloConfigImpl(manager1Dir, password);
+      manager1Cfg.setNumTservers(1);
+      manager1Cfg.setInstanceName("manager1");
 
       // Set up SSL if needed
-      ConfigurableMacBase.configureForEnvironment(master1Cfg,
-          ConfigurableMacBase.getSslDir(master1Dir));
+      ConfigurableMacBase.configureForEnvironment(manager1Cfg,
+          ConfigurableMacBase.getSslDir(manager1Dir));
 
-      master1Cfg.setProperty(Property.REPLICATION_NAME, master1Cfg.getInstanceName());
-      master1Cfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, "5M");
-      master1Cfg.setProperty(Property.REPLICATION_THREADCHECK, "5m");
-      master1Cfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
-      master1Cfg.setProperty(Property.MANAGER_REPLICATION_SCAN_INTERVAL, "1s");
-      master1Cluster = new MiniAccumuloClusterImpl(master1Cfg);
-      setCoreSite(master1Cluster);
+      manager1Cfg.setProperty(Property.REPLICATION_NAME, manager1Cfg.getInstanceName());
+      manager1Cfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, "5M");
+      manager1Cfg.setProperty(Property.REPLICATION_THREADCHECK, "5m");
+      manager1Cfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
+      manager1Cfg.setProperty(Property.MANAGER_REPLICATION_SCAN_INTERVAL, "1s");
+      manager1Cluster = new MiniAccumuloClusterImpl(manager1Cfg);
+      setCoreSite(manager1Cluster);
 
       try {
-        master1Cluster.start();
+        manager1Cluster.start();
         break;
       } catch (ZooKeeperBindException e) {
-        log.warn("Failed to start ZooKeeper on {}, will retry", master1Cfg.getZooKeeperPort());
+        log.warn("Failed to start ZooKeeper on {}, will retry", manager1Cfg.getZooKeeperPort());
       }
     }
 
-    MiniAccumuloConfigImpl master2Cfg;
-    MiniAccumuloClusterImpl master2Cluster;
+    MiniAccumuloConfigImpl manager2Cfg;
+    MiniAccumuloClusterImpl manager2Cluster;
     while (true) {
-      master2Cfg = new MiniAccumuloConfigImpl(master2Dir, password);
-      master2Cfg.setNumTservers(1);
-      master2Cfg.setInstanceName("master2");
+      manager2Cfg = new MiniAccumuloConfigImpl(manager2Dir, password);
+      manager2Cfg.setNumTservers(1);
+      manager2Cfg.setInstanceName("manager2");
 
-      // Set up SSL if needed. Need to share the same SSL truststore as master1
-      this.updatePeerConfigFromPrimary(master1Cfg, master2Cfg);
+      // Set up SSL if needed. Need to share the same SSL truststore as manager1
+      this.updatePeerConfigFromPrimary(manager1Cfg, manager2Cfg);
 
-      master2Cfg.setProperty(Property.REPLICATION_NAME, master2Cfg.getInstanceName());
-      master2Cfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, "5M");
-      master2Cfg.setProperty(Property.REPLICATION_THREADCHECK, "5m");
-      master2Cfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
-      master2Cfg.setProperty(Property.MANAGER_REPLICATION_SCAN_INTERVAL, "1s");
-      master2Cluster = new MiniAccumuloClusterImpl(master2Cfg);
-      setCoreSite(master2Cluster);
+      manager2Cfg.setProperty(Property.REPLICATION_NAME, manager2Cfg.getInstanceName());
+      manager2Cfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, "5M");
+      manager2Cfg.setProperty(Property.REPLICATION_THREADCHECK, "5m");
+      manager2Cfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
+      manager2Cfg.setProperty(Property.MANAGER_REPLICATION_SCAN_INTERVAL, "1s");
+      manager2Cluster = new MiniAccumuloClusterImpl(manager2Cfg);
+      setCoreSite(manager2Cluster);
 
       try {
-        master2Cluster.start();
+        manager2Cluster.start();
         break;
       } catch (ZooKeeperBindException e) {
-        log.warn("Failed to start ZooKeeper on {}, will retry", master2Cfg.getZooKeeperPort());
+        log.warn("Failed to start ZooKeeper on {}, will retry", manager2Cfg.getZooKeeperPort());
       }
     }
 
     try {
-      AccumuloClient clientMaster1 =
-          master1Cluster.createAccumuloClient("root", new PasswordToken(password)),
-          clientMaster2 = master2Cluster.createAccumuloClient("root", new PasswordToken(password));
+      AccumuloClient clientManager1 =
+          manager1Cluster.createAccumuloClient("root", new PasswordToken(password)),
+          clientManager2 =
+              manager2Cluster.createAccumuloClient("root", new PasswordToken(password));
 
-      String master1UserName = "master1", master1Password = "foo";
-      String master2UserName = "master2", master2Password = "bar";
-      String master1Table = master1Cluster.getInstanceName(),
-          master2Table = master2Cluster.getInstanceName();
+      String manager1UserName = "manager1", manager1Password = "foo";
+      String manager2UserName = "manager2", manager2Password = "bar";
+      String manager1Table = manager1Cluster.getInstanceName(),
+          manager2Table = manager2Cluster.getInstanceName();
 
-      clientMaster1.securityOperations().createLocalUser(master1UserName,
-          new PasswordToken(master1Password));
-      clientMaster2.securityOperations().createLocalUser(master2UserName,
-          new PasswordToken(master2Password));
+      clientManager1.securityOperations().createLocalUser(manager1UserName,
+          new PasswordToken(manager1Password));
+      clientManager2.securityOperations().createLocalUser(manager2UserName,
+          new PasswordToken(manager2Password));
 
       // Configure the credentials we should use to authenticate ourselves to the peer for
       // replication
-      clientMaster1.instanceOperations().setProperty(
-          Property.REPLICATION_PEER_USER.getKey() + master2Cluster.getInstanceName(),
-          master2UserName);
-      clientMaster1.instanceOperations().setProperty(
-          Property.REPLICATION_PEER_PASSWORD.getKey() + master2Cluster.getInstanceName(),
-          master2Password);
+      clientManager1.instanceOperations().setProperty(
+          Property.REPLICATION_PEER_USER.getKey() + manager2Cluster.getInstanceName(),
+          manager2UserName);
+      clientManager1.instanceOperations().setProperty(
+          Property.REPLICATION_PEER_PASSWORD.getKey() + manager2Cluster.getInstanceName(),
+          manager2Password);
 
-      clientMaster2.instanceOperations().setProperty(
-          Property.REPLICATION_PEER_USER.getKey() + master1Cluster.getInstanceName(),
-          master1UserName);
-      clientMaster2.instanceOperations().setProperty(
-          Property.REPLICATION_PEER_PASSWORD.getKey() + master1Cluster.getInstanceName(),
-          master1Password);
+      clientManager2.instanceOperations().setProperty(
+          Property.REPLICATION_PEER_USER.getKey() + manager1Cluster.getInstanceName(),
+          manager1UserName);
+      clientManager2.instanceOperations().setProperty(
+          Property.REPLICATION_PEER_PASSWORD.getKey() + manager1Cluster.getInstanceName(),
+          manager1Password);
 
-      clientMaster1.instanceOperations().setProperty(
-          Property.REPLICATION_PEERS.getKey() + master2Cluster.getInstanceName(),
+      clientManager1.instanceOperations().setProperty(
+          Property.REPLICATION_PEERS.getKey() + manager2Cluster.getInstanceName(),
           ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
-              AccumuloReplicaSystem.buildConfiguration(master2Cluster.getInstanceName(),
-                  master2Cluster.getZooKeepers())));
+              AccumuloReplicaSystem.buildConfiguration(manager2Cluster.getInstanceName(),
+                  manager2Cluster.getZooKeepers())));
 
-      clientMaster2.instanceOperations().setProperty(
-          Property.REPLICATION_PEERS.getKey() + master1Cluster.getInstanceName(),
+      clientManager2.instanceOperations().setProperty(
+          Property.REPLICATION_PEERS.getKey() + manager1Cluster.getInstanceName(),
           ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
-              AccumuloReplicaSystem.buildConfiguration(master1Cluster.getInstanceName(),
-                  master1Cluster.getZooKeepers())));
+              AccumuloReplicaSystem.buildConfiguration(manager1Cluster.getInstanceName(),
+                  manager1Cluster.getZooKeepers())));
 
-      clientMaster1.tableOperations().create(master1Table,
+      clientManager1.tableOperations().create(manager1Table,
           new NewTableConfiguration().withoutDefaultIterators());
-      String master1TableId = clientMaster1.tableOperations().tableIdMap().get(master1Table);
-      assertNotNull(master1TableId);
+      String manager1TableId = clientManager1.tableOperations().tableIdMap().get(manager1Table);
+      assertNotNull(manager1TableId);
 
-      clientMaster2.tableOperations().create(master2Table,
+      clientManager2.tableOperations().create(manager2Table,
           new NewTableConfiguration().withoutDefaultIterators());
-      String master2TableId = clientMaster2.tableOperations().tableIdMap().get(master2Table);
-      assertNotNull(master2TableId);
+      String manager2TableId = clientManager2.tableOperations().tableIdMap().get(manager2Table);
+      assertNotNull(manager2TableId);
 
-      // Replicate master1 in the master1 cluster to master2 in the master2 cluster
-      clientMaster1.tableOperations().setProperty(master1Table, Property.TABLE_REPLICATION.getKey(),
-          "true");
-      clientMaster1.tableOperations().setProperty(master1Table,
-          Property.TABLE_REPLICATION_TARGET.getKey() + master2Cluster.getInstanceName(),
-          master2TableId);
+      // Replicate manager1 in the manager1 cluster to manager2 in the manager2 cluster
+      clientManager1.tableOperations().setProperty(manager1Table,
+          Property.TABLE_REPLICATION.getKey(), "true");
+      clientManager1.tableOperations().setProperty(manager1Table,
+          Property.TABLE_REPLICATION_TARGET.getKey() + manager2Cluster.getInstanceName(),
+          manager2TableId);
 
-      // Replicate master2 in the master2 cluster to master1 in the master2 cluster
-      clientMaster2.tableOperations().setProperty(master2Table, Property.TABLE_REPLICATION.getKey(),
-          "true");
-      clientMaster2.tableOperations().setProperty(master2Table,
-          Property.TABLE_REPLICATION_TARGET.getKey() + master1Cluster.getInstanceName(),
-          master1TableId);
+      // Replicate manager2 in the manager2 cluster to manager1 in the manager2 cluster
+      clientManager2.tableOperations().setProperty(manager2Table,
+          Property.TABLE_REPLICATION.getKey(), "true");
+      clientManager2.tableOperations().setProperty(manager2Table,
+          Property.TABLE_REPLICATION_TARGET.getKey() + manager1Cluster.getInstanceName(),
+          manager1TableId);
 
       // Give our replication user the ability to write to the respective table
-      clientMaster1.securityOperations().grantTablePermission(master1UserName, master1Table,
+      clientManager1.securityOperations().grantTablePermission(manager1UserName, manager1Table,
           TablePermission.WRITE);
-      clientMaster2.securityOperations().grantTablePermission(master2UserName, master2Table,
+      clientManager2.securityOperations().grantTablePermission(manager2UserName, manager2Table,
           TablePermission.WRITE);
 
       IteratorSetting summingCombiner = new IteratorSetting(50, SummingCombiner.class);
@@ -297,81 +298,81 @@
 
       // Set a combiner on both instances that will sum multiple values
       // We can use this to verify that the mutation was not sent multiple times
-      clientMaster1.tableOperations().attachIterator(master1Table, summingCombiner);
-      clientMaster2.tableOperations().attachIterator(master2Table, summingCombiner);
+      clientManager1.tableOperations().attachIterator(manager1Table, summingCombiner);
+      clientManager2.tableOperations().attachIterator(manager2Table, summingCombiner);
 
       // Write a single entry
-      try (BatchWriter bw = clientMaster1.createBatchWriter(master1Table)) {
+      try (BatchWriter bw = clientManager1.createBatchWriter(manager1Table)) {
         Mutation m = new Mutation("row");
         m.put("count", "", "1");
         bw.addMutation(m);
       }
 
-      Set<String> files = clientMaster1.replicationOperations().referencedFiles(master1Table);
+      Set<String> files = clientManager1.replicationOperations().referencedFiles(manager1Table);
 
-      log.info("Found {} that need replication from master1", files);
+      log.info("Found {} that need replication from manager1", files);
 
-      // Kill and restart the tserver to close the WAL on master1
-      for (ProcessReference proc : master1Cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
-        master1Cluster.killProcess(ServerType.TABLET_SERVER, proc);
+      // Kill and restart the tserver to close the WAL on manager1
+      for (ProcessReference proc : manager1Cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
+        manager1Cluster.killProcess(ServerType.TABLET_SERVER, proc);
       }
 
-      master1Cluster.exec(TabletServer.class);
+      manager1Cluster.exec(TabletServer.class);
 
-      log.info("Restarted tserver on master1");
+      log.info("Restarted tserver on manager1");
 
       // Try to avoid ACCUMULO-2964
       Thread.sleep(1000);
 
-      // Sanity check that the element is there on master1
+      // Sanity check that the element is there on manager1
       Entry<Key,Value> entry;
-      try (Scanner s = clientMaster1.createScanner(master1Table, Authorizations.EMPTY)) {
+      try (Scanner s = clientManager1.createScanner(manager1Table, Authorizations.EMPTY)) {
         entry = Iterables.getOnlyElement(s);
         assertEquals("1", entry.getValue().toString());
 
         // Wait for this table to replicate
-        clientMaster1.replicationOperations().drain(master1Table, files);
+        clientManager1.replicationOperations().drain(manager1Table, files);
 
         Thread.sleep(5000);
       }
 
-      // Check that the element made it to master2 only once
-      try (Scanner s = clientMaster2.createScanner(master2Table, Authorizations.EMPTY)) {
+      // Check that the element made it to manager2 only once
+      try (Scanner s = clientManager2.createScanner(manager2Table, Authorizations.EMPTY)) {
         entry = Iterables.getOnlyElement(s);
         assertEquals("1", entry.getValue().toString());
 
-        // Wait for master2 to finish replicating it back
-        files = clientMaster2.replicationOperations().referencedFiles(master2Table);
+        // Wait for manager2 to finish replicating it back
+        files = clientManager2.replicationOperations().referencedFiles(manager2Table);
 
-        // Kill and restart the tserver to close the WAL on master2
-        for (ProcessReference proc : master2Cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
-          master2Cluster.killProcess(ServerType.TABLET_SERVER, proc);
+        // Kill and restart the tserver to close the WAL on manager2
+        for (ProcessReference proc : manager2Cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
+          manager2Cluster.killProcess(ServerType.TABLET_SERVER, proc);
         }
 
-        master2Cluster.exec(TabletServer.class);
+        manager2Cluster.exec(TabletServer.class);
 
         // Try to avoid ACCUMULO-2964
         Thread.sleep(1000);
       }
 
-      // Check that the element made it to master2 only once
-      try (Scanner s = clientMaster2.createScanner(master2Table, Authorizations.EMPTY)) {
+      // Check that the element made it to manager2 only once
+      try (Scanner s = clientManager2.createScanner(manager2Table, Authorizations.EMPTY)) {
         entry = Iterables.getOnlyElement(s);
         assertEquals("1", entry.getValue().toString());
 
-        clientMaster2.replicationOperations().drain(master2Table, files);
+        clientManager2.replicationOperations().drain(manager2Table, files);
 
         Thread.sleep(5000);
       }
 
-      // Verify that the entry wasn't sent back to master1
-      try (Scanner s = clientMaster1.createScanner(master1Table, Authorizations.EMPTY)) {
+      // Verify that the entry wasn't sent back to manager1
+      try (Scanner s = clientManager1.createScanner(manager1Table, Authorizations.EMPTY)) {
         entry = Iterables.getOnlyElement(s);
         assertEquals("1", entry.getValue().toString());
       }
     } finally {
-      master1Cluster.stop();
-      master2Cluster.stop();
+      manager1Cluster.stop();
+      manager2Cluster.stop();
     }
   }
 
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java b/test/src/main/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
index 92a94e0..3fa0495 100644
--- a/test/src/main/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
@@ -97,8 +97,8 @@
     cfg.setProperty(Property.GC_CYCLE_DELAY, GC_PERIOD_SECONDS + "s");
     // Wait longer to try to let the replication table come online before a cycle runs
     cfg.setProperty(Property.GC_CYCLE_START, "10s");
-    cfg.setProperty(Property.REPLICATION_NAME, "master");
-    // Set really long delays for the master to do stuff for replication. We don't need
+    cfg.setProperty(Property.REPLICATION_NAME, "manager");
+    // Set really long delays for the manager to do stuff for replication. We don't need
     // it to be doing anything, so just let it sleep
     cfg.setProperty(Property.REPLICATION_WORK_PROCESSOR_DELAY, "240s");
     cfg.setProperty(Property.MANAGER_REPLICATION_SCAN_INTERVAL, "240s");
@@ -384,7 +384,7 @@
 
     client.tableOperations().flush(otherTable, null, null, true);
 
-    // Get the tservers which the master deems as active
+    // Get the tservers which the manager deems as active
     final ClientContext context = (ClientContext) client;
     List<String> tservers = ManagerClient.execute(context,
         cli -> cli.getActiveTservers(TraceUtil.traceInfo(), context.rpcCreds()));
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/KerberosReplicationIT.java b/test/src/main/java/org/apache/accumulo/test/replication/KerberosReplicationIT.java
index 09c2fe2..b9be874 100644
--- a/test/src/main/java/org/apache/accumulo/test/replication/KerberosReplicationIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/replication/KerberosReplicationIT.java
@@ -211,22 +211,22 @@
 
         primaryclient.tableOperations().create(primaryTable1,
             new NewTableConfiguration().setProperties(props));
-        String masterTableId1 = primaryclient.tableOperations().tableIdMap().get(primaryTable1);
-        assertNotNull(masterTableId1);
+        String managerTableId1 = primaryclient.tableOperations().tableIdMap().get(primaryTable1);
+        assertNotNull(managerTableId1);
 
         // Grant write permission
         peerclient.securityOperations().grantTablePermission(replicationUser.getPrincipal(),
             peerTable1, TablePermission.WRITE);
 
         // Write some data to table1
-        long masterTable1Records = 0L;
+        long managerTable1Records = 0L;
         try (BatchWriter bw = primaryclient.createBatchWriter(primaryTable1)) {
           for (int rows = 0; rows < 2500; rows++) {
             Mutation m = new Mutation(primaryTable1 + rows);
             for (int cols = 0; cols < 100; cols++) {
               String value = Integer.toString(cols);
               m.put(value, "", value);
-              masterTable1Records++;
+              managerTable1Records++;
             }
             bw.addMutation(m);
           }
@@ -264,7 +264,7 @@
         }
 
         log.info("Found {} records in {}", countTable, peerTable1);
-        assertEquals(masterTable1Records, countTable);
+        assertEquals(managerTable1Records, countTable);
 
         return null;
       }
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java b/test/src/main/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
index 79d22ad..aa5fb53 100644
--- a/test/src/main/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
@@ -115,7 +115,7 @@
     cfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
     cfg.setProperty(Property.MANAGER_REPLICATION_SCAN_INTERVAL, "1s");
     cfg.setProperty(Property.REPLICATION_MAX_UNIT_SIZE, "8M");
-    cfg.setProperty(Property.REPLICATION_NAME, "master");
+    cfg.setProperty(Property.REPLICATION_NAME, "manager");
     cfg.setProperty(Property.REPLICATION_WORK_ASSIGNER, SequentialWorkAssigner.class.getName());
     cfg.setProperty(Property.TSERV_TOTAL_MUTATION_QUEUE_MAX, "1M");
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
@@ -180,11 +180,11 @@
 
     peerCluster.start();
 
-    try (AccumuloClient clientMaster = Accumulo.newClient().from(getClientProperties()).build();
+    try (AccumuloClient clientManager = Accumulo.newClient().from(getClientProperties()).build();
         AccumuloClient clientPeer =
             peerCluster.createAccumuloClient("root", new PasswordToken(ROOT_PASSWORD))) {
 
-      ReplicationTable.setOnline(clientMaster);
+      ReplicationTable.setOnline(clientManager);
 
       String peerUserName = "peer", peerPassword = "foo";
 
@@ -193,19 +193,19 @@
       clientPeer.securityOperations().createLocalUser(peerUserName,
           new PasswordToken(peerPassword));
 
-      clientMaster.instanceOperations()
+      clientManager.instanceOperations()
           .setProperty(Property.REPLICATION_PEER_USER.getKey() + peerClusterName, peerUserName);
-      clientMaster.instanceOperations()
+      clientManager.instanceOperations()
           .setProperty(Property.REPLICATION_PEER_PASSWORD.getKey() + peerClusterName, peerPassword);
 
       // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
-      clientMaster.instanceOperations().setProperty(
+      clientManager.instanceOperations().setProperty(
           Property.REPLICATION_PEERS.getKey() + peerClusterName,
           ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
               AccumuloReplicaSystem.buildConfiguration(peerCluster.getInstanceName(),
                   peerCluster.getZooKeepers())));
 
-      final String masterTable = "master", peerTable = "peer";
+      final String managerTable = "manager", peerTable = "peer";
 
       clientPeer.tableOperations().create(peerTable, new NewTableConfiguration());
       String peerTableId = clientPeer.tableOperations().tableIdMap().get(peerTable);
@@ -219,13 +219,13 @@
       props.put(Property.TABLE_REPLICATION.getKey(), "true");
       props.put(Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId);
 
-      clientMaster.tableOperations().create(masterTable,
+      clientManager.tableOperations().create(managerTable,
           new NewTableConfiguration().setProperties(props));
-      String masterTableId = clientMaster.tableOperations().tableIdMap().get(masterTable);
-      assertNotNull(masterTableId);
+      String managerTableId = clientManager.tableOperations().tableIdMap().get(managerTable);
+      assertNotNull(managerTableId);
 
       // Write some data to table1
-      try (BatchWriter bw = clientMaster.createBatchWriter(masterTable)) {
+      try (BatchWriter bw = clientManager.createBatchWriter(managerTable)) {
         for (int rows = 0; rows < 5000; rows++) {
           Mutation m = new Mutation(Integer.toString(rows));
           for (int cols = 0; cols < 100; cols++) {
@@ -236,10 +236,10 @@
         }
       }
 
-      log.info("Wrote all data to master cluster");
+      log.info("Wrote all data to manager cluster");
 
       final Set<String> filesNeedingReplication =
-          clientMaster.replicationOperations().referencedFiles(masterTable);
+          clientManager.replicationOperations().referencedFiles(managerTable);
 
       log.info("Files to replicate: " + filesNeedingReplication);
 
@@ -249,17 +249,17 @@
       cluster.exec(TabletServer.class);
 
       log.info("TabletServer restarted");
-      Iterators.size(ReplicationTable.getScanner(clientMaster).iterator());
+      Iterators.size(ReplicationTable.getScanner(clientManager).iterator());
       log.info("TabletServer is online");
 
-      while (!ReplicationTable.isOnline(clientMaster)) {
+      while (!ReplicationTable.isOnline(clientManager)) {
         log.info("Replication table still offline, waiting");
         Thread.sleep(5000);
       }
 
       log.info("");
       log.info("Fetching metadata records:");
-      try (var scanner = clientMaster.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+      try (var scanner = clientManager.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
         for (Entry<Key,Value> kv : scanner) {
           if (ReplicationSection.COLF.equals(kv.getKey().getColumnFamily())) {
             log.info("{} {}", kv.getKey().toStringNoTruncate(),
@@ -272,7 +272,7 @@
 
       log.info("");
       log.info("Fetching replication records:");
-      try (var scanner = ReplicationTable.getScanner(clientMaster)) {
+      try (var scanner = ReplicationTable.getScanner(clientManager)) {
         for (Entry<Key,Value> kv : scanner) {
           log.info("{} {}", kv.getKey().toStringNoTruncate(),
               ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
@@ -281,7 +281,7 @@
 
       Future<Boolean> future = executor.submit(() -> {
         long then = System.currentTimeMillis();
-        clientMaster.replicationOperations().drain(masterTable, filesNeedingReplication);
+        clientManager.replicationOperations().drain(managerTable, filesNeedingReplication);
         long now = System.currentTimeMillis();
         log.info("Drain completed in " + (now - then) + "ms");
         return true;
@@ -300,7 +300,7 @@
 
       log.info("");
       log.info("Fetching metadata records:");
-      try (var scanner = clientMaster.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+      try (var scanner = clientManager.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
         for (Entry<Key,Value> kv : scanner) {
           if (ReplicationSection.COLF.equals(kv.getKey().getColumnFamily())) {
             log.info("{} {}", kv.getKey().toStringNoTruncate(),
@@ -313,30 +313,30 @@
 
       log.info("");
       log.info("Fetching replication records:");
-      try (var scanner = ReplicationTable.getScanner(clientMaster)) {
+      try (var scanner = ReplicationTable.getScanner(clientManager)) {
         for (Entry<Key,Value> kv : scanner) {
           log.info("{} {}", kv.getKey().toStringNoTruncate(),
               ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
         }
       }
 
-      try (Scanner master = clientMaster.createScanner(masterTable, Authorizations.EMPTY);
+      try (Scanner manager = clientManager.createScanner(managerTable, Authorizations.EMPTY);
           Scanner peer = clientPeer.createScanner(peerTable, Authorizations.EMPTY)) {
-        Iterator<Entry<Key,Value>> masterIter = master.iterator(), peerIter = peer.iterator();
-        Entry<Key,Value> masterEntry = null, peerEntry = null;
-        while (masterIter.hasNext() && peerIter.hasNext()) {
-          masterEntry = masterIter.next();
+        Iterator<Entry<Key,Value>> managerIter = manager.iterator(), peerIter = peer.iterator();
+        Entry<Key,Value> managerEntry = null, peerEntry = null;
+        while (managerIter.hasNext() && peerIter.hasNext()) {
+          managerEntry = managerIter.next();
           peerEntry = peerIter.next();
-          assertEquals(masterEntry.getKey() + " was not equal to " + peerEntry.getKey(), 0,
-              masterEntry.getKey().compareTo(peerEntry.getKey(),
+          assertEquals(managerEntry.getKey() + " was not equal to " + peerEntry.getKey(), 0,
+              managerEntry.getKey().compareTo(peerEntry.getKey(),
                   PartialKey.ROW_COLFAM_COLQUAL_COLVIS));
-          assertEquals(masterEntry.getValue(), peerEntry.getValue());
+          assertEquals(managerEntry.getValue(), peerEntry.getValue());
         }
 
-        log.info("Last master entry: {}", masterEntry);
+        log.info("Last manager entry: {}", managerEntry);
         log.info("Last peer entry: {}", peerEntry);
 
-        assertFalse("Had more data to read from the master", masterIter.hasNext());
+        assertFalse("Had more data to read from the manager", managerIter.hasNext());
         assertFalse("Had more data to read from the peer", peerIter.hasNext());
       }
     } finally {
@@ -359,7 +359,7 @@
 
     peer1Cluster.start();
 
-    try (AccumuloClient clientMaster = Accumulo.newClient().from(getClientProperties()).build();
+    try (AccumuloClient clientManager = Accumulo.newClient().from(getClientProperties()).build();
         AccumuloClient clientPeer =
             peer1Cluster.createAccumuloClient("root", new PasswordToken(ROOT_PASSWORD))) {
       String peerClusterName = "peer";
@@ -369,19 +369,19 @@
       clientPeer.securityOperations().createLocalUser(peerUserName,
           new PasswordToken(peerPassword));
 
-      clientMaster.instanceOperations()
+      clientManager.instanceOperations()
           .setProperty(Property.REPLICATION_PEER_USER.getKey() + peerClusterName, peerUserName);
-      clientMaster.instanceOperations()
+      clientManager.instanceOperations()
           .setProperty(Property.REPLICATION_PEER_PASSWORD.getKey() + peerClusterName, peerPassword);
 
       // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
-      clientMaster.instanceOperations().setProperty(
+      clientManager.instanceOperations().setProperty(
           Property.REPLICATION_PEERS.getKey() + peerClusterName,
           ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
               AccumuloReplicaSystem.buildConfiguration(peer1Cluster.getInstanceName(),
                   peer1Cluster.getZooKeepers())));
 
-      String masterTable1 = "master1", peerTable1 = "peer1", masterTable2 = "master2",
+      String managerTable1 = "manager1", peerTable1 = "peer1", managerTable2 = "manager2",
           peerTable2 = "peer2";
 
       // Create tables
@@ -397,18 +397,18 @@
       props1.put(Property.TABLE_REPLICATION.getKey(), "true");
       props1.put(Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId1);
 
-      clientMaster.tableOperations().create(masterTable1,
+      clientManager.tableOperations().create(managerTable1,
           new NewTableConfiguration().setProperties(props1));
-      String masterTableId1 = clientMaster.tableOperations().tableIdMap().get(masterTable1);
-      assertNotNull(masterTableId1);
+      String managerTableId1 = clientManager.tableOperations().tableIdMap().get(managerTable1);
+      assertNotNull(managerTableId1);
       Map<String,String> props2 = new HashMap<>();
       props2.put(Property.TABLE_REPLICATION.getKey(), "true");
       props2.put(Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId2);
 
-      clientMaster.tableOperations().create(masterTable2,
+      clientManager.tableOperations().create(managerTable2,
           new NewTableConfiguration().setProperties(props2));
-      String masterTableId2 = clientMaster.tableOperations().tableIdMap().get(masterTable2);
-      assertNotNull(masterTableId2);
+      String managerTableId2 = clientManager.tableOperations().tableIdMap().get(managerTable2);
+      assertNotNull(managerTableId2);
 
       // Give our replication user the ability to write to the tables
       clientPeer.securityOperations().grantTablePermission(peerUserName, peerTable1,
@@ -417,37 +417,37 @@
           TablePermission.WRITE);
 
       // Write some data to table1
-      long masterTable1Records = 0L;
-      try (BatchWriter bw = clientMaster.createBatchWriter(masterTable1)) {
+      long managerTable1Records = 0L;
+      try (BatchWriter bw = clientManager.createBatchWriter(managerTable1)) {
         for (int rows = 0; rows < 2500; rows++) {
-          Mutation m = new Mutation(masterTable1 + rows);
+          Mutation m = new Mutation(managerTable1 + rows);
           for (int cols = 0; cols < 100; cols++) {
             String value = Integer.toString(cols);
             m.put(value, "", value);
-            masterTable1Records++;
+            managerTable1Records++;
           }
           bw.addMutation(m);
         }
       }
 
       // Write some data to table2
-      long masterTable2Records = 0L;
-      try (BatchWriter bw = clientMaster.createBatchWriter(masterTable2)) {
+      long managerTable2Records = 0L;
+      try (BatchWriter bw = clientManager.createBatchWriter(managerTable2)) {
         for (int rows = 0; rows < 2500; rows++) {
-          Mutation m = new Mutation(masterTable2 + rows);
+          Mutation m = new Mutation(managerTable2 + rows);
           for (int cols = 0; cols < 100; cols++) {
             String value = Integer.toString(cols);
             m.put(value, "", value);
-            masterTable2Records++;
+            managerTable2Records++;
           }
           bw.addMutation(m);
         }
       }
 
-      log.info("Wrote all data to master cluster");
+      log.info("Wrote all data to manager cluster");
 
-      Set<String> filesFor1 = clientMaster.replicationOperations().referencedFiles(masterTable1),
-          filesFor2 = clientMaster.replicationOperations().referencedFiles(masterTable2);
+      Set<String> filesFor1 = clientManager.replicationOperations().referencedFiles(managerTable1),
+          filesFor2 = clientManager.replicationOperations().referencedFiles(managerTable2);
 
       log.info("Files to replicate for table1: " + filesFor1);
       log.info("Files to replicate for table2: " + filesFor2);
@@ -461,43 +461,43 @@
       log.info("Restarted the tserver");
 
       // Read the data -- the tserver is back up and running
-      Iterators.size(clientMaster.createScanner(masterTable1, Authorizations.EMPTY).iterator());
+      Iterators.size(clientManager.createScanner(managerTable1, Authorizations.EMPTY).iterator());
 
-      while (!ReplicationTable.isOnline(clientMaster)) {
+      while (!ReplicationTable.isOnline(clientManager)) {
         log.info("Replication table still offline, waiting");
         Thread.sleep(5000);
       }
 
       // Wait for both tables to be replicated
-      log.info("Waiting for {} for {}", filesFor1, masterTable1);
-      clientMaster.replicationOperations().drain(masterTable1, filesFor1);
+      log.info("Waiting for {} for {}", filesFor1, managerTable1);
+      clientManager.replicationOperations().drain(managerTable1, filesFor1);
 
-      log.info("Waiting for {} for {}", filesFor2, masterTable2);
-      clientMaster.replicationOperations().drain(masterTable2, filesFor2);
+      log.info("Waiting for {} for {}", filesFor2, managerTable2);
+      clientManager.replicationOperations().drain(managerTable2, filesFor2);
 
       long countTable = 0L;
       try (var scanner = clientPeer.createScanner(peerTable1, Authorizations.EMPTY)) {
         for (Entry<Key,Value> entry : scanner) {
           countTable++;
           assertTrue("Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " "
-              + entry.getValue(), entry.getKey().getRow().toString().startsWith(masterTable1));
+              + entry.getValue(), entry.getKey().getRow().toString().startsWith(managerTable1));
         }
       }
 
       log.info("Found {} records in {}", countTable, peerTable1);
-      assertEquals(masterTable1Records, countTable);
+      assertEquals(managerTable1Records, countTable);
 
       countTable = 0L;
       try (var scanner = clientPeer.createScanner(peerTable2, Authorizations.EMPTY)) {
         for (Entry<Key,Value> entry : scanner) {
           countTable++;
           assertTrue("Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " "
-              + entry.getValue(), entry.getKey().getRow().toString().startsWith(masterTable2));
+              + entry.getValue(), entry.getKey().getRow().toString().startsWith(managerTable2));
         }
       }
 
       log.info("Found {} records in {}", countTable, peerTable2);
-      assertEquals(masterTable2Records, countTable);
+      assertEquals(managerTable2Records, countTable);
 
     } finally {
       peer1Cluster.stop();
@@ -519,7 +519,7 @@
 
     peerCluster.start();
 
-    try (AccumuloClient clientMaster = Accumulo.newClient().from(getClientProperties()).build();
+    try (AccumuloClient clientManager = Accumulo.newClient().from(getClientProperties()).build();
         AccumuloClient clientPeer =
             peerCluster.createAccumuloClient("root", new PasswordToken(ROOT_PASSWORD))) {
 
@@ -533,7 +533,7 @@
       String peerClusterName = "peer";
 
       // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
-      clientMaster.instanceOperations().setProperty(
+      clientManager.instanceOperations().setProperty(
           Property.REPLICATION_PEERS.getKey() + peerClusterName,
           ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
               AccumuloReplicaSystem.buildConfiguration(peerCluster.getInstanceName(),
@@ -541,12 +541,12 @@
 
       // Configure the credentials we should use to authenticate ourselves to the peer for
       // replication
-      clientMaster.instanceOperations()
+      clientManager.instanceOperations()
           .setProperty(Property.REPLICATION_PEER_USER.getKey() + peerClusterName, peerUserName);
-      clientMaster.instanceOperations()
+      clientManager.instanceOperations()
           .setProperty(Property.REPLICATION_PEER_PASSWORD.getKey() + peerClusterName, peerPassword);
 
-      String masterTable = "master", peerTable = "peer";
+      String managerTable = "manager", peerTable = "peer";
       clientPeer.tableOperations().create(peerTable, new NewTableConfiguration());
       String peerTableId = clientPeer.tableOperations().tableIdMap().get(peerTable);
       assertNotNull(peerTableId);
@@ -559,13 +559,13 @@
       props.put(Property.TABLE_REPLICATION.getKey(), "true");
       // Replicate this table to the peerClusterName in a table with the peerTableId table id
       props.put(Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId);
-      clientMaster.tableOperations().create(masterTable,
+      clientManager.tableOperations().create(managerTable,
           new NewTableConfiguration().setProperties(props));
-      String masterTableId = clientMaster.tableOperations().tableIdMap().get(masterTable);
-      assertNotNull(masterTableId);
+      String managerTableId = clientManager.tableOperations().tableIdMap().get(managerTable);
+      assertNotNull(managerTableId);
 
       // Write some data to table1
-      try (BatchWriter bw = clientMaster.createBatchWriter(masterTable)) {
+      try (BatchWriter bw = clientManager.createBatchWriter(managerTable)) {
         for (int rows = 0; rows < 5000; rows++) {
           Mutation m = new Mutation(Integer.toString(rows));
           for (int cols = 0; cols < 100; cols++) {
@@ -576,9 +576,9 @@
         }
       }
 
-      log.info("Wrote all data to master cluster");
+      log.info("Wrote all data to manager cluster");
 
-      Set<String> files = clientMaster.replicationOperations().referencedFiles(masterTable);
+      Set<String> files = clientManager.replicationOperations().referencedFiles(managerTable);
 
       log.info("Files to replicate:" + files);
 
@@ -588,34 +588,34 @@
 
       cluster.exec(TabletServer.class);
 
-      while (!ReplicationTable.isOnline(clientMaster)) {
+      while (!ReplicationTable.isOnline(clientManager)) {
         log.info("Replication table still offline, waiting");
         Thread.sleep(5000);
       }
 
-      Iterators.size(clientMaster.createScanner(masterTable, Authorizations.EMPTY).iterator());
+      Iterators.size(clientManager.createScanner(managerTable, Authorizations.EMPTY).iterator());
 
-      try (var scanner = ReplicationTable.getScanner(clientMaster)) {
+      try (var scanner = ReplicationTable.getScanner(clientManager)) {
         for (Entry<Key,Value> kv : scanner) {
           log.debug("{} {}", kv.getKey().toStringNoTruncate(),
               ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
         }
       }
 
-      clientMaster.replicationOperations().drain(masterTable, files);
+      clientManager.replicationOperations().drain(managerTable, files);
 
-      try (Scanner master = clientMaster.createScanner(masterTable, Authorizations.EMPTY);
+      try (Scanner manager = clientManager.createScanner(managerTable, Authorizations.EMPTY);
           Scanner peer = clientPeer.createScanner(peerTable, Authorizations.EMPTY)) {
-        Iterator<Entry<Key,Value>> masterIter = master.iterator(), peerIter = peer.iterator();
-        while (masterIter.hasNext() && peerIter.hasNext()) {
-          Entry<Key,Value> masterEntry = masterIter.next(), peerEntry = peerIter.next();
+        Iterator<Entry<Key,Value>> managerIter = manager.iterator(), peerIter = peer.iterator();
+        while (managerIter.hasNext() && peerIter.hasNext()) {
+          Entry<Key,Value> managerEntry = managerIter.next(), peerEntry = peerIter.next();
           assertEquals(peerEntry.getKey() + " was not equal to " + peerEntry.getKey(), 0,
-              masterEntry.getKey().compareTo(peerEntry.getKey(),
+              managerEntry.getKey().compareTo(peerEntry.getKey(),
                   PartialKey.ROW_COLFAM_COLQUAL_COLVIS));
-          assertEquals(masterEntry.getValue(), peerEntry.getValue());
+          assertEquals(managerEntry.getValue(), peerEntry.getValue());
         }
 
-        assertFalse("Had more data to read from the master", masterIter.hasNext());
+        assertFalse("Had more data to read from the manager", managerIter.hasNext());
         assertFalse("Had more data to read from the peer", peerIter.hasNext());
       }
     } finally {
@@ -638,7 +638,7 @@
 
     peer1Cluster.start();
 
-    try (AccumuloClient clientMaster = Accumulo.newClient().from(getClientProperties()).build();
+    try (AccumuloClient clientManager = Accumulo.newClient().from(getClientProperties()).build();
         AccumuloClient clientPeer =
             peer1Cluster.createAccumuloClient("root", new PasswordToken(ROOT_PASSWORD))) {
 
@@ -653,19 +653,19 @@
 
       // Configure the credentials we should use to authenticate ourselves to the peer for
       // replication
-      clientMaster.instanceOperations()
+      clientManager.instanceOperations()
           .setProperty(Property.REPLICATION_PEER_USER.getKey() + peerClusterName, peerUserName);
-      clientMaster.instanceOperations()
+      clientManager.instanceOperations()
           .setProperty(Property.REPLICATION_PEER_PASSWORD.getKey() + peerClusterName, peerPassword);
 
       // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
-      clientMaster.instanceOperations().setProperty(
+      clientManager.instanceOperations().setProperty(
           Property.REPLICATION_PEERS.getKey() + peerClusterName,
           ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
               AccumuloReplicaSystem.buildConfiguration(peer1Cluster.getInstanceName(),
                   peer1Cluster.getZooKeepers())));
 
-      String masterTable1 = "master1", peerTable1 = "peer1", masterTable2 = "master2",
+      String managerTable1 = "manager1", peerTable1 = "peer1", managerTable2 = "manager2",
           peerTable2 = "peer2";
       // Create tables
       clientPeer.tableOperations().create(peerTable1, new NewTableConfiguration());
@@ -680,19 +680,19 @@
       props1.put(Property.TABLE_REPLICATION.getKey(), "true");
       props1.put(Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId2);
 
-      clientMaster.tableOperations().create(masterTable1,
+      clientManager.tableOperations().create(managerTable1,
           new NewTableConfiguration().setProperties(props1));
-      String masterTableId1 = clientMaster.tableOperations().tableIdMap().get(masterTable1);
-      assertNotNull(masterTableId1);
+      String managerTableId1 = clientManager.tableOperations().tableIdMap().get(managerTable1);
+      assertNotNull(managerTableId1);
 
       Map<String,String> props2 = new HashMap<>();
       props2.put(Property.TABLE_REPLICATION.getKey(), "true");
       props2.put(Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId2);
 
-      clientMaster.tableOperations().create(masterTable2,
+      clientManager.tableOperations().create(managerTable2,
           new NewTableConfiguration().setProperties(props2));
-      String masterTableId2 = clientMaster.tableOperations().tableIdMap().get(masterTable2);
-      assertNotNull(masterTableId2);
+      String managerTableId2 = clientManager.tableOperations().tableIdMap().get(managerTable2);
+      assertNotNull(managerTableId2);
 
       // Give our replication user the ability to write to the tables
       clientPeer.securityOperations().grantTablePermission(peerUserName, peerTable1,
@@ -701,15 +701,15 @@
           TablePermission.WRITE);
 
       // Replicate this table to the peerClusterName in a table with the peerTableId table id
-      clientMaster.tableOperations().setProperty(masterTable1,
+      clientManager.tableOperations().setProperty(managerTable1,
           Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId1);
-      clientMaster.tableOperations().setProperty(masterTable2,
+      clientManager.tableOperations().setProperty(managerTable2,
           Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId2);
 
       // Write some data to table1
-      try (BatchWriter bw = clientMaster.createBatchWriter(masterTable1)) {
+      try (BatchWriter bw = clientManager.createBatchWriter(managerTable1)) {
         for (int rows = 0; rows < 2500; rows++) {
-          Mutation m = new Mutation(masterTable1 + rows);
+          Mutation m = new Mutation(managerTable1 + rows);
           for (int cols = 0; cols < 100; cols++) {
             String value = Integer.toString(cols);
             m.put(value, "", value);
@@ -719,9 +719,9 @@
       }
 
       // Write some data to table2
-      try (BatchWriter bw = clientMaster.createBatchWriter(masterTable2)) {
+      try (BatchWriter bw = clientManager.createBatchWriter(managerTable2)) {
         for (int rows = 0; rows < 2500; rows++) {
-          Mutation m = new Mutation(masterTable2 + rows);
+          Mutation m = new Mutation(managerTable2 + rows);
           for (int cols = 0; cols < 100; cols++) {
             String value = Integer.toString(cols);
             m.put(value, "", value);
@@ -730,7 +730,7 @@
         }
       }
 
-      log.info("Wrote all data to master cluster");
+      log.info("Wrote all data to manager cluster");
 
       for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
         cluster.killProcess(ServerType.TABLET_SERVER, proc);
@@ -738,7 +738,7 @@
 
       cluster.exec(TabletServer.class);
 
-      while (!ReplicationTable.isOnline(clientMaster)) {
+      while (!ReplicationTable.isOnline(clientManager)) {
         log.info("Replication table still offline, waiting");
         Thread.sleep(5000);
       }
@@ -748,7 +748,7 @@
       for (int i = 0; i < 10 && !fullyReplicated; i++) {
         sleepUninterruptibly(2, TimeUnit.SECONDS);
 
-        try (Scanner s = ReplicationTable.getScanner(clientMaster)) {
+        try (Scanner s = ReplicationTable.getScanner(clientManager)) {
           WorkSection.limit(s);
           for (Entry<Key,Value> entry : s) {
             Status status = Status.parseFrom(entry.getValue().get());
@@ -761,7 +761,7 @@
 
       assertNotEquals(0, fullyReplicated);
 
-      // We have to wait for the master to assign the replication work, a local tserver to process
+      // We have to wait for the manager to assign the replication work, a local tserver to process
       // it, and then the remote tserver to replay it
       // Be cautious in how quickly we assert that the data is present on the peer
       long countTable = 0L;
@@ -769,7 +769,7 @@
         for (Entry<Key,Value> entry : clientPeer.createScanner(peerTable1, Authorizations.EMPTY)) {
           countTable++;
           assertTrue("Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " "
-              + entry.getValue(), entry.getKey().getRow().toString().startsWith(masterTable1));
+              + entry.getValue(), entry.getKey().getRow().toString().startsWith(managerTable1));
         }
 
         log.info("Found {} records in {}", countTable, peerTable1);
@@ -783,7 +783,7 @@
 
       assertTrue("Found no records in " + peerTable1 + " in the peer cluster", countTable > 0);
 
-      // We have to wait for the master to assign the replication work, a local tserver to process
+      // We have to wait for the manager to assign the replication work, a local tserver to process
       // it, and then the remote tserver to replay it
       // Be cautious in how quickly we assert that the data is present on the peer
       for (int i = 0; i < 10; i++) {
@@ -791,7 +791,7 @@
         for (Entry<Key,Value> entry : clientPeer.createScanner(peerTable2, Authorizations.EMPTY)) {
           countTable++;
           assertTrue("Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " "
-              + entry.getValue(), entry.getKey().getRow().toString().startsWith(masterTable2));
+              + entry.getValue(), entry.getKey().getRow().toString().startsWith(managerTable2));
         }
 
         log.info("Found {} records in {}", countTable, peerTable2);
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/MultiTserverReplicationIT.java b/test/src/main/java/org/apache/accumulo/test/replication/MultiTserverReplicationIT.java
index 54913e0..524a425 100644
--- a/test/src/main/java/org/apache/accumulo/test/replication/MultiTserverReplicationIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/replication/MultiTserverReplicationIT.java
@@ -97,7 +97,7 @@
   }
 
   @Test
-  public void masterReplicationServicePortsAreAdvertised() throws Exception {
+  public void managerReplicationServicePortsAreAdvertised() throws Exception {
     // Wait for the cluster to be up
     AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build();
     ClientContext context = (ClientContext) client;
@@ -110,22 +110,22 @@
       ZooReader zreader =
           new ZooReader(context.getZooKeepers(), context.getZooKeepersSessionTimeOut());
 
-      // Should have one master instance
-      assertEquals(1, context.getMasterLocations().size());
+      // Should have one manager instance
+      assertEquals(1, context.getManagerLocations().size());
 
-      // Get the master thrift service addr
-      String masterAddr = Iterables.getOnlyElement(context.getMasterLocations());
+      // Get the manager thrift service addr
+      String managerAddr = Iterables.getOnlyElement(context.getManagerLocations());
 
-      // Get the master replication coordinator addr
+      // Get the manager replication coordinator addr
       String replCoordAddr =
           new String(zreader.getData(ZooUtil.getRoot(client.instanceOperations().getInstanceID())
-              + Constants.ZMASTER_REPLICATION_COORDINATOR_ADDR), UTF_8);
+              + Constants.ZMANAGER_REPLICATION_COORDINATOR_ADDR), UTF_8);
 
       // They shouldn't be the same
-      assertNotEquals(masterAddr, replCoordAddr);
+      assertNotEquals(managerAddr, replCoordAddr);
 
       // Neither should be zero as the port
-      assertNotEquals(0, HostAndPort.fromString(masterAddr).getPort());
+      assertNotEquals(0, HostAndPort.fromString(managerAddr).getPort());
       assertNotEquals(0, HostAndPort.fromString(replCoordAddr).getPort());
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/ReplicationIT.java b/test/src/main/java/org/apache/accumulo/test/replication/ReplicationIT.java
index acdd50e..af50df8 100644
--- a/test/src/main/java/org/apache/accumulo/test/replication/ReplicationIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/replication/ReplicationIT.java
@@ -135,7 +135,7 @@
 
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    // Run the master replication loop run frequently
+    // Run the manager replication loop run frequently
     cfg.setClientProperty(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT, "15s");
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
     cfg.setProperty(Property.MANAGER_REPLICATION_SCAN_INTERVAL, "1s");
@@ -143,7 +143,7 @@
     cfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, "1M");
     cfg.setProperty(Property.GC_CYCLE_START, "1s");
     cfg.setProperty(Property.GC_CYCLE_DELAY, "0");
-    cfg.setProperty(Property.REPLICATION_NAME, "master");
+    cfg.setProperty(Property.REPLICATION_NAME, "manager");
     cfg.setProperty(Property.REPLICATION_WORK_PROCESSOR_DELAY, "1s");
     cfg.setProperty(Property.REPLICATION_WORK_PROCESSOR_PERIOD, "1s");
     cfg.setProperty(Property.TSERV_TOTAL_MUTATION_QUEUE_MAX, "1M");
@@ -595,7 +595,8 @@
       keepRunning.set(false);
       t.join(5000);
 
-      // The master is only running every second to create records in the replication table from the
+      // The manager is only running every second to create records in the replication table from
+      // the
       // metadata table
       // Sleep a sufficient amount of time to ensure that we get the straggling WALs that might have
       // been created at the end
@@ -906,7 +907,7 @@
       assertTrue("Replication table was never created", ReplicationTable.isOnline(client));
 
       // ACCUMULO-2743 The Observer in the tserver has to be made aware of the change to get the
-      // combiner (made by the master)
+      // combiner (made by the manager)
       for (int i = 0; i < 10 && !client.tableOperations().listIterators(ReplicationTable.NAME)
           .containsKey(ReplicationTable.COMBINER_NAME); i++) {
         sleepUninterruptibly(2, TimeUnit.SECONDS);
@@ -932,7 +933,7 @@
             Status actual = Status.parseFrom(entry.getValue().get());
             if (actual.getInfiniteEnd() != expectedStatus.getInfiniteEnd()) {
               entry = null;
-              // the master process didn't yet fire and write the new mutation, wait for it to do
+              // the manager process didn't yet fire and write the new mutation, wait for it to do
               // so and try to read it again
               Thread.sleep(1000);
             }
@@ -989,9 +990,10 @@
         client.tableOperations().compact(table1, null, null, true, true);
         log.info("Compaction completed");
 
-        // Master is creating entries in the replication table from the metadata table every second.
+        // Manager is creating entries in the replication table from the metadata table every
+        // second.
         // Compaction should trigger the record to be written to metadata. Wait a bit to ensure
-        // that the master has time to work.
+        // that the manager has time to work.
         Thread.sleep(5000);
 
         try (Scanner s2 = ReplicationTable.getScanner(client)) {
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/ReplicationOperationsImplIT.java b/test/src/main/java/org/apache/accumulo/test/replication/ReplicationOperationsImplIT.java
index 202d3db..68b2ff6 100644
--- a/test/src/main/java/org/apache/accumulo/test/replication/ReplicationOperationsImplIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/replication/ReplicationOperationsImplIT.java
@@ -83,14 +83,14 @@
   }
 
   /**
-   * Spoof out the Master so we can call the implementation without starting a full instance.
+   * Spoof out the Manager so we can call the implementation without starting a full instance.
    */
   private ReplicationOperationsImpl getReplicationOperations() {
-    Manager master = EasyMock.createMock(Manager.class);
-    EasyMock.expect(master.getContext()).andReturn(serverContext).anyTimes();
-    EasyMock.replay(master);
+    Manager manager = EasyMock.createMock(Manager.class);
+    EasyMock.expect(manager.getContext()).andReturn(serverContext).anyTimes();
+    EasyMock.replay(manager);
 
-    final ManagerClientServiceHandler mcsh = new ManagerClientServiceHandler(master) {
+    final ManagerClientServiceHandler mcsh = new ManagerClientServiceHandler(manager) {
       @Override
       protected TableId getTableId(ClientContext context, String tableName) {
         try {
@@ -104,7 +104,7 @@
     ClientContext context = (ClientContext) client;
     return new ReplicationOperationsImpl(context) {
       @Override
-      protected boolean getMasterDrain(final TInfo tinfo, final TCredentials rpcCreds,
+      protected boolean getManagerDrain(final TInfo tinfo, final TCredentials rpcCreds,
           final String tableName, final Set<String> wals) {
         try {
           return mcsh.drainReplicationTable(tinfo, rpcCreds, tableName, wals);
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerReplicationIT.java b/test/src/main/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerReplicationIT.java
index 2b274cb..92fe989 100644
--- a/test/src/main/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerReplicationIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerReplicationIT.java
@@ -122,7 +122,7 @@
     cfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
     cfg.setProperty(Property.MANAGER_REPLICATION_SCAN_INTERVAL, "1s");
     cfg.setProperty(Property.REPLICATION_MAX_UNIT_SIZE, "8M");
-    cfg.setProperty(Property.REPLICATION_NAME, "master");
+    cfg.setProperty(Property.REPLICATION_NAME, "manager");
     cfg.setProperty(Property.REPLICATION_WORK_ASSIGNER, UnorderedWorkAssigner.class.getName());
     cfg.setProperty(Property.TSERV_TOTAL_MUTATION_QUEUE_MAX, "1M");
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
@@ -185,11 +185,11 @@
 
     peerCluster.start();
 
-    try (AccumuloClient clientMaster = Accumulo.newClient().from(getClientProperties()).build();
+    try (AccumuloClient clientManager = Accumulo.newClient().from(getClientProperties()).build();
         AccumuloClient clientPeer =
             peerCluster.createAccumuloClient("root", new PasswordToken(ROOT_PASSWORD))) {
 
-      ReplicationTable.setOnline(clientMaster);
+      ReplicationTable.setOnline(clientManager);
 
       String peerUserName = "peer", peerPassword = "foo";
 
@@ -198,19 +198,19 @@
       clientPeer.securityOperations().createLocalUser(peerUserName,
           new PasswordToken(peerPassword));
 
-      clientMaster.instanceOperations()
+      clientManager.instanceOperations()
           .setProperty(Property.REPLICATION_PEER_USER.getKey() + peerClusterName, peerUserName);
-      clientMaster.instanceOperations()
+      clientManager.instanceOperations()
           .setProperty(Property.REPLICATION_PEER_PASSWORD.getKey() + peerClusterName, peerPassword);
 
       // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
-      clientMaster.instanceOperations().setProperty(
+      clientManager.instanceOperations().setProperty(
           Property.REPLICATION_PEERS.getKey() + peerClusterName,
           ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
               AccumuloReplicaSystem.buildConfiguration(peerCluster.getInstanceName(),
                   peerCluster.getZooKeepers())));
 
-      final String masterTable = "master", peerTable = "peer";
+      final String managerTable = "manager", peerTable = "peer";
 
       clientPeer.tableOperations().create(peerTable);
       String peerTableId = clientPeer.tableOperations().tableIdMap().get(peerTable);
@@ -220,10 +220,10 @@
       props.put(Property.TABLE_REPLICATION.getKey(), "true");
       props.put(Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId);
 
-      clientMaster.tableOperations().create(masterTable,
+      clientManager.tableOperations().create(managerTable,
           new NewTableConfiguration().setProperties(props));
-      String masterTableId = clientMaster.tableOperations().tableIdMap().get(masterTable);
-      assertNotNull(masterTableId);
+      String managerTableId = clientManager.tableOperations().tableIdMap().get(managerTable);
+      assertNotNull(managerTableId);
 
       clientPeer.securityOperations().grantTablePermission(peerUserName, peerTable,
           TablePermission.WRITE);
@@ -232,7 +232,7 @@
       sleepUninterruptibly(3, TimeUnit.SECONDS);
 
       // Write some data to table1
-      try (BatchWriter bw = clientMaster.createBatchWriter(masterTable)) {
+      try (BatchWriter bw = clientManager.createBatchWriter(managerTable)) {
         for (int rows = 0; rows < 5000; rows++) {
           Mutation m = new Mutation(Integer.toString(rows));
           for (int cols = 0; cols < 100; cols++) {
@@ -243,10 +243,10 @@
         }
       }
 
-      log.info("Wrote all data to master cluster");
+      log.info("Wrote all data to manager cluster");
 
       final Set<String> filesNeedingReplication =
-          clientMaster.replicationOperations().referencedFiles(masterTable);
+          clientManager.replicationOperations().referencedFiles(managerTable);
 
       for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
         cluster.killProcess(ServerType.TABLET_SERVER, proc);
@@ -254,12 +254,12 @@
       cluster.exec(TabletServer.class);
 
       log.info("TabletServer restarted");
-      Iterators.size(ReplicationTable.getScanner(clientMaster).iterator());
+      Iterators.size(ReplicationTable.getScanner(clientManager).iterator());
       log.info("TabletServer is online");
 
       log.info("");
       log.info("Fetching metadata records:");
-      try (var scanner = clientMaster.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+      try (var scanner = clientManager.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
         for (Entry<Key,Value> kv : scanner) {
           if (ReplicationSection.COLF.equals(kv.getKey().getColumnFamily())) {
             log.info("{} {}", kv.getKey().toStringNoTruncate(),
@@ -272,7 +272,7 @@
 
       log.info("");
       log.info("Fetching replication records:");
-      try (var scanner = ReplicationTable.getScanner(clientMaster)) {
+      try (var scanner = ReplicationTable.getScanner(clientManager)) {
         for (Entry<Key,Value> kv : scanner) {
           log.info("{} {}", kv.getKey().toStringNoTruncate(),
               ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
@@ -280,7 +280,7 @@
       }
 
       Future<Boolean> future = executor.submit(() -> {
-        clientMaster.replicationOperations().drain(masterTable, filesNeedingReplication);
+        clientManager.replicationOperations().drain(managerTable, filesNeedingReplication);
         log.info("Drain completed");
         return true;
       });
@@ -297,7 +297,7 @@
 
       log.info("");
       log.info("Fetching metadata records:");
-      try (var scanner = clientMaster.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+      try (var scanner = clientManager.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
         for (Entry<Key,Value> kv : scanner) {
           if (ReplicationSection.COLF.equals(kv.getKey().getColumnFamily())) {
             log.info("{} {}", kv.getKey().toStringNoTruncate(),
@@ -310,30 +310,30 @@
 
       log.info("");
       log.info("Fetching replication records:");
-      try (var scanner = ReplicationTable.getScanner(clientMaster)) {
+      try (var scanner = ReplicationTable.getScanner(clientManager)) {
         for (Entry<Key,Value> kv : scanner) {
           log.info("{} {}", kv.getKey().toStringNoTruncate(),
               ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
         }
       }
 
-      try (Scanner master = clientMaster.createScanner(masterTable, Authorizations.EMPTY);
+      try (Scanner manager = clientManager.createScanner(managerTable, Authorizations.EMPTY);
           Scanner peer = clientPeer.createScanner(peerTable, Authorizations.EMPTY)) {
-        Iterator<Entry<Key,Value>> masterIter = master.iterator(), peerIter = peer.iterator();
-        Entry<Key,Value> masterEntry = null, peerEntry = null;
-        while (masterIter.hasNext() && peerIter.hasNext()) {
-          masterEntry = masterIter.next();
+        Iterator<Entry<Key,Value>> managerIter = manager.iterator(), peerIter = peer.iterator();
+        Entry<Key,Value> managerEntry = null, peerEntry = null;
+        while (managerIter.hasNext() && peerIter.hasNext()) {
+          managerEntry = managerIter.next();
           peerEntry = peerIter.next();
-          assertEquals(masterEntry.getKey() + " was not equal to " + peerEntry.getKey(), 0,
-              masterEntry.getKey().compareTo(peerEntry.getKey(),
+          assertEquals(managerEntry.getKey() + " was not equal to " + peerEntry.getKey(), 0,
+              managerEntry.getKey().compareTo(peerEntry.getKey(),
                   PartialKey.ROW_COLFAM_COLQUAL_COLVIS));
-          assertEquals(masterEntry.getValue(), peerEntry.getValue());
+          assertEquals(managerEntry.getValue(), peerEntry.getValue());
         }
 
-        log.info("Last master entry: {}", masterEntry);
+        log.info("Last manager entry: {}", managerEntry);
         log.info("Last peer entry: {}", peerEntry);
 
-        assertFalse("Had more data to read from the master", masterIter.hasNext());
+        assertFalse("Had more data to read from the manager", managerIter.hasNext());
         assertFalse("Had more data to read from the peer", peerIter.hasNext());
       }
     } finally {
@@ -354,7 +354,7 @@
 
     peer1Cluster.start();
 
-    try (AccumuloClient clientMaster = Accumulo.newClient().from(getClientProperties()).build();
+    try (AccumuloClient clientManager = Accumulo.newClient().from(getClientProperties()).build();
         AccumuloClient clientPeer =
             peer1Cluster.createAccumuloClient("root", new PasswordToken(ROOT_PASSWORD))) {
 
@@ -365,19 +365,19 @@
       clientPeer.securityOperations().createLocalUser(peerUserName,
           new PasswordToken(peerPassword));
 
-      clientMaster.instanceOperations()
+      clientManager.instanceOperations()
           .setProperty(Property.REPLICATION_PEER_USER.getKey() + peerClusterName, peerUserName);
-      clientMaster.instanceOperations()
+      clientManager.instanceOperations()
           .setProperty(Property.REPLICATION_PEER_PASSWORD.getKey() + peerClusterName, peerPassword);
 
       // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
-      clientMaster.instanceOperations().setProperty(
+      clientManager.instanceOperations().setProperty(
           Property.REPLICATION_PEERS.getKey() + peerClusterName,
           ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
               AccumuloReplicaSystem.buildConfiguration(peer1Cluster.getInstanceName(),
                   peer1Cluster.getZooKeepers())));
 
-      String masterTable1 = "master1", peerTable1 = "peer1", masterTable2 = "master2",
+      String managerTable1 = "manager1", peerTable1 = "peer1", managerTable2 = "manager2",
           peerTable2 = "peer2";
 
       // Create tables
@@ -400,57 +400,57 @@
       props1.put(Property.TABLE_REPLICATION.getKey(), "true");
       props1.put(Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId1);
 
-      clientMaster.tableOperations().create(masterTable1,
+      clientManager.tableOperations().create(managerTable1,
           new NewTableConfiguration().setProperties(props1));
-      String masterTableId1 = clientMaster.tableOperations().tableIdMap().get(masterTable1);
-      assertNotNull(masterTableId1);
+      String managerTableId1 = clientManager.tableOperations().tableIdMap().get(managerTable1);
+      assertNotNull(managerTableId1);
 
       Map<String,String> props2 = new HashMap<>();
       props2.put(Property.TABLE_REPLICATION.getKey(), "true");
       props2.put(Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId2);
 
-      clientMaster.tableOperations().create(masterTable2,
+      clientManager.tableOperations().create(managerTable2,
           new NewTableConfiguration().setProperties(props2));
-      String masterTableId2 = clientMaster.tableOperations().tableIdMap().get(masterTable2);
-      assertNotNull(masterTableId2);
+      String managerTableId2 = clientManager.tableOperations().tableIdMap().get(managerTable2);
+      assertNotNull(managerTableId2);
 
       // Wait for zookeeper updates (configuration) to propagate
       sleepUninterruptibly(3, TimeUnit.SECONDS);
 
       // Write some data to table1
-      long masterTable1Records = 0L;
-      try (BatchWriter bw = clientMaster.createBatchWriter(masterTable1)) {
+      long managerTable1Records = 0L;
+      try (BatchWriter bw = clientManager.createBatchWriter(managerTable1)) {
         for (int rows = 0; rows < 2500; rows++) {
-          Mutation m = new Mutation(masterTable1 + rows);
+          Mutation m = new Mutation(managerTable1 + rows);
           for (int cols = 0; cols < 100; cols++) {
             String value = Integer.toString(cols);
             m.put(value, "", value);
-            masterTable1Records++;
+            managerTable1Records++;
           }
           bw.addMutation(m);
         }
       }
 
       // Write some data to table2
-      long masterTable2Records = 0L;
-      try (BatchWriter bw = clientMaster.createBatchWriter(masterTable2)) {
+      long managerTable2Records = 0L;
+      try (BatchWriter bw = clientManager.createBatchWriter(managerTable2)) {
         for (int rows = 0; rows < 2500; rows++) {
-          Mutation m = new Mutation(masterTable2 + rows);
+          Mutation m = new Mutation(managerTable2 + rows);
           for (int cols = 0; cols < 100; cols++) {
             String value = Integer.toString(cols);
             m.put(value, "", value);
-            masterTable2Records++;
+            managerTable2Records++;
           }
           bw.addMutation(m);
         }
       }
 
-      log.info("Wrote all data to master cluster");
+      log.info("Wrote all data to manager cluster");
 
-      Set<String> filesFor1 = clientMaster.replicationOperations().referencedFiles(masterTable1),
-          filesFor2 = clientMaster.replicationOperations().referencedFiles(masterTable2);
+      Set<String> filesFor1 = clientManager.replicationOperations().referencedFiles(managerTable1),
+          filesFor2 = clientManager.replicationOperations().referencedFiles(managerTable2);
 
-      while (!ReplicationTable.isOnline(clientMaster)) {
+      while (!ReplicationTable.isOnline(clientManager)) {
         Thread.sleep(500);
       }
 
@@ -463,14 +463,14 @@
       log.info("Restarted the tserver");
 
       // Read the data -- the tserver is back up and running
-      Iterators.size(clientMaster.createScanner(masterTable1, Authorizations.EMPTY).iterator());
+      Iterators.size(clientManager.createScanner(managerTable1, Authorizations.EMPTY).iterator());
 
       // Wait for both tables to be replicated
-      log.info("Waiting for {} for {}", filesFor1, masterTable1);
-      clientMaster.replicationOperations().drain(masterTable1, filesFor1);
+      log.info("Waiting for {} for {}", filesFor1, managerTable1);
+      clientManager.replicationOperations().drain(managerTable1, filesFor1);
 
-      log.info("Waiting for {} for {}", filesFor2, masterTable2);
-      clientMaster.replicationOperations().drain(masterTable2, filesFor2);
+      log.info("Waiting for {} for {}", filesFor2, managerTable2);
+      clientManager.replicationOperations().drain(managerTable2, filesFor2);
 
       long countTable = 0L;
       for (int i = 0; i < 5; i++) {
@@ -478,36 +478,36 @@
         for (Entry<Key,Value> entry : clientPeer.createScanner(peerTable1, Authorizations.EMPTY)) {
           countTable++;
           assertTrue("Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " "
-              + entry.getValue(), entry.getKey().getRow().toString().startsWith(masterTable1));
+              + entry.getValue(), entry.getKey().getRow().toString().startsWith(managerTable1));
         }
 
         log.info("Found {} records in {}", countTable, peerTable1);
 
-        if (masterTable1Records != countTable) {
-          log.warn("Did not find {} expected records in {}, only found {}", masterTable1Records,
+        if (managerTable1Records != countTable) {
+          log.warn("Did not find {} expected records in {}, only found {}", managerTable1Records,
               peerTable1, countTable);
         }
       }
 
-      assertEquals(masterTable1Records, countTable);
+      assertEquals(managerTable1Records, countTable);
 
       for (int i = 0; i < 5; i++) {
         countTable = 0L;
         for (Entry<Key,Value> entry : clientPeer.createScanner(peerTable2, Authorizations.EMPTY)) {
           countTable++;
           assertTrue("Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " "
-              + entry.getValue(), entry.getKey().getRow().toString().startsWith(masterTable2));
+              + entry.getValue(), entry.getKey().getRow().toString().startsWith(managerTable2));
         }
 
         log.info("Found {} records in {}", countTable, peerTable2);
 
-        if (masterTable2Records != countTable) {
-          log.warn("Did not find {} expected records in {}, only found {}", masterTable2Records,
+        if (managerTable2Records != countTable) {
+          log.warn("Did not find {} expected records in {}, only found {}", managerTable2Records,
               peerTable2, countTable);
         }
       }
 
-      assertEquals(masterTable2Records, countTable);
+      assertEquals(managerTable2Records, countTable);
 
     } finally {
       peer1Cluster.stop();
@@ -527,7 +527,7 @@
 
     peerCluster.start();
 
-    try (AccumuloClient clientMaster = Accumulo.newClient().from(getClientProperties()).build();
+    try (AccumuloClient clientManager = Accumulo.newClient().from(getClientProperties()).build();
         AccumuloClient clientPeer =
             peerCluster.createAccumuloClient("root", new PasswordToken(ROOT_PASSWORD))) {
 
@@ -541,7 +541,7 @@
       String peerClusterName = "peer";
 
       // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
-      clientMaster.instanceOperations().setProperty(
+      clientManager.instanceOperations().setProperty(
           Property.REPLICATION_PEERS.getKey() + peerClusterName,
           ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
               AccumuloReplicaSystem.buildConfiguration(peerCluster.getInstanceName(),
@@ -549,12 +549,12 @@
 
       // Configure the credentials we should use to authenticate ourselves to the peer for
       // replication
-      clientMaster.instanceOperations()
+      clientManager.instanceOperations()
           .setProperty(Property.REPLICATION_PEER_USER.getKey() + peerClusterName, peerUserName);
-      clientMaster.instanceOperations()
+      clientManager.instanceOperations()
           .setProperty(Property.REPLICATION_PEER_PASSWORD.getKey() + peerClusterName, peerPassword);
 
-      String masterTable = "master", peerTable = "peer";
+      String managerTable = "manager", peerTable = "peer";
 
       clientPeer.tableOperations().create(peerTable);
       String peerTableId = clientPeer.tableOperations().tableIdMap().get(peerTable);
@@ -564,17 +564,17 @@
       props.put(Property.TABLE_REPLICATION.getKey(), "true");
       props.put(Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId);
 
-      clientMaster.tableOperations().create(masterTable,
+      clientManager.tableOperations().create(managerTable,
           new NewTableConfiguration().setProperties(props));
-      String masterTableId = clientMaster.tableOperations().tableIdMap().get(masterTable);
-      assertNotNull(masterTableId);
+      String managerTableId = clientManager.tableOperations().tableIdMap().get(managerTable);
+      assertNotNull(managerTableId);
 
       // Give our replication user the ability to write to the table
       clientPeer.securityOperations().grantTablePermission(peerUserName, peerTable,
           TablePermission.WRITE);
 
       // Write some data to table1
-      try (BatchWriter bw = clientMaster.createBatchWriter(masterTable)) {
+      try (BatchWriter bw = clientManager.createBatchWriter(managerTable)) {
         for (int rows = 0; rows < 5000; rows++) {
           Mutation m = new Mutation(Integer.toString(rows));
           for (int cols = 0; cols < 100; cols++) {
@@ -585,11 +585,11 @@
         }
       }
 
-      log.info("Wrote all data to master cluster");
+      log.info("Wrote all data to manager cluster");
 
-      Set<String> files = clientMaster.replicationOperations().referencedFiles(masterTable);
+      Set<String> files = clientManager.replicationOperations().referencedFiles(managerTable);
       for (String s : files) {
-        log.info("Found referenced file for {}: {}", masterTable, s);
+        log.info("Found referenced file for {}: {}", managerTable, s);
       }
 
       for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
@@ -598,31 +598,31 @@
 
       cluster.exec(TabletServer.class);
 
-      Iterators.size(clientMaster.createScanner(masterTable, Authorizations.EMPTY).iterator());
+      Iterators.size(clientManager.createScanner(managerTable, Authorizations.EMPTY).iterator());
 
-      try (var scanner = clientMaster.createScanner(ReplicationTable.NAME, Authorizations.EMPTY)) {
+      try (var scanner = clientManager.createScanner(ReplicationTable.NAME, Authorizations.EMPTY)) {
         for (Entry<Key,Value> kv : scanner) {
           log.debug("{} {}", kv.getKey().toStringNoTruncate(),
               ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
         }
       }
 
-      clientMaster.replicationOperations().drain(masterTable, files);
+      clientManager.replicationOperations().drain(managerTable, files);
 
-      try (Scanner master = clientMaster.createScanner(masterTable, Authorizations.EMPTY);
+      try (Scanner manager = clientManager.createScanner(managerTable, Authorizations.EMPTY);
           Scanner peer = clientPeer.createScanner(peerTable, Authorizations.EMPTY)) {
-        Iterator<Entry<Key,Value>> masterIter = master.iterator(), peerIter = peer.iterator();
-        assertTrue("No data in master table", masterIter.hasNext());
+        Iterator<Entry<Key,Value>> managerIter = manager.iterator(), peerIter = peer.iterator();
+        assertTrue("No data in manager table", managerIter.hasNext());
         assertTrue("No data in peer table", peerIter.hasNext());
-        while (masterIter.hasNext() && peerIter.hasNext()) {
-          Entry<Key,Value> masterEntry = masterIter.next(), peerEntry = peerIter.next();
+        while (managerIter.hasNext() && peerIter.hasNext()) {
+          Entry<Key,Value> managerEntry = managerIter.next(), peerEntry = peerIter.next();
           assertEquals(peerEntry.getKey() + " was not equal to " + peerEntry.getKey(), 0,
-              masterEntry.getKey().compareTo(peerEntry.getKey(),
+              managerEntry.getKey().compareTo(peerEntry.getKey(),
                   PartialKey.ROW_COLFAM_COLQUAL_COLVIS));
-          assertEquals(masterEntry.getValue(), peerEntry.getValue());
+          assertEquals(managerEntry.getValue(), peerEntry.getValue());
         }
 
-        assertFalse("Had more data to read from the master", masterIter.hasNext());
+        assertFalse("Had more data to read from the manager", managerIter.hasNext());
         assertFalse("Had more data to read from the peer", peerIter.hasNext());
       }
       peerCluster.stop();
@@ -642,7 +642,7 @@
 
     peer1Cluster.start();
 
-    try (AccumuloClient clientMaster = Accumulo.newClient().from(getClientProperties()).build();
+    try (AccumuloClient clientManager = Accumulo.newClient().from(getClientProperties()).build();
         AccumuloClient clientPeer =
             peer1Cluster.createAccumuloClient("root", new PasswordToken(ROOT_PASSWORD))) {
 
@@ -657,19 +657,19 @@
 
       // Configure the credentials we should use to authenticate ourselves to the peer for
       // replication
-      clientMaster.instanceOperations()
+      clientManager.instanceOperations()
           .setProperty(Property.REPLICATION_PEER_USER.getKey() + peerClusterName, peerUserName);
-      clientMaster.instanceOperations()
+      clientManager.instanceOperations()
           .setProperty(Property.REPLICATION_PEER_PASSWORD.getKey() + peerClusterName, peerPassword);
 
       // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
-      clientMaster.instanceOperations().setProperty(
+      clientManager.instanceOperations().setProperty(
           Property.REPLICATION_PEERS.getKey() + peerClusterName,
           ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
               AccumuloReplicaSystem.buildConfiguration(peer1Cluster.getInstanceName(),
                   peer1Cluster.getZooKeepers())));
 
-      String masterTable1 = "master1", peerTable1 = "peer1", masterTable2 = "master2",
+      String managerTable1 = "manager1", peerTable1 = "peer1", managerTable2 = "manager2",
           peerTable2 = "peer2";
       clientPeer.tableOperations().create(peerTable1, new NewTableConfiguration());
       String peerTableId1 = clientPeer.tableOperations().tableIdMap().get(peerTable1);
@@ -683,19 +683,19 @@
       props1.put(Property.TABLE_REPLICATION.getKey(), "true");
       props1.put(Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId1);
 
-      clientMaster.tableOperations().create(masterTable1,
+      clientManager.tableOperations().create(managerTable1,
           new NewTableConfiguration().setProperties(props1));
-      String masterTableId1 = clientMaster.tableOperations().tableIdMap().get(masterTable1);
-      assertNotNull(masterTableId1);
+      String managerTableId1 = clientManager.tableOperations().tableIdMap().get(managerTable1);
+      assertNotNull(managerTableId1);
 
       Map<String,String> props2 = new HashMap<>();
       props2.put(Property.TABLE_REPLICATION.getKey(), "true");
       props2.put(Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId2);
 
-      clientMaster.tableOperations().create(masterTable2,
+      clientManager.tableOperations().create(managerTable2,
           new NewTableConfiguration().setProperties(props2));
-      String masterTableId2 = clientMaster.tableOperations().tableIdMap().get(masterTable2);
-      assertNotNull(masterTableId2);
+      String managerTableId2 = clientManager.tableOperations().tableIdMap().get(managerTable2);
+      assertNotNull(managerTableId2);
 
       // Give our replication user the ability to write to the tables
       clientPeer.securityOperations().grantTablePermission(peerUserName, peerTable1,
@@ -707,9 +707,9 @@
       sleepUninterruptibly(3, TimeUnit.SECONDS);
 
       // Write some data to table1
-      try (BatchWriter bw = clientMaster.createBatchWriter(masterTable1)) {
+      try (BatchWriter bw = clientManager.createBatchWriter(managerTable1)) {
         for (int rows = 0; rows < 2500; rows++) {
-          Mutation m = new Mutation(masterTable1 + rows);
+          Mutation m = new Mutation(managerTable1 + rows);
           for (int cols = 0; cols < 100; cols++) {
             String value = Integer.toString(cols);
             m.put(value, "", value);
@@ -719,9 +719,9 @@
       }
 
       // Write some data to table2
-      try (BatchWriter bw = clientMaster.createBatchWriter(masterTable2)) {
+      try (BatchWriter bw = clientManager.createBatchWriter(managerTable2)) {
         for (int rows = 0; rows < 2500; rows++) {
-          Mutation m = new Mutation(masterTable2 + rows);
+          Mutation m = new Mutation(managerTable2 + rows);
           for (int cols = 0; cols < 100; cols++) {
             String value = Integer.toString(cols);
             m.put(value, "", value);
@@ -730,9 +730,9 @@
         }
       }
 
-      log.info("Wrote all data to master cluster");
+      log.info("Wrote all data to manager cluster");
 
-      while (!ReplicationTable.isOnline(clientMaster)) {
+      while (!ReplicationTable.isOnline(clientManager)) {
         Thread.sleep(500);
       }
 
@@ -747,7 +747,7 @@
       for (int i = 0; i < 10 && !fullyReplicated; i++) {
         sleepUninterruptibly(timeoutFactor * 2, TimeUnit.SECONDS);
 
-        try (Scanner s = ReplicationTable.getScanner(clientMaster)) {
+        try (Scanner s = ReplicationTable.getScanner(clientManager)) {
           WorkSection.limit(s);
           for (Entry<Key,Value> entry : s) {
             Status status = Status.parseFrom(entry.getValue().get());
@@ -768,7 +768,7 @@
         for (Entry<Key,Value> entry : clientPeer.createScanner(peerTable1, Authorizations.EMPTY)) {
           countTable++;
           assertTrue("Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " "
-              + entry.getValue(), entry.getKey().getRow().toString().startsWith(masterTable1));
+              + entry.getValue(), entry.getKey().getRow().toString().startsWith(managerTable1));
         }
         log.info("Found {} records in {}", countTable, peerTable1);
         if (countTable > 0) {
@@ -784,7 +784,7 @@
         for (Entry<Key,Value> entry : clientPeer.createScanner(peerTable2, Authorizations.EMPTY)) {
           countTable++;
           assertTrue("Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " "
-              + entry.getValue(), entry.getKey().getRow().toString().startsWith(masterTable2));
+              + entry.getValue(), entry.getKey().getRow().toString().startsWith(managerTable2));
         }
 
         log.info("Found {} records in {}", countTable, peerTable2);
diff --git a/test/src/main/resources/hadoop-metrics2-accumulo.properties b/test/src/main/resources/hadoop-metrics2-accumulo.properties
index a7a99ac..416a0f7 100644
--- a/test/src/main/resources/hadoop-metrics2-accumulo.properties
+++ b/test/src/main/resources/hadoop-metrics2-accumulo.properties
@@ -45,13 +45,13 @@
 # accumulo.sink.file-tserver.context=tserver
 # accumulo.sink.file-tserver.filename=tserver.metrics
 
-# File sink for master metrics
-accumulo.sink.file-master.class=org.apache.hadoop.metrics2.sink.FileSink
-accumulo.sink.file-master.context=master
-accumulo.sink.file-master.filename=./target/master.metrics
-accumulo.sink.file-master.period=5
+# File sink for manager metrics
+accumulo.sink.file-manager.class=org.apache.hadoop.metrics2.sink.FileSink
+accumulo.sink.file-manager.context=manager
+accumulo.sink.file-manager.filename=./target/manager.metrics
+accumulo.sink.file-manager.period=5
 
-accumulo.jmx.master.context=master
+accumulo.jmx.manager.context=manager
 
 # File sink for thrift server metrics
 # accumulo.sink.file-thrift.class=org.apache.hadoop.metrics2.sink.FileSink
diff --git a/test/src/main/resources/log4j2-test.properties b/test/src/main/resources/log4j2-test.properties
index 5343437..35891b8 100644
--- a/test/src/main/resources/log4j2-test.properties
+++ b/test/src/main/resources/log4j2-test.properties
@@ -129,7 +129,7 @@
 logger.33.level = info
 
 # For debugging replication tests.
-logger.34.name = org.apache.accumulo.master.MasterDrainImpl
+logger.34.name = org.apache.accumulo.manager.ManagerDrainImpl
 logger.34.level = trace
 
 # This is spammy