You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by br...@apache.org on 2021/01/26 13:40:22 UTC

[accumulo] branch main updated: Deprecate/replace 'master.*' properties with 'manager.*'. (fixes #1640) (#1873)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 300c933  Deprecate/replace 'master.*' properties with 'manager.*'. (fixes #1640) (#1873)
300c933 is described below

commit 300c933baa5800fd3bdcee2793dd416aeb26a9e6
Author: Brian Loss <br...@apache.org>
AuthorDate: Tue Jan 26 08:40:13 2021 -0500

    Deprecate/replace 'master.*' properties with 'manager.*'. (fixes #1640) (#1873)
    
    This change fixes #1640 by updating the 'master.*' properties to
    'manager.*' while allowing the user to continue to use the deprecated
    'master.*' names in the site configuration and in zookeeper (system
    properties only--not table or namespace properties). The user is warned
    about usages of the deprecated properties.
    
    * Mark Property.MASTER_PREFIX as deprecated and update its description
      (which is used in generated documentation) to indicate that users
      should switch to the replacement 'manager.*' properties instead.
      This is a tradeoff vs leaving all of the Property.MASTER_* properties
      there and deprecated. This would lead to a huge list of properties in
      the generated documentation and in the end that would make the docs
      less readable.
    * Add Property.MANAGER_PREFIX and rename all other Property.MASTER_*
      properties to Property.MANAGER_*.
    * Introduce DeprecatedPropertyUtil to have a central place for
      determining the replacement name to use for a deprecated property.
    * When system properties are set or removed in zookeeper (either via
      the 'config' shell command or the API) rename 'master.*' properties
      to 'manager.*' before executing the command. In this case, a warning
      is emitted on both the client and server side indicating that the
      replacement property was used instead and the user should switch to
      using the new names.
    * Rename all 'master.*' properties to 'manager.*' when the site
      configuration is read. A warning is emitted when deprecated property
      names are found in the site configuration. If both deprecated and
      replacement names are found in the site configuration, then an
      exception is thrown to fail the current process.
    * Upon Accumulo upgrade (the 9 to 10 upgrade), rename in zookeeper all
      system properties with 'master.*' to 'manager.*'
    
    Co-authored-by: Christopher Tubbs <ct...@apache.org>
---
 .../core/clientImpl/InstanceOperationsImpl.java    |  16 ++-
 .../accumulo/core/conf/DeprecatedPropertyUtil.java | 118 +++++++++++++++++++++
 .../org/apache/accumulo/core/conf/Property.java    |  85 ++++++++-------
 .../accumulo/core/conf/SiteConfiguration.java      |  11 +-
 .../accumulo/core/util/threads/ThreadPools.java    |  10 +-
 .../main/java/org/apache/accumulo/fate/Fate.java   |   4 +-
 .../accumulo/core/conf/ConfigSanityCheckTest.java  |  16 +--
 .../core/conf/DefaultConfigurationTest.java        |   8 +-
 .../core/conf/DeprecatedPropertyUtilTest.java      | 100 +++++++++++++++++
 .../miniclusterImpl/MiniAccumuloConfigImpl.java    |   4 +-
 .../minicluster/MiniAccumuloClusterTest.java       |   2 +-
 .../server/master/recovery/HadoopLogCloser.java    |   2 +-
 .../accumulo/server/util/SystemPropUtil.java       |  18 ++++
 .../server/conf/ZooCachePropertyAccessorTest.java  |   2 +-
 .../accumulo/server/util/TServerUtilsTest.java     |   4 +-
 .../java/org/apache/accumulo/master/Master.java    |  30 +++---
 .../master/MasterClientServiceHandler.java         |  11 +-
 .../master/metrics/MasterMetricsFactory.java       |   4 +-
 .../accumulo/master/recovery/RecoveryManager.java  |   5 +-
 .../master/replication/ReplicationDriver.java      |   2 +-
 .../master/tableOps/bulkVer1/BulkImport.java       |   4 +-
 .../master/tableOps/bulkVer1/LoadFiles.java        |  10 +-
 .../master/tableOps/bulkVer2/BulkImportMove.java   |   2 +-
 .../master/tableOps/bulkVer2/LoadFiles.java        |   2 +-
 .../tableOps/tableImport/MoveExportedFiles.java    |   2 +-
 .../accumulo/master/upgrade/Upgrader9to10.java     |  26 +++++
 .../accumulo/tserver/UnloadTabletHandler.java      |   2 +-
 .../accumulo/test/DeprecatedPropertyUtilIT.java    | 104 ++++++++++++++++++
 .../test/MasterRepairsDualAssignmentIT.java        |   2 +-
 .../test/MissingWalHeaderCompletesRecoveryIT.java  |   2 +-
 .../ThriftServerBindsBeforeZooKeeperLockIT.java    |   2 +-
 .../test/functional/ManyWriteAheadLogsIT.java      |   2 +-
 .../accumulo/test/functional/MasterMetricsIT.java  |   4 +-
 .../accumulo/test/functional/RestartStressIT.java  |   2 +-
 .../accumulo/test/functional/WriteAheadLogIT.java  |   2 +-
 .../test/replication/CyclicReplicationIT.java      |   4 +-
 ...GarbageCollectorCommunicatesWithTServersIT.java |   2 +-
 .../test/replication/KerberosReplicationIT.java    |   2 +-
 .../replication/MultiInstanceReplicationIT.java    |   2 +-
 .../accumulo/test/replication/ReplicationIT.java   |   2 +-
 .../UnorderedWorkAssignerReplicationIT.java        |   2 +-
 .../upgrade/DeprecatedPropertyUpgrade9to10IT.java  |  67 ++++++++++++
 42 files changed, 585 insertions(+), 116 deletions(-)

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 6dd3e08..5e1d8a3 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
@@ -40,6 +40,7 @@ import org.apache.accumulo.core.client.admin.ActiveScan;
 import org.apache.accumulo.core.client.admin.InstanceOperations;
 import org.apache.accumulo.core.clientImpl.thrift.ConfigurationType;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.conf.DeprecatedPropertyUtil;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
 import org.apache.accumulo.core.trace.TraceUtil;
 import org.apache.accumulo.core.util.AddressUtil;
@@ -67,6 +68,12 @@ public class InstanceOperationsImpl implements InstanceOperations {
       throws AccumuloException, AccumuloSecurityException, IllegalArgumentException {
     checkArgument(property != null, "property is null");
     checkArgument(value != null, "value is null");
+    DeprecatedPropertyUtil.getReplacementName(property, (log, replacement) -> {
+      // force a warning on the client side, but send the name the user used to the server-side
+      // to trigger a warning in the server logs, and to handle it there
+      log.warn("{} was deprecated and will be removed in a future release;"
+          + " setting its replacement {} instead", property, replacement);
+    });
     MasterClient.executeVoid(context, client -> client.setSystemProperty(TraceUtil.traceInfo(),
         context.rpcCreds(), property, value));
     checkLocalityGroups(property);
@@ -76,12 +83,19 @@ public class InstanceOperationsImpl implements InstanceOperations {
   public void removeProperty(final String property)
       throws AccumuloException, AccumuloSecurityException {
     checkArgument(property != null, "property is null");
+    DeprecatedPropertyUtil.getReplacementName(property, (log, replacement) -> {
+      // force a warning on the client side, but send the name the user used to the server-side
+      // to trigger a warning in the server logs, and to handle it there
+      log.warn("{} was deprecated and will be removed in a future release; assuming user meant"
+          + " its replacement {} and will remove that instead", property, replacement);
+    });
     MasterClient.executeVoid(context,
         client -> client.removeSystemProperty(TraceUtil.traceInfo(), context.rpcCreds(), property));
     checkLocalityGroups(property);
   }
 
-  void checkLocalityGroups(String propChanged) throws AccumuloSecurityException, AccumuloException {
+  private void checkLocalityGroups(String propChanged)
+      throws AccumuloSecurityException, AccumuloException {
     if (LocalityGroupUtil.isLocalityGroupProperty(propChanged)) {
       try {
         LocalityGroupUtil.checkLocalityGroups(getSystemConfiguration().entrySet());
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/DeprecatedPropertyUtil.java b/core/src/main/java/org/apache/accumulo/core/conf/DeprecatedPropertyUtil.java
new file mode 100644
index 0000000..87e467a
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/conf/DeprecatedPropertyUtil.java
@@ -0,0 +1,118 @@
+/*
+ * 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.core.conf;
+
+import static java.util.Objects.requireNonNull;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Spliterator;
+import java.util.Spliterators;
+import java.util.function.BiConsumer;
+import java.util.function.Predicate;
+import java.util.function.UnaryOperator;
+import java.util.stream.StreamSupport;
+
+import org.apache.commons.configuration2.AbstractConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DeprecatedPropertyUtil {
+
+  public static class PropertyRenamer {
+    final Predicate<String> keyFilter;
+    final UnaryOperator<String> keyMapper;
+
+    public PropertyRenamer(Predicate<String> keyFilter, UnaryOperator<String> keyMapper) {
+      this.keyFilter = requireNonNull(keyFilter);
+      this.keyMapper = requireNonNull(keyMapper);
+    }
+
+    public static PropertyRenamer renamePrefix(String oldPrefix, String newPrefix) {
+      return new PropertyRenamer(p -> p.startsWith(oldPrefix),
+          p -> newPrefix + p.substring(oldPrefix.length()));
+    }
+  }
+
+  private static final Logger log = LoggerFactory.getLogger(DeprecatedPropertyUtil.class);
+
+  @SuppressWarnings("deprecation")
+  public static final PropertyRenamer MASTER_MANAGER_RENAMER = PropertyRenamer
+      .renamePrefix(Property.MASTER_PREFIX.getKey(), Property.MANAGER_PREFIX.getKey());
+
+  /**
+   * Ordered list of renamers
+   */
+  protected static final List<PropertyRenamer> renamers =
+      new ArrayList<>(List.of(MASTER_MANAGER_RENAMER));
+
+  /**
+   * Checks if {@code propertyName} is a deprecated property name and return its replacement name,
+   * if one is available, or the original name if no replacement is available. If a property has a
+   * replacement that itself was replaced, this method will return the final recommended property,
+   * after processing each replacement in order. If the final name has changed from the original
+   * name, the logging action is triggered with a provided logger, the original name, and the
+   * replacement name.
+   * <p>
+   * This is expected to be used only with system properties stored in the SiteConfiguration and
+   * ZooConfiguration, and not for per-table or per-namespace configuration in ZooKeeper.
+   *
+   * @param propertyName
+   *          the name of the potentially deprecated property to check for a replacement name
+   * @param loggerActionOnReplace
+   *          the action to execute, if not null, if a replacement name was found
+   * @return either the replacement for {@code propertyName}, or {@code propertyName} if the
+   *         property is not deprecated
+   */
+  public static String getReplacementName(final String propertyName,
+      BiConsumer<Logger,String> loggerActionOnReplace) {
+    String replacement = requireNonNull(propertyName);
+    requireNonNull(loggerActionOnReplace);
+    for (PropertyRenamer renamer : renamers) {
+      if (renamer.keyFilter.test(replacement)) {
+        replacement = renamer.keyMapper.apply(replacement);
+      }
+    }
+    // perform the logger action if the property was replaced
+    if (!replacement.equals(propertyName)) {
+      loggerActionOnReplace.accept(log, replacement);
+    }
+    return replacement;
+  }
+
+  /**
+   * Ensures that for any deprecated properties, both the deprecated and replacement property name
+   * are not both used in {@code config}.
+   *
+   * @param config
+   *          the configuration to check for invalid use of deprecated and replacement properties
+   */
+  static void sanityCheckManagerProperties(AbstractConfiguration config) {
+    boolean foundMasterPrefix = StreamSupport
+        .stream(Spliterators.spliteratorUnknownSize(config.getKeys(), Spliterator.ORDERED), false)
+        .anyMatch(MASTER_MANAGER_RENAMER.keyFilter);
+    boolean foundManagerPrefix = StreamSupport
+        .stream(Spliterators.spliteratorUnknownSize(config.getKeys(), Spliterator.ORDERED), false)
+        .anyMatch(k -> k.startsWith(Property.MANAGER_PREFIX.getKey()));
+    if (foundMasterPrefix && foundManagerPrefix) {
+      throw new IllegalStateException("Found both old 'master.*' and new 'manager.*' "
+          + "naming conventions in the same startup configuration");
+    }
+  }
+}
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index bbc956d..9525f34 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -245,80 +245,88 @@ public enum Property {
       PropertyType.TIMEDURATION,
       "The maximum amount of time that a Scanner should wait before retrying a failed RPC"),
 
-  // properties that are specific to master server behavior
+  // properties that are specific to manager server behavior
+  MANAGER_PREFIX("manager.", null, PropertyType.PREFIX,
+      "Properties in this category affect the behavior of the manager server."),
+  @Deprecated(since = "2.1.0")
+  @ReplacedBy(property = Property.MANAGER_PREFIX)
   MASTER_PREFIX("master.", null, PropertyType.PREFIX,
-      "Properties in this category affect the behavior of the master server"),
-  MASTER_CLIENTPORT("master.port.client", "9999", PropertyType.PORT,
-      "The port used for handling client connections on the master"),
-  MASTER_TABLET_BALANCER("master.tablet.balancer",
+      "Properties in this category affect the behavior of the manager (formerly named master) server. "
+          + "Since 2.1.0, all properties in this category are deprecated and replaced with corresponding "
+          + "`manager.*` properties. The old `master.*` names can still be used until at release 3.0, but a warning "
+          + "will be emitted. Configuration files should be updated to use the new property names."),
+  MANAGER_CLIENTPORT("manager.port.client", "9999", PropertyType.PORT,
+      "The port used for handling client connections on the manager"),
+  MANAGER_TABLET_BALANCER("manager.tablet.balancer",
       "org.apache.accumulo.server.master.balancer.TableLoadBalancer", PropertyType.CLASSNAME,
       "The balancer class that accumulo will use to make tablet assignment and "
           + "migration decisions."),
-  MASTER_BULK_RETRIES("master.bulk.retries", "3", PropertyType.COUNT,
+  MANAGER_BULK_RETRIES("manager.bulk.retries", "3", PropertyType.COUNT,
       "The number of attempts to bulk import a RFile before giving up."),
-  MASTER_BULK_THREADPOOL_SIZE("master.bulk.threadpool.size", "5", PropertyType.COUNT,
+  MANAGER_BULK_THREADPOOL_SIZE("manager.bulk.threadpool.size", "5", PropertyType.COUNT,
       "The number of threads to use when coordinating a bulk import."),
-  MASTER_BULK_THREADPOOL_TIMEOUT("master.bulk.threadpool.timeout", "0s", PropertyType.TIMEDURATION,
+  MANAGER_BULK_THREADPOOL_TIMEOUT("manager.bulk.threadpool.timeout", "0s",
+      PropertyType.TIMEDURATION,
       "The time after which bulk import threads terminate with no work available.  Zero (0) will keep the threads alive indefinitely."),
-  MASTER_BULK_TIMEOUT("master.bulk.timeout", "5m", PropertyType.TIMEDURATION,
+  MANAGER_BULK_TIMEOUT("manager.bulk.timeout", "5m", PropertyType.TIMEDURATION,
       "The time to wait for a tablet server to process a bulk import request"),
-  MASTER_RENAME_THREADS("master.rename.threadpool.size", "20", PropertyType.COUNT,
+  MANAGER_RENAME_THREADS("manager.rename.threadpool.size", "20", PropertyType.COUNT,
       "The number of threads to use when renaming user files during table import or bulk ingest."),
   @Deprecated(since = "2.1.0")
-  @ReplacedBy(property = MASTER_RENAME_THREADS)
-  MASTER_BULK_RENAME_THREADS("master.bulk.rename.threadpool.size", "20", PropertyType.COUNT,
+  @ReplacedBy(property = MANAGER_RENAME_THREADS)
+  MANAGER_BULK_RENAME_THREADS("manager.bulk.rename.threadpool.size", "20", PropertyType.COUNT,
       "This property is deprecated since 2.1.0. The number of threads to use when moving user files to bulk ingest "
           + "directories under accumulo control"),
-  MASTER_BULK_TSERVER_REGEX("master.bulk.tserver.regex", "", PropertyType.STRING,
+  MANAGER_BULK_TSERVER_REGEX("manager.bulk.tserver.regex", "", PropertyType.STRING,
       "Regular expression that defines the set of Tablet Servers that will perform bulk imports"),
-  MASTER_MINTHREADS("master.server.threads.minimum", "20", PropertyType.COUNT,
+  MANAGER_MINTHREADS("manager.server.threads.minimum", "20", PropertyType.COUNT,
       "The minimum number of threads to use to handle incoming requests."),
-  MASTER_MINTHREADS_TIMEOUT("master.server.threads.timeout", "0s", PropertyType.TIMEDURATION,
+  MANAGER_MINTHREADS_TIMEOUT("manager.server.threads.timeout", "0s", PropertyType.TIMEDURATION,
       "The time after which incoming request threads terminate with no work available.  Zero (0) will keep the threads alive indefinitely."),
-  MASTER_THREADCHECK("master.server.threadcheck.time", "1s", PropertyType.TIMEDURATION,
+  MANAGER_THREADCHECK("manager.server.threadcheck.time", "1s", PropertyType.TIMEDURATION,
       "The time between adjustments of the server thread pool."),
-  MASTER_RECOVERY_DELAY("master.recovery.delay", "10s", PropertyType.TIMEDURATION,
+  MANAGER_RECOVERY_DELAY("manager.recovery.delay", "10s", PropertyType.TIMEDURATION,
       "When a tablet server's lock is deleted, it takes time for it to "
           + "completely quit. This delay gives it time before log recoveries begin."),
-  MASTER_LEASE_RECOVERY_WAITING_PERIOD("master.lease.recovery.interval", "5s",
+  MANAGER_LEASE_RECOVERY_WAITING_PERIOD("manager.lease.recovery.interval", "5s",
       PropertyType.TIMEDURATION,
       "The amount of time to wait after requesting a write-ahead log to be recovered"),
-  MASTER_WALOG_CLOSER_IMPLEMETATION("master.walog.closer.implementation",
+  MANAGER_WALOG_CLOSER_IMPLEMETATION("manager.walog.closer.implementation",
       "org.apache.accumulo.server.master.recovery.HadoopLogCloser", PropertyType.CLASSNAME,
       "A class that implements a mechanism to steal write access to a write-ahead log"),
-  MASTER_FATE_METRICS_ENABLED("master.fate.metrics.enabled", "true", PropertyType.BOOLEAN,
+  MANAGER_FATE_METRICS_ENABLED("manager.fate.metrics.enabled", "true", PropertyType.BOOLEAN,
       "Enable reporting of FATE metrics in JMX (and logging with Hadoop Metrics2"),
-  MASTER_FATE_METRICS_MIN_UPDATE_INTERVAL("master.fate.metrics.min.update.interval", "60s",
+  MANAGER_FATE_METRICS_MIN_UPDATE_INTERVAL("manager.fate.metrics.min.update.interval", "60s",
       PropertyType.TIMEDURATION, "Limit calls from metric sinks to zookeeper to update interval"),
-  MASTER_FATE_THREADPOOL_SIZE("master.fate.threadpool.size", "4", PropertyType.COUNT,
+  MANAGER_FATE_THREADPOOL_SIZE("manager.fate.threadpool.size", "4", PropertyType.COUNT,
       "The number of threads used to run fault-tolerant executions (FATE)."
           + " These are primarily table operations like merge."),
-  MASTER_REPLICATION_SCAN_INTERVAL("master.replication.status.scan.interval", "30s",
+  MANAGER_REPLICATION_SCAN_INTERVAL("manager.replication.status.scan.interval", "30s",
       PropertyType.TIMEDURATION,
       "Amount of time to sleep before scanning the status section of the "
           + "replication table for new data"),
-  MASTER_REPLICATION_COORDINATOR_PORT("master.replication.coordinator.port", "10001",
+  MANAGER_REPLICATION_COORDINATOR_PORT("manager.replication.coordinator.port", "10001",
       PropertyType.PORT, "Port for the replication coordinator service"),
-  MASTER_REPLICATION_COORDINATOR_MINTHREADS("master.replication.coordinator.minthreads", "4",
+  MANAGER_REPLICATION_COORDINATOR_MINTHREADS("manager.replication.coordinator.minthreads", "4",
       PropertyType.COUNT, "Minimum number of threads dedicated to answering coordinator requests"),
-  MASTER_REPLICATION_COORDINATOR_THREADCHECK("master.replication.coordinator.threadcheck.time",
+  MANAGER_REPLICATION_COORDINATOR_THREADCHECK("manager.replication.coordinator.threadcheck.time",
       "5s", PropertyType.TIMEDURATION,
       "The time between adjustments of the coordinator thread pool"),
-  MASTER_STATUS_THREAD_POOL_SIZE("master.status.threadpool.size", "0", PropertyType.COUNT,
+  MANAGER_STATUS_THREAD_POOL_SIZE("manager.status.threadpool.size", "0", PropertyType.COUNT,
       "The number of threads to use when fetching the tablet server status for balancing.  Zero "
           + "indicates an unlimited number of threads will be used."),
-  MASTER_METADATA_SUSPENDABLE("master.metadata.suspendable", "false", PropertyType.BOOLEAN,
+  MANAGER_METADATA_SUSPENDABLE("manager.metadata.suspendable", "false", PropertyType.BOOLEAN,
       "Allow tablets for the " + MetadataTable.NAME
           + " table to be suspended via table.suspend.duration."),
-  MASTER_STARTUP_TSERVER_AVAIL_MIN_COUNT("master.startup.tserver.avail.min.count", "0",
+  MANAGER_STARTUP_TSERVER_AVAIL_MIN_COUNT("manager.startup.tserver.avail.min.count", "0",
       PropertyType.COUNT,
-      "Minimum number of tservers that need to be registered before master will "
-          + "start tablet assignment - checked at master initialization, when master gets lock. "
+      "Minimum number of tservers that need to be registered before manager will "
+          + "start tablet assignment - checked at manager initialization, when manager gets lock. "
           + " When set to 0 or less, no blocking occurs. Default is 0 (disabled) to keep original "
           + " behaviour. Added with version 1.10"),
-  MASTER_STARTUP_TSERVER_AVAIL_MAX_WAIT("master.startup.tserver.avail.max.wait", "0",
+  MANAGER_STARTUP_TSERVER_AVAIL_MAX_WAIT("manager.startup.tserver.avail.max.wait", "0",
       PropertyType.TIMEDURATION,
-      "Maximum time master will wait for tserver available threshold "
+      "Maximum time manager will wait for tserver available threshold "
           + "to be reached before continuing. When set to 0 or less, will block "
           + "indefinitely. Default is 0 to block indefinitely. Only valid when tserver available "
           + "threshold is set greater than 0. Added with version 1.10"),
@@ -507,11 +515,11 @@ public enum Property {
           + " minor compacted file because it may have been modified by iterators. The"
           + " file dumped to the local dir is an exact copy of what was in memory."),
   TSERV_BULK_PROCESS_THREADS("tserver.bulk.process.threads", "1", PropertyType.COUNT,
-      "The master will task a tablet server with pre-processing a bulk import"
+      "The manager will task a tablet server with pre-processing a bulk import"
           + " RFile prior to assigning it to the appropriate tablet servers. This"
           + " configuration value controls the number of threads used to process the" + " files."),
   TSERV_BULK_ASSIGNMENT_THREADS("tserver.bulk.assign.threads", "1", PropertyType.COUNT,
-      "The master delegates bulk import RFile processing and assignment to"
+      "The manager delegates bulk import RFile processing and assignment to"
           + " tablet servers. After file has been processed, the tablet server will"
           + " assign the file to the appropriate tablets on all servers. This property"
           + " controls the number of threads used to communicate to the other" + " servers."),
@@ -1001,7 +1009,7 @@ public enum Property {
       "org.apache.accumulo.master.replication.UnorderedWorkAssigner", PropertyType.CLASSNAME,
       "Replication WorkAssigner implementation to use"),
   REPLICATION_DRIVER_DELAY("replication.driver.delay", "0s", PropertyType.TIMEDURATION,
-      "Amount of time to wait before the replication work loop begins in the master."),
+      "Amount of time to wait before the replication work loop begins in the manager."),
   REPLICATION_WORK_PROCESSOR_DELAY("replication.work.processor.delay", "0s",
       PropertyType.TIMEDURATION,
       "Amount of time to wait before first checking for replication work, not"
@@ -1073,7 +1081,7 @@ public enum Property {
   private Property replacedBy = null;
   private PropertyType type;
 
-  private Property(String name, String defaultValue, PropertyType type, String description) {
+  Property(String name, String defaultValue, PropertyType type, String description) {
     this.key = name;
     this.defaultValue = defaultValue;
     this.description = description;
@@ -1310,7 +1318,7 @@ public enum Property {
 
   private static final EnumSet<Property> fixedProperties =
       EnumSet.of(Property.TSERV_CLIENTPORT, Property.TSERV_NATIVEMAP_ENABLED,
-          Property.TSERV_SCAN_MAX_OPENFILES, Property.MASTER_CLIENTPORT, Property.GC_PORT);
+          Property.TSERV_SCAN_MAX_OPENFILES, Property.MANAGER_CLIENTPORT, Property.GC_PORT);
 
   /**
    * Checks if the given property may be changed via Zookeeper, but not recognized until the restart
@@ -1335,6 +1343,7 @@ public enum Property {
     // white list prefixes
     return key.startsWith(Property.TABLE_PREFIX.getKey())
         || key.startsWith(Property.TSERV_PREFIX.getKey())
+        || key.startsWith(Property.MANAGER_PREFIX.getKey())
         || key.startsWith(Property.MASTER_PREFIX.getKey())
         || key.startsWith(Property.GC_PREFIX.getKey())
         || key.startsWith(Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey())
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java b/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java
index 81f5746..35bda16 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java
@@ -167,8 +167,17 @@ public class SiteConfiguration extends AccumuloConfiguration {
       config.addConfiguration(overrideConfig);
       config.addConfiguration(propsFileConfig);
 
+      // Make sure any deprecated property names aren't using both the old and new name.
+      DeprecatedPropertyUtil.sanityCheckManagerProperties(config);
+
       var result = new HashMap<String,String>();
-      config.getKeys().forEachRemaining(k -> result.put(k, config.getString(k)));
+      config.getKeys().forEachRemaining(orig -> {
+        String resolved = DeprecatedPropertyUtil.getReplacementName(orig, (log, replacement) -> {
+          log.warn("{} has been deprecated and will be removed in a future release;"
+              + " loading its replacement {} instead.", orig, replacement);
+        });
+        result.put(resolved, config.getString(orig));
+      });
       return new SiteConfiguration(Collections.unmodifiableMap(result));
     }
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java b/core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java
index cb1d389..c2b18bb 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java
@@ -51,15 +51,15 @@ public class ThreadPools {
     switch (p) {
       case GENERAL_SIMPLETIMER_THREADPOOL_SIZE:
         return createScheduledExecutorService(conf.getCount(p), "SimpleTimer", false);
-      case MASTER_BULK_THREADPOOL_SIZE:
+      case MANAGER_BULK_THREADPOOL_SIZE:
         return createFixedThreadPool(conf.getCount(p),
-            conf.getTimeInMillis(Property.MASTER_BULK_THREADPOOL_TIMEOUT), TimeUnit.MILLISECONDS,
+            conf.getTimeInMillis(Property.MANAGER_BULK_THREADPOOL_TIMEOUT), TimeUnit.MILLISECONDS,
             "bulk import", true);
-      case MASTER_RENAME_THREADS:
+      case MANAGER_RENAME_THREADS:
         return createFixedThreadPool(conf.getCount(p), "bulk move", false);
-      case MASTER_FATE_THREADPOOL_SIZE:
+      case MANAGER_FATE_THREADPOOL_SIZE:
         return createFixedThreadPool(conf.getCount(p), "Repo Runner", false);
-      case MASTER_STATUS_THREAD_POOL_SIZE:
+      case MANAGER_STATUS_THREAD_POOL_SIZE:
         int threads = conf.getCount(p);
         if (threads == 0) {
           return createThreadPool(0, Integer.MAX_VALUE, 60L, TimeUnit.SECONDS,
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 5ea36f3..85bec1e 100644
--- a/core/src/main/java/org/apache/accumulo/fate/Fate.java
+++ b/core/src/main/java/org/apache/accumulo/fate/Fate.java
@@ -226,8 +226,8 @@ public class Fate<T> {
    * Launches the specified number of worker threads.
    */
   public void startTransactionRunners(AccumuloConfiguration conf) {
-    int numThreads = conf.getCount(Property.MASTER_FATE_THREADPOOL_SIZE);
-    executor = ThreadPools.createExecutorService(conf, Property.MASTER_FATE_THREADPOOL_SIZE);
+    int numThreads = conf.getCount(Property.MANAGER_FATE_THREADPOOL_SIZE);
+    executor = ThreadPools.createExecutorService(conf, Property.MANAGER_FATE_THREADPOOL_SIZE);
     for (int i = 0; i < numThreads; i++) {
       executor.execute(new TransactionRunner());
     }
diff --git a/core/src/test/java/org/apache/accumulo/core/conf/ConfigSanityCheckTest.java b/core/src/test/java/org/apache/accumulo/core/conf/ConfigSanityCheckTest.java
index 7febba7..b662b81 100644
--- a/core/src/test/java/org/apache/accumulo/core/conf/ConfigSanityCheckTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/conf/ConfigSanityCheckTest.java
@@ -34,10 +34,10 @@ public class ConfigSanityCheckTest {
 
   @Test
   public void testPass() {
-    m.put(Property.MASTER_CLIENTPORT.getKey(), "9999");
-    m.put(Property.MASTER_TABLET_BALANCER.getKey(),
+    m.put(Property.MANAGER_CLIENTPORT.getKey(), "9999");
+    m.put(Property.MANAGER_TABLET_BALANCER.getKey(),
         "org.apache.accumulo.server.master.balancer.TableLoadBalancer");
-    m.put(Property.MASTER_BULK_RETRIES.getKey(), "3");
+    m.put(Property.MANAGER_BULK_RETRIES.getKey(), "3");
     ConfigSanityCheck.validate(m.entrySet());
   }
 
@@ -48,22 +48,22 @@ public class ConfigSanityCheckTest {
 
   @Test
   public void testPass_UnrecognizedValidProperty() {
-    m.put(Property.MASTER_CLIENTPORT.getKey(), "9999");
-    m.put(Property.MASTER_PREFIX.getKey() + "something", "abcdefg");
+    m.put(Property.MANAGER_CLIENTPORT.getKey(), "9999");
+    m.put(Property.MANAGER_PREFIX.getKey() + "something", "abcdefg");
     ConfigSanityCheck.validate(m.entrySet());
   }
 
   @Test
   public void testPass_UnrecognizedProperty() {
-    m.put(Property.MASTER_CLIENTPORT.getKey(), "9999");
+    m.put(Property.MANAGER_CLIENTPORT.getKey(), "9999");
     m.put("invalid.prefix.value", "abcdefg");
     ConfigSanityCheck.validate(m.entrySet());
   }
 
   @Test(expected = SanityCheckException.class)
   public void testFail_Prefix() {
-    m.put(Property.MASTER_CLIENTPORT.getKey(), "9999");
-    m.put(Property.MASTER_PREFIX.getKey(), "oops");
+    m.put(Property.MANAGER_CLIENTPORT.getKey(), "9999");
+    m.put(Property.MANAGER_PREFIX.getKey(), "oops");
     ConfigSanityCheck.validate(m.entrySet());
   }
 
diff --git a/core/src/test/java/org/apache/accumulo/core/conf/DefaultConfigurationTest.java b/core/src/test/java/org/apache/accumulo/core/conf/DefaultConfigurationTest.java
index 93772ee..457fd2c 100644
--- a/core/src/test/java/org/apache/accumulo/core/conf/DefaultConfigurationTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/conf/DefaultConfigurationTest.java
@@ -37,16 +37,16 @@ public class DefaultConfigurationTest {
 
   @Test
   public void testGet() {
-    assertEquals(Property.MASTER_CLIENTPORT.getDefaultValue(), c.get(Property.MASTER_CLIENTPORT));
+    assertEquals(Property.MANAGER_CLIENTPORT.getDefaultValue(), c.get(Property.MANAGER_CLIENTPORT));
   }
 
   @Test
   public void testGetProperties() {
     Map<String,String> p = new java.util.HashMap<>();
     c.getProperties(p, x -> true);
-    assertEquals(Property.MASTER_CLIENTPORT.getDefaultValue(),
-        p.get(Property.MASTER_CLIENTPORT.getKey()));
-    assertFalse(p.containsKey(Property.MASTER_PREFIX.getKey()));
+    assertEquals(Property.MANAGER_CLIENTPORT.getDefaultValue(),
+        p.get(Property.MANAGER_CLIENTPORT.getKey()));
+    assertFalse(p.containsKey(Property.MANAGER_PREFIX.getKey()));
     assertTrue(p.containsKey(Property.TSERV_DEFAULT_BLOCKSIZE.getKey()));
   }
 
diff --git a/core/src/test/java/org/apache/accumulo/core/conf/DeprecatedPropertyUtilTest.java b/core/src/test/java/org/apache/accumulo/core/conf/DeprecatedPropertyUtilTest.java
new file mode 100644
index 0000000..4f5c8aa
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/conf/DeprecatedPropertyUtilTest.java
@@ -0,0 +1,100 @@
+/*
+ * 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.core.conf;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThrows;
+
+import java.util.Arrays;
+import java.util.function.BiConsumer;
+
+import org.apache.commons.configuration2.BaseConfiguration;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+
+public class DeprecatedPropertyUtilTest {
+
+  private static class TestPropertyUtil extends DeprecatedPropertyUtil {
+    private static final String OLD_PREFIX = "old.";
+    private static final String MIDDLE_PREFIX = "middle.";
+    private static final String NEW_PREFIX = "new.";
+
+    public static void registerTestRenamer() {
+      renamers.add(PropertyRenamer.renamePrefix(OLD_PREFIX, MIDDLE_PREFIX));
+      renamers.add(PropertyRenamer.renamePrefix(MIDDLE_PREFIX, NEW_PREFIX));
+    }
+  }
+
+  private static final BiConsumer<Logger,String> NOOP = (log, replacement) -> {};
+
+  @BeforeClass
+  public static void setup() {
+    TestPropertyUtil.registerTestRenamer();
+  }
+
+  @Test
+  public void testNonDeprecatedPropertyRename() {
+    String oldProp = "some_property_name";
+    String newProp = DeprecatedPropertyUtil.getReplacementName(oldProp, NOOP);
+    assertSame(oldProp, newProp);
+  }
+
+  @Test
+  public void testDeprecatedPropertyRename() {
+    // 'middle.test' -> 'new.test'
+    String newProp = DeprecatedPropertyUtil.getReplacementName("middle.test", NOOP);
+    assertEquals("new.test", newProp);
+    // 'old.test' -> 'middle.test' -> 'new.test'
+    String newProp2 = DeprecatedPropertyUtil.getReplacementName("old.test", NOOP);
+    assertEquals("new.test", newProp2);
+  }
+
+  @Test
+  public void testMasterManagerPropertyRename() {
+    Arrays.stream(Property.values()).filter(p -> p.getType() != PropertyType.PREFIX)
+        .filter(p -> p.getKey().startsWith(Property.MANAGER_PREFIX.getKey())).forEach(p -> {
+          String oldProp =
+              "master." + p.getKey().substring(Property.MANAGER_PREFIX.getKey().length());
+          assertEquals(p.getKey(), DeprecatedPropertyUtil.getReplacementName(oldProp, NOOP));
+        });
+  }
+
+  @Test
+  public void testSanityCheckManagerProperties() {
+    var config = new BaseConfiguration();
+    config.setProperty("regular.prop1", "value");
+    config.setProperty("regular.prop2", "value");
+    assertEquals(2, config.size());
+    DeprecatedPropertyUtil.sanityCheckManagerProperties(config); // should succeed
+    config.setProperty("master.deprecatedProp", "value");
+    assertEquals(3, config.size());
+    DeprecatedPropertyUtil.sanityCheckManagerProperties(config); // should succeed
+    config.setProperty("manager.replacementProp", "value");
+    assertEquals(4, config.size());
+    assertThrows("Sanity check should fail when 'master.*' and 'manager.*' appear in same config",
+        IllegalStateException.class,
+        () -> DeprecatedPropertyUtil.sanityCheckManagerProperties(config));
+    config.clearProperty("master.deprecatedProp");
+    assertEquals(3, config.size());
+    DeprecatedPropertyUtil.sanityCheckManagerProperties(config); // should succeed
+  }
+
+}
diff --git a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java
index 7e428d5..71a711c 100644
--- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java
@@ -159,13 +159,13 @@ public class MiniAccumuloConfigImpl {
       mergeProp(generalDynamicClasspaths.getKey(), libExtDir.getAbsolutePath() + "/[^.].*[.]jar");
       mergeProp(Property.GC_CYCLE_DELAY.getKey(), "4s");
       mergeProp(Property.GC_CYCLE_START.getKey(), "0s");
-      mergePropWithRandomPort(Property.MASTER_CLIENTPORT.getKey());
+      mergePropWithRandomPort(Property.MANAGER_CLIENTPORT.getKey());
       mergePropWithRandomPort(Property.TRACE_PORT.getKey());
       mergePropWithRandomPort(Property.TSERV_CLIENTPORT.getKey());
       mergePropWithRandomPort(Property.MONITOR_PORT.getKey());
       mergePropWithRandomPort(Property.GC_PORT.getKey());
       mergePropWithRandomPort(Property.REPLICATION_RECEIPT_SERVICE_PORT.getKey());
-      mergePropWithRandomPort(Property.MASTER_REPLICATION_COORDINATOR_PORT.getKey());
+      mergePropWithRandomPort(Property.MANAGER_REPLICATION_COORDINATOR_PORT.getKey());
 
       if (isUseCredentialProvider()) {
         updateConfigForCredentialProvider();
diff --git a/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java b/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java
index 398322e..9cee84f 100644
--- a/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java
+++ b/minicluster/src/test/java/org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java
@@ -249,7 +249,7 @@ public class MiniAccumuloClusterTest {
       config.read(reader);
     }
     for (Property randomPortProp : new Property[] {Property.TSERV_CLIENTPORT, Property.MONITOR_PORT,
-        Property.MASTER_CLIENTPORT, Property.TRACE_PORT, Property.GC_PORT}) {
+        Property.MANAGER_CLIENTPORT, Property.TRACE_PORT, Property.GC_PORT}) {
       String value = config.getString(randomPortProp.getKey());
       assertNotNull("Found no value for " + randomPortProp, value);
       assertEquals("0", value);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/recovery/HadoopLogCloser.java b/server/base/src/main/java/org/apache/accumulo/server/master/recovery/HadoopLogCloser.java
index 74a4407..ef3ad46 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/recovery/HadoopLogCloser.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/recovery/HadoopLogCloser.java
@@ -57,7 +57,7 @@ public class HadoopLogCloser implements LogCloser {
       try {
         if (!dfs.recoverLease(source)) {
           log.info("Waiting for file to be closed {}", source);
-          return conf.getTimeInMillis(Property.MASTER_LEASE_RECOVERY_WAITING_PERIOD);
+          return conf.getTimeInMillis(Property.MANAGER_LEASE_RECOVERY_WAITING_PERIOD);
         }
         log.info("Recovered lease on {}", source);
       } catch (FileNotFoundException ex) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/SystemPropUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/SystemPropUtil.java
index 31df495..35ba55d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/SystemPropUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/SystemPropUtil.java
@@ -21,6 +21,7 @@ package org.apache.accumulo.server.util;
 import static java.nio.charset.StandardCharsets.UTF_8;
 
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.conf.DeprecatedPropertyUtil;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.conf.PropertyType;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
@@ -36,6 +37,14 @@ public class SystemPropUtil {
 
   public static void setSystemProperty(ServerContext context, String property, String value)
       throws KeeperException, InterruptedException {
+    // Retrieve the replacement name for this property, if there is one.
+    // Do this before we check if the name is a valid zookeeper name.
+    final var original = property;
+    property = DeprecatedPropertyUtil.getReplacementName(property, (log, replacement) -> {
+      log.warn("{} was deprecated and will be removed in a future release;"
+          + " setting its replacement {} instead", original, replacement);
+    });
+
     if (!Property.isValidZooPropertyKey(property)) {
       IllegalArgumentException iae =
           new IllegalArgumentException("Zookeeper property is not mutable: " + property);
@@ -70,6 +79,15 @@ public class SystemPropUtil {
 
   public static void removeSystemProperty(ServerContext context, String property)
       throws InterruptedException, KeeperException {
+    String resolved = DeprecatedPropertyUtil.getReplacementName(property, (log, replacement) -> {
+      log.warn("{} was deprecated and will be removed in a future release; assuming user meant"
+          + " its replacement {} and will remove that instead", property, replacement);
+    });
+    removePropWithoutDeprecationWarning(context, resolved);
+  }
+
+  public static void removePropWithoutDeprecationWarning(ServerContext context, String property)
+      throws InterruptedException, KeeperException {
     String zPath = context.getZooKeeperRoot() + Constants.ZCONFIG + "/" + property;
     context.getZooReaderWriter().recursiveDelete(zPath, NodeMissingPolicy.FAIL);
   }
diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/ZooCachePropertyAccessorTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/ZooCachePropertyAccessorTest.java
index 2cf1ef9..0517af3 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/conf/ZooCachePropertyAccessorTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/conf/ZooCachePropertyAccessorTest.java
@@ -95,7 +95,7 @@ public class ZooCachePropertyAccessorTest {
 
   @Test
   public void testGet_InvalidFormat() {
-    Property badProp = Property.MASTER_CLIENTPORT;
+    Property badProp = Property.MANAGER_CLIENTPORT;
     expect(zc.get(PATH + "/" + badProp.getKey())).andReturn(VALUE_BYTES);
     replay(zc);
     AccumuloConfiguration parent = createMock(AccumuloConfiguration.class);
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 c6ee2a1..6c69a9f 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
@@ -231,13 +231,13 @@ public class TServerUtilsTest {
 
     ports = findTwoFreeSequentialPorts(gcPort + 1);
     int masterPort = ports[0];
-    conf.set(Property.MASTER_CLIENTPORT, Integer.toString(masterPort));
+    conf.set(Property.MANAGER_CLIENTPORT, Integer.toString(masterPort));
     int monitorPort = ports[1];
     conf.set(Property.MONITOR_PORT, Integer.toString(monitorPort));
 
     ports = findTwoFreeSequentialPorts(monitorPort + 1);
     int masterReplCoordPort = ports[0];
-    conf.set(Property.MASTER_REPLICATION_COORDINATOR_PORT, Integer.toString(masterReplCoordPort));
+    conf.set(Property.MANAGER_REPLICATION_COORDINATOR_PORT, Integer.toString(masterReplCoordPort));
     int tserverFinalPort = ports[1];
 
     conf.set(Property.TSERV_PORTSEARCH, "true");
diff --git a/server/manager/src/main/java/org/apache/accumulo/master/Master.java b/server/manager/src/main/java/org/apache/accumulo/master/Master.java
index 3bf6651..cf2ec31 100644
--- a/server/manager/src/main/java/org/apache/accumulo/master/Master.java
+++ b/server/manager/src/main/java/org/apache/accumulo/master/Master.java
@@ -373,7 +373,7 @@ public class Master extends AbstractServer
         .setIdleTime(aconf.getTimeInMillis(Property.GENERAL_RPC_TIMEOUT));
     tserverSet = new LiveTServerSet(context, this);
     this.tabletBalancer = Property.createInstanceFromPropertyName(aconf,
-        Property.MASTER_TABLET_BALANCER, TabletBalancer.class, new DefaultLoadBalancer());
+        Property.MANAGER_TABLET_BALANCER, TabletBalancer.class, new DefaultLoadBalancer());
     this.tabletBalancer.init(context);
 
     this.security = AuditedSecurityOperation.getInstance(context);
@@ -881,9 +881,9 @@ public class Master extends AbstractServer
   private SortedMap<TServerInstance,TabletServerStatus>
       gatherTableInformation(Set<TServerInstance> currentServers) {
     final long rpcTimeout = getConfiguration().getTimeInMillis(Property.GENERAL_RPC_TIMEOUT);
-    int threads = getConfiguration().getCount(Property.MASTER_STATUS_THREAD_POOL_SIZE);
+    int threads = getConfiguration().getCount(Property.MANAGER_STATUS_THREAD_POOL_SIZE);
     ExecutorService tp = ThreadPools.createExecutorService(getConfiguration(),
-        Property.MASTER_STATUS_THREAD_POOL_SIZE);
+        Property.MANAGER_STATUS_THREAD_POOL_SIZE);
     long start = System.currentTimeMillis();
     final SortedMap<TServerInstance,TabletServerStatus> result = new ConcurrentSkipListMap<>();
     final RateLimiter shutdownServerRateLimiter = RateLimiter.create(MAX_SHUTDOWNS_PER_SEC);
@@ -992,9 +992,9 @@ public class Master extends AbstractServer
     ServerAddress sa;
     try {
       sa = TServerUtils.startServer(getMetricsSystem(), context, getHostname(),
-          Property.MASTER_CLIENTPORT, processor, "Master", "Master Client Service Handler", null,
-          Property.MASTER_MINTHREADS, Property.MASTER_MINTHREADS_TIMEOUT,
-          Property.MASTER_THREADCHECK, Property.GENERAL_MAX_MESSAGE_SIZE);
+          Property.MANAGER_CLIENTPORT, processor, "Master", "Master 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);
     }
@@ -1061,7 +1061,7 @@ public class Master extends AbstractServer
         // we'll want metadata tablets to
         // be immediately reassigned, even if there's a global table.suspension.duration
         // setting.
-        return getConfiguration().getBoolean(Property.MASTER_METADATA_SUSPENDABLE);
+        return getConfiguration().getBoolean(Property.MANAGER_METADATA_SUSPENDABLE);
       }
     });
 
@@ -1240,20 +1240,20 @@ public class Master extends AbstractServer
     long waitStart = System.currentTimeMillis();
 
     long minTserverCount =
-        getConfiguration().getCount(Property.MASTER_STARTUP_TSERVER_AVAIL_MIN_COUNT);
+        getConfiguration().getCount(Property.MANAGER_STARTUP_TSERVER_AVAIL_MIN_COUNT);
 
     if (minTserverCount <= 0) {
       log.info("tserver availability check disabled, continuing with-{} servers. To enable, set {}",
-          tserverSet.size(), Property.MASTER_STARTUP_TSERVER_AVAIL_MIN_COUNT.getKey());
+          tserverSet.size(), Property.MANAGER_STARTUP_TSERVER_AVAIL_MIN_COUNT.getKey());
       return;
     }
 
     long maxWait =
-        getConfiguration().getTimeInMillis(Property.MASTER_STARTUP_TSERVER_AVAIL_MAX_WAIT);
+        getConfiguration().getTimeInMillis(Property.MANAGER_STARTUP_TSERVER_AVAIL_MAX_WAIT);
 
     if (maxWait <= 0) {
       log.info("tserver availability check set to block indefinitely, To change, set {} > 0.",
-          Property.MASTER_STARTUP_TSERVER_AVAIL_MAX_WAIT.getKey());
+          Property.MANAGER_STARTUP_TSERVER_AVAIL_MAX_WAIT.getKey());
       maxWait = Long.MAX_VALUE;
     }
 
@@ -1313,10 +1313,10 @@ public class Master extends AbstractServer
     ReplicationCoordinator.Processor<ReplicationCoordinator.Iface> replicationCoordinatorProcessor =
         new ReplicationCoordinator.Processor<>(TraceUtil.wrapService(haReplicationProxy));
     ServerAddress replAddress = TServerUtils.startServer(getMetricsSystem(), context, getHostname(),
-        Property.MASTER_REPLICATION_COORDINATOR_PORT, replicationCoordinatorProcessor,
+        Property.MANAGER_REPLICATION_COORDINATOR_PORT, replicationCoordinatorProcessor,
         "Master Replication Coordinator", "Replication Coordinator", null,
-        Property.MASTER_REPLICATION_COORDINATOR_MINTHREADS, null,
-        Property.MASTER_REPLICATION_COORDINATOR_THREADCHECK, Property.GENERAL_MAX_MESSAGE_SIZE);
+        Property.MANAGER_REPLICATION_COORDINATOR_MINTHREADS, null,
+        Property.MANAGER_REPLICATION_COORDINATOR_THREADCHECK, Property.GENERAL_MAX_MESSAGE_SIZE);
 
     log.info("Started replication coordinator service at " + replAddress.address);
     // Start the daemon to scan the replication table and make units of work
@@ -1405,7 +1405,7 @@ public class Master extends AbstractServer
     log.info("trying to get master lock");
 
     final String masterClientAddress =
-        getHostname() + ":" + getConfiguration().getPort(Property.MASTER_CLIENTPORT)[0];
+        getHostname() + ":" + getConfiguration().getPort(Property.MANAGER_CLIENTPORT)[0];
 
     while (true) {
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java b/server/manager/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
index 6784e95..d9bab5b 100644
--- a/server/manager/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
+++ b/server/manager/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
@@ -49,6 +49,7 @@ import org.apache.accumulo.core.clientImpl.thrift.TableOperationExceptionType;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.clientImpl.thrift.ThriftTableOperationException;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.DeprecatedPropertyUtil;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.NamespaceId;
@@ -395,8 +396,8 @@ public class MasterClientServiceHandler extends FateServiceHandler
       String value, TableOperation op)
       throws ThriftSecurityException, ThriftTableOperationException {
 
-    NamespaceId namespaceId = null;
-    namespaceId = ClientServiceHandler.checkNamespaceId(master.getContext(), namespace, op);
+    NamespaceId namespaceId =
+        ClientServiceHandler.checkNamespaceId(master.getContext(), namespace, op);
 
     if (!master.security.canAlterNamespace(c, namespaceId))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
@@ -449,10 +450,12 @@ public class MasterClientServiceHandler extends FateServiceHandler
   }
 
   private void updatePlugins(String property) {
-    if (property.equals(Property.MASTER_TABLET_BALANCER.getKey())) {
+    // resolve without warning; any warnings should have already occurred
+    String resolved = DeprecatedPropertyUtil.getReplacementName(property, (log, replacement) -> {});
+    if (resolved.equals(Property.MANAGER_TABLET_BALANCER.getKey())) {
       AccumuloConfiguration conf = master.getConfiguration();
       TabletBalancer balancer = Property.createInstanceFromPropertyName(conf,
-          Property.MASTER_TABLET_BALANCER, TabletBalancer.class, new DefaultLoadBalancer());
+          Property.MANAGER_TABLET_BALANCER, TabletBalancer.class, new DefaultLoadBalancer());
       balancer.init(master.getContext());
       master.tabletBalancer = balancer;
       log.info("tablet balancer changed to {}", master.tabletBalancer.getClass().getName());
diff --git a/server/manager/src/main/java/org/apache/accumulo/master/metrics/MasterMetricsFactory.java b/server/manager/src/main/java/org/apache/accumulo/master/metrics/MasterMetricsFactory.java
index ed3bcf8..857e6b8 100644
--- a/server/manager/src/main/java/org/apache/accumulo/master/metrics/MasterMetricsFactory.java
+++ b/server/manager/src/main/java/org/apache/accumulo/master/metrics/MasterMetricsFactory.java
@@ -42,8 +42,8 @@ public class MasterMetricsFactory {
 
   public MasterMetricsFactory(AccumuloConfiguration conf) {
     requireNonNull(conf, "AccumuloConfiguration must not be null");
-    enableFateMetrics = conf.getBoolean(Property.MASTER_FATE_METRICS_ENABLED);
-    fateMinUpdateInterval = conf.getTimeInMillis(Property.MASTER_FATE_METRICS_MIN_UPDATE_INTERVAL);
+    enableFateMetrics = conf.getBoolean(Property.MANAGER_FATE_METRICS_ENABLED);
+    fateMinUpdateInterval = conf.getTimeInMillis(Property.MANAGER_FATE_METRICS_MIN_UPDATE_INTERVAL);
   }
 
   public int register(Master master) {
diff --git a/server/manager/src/main/java/org/apache/accumulo/master/recovery/RecoveryManager.java b/server/manager/src/main/java/org/apache/accumulo/master/recovery/RecoveryManager.java
index 60c82bb..a3b63b3 100644
--- a/server/manager/src/main/java/org/apache/accumulo/master/recovery/RecoveryManager.java
+++ b/server/manager/src/main/java/org/apache/accumulo/master/recovery/RecoveryManager.java
@@ -197,10 +197,11 @@ public class RecoveryManager {
           if (!closeTasksQueued.contains(sortId) && !sortsQueued.contains(sortId)) {
             AccumuloConfiguration aconf = master.getConfiguration();
             LogCloser closer = Property.createInstanceFromPropertyName(aconf,
-                Property.MASTER_WALOG_CLOSER_IMPLEMETATION, LogCloser.class, new HadoopLogCloser());
+                Property.MANAGER_WALOG_CLOSER_IMPLEMETATION, LogCloser.class,
+                new HadoopLogCloser());
             Long delay = recoveryDelay.get(sortId);
             if (delay == null) {
-              delay = aconf.getTimeInMillis(Property.MASTER_RECOVERY_DELAY);
+              delay = aconf.getTimeInMillis(Property.MANAGER_RECOVERY_DELAY);
             } else {
               delay = Math.min(2 * delay, 1000 * 60 * 5L);
             }
diff --git a/server/manager/src/main/java/org/apache/accumulo/master/replication/ReplicationDriver.java b/server/manager/src/main/java/org/apache/accumulo/master/replication/ReplicationDriver.java
index cb73f51..1bee7da 100644
--- a/server/manager/src/main/java/org/apache/accumulo/master/replication/ReplicationDriver.java
+++ b/server/manager/src/main/java/org/apache/accumulo/master/replication/ReplicationDriver.java
@@ -109,7 +109,7 @@ public class ReplicationDriver implements Runnable {
       }
 
       // Sleep for a bit
-      long sleepMillis = conf.getTimeInMillis(Property.MASTER_REPLICATION_SCAN_INTERVAL);
+      long sleepMillis = conf.getTimeInMillis(Property.MANAGER_REPLICATION_SCAN_INTERVAL);
       log.trace("Sleeping for {}ms before re-running", sleepMillis);
       try {
         Thread.sleep(sleepMillis);
diff --git a/server/manager/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/BulkImport.java b/server/manager/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/BulkImport.java
index 1193f82..bbb4514 100644
--- a/server/manager/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/BulkImport.java
+++ b/server/manager/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/BulkImport.java
@@ -203,8 +203,8 @@ public class BulkImport extends MasterRepo {
 
     AccumuloConfiguration serverConfig = master.getConfiguration();
     @SuppressWarnings("deprecation")
-    ExecutorService workers = ThreadPools.createExecutorService(serverConfig,
-        serverConfig.resolve(Property.MASTER_RENAME_THREADS, Property.MASTER_BULK_RENAME_THREADS));
+    ExecutorService workers = ThreadPools.createExecutorService(serverConfig, serverConfig
+        .resolve(Property.MANAGER_RENAME_THREADS, Property.MANAGER_BULK_RENAME_THREADS));
     List<Future<Exception>> results = new ArrayList<>();
 
     for (FileStatus file : mapFiles) {
diff --git a/server/manager/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/LoadFiles.java b/server/manager/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/LoadFiles.java
index 6869838..a62b79f 100644
--- a/server/manager/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/LoadFiles.java
+++ b/server/manager/src/main/java/org/apache/accumulo/master/tableOps/bulkVer1/LoadFiles.java
@@ -91,7 +91,7 @@ class LoadFiles extends MasterRepo {
   private static synchronized ExecutorService getThreadPool(Master master) {
     if (threadPool == null) {
       threadPool = ThreadPools.createExecutorService(master.getConfiguration(),
-          Property.MASTER_BULK_THREADPOOL_SIZE);
+          Property.MANAGER_BULK_THREADPOOL_SIZE);
     }
     return threadPool;
   }
@@ -121,7 +121,7 @@ class LoadFiles extends MasterRepo {
     for (FileStatus f : files)
       filesToLoad.add(f.getPath().toString());
 
-    final int RETRIES = Math.max(1, conf.getCount(Property.MASTER_BULK_RETRIES));
+    final int RETRIES = Math.max(1, conf.getCount(Property.MANAGER_BULK_RETRIES));
     for (int attempt = 0; attempt < RETRIES && !filesToLoad.isEmpty(); attempt++) {
       List<Future<Void>> results = new ArrayList<>();
 
@@ -137,7 +137,7 @@ class LoadFiles extends MasterRepo {
       final List<String> loaded = Collections.synchronizedList(new ArrayList<>());
       final Random random = new SecureRandom();
       final TServerInstance[] servers;
-      String prop = conf.get(Property.MASTER_BULK_TSERVER_REGEX);
+      String prop = conf.get(Property.MANAGER_BULK_TSERVER_REGEX);
       if (prop == null || "".equals(prop)) {
         servers = master.onlineTabletServers().toArray(new TServerInstance[0]);
       } else {
@@ -150,7 +150,7 @@ class LoadFiles extends MasterRepo {
         });
         if (subset.isEmpty()) {
           log.warn("There are no tablet servers online that match supplied regex: {}",
-              conf.get(Property.MASTER_BULK_TSERVER_REGEX));
+              conf.get(Property.MANAGER_BULK_TSERVER_REGEX));
         }
         servers = subset.toArray(new TServerInstance[0]);
       }
@@ -164,7 +164,7 @@ class LoadFiles extends MasterRepo {
               // because this is running on the master and there are lots of connections to tablet
               // servers serving the metadata tablets
               long timeInMillis =
-                  master.getConfiguration().getTimeInMillis(Property.MASTER_BULK_TIMEOUT);
+                  master.getConfiguration().getTimeInMillis(Property.MANAGER_BULK_TIMEOUT);
               server = servers[random.nextInt(servers.length)].getHostAndPort();
               client = ThriftUtil.getTServerClient(server, master.getContext(), timeInMillis);
               List<String> attempt1 = Collections.singletonList(file);
diff --git a/server/manager/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/BulkImportMove.java b/server/manager/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/BulkImportMove.java
index f10d0c0..4faa608 100644
--- a/server/manager/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/BulkImportMove.java
+++ b/server/manager/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/BulkImportMove.java
@@ -107,7 +107,7 @@ class BulkImportMove extends MasterRepo {
     AccumuloConfiguration aConf = master.getConfiguration();
     @SuppressWarnings("deprecation")
     int workerCount = aConf.getCount(
-        aConf.resolve(Property.MASTER_RENAME_THREADS, Property.MASTER_BULK_RENAME_THREADS));
+        aConf.resolve(Property.MANAGER_RENAME_THREADS, Property.MANAGER_BULK_RENAME_THREADS));
     Map<Path,Path> oldToNewMap = new HashMap<>();
     String fmtTid = FateTxId.formatTid(tid);
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/LoadFiles.java b/server/manager/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/LoadFiles.java
index 817497f..0bfb011 100644
--- a/server/manager/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/LoadFiles.java
+++ b/server/manager/src/main/java/org/apache/accumulo/master/tableOps/bulkVer2/LoadFiles.java
@@ -148,7 +148,7 @@ class LoadFiles extends MasterRepo {
     void start(Path bulkDir, Master master, long tid, boolean setTime) throws Exception {
       super.start(bulkDir, master, tid, setTime);
 
-      timeInMillis = master.getConfiguration().getTimeInMillis(Property.MASTER_BULK_TIMEOUT);
+      timeInMillis = master.getConfiguration().getTimeInMillis(Property.MANAGER_BULK_TIMEOUT);
       fmtTid = FateTxId.formatTid(tid);
 
       loadMsgs = new MapCounter<>();
diff --git a/server/manager/src/main/java/org/apache/accumulo/master/tableOps/tableImport/MoveExportedFiles.java b/server/manager/src/main/java/org/apache/accumulo/master/tableOps/tableImport/MoveExportedFiles.java
index ae99357..701ed06 100644
--- a/server/manager/src/main/java/org/apache/accumulo/master/tableOps/tableImport/MoveExportedFiles.java
+++ b/server/manager/src/main/java/org/apache/accumulo/master/tableOps/tableImport/MoveExportedFiles.java
@@ -58,7 +58,7 @@ class MoveExportedFiles extends MasterRepo {
   public Repo<Master> call(long tid, Master master) throws Exception {
     String fmtTid = FateTxId.formatTid(tid);
 
-    int workerCount = master.getConfiguration().getCount(Property.MASTER_RENAME_THREADS);
+    int workerCount = master.getConfiguration().getCount(Property.MANAGER_RENAME_THREADS);
     VolumeManager fs = master.getVolumeManager();
     Map<Path,Path> oldToNewPaths = new HashMap<>();
 
diff --git a/server/manager/src/main/java/org/apache/accumulo/master/upgrade/Upgrader9to10.java b/server/manager/src/main/java/org/apache/accumulo/master/upgrade/Upgrader9to10.java
index 07e292e..d9cff12 100644
--- a/server/manager/src/main/java/org/apache/accumulo/master/upgrade/Upgrader9to10.java
+++ b/server/manager/src/main/java/org/apache/accumulo/master/upgrade/Upgrader9to10.java
@@ -45,6 +45,8 @@ import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.TimeType;
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.conf.DeprecatedPropertyUtil;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -74,10 +76,12 @@ import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.conf.ZooConfiguration;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.gc.GcVolumeUtil;
 import org.apache.accumulo.server.metadata.RootGcCandidates;
 import org.apache.accumulo.server.metadata.TabletMutatorBase;
+import org.apache.accumulo.server.util.SystemPropUtil;
 import org.apache.accumulo.server.util.TablePropUtil;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -113,6 +117,7 @@ public class Upgrader9to10 implements Upgrader {
   public void upgradeZookeeper(ServerContext ctx) {
     setMetaTableProps(ctx);
     upgradeRootTabletMetadata(ctx);
+    renameOldMasterPropsinZK(ctx);
   }
 
   @Override
@@ -201,6 +206,27 @@ public class Upgrader9to10 implements Upgrader {
     delete(ctx, ZROOT_TABLET_PATH);
   }
 
+  @SuppressWarnings("deprecation")
+  private void renameOldMasterPropsinZK(ServerContext ctx) {
+    // Rename all of the properties only set in ZooKeeper that start with "master." to rename and
+    // store them starting with "manager." instead.
+    var zooConfiguration = new ZooConfiguration(ctx, ctx.getZooCache(), new ConfigurationCopy());
+    zooConfiguration.getAllPropertiesWithPrefix(Property.MASTER_PREFIX)
+        .forEach((original, value) -> {
+          DeprecatedPropertyUtil.getReplacementName(original, (log, replacement) -> {
+            log.info("Automatically renaming deprecated property '{}' with its replacement '{}'"
+                + " in ZooKeeper on upgrade.", original, replacement);
+            try {
+              // Set the property under the new name
+              SystemPropUtil.setSystemProperty(ctx, replacement, value);
+              SystemPropUtil.removePropWithoutDeprecationWarning(ctx, original);
+            } catch (KeeperException | InterruptedException e) {
+              throw new RuntimeException("Unable to upgrade system properties", e);
+            }
+          });
+        });
+  }
+
   private static class UpgradeMutator extends TabletMutatorBase {
 
     private ServerContext context;
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 f77bbfe..e3978f0 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
@@ -117,7 +117,7 @@ class UnloadTabletHandler implements Runnable {
       }
       if (!goalState.equals(TUnloadTabletGoal.SUSPENDED) || extent.isRootTablet()
           || (extent.isMeta()
-              && !server.getConfiguration().getBoolean(Property.MASTER_METADATA_SUSPENDABLE))) {
+              && !server.getConfiguration().getBoolean(Property.MANAGER_METADATA_SUSPENDABLE))) {
         TabletStateStore.unassign(server.getContext(), tls, null);
       } else {
         TabletStateStore.suspend(server.getContext(), tls, null,
diff --git a/test/src/main/java/org/apache/accumulo/test/DeprecatedPropertyUtilIT.java b/test/src/main/java/org/apache/accumulo/test/DeprecatedPropertyUtilIT.java
new file mode 100644
index 0000000..c168812
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/DeprecatedPropertyUtilIT.java
@@ -0,0 +1,104 @@
+/*
+ * 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.test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Map;
+
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.conf.DeprecatedPropertyUtil;
+import org.apache.accumulo.core.conf.DeprecatedPropertyUtil.PropertyRenamer;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.server.util.SystemPropUtil;
+import org.apache.accumulo.test.functional.ConfigurableMacBase;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class DeprecatedPropertyUtilIT extends ConfigurableMacBase {
+  private static final String OLD_SYSTEM_PREFIX = "old.system.custom.";
+
+  private static final PropertyRenamer TEST_SYS_RENAMER = PropertyRenamer
+      .renamePrefix(OLD_SYSTEM_PREFIX, Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey());
+
+  private static class TestPropertyUtil extends DeprecatedPropertyUtil {
+    public static void registerTestRenamer() {
+      renamers.add(TEST_SYS_RENAMER);
+    }
+
+    public static void removeTestRenamer() {
+      renamers.remove(TEST_SYS_RENAMER);
+    }
+  }
+
+  @Before
+  public void setUpRenamers() throws Exception {
+    super.setUp();
+    TestPropertyUtil.registerTestRenamer();
+  }
+
+  @After
+  public void tearDownRenamers() {
+    super.tearDown();
+    TestPropertyUtil.removeTestRenamer();
+  }
+
+  @Test
+  public void testSystemProperty() throws Exception {
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) {
+      String oldProp = OLD_SYSTEM_PREFIX + "test.prop";
+      String newProp = Property.GENERAL_ARBITRARY_PROP_PREFIX.getKey() + "test.prop";
+      String propValue = "dummy prop value";
+      Map<String,String> config = client.instanceOperations().getSystemConfiguration();
+      assertFalse(oldProp + " was in the config!", config.containsKey(newProp));
+      assertFalse(newProp + " was in the config!", config.containsKey(newProp));
+
+      // create using old prop and verify new prop was created
+      SystemPropUtil.setSystemProperty(getServerContext(), oldProp, propValue);
+      config = client.instanceOperations().getSystemConfiguration();
+      assertFalse(oldProp + " was in the config after set call!", config.containsKey(oldProp));
+      assertTrue(newProp + " was not in the config after set call!", config.containsKey(newProp));
+      assertEquals(propValue, config.get(newProp));
+
+      // remove using new prop and verify both are gone
+      SystemPropUtil.removeSystemProperty(getServerContext(), newProp);
+      config = client.instanceOperations().getSystemConfiguration();
+      assertFalse(oldProp + " was in the config after remove call!", config.containsKey(oldProp));
+      assertFalse(newProp + " was in the config after remove call!", config.containsKey(newProp));
+
+      // re-create using new prop and verify new prop was created
+      SystemPropUtil.setSystemProperty(getServerContext(), newProp, propValue);
+      config = client.instanceOperations().getSystemConfiguration();
+      assertFalse(oldProp + " was in the config after set call!", config.containsKey(oldProp));
+      assertTrue(newProp + " was not in the config after set call!", config.containsKey(newProp));
+      assertEquals(propValue, config.get(newProp));
+
+      // remove using old prop and verify both are gone
+      SystemPropUtil.removeSystemProperty(getServerContext(), oldProp);
+      config = client.instanceOperations().getSystemConfiguration();
+      assertFalse(oldProp + " was in the config after remove call!", config.containsKey(oldProp));
+      assertFalse(newProp + " was in the config after remove call!", config.containsKey(newProp));
+    }
+  }
+
+}
diff --git a/test/src/main/java/org/apache/accumulo/test/MasterRepairsDualAssignmentIT.java b/test/src/main/java/org/apache/accumulo/test/MasterRepairsDualAssignmentIT.java
index b1a95da..22a23d0 100644
--- a/test/src/main/java/org/apache/accumulo/test/MasterRepairsDualAssignmentIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/MasterRepairsDualAssignmentIT.java
@@ -64,7 +64,7 @@ public class MasterRepairsDualAssignmentIT extends ConfigurableMacBase {
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
-    cfg.setProperty(Property.MASTER_RECOVERY_DELAY, "5s");
+    cfg.setProperty(Property.MANAGER_RECOVERY_DELAY, "5s");
     // use raw local file system so walogs sync and flush will work
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
index 2030166..2dc63ab 100644
--- a/test/src/main/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
@@ -72,7 +72,7 @@ public class MissingWalHeaderCompletesRecoveryIT extends ConfigurableMacBase {
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration conf) {
     cfg.setNumTservers(1);
-    cfg.setProperty(Property.MASTER_RECOVERY_DELAY, "1s");
+    cfg.setProperty(Property.MANAGER_RECOVERY_DELAY, "1s");
     // Make sure the GC doesn't delete the file before the metadata reference is added
     cfg.setProperty(Property.GC_CYCLE_START, "999999s");
     conf.set("fs.file.impl", RawLocalFileSystem.class.getName());
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 bdf6dcf..2862ace 100644
--- a/test/src/main/java/org/apache/accumulo/test/ThriftServerBindsBeforeZooKeeperLockIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ThriftServerBindsBeforeZooKeeperLockIT.java
@@ -258,7 +258,7 @@ public class ThriftServerBindsBeforeZooKeeperLockIT extends AccumuloClusterHarne
         service = Monitor.class;
         break;
       case MANAGER:
-        property = Property.MASTER_CLIENTPORT;
+        property = Property.MANAGER_CLIENTPORT;
         service = Master.class;
         break;
       case GARBAGE_COLLECTOR:
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ManyWriteAheadLogsIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ManyWriteAheadLogsIT.java
index a1e6411..9f1a8bc 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ManyWriteAheadLogsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ManyWriteAheadLogsIT.java
@@ -62,7 +62,7 @@ public class ManyWriteAheadLogsIT extends AccumuloClusterHarness {
     cfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, "1M");
     cfg.setProperty(Property.GC_CYCLE_DELAY, "1");
     cfg.setProperty(Property.GC_CYCLE_START, "1");
-    cfg.setProperty(Property.MASTER_RECOVERY_DELAY, "1s");
+    cfg.setProperty(Property.MANAGER_RECOVERY_DELAY, "1s");
     cfg.setProperty(Property.TSERV_MAJC_DELAY, "1");
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
     // idle compactions may addess the problem this test is creating, however they will not prevent
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MasterMetricsIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MasterMetricsIT.java
index 721a218..a6e05b6 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/MasterMetricsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MasterMetricsIT.java
@@ -104,7 +104,7 @@ public class MasterMetricsIT extends AccumuloClusterHarness {
   public void metricsPublished() throws AccumuloException, AccumuloSecurityException {
 
     assumeTrue(accumuloClient.instanceOperations().getSystemConfiguration()
-        .get(Property.MASTER_FATE_METRICS_ENABLED.getKey()).compareTo("true") == 0);
+        .get(Property.MANAGER_FATE_METRICS_ENABLED.getKey()).compareTo("true") == 0);
 
     log.trace("Client started, properties:{}", accumuloClient.properties());
 
@@ -142,7 +142,7 @@ public class MasterMetricsIT extends AccumuloClusterHarness {
   public void compactionMetrics() throws AccumuloSecurityException, AccumuloException {
 
     assumeTrue(accumuloClient.instanceOperations().getSystemConfiguration()
-        .get(Property.MASTER_FATE_METRICS_ENABLED.getKey()).compareTo("true") == 0);
+        .get(Property.MANAGER_FATE_METRICS_ENABLED.getKey()).compareTo("true") == 0);
 
     MetricsFileTailer.LineUpdate firstUpdate =
         metricsTail.waitForUpdate(-1, NUM_TAIL_ATTEMPTS, TAIL_DELAY);
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/RestartStressIT.java b/test/src/main/java/org/apache/accumulo/test/functional/RestartStressIT.java
index 40f656c..6f115f1 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/RestartStressIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/RestartStressIT.java
@@ -55,7 +55,7 @@ public class RestartStressIT extends AccumuloClusterHarness {
     opts.put(Property.TSERV_MAJC_DELAY.getKey(), "100ms");
     opts.put(Property.TSERV_WALOG_MAX_SIZE.getKey(), "1M");
     opts.put(Property.INSTANCE_ZK_TIMEOUT.getKey(), "15s");
-    opts.put(Property.MASTER_RECOVERY_DELAY.getKey(), "1s");
+    opts.put(Property.MANAGER_RECOVERY_DELAY.getKey(), "1s");
     cfg.setSiteConfig(opts);
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java b/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java
index 678b138..843eaec 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java
@@ -42,7 +42,7 @@ public class WriteAheadLogIT extends AccumuloClusterHarness {
     cfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, "2M");
     cfg.setProperty(Property.GC_CYCLE_DELAY, "1");
     cfg.setProperty(Property.GC_CYCLE_START, "1");
-    cfg.setProperty(Property.MASTER_RECOVERY_DELAY, "1s");
+    cfg.setProperty(Property.MANAGER_RECOVERY_DELAY, "1s");
     cfg.setProperty(Property.TSERV_MAJC_DELAY, "1");
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
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 86b437f..736f6d3 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
@@ -180,7 +180,7 @@ public class CyclicReplicationIT {
       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.MASTER_REPLICATION_SCAN_INTERVAL, "1s");
+      master1Cfg.setProperty(Property.MANAGER_REPLICATION_SCAN_INTERVAL, "1s");
       master1Cluster = new MiniAccumuloClusterImpl(master1Cfg);
       setCoreSite(master1Cluster);
 
@@ -206,7 +206,7 @@ public class CyclicReplicationIT {
       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.MASTER_REPLICATION_SCAN_INTERVAL, "1s");
+      master2Cfg.setProperty(Property.MANAGER_REPLICATION_SCAN_INTERVAL, "1s");
       master2Cluster = new MiniAccumuloClusterImpl(master2Cfg);
       setCoreSite(master2Cluster);
 
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 de7d2bc..bf8a390 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
@@ -101,7 +101,7 @@ public class GarbageCollectorCommunicatesWithTServersIT extends ConfigurableMacB
     // Set really long delays for the master 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.MASTER_REPLICATION_SCAN_INTERVAL, "240s");
+    cfg.setProperty(Property.MANAGER_REPLICATION_SCAN_INTERVAL, "240s");
     cfg.setProperty(Property.REPLICATION_DRIVER_DELAY, "240s");
     // Pull down the maximum size of the wal so we can test close()'ing it.
     cfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, "1M");
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 56a3d92..c88aa40 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
@@ -124,7 +124,7 @@ public class KerberosReplicationIT extends AccumuloITBase {
         cfg.setProperty(Property.GC_CYCLE_START, "1s");
         cfg.setProperty(Property.GC_CYCLE_DELAY, "5s");
         cfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
-        cfg.setProperty(Property.MASTER_REPLICATION_SCAN_INTERVAL, "1s");
+        cfg.setProperty(Property.MANAGER_REPLICATION_SCAN_INTERVAL, "1s");
         cfg.setProperty(Property.REPLICATION_NAME, name);
         cfg.setProperty(Property.REPLICATION_MAX_UNIT_SIZE, "8M");
         cfg.setProperty(Property.REPLICATION_WORK_ASSIGNER, SequentialWorkAssigner.class.getName());
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 9733308..287fad4 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
@@ -113,7 +113,7 @@ public class MultiInstanceReplicationIT extends ConfigurableMacBase {
     cfg.setProperty(Property.GC_CYCLE_START, "1s");
     cfg.setProperty(Property.GC_CYCLE_DELAY, "5s");
     cfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
-    cfg.setProperty(Property.MASTER_REPLICATION_SCAN_INTERVAL, "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_WORK_ASSIGNER, SequentialWorkAssigner.class.getName());
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 7b4b67f..acdd50e 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
@@ -138,7 +138,7 @@ public class ReplicationIT extends ConfigurableMacBase {
     // Run the master replication loop run frequently
     cfg.setClientProperty(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT, "15s");
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
-    cfg.setProperty(Property.MASTER_REPLICATION_SCAN_INTERVAL, "1s");
+    cfg.setProperty(Property.MANAGER_REPLICATION_SCAN_INTERVAL, "1s");
     cfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
     cfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, "1M");
     cfg.setProperty(Property.GC_CYCLE_START, "1s");
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 62e2567..382a5f3 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
@@ -120,7 +120,7 @@ public class UnorderedWorkAssignerReplicationIT extends ConfigurableMacBase {
     cfg.setProperty(Property.GC_CYCLE_START, "1s");
     cfg.setProperty(Property.GC_CYCLE_DELAY, "5s");
     cfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
-    cfg.setProperty(Property.MASTER_REPLICATION_SCAN_INTERVAL, "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_WORK_ASSIGNER, UnorderedWorkAssigner.class.getName());
diff --git a/test/src/main/java/org/apache/accumulo/test/upgrade/DeprecatedPropertyUpgrade9to10IT.java b/test/src/main/java/org/apache/accumulo/test/upgrade/DeprecatedPropertyUpgrade9to10IT.java
new file mode 100644
index 0000000..3e0a7d3
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/upgrade/DeprecatedPropertyUpgrade9to10IT.java
@@ -0,0 +1,67 @@
+/*
+ * 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.test.upgrade;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Map;
+
+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;
+import org.apache.accumulo.fate.zookeeper.ZooUtil;
+import org.apache.accumulo.master.upgrade.Upgrader9to10;
+import org.apache.accumulo.test.functional.ConfigurableMacBase;
+import org.junit.Test;
+
+public class DeprecatedPropertyUpgrade9to10IT extends ConfigurableMacBase {
+  private static final Upgrader9to10 upgrader = new Upgrader9to10();
+
+  @Test
+  public void testZookeeperUpdate() throws Exception {
+    String oldProp = "master.bulk.retries";
+    String newProp = Property.MANAGER_BULK_RETRIES.getKey();
+    String propValue =
+        Integer.toString(Integer.parseInt(Property.MANAGER_BULK_RETRIES.getDefaultValue()) * 2);
+
+    String zPath = getServerContext().getZooKeeperRoot() + Constants.ZCONFIG + "/" + oldProp;
+    getServerContext().getZooReaderWriter().putPersistentData(zPath, propValue.getBytes(UTF_8),
+        ZooUtil.NodeExistsPolicy.OVERWRITE);
+
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) {
+      Map<String,String> systemConfig = client.instanceOperations().getSystemConfiguration();
+      assertTrue(oldProp + " missing from system config before upgrade",
+          systemConfig.containsKey(oldProp));
+      assertEquals(propValue, systemConfig.get(oldProp));
+      assertEquals(Property.MANAGER_BULK_RETRIES.getDefaultValue(), systemConfig.get(newProp));
+      assertNotEquals(propValue, systemConfig.get(newProp));
+
+      upgrader.upgradeZookeeper(getServerContext());
+
+      systemConfig = client.instanceOperations().getSystemConfiguration();
+      assertFalse(oldProp + " is still set after upgrade", systemConfig.containsKey(oldProp));
+      assertEquals(propValue, systemConfig.get(newProp));
+    }
+  }
+}