You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by tf...@apache.org on 2020/08/07 23:11:08 UTC

[lucene-solr] branch branch_8x updated: SOLR-14702: Remove Master and Slave from Code Base and Docs (8.x) (#1719)

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

tflobbe pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/branch_8x by this push:
     new f0eef22  SOLR-14702: Remove Master and Slave from Code Base and Docs (8.x) (#1719)
f0eef22 is described below

commit f0eef2270cc5d2659a0f837760415dbb80fa0c2c
Author: Tomas Fernandez Lobbe <tf...@apache.org>
AuthorDate: Fri Aug 7 16:10:57 2020 -0700

    SOLR-14702: Remove Master and Slave from Code Base and Docs (8.x) (#1719)
    
    This commit brings the changes from #1711 into 8.x and adds a commit to bring back compatibility. With this code change, a Solr instance running 8.7 should be able to replicate from a Solr instance running 8.x < 8.7, and a Solr instance running 9.0 should be able to replicate from an instance running 8.7
    
    Co-authored-by: Marcus <ma...@gmail.com>
---
 dev-tools/scripts/SOLR-2452.patch.hack.pl          |    4 +-
 .../collection1/conf/dataimport-solrconfig.xml     |    2 +-
 .../collection1/conf/contentstream-solrconfig.xml  |    2 +-
 .../conf/dataimport-nodatasource-solrconfig.xml    |    2 +-
 .../collection1/conf/dataimport-solrconfig.xml     |    2 +-
 .../dataimport/TestContentStreamDataSource.java    |    2 +-
 .../org/apache/solr/cloud/RecoveryStrategy.java    |    4 +-
 .../org/apache/solr/cloud/ReplicateFromLeader.java |   10 +-
 .../apache/solr/handler/CdcrReplicatorManager.java |    2 +-
 .../apache/solr/handler/CdcrRequestHandler.java    |   14 +-
 .../java/org/apache/solr/handler/IndexFetcher.java |  112 +-
 .../apache/solr/handler/ReplicationHandler.java    |  242 ++--
 .../solr/handler/component/FacetComponent.java     |    6 +-
 .../solr/handler/component/StatsValuesFactory.java |    2 +-
 .../solr/search/stats/ExactSharedStatsCache.java   |    2 +-
 .../apache/solr/search/stats/LRUStatsCache.java    |    2 +-
 .../org/apache/solr/search/stats/StatsCache.java   |    2 +-
 .../solr/security/JWTVerificationkeyResolver.java  |    2 +-
 .../java/org/apache/solr/util/TestInjection.java   |   12 +-
 ...olrconfig-slave.xml => solrconfig-follower.xml} |    4 +-
 ...rconfig-slave1.xml => solrconfig-follower1.xml} |    0
 ...rottled.xml => solrconfig-leader-throttled.xml} |    0
 ...solrconfig-master.xml => solrconfig-leader.xml} |    2 +-
 ...up.xml => solrconfig-leader1-keepOneBackup.xml} |    2 +-
 ...lrconfig-master1.xml => solrconfig-leader1.xml} |    2 +-
 ...lrconfig-master2.xml => solrconfig-leader2.xml} |    2 +-
 ...lrconfig-master3.xml => solrconfig-leader3.xml} |    2 +-
 .../solr/collection1/conf/solrconfig-repeater.xml  |    6 +-
 ...slave.xml => solrconfig-replication-legacy.xml} |    3 +
 .../solr/analysis/TestDeprecatedFilters.java       |    2 +-
 .../cloud/cdcr/CdcrReplicationHandlerTest.java     |   88 +-
 .../solr/core/snapshots/TestSolrCoreSnapshots.java |   22 +-
 .../apache/solr/handler/BackupRestoreUtils.java    |   20 +-
 .../solr/handler/TestHdfsBackupRestoreCore.java    |   14 +-
 .../solr/handler/TestReplicationHandler.java       | 1216 +++++++++++---------
 .../solr/handler/TestReplicationHandlerBackup.java |   68 +-
 .../TestReplicationHandlerDiskOverFlow.java        |   74 +-
 .../org/apache/solr/handler/TestRestoreCore.java   |   78 +-
 .../solr/security/BasicAuthStandaloneTest.java     |    2 +-
 solr/solr-ref-guide/src/cloud-screens.adoc         |    2 +-
 .../src/collection-specific-tools.adoc             |    2 +-
 .../combining-distribution-and-replication.adoc    |   12 +-
 solr/solr-ref-guide/src/coreadmin-api.adoc         |    2 +-
 .../distributed-search-with-index-sharding.adoc    |    2 +-
 .../src/implicit-requesthandlers.adoc              |    2 +-
 solr/solr-ref-guide/src/index-replication.adoc     |  156 +--
 .../src/major-changes-in-solr-8.adoc               |    4 +-
 .../src/near-real-time-searching.adoc              |    2 +-
 .../src/query-settings-in-solrconfig.adoc          |    2 +-
 solr/solr-ref-guide/src/replication-screen.adoc    |    8 +-
 .../solr-ref-guide/src/request-parameters-api.adoc |    2 +-
 .../src/shards-and-indexing-data-in-solrcloud.adoc |    2 +-
 solr/solr-ref-guide/src/solr-glossary.adoc         |    2 +-
 solr/solr-ref-guide/src/solrcloud.adoc             |    2 +-
 .../src/updatehandlers-in-solrconfig.adoc          |    2 +-
 .../solr/client/solrj/impl/LBHttp2SolrClient.java  |    4 +-
 .../solr/client/solrj/impl/LBHttpSolrClient.java   |    4 +-
 ...rconfig-slave1.xml => solrconfig-follower1.xml} |    0
 .../solr/client/solrj/TestLBHttp2SolrClient.java   |    2 +-
 .../solr/client/solrj/TestLBHttpSolrClient.java    |    2 +-
 solr/webapp/web/css/angular/collections.css        |    2 +-
 solr/webapp/web/css/angular/dashboard.css          |    4 +-
 solr/webapp/web/css/angular/menu.css               |    2 +-
 solr/webapp/web/css/angular/replication.css        |   10 +-
 .../img/ico/{node-slave.png => node-follower.png}  |  Bin
 .../img/ico/{node-master.png => node-leader.png}   |  Bin
 .../web/js/angular/controllers/core-overview.js    |    4 +-
 .../web/js/angular/controllers/replication.js      |   92 +-
 solr/webapp/web/partials/core_overview.html        |   38 +-
 solr/webapp/web/partials/replication.html          |   62 +-
 70 files changed, 1286 insertions(+), 1177 deletions(-)

diff --git a/dev-tools/scripts/SOLR-2452.patch.hack.pl b/dev-tools/scripts/SOLR-2452.patch.hack.pl
index 7da5c54..2f6c7fc 100755
--- a/dev-tools/scripts/SOLR-2452.patch.hack.pl
+++ b/dev-tools/scripts/SOLR-2452.patch.hack.pl
@@ -167,8 +167,8 @@ my @copies = (
     'solr/core/src/test-files/solr/conf/schema-replication1.xml'
  => 'solr/solrj/src/test-files/solrj/solr/conf/schema-replication1.xml',
 
-    'solr/core/src/test-files/solr/conf/solrconfig-slave1.xml'
- => 'solr/solrj/src/test-files/solrj/solr/conf/solrconfig-slave1.xml',
+    'solr/core/src/test-files/solr/conf/solrconfig-follower1.xml'
+ => 'solr/solrj/src/test-files/solrj/solr/conf/solrconfig-follower1.xml',
 );
 
 my $diff;
diff --git a/solr/contrib/dataimporthandler-extras/src/test-files/dihextras/solr/collection1/conf/dataimport-solrconfig.xml b/solr/contrib/dataimporthandler-extras/src/test-files/dihextras/solr/collection1/conf/dataimport-solrconfig.xml
index b851f51..f74c48b 100644
--- a/solr/contrib/dataimporthandler-extras/src/test-files/dihextras/solr/collection1/conf/dataimport-solrconfig.xml
+++ b/solr/contrib/dataimporthandler-extras/src/test-files/dihextras/solr/collection1/conf/dataimport-solrconfig.xml
@@ -162,7 +162,7 @@
 
     <!-- Maximum number of searchers that may be warming in the background
       concurrently.  An error is returned if this limit is exceeded. Recommend
-      1-2 for read-only slaves, higher for masters w/o cache warming. -->
+      1-2 for read-only followers, higher for leaders w/o cache warming. -->
     <maxWarmingSearchers>4</maxWarmingSearchers>
 
   </query>
diff --git a/solr/contrib/dataimporthandler/src/test-files/dih/solr/collection1/conf/contentstream-solrconfig.xml b/solr/contrib/dataimporthandler/src/test-files/dih/solr/collection1/conf/contentstream-solrconfig.xml
index 616a65b..dfe50f1 100644
--- a/solr/contrib/dataimporthandler/src/test-files/dih/solr/collection1/conf/contentstream-solrconfig.xml
+++ b/solr/contrib/dataimporthandler/src/test-files/dih/solr/collection1/conf/contentstream-solrconfig.xml
@@ -162,7 +162,7 @@
 
     <!-- Maximum number of searchers that may be warming in the background
       concurrently.  An error is returned if this limit is exceeded. Recommend
-      1-2 for read-only slaves, higher for masters w/o cache warming. -->
+      1-2 for read-only followers, higher for leaders w/o cache warming. -->
     <maxWarmingSearchers>4</maxWarmingSearchers>
 
   </query>
diff --git a/solr/contrib/dataimporthandler/src/test-files/dih/solr/collection1/conf/dataimport-nodatasource-solrconfig.xml b/solr/contrib/dataimporthandler/src/test-files/dih/solr/collection1/conf/dataimport-nodatasource-solrconfig.xml
index c7ebd7d..0e84b89 100644
--- a/solr/contrib/dataimporthandler/src/test-files/dih/solr/collection1/conf/dataimport-nodatasource-solrconfig.xml
+++ b/solr/contrib/dataimporthandler/src/test-files/dih/solr/collection1/conf/dataimport-nodatasource-solrconfig.xml
@@ -164,7 +164,7 @@
 
     <!-- Maximum number of searchers that may be warming in the background
       concurrently.  An error is returned if this limit is exceeded. Recommend
-      1-2 for read-only slaves, higher for masters w/o cache warming. -->
+      1-2 for read-only followers, higher for leaders w/o cache warming. -->
     <maxWarmingSearchers>4</maxWarmingSearchers>
 
   </query>
diff --git a/solr/contrib/dataimporthandler/src/test-files/dih/solr/collection1/conf/dataimport-solrconfig.xml b/solr/contrib/dataimporthandler/src/test-files/dih/solr/collection1/conf/dataimport-solrconfig.xml
index 116c148..73e16f6 100644
--- a/solr/contrib/dataimporthandler/src/test-files/dih/solr/collection1/conf/dataimport-solrconfig.xml
+++ b/solr/contrib/dataimporthandler/src/test-files/dih/solr/collection1/conf/dataimport-solrconfig.xml
@@ -162,7 +162,7 @@
 
     <!-- Maximum number of searchers that may be warming in the background
       concurrently.  An error is returned if this limit is exceeded. Recommend
-      1-2 for read-only slaves, higher for masters w/o cache warming. -->
+      1-2 for read-only followers, higher for leaders w/o cache warming. -->
     <maxWarmingSearchers>4</maxWarmingSearchers>
 
   </query>
diff --git a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestContentStreamDataSource.java b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestContentStreamDataSource.java
index 34e50e0..4e4cf23 100644
--- a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestContentStreamDataSource.java
+++ b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestContentStreamDataSource.java
@@ -120,7 +120,7 @@ public class TestContentStreamDataSource extends AbstractDataImportHandlerTestCa
     File dataDir;
     
     /**
-     * if masterPort is null, this instance is a master -- otherwise this instance is a slave, and assumes the master is
+     * if leaderPort is null, this instance is a leader -- otherwise this instance is a follower, and assumes the leader is
      * on localhost at the specified port.
      */
     public SolrInstance(String name, Integer port) {
diff --git a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
index 2be35fb..d6d574d 100644
--- a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
+++ b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
@@ -239,8 +239,8 @@ public class RecoveryStrategy implements Runnable, Closeable {
     }
 
     ModifiableSolrParams solrParams = new ModifiableSolrParams();
-    solrParams.set(ReplicationHandler.MASTER_URL, leaderUrl);
-    solrParams.set(ReplicationHandler.SKIP_COMMIT_ON_MASTER_VERSION_ZERO, replicaType == Replica.Type.TLOG);
+    solrParams.set(ReplicationHandler.LEGACY_LEADER_URL, leaderUrl);
+    solrParams.set(ReplicationHandler.LEGACY_SKIP_COMMIT_ON_LEADER_VERSION_ZERO, replicaType == Replica.Type.TLOG);
     // always download the tlogs from the leader when running with cdcr enabled. We need to have all the tlogs
     // to ensure leader failover doesn't cause missing docs on the target
     if (core.getUpdateHandler().getUpdateLog() != null
diff --git a/solr/core/src/java/org/apache/solr/cloud/ReplicateFromLeader.java b/solr/core/src/java/org/apache/solr/cloud/ReplicateFromLeader.java
index 17a6ec3..7e2b872 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ReplicateFromLeader.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ReplicateFromLeader.java
@@ -76,12 +76,12 @@ public class ReplicateFromLeader {
       }
       log.info("Will start replication from leader with poll interval: {}", pollIntervalStr );
 
-      NamedList<Object> slaveConfig = new NamedList<>();
-      slaveConfig.add("fetchFromLeader", Boolean.TRUE);
-      slaveConfig.add(ReplicationHandler.SKIP_COMMIT_ON_MASTER_VERSION_ZERO, switchTransactionLog);
-      slaveConfig.add("pollInterval", pollIntervalStr);
+      NamedList<Object> followerConfig = new NamedList<>();
+      followerConfig.add("fetchFromLeader", Boolean.TRUE);
+      followerConfig.add(ReplicationHandler.SKIP_COMMIT_ON_LEADER_VERSION_ZERO, switchTransactionLog);
+      followerConfig.add("pollInterval", pollIntervalStr);
       NamedList<Object> replicationConfig = new NamedList<>();
-      replicationConfig.add("slave", slaveConfig);
+      replicationConfig.add("follower", followerConfig);
 
       String lastCommitVersion = getCommitVersion(core);
       if (lastCommitVersion != null) {
diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorManager.java b/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorManager.java
index c5881c7..84aa8db 100644
--- a/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorManager.java
+++ b/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorManager.java
@@ -369,7 +369,7 @@ class CdcrReplicatorManager implements CdcrStateManager.CdcrStateObserver {
         log.info("Attempting to bootstrap target collection: {} shard: {} leader: {}", targetCollection, shard, leaderCoreUrl);
         try {
           @SuppressWarnings({"rawtypes"})
-          NamedList response = sendCdcrCommand(client, CdcrParams.CdcrAction.BOOTSTRAP, ReplicationHandler.MASTER_URL, myCoreUrl);
+          NamedList response = sendCdcrCommand(client, CdcrParams.CdcrAction.BOOTSTRAP, ReplicationHandler.LEGACY_LEADER_URL, myCoreUrl);
           log.debug("CDCR Bootstrap response: {}", response);
           String status = response.get(RESPONSE_STATUS).toString();
           return BootstrapStatus.valueOf(status.toUpperCase(Locale.ROOT));
diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrRequestHandler.java b/solr/core/src/java/org/apache/solr/handler/CdcrRequestHandler.java
index d50d39a..4f3e28b 100644
--- a/solr/core/src/java/org/apache/solr/handler/CdcrRequestHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/CdcrRequestHandler.java
@@ -652,8 +652,8 @@ public class CdcrRequestHandler extends RequestHandlerBase implements SolrCoreAw
           coreState.setCdcrBootstrapRunning(true);
           latch.countDown(); // free the latch as current bootstrap is executing
           //running.set(true);
-          String masterUrl = req.getParams().get(ReplicationHandler.MASTER_URL);
-          BootstrapCallable bootstrapCallable = new BootstrapCallable(masterUrl, core);
+          String leaderUrl = ReplicationHandler.getObjectWithBackwardCompatibility(req.getParams(), ReplicationHandler.LEADER_URL, ReplicationHandler.LEGACY_LEADER_URL, null);
+          BootstrapCallable bootstrapCallable = new BootstrapCallable(leaderUrl, core);
           coreState.setCdcrBootstrapCallable(bootstrapCallable);
           Future<Boolean> bootstrapFuture = core.getCoreContainer().getUpdateShardHandler().getRecoveryExecutor()
               .submit(bootstrapCallable);
@@ -733,12 +733,12 @@ public class CdcrRequestHandler extends RequestHandlerBase implements SolrCoreAw
   }
 
   static class BootstrapCallable implements Callable<Boolean>, Closeable {
-    private final String masterUrl;
+    private final String leaderUrl;
     private final SolrCore core;
     private volatile boolean closed = false;
 
-    BootstrapCallable(String masterUrl, SolrCore core) {
-      this.masterUrl = masterUrl;
+    BootstrapCallable(String leaderUrl, SolrCore core) {
+      this.leaderUrl = leaderUrl;
       this.core = core;
     }
 
@@ -762,7 +762,7 @@ public class CdcrRequestHandler extends RequestHandlerBase implements SolrCoreAw
       // to receive any updates from the source during bootstrap
       ulog.bufferUpdates();
       try {
-        commitOnLeader(masterUrl);
+        commitOnLeader(leaderUrl);
         // use rep handler directly, so we can do this sync rather than async
         SolrRequestHandler handler = core.getRequestHandler(ReplicationHandler.PATH);
         ReplicationHandler replicationHandler = (ReplicationHandler) handler;
@@ -773,7 +773,7 @@ public class CdcrRequestHandler extends RequestHandlerBase implements SolrCoreAw
         }
 
         ModifiableSolrParams solrParams = new ModifiableSolrParams();
-        solrParams.set(ReplicationHandler.MASTER_URL, masterUrl);
+        solrParams.set(ReplicationHandler.LEGACY_LEADER_URL, leaderUrl);
         // we do not want the raw tlog files from the source
         solrParams.set(ReplicationHandler.TLOG_FILES, false);
 
diff --git a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
index e78028e..57f8ba0 100644
--- a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
+++ b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
@@ -112,7 +112,7 @@ import static org.apache.solr.handler.ReplicationHandler.*;
 
 /**
  * <p> Provides functionality of downloading changed index files as well as config files and a timer for scheduling fetches from the
- * master. </p>
+ * leader. </p>
  *
  *
  * @since solr 1.4
@@ -124,7 +124,7 @@ public class IndexFetcher {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  private String masterUrl;
+  private String leaderUrl;
 
   final ReplicationHandler replicationHandler;
 
@@ -169,7 +169,7 @@ public class IndexFetcher {
 
   private boolean downloadTlogFiles = false;
 
-  private boolean skipCommitOnMasterVersionZero = true;
+  private boolean skipCommitOnLeaderVersionZero = true;
 
   private boolean clearLocalIndexFirst = false;
 
@@ -189,7 +189,7 @@ public class IndexFetcher {
     public static final IndexFetchResult INDEX_FETCH_SUCCESS = new IndexFetchResult("Fetching latest index is successful", true, null);
     public static final IndexFetchResult LOCK_OBTAIN_FAILED = new IndexFetchResult("Obtaining SnapPuller lock failed", false, null);
     public static final IndexFetchResult CONTAINER_IS_SHUTTING_DOWN = new IndexFetchResult("I was asked to replicate but CoreContainer is shutting down", false, null);
-    public static final IndexFetchResult MASTER_VERSION_ZERO = new IndexFetchResult("Index in peer is empty and never committed yet", true, null);
+    public static final IndexFetchResult LEADER_VERSION_ZERO = new IndexFetchResult("Index in peer is empty and never committed yet", true, null);
     public static final IndexFetchResult NO_INDEX_COMMIT_EXIST = new IndexFetchResult("No IndexCommit in local index", false, null);
     public static final IndexFetchResult PEER_INDEX_COMMIT_DELETED = new IndexFetchResult("No files to download because IndexCommit in peer was deleted", false, null);
     public static final IndexFetchResult LOCAL_ACTIVITY_DURING_REPLICATION = new IndexFetchResult("Local index modification during replication", false, null);
@@ -236,19 +236,19 @@ public class IndexFetcher {
     if (fetchFromLeader != null && fetchFromLeader instanceof Boolean) {
       this.fetchFromLeader = (boolean) fetchFromLeader;
     }
-    Object skipCommitOnMasterVersionZero = initArgs.get(SKIP_COMMIT_ON_MASTER_VERSION_ZERO);
-    if (skipCommitOnMasterVersionZero != null && skipCommitOnMasterVersionZero instanceof Boolean) {
-      this.skipCommitOnMasterVersionZero = (boolean) skipCommitOnMasterVersionZero;
+    Object skipCommitOnLeaderVersionZero = ReplicationHandler.getObjectWithBackwardCompatibility(initArgs, SKIP_COMMIT_ON_LEADER_VERSION_ZERO, LEGACY_SKIP_COMMIT_ON_LEADER_VERSION_ZERO);
+    if (skipCommitOnLeaderVersionZero != null && skipCommitOnLeaderVersionZero instanceof Boolean) {
+      this.skipCommitOnLeaderVersionZero = (boolean) skipCommitOnLeaderVersionZero;
     }
-    String masterUrl = (String) initArgs.get(MASTER_URL);
-    if (masterUrl == null && !this.fetchFromLeader)
+    String leaderUrl = ReplicationHandler.getObjectWithBackwardCompatibility(initArgs, LEADER_URL, LEGACY_LEADER_URL);
+    if (leaderUrl == null && !this.fetchFromLeader)
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
-              "'masterUrl' is required for a slave");
-    if (masterUrl != null && masterUrl.endsWith(ReplicationHandler.PATH)) {
-      masterUrl = masterUrl.substring(0, masterUrl.length()-12);
-      log.warn("'masterUrl' must be specified without the {} suffix", ReplicationHandler.PATH);
+              "'leaderUrl' is required for a follower");
+    if (leaderUrl != null && leaderUrl.endsWith(ReplicationHandler.PATH)) {
+      leaderUrl = leaderUrl.substring(0, leaderUrl.length()-12);
+      log.warn("'leaderUrl' must be specified without the {} suffix", ReplicationHandler.PATH);
     }
-    this.masterUrl = masterUrl;
+    this.leaderUrl = leaderUrl;
 
     this.replicationHandler = handler;
     String compress = (String) initArgs.get(COMPRESSION);
@@ -256,7 +256,7 @@ public class IndexFetcher {
     useExternalCompression = EXTERNAL.equals(compress);
     connTimeout = getParameter(initArgs, HttpClientUtil.PROP_CONNECTION_TIMEOUT, 30000, null);
     
-    // allow a master override for tests - you specify this in /replication slave section of solrconfig and some 
+    // allow a leader override for tests - you specify this in /replication follower section of solrconfig and some
     // test don't want to define this
     soTimeout = Integer.getInteger("solr.indexfetcher.sotimeout", -1);
     if (soTimeout == -1) {
@@ -284,7 +284,7 @@ public class IndexFetcher {
   }
 
   /**
-   * Gets the latest commit version and generation from the master
+   * Gets the latest commit version and generation from the leader
    */
   @SuppressWarnings({"unchecked", "rawtypes"})
   NamedList getLatestVersion() throws IOException {
@@ -295,7 +295,7 @@ public class IndexFetcher {
     QueryRequest req = new QueryRequest(params);
 
     // TODO modify to use shardhandler
-    try (HttpSolrClient client = new Builder(masterUrl)
+    try (HttpSolrClient client = new Builder(leaderUrl)
         .withHttpClient(myHttpClient)
         .withConnectionTimeout(connTimeout)
         .withSocketTimeout(soTimeout)
@@ -321,7 +321,7 @@ public class IndexFetcher {
     QueryRequest req = new QueryRequest(params);
 
     // TODO modify to use shardhandler
-    try (HttpSolrClient client = new HttpSolrClient.Builder(masterUrl)
+    try (HttpSolrClient client = new HttpSolrClient.Builder(leaderUrl)
         .withHttpClient(myHttpClient)
         .withConnectionTimeout(connTimeout)
         .withSocketTimeout(soTimeout)
@@ -355,12 +355,12 @@ public class IndexFetcher {
   }
 
   /**
-   * This command downloads all the necessary files from master to install a index commit point. Only changed files are
+   * This command downloads all the necessary files from leader to install a index commit point. Only changed files are
    * downloaded. It also downloads the conf files (if they are modified).
    *
    * @param forceReplication force a replication in all cases
    * @param forceCoreReload force a core reload in all cases
-   * @return true on success, false if slave is already in sync
+   * @return true on success, false if follower is already in sync
    * @throws IOException if an exception occurs
    */
   IndexFetchResult fetchLatestIndex(boolean forceReplication, boolean forceCoreReload) throws IOException, InterruptedException {
@@ -404,15 +404,15 @@ public class IndexFetcher {
           }
           return IndexFetchResult.LEADER_IS_NOT_ACTIVE;
         }
-        if (!replica.getCoreUrl().equals(masterUrl)) {
-          masterUrl = replica.getCoreUrl();
-          log.info("Updated masterUrl to {}", masterUrl);
+        if (!replica.getCoreUrl().equals(leaderUrl)) {
+          leaderUrl = replica.getCoreUrl();
+          log.info("Updated leaderUrl to {}", leaderUrl);
           // TODO: Do we need to set forceReplication = true?
         } else {
-          log.debug("masterUrl didn't change");
+          log.debug("leaderUrl didn't change");
         }
       }
-      //get the current 'replicateable' index version in the master
+      //get the current 'replicateable' index version in the leader
       @SuppressWarnings({"rawtypes"})
       NamedList response;
       try {
@@ -420,10 +420,10 @@ public class IndexFetcher {
       } catch (Exception e) {
         final String errorMsg = e.toString();
         if (!Strings.isNullOrEmpty(errorMsg) && errorMsg.contains(INTERRUPT_RESPONSE_MESSAGE)) {
-            log.warn("Master at: {} is not available. Index fetch failed by interrupt. Exception: {}", masterUrl, errorMsg);
+            log.warn("Leader at: {} is not available. Index fetch failed by interrupt. Exception: {}", leaderUrl, errorMsg);
             return new IndexFetchResult(IndexFetchResult.FAILED_BY_INTERRUPT_MESSAGE, false, e);
         } else {
-            log.warn("Master at: {} is not available. Index fetch failed by exception: {}", masterUrl, errorMsg);
+            log.warn("Leader at: {} is not available. Index fetch failed by exception: {}", leaderUrl, errorMsg);
             return new IndexFetchResult(IndexFetchResult.FAILED_BY_EXCEPTION_MESSAGE, false, e);
         }
     }
@@ -431,8 +431,8 @@ public class IndexFetcher {
       long latestVersion = (Long) response.get(CMD_INDEX_VERSION);
       long latestGeneration = (Long) response.get(GENERATION);
 
-      log.info("Master's generation: {}", latestGeneration);
-      log.info("Master's version: {}", latestVersion);
+      log.info("Leader's generation: {}", latestGeneration);
+      log.info("Leader's version: {}", latestVersion);
 
       // TODO: make sure that getLatestCommit only returns commit points for the main index (i.e. no side-car indexes)
       IndexCommit commit = solrCore.getDeletionPolicy().getLatestCommit();
@@ -453,23 +453,23 @@ public class IndexFetcher {
       }
 
       if (log.isInfoEnabled()) {
-        log.info("Slave's generation: {}", commit.getGeneration());
-        log.info("Slave's version: {}", IndexDeletionPolicyWrapper.getCommitTimestamp(commit)); // logOK
+        log.info("Follower's generation: {}", commit.getGeneration());
+        log.info("Follower's version: {}", IndexDeletionPolicyWrapper.getCommitTimestamp(commit)); // logOK
       }
 
       if (latestVersion == 0L) {
         if (commit.getGeneration() != 0) {
           // since we won't get the files for an empty index,
           // we just clear ours and commit
-          log.info("New index in Master. Deleting mine...");
+          log.info("New index in Leader. Deleting mine...");
           RefCounted<IndexWriter> iw = solrCore.getUpdateHandler().getSolrCoreState().getIndexWriter(solrCore);
           try {
             iw.get().deleteAll();
           } finally {
             iw.decref();
           }
-          assert TestInjection.injectDelayBeforeSlaveCommitRefresh();
-          if (skipCommitOnMasterVersionZero) {
+          assert TestInjection.injectDelayBeforeFollowerCommitRefresh();
+          if (skipCommitOnLeaderVersionZero) {
             openNewSearcherAndUpdateCommitPoint();
           } else {
             SolrQueryRequest req = new LocalSolrQueryRequest(solrCore, new ModifiableSolrParams());
@@ -479,14 +479,14 @@ public class IndexFetcher {
 
         //there is nothing to be replicated
         successfulInstall = true;
-        log.debug("Nothing to replicate, master's version is 0");
-        return IndexFetchResult.MASTER_VERSION_ZERO;
+        log.debug("Nothing to replicate, leader's version is 0");
+        return IndexFetchResult.LEADER_VERSION_ZERO;
       }
 
       // TODO: Should we be comparing timestamps (across machines) here?
       if (!forceReplication && IndexDeletionPolicyWrapper.getCommitTimestamp(commit) == latestVersion) {
-        //master and slave are already in sync just return
-        log.info("Slave in sync with master.");
+        //leader and follower are already in sync just return
+        log.info("Follower in sync with leader.");
         successfulInstall = true;
         return IndexFetchResult.ALREADY_IN_SYNC;
       }
@@ -498,11 +498,11 @@ public class IndexFetcher {
         return IndexFetchResult.PEER_INDEX_COMMIT_DELETED;
       }
       if (log.isInfoEnabled()) {
-        log.info("Number of files in latest index in master: {}", filesToDownload.size());
+        log.info("Number of files in latest index in leader: {}", filesToDownload.size());
       }
       if (tlogFilesToDownload != null) {
         if (log.isInfoEnabled()) {
-          log.info("Number of tlog files in master: {}", tlogFilesToDownload.size());
+          log.info("Number of tlog files in leader: {}", tlogFilesToDownload.size());
         }
       }
 
@@ -510,7 +510,7 @@ public class IndexFetcher {
       fsyncService = ExecutorUtil.newMDCAwareSingleThreadExecutor(new SolrNamedThreadFactory("fsyncService"));
       // use a synchronized list because the list is read by other threads (to show details)
       filesDownloaded = Collections.synchronizedList(new ArrayList<Map<String, Object>>());
-      // if the generation of master is older than that of the slave , it means they are not compatible to be copied
+      // if the generation of leader is older than that of the follower , it means they are not compatible to be copied
       // then a new index directory to be created and all the files need to be copied
       boolean isFullCopyNeeded = IndexDeletionPolicyWrapper
           .getCommitTimestamp(commit) >= latestVersion
@@ -533,7 +533,7 @@ public class IndexFetcher {
 
       try {
 
-        // We will compare all the index files from the master vs the index files on disk to see if there is a mismatch
+        // We will compare all the index files from the leader vs the index files on disk to see if there is a mismatch
         // in the metadata. If there is a mismatch for the same index file then we download the entire index
         // (except when differential copy is applicable) again.
         if (!isFullCopyNeeded && isIndexStale(indexDir)) {
@@ -964,7 +964,7 @@ public class IndexFetcher {
   }
 
   private void downloadConfFiles(List<Map<String, Object>> confFilesToDownload, long latestGeneration) throws Exception {
-    log.info("Starting download of configuration files from master: {}", confFilesToDownload);
+    log.info("Starting download of configuration files from leader: {}", confFilesToDownload);
     confFilesDownloaded = Collections.synchronizedList(new ArrayList<>());
     File tmpconfDir = new File(solrCore.getResourceLoader().getConfigDir(), "conf." + getDateAsStr(new Date()));
     try {
@@ -993,7 +993,7 @@ public class IndexFetcher {
    * Download all the tlog files to the temp tlog directory.
    */
   private long downloadTlogFiles(File tmpTlogDir, long latestGeneration) throws Exception {
-    log.info("Starting download of tlog files from master: {}", tlogFilesToDownload);
+    log.info("Starting download of tlog files from leader: {}", tlogFilesToDownload);
     tlogFilesDownloaded = Collections.synchronizedList(new ArrayList<>());
     long bytesDownloaded = 0;
 
@@ -1146,7 +1146,7 @@ public class IndexFetcher {
       // after considering the files actually available locally we really don't need to do any delete
       return;
     }
-    log.info("This disk does not have enough space to download the index from leader/master. So cleaning up the local index. "
+    log.info("This disk does not have enough space to download the index from leader. So cleaning up the local index. "
         + " This may lead to loss of data/or node if index replication fails in between");
     //now we should disable searchers and index writers because this core will not have all the required files
     this.clearLocalIndexFirst = true;
@@ -1247,7 +1247,7 @@ public class IndexFetcher {
   }
 
   /**
-   * All the files which are common between master and slave must have same size and same checksum else we assume
+   * All the files which are common between leader and follower must have same size and same checksum else we assume
    * they are not compatible (stale).
    *
    * @return true if the index stale and we need to download a fresh copy, false otherwise.
@@ -1480,11 +1480,11 @@ public class IndexFetcher {
   private final Map<String, FileInfo> confFileInfoCache = new HashMap<>();
 
   /**
-   * The local conf files are compared with the conf files in the master. If they are same (by checksum) do not copy.
+   * The local conf files are compared with the conf files in the leader. If they are same (by checksum) do not copy.
    *
-   * @param confFilesToDownload The list of files obtained from master
+   * @param confFilesToDownload The list of files obtained from leader
    *
-   * @return a list of configuration files which have changed on the master and need to be downloaded.
+   * @return a list of configuration files which have changed on the leader and need to be downloaded.
    */
   @SuppressWarnings({"unchecked"})
   private Collection<Map<String, Object>> getModifiedConfFiles(List<Map<String, Object>> confFilesToDownload) {
@@ -1496,7 +1496,7 @@ public class IndexFetcher {
     @SuppressWarnings({"rawtypes"})
     NamedList names = new NamedList();
     for (Map<String, Object> map : confFilesToDownload) {
-      //if alias is present that is the name the file may have in the slave
+      //if alias is present that is the name the file may have in the follower
       String name = (String) (map.get(ALIAS) == null ? map.get(NAME) : map.get(ALIAS));
       nameVsFile.put(name, map);
       names.add(name, null);
@@ -1752,7 +1752,7 @@ public class IndexFetcher {
           }
           //then read the packet of bytes
           fis.readFully(buf, 0, packetSize);
-          //compare the checksum as sent from the master
+          //compare the checksum as sent from the leader
           if (includeChecksum) {
             checksum.reset();
             checksum.update(buf, 0, packetSize);
@@ -1870,7 +1870,7 @@ public class IndexFetcher {
       InputStream is = null;
 
       // TODO use shardhandler
-      try (HttpSolrClient client = new Builder(masterUrl)
+      try (HttpSolrClient client = new Builder(leaderUrl)
           .withHttpClient(myHttpClient)
           .withResponseParser(null)
           .withConnectionTimeout(connTimeout)
@@ -1979,11 +1979,11 @@ public class IndexFetcher {
   NamedList getDetails() throws IOException, SolrServerException {
     ModifiableSolrParams params = new ModifiableSolrParams();
     params.set(COMMAND, CMD_DETAILS);
-    params.set("slave", false);
+    params.set("follower", false);
     params.set(CommonParams.QT, ReplicationHandler.PATH);
 
     // TODO use shardhandler
-    try (HttpSolrClient client = new HttpSolrClient.Builder(masterUrl)
+    try (HttpSolrClient client = new HttpSolrClient.Builder(leaderUrl)
         .withHttpClient(myHttpClient)
         .withConnectionTimeout(connTimeout)
         .withSocketTimeout(soTimeout)
@@ -1998,8 +1998,8 @@ public class IndexFetcher {
     HttpClientUtil.close(myHttpClient);
   }
 
-  String getMasterUrl() {
-    return masterUrl;
+  String getLeaderUrl() {
+    return leaderUrl;
   }
 
   private static final int MAX_RETRIES = 5;
diff --git a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
index ec711dc..b758134 100644
--- a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
@@ -113,15 +113,15 @@ import static org.apache.solr.common.params.CommonParams.NAME;
 import static org.apache.solr.core.XmlConfigFile.assertWarnOrFail;
 
 /**
- * <p> A Handler which provides a REST API for replication and serves replication requests from Slaves. </p>
- * <p>When running on the master, it provides the following commands <ol> <li>Get the current replicable index version
+ * <p> A Handler which provides a REST API for replication and serves replication requests from Followers. </p>
+ * <p>When running on the leader, it provides the following commands <ol> <li>Get the current replicable index version
  * (command=indexversion)</li> <li>Get the list of files for a given index version
  * (command=filelist&amp;indexversion=&lt;VERSION&gt;)</li> <li>Get full or a part (chunk) of a given index or a config
  * file (command=filecontent&amp;file=&lt;FILE_NAME&gt;) You can optionally specify an offset and length to get that
  * chunk of the file. You can request a configuration file by using "cf" parameter instead of the "file" parameter.</li>
- * <li>Get status/statistics (command=details)</li> </ol> <p>When running on the slave, it provides the following
+ * <li>Get status/statistics (command=details)</li> </ol> <p>When running on the follower, it provides the following
  * commands <ol> <li>Perform an index fetch now (command=snappull)</li> <li>Get status/statistics (command=details)</li>
- * <li>Abort an index fetch (command=abort)</li> <li>Enable/Disable polling the master for new versions (command=enablepoll
+ * <li>Abort an index fetch (command=abort)</li> <li>Enable/Disable polling the leader for new versions (command=enablepoll
  * or command=disablepoll)</li> </ol>
  *
  *
@@ -187,9 +187,9 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
 
   private NamedList<String> confFileNameAlias = new NamedList<>();
 
-  private boolean isMaster = false;
+  private boolean isLeader = false;
 
-  private boolean isSlave = false;
+  private boolean isFollower = false;
 
   private boolean replicateOnOptimize = false;
 
@@ -242,7 +242,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
     final SolrParams solrParams = req.getParams();
     String command = solrParams.required().get(COMMAND);
 
-    // This command does not give the current index version of the master
+    // This command does not give the current index version of the leader
     // It gives the current 'replicateable' index version
     if (command.equals(CMD_INDEX_VERSION)) {
       IndexCommit commitPoint = indexCommitPoint;  // make a copy so it won't change
@@ -293,12 +293,12 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
       if (abortFetch()) {
         rsp.add(STATUS, OK_STATUS);
       } else {
-        reportErrorOnResponse(rsp, "No slave configured", null);
+        reportErrorOnResponse(rsp, "No follower configured", null);
       }
     } else if (command.equals(CMD_SHOW_COMMITS)) {
       populateCommitInfo(rsp);
     } else if (command.equals(CMD_DETAILS)) {
-      getReplicationDetails(rsp, solrParams.getBool("slave", true));
+      getReplicationDetails(rsp, getBoolWithBackwardCompatibility(solrParams, "follower", "slave", true));
     } else if (CMD_ENABLE_REPL.equalsIgnoreCase(command)) {
       replicationEnabled.set(true);
       rsp.add(STATUS, OK_STATUS);
@@ -307,6 +307,36 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
       rsp.add(STATUS, OK_STATUS);
     }
   }
+  
+  static boolean getBoolWithBackwardCompatibility(SolrParams params, String preferredKey, String alternativeKey, boolean defaultValue) {
+    Boolean value = params.getBool(preferredKey);
+    if (value != null) {
+      return value;
+    }
+    return params.getBool(alternativeKey, defaultValue);
+  }
+  
+  @SuppressWarnings("unchecked")
+  static <T> T getObjectWithBackwardCompatibility(SolrParams params, String preferredKey, String alternativeKey, T defaultValue) {
+    Object value = params.get(preferredKey);
+    if (value != null) {
+      return (T) value;
+    }
+    value = params.get(alternativeKey);
+    if (value != null) {
+      return (T) value;
+    }
+    return defaultValue;
+  }
+  
+  @SuppressWarnings("unchecked")
+  static <T> T getObjectWithBackwardCompatibility(NamedList<?> params, String preferredKey, String alternativeKey) {
+    Object value = params.get(preferredKey);
+    if (value != null) {
+      return (T) value;
+    }
+    return (T) params.get(alternativeKey);
+  }
 
   private void reportErrorOnResponse(SolrQueryResponse response, String message, Exception e) {
     response.add(STATUS, ERR_STATUS);
@@ -339,9 +369,9 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
   }
 
   private void fetchIndex(SolrParams solrParams, SolrQueryResponse rsp) throws InterruptedException {
-    String masterUrl = solrParams.get(MASTER_URL);
-    if (!isSlave && masterUrl == null) {
-      reportErrorOnResponse(rsp, "No slave configured or no 'masterUrl' specified", null);
+    String leaderUrl = getObjectWithBackwardCompatibility(solrParams, LEADER_URL, LEGACY_LEADER_URL, null);
+    if (!isFollower && leaderUrl == null) {
+      reportErrorOnResponse(rsp, "No follower configured or no 'leaderUrl' specified", null);
       return;
     }
     final SolrParams paramsCopy = new ModifiableSolrParams(solrParams);
@@ -408,7 +438,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
   private volatile IndexFetcher currentIndexFetcher;
 
   public IndexFetchResult doFetch(SolrParams solrParams, boolean forceReplication) {
-    String masterUrl = solrParams == null ? null : solrParams.get(MASTER_URL);
+    String leaderUrl = solrParams == null ? null : ReplicationHandler.getObjectWithBackwardCompatibility(solrParams, LEADER_URL, LEGACY_LEADER_URL, null);
     if (!indexFetchLock.tryLock())
       return IndexFetchResult.LOCK_OBTAIN_FAILED;
     if (core.getCoreContainer().isShutDown()) {
@@ -416,7 +446,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
       return IndexFetchResult.CONTAINER_IS_SHUTTING_DOWN; 
     }
     try {
-      if (masterUrl != null) {
+      if (leaderUrl != null) {
         if (currentIndexFetcher != null && currentIndexFetcher != pollingIndexFetcher) {
           currentIndexFetcher.destroy();
         }
@@ -828,7 +858,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
       log.info("inside disable poll, value of pollDisabled = {}", pollDisabled);
       rsp.add(STATUS, OK_STATUS);
     } else {
-      reportErrorOnResponse(rsp, "No slave configured", null);
+      reportErrorOnResponse(rsp, "No follower configured", null);
     }
   }
 
@@ -838,7 +868,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
       log.info("inside enable poll, value of pollDisabled = {}", pollDisabled);
       rsp.add(STATUS, OK_STATUS);
     } else {
-      reportErrorOnResponse(rsp, "No slave configured", null);
+      reportErrorOnResponse(rsp, "No follower configured", null);
     }
   }
 
@@ -873,7 +903,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
 
   @Override
   public String getDescription() {
-    return "ReplicationHandler provides replication of index and configuration files from Master to Slaves";
+    return "ReplicationHandler provides replication of index and configuration files from Leader to Followers";
   }
 
   /**
@@ -899,14 +929,14 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
         true, GENERATION, getCategory().toString(), scope);
     solrMetricsContext.gauge(this, () -> (core != null && !core.isClosed() ? core.getIndexDir() : ""),
         true, "indexPath", getCategory().toString(), scope);
-    solrMetricsContext.gauge(this, () -> isMaster,
+    solrMetricsContext.gauge(this, () -> isLeader,
          true, "isMaster", getCategory().toString(), scope);
-    solrMetricsContext.gauge(this, () -> isSlave,
+    solrMetricsContext.gauge(this, () -> isFollower,
          true, "isSlave", getCategory().toString(), scope);
     final MetricsMap fetcherMap = new MetricsMap((detailed, map) -> {
       IndexFetcher fetcher = currentIndexFetcher;
       if (fetcher != null) {
-        map.put(MASTER_URL, fetcher.getMasterUrl());
+        map.put(LEGACY_LEADER_URL, fetcher.getLeaderUrl());
         if (getPollInterval() != null) {
           map.put(POLL_INTERVAL, getPollInterval());
         }
@@ -932,11 +962,11 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
       }
     });
     solrMetricsContext.gauge(this , fetcherMap, true, "fetcher", getCategory().toString(), scope);
-    solrMetricsContext.gauge(this, () -> isMaster && includeConfFiles != null ? includeConfFiles : "",
+    solrMetricsContext.gauge(this, () -> isLeader && includeConfFiles != null ? includeConfFiles : "",
          true, "confFilesToReplicate", getCategory().toString(), scope);
-    solrMetricsContext.gauge(this, () -> isMaster ? getReplicateAfterStrings() : Collections.<String>emptyList(),
+    solrMetricsContext.gauge(this, () -> isLeader ? getReplicateAfterStrings() : Collections.<String>emptyList(),
         true, REPLICATE_AFTER, getCategory().toString(), scope);
-    solrMetricsContext.gauge(this,  () -> isMaster && replicationEnabled.get(),
+    solrMetricsContext.gauge(this,  () -> isLeader && replicationEnabled.get(),
         true, "replicationEnabled", getCategory().toString(), scope);
   }
 
@@ -944,76 +974,76 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
   /**
    * Used for showing statistics and progress information.
    */
-  private NamedList<Object> getReplicationDetails(SolrQueryResponse rsp, boolean showSlaveDetails) {
+  private NamedList<Object> getReplicationDetails(SolrQueryResponse rsp, boolean showFollowerDetails) {
     NamedList<Object> details = new SimpleOrderedMap<>();
-    NamedList<Object> master = new SimpleOrderedMap<>();
-    NamedList<Object> slave = new SimpleOrderedMap<>();
+    NamedList<Object> leader = new SimpleOrderedMap<>();
+    NamedList<Object> follower = new SimpleOrderedMap<>();
 
     details.add("indexSize", NumberUtils.readableSize(core.getIndexSize()));
     details.add("indexPath", core.getIndexDir());
     details.add(CMD_SHOW_COMMITS, getCommits());
-    details.add("isMaster", String.valueOf(isMaster));
-    details.add("isSlave", String.valueOf(isSlave));
+    details.add("isMaster", String.valueOf(isLeader));
+    details.add("isSlave", String.valueOf(isFollower));
     CommitVersionInfo vInfo = getIndexVersion();
     details.add("indexVersion", null == vInfo ? 0 : vInfo.version);
     details.add(GENERATION, null == vInfo ? 0 : vInfo.generation);
 
     IndexCommit commit = indexCommitPoint;  // make a copy so it won't change
 
-    if (isMaster) {
-      if (includeConfFiles != null) master.add(CONF_FILES, includeConfFiles);
-      master.add(REPLICATE_AFTER, getReplicateAfterStrings());
-      master.add("replicationEnabled", String.valueOf(replicationEnabled.get()));
+    if (isLeader) {
+      if (includeConfFiles != null) leader.add(CONF_FILES, includeConfFiles);
+      leader.add(REPLICATE_AFTER, getReplicateAfterStrings());
+      leader.add("replicationEnabled", String.valueOf(replicationEnabled.get()));
     }
 
-    if (isMaster && commit != null) {
+    if (isLeader && commit != null) {
       CommitVersionInfo repCommitInfo = CommitVersionInfo.build(commit);
-      master.add("replicableVersion", repCommitInfo.version);
-      master.add("replicableGeneration", repCommitInfo.generation);
+      leader.add("replicableVersion", repCommitInfo.version);
+      leader.add("replicableGeneration", repCommitInfo.generation);
     }
 
     IndexFetcher fetcher = currentIndexFetcher;
     if (fetcher != null) {
       Properties props = loadReplicationProperties();
-      if (showSlaveDetails) {
+      if (showFollowerDetails) {
         try {
           @SuppressWarnings({"rawtypes"})
           NamedList nl = fetcher.getDetails();
-          slave.add("masterDetails", nl.get(CMD_DETAILS));
+          follower.add("masterDetails", nl.get(CMD_DETAILS));
         } catch (Exception e) {
           log.warn(
-              "Exception while invoking 'details' method for replication on master ",
+              "Exception while invoking 'details' method for replication on leader ",
               e);
-          slave.add(ERR_STATUS, "invalid_master");
+          follower.add(ERR_STATUS, "invalid_leader");
         }
       }
-      slave.add(MASTER_URL, fetcher.getMasterUrl());
+      follower.add(LEGACY_LEADER_URL, fetcher.getLeaderUrl());
       if (getPollInterval() != null) {
-        slave.add(POLL_INTERVAL, getPollInterval());
+        follower.add(POLL_INTERVAL, getPollInterval());
       }
       Date nextScheduled = getNextScheduledExecTime();
       if (nextScheduled != null && !isPollingDisabled()) {
-        slave.add(NEXT_EXECUTION_AT, nextScheduled.toString());
+        follower.add(NEXT_EXECUTION_AT, nextScheduled.toString());
       } else if (isPollingDisabled()) {
-        slave.add(NEXT_EXECUTION_AT, "Polling disabled");
+        follower.add(NEXT_EXECUTION_AT, "Polling disabled");
       }
-      addVal(slave, IndexFetcher.INDEX_REPLICATED_AT, props, Date.class);
-      addVal(slave, IndexFetcher.INDEX_REPLICATED_AT_LIST, props, List.class);
-      addVal(slave, IndexFetcher.REPLICATION_FAILED_AT_LIST, props, List.class);
-      addVal(slave, IndexFetcher.TIMES_INDEX_REPLICATED, props, Integer.class);
-      addVal(slave, IndexFetcher.CONF_FILES_REPLICATED, props, Integer.class);
-      addVal(slave, IndexFetcher.TIMES_CONFIG_REPLICATED, props, Integer.class);
-      addVal(slave, IndexFetcher.CONF_FILES_REPLICATED_AT, props, Integer.class);
-      addVal(slave, IndexFetcher.LAST_CYCLE_BYTES_DOWNLOADED, props, Long.class);
-      addVal(slave, IndexFetcher.TIMES_FAILED, props, Integer.class);
-      addVal(slave, IndexFetcher.REPLICATION_FAILED_AT, props, Date.class);
-      addVal(slave, IndexFetcher.PREVIOUS_CYCLE_TIME_TAKEN, props, Long.class);
-      addVal(slave, IndexFetcher.CLEARED_LOCAL_IDX, props, Long.class);
-
-      slave.add("currentDate", new Date().toString());
-      slave.add("isPollingDisabled", String.valueOf(isPollingDisabled()));
+      addVal(follower, IndexFetcher.INDEX_REPLICATED_AT, props, Date.class);
+      addVal(follower, IndexFetcher.INDEX_REPLICATED_AT_LIST, props, List.class);
+      addVal(follower, IndexFetcher.REPLICATION_FAILED_AT_LIST, props, List.class);
+      addVal(follower, IndexFetcher.TIMES_INDEX_REPLICATED, props, Integer.class);
+      addVal(follower, IndexFetcher.CONF_FILES_REPLICATED, props, Integer.class);
+      addVal(follower, IndexFetcher.TIMES_CONFIG_REPLICATED, props, Integer.class);
+      addVal(follower, IndexFetcher.CONF_FILES_REPLICATED_AT, props, Integer.class);
+      addVal(follower, IndexFetcher.LAST_CYCLE_BYTES_DOWNLOADED, props, Long.class);
+      addVal(follower, IndexFetcher.TIMES_FAILED, props, Integer.class);
+      addVal(follower, IndexFetcher.REPLICATION_FAILED_AT, props, Date.class);
+      addVal(follower, IndexFetcher.PREVIOUS_CYCLE_TIME_TAKEN, props, Long.class);
+      addVal(follower, IndexFetcher.CLEARED_LOCAL_IDX, props, Long.class);
+
+      follower.add("currentDate", new Date().toString());
+      follower.add("isPollingDisabled", String.valueOf(isPollingDisabled()));
       boolean isReplicating = isReplicating();
-      slave.add("isReplicating", String.valueOf(isReplicating));
+      follower.add("isReplicating", String.valueOf(isReplicating));
       if (isReplicating) {
         try {
           long bytesToDownload = 0;
@@ -1029,9 +1059,9 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
             bytesToDownload += (Long) file.get(SIZE);
           }
 
-          slave.add("filesToDownload", filesToDownload);
-          slave.add("numFilesToDownload", String.valueOf(filesToDownload.size()));
-          slave.add("bytesToDownload", NumberUtils.readableSize(bytesToDownload));
+          follower.add("filesToDownload", filesToDownload);
+          follower.add("numFilesToDownload", String.valueOf(filesToDownload.size()));
+          follower.add("bytesToDownload", NumberUtils.readableSize(bytesToDownload));
 
           long bytesDownloaded = 0;
           List<String> filesDownloaded = new ArrayList<>();
@@ -1060,17 +1090,17 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
                 percentDownloaded = (currFileSizeDownloaded * 100) / currFileSize;
             }
           }
-          slave.add("filesDownloaded", filesDownloaded);
-          slave.add("numFilesDownloaded", String.valueOf(filesDownloaded.size()));
+          follower.add("filesDownloaded", filesDownloaded);
+          follower.add("numFilesDownloaded", String.valueOf(filesDownloaded.size()));
 
           long estimatedTimeRemaining = 0;
 
           Date replicationStartTimeStamp = fetcher.getReplicationStartTimeStamp();
           if (replicationStartTimeStamp != null) {
-            slave.add("replicationStartTime", replicationStartTimeStamp.toString());
+            follower.add("replicationStartTime", replicationStartTimeStamp.toString());
           }
           long elapsed = fetcher.getReplicationTimeElapsed();
-          slave.add("timeElapsed", String.valueOf(elapsed) + "s");
+          follower.add("timeElapsed", String.valueOf(elapsed) + "s");
 
           if (bytesDownloaded > 0)
             estimatedTimeRemaining = ((bytesToDownload - bytesDownloaded) * elapsed) / bytesDownloaded;
@@ -1081,24 +1111,24 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
           if (elapsed > 0)
             downloadSpeed = (bytesDownloaded / elapsed);
           if (currFile != null)
-            slave.add("currentFile", currFile);
-          slave.add("currentFileSize", NumberUtils.readableSize(currFileSize));
-          slave.add("currentFileSizeDownloaded", NumberUtils.readableSize(currFileSizeDownloaded));
-          slave.add("currentFileSizePercent", String.valueOf(percentDownloaded));
-          slave.add("bytesDownloaded", NumberUtils.readableSize(bytesDownloaded));
-          slave.add("totalPercent", String.valueOf(totalPercent));
-          slave.add("timeRemaining", String.valueOf(estimatedTimeRemaining) + "s");
-          slave.add("downloadSpeed", NumberUtils.readableSize(downloadSpeed));
+            follower.add("currentFile", currFile);
+          follower.add("currentFileSize", NumberUtils.readableSize(currFileSize));
+          follower.add("currentFileSizeDownloaded", NumberUtils.readableSize(currFileSizeDownloaded));
+          follower.add("currentFileSizePercent", String.valueOf(percentDownloaded));
+          follower.add("bytesDownloaded", NumberUtils.readableSize(bytesDownloaded));
+          follower.add("totalPercent", String.valueOf(totalPercent));
+          follower.add("timeRemaining", String.valueOf(estimatedTimeRemaining) + "s");
+          follower.add("downloadSpeed", NumberUtils.readableSize(downloadSpeed));
         } catch (Exception e) {
           log.error("Exception while writing replication details: ", e);
         }
       }
     }
 
-    if (isMaster)
-      details.add("master", master);
-    if (slave.size() > 0)
-      details.add("slave", slave);
+    if (isLeader)
+      details.add("master", leader);
+    if (follower.size() > 0)
+      details.add("slave", follower);
 
     @SuppressWarnings({"rawtypes"})
     NamedList snapshotStats = snapShootDetails;
@@ -1232,7 +1262,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
   }
 
   @Override
-  @SuppressWarnings({"unchecked", "resource"})
+  @SuppressWarnings({"resource"})
   public void inform(SolrCore core) {
     this.core = core;
     registerCloseHook();
@@ -1244,33 +1274,33 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
       numberBackupsToKeep = 0;
     }
     @SuppressWarnings({"rawtypes"})
-    NamedList slave = (NamedList) initArgs.get("slave");
-    boolean enableSlave = isEnabled( slave );
-    if (enableSlave) {
-      currentIndexFetcher = pollingIndexFetcher = new IndexFetcher(slave, this, core);
-      setupPolling((String) slave.get(POLL_INTERVAL));
-      isSlave = true;
+    NamedList follower = getObjectWithBackwardCompatibility(initArgs,  "follower",  "slave");
+    boolean enableFollower = isEnabled( follower );
+    if (enableFollower) {
+      currentIndexFetcher = pollingIndexFetcher = new IndexFetcher(follower, this, core);
+      setupPolling((String) follower.get(POLL_INTERVAL));
+      isFollower = true;
     }
     @SuppressWarnings({"rawtypes"})
-    NamedList master = (NamedList) initArgs.get("master");
-    boolean enableMaster = isEnabled( master );
+    NamedList leader = getObjectWithBackwardCompatibility(initArgs, "leader", "master");
+    boolean enableLeader = isEnabled( leader );
 
-    if (enableMaster || (enableSlave && !currentIndexFetcher.fetchFromLeader)) {
+    if (enableLeader || (enableFollower && !currentIndexFetcher.fetchFromLeader)) {
       if (core.getCoreContainer().getZkController() != null) {
         log.warn("SolrCloud is enabled for core {} but so is old-style replication. "
                 + "Make sure you intend this behavior, it usually indicates a mis-configuration. "
-                + "Master setting is {} and slave setting is {}"
-        , core.getName(), enableMaster, enableSlave);
+                + "Leader setting is {} and follower setting is {}"
+        , core.getName(), enableLeader, enableFollower);
       }
     }
 
-    if (!enableSlave && !enableMaster) {
-      enableMaster = true;
-      master = new NamedList<>();
+    if (!enableFollower && !enableLeader) {
+      enableLeader = true;
+      leader = new NamedList<>();
     }
 
-    if (enableMaster) {
-      includeConfFiles = (String) master.get(CONF_FILES);
+    if (enableLeader) {
+      includeConfFiles = (String) leader.get(CONF_FILES);
       if (includeConfFiles != null && includeConfFiles.trim().length() > 0) {
         List<String> files = Arrays.asList(includeConfFiles.split(","));
         for (String file : files) {
@@ -1282,11 +1312,11 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
         log.info("Replication enabled for following config files: {}", includeConfFiles);
       }
       @SuppressWarnings({"rawtypes"})
-      List backup = master.getAll("backupAfter");
+      List backup = leader.getAll("backupAfter");
       boolean backupOnCommit = backup.contains("commit");
       boolean backupOnOptimize = !backupOnCommit && backup.contains("optimize");
       @SuppressWarnings({"rawtypes"})
-      List replicateAfter = master.getAll(REPLICATE_AFTER);
+      List replicateAfter = leader.getAll(REPLICATE_AFTER);
       replicateOnCommit = replicateAfter.contains("commit");
       replicateOnOptimize = !replicateOnCommit && replicateAfter.contains("optimize");
 
@@ -1354,7 +1384,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
           if (s!=null) s.decref();
         }
       }
-      String reserve = (String) master.get(RESERVE);
+      String reserve = (String) leader.get(RESERVE);
       if (reserve != null && !reserve.trim().equals("")) {
         reserveCommitDuration = readIntervalMs(reserve);
         deprecatedReserveCommitDuration = reserveCommitDuration;
@@ -1364,7 +1394,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
           (null == reserve),
           core.getSolrConfig().luceneMatchVersion.onOrAfter(Version.LUCENE_7_1_0));
       }
-      isMaster = true;
+      isLeader = true;
     }
 
     {
@@ -1379,7 +1409,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
     log.info("Commits will be reserved for {} ms", reserveCommitDuration);
   }
 
-  // check master or slave is enabled
+  // check leader or follower is enabled
   private boolean isEnabled( @SuppressWarnings({"rawtypes"})NamedList params ){
     if( params == null ) return false;
     Object enable = params.get( "enable" );
@@ -1784,13 +1814,19 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
 
   private static final String EXCEPTION = "exception";
 
-  public static final String MASTER_URL = "masterUrl";
+  public static final String LEADER_URL = "leaderUrl";
+  @Deprecated
+  /** @deprecated: Only used for backwards compatibility. Use {@link #LEADER_URL} */
+  public static final String LEGACY_LEADER_URL = "masterUrl";
 
   public static final String FETCH_FROM_LEADER = "fetchFromLeader";
 
-  // in case of TLOG replica, if masterVersion = zero, don't do commit
+  // in case of TLOG replica, if leaderVersion = zero, don't do commit
   // otherwise updates from current tlog won't copied over properly to the new tlog, leading to data loss
-  public static final String SKIP_COMMIT_ON_MASTER_VERSION_ZERO = "skipCommitOnMasterVersionZero";
+  public static final String SKIP_COMMIT_ON_LEADER_VERSION_ZERO = "skipCommitOnLeaderVersionZero";
+  @Deprecated
+  /** @deprecated: Only used for backwards compatibility. Use {@link #SKIP_COMMIT_ON_LEADER_VERSION_ZERO} */
+  public static final String LEGACY_SKIP_COMMIT_ON_LEADER_VERSION_ZERO = "skipCommitOnMasterVersionZero";
 
   public static final String STATUS = "status";
 
diff --git a/solr/core/src/java/org/apache/solr/handler/component/FacetComponent.java b/solr/core/src/java/org/apache/solr/handler/component/FacetComponent.java
index 698525a..ecce09f 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/FacetComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/FacetComponent.java
@@ -1033,9 +1033,9 @@ public class FacetComponent extends SearchComponent {
       }
       
       for (Entry<String,List<NamedList<Object>>> pivotFacetResponseFromShard : pivotFacetResponsesFromShard) {
-        PivotFacet masterPivotFacet = fi.pivotFacets.get(pivotFacetResponseFromShard.getKey());
-        masterPivotFacet.mergeResponseFromShard(shardNumber, rb, pivotFacetResponseFromShard.getValue());  
-        masterPivotFacet.removeAllRefinementsForShard(shardNumber);
+        PivotFacet aggregatedPivotFacet = fi.pivotFacets.get(pivotFacetResponseFromShard.getKey());
+        aggregatedPivotFacet.mergeResponseFromShard(shardNumber, rb, pivotFacetResponseFromShard.getValue());
+        aggregatedPivotFacet.removeAllRefinementsForShard(shardNumber);
       }
     }
     
diff --git a/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java b/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java
index 4a34265..776a140 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/StatsValuesFactory.java
@@ -183,7 +183,7 @@ abstract class AbstractStatsValues<T> implements StatsValues {
     // "NumericValueSourceStatsValues" which would have diff parent classes
     //
     // part of the complexity here being that the StatsValues API serves two
-    // masters: collecting concrete Values from things like DocValuesStats and
+    // leaders: collecting concrete Values from things like DocValuesStats and
     // the distributed aggregation logic, but also collecting docIds which it
     // then
     // uses to go out and pull concreate values from the ValueSource
diff --git a/solr/core/src/java/org/apache/solr/search/stats/ExactSharedStatsCache.java b/solr/core/src/java/org/apache/solr/search/stats/ExactSharedStatsCache.java
index 01c479b..0b1d265 100644
--- a/solr/core/src/java/org/apache/solr/search/stats/ExactSharedStatsCache.java
+++ b/solr/core/src/java/org/apache/solr/search/stats/ExactSharedStatsCache.java
@@ -40,7 +40,7 @@ public class ExactSharedStatsCache extends ExactStatsCache {
   // local stats obtained from shard servers
   private final Map<String,Map<String,TermStats>> perShardTermStats = new ConcurrentHashMap<>();
   private final Map<String,Map<String,CollectionStats>> perShardColStats = new ConcurrentHashMap<>();
-  // global stats synchronized from the master
+  // global stats synchronized from the leader
   private final Map<String,TermStats> currentGlobalTermStats = new ConcurrentHashMap<>();
   private final Map<String,CollectionStats> currentGlobalColStats = new ConcurrentHashMap<>();
 
diff --git a/solr/core/src/java/org/apache/solr/search/stats/LRUStatsCache.java b/solr/core/src/java/org/apache/solr/search/stats/LRUStatsCache.java
index e02eb19..7406ad6 100644
--- a/solr/core/src/java/org/apache/solr/search/stats/LRUStatsCache.java
+++ b/solr/core/src/java/org/apache/solr/search/stats/LRUStatsCache.java
@@ -65,7 +65,7 @@ public class LRUStatsCache extends ExactStatsCache {
   // map of <shardName, <field, collStats>>
   private final Map<String,Map<String,CollectionStats>> perShardColStats = new ConcurrentHashMap<>();
   
-  // global stats synchronized from the master
+  // global stats synchronized from the leader
 
   // cache of <term, termStats>
   private final FastLRUCache<String,TermStats> currentGlobalTermStats = new FastLRUCache<>();
diff --git a/solr/core/src/java/org/apache/solr/search/stats/StatsCache.java b/solr/core/src/java/org/apache/solr/search/stats/StatsCache.java
index 238bb12..3049fb6 100644
--- a/solr/core/src/java/org/apache/solr/search/stats/StatsCache.java
+++ b/solr/core/src/java/org/apache/solr/search/stats/StatsCache.java
@@ -176,7 +176,7 @@ public abstract class StatsCache implements PluginInfoInitialized {
   protected abstract void doMergeToGlobalStats(SolrQueryRequest req, List<ShardResponse> responses);
 
   /**
-   * Receive global stats data from the master and update a local cache of global stats
+   * Receive global stats data from the leader and update a local cache of global stats
    * with this global data. This event occurs either as a separate request, or
    * together with the regular query request, in which case this method is
    * called first, before preparing a {@link QueryCommand} to be submitted to
diff --git a/solr/core/src/java/org/apache/solr/security/JWTVerificationkeyResolver.java b/solr/core/src/java/org/apache/solr/security/JWTVerificationkeyResolver.java
index 3aca77c..4a0ada0 100644
--- a/solr/core/src/java/org/apache/solr/security/JWTVerificationkeyResolver.java
+++ b/solr/core/src/java/org/apache/solr/security/JWTVerificationkeyResolver.java
@@ -106,7 +106,7 @@ public class JWTVerificationkeyResolver implements VerificationKeyResolver {
         }
       }
 
-      // Add all keys into a master list
+      // Add all keys into a leader list
       if (issuerConfig.usesHttpsJwk()) {
         keysSource = "[" + String.join(", ", issuerConfig.getJwksUrls()) + "]";
         for (HttpsJwks hjwks : issuerConfig.getHttpsJwks()) {
diff --git a/solr/core/src/java/org/apache/solr/util/TestInjection.java b/solr/core/src/java/org/apache/solr/util/TestInjection.java
index 2656155..3ae2349 100644
--- a/solr/core/src/java/org/apache/solr/util/TestInjection.java
+++ b/solr/core/src/java/org/apache/solr/util/TestInjection.java
@@ -143,7 +143,7 @@ public class TestInjection {
 
   private volatile static AtomicInteger countPrepRecoveryOpPauseForever = new AtomicInteger(0);
 
-  public volatile static Integer delayBeforeSlaveCommitRefresh=null;
+  public volatile static Integer delayBeforeFollowerCommitRefresh=null;
 
   public volatile static Integer delayInExecutePlanAction=null;
 
@@ -185,7 +185,7 @@ public class TestInjection {
     countPrepRecoveryOpPauseForever = new AtomicInteger(0);
     failIndexFingerprintRequests = null;
     wrongIndexFingerprint = null;
-    delayBeforeSlaveCommitRefresh = null;
+    delayBeforeFollowerCommitRefresh = null;
     delayInExecutePlanAction = null;
     failInExecutePlanAction = false;
     skipIndexWriterCommitOnClose = false;
@@ -521,11 +521,11 @@ public class TestInjection {
     return new Pair<>(Boolean.parseBoolean(val), Integer.parseInt(percent));
   }
 
-  public static boolean injectDelayBeforeSlaveCommitRefresh() {
-    if (delayBeforeSlaveCommitRefresh!=null) {
+  public static boolean injectDelayBeforeFollowerCommitRefresh() {
+    if (delayBeforeFollowerCommitRefresh!=null) {
       try {
-        log.info("Pausing IndexFetcher for {}ms", delayBeforeSlaveCommitRefresh);
-        Thread.sleep(delayBeforeSlaveCommitRefresh);
+        log.info("Pausing IndexFetcher for {}ms", delayBeforeFollowerCommitRefresh);
+        Thread.sleep(delayBeforeFollowerCommitRefresh);
       } catch (InterruptedException e) {
         Thread.currentThread().interrupt();
       }
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-slave.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-follower.xml
similarity index 95%
copy from solr/core/src/test-files/solr/collection1/conf/solrconfig-slave.xml
copy to solr/core/src/test-files/solr/collection1/conf/solrconfig-follower.xml
index 39a7870..217d568 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-slave.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-follower.xml
@@ -42,8 +42,8 @@
   </requestHandler>
 
   <requestHandler name="/replication" class="solr.ReplicationHandler">
-    <lst name="slave">
-      <str name="masterUrl">http://127.0.0.1:TEST_PORT/solr/collection1</str>
+    <lst name="follower">
+      <str name="leaderUrl">http://127.0.0.1:TEST_PORT/solr/collection1</str>
       <str name="pollInterval">00:00:01</str>
       <str name="compression">COMPRESSION</str>
     </lst>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-slave1.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-follower1.xml
similarity index 100%
rename from solr/core/src/test-files/solr/collection1/conf/solrconfig-slave1.xml
rename to solr/core/src/test-files/solr/collection1/conf/solrconfig-follower1.xml
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-master-throttled.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader-throttled.xml
similarity index 100%
rename from solr/core/src/test-files/solr/collection1/conf/solrconfig-master-throttled.xml
rename to solr/core/src/test-files/solr/collection1/conf/solrconfig-leader-throttled.xml
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-master.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader.xml
similarity index 99%
rename from solr/core/src/test-files/solr/collection1/conf/solrconfig-master.xml
rename to solr/core/src/test-files/solr/collection1/conf/solrconfig-leader.xml
index e501af2..374c413 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-master.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader.xml
@@ -33,7 +33,7 @@
   </requestHandler>
 
   <requestHandler name="/replication" class="solr.ReplicationHandler">
-    <lst name="master">
+    <lst name="leader">
       <str name="replicateAfter">commit</str>
       <!-- we don't really need dummy.xsl, but we want to be sure subdir 
            files replicate (see SOLR-3809)
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-master1-keepOneBackup.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader1-keepOneBackup.xml
similarity index 98%
rename from solr/core/src/test-files/solr/collection1/conf/solrconfig-master1-keepOneBackup.xml
rename to solr/core/src/test-files/solr/collection1/conf/solrconfig-leader1-keepOneBackup.xml
index bcd7874..101ba30 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-master1-keepOneBackup.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader1-keepOneBackup.xml
@@ -30,7 +30,7 @@
   <requestHandler name="/select" class="solr.SearchHandler" />
 
   <requestHandler name="/replication" class="solr.ReplicationHandler">
-    <lst name="master">
+    <lst name="leader">
       <str name="replicateAfter">commit</str>
       <str name="confFiles">schema-replication2.xml:schema.xml</str>
       <str name="backupAfter">commit</str>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-master1.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader1.xml
similarity index 99%
rename from solr/core/src/test-files/solr/collection1/conf/solrconfig-master1.xml
rename to solr/core/src/test-files/solr/collection1/conf/solrconfig-leader1.xml
index 9271686..0c3eb86 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-master1.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader1.xml
@@ -33,7 +33,7 @@
   </requestHandler>
 
   <requestHandler name="/replication" class="solr.ReplicationHandler">
-    <lst name="master">
+    <lst name="leader">
       <str name="replicateAfter">commit</str>
       <str name="backupAfter">commit</str>
       <str name="confFiles">schema-replication2.xml:schema.xml</str>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-master2.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader2.xml
similarity index 99%
rename from solr/core/src/test-files/solr/collection1/conf/solrconfig-master2.xml
rename to solr/core/src/test-files/solr/collection1/conf/solrconfig-leader2.xml
index 55301c2..5eca462 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-master2.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader2.xml
@@ -32,7 +32,7 @@
   </requestHandler>
 
   <requestHandler name="/replication" class="solr.ReplicationHandler">
-    <lst name="master">
+    <lst name="leader">
       <str name="replicateAfter">startup</str>
       <str name="confFiles">schema.xml</str>
     </lst>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-master3.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader3.xml
similarity index 99%
rename from solr/core/src/test-files/solr/collection1/conf/solrconfig-master3.xml
rename to solr/core/src/test-files/solr/collection1/conf/solrconfig-leader3.xml
index 1c1dd40..5d97350 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-master3.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-leader3.xml
@@ -32,7 +32,7 @@
   </requestHandler>
 
   <requestHandler name="/replication" class="solr.ReplicationHandler">
-    <lst name="master">
+    <lst name="leader">
       <str name="replicateAfter">commit</str>
       <str name="replicateAfter">startup</str>
       <str name="confFiles">schema.xml</str>
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-repeater.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-repeater.xml
index 761d45d..160bc4e 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-repeater.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-repeater.xml
@@ -42,12 +42,12 @@
   </requestHandler>
 
   <requestHandler name="/replication" class="solr.ReplicationHandler">
-    <lst name="master">
+    <lst name="leader">
       <str name="replicateAfter">commit</str>
       <str name="confFiles">schema.xml</str>
     </lst>
-    <lst name="slave">
-      <str name="masterUrl">http://127.0.0.1:TEST_PORT/solr/replication</str>
+    <lst name="follower">
+      <str name="leaderUrl">http://127.0.0.1:TEST_PORT/solr/replication</str>
     </lst>
   </requestHandler>
 
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-slave.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-replication-legacy.xml
similarity index 96%
rename from solr/core/src/test-files/solr/collection1/conf/solrconfig-slave.xml
rename to solr/core/src/test-files/solr/collection1/conf/solrconfig-replication-legacy.xml
index 39a7870..c2f25ba 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-slave.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-replication-legacy.xml
@@ -42,6 +42,9 @@
   </requestHandler>
 
   <requestHandler name="/replication" class="solr.ReplicationHandler">
+    <lst name="master">
+      <str name="replicateAfter">commit</str>
+    </lst>
     <lst name="slave">
       <str name="masterUrl">http://127.0.0.1:TEST_PORT/solr/collection1</str>
       <str name="pollInterval">00:00:01</str>
diff --git a/solr/core/src/test/org/apache/solr/analysis/TestDeprecatedFilters.java b/solr/core/src/test/org/apache/solr/analysis/TestDeprecatedFilters.java
index 120fda1..fea1ca8 100644
--- a/solr/core/src/test/org/apache/solr/analysis/TestDeprecatedFilters.java
+++ b/solr/core/src/test/org/apache/solr/analysis/TestDeprecatedFilters.java
@@ -24,7 +24,7 @@ public class TestDeprecatedFilters extends SolrTestCaseJ4 {
 
   @BeforeClass
   public static void beforeClass() throws Exception {
-    initCore("solrconfig-master.xml","schema-deprecations.xml");
+    initCore("solrconfig-leader.xml","schema-deprecations.xml");
   }
 
   public void testLowerCaseTokenizer() {
diff --git a/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrReplicationHandlerTest.java b/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrReplicationHandlerTest.java
index 7bd371f..271bf67 100644
--- a/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrReplicationHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrReplicationHandlerTest.java
@@ -59,14 +59,14 @@ public class CdcrReplicationHandlerTest extends BaseCdcrDistributedZkTest {
   }
 
   /**
-   * Test the scenario where the slave is killed from the start. The replication
+   * Test the scenario where the follower is killed from the start. The replication
    * strategy should fetch all the missing tlog files from the leader.
    */
   @Test
   @ShardsFixed(num = 2)
   public void testFullReplication() throws Exception {
-    List<CloudJettyRunner> slaves = this.getShardToSlaveJetty(SOURCE_COLLECTION, SHARD1);
-    slaves.get(0).jetty.stop();
+    List<CloudJettyRunner> followers = this.getShardToFollowerJetty(SOURCE_COLLECTION, SHARD1);
+    followers.get(0).jetty.stop();
 
     for (int i = 0; i < 10; i++) {
       List<SolrInputDocument> docs = new ArrayList<>();
@@ -78,14 +78,14 @@ public class CdcrReplicationHandlerTest extends BaseCdcrDistributedZkTest {
 
     assertNumDocs(100, SOURCE_COLLECTION);
 
-    // Restart the slave node to trigger Replication strategy
-    this.restartServer(slaves.get(0));
+    // Restart the follower node to trigger Replication strategy
+    this.restartServer(followers.get(0));
 
     this.assertUpdateLogsEquals(SOURCE_COLLECTION, 10);
   }
 
   /**
-   * Test the scenario where the slave is killed before receiving all the documents. The replication
+   * Test the scenario where the follower is killed before receiving all the documents. The replication
    * strategy should fetch all the missing tlog files from the leader.
    */
   @Test
@@ -99,8 +99,8 @@ public class CdcrReplicationHandlerTest extends BaseCdcrDistributedZkTest {
       index(SOURCE_COLLECTION, docs);
     }
 
-    List<CloudJettyRunner> slaves = this.getShardToSlaveJetty(SOURCE_COLLECTION, SHARD1);
-    slaves.get(0).jetty.stop();
+    List<CloudJettyRunner> followers = this.getShardToFollowerJetty(SOURCE_COLLECTION, SHARD1);
+    followers.get(0).jetty.stop();
 
     for (int i = 5; i < 10; i++) {
       List<SolrInputDocument> docs = new ArrayList<>();
@@ -112,32 +112,32 @@ public class CdcrReplicationHandlerTest extends BaseCdcrDistributedZkTest {
 
     assertNumDocs(200, SOURCE_COLLECTION);
 
-    // Restart the slave node to trigger Replication strategy
-    this.restartServer(slaves.get(0));
+    // Restart the follower node to trigger Replication strategy
+    this.restartServer(followers.get(0));
 
-    // at this stage, the slave should have replicated the 5 missing tlog files
+    // at this stage, the follower should have replicated the 5 missing tlog files
     this.assertUpdateLogsEquals(SOURCE_COLLECTION, 10);
   }
 
   /**
-   * Test the scenario where the slave is killed before receiving a commit. This creates a truncated tlog
-   * file on the slave node. The replication strategy should detect this truncated file, and fetch the
+   * Test the scenario where the follower is killed before receiving a commit. This creates a truncated tlog
+   * file on the follower node. The replication strategy should detect this truncated file, and fetch the
    * non-truncated file from the leader.
    */
   @Test
   @ShardsFixed(num = 2)
   public void testPartialReplicationWithTruncatedTlog() throws Exception {
     CloudSolrClient client = createCloudClient(SOURCE_COLLECTION);
-    List<CloudJettyRunner> slaves = this.getShardToSlaveJetty(SOURCE_COLLECTION, SHARD1);
+    List<CloudJettyRunner> followers = this.getShardToFollowerJetty(SOURCE_COLLECTION, SHARD1);
 
     try {
       for (int i = 0; i < 10; i++) {
         for (int j = i * 20; j < (i * 20) + 20; j++) {
           client.add(getDoc(id, Integer.toString(j)));
 
-          // Stop the slave in the middle of a batch to create a truncated tlog on the slave
+          // Stop the follower in the middle of a batch to create a truncated tlog on the follower
           if (j == 45) {
-            slaves.get(0).jetty.stop();
+            followers.get(0).jetty.stop();
           }
 
         }
@@ -149,16 +149,16 @@ public class CdcrReplicationHandlerTest extends BaseCdcrDistributedZkTest {
 
     assertNumDocs(200, SOURCE_COLLECTION);
 
-    // Restart the slave node to trigger Replication recovery
-    this.restartServer(slaves.get(0));
+    // Restart the follower node to trigger Replication recovery
+    this.restartServer(followers.get(0));
 
-    // at this stage, the slave should have replicated the 5 missing tlog files
+    // at this stage, the follower should have replicated the 5 missing tlog files
     this.assertUpdateLogsEquals(SOURCE_COLLECTION, 10);
   }
 
   /**
-   * Test the scenario where the slave first recovered with a PeerSync strategy, then with a Replication strategy.
-   * The PeerSync strategy will generate a single tlog file for all the missing updates on the slave node.
+   * Test the scenario where the follower first recovered with a PeerSync strategy, then with a Replication strategy.
+   * The PeerSync strategy will generate a single tlog file for all the missing updates on the follower node.
    * If a Replication strategy occurs at a later stage, it should remove this tlog file generated by PeerSync
    * and fetch the corresponding tlog files from the leader.
    */
@@ -173,8 +173,8 @@ public class CdcrReplicationHandlerTest extends BaseCdcrDistributedZkTest {
       index(SOURCE_COLLECTION, docs);
     }
 
-    List<CloudJettyRunner> slaves = this.getShardToSlaveJetty(SOURCE_COLLECTION, SHARD1);
-    slaves.get(0).jetty.stop();
+    List<CloudJettyRunner> followers = this.getShardToFollowerJetty(SOURCE_COLLECTION, SHARD1);
+    followers.get(0).jetty.stop();
 
     for (int i = 5; i < 10; i++) {
       List<SolrInputDocument> docs = new ArrayList<>();
@@ -186,11 +186,11 @@ public class CdcrReplicationHandlerTest extends BaseCdcrDistributedZkTest {
 
     assertNumDocs(100, SOURCE_COLLECTION);
 
-    // Restart the slave node to trigger PeerSync recovery
-    // (the update windows between leader and slave is small enough)
-    this.restartServer(slaves.get(0));
+    // Restart the follower node to trigger PeerSync recovery
+    // (the update windows between leader and follower is small enough)
+    this.restartServer(followers.get(0));
 
-    slaves.get(0).jetty.stop();
+    followers.get(0).jetty.stop();
 
     for (int i = 10; i < 15; i++) {
       List<SolrInputDocument> docs = new ArrayList<>();
@@ -200,30 +200,30 @@ public class CdcrReplicationHandlerTest extends BaseCdcrDistributedZkTest {
       index(SOURCE_COLLECTION, docs);
     }
 
-    // restart the slave node to trigger Replication recovery
-    this.restartServer(slaves.get(0));
+    // restart the follower node to trigger Replication recovery
+    this.restartServer(followers.get(0));
 
-    // at this stage, the slave should have replicated the 5 missing tlog files
+    // at this stage, the follower should have replicated the 5 missing tlog files
     this.assertUpdateLogsEquals(SOURCE_COLLECTION, 15);
   }
 
   /**
-   * Test the scenario where the slave is killed while the leader is still receiving updates.
-   * The slave should buffer updates while in recovery, then replay them at the end of the recovery.
-   * If updates were properly buffered and replayed, then the slave should have the same number of documents
+   * Test the scenario where the follower is killed while the leader is still receiving updates.
+   * The follower should buffer updates while in recovery, then replay them at the end of the recovery.
+   * If updates were properly buffered and replayed, then the follower should have the same number of documents
    * than the leader. This checks if cdcr tlog replication interferes with buffered updates - SOLR-8263.
    */
   @Test
   @ShardsFixed(num = 2)
   public void testReplicationWithBufferedUpdates() throws Exception {
-    List<CloudJettyRunner> slaves = this.getShardToSlaveJetty(SOURCE_COLLECTION, SHARD1);
+    List<CloudJettyRunner> followers = this.getShardToFollowerJetty(SOURCE_COLLECTION, SHARD1);
 
     AtomicInteger numDocs = new AtomicInteger(0);
     ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(new SolrNamedThreadFactory("cdcr-test-update-scheduler"));
     executor.scheduleWithFixedDelay(new UpdateThread(numDocs), 10, 10, TimeUnit.MILLISECONDS);
 
-    // Restart the slave node to trigger Replication strategy
-    this.restartServer(slaves.get(0));
+    // Restart the follower node to trigger Replication strategy
+    this.restartServer(followers.get(0));
 
     // shutdown the update thread and wait for its completion
     executor.shutdown();
@@ -232,8 +232,8 @@ public class CdcrReplicationHandlerTest extends BaseCdcrDistributedZkTest {
     // check that we have the expected number of documents in the cluster
     assertNumDocs(numDocs.get(), SOURCE_COLLECTION);
 
-    // check that we have the expected number of documents on the slave
-    assertNumDocs(numDocs.get(), slaves.get(0));
+    // check that we have the expected number of documents on the follower
+    assertNumDocs(numDocs.get(), followers.get(0));
   }
 
   private void assertNumDocs(int expectedNumDocs, CloudJettyRunner jetty)
@@ -287,7 +287,7 @@ public class CdcrReplicationHandlerTest extends BaseCdcrDistributedZkTest {
 
   }
 
-  private List<CloudJettyRunner> getShardToSlaveJetty(String collection, String shard) {
+  private List<CloudJettyRunner> getShardToFollowerJetty(String collection, String shard) {
     List<CloudJettyRunner> jetties = new ArrayList<>(shardToJetty.get(collection).get(shard));
     CloudJettyRunner leader = shardToLeaderJetty.get(collection).get(shard);
     jetties.remove(leader);
@@ -295,7 +295,7 @@ public class CdcrReplicationHandlerTest extends BaseCdcrDistributedZkTest {
   }
 
   /**
-   * Asserts that the update logs are in sync between the leader and slave. The leader and the slaves
+   * Asserts that the update logs are in sync between the leader and follower. The leader and the followers
    * must have identical tlog files.
    */
   protected void assertUpdateLogsEquals(String collection, int numberOfTLogs) throws Exception {
@@ -304,14 +304,14 @@ public class CdcrReplicationHandlerTest extends BaseCdcrDistributedZkTest {
 
     for (String shard : shardToCoresMap.keySet()) {
       Map<Long, Long> leaderFilesMeta = this.getFilesMeta(info.getLeader(shard).ulogDir);
-      Map<Long, Long> slaveFilesMeta = this.getFilesMeta(info.getReplicas(shard).get(0).ulogDir);
+      Map<Long, Long> followerFilesMeta = this.getFilesMeta(info.getReplicas(shard).get(0).ulogDir);
 
       assertEquals("Incorrect number of tlog files on the leader", numberOfTLogs, leaderFilesMeta.size());
-      assertEquals("Incorrect number of tlog files on the slave", numberOfTLogs, slaveFilesMeta.size());
+      assertEquals("Incorrect number of tlog files on the follower", numberOfTLogs, followerFilesMeta.size());
 
       for (Long leaderFileVersion : leaderFilesMeta.keySet()) {
-        assertTrue("Slave is missing a tlog for version " + leaderFileVersion, slaveFilesMeta.containsKey(leaderFileVersion));
-        assertEquals("Slave's tlog file size differs for version " + leaderFileVersion, leaderFilesMeta.get(leaderFileVersion), slaveFilesMeta.get(leaderFileVersion));
+        assertTrue("Follower is missing a tlog for version " + leaderFileVersion, followerFilesMeta.containsKey(leaderFileVersion));
+        assertEquals("Follower's tlog file size differs for version " + leaderFileVersion, leaderFilesMeta.get(leaderFileVersion), followerFilesMeta.get(leaderFileVersion));
       }
     }
   }
diff --git a/solr/core/src/test/org/apache/solr/core/snapshots/TestSolrCoreSnapshots.java b/solr/core/src/test/org/apache/solr/core/snapshots/TestSolrCoreSnapshots.java
index 751d61e..2593c35 100644
--- a/solr/core/src/test/org/apache/solr/core/snapshots/TestSolrCoreSnapshots.java
+++ b/solr/core/src/test/org/apache/solr/core/snapshots/TestSolrCoreSnapshots.java
@@ -104,7 +104,7 @@ public class TestSolrCoreSnapshots extends SolrCloudTestCase {
 
     try (
         SolrClient adminClient = getHttpSolrClient(cluster.getJettySolrRunners().get(0).getBaseUrl().toString());
-        SolrClient masterClient = getHttpSolrClient(replica.getCoreUrl())) {
+        SolrClient leaderClient = getHttpSolrClient(replica.getCoreUrl())) {
 
       SnapshotMetaData metaData = createSnapshot(adminClient, coreName, commitName);
       // Create another snapshot referring to the same index commit to verify the
@@ -115,8 +115,8 @@ public class TestSolrCoreSnapshots extends SolrCloudTestCase {
       assertEquals (metaData.getGenerationNumber(), duplicateCommit.getGenerationNumber());
 
       // Delete all documents
-      masterClient.deleteByQuery("*:*");
-      masterClient.commit();
+      leaderClient.deleteByQuery("*:*");
+      leaderClient.commit();
       BackupRestoreUtils.verifyDocs(0, cluster.getSolrClient(), collectionName);
 
       // Verify that the index directory contains at least 2 index commits - one referred by the snapshots
@@ -191,7 +191,7 @@ public class TestSolrCoreSnapshots extends SolrCloudTestCase {
 
     try (
         SolrClient adminClient = getHttpSolrClient(cluster.getJettySolrRunners().get(0).getBaseUrl().toString());
-        SolrClient masterClient = getHttpSolrClient(replica.getCoreUrl())) {
+        SolrClient leaderClient = getHttpSolrClient(replica.getCoreUrl())) {
 
       SnapshotMetaData metaData = createSnapshot(adminClient, coreName, commitName);
 
@@ -202,7 +202,7 @@ public class TestSolrCoreSnapshots extends SolrCloudTestCase {
           //Delete a few docs
           int numDeletes = TestUtil.nextInt(random(), 1, nDocs);
           for(int i=0; i<numDeletes; i++) {
-            masterClient.deleteByQuery("id:" + i);
+            leaderClient.deleteByQuery("id:" + i);
           }
           //Add a few more
           int moreAdds = TestUtil.nextInt(random(), 1, 100);
@@ -210,9 +210,9 @@ public class TestSolrCoreSnapshots extends SolrCloudTestCase {
             SolrInputDocument doc = new SolrInputDocument();
             doc.addField("id", i + nDocs);
             doc.addField("name", "name = " + (i + nDocs));
-            masterClient.add(doc);
+            leaderClient.add(doc);
           }
-          masterClient.commit();
+          leaderClient.commit();
         }
       }
 
@@ -226,7 +226,7 @@ public class TestSolrCoreSnapshots extends SolrCloudTestCase {
       }
 
       // Optimize the index.
-      masterClient.optimize(true, true, 1);
+      leaderClient.optimize(true, true, 1);
 
       // After invoking optimize command, verify that the index directory contains multiple commits (including the one we snapshotted earlier).
       {
@@ -247,13 +247,13 @@ public class TestSolrCoreSnapshots extends SolrCloudTestCase {
           SolrInputDocument doc = new SolrInputDocument();
           doc.addField("id", i + nDocs);
           doc.addField("name", "name = " + (i + nDocs));
-          masterClient.add(doc);
+          leaderClient.add(doc);
         }
-        masterClient.commit();
+        leaderClient.commit();
       }
 
       // Optimize the index.
-      masterClient.optimize(true, true, 1);
+      leaderClient.optimize(true, true, 1);
 
       // Verify that the index directory contains only 1 index commit (which is not the same as the snapshotted commit).
       Collection<IndexCommit> commits = listCommits(metaData.getIndexDirPath());
diff --git a/solr/core/src/test/org/apache/solr/handler/BackupRestoreUtils.java b/solr/core/src/test/org/apache/solr/handler/BackupRestoreUtils.java
index 74add18..7617dc5 100644
--- a/solr/core/src/test/org/apache/solr/handler/BackupRestoreUtils.java
+++ b/solr/core/src/test/org/apache/solr/handler/BackupRestoreUtils.java
@@ -41,8 +41,8 @@ public class BackupRestoreUtils extends SolrTestCase {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  public static int indexDocs(SolrClient masterClient, String collectionName, long docsSeed) throws IOException, SolrServerException {
-    masterClient.deleteByQuery(collectionName, "*:*");
+  public static int indexDocs(SolrClient leaderClient, String collectionName, long docsSeed) throws IOException, SolrServerException {
+    leaderClient.deleteByQuery(collectionName, "*:*");
 
     Random random = new Random(docsSeed);// use a constant seed for the whole test run so that we can easily re-index.
     int nDocs = TestUtil.nextInt(random, 1, 100);
@@ -55,15 +55,15 @@ public class BackupRestoreUtils extends SolrTestCase {
       doc.addField("name", "name = " + i);
       docs.add(doc);
     }
-    masterClient.add(collectionName, docs);
-    masterClient.commit(collectionName);
+    leaderClient.add(collectionName, docs);
+    leaderClient.commit(collectionName);
     return nDocs;
   }
 
-  public static void verifyDocs(int nDocs, SolrClient masterClient, String collectionName) throws SolrServerException, IOException {
+  public static void verifyDocs(int nDocs, SolrClient leaderClient, String collectionName) throws SolrServerException, IOException {
     ModifiableSolrParams queryParams = new ModifiableSolrParams();
     queryParams.set("q", "*:*");
-    QueryResponse response = masterClient.query(collectionName, queryParams);
+    QueryResponse response = leaderClient.query(collectionName, queryParams);
 
     assertEquals(0, response.getStatus());
     assertEquals(nDocs, response.getResults().getNumFound());
@@ -82,13 +82,13 @@ public class BackupRestoreUtils extends SolrTestCase {
       builder.append("=");
       builder.append(p.getValue());
     }
-    String masterUrl = builder.toString();
-    executeHttpRequest(masterUrl);
+    String leaderUrl = builder.toString();
+    executeHttpRequest(leaderUrl);
   }
 
   public static void runReplicationHandlerCommand(String baseUrl, String coreName, String action, String repoName, String backupName) throws IOException {
-    String masterUrl = baseUrl + "/" + coreName + ReplicationHandler.PATH + "?command=" + action + "&repository="+repoName+"&name="+backupName;
-    executeHttpRequest(masterUrl);
+    String leaderUrl = baseUrl + "/" + coreName + ReplicationHandler.PATH + "?command=" + action + "&repository="+repoName+"&name="+backupName;
+    executeHttpRequest(leaderUrl);
   }
 
   static void executeHttpRequest(String requestUrl) throws IOException {
diff --git a/solr/core/src/test/org/apache/solr/handler/TestHdfsBackupRestoreCore.java b/solr/core/src/test/org/apache/solr/handler/TestHdfsBackupRestoreCore.java
index 010cc8b..3347c02 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestHdfsBackupRestoreCore.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestHdfsBackupRestoreCore.java
@@ -186,13 +186,13 @@ public class TestHdfsBackupRestoreCore extends SolrCloudTestCase {
     boolean testViaReplicationHandler = random().nextBoolean();
     String baseUrl = cluster.getJettySolrRunners().get(0).getBaseUrl().toString();
 
-    try (HttpSolrClient masterClient = getHttpSolrClient(replicaBaseUrl)) {
+    try (HttpSolrClient leaderClient = getHttpSolrClient(replicaBaseUrl)) {
       // Create a backup.
       if (testViaReplicationHandler) {
         log.info("Running Backup via replication handler");
         BackupRestoreUtils.runReplicationHandlerCommand(baseUrl, coreName, ReplicationHandler.CMD_BACKUP, "hdfs", backupName);
         final BackupStatusChecker backupStatus
-          = new BackupStatusChecker(masterClient, "/" + coreName + "/replication");
+          = new BackupStatusChecker(leaderClient, "/" + coreName + "/replication");
         backupStatus.waitForBackupSuccess(backupName, 30);
       } else {
         log.info("Running Backup via core admin api");
@@ -209,9 +209,9 @@ public class TestHdfsBackupRestoreCore extends SolrCloudTestCase {
           //Delete a few docs
           int numDeletes = TestUtil.nextInt(random(), 1, nDocs);
           for(int i=0; i<numDeletes; i++) {
-            masterClient.deleteByQuery(collectionName, "id:" + i);
+            leaderClient.deleteByQuery(collectionName, "id:" + i);
           }
-          masterClient.commit(collectionName);
+          leaderClient.commit(collectionName);
 
           //Add a few more
           int moreAdds = TestUtil.nextInt(random(), 1, 100);
@@ -219,11 +219,11 @@ public class TestHdfsBackupRestoreCore extends SolrCloudTestCase {
             SolrInputDocument doc = new SolrInputDocument();
             doc.addField("id", i + nDocs);
             doc.addField("name", "name = " + (i + nDocs));
-            masterClient.add(collectionName, doc);
+            leaderClient.add(collectionName, doc);
           }
           //Purposely not calling commit once in a while. There can be some docs which are not committed
           if (usually()) {
-            masterClient.commit(collectionName);
+            leaderClient.commit(collectionName);
           }
         }
         // Snapshooter prefixes "snapshot." to the backup name.
@@ -242,7 +242,7 @@ public class TestHdfsBackupRestoreCore extends SolrCloudTestCase {
           BackupRestoreUtils.runCoreAdminCommand(replicaBaseUrl, coreName, CoreAdminAction.RESTORECORE.toString(), params);
         }
         //See if restore was successful by checking if all the docs are present again
-        BackupRestoreUtils.verifyDocs(nDocs, masterClient, coreName);
+        BackupRestoreUtils.verifyDocs(nDocs, leaderClient, coreName);
 
         // Verify the permissions for the backup folder.
         FileStatus status = fs.getFileStatus(new org.apache.hadoop.fs.Path("/backup/snapshot."+backupName));
diff --git a/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java b/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
index 2c91e3a..1b02f77 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
@@ -109,9 +109,9 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
       + File.separator + "collection1" + File.separator + "conf"
       + File.separator;
 
-  JettySolrRunner masterJetty, slaveJetty, repeaterJetty;
-  HttpSolrClient masterClient, slaveClient, repeaterClient;
-  SolrInstance master = null, slave = null, repeater = null;
+  JettySolrRunner leaderJetty, followerJetty, repeaterJetty;
+  HttpSolrClient leaderClient, followerClient, repeaterClient;
+  SolrInstance leader = null, follower = null, repeater = null;
 
   static String context = "/solr";
 
@@ -119,8 +119,12 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
   // index from previous test method
   static int nDocs = 500;
   
+  /* For testing backward compatibility, remove for 10.x */
+  private static boolean useLegacyParams = false; 
+  
   @BeforeClass
   public static void beforeClass() {
+    useLegacyParams = usually();
 
   }
   
@@ -130,25 +134,25 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
 //    System.setProperty("solr.directoryFactory", "solr.StandardDirectoryFactory");
     // For manual testing only
     // useFactory(null); // force an FS factory.
-    master = new SolrInstance(createTempDir("solr-instance").toFile(), "master", null);
-    master.setUp();
-    masterJetty = createAndStartJetty(master);
-    masterClient = createNewSolrClient(masterJetty.getLocalPort());
-
-    slave = new SolrInstance(createTempDir("solr-instance").toFile(), "slave", masterJetty.getLocalPort());
-    slave.setUp();
-    slaveJetty = createAndStartJetty(slave);
-    slaveClient = createNewSolrClient(slaveJetty.getLocalPort());
+    leader = new SolrInstance(createTempDir("solr-instance").toFile(), "leader", null);
+    leader.setUp();
+    leaderJetty = createAndStartJetty(leader);
+    leaderClient = createNewSolrClient(leaderJetty.getLocalPort());
+
+    follower = new SolrInstance(createTempDir("solr-instance").toFile(), "follower", leaderJetty.getLocalPort());
+    follower.setUp();
+    followerJetty = createAndStartJetty(follower);
+    followerClient = createNewSolrClient(followerJetty.getLocalPort());
     
     System.setProperty("solr.indexfetcher.sotimeout2", "45000");
   }
 
   public void clearIndexWithReplication() throws Exception {
-    if (numFound(query("*:*", masterClient)) != 0) {
-      masterClient.deleteByQuery("*:*");
-      masterClient.commit();
+    if (numFound(query("*:*", leaderClient)) != 0) {
+      leaderClient.deleteByQuery("*:*");
+      leaderClient.commit();
       // wait for replication to sync & verify
-      assertEquals(0, numFound(rQuery(0, "*:*", slaveClient)));
+      assertEquals(0, numFound(rQuery(0, "*:*", followerClient)));
     }
   }
 
@@ -156,21 +160,21 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
   @After
   public void tearDown() throws Exception {
     super.tearDown();
-    if (null != masterJetty) {
-      masterJetty.stop();
-      masterJetty = null;
+    if (null != leaderJetty) {
+      leaderJetty.stop();
+      leaderJetty = null;
     }
-    if (null != slaveJetty) {
-      slaveJetty.stop();
-      slaveJetty = null;
+    if (null != followerJetty) {
+      followerJetty.stop();
+      followerJetty = null;
     }
-    if (null != masterClient) {
-      masterClient.close();
-      masterClient = null;
+    if (null != leaderClient) {
+      leaderClient.close();
+      leaderClient = null;
     }
-    if (null != slaveClient) {
-      slaveClient.close();
-      slaveClient = null;
+    if (null != followerClient) {
+      followerClient.close();
+      followerClient = null;
     }
     System.clearProperty("solr.indexfetcher.sotimeout");
   }
@@ -300,67 +304,67 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
 
   @Test
   public void doTestDetails() throws Exception {
-    slaveJetty.stop();
+    followerJetty.stop();
     
-    slave.setTestPort(masterJetty.getLocalPort());
-    slave.copyConfigFile(CONF_DIR + "solrconfig-slave.xml", "solrconfig.xml");
-    slaveJetty = createAndStartJetty(slave);
+    follower.setTestPort(leaderJetty.getLocalPort());
+    follower.copyConfigFile(CONF_DIR + "solrconfig-follower.xml", "solrconfig.xml");
+    followerJetty = createAndStartJetty(follower);
     
-    slaveClient.close();
-    masterClient.close();
-    masterClient = createNewSolrClient(masterJetty.getLocalPort());
-    slaveClient = createNewSolrClient(slaveJetty.getLocalPort());
+    followerClient.close();
+    leaderClient.close();
+    leaderClient = createNewSolrClient(leaderJetty.getLocalPort());
+    followerClient = createNewSolrClient(followerJetty.getLocalPort());
     
     clearIndexWithReplication();
     { 
-      NamedList<Object> details = getDetails(masterClient);
+      NamedList<Object> details = getDetails(leaderClient);
       
-      assertEquals("master isMaster?", 
+      assertEquals("leader isLeader?",
                    "true", details.get("isMaster"));
-      assertEquals("master isSlave?", 
+      assertEquals("leader isFollower?",
                    "false", details.get("isSlave"));
-      assertNotNull("master has master section", 
+      assertNotNull("leader has leader section",
                     details.get("master"));
     }
 
-    // check details on the slave a couple of times before & after fetching
+    // check details on the follower a couple of times before & after fetching
     for (int i = 0; i < 3; i++) {
-      NamedList<Object> details = getDetails(slaveClient);
+      NamedList<Object> details = getDetails(followerClient);
       assertNotNull(i + ": " + details);
       assertNotNull(i + ": " + details.toString(), details.get("slave"));
 
       if (i > 0) {
-        rQuery(i, "*:*", slaveClient);
+        rQuery(i, "*:*", followerClient);
         @SuppressWarnings({"rawtypes"})
         List replicatedAtCount = (List) ((NamedList) details.get("slave")).get("indexReplicatedAtList");
         int tries = 0;
         while ((replicatedAtCount == null || replicatedAtCount.size() < i) && tries++ < 5) {
           Thread.sleep(1000);
-          details = getDetails(slaveClient);
+          details = getDetails(followerClient);
           replicatedAtCount = (List) ((NamedList) details.get("slave")).get("indexReplicatedAtList");
         }
         
-        assertNotNull("Expected to see that the slave has replicated" + i + ": " + details.toString(), replicatedAtCount);
+        assertNotNull("Expected to see that the follower has replicated" + i + ": " + details.toString(), replicatedAtCount);
         
         // we can have more replications than we added docs because a replication can legally fail and try 
         // again (sometimes we cannot merge into a live index and have to try again)
         assertTrue("i:" + i + " replicationCount:" + replicatedAtCount.size(), replicatedAtCount.size() >= i); 
       }
 
-      assertEquals(i + ": " + "slave isMaster?", "false", details.get("isMaster"));
-      assertEquals(i + ": " + "slave isSlave?", "true", details.get("isSlave"));
-      assertNotNull(i + ": " + "slave has slave section", details.get("slave"));
+      assertEquals(i + ": " + "follower isLeader?", "false", details.get("isMaster"));
+      assertEquals(i + ": " + "follower isFollower?", "true", details.get("isSlave"));
+      assertNotNull(i + ": " + "follower has follower section", details.get("slave"));
       // SOLR-2677: assert not false negatives
       Object timesFailed = ((NamedList)details.get("slave")).get(IndexFetcher.TIMES_FAILED);
       // SOLR-7134: we can have a fail because some mock index files have no checksum, will
       // always be downloaded, and may not be able to be moved into the existing index
-      assertTrue(i + ": " + "slave has fetch error count: " + (String)timesFailed, timesFailed == null || ((String) timesFailed).equals("1"));
+      assertTrue(i + ": " + "follower has fetch error count: " + (String)timesFailed, timesFailed == null || ((String) timesFailed).equals("1"));
 
       if (3 != i) {
         // index & fetch
-        index(masterClient, "id", i, "name", "name = " + i);
-        masterClient.commit();
-        pullFromTo(masterJetty, slaveJetty);
+        index(leaderClient, "id", i, "name", "name = " + i);
+        leaderClient.commit();
+        pullFromTo(leaderJetty, followerJetty);
       }
     }
 
@@ -368,7 +372,7 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
     JettySolrRunner repeaterJetty = null;
     SolrClient repeaterClient = null;
     try {
-      repeater = new SolrInstance(createTempDir("solr-instance").toFile(), "repeater", masterJetty.getLocalPort());
+      repeater = new SolrInstance(createTempDir("solr-instance").toFile(), "repeater", leaderJetty.getLocalPort());
       repeater.setUp();
       repeaterJetty = createAndStartJetty(repeater);
       repeaterClient = createNewSolrClient(repeaterJetty.getLocalPort());
@@ -376,13 +380,13 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
       
       NamedList<Object> details = getDetails(repeaterClient);
       
-      assertEquals("repeater isMaster?", 
+      assertEquals("repeater isLeader?",
                    "true", details.get("isMaster"));
-      assertEquals("repeater isSlave?", 
+      assertEquals("repeater isFollower?",
                    "true", details.get("isSlave"));
-      assertNotNull("repeater has master section", 
+      assertNotNull("repeater has leader section",
                     details.get("master"));
-      assertNotNull("repeater has slave section", 
+      assertNotNull("repeater has follower section",
                     details.get("slave"));
 
     } finally {
@@ -392,104 +396,135 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
       if (repeaterClient != null) repeaterClient.close();
     }
   }
+  
+  @Test
+  public void testLegacyConfiguration() throws Exception {
+    SolrInstance solrInstance = null;
+    JettySolrRunner instanceJetty = null;
+    SolrClient client = null;
+    try {
+      solrInstance = new SolrInstance(createTempDir("solr-instance").toFile(), "replication-legacy", leaderJetty.getLocalPort());
+      solrInstance.setUp();
+      instanceJetty = createAndStartJetty(solrInstance);
+      client = createNewSolrClient(instanceJetty.getLocalPort());
+
+      
+      NamedList<Object> details = getDetails(client);
+      
+      assertEquals("repeater isLeader?",
+                   "true", details.get("isMaster"));
+      assertEquals("repeater isFollower?",
+                   "true", details.get("isSlave"));
+      assertNotNull("repeater has leader section",
+                    details.get("master"));
+      assertNotNull("repeater has follower section",
+                    details.get("slave"));
+
+    } finally {
+      if (instanceJetty != null) {
+        instanceJetty.stop();
+      }
+      if (client != null) client.close();
+    }
+  }
 
 
   /**
    * Verify that empty commits and/or commits with openSearcher=false
-   * on the master do not cause subsequent replication problems on the slave 
+   * on the leader do not cause subsequent replication problems on the follower
    */
   public void testEmptyCommits() throws Exception {
     clearIndexWithReplication();
     
-    // add a doc to master and commit
-    index(masterClient, "id", "1", "name", "empty1");
-    emptyUpdate(masterClient, "commit", "true");
+    // add a doc to leader and commit
+    index(leaderClient, "id", "1", "name", "empty1");
+    emptyUpdate(leaderClient, "commit", "true");
     // force replication
-    pullFromMasterToSlave();
-    // verify doc is on slave
-    rQuery(1, "name:empty1", slaveClient);
-    assertVersions(masterClient, slaveClient);
-
-    // do a completely empty commit on master and force replication
-    emptyUpdate(masterClient, "commit", "true");
-    pullFromMasterToSlave();
-
-    // add another doc and verify slave gets it
-    index(masterClient, "id", "2", "name", "empty2");
-    emptyUpdate(masterClient, "commit", "true");
+    pullFromLeaderToFollower();
+    // verify doc is on follower
+    rQuery(1, "name:empty1", followerClient);
+    assertVersions(leaderClient, followerClient);
+
+    // do a completely empty commit on leader and force replication
+    emptyUpdate(leaderClient, "commit", "true");
+    pullFromLeaderToFollower();
+
+    // add another doc and verify follower gets it
+    index(leaderClient, "id", "2", "name", "empty2");
+    emptyUpdate(leaderClient, "commit", "true");
     // force replication
-    pullFromMasterToSlave();
+    pullFromLeaderToFollower();
 
-    rQuery(1, "name:empty2", slaveClient);
-    assertVersions(masterClient, slaveClient);
+    rQuery(1, "name:empty2", followerClient);
+    assertVersions(leaderClient, followerClient);
 
-    // add a third doc but don't open a new searcher on master
-    index(masterClient, "id", "3", "name", "empty3");
-    emptyUpdate(masterClient, "commit", "true", "openSearcher", "false");
-    pullFromMasterToSlave();
+    // add a third doc but don't open a new searcher on leader
+    index(leaderClient, "id", "3", "name", "empty3");
+    emptyUpdate(leaderClient, "commit", "true", "openSearcher", "false");
+    pullFromLeaderToFollower();
     
-    // verify slave can search the doc, but master doesn't
-    rQuery(0, "name:empty3", masterClient);
-    rQuery(1, "name:empty3", slaveClient);
+    // verify follower can search the doc, but leader doesn't
+    rQuery(0, "name:empty3", leaderClient);
+    rQuery(1, "name:empty3", followerClient);
 
-    // final doc with hard commit, slave and master both showing all docs
-    index(masterClient, "id", "4", "name", "empty4");
-    emptyUpdate(masterClient, "commit", "true");
-    pullFromMasterToSlave();
+    // final doc with hard commit, follower and leader both showing all docs
+    index(leaderClient, "id", "4", "name", "empty4");
+    emptyUpdate(leaderClient, "commit", "true");
+    pullFromLeaderToFollower();
 
     String q = "name:(empty1 empty2 empty3 empty4)";
-    rQuery(4, q, masterClient);
-    rQuery(4, q, slaveClient);
-    assertVersions(masterClient, slaveClient);
+    rQuery(4, q, leaderClient);
+    rQuery(4, q, followerClient);
+    assertVersions(leaderClient, followerClient);
 
   }
 
   @Test
-  public void doTestReplicateAfterWrite2Slave() throws Exception {
+  public void doTestReplicateAfterWrite2Follower() throws Exception {
     clearIndexWithReplication();
     nDocs--;
     for (int i = 0; i < nDocs; i++) {
-      index(masterClient, "id", i, "name", "name = " + i);
+      index(leaderClient, "id", i, "name", "name = " + i);
     }
 
-    invokeReplicationCommand(masterJetty.getLocalPort(), "disableReplication");
-    invokeReplicationCommand(slaveJetty.getLocalPort(), "disablepoll");
+    invokeReplicationCommand(leaderJetty.getLocalPort(), "disableReplication");
+    invokeReplicationCommand(followerJetty.getLocalPort(), "disablepoll");
     
-    masterClient.commit();
+    leaderClient.commit();
 
-    assertEquals(nDocs, numFound(rQuery(nDocs, "*:*", masterClient)));
+    assertEquals(nDocs, numFound(rQuery(nDocs, "*:*", leaderClient)));
 
-    // Make sure that both the index version and index generation on the slave is
-    // higher than that of the master, just to make the test harder.
+    // Make sure that both the index version and index generation on the follower is
+    // higher than that of the leader, just to make the test harder.
 
-    index(slaveClient, "id", 551, "name", "name = " + 551);
-    slaveClient.commit(true, true);
-    index(slaveClient, "id", 552, "name", "name = " + 552);
-    slaveClient.commit(true, true);
-    index(slaveClient, "id", 553, "name", "name = " + 553);
-    slaveClient.commit(true, true);
-    index(slaveClient, "id", 554, "name", "name = " + 554);
-    slaveClient.commit(true, true);
-    index(slaveClient, "id", 555, "name", "name = " + 555);
-    slaveClient.commit(true, true);
+    index(followerClient, "id", 551, "name", "name = " + 551);
+    followerClient.commit(true, true);
+    index(followerClient, "id", 552, "name", "name = " + 552);
+    followerClient.commit(true, true);
+    index(followerClient, "id", 553, "name", "name = " + 553);
+    followerClient.commit(true, true);
+    index(followerClient, "id", 554, "name", "name = " + 554);
+    followerClient.commit(true, true);
+    index(followerClient, "id", 555, "name", "name = " + 555);
+    followerClient.commit(true, true);
 
-    //this doc is added to slave so it should show an item w/ that result
-    assertEquals(1, numFound(rQuery(1, "id:555", slaveClient)));
+    //this doc is added to follower so it should show an item w/ that result
+    assertEquals(1, numFound(rQuery(1, "id:555", followerClient)));
 
     //Let's fetch the index rather than rely on the polling.
-    invokeReplicationCommand(masterJetty.getLocalPort(), "enablereplication");
-    invokeReplicationCommand(slaveJetty.getLocalPort(), "fetchindex");
+    invokeReplicationCommand(leaderJetty.getLocalPort(), "enablereplication");
+    invokeReplicationCommand(followerJetty.getLocalPort(), "fetchindex");
 
     /*
-    //the slave should have done a full copy of the index so the doc with id:555 should not be there in the slave now
-    slaveQueryRsp = rQuery(0, "id:555", slaveClient);
-    slaveQueryResult = (SolrDocumentList) slaveQueryRsp.get("response");
-    assertEquals(0, slaveQueryResult.getNumFound());
-
-    // make sure we replicated the correct index from the master
-    slaveQueryRsp = rQuery(nDocs, "*:*", slaveClient);
-    slaveQueryResult = (SolrDocumentList) slaveQueryRsp.get("response");
-    assertEquals(nDocs, slaveQueryResult.getNumFound());
+    //the follower should have done a full copy of the index so the doc with id:555 should not be there in the follower now
+    followerQueryRsp = rQuery(0, "id:555", followerClient);
+    followerQueryResult = (SolrDocumentList) followerQueryRsp.get("response");
+    assertEquals(0, followerQueryResult.getNumFound());
+
+    // make sure we replicated the correct index from the leader
+    followerQueryRsp = rQuery(nDocs, "*:*", followerClient);
+    followerQueryResult = (SolrDocumentList) followerQueryRsp.get("response");
+    assertEquals(nDocs, followerQueryResult.getNumFound());
     
     */
   }
@@ -498,8 +533,8 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
   //jetty servers.
   static void invokeReplicationCommand(int pJettyPort, String pCommand) throws IOException
   {
-    String masterUrl = buildUrl(pJettyPort) + "/" + DEFAULT_TEST_CORENAME + ReplicationHandler.PATH+"?command=" + pCommand;
-    URL u = new URL(masterUrl);
+    String leaderUrl = buildUrl(pJettyPort) + "/" + DEFAULT_TEST_CORENAME + ReplicationHandler.PATH+"?command=" + pCommand;
+    URL u = new URL(leaderUrl);
     InputStream stream = u.openStream();
     stream.close();
   }
@@ -507,80 +542,80 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
   @Test
   public void doTestIndexAndConfigReplication() throws Exception {
 
-    TestInjection.delayBeforeSlaveCommitRefresh = random().nextInt(10);
+    TestInjection.delayBeforeFollowerCommitRefresh = random().nextInt(10);
 
     clearIndexWithReplication();
 
     nDocs--;
     for (int i = 0; i < nDocs; i++)
-      index(masterClient, "id", i, "name", "name = " + i);
+      index(leaderClient, "id", i, "name", "name = " + i);
 
-    masterClient.commit();
+    leaderClient.commit();
 
     @SuppressWarnings({"rawtypes"})
-    NamedList masterQueryRsp = rQuery(nDocs, "*:*", masterClient);
-    SolrDocumentList masterQueryResult = (SolrDocumentList) masterQueryRsp.get("response");
-    assertEquals(nDocs, numFound(masterQueryRsp));
+    NamedList leaderQueryRsp = rQuery(nDocs, "*:*", leaderClient);
+    SolrDocumentList leaderQueryResult = (SolrDocumentList) leaderQueryRsp.get("response");
+    assertEquals(nDocs, numFound(leaderQueryRsp));
 
-    //get docs from slave and check if number is equal to master
+    //get docs from follower and check if number is equal to leader
     @SuppressWarnings({"rawtypes"})
-    NamedList slaveQueryRsp = rQuery(nDocs, "*:*", slaveClient);
-    SolrDocumentList slaveQueryResult = (SolrDocumentList) slaveQueryRsp.get("response");
-    assertEquals(nDocs, numFound(slaveQueryRsp));
+    NamedList followerQueryRsp = rQuery(nDocs, "*:*", followerClient);
+    SolrDocumentList followerQueryResult = (SolrDocumentList) followerQueryRsp.get("response");
+    assertEquals(nDocs, numFound(followerQueryRsp));
 
     //compare results
-    String cmp = BaseDistributedSearchTestCase.compare(masterQueryResult, slaveQueryResult, 0, null);
+    String cmp = BaseDistributedSearchTestCase.compare(leaderQueryResult, followerQueryResult, 0, null);
     assertEquals(null, cmp);
     
-    assertVersions(masterClient, slaveClient);
+    assertVersions(leaderClient, followerClient);
 
     //start config files replication test
-    masterClient.deleteByQuery("*:*");
-    masterClient.commit();
+    leaderClient.deleteByQuery("*:*");
+    leaderClient.commit();
 
-    //change the schema on master
-    master.copyConfigFile(CONF_DIR + "schema-replication2.xml", "schema.xml");
+    //change the schema on leader
+    leader.copyConfigFile(CONF_DIR + "schema-replication2.xml", "schema.xml");
 
-    masterJetty.stop();
+    leaderJetty.stop();
 
-    masterJetty = createAndStartJetty(master);
-    masterClient.close();
-    masterClient = createNewSolrClient(masterJetty.getLocalPort());
+    leaderJetty = createAndStartJetty(leader);
+    leaderClient.close();
+    leaderClient = createNewSolrClient(leaderJetty.getLocalPort());
 
-    slave.setTestPort(masterJetty.getLocalPort());
-    slave.copyConfigFile(slave.getSolrConfigFile(), "solrconfig.xml");
+    follower.setTestPort(leaderJetty.getLocalPort());
+    follower.copyConfigFile(follower.getSolrConfigFile(), "solrconfig.xml");
 
-    slaveJetty.stop();
+    followerJetty.stop();
 
     // setup an xslt dir to force subdir file replication
-    File masterXsltDir = new File(master.getConfDir() + File.separator + "xslt");
-    File masterXsl = new File(masterXsltDir, "dummy.xsl");
-    assertTrue("could not make dir " + masterXsltDir, masterXsltDir.mkdirs());
-    assertTrue(masterXsl.createNewFile());
-
-    File slaveXsltDir = new File(slave.getConfDir() + File.separator + "xslt");
-    File slaveXsl = new File(slaveXsltDir, "dummy.xsl");
-    assertFalse(slaveXsltDir.exists());
-
-    slaveJetty = createAndStartJetty(slave);
-    slaveClient.close();
-    slaveClient = createNewSolrClient(slaveJetty.getLocalPort());
-    //add a doc with new field and commit on master to trigger index fetch from slave.
-    index(masterClient, "id", "2000", "name", "name = " + 2000, "newname", "newname = " + 2000);
-    masterClient.commit();
-
-    assertEquals(1, numFound( rQuery(1, "*:*", masterClient)));
+    File leaderXsltDir = new File(leader.getConfDir() + File.separator + "xslt");
+    File leaderXsl = new File(leaderXsltDir, "dummy.xsl");
+    assertTrue("could not make dir " + leaderXsltDir, leaderXsltDir.mkdirs());
+    assertTrue(leaderXsl.createNewFile());
+
+    File followerXsltDir = new File(follower.getConfDir() + File.separator + "xslt");
+    File followerXsl = new File(followerXsltDir, "dummy.xsl");
+    assertFalse(followerXsltDir.exists());
+
+    followerJetty = createAndStartJetty(follower);
+    followerClient.close();
+    followerClient = createNewSolrClient(followerJetty.getLocalPort());
+    //add a doc with new field and commit on leader to trigger index fetch from follower.
+    index(leaderClient, "id", "2000", "name", "name = " + 2000, "newname", "newname = " + 2000);
+    leaderClient.commit();
+
+    assertEquals(1, numFound( rQuery(1, "*:*", leaderClient)));
     
-    slaveQueryRsp = rQuery(1, "*:*", slaveClient);
-    assertVersions(masterClient, slaveClient);
-    SolrDocument d = ((SolrDocumentList) slaveQueryRsp.get("response")).get(0);
+    followerQueryRsp = rQuery(1, "*:*", followerClient);
+    assertVersions(leaderClient, followerClient);
+    SolrDocument d = ((SolrDocumentList) followerQueryRsp.get("response")).get(0);
     assertEquals("newname = 2000", (String) d.getFieldValue("newname"));
 
-    assertTrue(slaveXsltDir.isDirectory());
-    assertTrue(slaveXsl.exists());
+    assertTrue(followerXsltDir.isDirectory());
+    assertTrue(followerXsl.exists());
     
-    checkForSingleIndex(masterJetty);
-    checkForSingleIndex(slaveJetty, true);
+    checkForSingleIndex(leaderJetty);
+    checkForSingleIndex(followerJetty, true);
   }
 
   @Test
@@ -588,136 +623,136 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
     clearIndexWithReplication();
 
     // Test:
-    // setup master/slave.
-    // stop polling on slave, add a doc to master and verify slave hasn't picked it.
+    // setup leader/follower.
+    // stop polling on follower, add a doc to leader and verify follower hasn't picked it.
     nDocs--;
     for (int i = 0; i < nDocs; i++)
-      index(masterClient, "id", i, "name", "name = " + i);
+      index(leaderClient, "id", i, "name", "name = " + i);
 
-    masterClient.commit();
+    leaderClient.commit();
 
     @SuppressWarnings({"rawtypes"})
-    NamedList masterQueryRsp = rQuery(nDocs, "*:*", masterClient);
-    SolrDocumentList masterQueryResult = (SolrDocumentList) masterQueryRsp.get("response");
-    assertEquals(nDocs, numFound(masterQueryRsp));
+    NamedList leaderQueryRsp = rQuery(nDocs, "*:*", leaderClient);
+    SolrDocumentList leaderQueryResult = (SolrDocumentList) leaderQueryRsp.get("response");
+    assertEquals(nDocs, numFound(leaderQueryRsp));
 
-    //get docs from slave and check if number is equal to master
+    //get docs from follower and check if number is equal to leader
     @SuppressWarnings({"rawtypes"})
-    NamedList slaveQueryRsp = rQuery(nDocs, "*:*", slaveClient);
-    SolrDocumentList slaveQueryResult = (SolrDocumentList) slaveQueryRsp.get("response");
-    assertEquals(nDocs, numFound(slaveQueryRsp));
+    NamedList followerQueryRsp = rQuery(nDocs, "*:*", followerClient);
+    SolrDocumentList followerQueryResult = (SolrDocumentList) followerQueryRsp.get("response");
+    assertEquals(nDocs, numFound(followerQueryRsp));
 
     //compare results
-    String cmp = BaseDistributedSearchTestCase.compare(masterQueryResult, slaveQueryResult, 0, null);
+    String cmp = BaseDistributedSearchTestCase.compare(leaderQueryResult, followerQueryResult, 0, null);
     assertEquals(null, cmp);
 
     // start stop polling test
-    invokeReplicationCommand(slaveJetty.getLocalPort(), "disablepoll");
+    invokeReplicationCommand(followerJetty.getLocalPort(), "disablepoll");
     
-    index(masterClient, "id", 501, "name", "name = " + 501);
-    masterClient.commit();
+    index(leaderClient, "id", 501, "name", "name = " + 501);
+    leaderClient.commit();
 
-    //get docs from master and check if number is equal to master
-    assertEquals(nDocs+1, numFound(rQuery(nDocs+1, "*:*", masterClient)));
+    //get docs from leader and check if number is equal to leader
+    assertEquals(nDocs+1, numFound(rQuery(nDocs+1, "*:*", leaderClient)));
     
     // NOTE: this test is wierd, we want to verify it DOESNT replicate...
     // for now, add a sleep for this.., but the logic is wierd.
     Thread.sleep(3000);
     
-    //get docs from slave and check if number is not equal to master; polling is disabled
-    assertEquals(nDocs, numFound(rQuery(nDocs, "*:*", slaveClient)));
+    //get docs from follower and check if number is not equal to leader; polling is disabled
+    assertEquals(nDocs, numFound(rQuery(nDocs, "*:*", followerClient)));
 
     // re-enable replication
-    invokeReplicationCommand(slaveJetty.getLocalPort(), "enablepoll");
+    invokeReplicationCommand(followerJetty.getLocalPort(), "enablepoll");
 
-    assertEquals(nDocs+1, numFound(rQuery(nDocs+1, "*:*", slaveClient)));
+    assertEquals(nDocs+1, numFound(rQuery(nDocs+1, "*:*", followerClient)));
   }
 
   /**
-   * We assert that if master is down for more than poll interval,
-   * the slave doesn't re-fetch the whole index from master again if
+   * We assert that if leader is down for more than poll interval,
+   * the follower doesn't re-fetch the whole index from leader again if
    * the index hasn't changed. See SOLR-9036
    */
   @Test
-  public void doTestIndexFetchOnMasterRestart() throws Exception  {
+  public void doTestIndexFetchOnLeaderRestart() throws Exception  {
     useFactory(null);
     try {
       clearIndexWithReplication();
-      // change solrconfig having 'replicateAfter startup' option on master
-      master.copyConfigFile(CONF_DIR + "solrconfig-master2.xml",
+      // change solrconfig having 'replicateAfter startup' option on leader
+      leader.copyConfigFile(CONF_DIR + "solrconfig-leader2.xml",
           "solrconfig.xml");
 
-      masterJetty.stop();
-      masterJetty.start();
+      leaderJetty.stop();
+      leaderJetty.start();
 
-      // close and re-create master client because its connection pool has stale connections
-      masterClient.close();
-      masterClient = createNewSolrClient(masterJetty.getLocalPort());
+      // close and re-create leader client because its connection pool has stale connections
+      leaderClient.close();
+      leaderClient = createNewSolrClient(leaderJetty.getLocalPort());
 
       nDocs--;
       for (int i = 0; i < nDocs; i++)
-        index(masterClient, "id", i, "name", "name = " + i);
+        index(leaderClient, "id", i, "name", "name = " + i);
 
-      masterClient.commit();
+      leaderClient.commit();
 
       @SuppressWarnings({"rawtypes"})
-      NamedList masterQueryRsp = rQuery(nDocs, "*:*", masterClient);
-      SolrDocumentList masterQueryResult = (SolrDocumentList) masterQueryRsp.get("response");
-      assertEquals(nDocs, numFound(masterQueryRsp));
+      NamedList leaderQueryRsp = rQuery(nDocs, "*:*", leaderClient);
+      SolrDocumentList leaderQueryResult = (SolrDocumentList) leaderQueryRsp.get("response");
+      assertEquals(nDocs, numFound(leaderQueryRsp));
 
-      //get docs from slave and check if number is equal to master
+      //get docs from follower and check if number is equal to leader
       @SuppressWarnings({"rawtypes"})
-      NamedList slaveQueryRsp = rQuery(nDocs, "*:*", slaveClient);
-      SolrDocumentList slaveQueryResult = (SolrDocumentList) slaveQueryRsp.get("response");
-      assertEquals(nDocs, numFound(slaveQueryRsp));
+      NamedList followerQueryRsp = rQuery(nDocs, "*:*", followerClient);
+      SolrDocumentList followerQueryResult = (SolrDocumentList) followerQueryRsp.get("response");
+      assertEquals(nDocs, numFound(followerQueryRsp));
 
       //compare results
-      String cmp = BaseDistributedSearchTestCase.compare(masterQueryResult, slaveQueryResult, 0, null);
+      String cmp = BaseDistributedSearchTestCase.compare(leaderQueryResult, followerQueryResult, 0, null);
       assertEquals(null, cmp);
 
-      String timesReplicatedString = getSlaveDetails("timesIndexReplicated");
+      String timesReplicatedString = getFollowerDetails("timesIndexReplicated");
       String timesFailed;
       Integer previousTimesFailed = null;
       if (timesReplicatedString == null) {
         timesFailed = "0";
       } else {
         int timesReplicated = Integer.parseInt(timesReplicatedString);
-        timesFailed = getSlaveDetails("timesFailed");
+        timesFailed = getFollowerDetails("timesFailed");
         if (null == timesFailed) {
           timesFailed = "0";
         }
 
         previousTimesFailed = Integer.parseInt(timesFailed);
-        // Sometimes replication will fail because master's core is still loading; make sure there was one success
+        // Sometimes replication will fail because leader's core is still loading; make sure there was one success
         assertEquals(1, timesReplicated - previousTimesFailed);
 
       }
 
-      masterJetty.stop();
+      leaderJetty.stop();
 
       final TimeOut waitForLeaderToShutdown = new TimeOut(300, TimeUnit.SECONDS, TimeSource.NANO_TIME);
       waitForLeaderToShutdown.waitFor
         ("Gave up after waiting an obscene amount of time for leader to shut down",
-         () -> masterJetty.isStopped() );
+         () -> leaderJetty.isStopped() );
         
       for(int retries=0; ;retries++) { 
 
         Thread.yield(); // might not be necessary at all
-        // poll interval on slave is 1 second, so we just sleep for a few seconds
+        // poll interval on follower is 1 second, so we just sleep for a few seconds
         Thread.sleep(2000);
         
-        NamedList<Object> slaveDetails=null;
+        NamedList<Object> followerDetails=null;
         try {
-          slaveDetails = getSlaveDetails();
-          int failed = Integer.parseInt(getStringOrNull(slaveDetails,"timesFailed"));
+          followerDetails = getFollowerDetails();
+          int failed = Integer.parseInt(getStringOrNull(followerDetails,"timesFailed"));
           if (previousTimesFailed != null) {
             assertTrue(failed > previousTimesFailed);
           }
-          assertEquals(1, Integer.parseInt(getStringOrNull(slaveDetails,"timesIndexReplicated")) - failed);
+          assertEquals(1, Integer.parseInt(getStringOrNull(followerDetails,"timesIndexReplicated")) - failed);
           break;
         } catch (NumberFormatException | AssertionError notYet) {
           if (log.isInfoEnabled()) {
-            log.info("{}th attempt failure on {} details are {}", retries + 1, notYet, slaveDetails); // logOk
+            log.info("{}th attempt failure on {} details are {}", retries + 1, notYet, followerDetails); // logOk
           }
           if (retries>9) {
             log.error("giving up: ", notYet);
@@ -726,22 +761,22 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
         }
       }
       
-      masterJetty.start();
+      leaderJetty.start();
 
-      // poll interval on slave is 1 second, so we just sleep for a few seconds
+      // poll interval on follower is 1 second, so we just sleep for a few seconds
       Thread.sleep(2000);
-      //get docs from slave and assert that they are still the same as before
-      slaveQueryRsp = rQuery(nDocs, "*:*", slaveClient);
-      slaveQueryResult = (SolrDocumentList) slaveQueryRsp.get("response");
-      assertEquals(nDocs, numFound(slaveQueryRsp));
+      //get docs from follower and assert that they are still the same as before
+      followerQueryRsp = rQuery(nDocs, "*:*", followerClient);
+      followerQueryResult = (SolrDocumentList) followerQueryRsp.get("response");
+      assertEquals(nDocs, numFound(followerQueryRsp));
 
     } finally {
       resetFactory();
     }
   }
 
-  private String getSlaveDetails(String keyName) throws SolrServerException, IOException {
-    NamedList<Object> details = getSlaveDetails();
+  private String getFollowerDetails(String keyName) throws SolrServerException, IOException {
+    NamedList<Object> details = getFollowerDetails();
     return getStringOrNull(details, keyName);
   }
 
@@ -750,149 +785,159 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
     return o != null ? o.toString() : null;
   }
 
-  private NamedList<Object> getSlaveDetails() throws SolrServerException, IOException {
+  private NamedList<Object> getFollowerDetails() throws SolrServerException, IOException {
     ModifiableSolrParams params = new ModifiableSolrParams();
     params.set(CommonParams.QT, "/replication");
     params.set("command", "details");
-    QueryResponse response = slaveClient.query(params);
+    if (useLegacyParams) {
+      params.set("slave", "true");
+    } else {
+      params.set("follower", "true");
+    }
+    QueryResponse response = followerClient.query(params);
 
-    // details/slave/timesIndexReplicated
+    // details/follower/timesIndexReplicated
     @SuppressWarnings({"unchecked"})
     NamedList<Object> details = (NamedList<Object>) response.getResponse().get("details");
     @SuppressWarnings({"unchecked"})
-    NamedList<Object> slave = (NamedList<Object>) details.get("slave");
-    return slave;
+    NamedList<Object> follower = (NamedList<Object>) details.get("slave");
+    return follower;
   }
 
   @Test
-  public void doTestIndexFetchWithMasterUrl() throws Exception {
-    //change solrconfig on slave
+  public void doTestIndexFetchWithLeaderUrl() throws Exception {
+    //change solrconfig on follower
     //this has no entry for pollinginterval
-    slave.setTestPort(masterJetty.getLocalPort());
-    slave.copyConfigFile(CONF_DIR + "solrconfig-slave1.xml", "solrconfig.xml");
-    slaveJetty.stop();
-    slaveJetty = createAndStartJetty(slave);
-    slaveClient.close();
-    slaveClient = createNewSolrClient(slaveJetty.getLocalPort());
-
-    masterClient.deleteByQuery("*:*");
-    slaveClient.deleteByQuery("*:*");
-    slaveClient.commit();
+    follower.setTestPort(leaderJetty.getLocalPort());
+    follower.copyConfigFile(CONF_DIR + "solrconfig-follower1.xml", "solrconfig.xml");
+    followerJetty.stop();
+    followerJetty = createAndStartJetty(follower);
+    followerClient.close();
+    followerClient = createNewSolrClient(followerJetty.getLocalPort());
+
+    leaderClient.deleteByQuery("*:*");
+    followerClient.deleteByQuery("*:*");
+    followerClient.commit();
     nDocs--;
     for (int i = 0; i < nDocs; i++)
-      index(masterClient, "id", i, "name", "name = " + i);
+      index(leaderClient, "id", i, "name", "name = " + i);
 
     // make sure prepareCommit doesn't mess up commit  (SOLR-3938)
     
     // todo: make SolrJ easier to pass arbitrary params to
     // TODO: precommit WILL screw with the rest of this test
 
-    masterClient.commit();
+    leaderClient.commit();
 
     @SuppressWarnings({"rawtypes"})
-    NamedList masterQueryRsp = rQuery(nDocs, "*:*", masterClient);
-    SolrDocumentList masterQueryResult = (SolrDocumentList) masterQueryRsp.get("response");
-    assertEquals(nDocs, masterQueryResult.getNumFound());
+    NamedList leaderQueryRsp = rQuery(nDocs, "*:*", leaderClient);
+    SolrDocumentList leaderQueryResult = (SolrDocumentList) leaderQueryRsp.get("response");
+    assertEquals(nDocs, leaderQueryResult.getNumFound());
+    
+    String urlKey = "leaderUrl";
+    if (useLegacyParams) {
+      urlKey = "masterUrl";
+    }
 
     // index fetch
-    String masterUrl = buildUrl(slaveJetty.getLocalPort()) + "/" + DEFAULT_TEST_CORENAME + ReplicationHandler.PATH+"?command=fetchindex&masterUrl=";
-    masterUrl += buildUrl(masterJetty.getLocalPort()) + "/" + DEFAULT_TEST_CORENAME + ReplicationHandler.PATH;
-    URL url = new URL(masterUrl);
+    String leaderUrl = buildUrl(followerJetty.getLocalPort()) + "/" + DEFAULT_TEST_CORENAME + ReplicationHandler.PATH+"?command=fetchindex&" + urlKey + "=";
+    leaderUrl += buildUrl(leaderJetty.getLocalPort()) + "/" + DEFAULT_TEST_CORENAME + ReplicationHandler.PATH;
+    URL url = new URL(leaderUrl);
     InputStream stream = url.openStream();
     stream.close();
     
-    //get docs from slave and check if number is equal to master
+    //get docs from follower and check if number is equal to leader
     @SuppressWarnings({"rawtypes"})
-    NamedList slaveQueryRsp = rQuery(nDocs, "*:*", slaveClient);
-    SolrDocumentList slaveQueryResult = (SolrDocumentList) slaveQueryRsp.get("response");
-    assertEquals(nDocs, slaveQueryResult.getNumFound());
+    NamedList followerQueryRsp = rQuery(nDocs, "*:*", followerClient);
+    SolrDocumentList followerQueryResult = (SolrDocumentList) followerQueryRsp.get("response");
+    assertEquals(nDocs, followerQueryResult.getNumFound());
     //compare results
-    String cmp = BaseDistributedSearchTestCase.compare(masterQueryResult, slaveQueryResult, 0, null);
+    String cmp = BaseDistributedSearchTestCase.compare(leaderQueryResult, followerQueryResult, 0, null);
     assertEquals(null, cmp);
 
-    // index fetch from the slave to the master
+    // index fetch from the follower to the leader
     
     for (int i = nDocs; i < nDocs + 3; i++)
-      index(slaveClient, "id", i, "name", "name = " + i);
+      index(followerClient, "id", i, "name", "name = " + i);
 
-    slaveClient.commit();
+    followerClient.commit();
     
-    pullFromSlaveToMaster();
-    rQuery(nDocs + 3, "*:*", masterClient);
+    pullFromFollowerToLeader();
+    rQuery(nDocs + 3, "*:*", leaderClient);
     
-    //get docs from slave and check if number is equal to master
-    slaveQueryRsp = rQuery(nDocs + 3, "*:*", slaveClient);
-    slaveQueryResult = (SolrDocumentList) slaveQueryRsp.get("response");
-    assertEquals(nDocs + 3, slaveQueryResult.getNumFound());
+    //get docs from follower and check if number is equal to leader
+    followerQueryRsp = rQuery(nDocs + 3, "*:*", followerClient);
+    followerQueryResult = (SolrDocumentList) followerQueryRsp.get("response");
+    assertEquals(nDocs + 3, followerQueryResult.getNumFound());
     //compare results
-    masterQueryRsp = rQuery(nDocs + 3, "*:*", masterClient);
-    masterQueryResult = (SolrDocumentList) masterQueryRsp.get("response");
-    cmp = BaseDistributedSearchTestCase.compare(masterQueryResult, slaveQueryResult, 0, null);
+    leaderQueryRsp = rQuery(nDocs + 3, "*:*", leaderClient);
+    leaderQueryResult = (SolrDocumentList) leaderQueryRsp.get("response");
+    cmp = BaseDistributedSearchTestCase.compare(leaderQueryResult, followerQueryResult, 0, null);
     assertEquals(null, cmp);
 
-    assertVersions(masterClient, slaveClient);
+    assertVersions(leaderClient, followerClient);
     
-    pullFromSlaveToMaster();
+    pullFromFollowerToLeader();
     
-    //get docs from slave and check if number is equal to master
-    slaveQueryRsp = rQuery(nDocs + 3, "*:*", slaveClient);
-    slaveQueryResult = (SolrDocumentList) slaveQueryRsp.get("response");
-    assertEquals(nDocs + 3, slaveQueryResult.getNumFound());
+    //get docs from follower and check if number is equal to leader
+    followerQueryRsp = rQuery(nDocs + 3, "*:*", followerClient);
+    followerQueryResult = (SolrDocumentList) followerQueryRsp.get("response");
+    assertEquals(nDocs + 3, followerQueryResult.getNumFound());
     //compare results
-    masterQueryRsp = rQuery(nDocs + 3, "*:*", masterClient);
-    masterQueryResult = (SolrDocumentList) masterQueryRsp.get("response");
-    cmp = BaseDistributedSearchTestCase.compare(masterQueryResult, slaveQueryResult, 0, null);
+    leaderQueryRsp = rQuery(nDocs + 3, "*:*", leaderClient);
+    leaderQueryResult = (SolrDocumentList) leaderQueryRsp.get("response");
+    cmp = BaseDistributedSearchTestCase.compare(leaderQueryResult, followerQueryResult, 0, null);
     assertEquals(null, cmp);
     
-    assertVersions(masterClient, slaveClient);
+    assertVersions(leaderClient, followerClient);
     
     // now force a new index directory
     for (int i = nDocs + 3; i < nDocs + 7; i++)
-      index(masterClient, "id", i, "name", "name = " + i);
+      index(leaderClient, "id", i, "name", "name = " + i);
     
-    masterClient.commit();
+    leaderClient.commit();
     
-    pullFromSlaveToMaster();
-    rQuery((int) slaveQueryResult.getNumFound(), "*:*", masterClient);
+    pullFromFollowerToLeader();
+    rQuery((int) followerQueryResult.getNumFound(), "*:*", leaderClient);
     
-    //get docs from slave and check if number is equal to master
-    slaveQueryRsp = rQuery(nDocs + 3, "*:*", slaveClient);
-    slaveQueryResult = (SolrDocumentList) slaveQueryRsp.get("response");
-    assertEquals(nDocs + 3, slaveQueryResult.getNumFound());
+    //get docs from follower and check if number is equal to leader
+    followerQueryRsp = rQuery(nDocs + 3, "*:*", followerClient);
+    followerQueryResult = (SolrDocumentList) followerQueryRsp.get("response");
+    assertEquals(nDocs + 3, followerQueryResult.getNumFound());
     //compare results
-    masterQueryRsp = rQuery(nDocs + 3, "*:*", masterClient);
-    masterQueryResult = (SolrDocumentList) masterQueryRsp.get("response");
-    cmp = BaseDistributedSearchTestCase.compare(masterQueryResult, slaveQueryResult, 0, null);
+    leaderQueryRsp = rQuery(nDocs + 3, "*:*", leaderClient);
+    leaderQueryResult = (SolrDocumentList) leaderQueryRsp.get("response");
+    cmp = BaseDistributedSearchTestCase.compare(leaderQueryResult, followerQueryResult, 0, null);
     assertEquals(null, cmp);
     
-    assertVersions(masterClient, slaveClient);
-    pullFromSlaveToMaster();
+    assertVersions(leaderClient, followerClient);
+    pullFromFollowerToLeader();
     
-    //get docs from slave and check if number is equal to master
-    slaveQueryRsp = rQuery(nDocs + 3, "*:*", slaveClient);
-    slaveQueryResult = (SolrDocumentList) slaveQueryRsp.get("response");
-    assertEquals(nDocs + 3, slaveQueryResult.getNumFound());
+    //get docs from follower and check if number is equal to leader
+    followerQueryRsp = rQuery(nDocs + 3, "*:*", followerClient);
+    followerQueryResult = (SolrDocumentList) followerQueryRsp.get("response");
+    assertEquals(nDocs + 3, followerQueryResult.getNumFound());
     //compare results
-    masterQueryRsp = rQuery(nDocs + 3, "*:*", masterClient);
-    masterQueryResult = (SolrDocumentList) masterQueryRsp.get("response");
-    cmp = BaseDistributedSearchTestCase.compare(masterQueryResult, slaveQueryResult, 0, null);
+    leaderQueryRsp = rQuery(nDocs + 3, "*:*", leaderClient);
+    leaderQueryResult = (SolrDocumentList) leaderQueryRsp.get("response");
+    cmp = BaseDistributedSearchTestCase.compare(leaderQueryResult, followerQueryResult, 0, null);
     assertEquals(null, cmp);
     
-    assertVersions(masterClient, slaveClient);
+    assertVersions(leaderClient, followerClient);
     
-    NamedList<Object> details = getDetails(masterClient);
+    NamedList<Object> details = getDetails(leaderClient);
    
-    details = getDetails(slaveClient);
+    details = getDetails(followerClient);
     
-    checkForSingleIndex(masterJetty);
-    checkForSingleIndex(slaveJetty);
+    checkForSingleIndex(leaderJetty);
+    checkForSingleIndex(followerJetty);
   }
   
   
   @Test
   //commented 20-Sep-2018  @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // added 17-Aug-2018
   public void doTestStressReplication() throws Exception {
-    // change solrconfig on slave
+    // change solrconfig on follower
     // this has no entry for pollinginterval
     
     // get us a straight standard fs dir rather than mock*dir
@@ -901,30 +946,30 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
     if (useStraightStandardDirectory) {
       useFactory(null);
     }
-    final String SLAVE_SCHEMA_1 = "schema-replication1.xml";
-    final String SLAVE_SCHEMA_2 = "schema-replication2.xml";
-    String slaveSchema = SLAVE_SCHEMA_1;
+    final String FOLLOWER_SCHEMA_1 = "schema-replication1.xml";
+    final String FOLLOWER_SCHEMA_2 = "schema-replication2.xml";
+    String followerSchema = FOLLOWER_SCHEMA_1;
 
     try {
 
-      slave.setTestPort(masterJetty.getLocalPort());
-      slave.copyConfigFile(CONF_DIR +"solrconfig-slave1.xml", "solrconfig.xml");
-      slave.copyConfigFile(CONF_DIR +slaveSchema, "schema.xml");
-      slaveJetty.stop();
-      slaveJetty = createAndStartJetty(slave);
-      slaveClient.close();
-      slaveClient = createNewSolrClient(slaveJetty.getLocalPort());
+      follower.setTestPort(leaderJetty.getLocalPort());
+      follower.copyConfigFile(CONF_DIR +"solrconfig-follower1.xml", "solrconfig.xml");
+      follower.copyConfigFile(CONF_DIR +followerSchema, "schema.xml");
+      followerJetty.stop();
+      followerJetty = createAndStartJetty(follower);
+      followerClient.close();
+      followerClient = createNewSolrClient(followerJetty.getLocalPort());
 
-      master.copyConfigFile(CONF_DIR + "solrconfig-master3.xml",
+      leader.copyConfigFile(CONF_DIR + "solrconfig-leader3.xml",
           "solrconfig.xml");
-      masterJetty.stop();
-      masterJetty = createAndStartJetty(master);
-      masterClient.close();
-      masterClient = createNewSolrClient(masterJetty.getLocalPort());
+      leaderJetty.stop();
+      leaderJetty = createAndStartJetty(leader);
+      leaderClient.close();
+      leaderClient = createNewSolrClient(leaderJetty.getLocalPort());
       
-      masterClient.deleteByQuery("*:*");
-      slaveClient.deleteByQuery("*:*");
-      slaveClient.commit();
+      leaderClient.deleteByQuery("*:*");
+      followerClient.deleteByQuery("*:*");
+      followerClient.commit();
       
       int maxDocs = TEST_NIGHTLY ? 1000 : 75;
       int rounds = TEST_NIGHTLY ? 45 : 3;
@@ -936,57 +981,57 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
         if (confCoreReload) {
           // toggle the schema file used
 
-          slaveSchema = slaveSchema.equals(SLAVE_SCHEMA_1) ? 
-            SLAVE_SCHEMA_2 : SLAVE_SCHEMA_1;
-          master.copyConfigFile(CONF_DIR + slaveSchema, "schema.xml");
+          followerSchema = followerSchema.equals(FOLLOWER_SCHEMA_1) ?
+            FOLLOWER_SCHEMA_2 : FOLLOWER_SCHEMA_1;
+          leader.copyConfigFile(CONF_DIR + followerSchema, "schema.xml");
         }
         
         int docs = random().nextInt(maxDocs) + 1;
         for (int i = 0; i < docs; i++) {
-          index(masterClient, "id", id++, "name", "name = " + i);
+          index(leaderClient, "id", id++, "name", "name = " + i);
         }
         
         totalDocs += docs;
-        masterClient.commit();
+        leaderClient.commit();
         
         @SuppressWarnings({"rawtypes"})
-        NamedList masterQueryRsp = rQuery(totalDocs, "*:*", masterClient);
-        SolrDocumentList masterQueryResult = (SolrDocumentList) masterQueryRsp
+        NamedList leaderQueryRsp = rQuery(totalDocs, "*:*", leaderClient);
+        SolrDocumentList leaderQueryResult = (SolrDocumentList) leaderQueryRsp
             .get("response");
-        assertEquals(totalDocs, masterQueryResult.getNumFound());
+        assertEquals(totalDocs, leaderQueryResult.getNumFound());
         
         // index fetch
-        Date slaveCoreStart = watchCoreStartAt(slaveClient, 30*1000, null);
-        pullFromMasterToSlave();
+        Date followerCoreStart = watchCoreStartAt(followerClient, 30*1000, null);
+        pullFromLeaderToFollower();
         if (confCoreReload) {
-          watchCoreStartAt(slaveClient, 30*1000, slaveCoreStart);
+          watchCoreStartAt(followerClient, 30*1000, followerCoreStart);
         }
 
-        // get docs from slave and check if number is equal to master
+        // get docs from follower and check if number is equal to leader
         @SuppressWarnings({"rawtypes"})
-        NamedList slaveQueryRsp = rQuery(totalDocs, "*:*", slaveClient);
-        SolrDocumentList slaveQueryResult = (SolrDocumentList) slaveQueryRsp
+        NamedList followerQueryRsp = rQuery(totalDocs, "*:*", followerClient);
+        SolrDocumentList followerQueryResult = (SolrDocumentList) followerQueryRsp
             .get("response");
-        assertEquals(totalDocs, slaveQueryResult.getNumFound());
+        assertEquals(totalDocs, followerQueryResult.getNumFound());
         // compare results
-        String cmp = BaseDistributedSearchTestCase.compare(masterQueryResult,
-            slaveQueryResult, 0, null);
+        String cmp = BaseDistributedSearchTestCase.compare(leaderQueryResult,
+            followerQueryResult, 0, null);
         assertEquals(null, cmp);
         
-        assertVersions(masterClient, slaveClient);
+        assertVersions(leaderClient, followerClient);
         
-        checkForSingleIndex(masterJetty);
+        checkForSingleIndex(leaderJetty);
         
         if (!Constants.WINDOWS) {
-          checkForSingleIndex(slaveJetty);
+          checkForSingleIndex(followerJetty);
         }
         
         if (random().nextBoolean()) {
-          // move the slave ahead
+          // move the follower ahead
           for (int i = 0; i < 3; i++) {
-            index(slaveClient, "id", id++, "name", "name = " + i);
+            index(followerClient, "id", id++, "name", "name = " + i);
           }
-          slaveClient.commit();
+          followerClient.commit();
         }
         
       }
@@ -1050,23 +1095,23 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
     return list.length;
   }
 
-  private void pullFromMasterToSlave() throws MalformedURLException,
+  private void pullFromLeaderToFollower() throws MalformedURLException,
       IOException {
-    pullFromTo(masterJetty, slaveJetty);
+    pullFromTo(leaderJetty, followerJetty);
   }
   
   @Test
   public void doTestRepeater() throws Exception {
     // no polling
-    slave.setTestPort(masterJetty.getLocalPort());
-    slave.copyConfigFile(CONF_DIR + "solrconfig-slave1.xml", "solrconfig.xml");
-    slaveJetty.stop();
-    slaveJetty = createAndStartJetty(slave);
-    slaveClient.close();
-    slaveClient = createNewSolrClient(slaveJetty.getLocalPort());
+    follower.setTestPort(leaderJetty.getLocalPort());
+    follower.copyConfigFile(CONF_DIR + "solrconfig-follower1.xml", "solrconfig.xml");
+    followerJetty.stop();
+    followerJetty = createAndStartJetty(follower);
+    followerClient.close();
+    followerClient = createNewSolrClient(followerJetty.getLocalPort());
 
     try {
-      repeater = new SolrInstance(createTempDir("solr-instance").toFile(), "repeater", masterJetty.getLocalPort());
+      repeater = new SolrInstance(createTempDir("solr-instance").toFile(), "repeater", leaderJetty.getLocalPort());
       repeater.setUp();
       repeater.copyConfigFile(CONF_DIR + "solrconfig-repeater.xml",
           "solrconfig.xml");
@@ -1077,45 +1122,45 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
       repeaterClient = createNewSolrClient(repeaterJetty.getLocalPort());
       
       for (int i = 0; i < 3; i++)
-        index(masterClient, "id", i, "name", "name = " + i);
+        index(leaderClient, "id", i, "name", "name = " + i);
 
-      masterClient.commit();
+      leaderClient.commit();
       
-      pullFromTo(masterJetty, repeaterJetty);
+      pullFromTo(leaderJetty, repeaterJetty);
       
       rQuery(3, "*:*", repeaterClient);
       
-      pullFromTo(repeaterJetty, slaveJetty);
+      pullFromTo(repeaterJetty, followerJetty);
       
-      rQuery(3, "*:*", slaveClient);
+      rQuery(3, "*:*", followerClient);
       
-      assertVersions(masterClient, repeaterClient);
-      assertVersions(repeaterClient, slaveClient);
+      assertVersions(leaderClient, repeaterClient);
+      assertVersions(repeaterClient, followerClient);
       
       for (int i = 0; i < 4; i++)
         index(repeaterClient, "id", i, "name", "name = " + i);
       repeaterClient.commit();
       
-      pullFromTo(masterJetty, repeaterJetty);
+      pullFromTo(leaderJetty, repeaterJetty);
       
       rQuery(3, "*:*", repeaterClient);
       
-      pullFromTo(repeaterJetty, slaveJetty);
+      pullFromTo(repeaterJetty, followerJetty);
       
-      rQuery(3, "*:*", slaveClient);
+      rQuery(3, "*:*", followerClient);
       
       for (int i = 3; i < 6; i++)
-        index(masterClient, "id", i, "name", "name = " + i);
+        index(leaderClient, "id", i, "name", "name = " + i);
       
-      masterClient.commit();
+      leaderClient.commit();
       
-      pullFromTo(masterJetty, repeaterJetty);
+      pullFromTo(leaderJetty, repeaterJetty);
       
       rQuery(6, "*:*", repeaterClient);
       
-      pullFromTo(repeaterJetty, slaveJetty);
+      pullFromTo(repeaterJetty, followerJetty);
       
-      rQuery(6, "*:*", slaveClient);
+      rQuery(6, "*:*", followerClient);
 
     } finally {
       if (repeater != null) {
@@ -1164,82 +1209,82 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
     ArrayList<NamedList<Object>> commits;
     details = getDetails(client);
     commits = (ArrayList<NamedList<Object>>) details.get("commits");
-    Long maxVersionSlave= 0L;
+    Long maxVersionFollower= 0L;
     for(NamedList<Object> commit : commits) {
       Long version = (Long) commit.get("indexVersion");
-      maxVersionSlave = Math.max(version, maxVersionSlave);
+      maxVersionFollower = Math.max(version, maxVersionFollower);
     }
-    return maxVersionSlave;
+    return maxVersionFollower;
   }
 
-  private void pullFromSlaveToMaster() throws MalformedURLException,
+  private void pullFromFollowerToLeader() throws MalformedURLException,
       IOException {
-    pullFromTo(slaveJetty, masterJetty);
+    pullFromTo(followerJetty, leaderJetty);
   }
   
   private void pullFromTo(JettySolrRunner from, JettySolrRunner to) throws IOException {
-    String masterUrl;
+    String leaderUrl;
     URL url;
     InputStream stream;
-    masterUrl = buildUrl(to.getLocalPort())
+    leaderUrl = buildUrl(to.getLocalPort())
         + "/" + DEFAULT_TEST_CORENAME
-        + ReplicationHandler.PATH+"?wait=true&command=fetchindex&masterUrl="
+        + ReplicationHandler.PATH+"?wait=true&command=fetchindex&leaderUrl="
         + buildUrl(from.getLocalPort())
         + "/" + DEFAULT_TEST_CORENAME + ReplicationHandler.PATH;
-    url = new URL(masterUrl);
+    url = new URL(leaderUrl);
     stream = url.openStream();
     stream.close();
   }
 
   @Test
   public void doTestReplicateAfterStartup() throws Exception {
-    //stop slave
-    slaveJetty.stop();
+    //stop follower
+    followerJetty.stop();
 
     nDocs--;
-    masterClient.deleteByQuery("*:*");
+    leaderClient.deleteByQuery("*:*");
 
-    masterClient.commit();
+    leaderClient.commit();
 
 
 
-    //change solrconfig having 'replicateAfter startup' option on master
-    master.copyConfigFile(CONF_DIR + "solrconfig-master2.xml",
+    //change solrconfig having 'replicateAfter startup' option on leader
+    leader.copyConfigFile(CONF_DIR + "solrconfig-leader2.xml",
                           "solrconfig.xml");
 
-    masterJetty.stop();
+    leaderJetty.stop();
 
-    masterJetty = createAndStartJetty(master);
-    masterClient.close();
-    masterClient = createNewSolrClient(masterJetty.getLocalPort());
+    leaderJetty = createAndStartJetty(leader);
+    leaderClient.close();
+    leaderClient = createNewSolrClient(leaderJetty.getLocalPort());
     
     for (int i = 0; i < nDocs; i++)
-      index(masterClient, "id", i, "name", "name = " + i);
+      index(leaderClient, "id", i, "name", "name = " + i);
 
-    masterClient.commit();
+    leaderClient.commit();
     
     @SuppressWarnings({"rawtypes"})
-    NamedList masterQueryRsp = rQuery(nDocs, "*:*", masterClient);
-    SolrDocumentList masterQueryResult = (SolrDocumentList) masterQueryRsp.get("response");
-    assertEquals(nDocs, masterQueryResult.getNumFound());
+    NamedList leaderQueryRsp = rQuery(nDocs, "*:*", leaderClient);
+    SolrDocumentList leaderQueryResult = (SolrDocumentList) leaderQueryRsp.get("response");
+    assertEquals(nDocs, leaderQueryResult.getNumFound());
     
 
-    slave.setTestPort(masterJetty.getLocalPort());
-    slave.copyConfigFile(slave.getSolrConfigFile(), "solrconfig.xml");
+    follower.setTestPort(leaderJetty.getLocalPort());
+    follower.copyConfigFile(follower.getSolrConfigFile(), "solrconfig.xml");
 
-    //start slave
-    slaveJetty = createAndStartJetty(slave);
-    slaveClient.close();
-    slaveClient = createNewSolrClient(slaveJetty.getLocalPort());
+    //start follower
+    followerJetty = createAndStartJetty(follower);
+    followerClient.close();
+    followerClient = createNewSolrClient(followerJetty.getLocalPort());
 
-    //get docs from slave and check if number is equal to master
+    //get docs from follower and check if number is equal to leader
     @SuppressWarnings({"rawtypes"})
-    NamedList slaveQueryRsp = rQuery(nDocs, "*:*", slaveClient);
-    SolrDocumentList slaveQueryResult = (SolrDocumentList) slaveQueryRsp.get("response");
-    assertEquals(nDocs, slaveQueryResult.getNumFound());
+    NamedList followerQueryRsp = rQuery(nDocs, "*:*", followerClient);
+    SolrDocumentList followerQueryResult = (SolrDocumentList) followerQueryRsp.get("response");
+    assertEquals(nDocs, followerQueryResult.getNumFound());
 
     //compare results
-    String cmp = BaseDistributedSearchTestCase.compare(masterQueryResult, slaveQueryResult, 0, null);
+    String cmp = BaseDistributedSearchTestCase.compare(leaderQueryResult, followerQueryResult, 0, null);
     assertEquals(null, cmp);
 
   }
@@ -1249,61 +1294,61 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
     useFactory(null);
     try {
       
-      // stop slave
-      slaveJetty.stop();
+      // stop follower
+      followerJetty.stop();
       
       nDocs--;
-      masterClient.deleteByQuery("*:*");
+      leaderClient.deleteByQuery("*:*");
       
-      masterClient.commit();
+      leaderClient.commit();
       
-      // change solrconfig having 'replicateAfter startup' option on master
-      master.copyConfigFile(CONF_DIR + "solrconfig-master2.xml",
+      // change solrconfig having 'replicateAfter startup' option on leader
+      leader.copyConfigFile(CONF_DIR + "solrconfig-leader2.xml",
           "solrconfig.xml");
       
-      masterJetty.stop();
+      leaderJetty.stop();
       
-      masterJetty = createAndStartJetty(master);
-      masterClient.close();
-      masterClient = createNewSolrClient(masterJetty.getLocalPort());
+      leaderJetty = createAndStartJetty(leader);
+      leaderClient.close();
+      leaderClient = createNewSolrClient(leaderJetty.getLocalPort());
       
       for (int i = 0; i < nDocs; i++)
-        index(masterClient, "id", i, "name", "name = " + i);
+        index(leaderClient, "id", i, "name", "name = " + i);
       
-      masterClient.commit();
+      leaderClient.commit();
       
-      // now we restart to test what happens with no activity before the slave
+      // now we restart to test what happens with no activity before the follower
       // tries to
       // replicate
-      masterJetty.stop();
-      masterJetty.start();
+      leaderJetty.stop();
+      leaderJetty.start();
       
-      // masterClient = createNewSolrClient(masterJetty.getLocalPort());
+      // leaderClient = createNewSolrClient(leaderJetty.getLocalPort());
       
       @SuppressWarnings({"rawtypes"})
-      NamedList masterQueryRsp = rQuery(nDocs, "*:*", masterClient);
-      SolrDocumentList masterQueryResult = (SolrDocumentList) masterQueryRsp
+      NamedList leaderQueryRsp = rQuery(nDocs, "*:*", leaderClient);
+      SolrDocumentList leaderQueryResult = (SolrDocumentList) leaderQueryRsp
           .get("response");
-      assertEquals(nDocs, masterQueryResult.getNumFound());
+      assertEquals(nDocs, leaderQueryResult.getNumFound());
       
-      slave.setTestPort(masterJetty.getLocalPort());
-      slave.copyConfigFile(slave.getSolrConfigFile(), "solrconfig.xml");
+      follower.setTestPort(leaderJetty.getLocalPort());
+      follower.copyConfigFile(follower.getSolrConfigFile(), "solrconfig.xml");
       
-      // start slave
-      slaveJetty = createAndStartJetty(slave);
-      slaveClient.close();
-      slaveClient = createNewSolrClient(slaveJetty.getLocalPort());
+      // start follower
+      followerJetty = createAndStartJetty(follower);
+      followerClient.close();
+      followerClient = createNewSolrClient(followerJetty.getLocalPort());
       
-      // get docs from slave and check if number is equal to master
+      // get docs from follower and check if number is equal to leader
       @SuppressWarnings({"rawtypes"})
-      NamedList slaveQueryRsp = rQuery(nDocs, "*:*", slaveClient);
-      SolrDocumentList slaveQueryResult = (SolrDocumentList) slaveQueryRsp
+      NamedList followerQueryRsp = rQuery(nDocs, "*:*", followerClient);
+      SolrDocumentList followerQueryResult = (SolrDocumentList) followerQueryRsp
           .get("response");
-      assertEquals(nDocs, slaveQueryResult.getNumFound());
+      assertEquals(nDocs, followerQueryResult.getNumFound());
       
       // compare results
-      String cmp = BaseDistributedSearchTestCase.compare(masterQueryResult,
-          slaveQueryResult, 0, null);
+      String cmp = BaseDistributedSearchTestCase.compare(leaderQueryResult,
+          followerQueryResult, 0, null);
       assertEquals(null, cmp);
       
     } finally {
@@ -1315,69 +1360,69 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
   public void doTestReplicateAfterCoreReload() throws Exception {
     int docs = TEST_NIGHTLY ? 200000 : 10;
     
-    //stop slave
-    slaveJetty.stop();
+    //stop follower
+    followerJetty.stop();
 
 
-    //change solrconfig having 'replicateAfter startup' option on master
-    master.copyConfigFile(CONF_DIR + "solrconfig-master3.xml",
+    //change solrconfig having 'replicateAfter startup' option on leader
+    leader.copyConfigFile(CONF_DIR + "solrconfig-leader3.xml",
                           "solrconfig.xml");
 
-    masterJetty.stop();
+    leaderJetty.stop();
 
-    masterJetty = createAndStartJetty(master);
-    masterClient.close();
-    masterClient = createNewSolrClient(masterJetty.getLocalPort());
+    leaderJetty = createAndStartJetty(leader);
+    leaderClient.close();
+    leaderClient = createNewSolrClient(leaderJetty.getLocalPort());
 
-    masterClient.deleteByQuery("*:*");
+    leaderClient.deleteByQuery("*:*");
     for (int i = 0; i < docs; i++)
-      index(masterClient, "id", i, "name", "name = " + i);
+      index(leaderClient, "id", i, "name", "name = " + i);
 
-    masterClient.commit();
+    leaderClient.commit();
 
     @SuppressWarnings({"rawtypes"})
-    NamedList masterQueryRsp = rQuery(docs, "*:*", masterClient);
-    SolrDocumentList masterQueryResult = (SolrDocumentList) masterQueryRsp.get("response");
-    assertEquals(docs, masterQueryResult.getNumFound());
+    NamedList leaderQueryRsp = rQuery(docs, "*:*", leaderClient);
+    SolrDocumentList leaderQueryResult = (SolrDocumentList) leaderQueryRsp.get("response");
+    assertEquals(docs, leaderQueryResult.getNumFound());
     
-    slave.setTestPort(masterJetty.getLocalPort());
-    slave.copyConfigFile(slave.getSolrConfigFile(), "solrconfig.xml");
+    follower.setTestPort(leaderJetty.getLocalPort());
+    follower.copyConfigFile(follower.getSolrConfigFile(), "solrconfig.xml");
 
-    //start slave
-    slaveJetty = createAndStartJetty(slave);
-    slaveClient.close();
-    slaveClient = createNewSolrClient(slaveJetty.getLocalPort());
+    //start follower
+    followerJetty = createAndStartJetty(follower);
+    followerClient.close();
+    followerClient = createNewSolrClient(followerJetty.getLocalPort());
     
-    //get docs from slave and check if number is equal to master
+    //get docs from follower and check if number is equal to leader
     @SuppressWarnings({"rawtypes"})
-    NamedList slaveQueryRsp = rQuery(docs, "*:*", slaveClient);
-    SolrDocumentList slaveQueryResult = (SolrDocumentList) slaveQueryRsp.get("response");
-    assertEquals(docs, slaveQueryResult.getNumFound());
+    NamedList followerQueryRsp = rQuery(docs, "*:*", followerClient);
+    SolrDocumentList followerQueryResult = (SolrDocumentList) followerQueryRsp.get("response");
+    assertEquals(docs, followerQueryResult.getNumFound());
     
     //compare results
-    String cmp = BaseDistributedSearchTestCase.compare(masterQueryResult, slaveQueryResult, 0, null);
+    String cmp = BaseDistributedSearchTestCase.compare(leaderQueryResult, followerQueryResult, 0, null);
     assertEquals(null, cmp);
     
-    Object version = getIndexVersion(masterClient).get("indexversion");
+    Object version = getIndexVersion(leaderClient).get("indexversion");
     
-    reloadCore(masterClient, "collection1");
+    reloadCore(leaderClient, "collection1");
     
-    assertEquals(version, getIndexVersion(masterClient).get("indexversion"));
+    assertEquals(version, getIndexVersion(leaderClient).get("indexversion"));
     
-    index(masterClient, "id", docs + 10, "name", "name = 1");
-    index(masterClient, "id", docs + 20, "name", "name = 2");
+    index(leaderClient, "id", docs + 10, "name", "name = 1");
+    index(leaderClient, "id", docs + 20, "name", "name = 2");
 
-    masterClient.commit();
+    leaderClient.commit();
     
     @SuppressWarnings({"rawtypes"})
-    NamedList resp =  rQuery(docs + 2, "*:*", masterClient);
-    masterQueryResult = (SolrDocumentList) resp.get("response");
-    assertEquals(docs + 2, masterQueryResult.getNumFound());
+    NamedList resp =  rQuery(docs + 2, "*:*", leaderClient);
+    leaderQueryResult = (SolrDocumentList) resp.get("response");
+    assertEquals(docs + 2, leaderQueryResult.getNumFound());
     
-    //get docs from slave and check if number is equal to master
-    slaveQueryRsp = rQuery(docs + 2, "*:*", slaveClient);
-    slaveQueryResult = (SolrDocumentList) slaveQueryRsp.get("response");
-    assertEquals(docs + 2, slaveQueryResult.getNumFound());
+    //get docs from follower and check if number is equal to leader
+    followerQueryRsp = rQuery(docs + 2, "*:*", followerClient);
+    followerQueryResult = (SolrDocumentList) followerQueryRsp.get("response");
+    assertEquals(docs + 2, followerQueryResult.getNumFound());
     
   }
 
@@ -1388,117 +1433,117 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
 
     nDocs--;
     for (int i = 0; i < nDocs; i++)
-      index(masterClient, "id", i, "name", "name = " + i);
+      index(leaderClient, "id", i, "name", "name = " + i);
 
-    masterClient.commit();
+    leaderClient.commit();
 
     @SuppressWarnings({"rawtypes"})
-    NamedList masterQueryRsp = rQuery(nDocs, "*:*", masterClient);
-    SolrDocumentList masterQueryResult = (SolrDocumentList) masterQueryRsp.get("response");
-    assertEquals(nDocs, masterQueryResult.getNumFound());
+    NamedList leaderQueryRsp = rQuery(nDocs, "*:*", leaderClient);
+    SolrDocumentList leaderQueryResult = (SolrDocumentList) leaderQueryRsp.get("response");
+    assertEquals(nDocs, leaderQueryResult.getNumFound());
 
-    //get docs from slave and check if number is equal to master
+    //get docs from follower and check if number is equal to leader
     @SuppressWarnings({"rawtypes"})
-    NamedList slaveQueryRsp = rQuery(nDocs, "*:*", slaveClient);
-    SolrDocumentList slaveQueryResult = (SolrDocumentList) slaveQueryRsp.get("response");
+    NamedList followerQueryRsp = rQuery(nDocs, "*:*", followerClient);
+    SolrDocumentList followerQueryResult = (SolrDocumentList) followerQueryRsp.get("response");
 
-    assertEquals(nDocs, slaveQueryResult.getNumFound());
+    assertEquals(nDocs, followerQueryResult.getNumFound());
 
     //compare results
-    String cmp = BaseDistributedSearchTestCase.compare(masterQueryResult, slaveQueryResult, 0, null);
+    String cmp = BaseDistributedSearchTestCase.compare(leaderQueryResult, followerQueryResult, 0, null);
     assertEquals(null, cmp);
 
     //start config files replication test
-    //clear master index
-    masterClient.deleteByQuery("*:*");
-    masterClient.commit();
-    rQuery(0, "*:*", masterClient); // sanity check w/retry
+    //clear leader index
+    leaderClient.deleteByQuery("*:*");
+    leaderClient.commit();
+    rQuery(0, "*:*", leaderClient); // sanity check w/retry
 
-    //change solrconfig on master
-    master.copyConfigFile(CONF_DIR + "solrconfig-master1.xml", 
+    //change solrconfig on leader
+    leader.copyConfigFile(CONF_DIR + "solrconfig-leader1.xml",
                           "solrconfig.xml");
 
-    //change schema on master
-    master.copyConfigFile(CONF_DIR + "schema-replication2.xml", 
+    //change schema on leader
+    leader.copyConfigFile(CONF_DIR + "schema-replication2.xml",
                           "schema.xml");
 
     //keep a copy of the new schema
-    master.copyConfigFile(CONF_DIR + "schema-replication2.xml", 
+    leader.copyConfigFile(CONF_DIR + "schema-replication2.xml",
                           "schema-replication2.xml");
 
-    masterJetty.stop();
+    leaderJetty.stop();
 
-    masterJetty = createAndStartJetty(master);
-    masterClient.close();
-    masterClient = createNewSolrClient(masterJetty.getLocalPort());
+    leaderJetty = createAndStartJetty(leader);
+    leaderClient.close();
+    leaderClient = createNewSolrClient(leaderJetty.getLocalPort());
 
-    slave.setTestPort(masterJetty.getLocalPort());
-    slave.copyConfigFile(slave.getSolrConfigFile(), "solrconfig.xml");
+    follower.setTestPort(leaderJetty.getLocalPort());
+    follower.copyConfigFile(follower.getSolrConfigFile(), "solrconfig.xml");
 
-    slaveJetty.stop();
-    slaveJetty = createAndStartJetty(slave);
-    slaveClient.close();
-    slaveClient = createNewSolrClient(slaveJetty.getLocalPort());
+    followerJetty.stop();
+    followerJetty = createAndStartJetty(follower);
+    followerClient.close();
+    followerClient = createNewSolrClient(followerJetty.getLocalPort());
 
-    slaveClient.deleteByQuery("*:*");
-    slaveClient.commit();
-    rQuery(0, "*:*", slaveClient); // sanity check w/retry
+    followerClient.deleteByQuery("*:*");
+    followerClient.commit();
+    rQuery(0, "*:*", followerClient); // sanity check w/retry
     
-    // record collection1's start time on slave
-    final Date slaveStartTime = watchCoreStartAt(slaveClient, 30*1000, null);
+    // record collection1's start time on follower
+    final Date followerStartTime = watchCoreStartAt(followerClient, 30*1000, null);
 
-    //add a doc with new field and commit on master to trigger index fetch from slave.
-    index(masterClient, "id", "2000", "name", "name = " + 2000, "newname", "n2000");
-    masterClient.commit();
-    rQuery(1, "newname:n2000", masterClient);  // sanity check
+    //add a doc with new field and commit on leader to trigger index fetch from follower.
+    index(leaderClient, "id", "2000", "name", "name = " + 2000, "newname", "n2000");
+    leaderClient.commit();
+    rQuery(1, "newname:n2000", leaderClient);  // sanity check
 
-    // wait for slave to reload core by watching updated startTime
-    watchCoreStartAt(slaveClient, 30*1000, slaveStartTime);
+    // wait for follower to reload core by watching updated startTime
+    watchCoreStartAt(followerClient, 30*1000, followerStartTime);
 
     @SuppressWarnings({"rawtypes"})
-    NamedList masterQueryRsp2 = rQuery(1, "id:2000", masterClient);
-    SolrDocumentList masterQueryResult2 = (SolrDocumentList) masterQueryRsp2.get("response");
-    assertEquals(1, masterQueryResult2.getNumFound());
+    NamedList leaderQueryRsp2 = rQuery(1, "id:2000", leaderClient);
+    SolrDocumentList leaderQueryResult2 = (SolrDocumentList) leaderQueryRsp2.get("response");
+    assertEquals(1, leaderQueryResult2.getNumFound());
 
     @SuppressWarnings({"rawtypes"})
-    NamedList slaveQueryRsp2 = rQuery(1, "id:2000", slaveClient);
-    SolrDocumentList slaveQueryResult2 = (SolrDocumentList) slaveQueryRsp2.get("response");
-    assertEquals(1, slaveQueryResult2.getNumFound());
+    NamedList followerQueryRsp2 = rQuery(1, "id:2000", followerClient);
+    SolrDocumentList followerQueryResult2 = (SolrDocumentList) followerQueryRsp2.get("response");
+    assertEquals(1, followerQueryResult2.getNumFound());
     
-    checkForSingleIndex(masterJetty);
-    checkForSingleIndex(slaveJetty, true);
+    checkForSingleIndex(leaderJetty);
+    checkForSingleIndex(followerJetty, true);
   }
 
   @Test
   public void testRateLimitedReplication() throws Exception {
 
     //clean index
-    masterClient.deleteByQuery("*:*");
-    slaveClient.deleteByQuery("*:*");
-    masterClient.commit();
-    slaveClient.commit();
+    leaderClient.deleteByQuery("*:*");
+    followerClient.deleteByQuery("*:*");
+    leaderClient.commit();
+    followerClient.commit();
 
-    masterJetty.stop();
-    slaveJetty.stop();
+    leaderJetty.stop();
+    followerJetty.stop();
 
-    //Start master with the new solrconfig
-    master.copyConfigFile(CONF_DIR + "solrconfig-master-throttled.xml", "solrconfig.xml");
+    //Start leader with the new solrconfig
+    leader.copyConfigFile(CONF_DIR + "solrconfig-leader-throttled.xml", "solrconfig.xml");
     useFactory(null);
-    masterJetty = createAndStartJetty(master);
-    masterClient.close();
-    masterClient = createNewSolrClient(masterJetty.getLocalPort());
+    leaderJetty = createAndStartJetty(leader);
+    leaderClient.close();
+    leaderClient = createNewSolrClient(leaderJetty.getLocalPort());
 
     //index docs
     final int totalDocs = TestUtil.nextInt(random(), 17, 53);
     for (int i = 0; i < totalDocs; i++)
-      index(masterClient, "id", i, "name", TestUtil.randomSimpleString(random(), 1000 , 5000));
+      index(leaderClient, "id", i, "name", TestUtil.randomSimpleString(random(), 1000 , 5000));
 
-    masterClient.commit();
+    leaderClient.commit();
 
     //Check Index Size
-    String dataDir = master.getDataDir();
-    masterClient.close();
-    masterJetty.stop();
+    String dataDir = leader.getDataDir();
+    leaderClient.close();
+    leaderJetty.stop();
 
     Directory dir = FSDirectory.open(Paths.get(dataDir).resolve("index"));
     String[] files = dir.listAll();
@@ -1511,29 +1556,29 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
 
     //Start again and replicate the data
     useFactory(null);
-    masterJetty = createAndStartJetty(master);
-    masterClient = createNewSolrClient(masterJetty.getLocalPort());
+    leaderJetty = createAndStartJetty(leader);
+    leaderClient = createNewSolrClient(leaderJetty.getLocalPort());
 
-    //start slave
-    slave.setTestPort(masterJetty.getLocalPort());
-    slave.copyConfigFile(CONF_DIR + "solrconfig-slave1.xml", "solrconfig.xml");
-    slaveJetty = createAndStartJetty(slave);
-    slaveClient.close();
-    slaveClient = createNewSolrClient(slaveJetty.getLocalPort());
+    //start follower
+    follower.setTestPort(leaderJetty.getLocalPort());
+    follower.copyConfigFile(CONF_DIR + "solrconfig-follower1.xml", "solrconfig.xml");
+    followerJetty = createAndStartJetty(follower);
+    followerClient.close();
+    followerClient = createNewSolrClient(followerJetty.getLocalPort());
 
     long startTime = System.nanoTime();
 
-    pullFromMasterToSlave();
+    pullFromLeaderToFollower();
 
-    //Add a few more docs in the master. Just to make sure that we are replicating the correct index point
+    //Add a few more docs in the leader. Just to make sure that we are replicating the correct index point
     //These extra docs should not get replicated
-    new Thread(new AddExtraDocs(masterClient, totalDocs)).start();
+    new Thread(new AddExtraDocs(leaderClient, totalDocs)).start();
 
     //Wait and make sure that it actually replicated correctly.
     @SuppressWarnings({"rawtypes"})
-    NamedList slaveQueryRsp = rQuery(totalDocs, "*:*", slaveClient);
-    SolrDocumentList slaveQueryResult = (SolrDocumentList) slaveQueryRsp.get("response");
-    assertEquals(totalDocs, slaveQueryResult.getNumFound());
+    NamedList followerQueryRsp = rQuery(totalDocs, "*:*", followerClient);
+    SolrDocumentList followerQueryResult = (SolrDocumentList) followerQueryRsp.get("response");
+    assertEquals(totalDocs, followerQueryResult.getNumFound());
 
     long timeTaken = System.nanoTime() - startTime;
 
@@ -1554,7 +1599,7 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
     for (String param : params) {
       for (String filename : illegalFilenames) {
         expectThrows(Exception.class, () ->
-            invokeReplicationCommand(masterJetty.getLocalPort(), "filecontent&" + param + "=" + filename));
+            invokeReplicationCommand(leaderJetty.getLocalPort(), "filecontent&" + param + "=" + filename));
       }
     }
   }
@@ -1566,7 +1611,7 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
         .add("wt", "json")
         .add("command", "filelist")
         .add("generation", "-2"); // A 'generation' value not matching any commit point should cause error.
-    QueryResponse response = slaveClient.query(q);
+    QueryResponse response = followerClient.query(q);
     NamedList<Object> resp = response.getResponse();
     assertNotNull(resp);
     assertEquals("ERROR", resp.get("status"));
@@ -1575,15 +1620,15 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
 
   @Test
   public void testFetchIndexShouldReportErrorsWhenTheyOccur() throws Exception  {
-    int masterPort = masterJetty.getLocalPort();
-    masterJetty.stop();
+    int leaderPort = leaderJetty.getLocalPort();
+    leaderJetty.stop();
     SolrQuery q = new SolrQuery();
     q.add("qt", "/replication")
         .add("wt", "json")
         .add("wait", "true")
         .add("command", "fetchindex")
-        .add("masterUrl", buildUrl(masterPort));
-    QueryResponse response = slaveClient.query(q);
+        .add("leaderUrl", buildUrl(leaderPort));
+    QueryResponse response = followerClient.query(q);
     NamedList<Object> resp = response.getResponse();
     assertNotNull(resp);
     assertEquals("Fetch index with wait=true should have returned an error response", "ERROR", resp.get("status"));
@@ -1595,7 +1640,7 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
     q.add("qt", "/replication")
         .add("wt", "json");
     SolrException thrown = expectThrows(SolrException.class, () -> {
-      slaveClient.query(q);
+      followerClient.query(q);
     });
     assertEquals(SolrException.ErrorCode.BAD_REQUEST.code, thrown.code());
     assertThat(thrown.getMessage(), containsString("Missing required parameter: command"));
@@ -1608,7 +1653,7 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
         .add("wt", "json")
         .add("command", "deletebackup");
     SolrException thrown = expectThrows(SolrException.class, () -> {
-      slaveClient.query(q);
+      followerClient.query(q);
     });
     assertEquals(SolrException.ErrorCode.BAD_REQUEST.code, thrown.code());
     assertThat(thrown.getMessage(), containsString("Missing required parameter: name"));
@@ -1617,9 +1662,9 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
   @Test
   public void testEmptyBackups() throws Exception {
     final File backupDir = createTempDir().toFile();
-    final BackupStatusChecker backupStatus = new BackupStatusChecker(masterClient);
+    final BackupStatusChecker backupStatus = new BackupStatusChecker(leaderClient);
 
-    masterJetty.getCoreContainer().getAllowPaths().add(backupDir.toPath());
+    leaderJetty.getCoreContainer().getAllowPaths().add(backupDir.toPath());
 
     { // initial request w/o any committed docs
       final String backupName = "empty_backup1";
@@ -1629,7 +1674,7 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
                 "location", backupDir.getAbsolutePath(),
                 "name", backupName));
       final TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
-      final SimpleSolrResponse rsp = req.process(masterClient);
+      final SimpleSolrResponse rsp = req.process(leaderClient);
 
       final String dirName = backupStatus.waitForBackupSuccess(backupName, timeout);
       assertEquals("Did not get expected dir name for backup, did API change?",
@@ -1638,7 +1683,7 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
                  new File(backupDir, dirName).exists());
     }
     
-    index(masterClient, "id", "1", "name", "foo");
+    index(leaderClient, "id", "1", "name", "foo");
     
     { // second backup w/uncommited doc
       final String backupName = "empty_backup2";
@@ -1648,7 +1693,7 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
                 "location", backupDir.getAbsolutePath(),
                 "name", backupName));
       final TimeOut timeout = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
-      final SimpleSolrResponse rsp = req.process(masterClient);
+      final SimpleSolrResponse rsp = req.process(leaderClient);
       
       final String dirName = backupStatus.waitForBackupSuccess(backupName, timeout);
       assertEquals("Did not get expected dir name for backup, did API change?",
@@ -1667,13 +1712,38 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
     }
   }
   
+  public void testGetBoolWithBackwardCompatibility() {
+    assertTrue(ReplicationHandler.getBoolWithBackwardCompatibility(params(), "foo", "bar", true));
+    assertFalse(ReplicationHandler.getBoolWithBackwardCompatibility(params(), "foo", "bar", false));
+    assertTrue(ReplicationHandler.getBoolWithBackwardCompatibility(params("foo", "true"), "foo", "bar", false));
+    assertTrue(ReplicationHandler.getBoolWithBackwardCompatibility(params("bar", "true"), "foo", "bar", false));
+    assertTrue(ReplicationHandler.getBoolWithBackwardCompatibility(params("foo", "true", "bar", "false"), "foo", "bar", false));
+  }
+  
+  public void testGetObjectWithBackwardCompatibility() {
+    assertEquals("aaa", ReplicationHandler.getObjectWithBackwardCompatibility(params(), "foo", "bar", "aaa"));
+    assertEquals("bbb", ReplicationHandler.getObjectWithBackwardCompatibility(params("foo", "bbb"), "foo", "bar", "aaa"));
+    assertEquals("bbb", ReplicationHandler.getObjectWithBackwardCompatibility(params("bar", "bbb"), "foo", "bar", "aaa"));
+    assertEquals("bbb", ReplicationHandler.getObjectWithBackwardCompatibility(params("foo", "bbb", "bar", "aaa"), "foo", "bar", "aaa"));
+    assertNull(ReplicationHandler.getObjectWithBackwardCompatibility(params(), "foo", "bar", null));
+  }
+  
+  public void testGetObjectWithBackwardCompatibilityFromNL() {
+    NamedList<Object> nl = new NamedList<>();
+    assertNull(ReplicationHandler.getObjectWithBackwardCompatibility(nl, "foo", "bar"));
+    nl.add("bar", "bbb");
+    assertEquals("bbb", ReplicationHandler.getObjectWithBackwardCompatibility(nl, "foo", "bar"));
+    nl.add("foo", "aaa");
+    assertEquals("aaa", ReplicationHandler.getObjectWithBackwardCompatibility(nl, "foo", "bar"));
+  }
+  
   
   private class AddExtraDocs implements Runnable {
 
-    SolrClient masterClient;
+    SolrClient leaderClient;
     int startId;
-    public AddExtraDocs(SolrClient masterClient, int startId) {
-      this.masterClient = masterClient;
+    public AddExtraDocs(SolrClient leaderClient, int startId) {
+      this.leaderClient = leaderClient;
       this.startId = startId;
     }
 
@@ -1682,13 +1752,13 @@ public class TestReplicationHandler extends SolrTestCaseJ4 {
       final int totalDocs = TestUtil.nextInt(random(), 1, 10);
       for (int i = 0; i < totalDocs; i++) {
         try {
-          index(masterClient, "id", i + startId, "name", TestUtil.randomSimpleString(random(), 1000 , 5000));
+          index(leaderClient, "id", i + startId, "name", TestUtil.randomSimpleString(random(), 1000 , 5000));
         } catch (Exception e) {
           //Do nothing. Wasn't able to add doc.
         }
       }
       try {
-        masterClient.commit();
+        leaderClient.commit();
       } catch (Exception e) {
         //Do nothing. No extra doc got committed.
       }
diff --git a/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerBackup.java b/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerBackup.java
index 68bf77c..08b42ef 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerBackup.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerBackup.java
@@ -55,9 +55,9 @@ import org.slf4j.LoggerFactory;
 @SolrTestCaseJ4.SuppressSSL     // Currently unknown why SSL does not work with this test
 public class TestReplicationHandlerBackup extends SolrJettyTestBase {
 
-  JettySolrRunner masterJetty;
-  TestReplicationHandler.SolrInstance master = null;
-  SolrClient masterClient;
+  JettySolrRunner leaderJetty;
+  TestReplicationHandler.SolrInstance leader = null;
+  SolrClient leaderClient;
   
   private static final String CONF_DIR = "solr" + File.separator + "collection1" + File.separator + "conf"
       + File.separator;
@@ -95,19 +95,19 @@ public class TestReplicationHandlerBackup extends SolrJettyTestBase {
   @Before
   public void setUp() throws Exception {
     super.setUp();
-    String configFile = "solrconfig-master1.xml";
+    String configFile = "solrconfig-leader1.xml";
 
     if(random().nextBoolean()) {
-      configFile = "solrconfig-master1-keepOneBackup.xml";
+      configFile = "solrconfig-leader1-keepOneBackup.xml";
       addNumberToKeepInRequest = false;
       backupKeepParamName = ReplicationHandler.NUMBER_BACKUPS_TO_KEEP_INIT_PARAM;
     }
-    master = new TestReplicationHandler.SolrInstance(createTempDir("solr-instance").toFile(), "master", null);
-    master.setUp();
-    master.copyConfigFile(CONF_DIR + configFile, "solrconfig.xml");
+    leader = new TestReplicationHandler.SolrInstance(createTempDir("solr-instance").toFile(), "leader", null);
+    leader.setUp();
+    leader.copyConfigFile(CONF_DIR + configFile, "solrconfig.xml");
 
-    masterJetty = createAndStartJetty(master);
-    masterClient = createNewSolrClient(masterJetty.getLocalPort());
+    leaderJetty = createAndStartJetty(leader);
+    leaderClient = createNewSolrClient(leaderJetty.getLocalPort());
     docsSeed = random().nextLong();
   }
 
@@ -115,21 +115,21 @@ public class TestReplicationHandlerBackup extends SolrJettyTestBase {
   @After
   public void tearDown() throws Exception {
     super.tearDown();
-    if (null != masterClient) {
-      masterClient.close();
-      masterClient  = null;
+    if (null != leaderClient) {
+      leaderClient.close();
+      leaderClient  = null;
     }
-    if (null != masterJetty) {
-      masterJetty.stop();
-      masterJetty = null;
+    if (null != leaderJetty) {
+      leaderJetty.stop();
+      leaderJetty = null;
     }
-    master = null;
+    leader = null;
   }
 
   @Test
   public void testBackupOnCommit() throws Exception {
     final BackupStatusChecker backupStatus
-      = new BackupStatusChecker(masterClient, "/" + DEFAULT_TEST_CORENAME + "/replication");
+      = new BackupStatusChecker(leaderClient, "/" + DEFAULT_TEST_CORENAME + "/replication");
 
     final String lastBackupDir = backupStatus.checkBackupSuccess();
     // sanity check no backups yet
@@ -137,11 +137,11 @@ public class TestReplicationHandlerBackup extends SolrJettyTestBase {
                lastBackupDir);
     
     //Index
-    int nDocs = BackupRestoreUtils.indexDocs(masterClient, DEFAULT_TEST_COLLECTION_NAME, docsSeed);
+    int nDocs = BackupRestoreUtils.indexDocs(leaderClient, DEFAULT_TEST_COLLECTION_NAME, docsSeed);
     
     final String newBackupDir = backupStatus.waitForDifferentBackupDir(lastBackupDir, 30);
     //Validate
-    verify(Paths.get(master.getDataDir(), newBackupDir), nDocs);
+    verify(Paths.get(leader.getDataDir(), newBackupDir), nDocs);
   }
 
   private void verify(Path backup, int nDocs) throws IOException {
@@ -158,7 +158,7 @@ public class TestReplicationHandlerBackup extends SolrJettyTestBase {
   @Test
   public void doTestBackup() throws Exception {
     final BackupStatusChecker backupStatus
-      = new BackupStatusChecker(masterClient, "/" + DEFAULT_TEST_CORENAME + "/replication");
+      = new BackupStatusChecker(leaderClient, "/" + DEFAULT_TEST_CORENAME + "/replication");
 
     String lastBackupDir = backupStatus.checkBackupSuccess();
     assertNull("Already have a successful backup",
@@ -167,10 +167,10 @@ public class TestReplicationHandlerBackup extends SolrJettyTestBase {
     final Path[] snapDir = new Path[5]; //One extra for the backup on commit
     //First snapshot location
     
-    int nDocs = BackupRestoreUtils.indexDocs(masterClient, DEFAULT_TEST_COLLECTION_NAME, docsSeed);
+    int nDocs = BackupRestoreUtils.indexDocs(leaderClient, DEFAULT_TEST_COLLECTION_NAME, docsSeed);
 
     lastBackupDir = backupStatus.waitForDifferentBackupDir(lastBackupDir, 30);
-    snapDir[0] = Paths.get(master.getDataDir(), lastBackupDir);
+    snapDir[0] = Paths.get(leader.getDataDir(), lastBackupDir);
 
     final boolean namedBackup = random().nextBoolean();
 
@@ -182,17 +182,17 @@ public class TestReplicationHandlerBackup extends SolrJettyTestBase {
       final String backupName = TestUtil.randomSimpleString(random(), 1, 20) + "_" + i;
       if (!namedBackup) {
         if (addNumberToKeepInRequest) {
-          runBackupCommand(masterJetty, ReplicationHandler.CMD_BACKUP, "&" + backupKeepParamName + "=2");
+          runBackupCommand(leaderJetty, ReplicationHandler.CMD_BACKUP, "&" + backupKeepParamName + "=2");
         } else {
-          runBackupCommand(masterJetty, ReplicationHandler.CMD_BACKUP, "");
+          runBackupCommand(leaderJetty, ReplicationHandler.CMD_BACKUP, "");
         }
         lastBackupDir = backupStatus.waitForDifferentBackupDir(lastBackupDir, 30);
       } else {
-        runBackupCommand(masterJetty, ReplicationHandler.CMD_BACKUP, "&name=" +  backupName);
+        runBackupCommand(leaderJetty, ReplicationHandler.CMD_BACKUP, "&name=" +  backupName);
         lastBackupDir = backupStatus.waitForBackupSuccess(backupName, 30);
         backupNames[i] = backupName;
       }
-      snapDir[i+1] = Paths.get(master.getDataDir(), lastBackupDir);
+      snapDir[i+1] = Paths.get(leader.getDataDir(), lastBackupDir);
       verify(snapDir[i+1], nDocs);
     }
 
@@ -205,7 +205,7 @@ public class TestReplicationHandlerBackup extends SolrJettyTestBase {
       // Only the last two should still exist.
       final List<String> remainingBackups = new ArrayList<>();
       
-      try (DirectoryStream<Path> stream = Files.newDirectoryStream(Paths.get(master.getDataDir()), "snapshot*")) {
+      try (DirectoryStream<Path> stream = Files.newDirectoryStream(Paths.get(leader.getDataDir()), "snapshot*")) {
         Iterator<Path> iter = stream.iterator();
         while (iter.hasNext()) {
           remainingBackups.add(iter.next().getFileName().toString());
@@ -235,12 +235,12 @@ public class TestReplicationHandlerBackup extends SolrJettyTestBase {
 
   private void testDeleteNamedBackup(String backupNames[]) throws Exception {
     final BackupStatusChecker backupStatus
-      = new BackupStatusChecker(masterClient, "/" + DEFAULT_TEST_CORENAME + "/replication");
+      = new BackupStatusChecker(leaderClient, "/" + DEFAULT_TEST_CORENAME + "/replication");
     for (int i = 0; i < 2; i++) {
-      final Path p = Paths.get(master.getDataDir(), "snapshot." + backupNames[i]);
+      final Path p = Paths.get(leader.getDataDir(), "snapshot." + backupNames[i]);
       assertTrue("WTF: Backup doesn't exist: " + p.toString(),
                  Files.exists(p));
-      runBackupCommand(masterJetty, ReplicationHandler.CMD_DELETE_BACKUP, "&name=" +backupNames[i]);
+      runBackupCommand(leaderJetty, ReplicationHandler.CMD_DELETE_BACKUP, "&name=" +backupNames[i]);
       backupStatus.waitForBackupDeletionSuccess(backupNames[i], 30);
       assertFalse("backup still exists after deletion: " + p.toString(),
                   Files.exists(p));
@@ -248,12 +248,12 @@ public class TestReplicationHandlerBackup extends SolrJettyTestBase {
     
   }
 
-  public static void runBackupCommand(JettySolrRunner masterJetty, String cmd, String params) throws IOException {
-    String masterUrl = buildUrl(masterJetty.getLocalPort(), context) + "/" + DEFAULT_TEST_CORENAME
+  public static void runBackupCommand(JettySolrRunner leaderJetty, String cmd, String params) throws IOException {
+    String leaderUrl = buildUrl(leaderJetty.getLocalPort(), context) + "/" + DEFAULT_TEST_CORENAME
         + ReplicationHandler.PATH+"?wt=xml&command=" + cmd + params;
     InputStream stream = null;
     try {
-      URL url = new URL(masterUrl);
+      URL url = new URL(leaderUrl);
       stream = url.openStream();
       stream.close();
     } finally {
diff --git a/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerDiskOverFlow.java b/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerDiskOverFlow.java
index 6018181b..9565945 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerDiskOverFlow.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerDiskOverFlow.java
@@ -59,9 +59,9 @@ public class TestReplicationHandlerDiskOverFlow extends SolrTestCaseJ4 {
   Function<String, Long> originalDiskSpaceprovider = null;
   BooleanSupplier originalTestWait = null;
   
-  JettySolrRunner masterJetty, slaveJetty;
-  SolrClient masterClient, slaveClient;
-  TestReplicationHandler.SolrInstance master = null, slave = null;
+  JettySolrRunner leaderJetty, followerJetty;
+  SolrClient leaderClient, followerClient;
+  TestReplicationHandler.SolrInstance leader = null, follower = null;
 
   static String context = "/solr";
 
@@ -74,15 +74,15 @@ public class TestReplicationHandlerDiskOverFlow extends SolrTestCaseJ4 {
     System.setProperty("solr.directoryFactory", "solr.StandardDirectoryFactory");
     String factory = random().nextInt(100) < 75 ? "solr.NRTCachingDirectoryFactory" : "solr.StandardDirectoryFactory"; // test the default most of the time
     System.setProperty("solr.directoryFactory", factory);
-    master = new TestReplicationHandler.SolrInstance(createTempDir("solr-instance").toFile(), "master", null);
-    master.setUp();
-    masterJetty = createAndStartJetty(master);
-    masterClient = createNewSolrClient(masterJetty.getLocalPort());
+    leader = new TestReplicationHandler.SolrInstance(createTempDir("solr-instance").toFile(), "leader", null);
+    leader.setUp();
+    leaderJetty = createAndStartJetty(leader);
+    leaderClient = createNewSolrClient(leaderJetty.getLocalPort());
 
-    slave = new TestReplicationHandler.SolrInstance(createTempDir("solr-instance").toFile(), "slave", masterJetty.getLocalPort());
-    slave.setUp();
-    slaveJetty = createAndStartJetty(slave);
-    slaveClient = createNewSolrClient(slaveJetty.getLocalPort());
+    follower = new TestReplicationHandler.SolrInstance(createTempDir("solr-instance").toFile(), "follower", leaderJetty.getLocalPort());
+    follower.setUp();
+    followerJetty = createAndStartJetty(follower);
+    followerClient = createNewSolrClient(followerJetty.getLocalPort());
 
     System.setProperty("solr.indexfetcher.sotimeout2", "45000");
   }
@@ -91,22 +91,22 @@ public class TestReplicationHandlerDiskOverFlow extends SolrTestCaseJ4 {
   @After
   public void tearDown() throws Exception {
     super.tearDown();
-    if (null != masterJetty) {
-      masterJetty.stop();
-      masterJetty = null;
+    if (null != leaderJetty) {
+      leaderJetty.stop();
+      leaderJetty = null;
     }
-    if (null != slaveJetty) {
-      slaveJetty.stop();
-       slaveJetty = null;
+    if (null != followerJetty) {
+      followerJetty.stop();
+       followerJetty = null;
     }
-    master = slave = null;
-    if (null != masterClient) {
-      masterClient.close();
-      masterClient = null;
+    leader = follower = null;
+    if (null != leaderClient) {
+      leaderClient.close();
+      leaderClient = null;
     }
-    if (null != slaveClient) {
-      slaveClient.close();
-      slaveClient = null;
+    if (null != followerClient) {
+      followerClient.close();
+      followerClient = null;
     }
     System.clearProperty("solr.indexfetcher.sotimeout");
     
@@ -116,18 +116,18 @@ public class TestReplicationHandlerDiskOverFlow extends SolrTestCaseJ4 {
 
   @Test
   public void testDiskOverFlow() throws Exception {
-    invokeReplicationCommand(slaveJetty.getLocalPort(), "disablepoll");
+    invokeReplicationCommand(followerJetty.getLocalPort(), "disablepoll");
     //index docs
-    log.info("Indexing to MASTER");
-    int docsInMaster = 1000;
-    long szMaster = indexDocs(masterClient, docsInMaster, 0);
-    log.info("Indexing to SLAVE");
-    long szSlave = indexDocs(slaveClient, 1200, 1000);
+    log.info("Indexing to LEADER");
+    int docsInLeader = 1000;
+    long szLeader = indexDocs(leaderClient, docsInLeader, 0);
+    log.info("Indexing to FOLLOWER");
+    long szFollower = indexDocs(followerClient, 1200, 1000);
 
     IndexFetcher.usableDiskSpaceProvider = new Function<String, Long>() {
       @Override
       public Long apply(String s) {
-        return szMaster;
+        return szLeader;
       }
     };
 
@@ -161,7 +161,7 @@ public class TestReplicationHandlerDiskOverFlow extends SolrTestCaseJ4 {
             assertNotNull("why is query thread still looping if barrier has already been cleared?",
                           barrier);
             try {
-              QueryResponse rsp = slaveClient.query(new SolrQuery()
+              QueryResponse rsp = followerClient.query(new SolrQuery()
                                                     .setQuery("*:*")
                                                     .setRows(0));
               Thread.sleep(200);
@@ -187,7 +187,7 @@ public class TestReplicationHandlerDiskOverFlow extends SolrTestCaseJ4 {
         }
       }).start();
 
-    QueryResponse response = slaveClient.query(new SolrQuery()
+    QueryResponse response = followerClient.query(new SolrQuery()
                                                .add("qt", "/replication")
                                                .add("command", CMD_FETCH_INDEX)
                                                .add("wait", "true")
@@ -198,17 +198,17 @@ public class TestReplicationHandlerDiskOverFlow extends SolrTestCaseJ4 {
     assertEquals("threads encountered failures (see logs for when)",
                  Collections.emptyList(), threadFailures);
 
-    response = slaveClient.query(new SolrQuery().setQuery("*:*").setRows(0));
-    assertEquals("docs in slave", docsInMaster, response.getResults().getNumFound());
+    response = followerClient.query(new SolrQuery().setQuery("*:*").setRows(0));
+    assertEquals("docs in follower", docsInLeader, response.getResults().getNumFound());
 
-    response = slaveClient.query(new SolrQuery()
+    response = followerClient.query(new SolrQuery()
         .add("qt", "/replication")
         .add("command", ReplicationHandler.CMD_DETAILS)
     );
     if (log.isInfoEnabled()) {
       log.info("DETAILS {}", Utils.writeJson(response, new StringWriter(), true).toString());
     }
-    assertEquals("slave's clearedLocalIndexFirst (from rep details)",
+    assertEquals("follower's clearedLocalIndexFirst (from rep details)",
                  "true", response._getStr("details/slave/clearedLocalIndexFirst", null));
   }
 
diff --git a/solr/core/src/test/org/apache/solr/handler/TestRestoreCore.java b/solr/core/src/test/org/apache/solr/handler/TestRestoreCore.java
index 77e43cb..1e16707 100644
--- a/solr/core/src/test/org/apache/solr/handler/TestRestoreCore.java
+++ b/solr/core/src/test/org/apache/solr/handler/TestRestoreCore.java
@@ -47,9 +47,9 @@ import org.junit.Test;
 @SolrTestCaseJ4.SuppressSSL     // Currently unknown why SSL does not work with this test
 public class TestRestoreCore extends SolrJettyTestBase {
 
-  JettySolrRunner masterJetty;
-  TestReplicationHandler.SolrInstance master = null;
-  SolrClient masterClient;
+  JettySolrRunner leaderJetty;
+  TestReplicationHandler.SolrInstance leader = null;
+  SolrClient leaderClient;
 
   private static final String CONF_DIR = "solr" + File.separator + DEFAULT_TEST_CORENAME + File.separator + "conf"
       + File.separator;
@@ -83,14 +83,14 @@ public class TestRestoreCore extends SolrJettyTestBase {
   @Before
   public void setUp() throws Exception {
     super.setUp();
-    String configFile = "solrconfig-master.xml";
+    String configFile = "solrconfig-leader.xml";
 
-    master = new TestReplicationHandler.SolrInstance(createTempDir("solr-instance").toFile(), "master", null);
-    master.setUp();
-    master.copyConfigFile(CONF_DIR + configFile, "solrconfig.xml");
+    leader = new TestReplicationHandler.SolrInstance(createTempDir("solr-instance").toFile(), "leader", null);
+    leader.setUp();
+    leader.copyConfigFile(CONF_DIR + configFile, "solrconfig.xml");
 
-    masterJetty = createAndStartJetty(master);
-    masterClient = createNewSolrClient(masterJetty.getLocalPort());
+    leaderJetty = createAndStartJetty(leader);
+    leaderClient = createNewSolrClient(leaderJetty.getLocalPort());
     docsSeed = random().nextLong();
   }
 
@@ -98,34 +98,34 @@ public class TestRestoreCore extends SolrJettyTestBase {
   @After
   public void tearDown() throws Exception {
     super.tearDown();
-    if (null != masterClient) {
-      masterClient.close();
-      masterClient  = null;
+    if (null != leaderClient) {
+      leaderClient.close();
+      leaderClient  = null;
     }
-    if (null != masterJetty) {
-      masterJetty.stop();
-      masterJetty = null;
+    if (null != leaderJetty) {
+      leaderJetty.stop();
+      leaderJetty = null;
     }
-    master = null;
+    leader = null;
   }
 
   @Test
   public void testSimpleRestore() throws Exception {
 
-    int nDocs = usually() ? BackupRestoreUtils.indexDocs(masterClient, "collection1", docsSeed) : 0;
+    int nDocs = usually() ? BackupRestoreUtils.indexDocs(leaderClient, "collection1", docsSeed) : 0;
 
     final BackupStatusChecker backupStatus
-      = new BackupStatusChecker(masterClient, "/" + DEFAULT_TEST_CORENAME + "/replication");
+      = new BackupStatusChecker(leaderClient, "/" + DEFAULT_TEST_CORENAME + "/replication");
     final String oldBackupDir = backupStatus.checkBackupSuccess();
     String snapshotName = null;
     String location;
     String params = "";
-    String baseUrl = masterJetty.getBaseUrl().toString();
+    String baseUrl = leaderJetty.getBaseUrl().toString();
 
     //Use the default backup location or an externally provided location.
     if (random().nextBoolean()) {
       location = createTempDir().toFile().getAbsolutePath();
-      masterJetty.getCoreContainer().getAllowPaths().add(Paths.get(location)); // Allow core to be created outside SOLR_HOME
+      leaderJetty.getCoreContainer().getAllowPaths().add(Paths.get(location)); // Allow core to be created outside SOLR_HOME
       params += "&location=" + URLEncoder.encode(location, "UTF-8");
     }
 
@@ -135,7 +135,7 @@ public class TestRestoreCore extends SolrJettyTestBase {
       params += "&name=" + snapshotName;
     }
 
-    TestReplicationHandlerBackup.runBackupCommand(masterJetty, ReplicationHandler.CMD_BACKUP, params);
+    TestReplicationHandlerBackup.runBackupCommand(leaderJetty, ReplicationHandler.CMD_BACKUP, params);
 
     if (null == snapshotName) {
       backupStatus.waitForDifferentBackupDir(oldBackupDir, 30);
@@ -152,9 +152,9 @@ public class TestRestoreCore extends SolrJettyTestBase {
         //Delete a few docs
         int numDeletes = TestUtil.nextInt(random(), 1, nDocs);
         for(int i=0; i<numDeletes; i++) {
-          masterClient.deleteByQuery(DEFAULT_TEST_CORENAME, "id:" + i);
+          leaderClient.deleteByQuery(DEFAULT_TEST_CORENAME, "id:" + i);
         }
-        masterClient.commit(DEFAULT_TEST_CORENAME);
+        leaderClient.commit(DEFAULT_TEST_CORENAME);
 
         //Add a few more
         int moreAdds = TestUtil.nextInt(random(), 1, 100);
@@ -162,22 +162,22 @@ public class TestRestoreCore extends SolrJettyTestBase {
           SolrInputDocument doc = new SolrInputDocument();
           doc.addField("id", i + nDocs);
           doc.addField("name", "name = " + (i + nDocs));
-          masterClient.add(DEFAULT_TEST_CORENAME, doc);
+          leaderClient.add(DEFAULT_TEST_CORENAME, doc);
         }
         //Purposely not calling commit once in a while. There can be some docs which are not committed
         if (usually()) {
-          masterClient.commit(DEFAULT_TEST_CORENAME);
+          leaderClient.commit(DEFAULT_TEST_CORENAME);
         }
       }
 
-      TestReplicationHandlerBackup.runBackupCommand(masterJetty, ReplicationHandler.CMD_RESTORE, params);
+      TestReplicationHandlerBackup.runBackupCommand(leaderJetty, ReplicationHandler.CMD_RESTORE, params);
 
       while (!fetchRestoreStatus(baseUrl, DEFAULT_TEST_CORENAME)) {
         Thread.sleep(1000);
       }
 
       //See if restore was successful by checking if all the docs are present again
-      BackupRestoreUtils.verifyDocs(nDocs, masterClient, DEFAULT_TEST_CORENAME);
+      BackupRestoreUtils.verifyDocs(nDocs, leaderClient, DEFAULT_TEST_CORENAME);
     }
 
   }
@@ -185,7 +185,7 @@ public class TestRestoreCore extends SolrJettyTestBase {
   public void testBackupFailsMissingAllowPaths() throws Exception {
     final String params = "&location=" + URLEncoder.encode(createTempDir().toFile().getAbsolutePath(), "UTF-8");
     Throwable t = expectThrows(IOException.class, () -> {
-      TestReplicationHandlerBackup.runBackupCommand(masterJetty, ReplicationHandler.CMD_BACKUP, params);
+      TestReplicationHandlerBackup.runBackupCommand(leaderJetty, ReplicationHandler.CMD_BACKUP, params);
     });
     // The backup command will fail since the tmp dir is outside allowPaths
     assertTrue(t.getMessage().contains("Server returned HTTP response code: 400"));
@@ -193,18 +193,18 @@ public class TestRestoreCore extends SolrJettyTestBase {
 
   @Test
   public void testFailedRestore() throws Exception {
-    int nDocs = BackupRestoreUtils.indexDocs(masterClient, "collection1", docsSeed);
+    int nDocs = BackupRestoreUtils.indexDocs(leaderClient, "collection1", docsSeed);
 
     String location = createTempDir().toFile().getAbsolutePath();
-    masterJetty.getCoreContainer().getAllowPaths().add(Paths.get(location));
+    leaderJetty.getCoreContainer().getAllowPaths().add(Paths.get(location));
     String snapshotName = TestUtil.randomSimpleString(random(), 1, 5);
     String params = "&name=" + snapshotName + "&location=" + URLEncoder.encode(location, "UTF-8");
-    String baseUrl = masterJetty.getBaseUrl().toString();
+    String baseUrl = leaderJetty.getBaseUrl().toString();
 
-    TestReplicationHandlerBackup.runBackupCommand(masterJetty, ReplicationHandler.CMD_BACKUP, params);
+    TestReplicationHandlerBackup.runBackupCommand(leaderJetty, ReplicationHandler.CMD_BACKUP, params);
 
     final BackupStatusChecker backupStatus
-      = new BackupStatusChecker(masterClient, "/" + DEFAULT_TEST_CORENAME + "/replication");
+      = new BackupStatusChecker(leaderClient, "/" + DEFAULT_TEST_CORENAME + "/replication");
     final String backupDirName = backupStatus.waitForBackupSuccess(snapshotName, 30);
 
     //Remove the segments_n file so that the backup index is corrupted.
@@ -216,7 +216,7 @@ public class TestRestoreCore extends SolrJettyTestBase {
       Files.delete(segmentFileName);
     }
 
-    TestReplicationHandlerBackup.runBackupCommand(masterJetty, ReplicationHandler.CMD_RESTORE, params);
+    TestReplicationHandlerBackup.runBackupCommand(leaderJetty, ReplicationHandler.CMD_RESTORE, params);
 
     expectThrows(AssertionError.class, () -> {
         for (int i = 0; i < 10; i++) {
@@ -227,22 +227,22 @@ public class TestRestoreCore extends SolrJettyTestBase {
         // if we never got an assertion let expectThrows complain
       });
 
-    BackupRestoreUtils.verifyDocs(nDocs, masterClient, DEFAULT_TEST_CORENAME);
+    BackupRestoreUtils.verifyDocs(nDocs, leaderClient, DEFAULT_TEST_CORENAME);
 
     //make sure we can write to the index again
-    nDocs = BackupRestoreUtils.indexDocs(masterClient, "collection1", docsSeed);
-    BackupRestoreUtils.verifyDocs(nDocs, masterClient, DEFAULT_TEST_CORENAME);
+    nDocs = BackupRestoreUtils.indexDocs(leaderClient, "collection1", docsSeed);
+    BackupRestoreUtils.verifyDocs(nDocs, leaderClient, DEFAULT_TEST_CORENAME);
 
   }
 
   public static boolean fetchRestoreStatus (String baseUrl, String coreName) throws IOException {
-    String masterUrl = baseUrl + "/" + coreName +
+    String leaderUrl = baseUrl + "/" + coreName +
         ReplicationHandler.PATH + "?wt=xml&command=" + ReplicationHandler.CMD_RESTORE_STATUS;
     final Pattern pException = Pattern.compile("<str name=\"exception\">(.*?)</str>");
 
     InputStream stream = null;
     try {
-      URL url = new URL(masterUrl);
+      URL url = new URL(leaderUrl);
       stream = url.openStream();
       String response = IOUtils.toString(stream, "UTF-8");
       Matcher matcher = pException.matcher(response);
diff --git a/solr/core/src/test/org/apache/solr/security/BasicAuthStandaloneTest.java b/solr/core/src/test/org/apache/solr/security/BasicAuthStandaloneTest.java
index 53dc18f..406ce58 100644
--- a/solr/core/src/test/org/apache/solr/security/BasicAuthStandaloneTest.java
+++ b/solr/core/src/test/org/apache/solr/security/BasicAuthStandaloneTest.java
@@ -184,7 +184,7 @@ public class BasicAuthStandaloneTest extends SolrTestCaseJ4 {
     Path dataDir;
     
     /**
-     * if masterPort is null, this instance is a master -- otherwise this instance is a slave, and assumes the master is
+     * if leaderPort is null, this instance is a leader -- otherwise this instance is a follower, and assumes the leader is
      * on localhost at the specified port.
      */
     public SolrInstance(String name, Integer port) {
diff --git a/solr/solr-ref-guide/src/cloud-screens.adoc b/solr/solr-ref-guide/src/cloud-screens.adoc
index f603e39..26aa2b5 100644
--- a/solr/solr-ref-guide/src/cloud-screens.adoc
+++ b/solr/solr-ref-guide/src/cloud-screens.adoc
@@ -23,7 +23,7 @@ This screen provides status information about each collection & node in your clu
 .Only Visible When using SolrCloud
 [NOTE]
 ====
-The "Cloud" menu option is only available on Solr instances running in <<getting-started-with-solrcloud.adoc#getting-started-with-solrcloud,SolrCloud mode>>. Single node or master/slave replication instances of Solr will not display this option.
+The "Cloud" menu option is only available on Solr instances running in <<getting-started-with-solrcloud.adoc#getting-started-with-solrcloud,SolrCloud mode>>. Single node or leader/follower replication instances of Solr will not display this option.
 ====
 
 Click on the "Cloud" option in the left-hand navigation, and a small sub-menu appears with options called "Nodes", "Tree", "ZK Status" and "Graph". The sub-view selected by default is "Nodes".
diff --git a/solr/solr-ref-guide/src/collection-specific-tools.adoc b/solr/solr-ref-guide/src/collection-specific-tools.adoc
index e3ae1c5..a46da7c 100644
--- a/solr/solr-ref-guide/src/collection-specific-tools.adoc
+++ b/solr/solr-ref-guide/src/collection-specific-tools.adoc
@@ -24,7 +24,7 @@ In the left-hand navigation bar, you will see a pull-down menu titled "Collectio
 ====
 The "Collection Selector" pull-down menu is only available on Solr instances running in <<solrcloud.adoc#solrcloud,SolrCloud mode>>.
 
-Single node or master/slave replication instances of Solr will not display this menu, instead the Collection specific UI pages described in this section will be available in the <<core-specific-tools.adoc#core-specific-tools,Core Selector pull-down menu>>.
+Single node or leader/follower replication instances of Solr will not display this menu, instead the Collection specific UI pages described in this section will be available in the <<core-specific-tools.adoc#core-specific-tools,Core Selector pull-down menu>>.
 ====
 
 Clicking on the Collection Selector pull-down menu will show a list of the collections in your Solr cluster, with a search box that can be used to find a specific collection by name. When you select a collection from the pull-down, the main display of the page will display some basic metadata about the collection, and a secondary menu will appear in the left nav with links to additional collection specific administration screens.
diff --git a/solr/solr-ref-guide/src/combining-distribution-and-replication.adoc b/solr/solr-ref-guide/src/combining-distribution-and-replication.adoc
index 17cae7d..0c5e3c3 100644
--- a/solr/solr-ref-guide/src/combining-distribution-and-replication.adoc
+++ b/solr/solr-ref-guide/src/combining-distribution-and-replication.adoc
@@ -18,16 +18,16 @@
 
 When your index is too large for a single machine and you have a query volume that single shards cannot keep up with, it's time to replicate each shard in your distributed search setup.
 
-The idea is to combine distributed search with replication. As shown in the figure below, a combined distributed-replication configuration features a master server for each shard and then 1-_n_ slaves that are replicated from the master. As in a standard replicated configuration, the master server handles updates and optimizations without adversely affecting query handling performance.
+The idea is to combine distributed search with replication. As shown in the figure below, a combined distributed-replication configuration features a leader server for each shard and then 1-_n_ followers that are replicated from the leader. As in a standard replicated configuration, the leader server handles updates and optimizations without adversely affecting query handling performance.
 
-Query requests should be load balanced across each of the shard slaves. This gives you both increased query handling capacity and fail-over backup if a server goes down.
+Query requests should be load balanced across each of the shard followers. This gives you both increased query handling capacity and fail-over backup if a server goes down.
 
-.A Solr configuration combining both replication and master-slave distribution.
+.A Solr configuration combining both replication and leader-follower distribution.
 image::images/combining-distribution-and-replication/worddav4101c16174820e932b44baa22abcfcd1.png[image,width=312,height=344]
 
 
-None of the master shards in this configuration know about each other. You index to each master, the index is replicated to each slave, and then searches are distributed across the slaves, using one slave from each master/slave shard.
+None of the leader shards in this configuration know about each other. You index to each leader, the index is replicated to each follower, and then searches are distributed across the followers, using one follower from each leader/follower shard.
 
-For high availability you can use a load balancer to set up a virtual IP for each shard's set of slaves. If you are new to load balancing, HAProxy (http://haproxy.1wt.eu/) is a good open source software load-balancer. If a slave server goes down, a good load-balancer will detect the failure using some technique (generally a heartbeat system), and forward all requests to the remaining live slaves that served with the failed slave. A single virtual IP should then be set up so that requests [...]
+For high availability you can use a load balancer to set up a virtual IP for each shard's set of followers. If you are new to load balancing, HAProxy (http://haproxy.1wt.eu/) is a good open source software load-balancer. If a follower server goes down, a good load-balancer will detect the failure using some technique (generally a heartbeat system), and forward all requests to the remaining live followers that served with the failed follower. A single virtual IP should then be set up so t [...]
 
-With this configuration you will have a fully load balanced, search-side fault-tolerant system (Solr does not yet support fault-tolerant indexing). Incoming searches will be handed off to one of the functioning slaves, then the slave will distribute the search request across a slave for each of the shards in your configuration. The slave will issue a request to each of the virtual IPs for each shard, and the load balancer will choose one of the available slaves. Finally, the results will [...]
+With this configuration you will have a fully load balanced, search-side fault-tolerant system (Solr does not yet support fault-tolerant indexing). Incoming searches will be handed off to one of the functioning followers, then the follower will distribute the search request across a follower for each of the shards in your configuration. The follower will issue a request to each of the virtual IPs for each shard, and the load balancer will choose one of the available followers. Finally, t [...]
diff --git a/solr/solr-ref-guide/src/coreadmin-api.adoc b/solr/solr-ref-guide/src/coreadmin-api.adoc
index 90f0ab8..7d390f6 100644
--- a/solr/solr-ref-guide/src/coreadmin-api.adoc
+++ b/solr/solr-ref-guide/src/coreadmin-api.adoc
@@ -19,7 +19,7 @@
 
 The Core Admin API is primarily used under the covers by the <<collections-api.adoc#collections-api,Collections API>> when running a <<solrcloud.adoc#solrcloud,SolrCloud>> cluster.
 
-SolrCloud users should not typically use the CoreAdmin API directly, but the API may be useful for users of single-node or master/slave Solr installations for core maintenance operations.
+SolrCloud users should not typically use the CoreAdmin API directly, but the API may be useful for users of single-node or leader/follower Solr installations for core maintenance operations.
 
 The CoreAdmin API is implemented by the CoreAdminHandler, which is a special purpose <<requesthandlers-and-searchcomponents-in-solrconfig.adoc#requesthandlers-and-searchcomponents-in-solrconfig,request handler>> that is used to manage Solr cores. Unlike other request handlers, the CoreAdminHandler is not attached to a single core. Instead, there is a single instance of the CoreAdminHandler in each Solr node that manages all the cores running in that node and is accessible at the `/solr/a [...]
 
diff --git a/solr/solr-ref-guide/src/distributed-search-with-index-sharding.adoc b/solr/solr-ref-guide/src/distributed-search-with-index-sharding.adoc
index f2c745f..d51c142 100644
--- a/solr/solr-ref-guide/src/distributed-search-with-index-sharding.adoc
+++ b/solr/solr-ref-guide/src/distributed-search-with-index-sharding.adoc
@@ -61,7 +61,7 @@ The following components support distributed search:
 
 === Shards Whitelist
 
-The nodes allowed in the `shards` parameter is configurable through the `shardsWhitelist` property in `solr.xml`. This whitelist is automatically configured for SolrCloud but needs explicit configuration for master/slave mode. Read more details in the section <<distributed-requests.adoc#configuring-the-shardhandlerfactory,Configuring the ShardHandlerFactory>>.
+The nodes allowed in the `shards` parameter is configurable through the `shardsWhitelist` property in `solr.xml`. This whitelist is automatically configured for SolrCloud but needs explicit configuration for leader/follower mode. Read more details in the section <<distributed-requests.adoc#configuring-the-shardhandlerfactory,Configuring the ShardHandlerFactory>>.
 
 == Limitations to Distributed Search
 
diff --git a/solr/solr-ref-guide/src/implicit-requesthandlers.adoc b/solr/solr-ref-guide/src/implicit-requesthandlers.adoc
index d37fc53..f763d1f 100644
--- a/solr/solr-ref-guide/src/implicit-requesthandlers.adoc
+++ b/solr/solr-ref-guide/src/implicit-requesthandlers.adoc
@@ -184,7 +184,7 @@ Dump:: Echo the request contents back to the client.
 |`solr/debug/dump` |{solr-javadocs}/solr-core/org/apache/solr/handler/DumpRequestHandler.html[DumpRequestHandler] |`_DEBUG_DUMP`
 |===
 
-Replication:: Replicate indexes for SolrCloud recovery and Master/Slave index distribution. This handler must have a core name in the path to the endpoint.
+Replication:: Replicate indexes for SolrCloud recovery and Leader/Follower index distribution. This handler must have a core name in the path to the endpoint.
 +
 [cols="3*.",frame=none,grid=cols,options="header"]
 |===
diff --git a/solr/solr-ref-guide/src/index-replication.adoc b/solr/solr-ref-guide/src/index-replication.adoc
index ce7d66e..e635ca1 100644
--- a/solr/solr-ref-guide/src/index-replication.adoc
+++ b/solr/solr-ref-guide/src/index-replication.adoc
@@ -16,11 +16,11 @@
 // specific language governing permissions and limitations
 // under the License.
 
-Index Replication distributes complete copies of a master index to one or more slave servers. The master server continues to manage updates to the index. All querying is handled by the slaves. This division of labor enables Solr to scale to provide adequate responsiveness to queries against large search volumes.
+Index Replication distributes complete copies of a leader index to one or more follower servers. The leader server continues to manage updates to the index. All querying is handled by the followers. This division of labor enables Solr to scale to provide adequate responsiveness to queries against large search volumes.
 
-The figure below shows a Solr configuration using index replication. The master server's index is replicated on the slaves.
+The figure below shows a Solr configuration using index replication. The leader server's index is replicated on the followers.
 
-.A Solr index can be replicated across multiple slave servers, which then process requests.
+.A Solr index can be replicated across multiple follower servers, which then process requests.
 image::images/index-replication/worddav2b7e14725d898b4104cdd9c502fc77cd.png[image,width=159,height=235]
 
 
@@ -38,9 +38,9 @@ Solr includes a Java implementation of index replication that works over HTTP:
 .Replication In SolrCloud
 [NOTE]
 ====
-Although there is no explicit concept of "master/slave" nodes in a <<solrcloud.adoc#solrcloud,SolrCloud>> cluster, the `ReplicationHandler` discussed on this page is still used by SolrCloud as needed to support "shard recovery" โ€“ but this is done in a peer to peer manner.
+Although there is no explicit concept of "leader/follower" nodes in a <<solrcloud.adoc#solrcloud,SolrCloud>> cluster, the `ReplicationHandler` discussed on this page is still used by SolrCloud as needed to support "shard recovery" โ€“ but this is done in a peer to peer manner.
 
-When using SolrCloud, the `ReplicationHandler` must be available via the `/replication` path. Solr does this implicitly unless overridden explicitly in your `solrconfig.xml`, but if you wish to override the default behavior, make certain that you do not explicitly set any of the "master" or "slave" configuration options mentioned below, or they will interfere with normal SolrCloud operation.
+When using SolrCloud, the `ReplicationHandler` must be available via the `/replication` path. Solr does this implicitly unless overridden explicitly in your `solrconfig.xml`, but if you wish to override the default behavior, make certain that you do not explicitly set any of the "leader" or "follower" configuration options mentioned below, or they will interfere with normal SolrCloud operation.
 ====
 
 == Replication Terminology
@@ -51,19 +51,19 @@ Index::
 A Lucene index is a directory of files. These files make up the searchable and returnable data of a Solr Core.
 
 Distribution::
-The copying of an index from the master server to all slaves. The distribution process takes advantage of Lucene's index file structure.
+The copying of an index from the leader server to all followers. The distribution process takes advantage of Lucene's index file structure.
 
 Inserts and Deletes::
 As inserts and deletes occur in the index, the directory remains unchanged. Documents are always inserted into newly created segment files. Documents that are deleted are not removed from the segment files. They are flagged in the file, deletable, and are not removed from the segments until the segment is merged as part of normal index updates.
 
-Master and Slave::
-A Solr replication master is a single node which receives all updates initially and keeps everything organized. Solr replication slave nodes receive no updates directly, instead all changes (such as inserts, updates, deletes, etc.) are made against the single master node. Changes made on the master are distributed to all the slave nodes which service all query requests from the clients.
+Leader and Follower::
+A Solr replication leader is a single node which receives all updates initially and keeps everything organized. Solr replication follower nodes receive no updates directly, instead all changes (such as inserts, updates, deletes, etc.) are made against the single leader node. Changes made on the leader are distributed to all the follower nodes which service all query requests from the clients.
 
 Update::
 An update is a single change request against a single Solr instance. It may be a request to delete a document, add a new document, change a document, delete all documents matching a query, etc. Updates are handled synchronously within an individual Solr instance.
 
 Optimization::
-A process that compacts the index and merges segments in order to improve query performance. Optimization should only be run on the master nodes. An optimized index may give query performance gains compared to an index that has become fragmented over a period of time with many updates. Distributing an optimized index requires a much longer time than the distribution of new segments to an un-optimized index.
+A process that compacts the index and merges segments in order to improve query performance. Optimization should only be run on the leader nodes. An optimized index may give query performance gains compared to an index that has become fragmented over a period of time with many updates. Distributing an optimized index requires a much longer time than the distribution of new segments to an un-optimized index.
 
 WARNING: optimizing is not recommended unless it can be performed regularly as it may lead to a significantly larger portion of the index consisting of deleted documents than would normally be the case.
 
@@ -74,17 +74,17 @@ mergeFactor::
 A parameter that controls the number of segments in an index. For example, when mergeFactor is set to 3, Solr will fill one segment with documents until the limit maxBufferedDocs is met, then it will start a new segment. When the number of segments specified by mergeFactor is reached (in this example, 3) then Solr will merge all the segments into a single index file, then begin writing new documents to a new segment.
 
 Snapshot::
-A directory containing hard links to the data files of an index. Snapshots are distributed from the master nodes when the slaves pull them, "smart copying" any segments the slave node does not have in snapshot directory that contains the hard links to the most recent index data files.
+A directory containing hard links to the data files of an index. Snapshots are distributed from the leader nodes when the followers pull them, "smart copying" any segments the follower node does not have in snapshot directory that contains the hard links to the most recent index data files.
 
 
 == Configuring the ReplicationHandler
 
-In addition to `ReplicationHandler` configuration options specific to the master/slave roles, there are a few special configuration options that are generally supported (even when using SolrCloud).
+In addition to `ReplicationHandler` configuration options specific to the leader/follower roles, there are a few special configuration options that are generally supported (even when using SolrCloud).
 
 * `maxNumberOfBackups` an integer value dictating the maximum number of backups this node will keep on disk as it receives `backup` commands.
 * Similar to most other request handlers in Solr you may configure a set of <<requesthandlers-and-searchcomponents-in-solrconfig.adoc#searchhandlers,defaults, invariants, and/or appends>> parameters corresponding with any request parameters supported by the `ReplicationHandler` when <<HTTP API Commands for the ReplicationHandler,processing commands>>.
 
-=== Configuring the Replication RequestHandler on a Master Server
+=== Configuring the Replication RequestHandler on a Leader Server
 
 Before running a replication, you should set the following parameters on initialization of the handler:
 
@@ -103,12 +103,12 @@ The configuration files to replicate, separated by a comma.
 `commitReserveDuration`::
 If your commits are very frequent and your network is slow, you can tweak this parameter to increase the amount of time expected to be required to transfer data. The default is `00:00:10` i.e., 10 seconds.
 
-The example below shows a possible 'master' configuration for the `ReplicationHandler`, including a fixed number of backups and an invariant setting for the `maxWriteMBPerSec` request parameter to prevent slaves from saturating its network interface
+The example below shows a possible 'leader' configuration for the `ReplicationHandler`, including a fixed number of backups and an invariant setting for the `maxWriteMBPerSec` request parameter to prevent followers from saturating its network interface
 
 [source,xml]
 ----
 <requestHandler name="/replication" class="solr.ReplicationHandler">
-  <lst name="master">
+  <lst name="leader">
     <str name="replicateAfter">optimize</str>
     <str name="backupAfter">optimize</str>
     <str name="confFiles">schema.xml,stopwords.txt,elevate.xml</str>
@@ -123,32 +123,32 @@ The example below shows a possible 'master' configuration for the `ReplicationHa
 
 ==== Replicating solrconfig.xml
 
-In the configuration file on the master server, include a line like the following:
+In the configuration file on the leader server, include a line like the following:
 
 [source,xml]
 ----
-<str name="confFiles">solrconfig_slave.xml:solrconfig.xml,x.xml,y.xml</str>
+<str name="confFiles">solrconfig_follower.xml:solrconfig.xml,x.xml,y.xml</str>
 ----
 
-This ensures that the local configuration `solrconfig_slave.xml` will be saved as `solrconfig.xml` on the slave. All other files will be saved with their original names.
+This ensures that the local configuration `solrconfig_follower.xml` will be saved as `solrconfig.xml` on the follower. All other files will be saved with their original names.
 
-On the master server, the file name of the slave configuration file can be anything, as long as the name is correctly identified in the `confFiles` string; then it will be saved as whatever file name appears after the colon ':'.
+On the leader server, the file name of the follower configuration file can be anything, as long as the name is correctly identified in the `confFiles` string; then it will be saved as whatever file name appears after the colon ':'.
 
-=== Configuring the Replication RequestHandler on a Slave Server
+=== Configuring the Replication RequestHandler on a Follower Server
 
-The code below shows how to configure a ReplicationHandler on a slave.
+The code below shows how to configure a ReplicationHandler on a follower.
 
 [source,xml]
 ----
 <requestHandler name="/replication" class="solr.ReplicationHandler">
-  <lst name="slave">
+  <lst name="follower">
 
-    <!-- fully qualified url for the replication handler of master. It is
+    <!-- fully qualified url for the replication handler of leader. It is
          possible to pass on this as a request param for the fetchindex command -->
-    <str name="masterUrl">http://remote_host:port/solr/core_name/replication</str>
+    <str name="leaderUrl">http://remote_host:port/solr/core_name/replication</str>
 
-    <!-- Interval in which the slave should poll master.  Format is HH:mm:ss .
-         If this is absent slave does not poll automatically.
+    <!-- Interval in which the follower should poll leader.  Format is HH:mm:ss .
+         If this is absent follower does not poll automatically.
 
          But a fetchindex can be triggered from the admin or the http API -->
 
@@ -158,13 +158,13 @@ The code below shows how to configure a ReplicationHandler on a slave.
 
     <!-- To use compression while transferring the index files. The possible
          values are internal|external.  If the value is 'external' make sure
-         that your master Solr has the settings to honor the accept-encoding header.
+         that your leader Solr has the settings to honor the accept-encoding header.
          If it is 'internal' everything will be taken care of automatically.
          USE THIS ONLY IF YOUR BANDWIDTH IS LOW.
          THIS CAN ACTUALLY SLOW DOWN REPLICATION IN A LAN -->
     <str name="compression">internal</str>
 
-    <!-- The following values are used when the slave connects to the master to
+    <!-- The following values are used when the follower connects to the leader to
          download the index files.  Default values implicitly set as 5000ms and
          10000ms respectively. The user DOES NOT need to specify these unless the
          bandwidth is extremely low or if there is an extremely high latency -->
@@ -172,7 +172,7 @@ The code below shows how to configure a ReplicationHandler on a slave.
     <str name="httpConnTimeout">5000</str>
     <str name="httpReadTimeout">10000</str>
 
-    <!-- If HTTP Basic authentication is enabled on the master, then the slave
+    <!-- If HTTP Basic authentication is enabled on the leader, then the follower
          can be configured with the following -->
 
     <str name="httpBasicAuthUser">username</str>
@@ -183,23 +183,23 @@ The code below shows how to configure a ReplicationHandler on a slave.
 
 == Setting Up a Repeater with the ReplicationHandler
 
-A master may be able to serve only so many slaves without affecting performance. Some organizations have deployed slave servers across multiple data centers. If each slave downloads the index from a remote data center, the resulting download may consume too much network bandwidth. To avoid performance degradation in cases like this, you can configure one or more slaves as repeaters. A repeater is simply a node that acts as both a master and a slave.
+A leader may be able to serve only so many followers without affecting performance. Some organizations have deployed follower servers across multiple data centers. If each follower downloads the index from a remote data center, the resulting download may consume too much network bandwidth. To avoid performance degradation in cases like this, you can configure one or more followers as repeaters. A repeater is simply a node that acts as both a leader and a follower.
 
-* To configure a server as a repeater, the definition of the Replication `requestHandler` in the `solrconfig.xml` file must include file lists of use for both masters and slaves.
-* Be sure to set the `replicateAfter` parameter to commit, even if `replicateAfter` is set to optimize on the main master. This is because on a repeater (or any slave), a commit is called only after the index is downloaded. The optimize command is never called on slaves.
-* Optionally, one can configure the repeater to fetch compressed files from the master through the compression parameter to reduce the index download time.
+* To configure a server as a repeater, the definition of the Replication `requestHandler` in the `solrconfig.xml` file must include file lists of use for both leaders and followers.
+* Be sure to set the `replicateAfter` parameter to commit, even if `replicateAfter` is set to optimize on the main leader. This is because on a repeater (or any follower), a commit is called only after the index is downloaded. The optimize command is never called on followers.
+* Optionally, one can configure the repeater to fetch compressed files from the leader through the compression parameter to reduce the index download time.
 
 Here is an example of a ReplicationHandler configuration for a repeater:
 
 [source,xml]
 ----
 <requestHandler name="/replication" class="solr.ReplicationHandler">
-  <lst name="master">
+  <lst name="leader">
     <str name="replicateAfter">commit</str>
     <str name="confFiles">schema.xml,stopwords.txt,synonyms.txt</str>
   </lst>
-  <lst name="slave">
-    <str name="masterUrl">http://master.solr.company.com:8983/solr/core_name/replication</str>
+  <lst name="follower">
+    <str name="leaderUrl">http://leader.solr.company.com:8983/solr/core_name/replication</str>
     <str name="pollInterval">00:00:60</str>
   </lst>
 </requestHandler>
@@ -207,13 +207,13 @@ Here is an example of a ReplicationHandler configuration for a repeater:
 
 == Commit and Optimize Operations
 
-When a commit or optimize operation is performed on the master, the RequestHandler reads the list of file names which are associated with each commit point. This relies on the `replicateAfter` parameter in the configuration to decide which types of events should trigger replication.
+When a commit or optimize operation is performed on the leader, the RequestHandler reads the list of file names which are associated with each commit point. This relies on the `replicateAfter` parameter in the configuration to decide which types of events should trigger replication.
 
 These operations are supported:
 
-* `commit`: Triggers replication whenever a commit is performed on the master index.
-* `optimize`: Triggers replication whenever the master index is optimized.
-* `startup`: Triggers replication whenever the master index starts up.
+* `commit`: Triggers replication whenever a commit is performed on the leader index.
+* `optimize`: Triggers replication whenever the leader index is optimized.
+* `startup`: Triggers replication whenever the leader index starts up.
 
 The `replicateAfter` parameter can accept multiple arguments. For example:
 
@@ -224,91 +224,91 @@ The `replicateAfter` parameter can accept multiple arguments. For example:
 <str name="replicateAfter">optimize</str>
 ----
 
-== Slave Replication
+== Follower Replication
 
-The master is totally unaware of the slaves.
+The leader is totally unaware of the followers.
 
-The slave continuously keeps polling the master (depending on the `pollInterval` parameter) to check the current index version of the master. If the slave finds out that the master has a newer version of the index it initiates a replication process. The steps are as follows:
+The follower continuously keeps polling the leader (depending on the `pollInterval` parameter) to check the current index version of the leader. If the follower finds out that the leader has a newer version of the index it initiates a replication process. The steps are as follows:
 
-* The slave issues a `filelist` command to get the list of the files. This command returns the names of the files as well as some metadata (for example, size, a lastmodified timestamp, an alias if any).
-* The slave checks with its own index if it has any of those files in the local index. It then runs the filecontent command to download the missing files. This uses a custom format (akin to the HTTP chunked encoding) to download the full content or a part of each file. If the connection breaks in between, the download resumes from the point it failed. At any point, the slave tries 5 times before giving up a replication altogether.
-* The files are downloaded into a temp directory, so that if either the slave or the master crashes during the download process, no files will be corrupted. Instead, the current replication will simply abort.
-* After the download completes, all the new files are moved to the live index directory and the file's timestamp is same as its counterpart on the master.
-* A commit command is issued on the slave by the Slave's ReplicationHandler and the new index is loaded.
+* The follower issues a `filelist` command to get the list of the files. This command returns the names of the files as well as some metadata (for example, size, a lastmodified timestamp, an alias if any).
+* The follower checks with its own index if it has any of those files in the local index. It then runs the filecontent command to download the missing files. This uses a custom format (akin to the HTTP chunked encoding) to download the full content or a part of each file. If the connection breaks in between, the download resumes from the point it failed. At any point, the follower tries 5 times before giving up a replication altogether.
+* The files are downloaded into a temp directory, so that if either the follower or the leader crashes during the download process, no files will be corrupted. Instead, the current replication will simply abort.
+* After the download completes, all the new files are moved to the live index directory and the file's timestamp is same as its counterpart on the leader.
+* A commit command is issued on the follower by the Follower's ReplicationHandler and the new index is loaded.
 
 === Replicating Configuration Files
 
-To replicate configuration files, list them using using the `confFiles` parameter. Only files found in the `conf` directory of the master's Solr instance will be replicated.
+To replicate configuration files, list them using using the `confFiles` parameter. Only files found in the `conf` directory of the leader's Solr instance will be replicated.
 
-Solr replicates configuration files only when the index itself is replicated. That means even if a configuration file is changed on the master, that file will be replicated only after there is a new commit/optimize on master's index.
+Solr replicates configuration files only when the index itself is replicated. That means even if a configuration file is changed on the leader, that file will be replicated only after there is a new commit/optimize on leader's index.
 
-Unlike the index files, where the timestamp is good enough to figure out if they are identical, configuration files are compared against their checksum. The `schema.xml` files (on master and slave) are judged to be identical if their checksums are identical.
+Unlike the index files, where the timestamp is good enough to figure out if they are identical, configuration files are compared against their checksum. The `schema.xml` files (on leader and follower) are judged to be identical if their checksums are identical.
 
 As a precaution when replicating configuration files, Solr copies configuration files to a temporary directory before moving them into their ultimate location in the conf directory. The old configuration files are then renamed and kept in the same `conf/` directory. The ReplicationHandler does not automatically clean up these old files.
 
 If a replication involved downloading of at least one configuration file, the ReplicationHandler issues a core-reload command instead of a commit command.
 
-=== Resolving Corruption Issues on Slave Servers
+=== Resolving Corruption Issues on Follower Servers
 
-If documents are added to the slave, then the slave is no longer in sync with its master. However, the slave will not undertake any action to put itself in sync, until the master has new index data.
+If documents are added to the follower, then the follower is no longer in sync with its leader. However, the follower will not undertake any action to put itself in sync, until the leader has new index data.
 
-When a commit operation takes place on the master, the index version of the master becomes different from that of the slave. The slave then fetches the list of files and finds that some of the files present on the master are also present in the local index but with different sizes and timestamps. This means that the master and slave have incompatible indexes.
+When a commit operation takes place on the leader, the index version of the leader becomes different from that of the follower. The follower then fetches the list of files and finds that some of the files present on the leader are also present in the local index but with different sizes and timestamps. This means that the leader and follower have incompatible indexes.
 
-To correct this problem, the slave then copies all the index files from master to a new index directory and asks the core to load the fresh index from the new directory.
+To correct this problem, the follower then copies all the index files from leader to a new index directory and asks the core to load the fresh index from the new directory.
 
 == HTTP API Commands for the ReplicationHandler
 
 You can use the HTTP commands below to control the ReplicationHandler's operations.
 
 `enablereplication`::
-Enable replication on the "master" for all its slaves.
+Enable replication on the "leader" for all its followers.
 +
 [source,bash]
-http://_master_host:port_/solr/_core_name_/replication?command=enablereplication
+http://_leader_host:port_/solr/_core_name_/replication?command=enablereplication
 
 `disablereplication`::
-Disable replication on the master for all its slaves.
+Disable replication on the leader for all its followers.
 +
 [source,bash]
-http://_master_host:port_/solr/_core_name_/replication?command=disablereplication
+http://_leader_host:port_/solr/_core_name_/replication?command=disablereplication
 
 `indexversion`::
-Return the version of the latest replicatable index on the specified master or slave.
+Return the version of the latest replicatable index on the specified leader or follower.
 +
 [source,bash]
 http://_host:port_/solr/_core_name_/replication?command=indexversion
 
 `fetchindex`::
-Force the specified slave to fetch a copy of the index from its master.
+Force the specified follower to fetch a copy of the index from its leader.
 +
 [source.bash]
-http://_slave_host:port_/solr/_core_name_/replication?command=fetchindex
+http://_follower_host:port_/solr/_core_name_/replication?command=fetchindex
 +
-If you like, you can pass an extra attribute such as `masterUrl` or `compression` (or any other parameter which is specified in the `<lst name="slave">` tag) to do a one time replication from a master. This obviates the need for hard-coding the master in the slave.
+If you like, you can pass an extra attribute such as `leaderUrl` or `compression` (or any other parameter which is specified in the `<lst name="follower">` tag) to do a one time replication from a leader. This obviates the need for hard-coding the leader in the follower.
 
 `abortfetch`::
-Abort copying an index from a master to the specified slave.
+Abort copying an index from a leader to the specified follower.
 +
 [source,bash]
-http://_slave_host:port_/solr/_core_name_/replication?command=abortfetch
+http://_follower_host:port_/solr/_core_name_/replication?command=abortfetch
 
 `enablepoll`::
-Enable the specified slave to poll for changes on the master.
+Enable the specified follower to poll for changes on the leader.
 +
 [source,bash]
-http://_slave_host:port_/solr/_core_name_/replication?command=enablepoll
+http://_follower_host:port_/solr/_core_name_/replication?command=enablepoll
 
 `disablepoll`::
-Disable the specified slave from polling for changes on the master.
+Disable the specified follower from polling for changes on the leader.
 +
 [source,bash]
-http://_slave_host:port_/solr/_core_name_/replication?command=disablepoll
+http://_follower_host:port_/solr/_core_name_/replication?command=disablepoll
 
 `details`::
 Retrieve configuration details and current status.
 +
 [source,bash]
-http://_slave_host:port_/solr/_core_name_/replication?command=details
+http://_follower_host:port_/solr/_core_name_/replication?command=details
 
 `filelist`::
 Retrieve a list of Lucene files present in the specified host's index.
@@ -319,10 +319,10 @@ http://_host:port_/solr/_core_name_/replication?command=filelist&generation=<_ge
 You can discover the generation number of the index by running the `indexversion` command.
 
 `backup`::
-Create a backup on master if there are committed index data in the server; otherwise, does nothing.
+Create a backup on leader if there are committed index data in the server; otherwise, does nothing.
 +
 [source,bash]
-http://_master_host:port_/solr/_core_name_/replication?command=backup
+http://_leader_host:port_/solr/_core_name_/replication?command=backup
 +
 This command is useful for making periodic backups. There are several supported request parameters:
 +
@@ -335,7 +335,7 @@ This command is useful for making periodic backups. There are several supported
 Restore a backup from a backup repository.
 +
 [source,bash]
-http://_master_host:port_/solr/_core_name_/replication?command=restore
+http://_leader_host:port_/solr/_core_name_/replication?command=restore
 +
 This command is used to restore a backup. There are several supported request parameters:
 +
@@ -347,7 +347,7 @@ This command is used to restore a backup. There are several supported request pa
 Check the status of a running restore operation.
 +
 [source,bash]
-http://_master_host:port_/solr/_core_name_/replication?command=restorestatus
+http://_leader_host:port_/solr/_core_name_/replication?command=restorestatus
 +
 This command is used to check the status of a restore operation. This command takes no parameters.
 +
@@ -357,7 +357,7 @@ The status value can be "In Progress" , "success" or "failed". If it failed then
 Delete any backup created using the `backup` command.
 +
 [source,bash]
-http://_master_host:port_ /solr/_core_name_/replication?command=deletebackup
+http://_leader_host:port_ /solr/_core_name_/replication?command=deletebackup
 +
 There are two supported parameters:
 
@@ -369,15 +369,15 @@ There are two supported parameters:
 
 Optimizing an index is not something most users should generally worry about - but in particular users should be aware of the impacts of optimizing an index when using the `ReplicationHandler`.
 
-The time required to optimize a master index can vary dramatically. A small index may be optimized in minutes. A very large index may take hours. The variables include the size of the index and the speed of the hardware.
+The time required to optimize a leader index can vary dramatically. A small index may be optimized in minutes. A very large index may take hours. The variables include the size of the index and the speed of the hardware.
 
-Distributing a newly optimized index may take only a few minutes or up to an hour or more, again depending on the size of the index and the performance capabilities of network connections and disks. During optimization the machine is under load and does not process queries very well. Given a schedule of updates being driven a few times an hour to the slaves, we cannot run an optimize with every committed snapshot.
+Distributing a newly optimized index may take only a few minutes or up to an hour or more, again depending on the size of the index and the performance capabilities of network connections and disks. During optimization the machine is under load and does not process queries very well. Given a schedule of updates being driven a few times an hour to the followers, we cannot run an optimize with every committed snapshot.
 
 Copying an optimized index means that the *entire* index will need to be transferred during the next `snappull`. This is a large expense, but not nearly as huge as running the optimize everywhere.
 
-Consider this example: on a three-slave one-master configuration, distributing a newly-optimized index takes approximately 80 seconds _total_. Rolling the change across a tier would require approximately ten minutes per machine (or machine group). If this optimize were rolled across the query tier, and if each slave node being optimized were disabled and not receiving queries, a rollout would take at least twenty minutes and potentially as long as an hour and a half. Additionally, the fi [...]
+Consider this example: on a three-follower one-leader configuration, distributing a newly-optimized index takes approximately 80 seconds _total_. Rolling the change across a tier would require approximately ten minutes per machine (or machine group). If this optimize were rolled across the query tier, and if each follower node being optimized were disabled and not receiving queries, a rollout would take at least twenty minutes and potentially as long as an hour and a half. Additionally,  [...]
 
-Optimizing on the master allows for a straight-forward optimization operation. No query slaves need to be taken out of service. The optimized index can be distributed in the background as queries are being normally serviced. The optimization can occur at any time convenient to the application providing index updates.
+Optimizing on the leader allows for a straight-forward optimization operation. No query followers need to be taken out of service. The optimized index can be distributed in the background as queries are being normally serviced. The optimization can occur at any time convenient to the application providing index updates.
 
 While optimizing may have some benefits in some situations, a rapidly changing index will not retain those benefits for long, and since optimization is an intensive process, it may be better to consider other options, such as lowering the merge factor (discussed in the section on <<indexconfig-in-solrconfig.adoc#merge-factors,Index Configuration>>).
 
diff --git a/solr/solr-ref-guide/src/major-changes-in-solr-8.adoc b/solr/solr-ref-guide/src/major-changes-in-solr-8.adoc
index b5628ce..be3ab69 100644
--- a/solr/solr-ref-guide/src/major-changes-in-solr-8.adoc
+++ b/solr/solr-ref-guide/src/major-changes-in-solr-8.adoc
@@ -392,7 +392,7 @@ When upgrading to Solr 7.4, users should be aware of the following major changes
 
 *Legacy Scaling (non-SolrCloud)*
 
-* In the <<index-replication.adoc#index-replication,master-slave model>> of scaling Solr, a slave no longer commits an empty index when a completely new index is detected on master during replication. To return to the previous behavior pass `false` to `skipCommitOnMasterVersionZero` in the slave section of replication handler configuration, or pass it to the `fetchindex` command.
+* In the <<index-replication.adoc#index-replication,leader-follower model>> of scaling Solr, a follower no longer commits an empty index when a completely new index is detected on leader during replication. To return to the previous behavior pass `false` to `skipCommitOnLeaderVersionZero` in the follower section of replication handler configuration, or pass it to the `fetchindex` command.
 
 If you are upgrading from a version earlier than Solr 7.3, please see previous version notes below.
 
@@ -497,7 +497,7 @@ See the section <<metrics-reporting.adoc#shard-and-cluster-reporters,Shard and C
 
 *ReplicationHandler*
 
-* In the ReplicationHandler, the `master.commitReserveDuration` sub-element is deprecated. Instead please configure a direct `commitReserveDuration` element for use in all modes (master, slave, cloud).
+* In the ReplicationHandler, the `leader.commitReserveDuration` sub-element is deprecated. Instead please configure a direct `commitReserveDuration` element for use in all modes (leader, follower, cloud).
 
 *RunExecutableListener*
 
diff --git a/solr/solr-ref-guide/src/near-real-time-searching.adoc b/solr/solr-ref-guide/src/near-real-time-searching.adoc
index ff632a3..ba453c8 100644
--- a/solr/solr-ref-guide/src/near-real-time-searching.adoc
+++ b/solr/solr-ref-guide/src/near-real-time-searching.adoc
@@ -16,7 +16,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
-Near Real Time (NRT) search means that documents are available for search soon after being indexed. NRT searching is one of the main features of SolrCloud and is rarely attempted in master/slave configurations.
+Near Real Time (NRT) search means that documents are available for search soon after being indexed. NRT searching is one of the main features of SolrCloud and is rarely attempted in leader/follower configurations.
 
 Document durability and searchability are controlled by `commits`. The "Near" in "Near Real Time" is configurable to meet the needs of your application. Commits are either "hard" or "soft" and can be issued by a client (say SolrJ), via a REST call or configured to occur automatically in `solrconfig.xml`. The recommendation usually gives is to configure your commit strategy in `solrconfig.xml` (see below) and avoid issuing commits externally.
 
diff --git a/solr/solr-ref-guide/src/query-settings-in-solrconfig.adoc b/solr/solr-ref-guide/src/query-settings-in-solrconfig.adoc
index c6eb229..fbbb05b 100644
--- a/solr/solr-ref-guide/src/query-settings-in-solrconfig.adoc
+++ b/solr/solr-ref-guide/src/query-settings-in-solrconfig.adoc
@@ -196,7 +196,7 @@ This setting controls whether search requests for which there is not a currently
 
 === maxWarmingSearchers
 
-This parameter sets the maximum number of searchers that may be warming up in the background at any given time. Exceeding this limit will raise an error. For read-only slaves, a value of two is reasonable. Masters should probably be set a little higher.
+This parameter sets the maximum number of searchers that may be warming up in the background at any given time. Exceeding this limit will raise an error. For read-only followers, a value of two is reasonable. Leaders should probably be set a little higher.
 
 [source,xml]
 ----
diff --git a/solr/solr-ref-guide/src/replication-screen.adoc b/solr/solr-ref-guide/src/replication-screen.adoc
index 2637bd9..757f156 100644
--- a/solr/solr-ref-guide/src/replication-screen.adoc
+++ b/solr/solr-ref-guide/src/replication-screen.adoc
@@ -16,11 +16,11 @@
 // specific language governing permissions and limitations
 // under the License.
 
-The Replication screen shows you the current replication state for the core you have specified. <<solrcloud.adoc#solrcloud,SolrCloud>> has supplanted much of this functionality, but if you are still using Master-Slave index replication, you can use this screen to:
+The Replication screen shows you the current replication state for the core you have specified. <<solrcloud.adoc#solrcloud,SolrCloud>> has supplanted much of this functionality, but if you are still using Leader-Follower index replication, you can use this screen to:
 
-. View the replicatable index state. (on a master node)
-. View the current replication status (on a slave node)
-. Disable replication. (on a master node)
+. View the replicatable index state. (on a leader node)
+. View the current replication status (on a follower node)
+. Disable replication. (on a leader node)
 
 .Caution When Using SolrCloud
 [IMPORTANT]
diff --git a/solr/solr-ref-guide/src/request-parameters-api.adoc b/solr/solr-ref-guide/src/request-parameters-api.adoc
index 94be9b0..2f26f8d 100644
--- a/solr/solr-ref-guide/src/request-parameters-api.adoc
+++ b/solr/solr-ref-guide/src/request-parameters-api.adoc
@@ -208,4 +208,4 @@ The RequestParams Object can be accessed using the method `SolrConfig#getRequest
 
 == Examples Using the Request Parameters API
 
-The Solr "films" example demonstrates the use of the parameters API. You can use this example in your Solr installation (in the `example/films` directory) or view the files in the Apache GitHub mirror at https://github.com/apache/lucene-solr/tree/master/solr/example/films.
+The Solr "films" example demonstrates the use of the parameters API. You can use this example in your Solr installation (in the `example/films` directory) or view the files in the Apache GitHub mirror at https://github.com/apache/lucene-solr/tree/leader/solr/example/films.
diff --git a/solr/solr-ref-guide/src/shards-and-indexing-data-in-solrcloud.adoc b/solr/solr-ref-guide/src/shards-and-indexing-data-in-solrcloud.adoc
index 3aa07cb..c1f14eb 100644
--- a/solr/solr-ref-guide/src/shards-and-indexing-data-in-solrcloud.adoc
+++ b/solr/solr-ref-guide/src/shards-and-indexing-data-in-solrcloud.adoc
@@ -32,7 +32,7 @@ SolrCloud addresses those limitations. There is support for distributing both th
 
 == Leaders and Replicas
 
-In SolrCloud there are no masters or slaves. Instead, every shard consists of at least one physical *replica*, exactly one of which is a *leader*. Leaders are automatically elected, initially on a first-come-first-served basis, and then based on the ZooKeeper process described at http://zookeeper.apache.org/doc/r{ivy-zookeeper-version}/recipes.html#sc_leaderElection.
+In SolrCloud there are no leaders or followers. Instead, every shard consists of at least one physical *replica*, exactly one of which is a *leader*. Leaders are automatically elected, initially on a first-come-first-served basis, and then based on the ZooKeeper process described at http://zookeeper.apache.org/doc/r{ivy-zookeeper-version}/recipes.html#sc_leaderElection.
 
 If a leader goes down, one of the other replicas is automatically elected as the new leader.
 
diff --git a/solr/solr-ref-guide/src/solr-glossary.adoc b/solr/solr-ref-guide/src/solr-glossary.adoc
index 5c471af..27f1c7e 100644
--- a/solr/solr-ref-guide/src/solr-glossary.adoc
+++ b/solr/solr-ref-guide/src/solr-glossary.adoc
@@ -144,7 +144,7 @@ A <<core,Core>> that acts as a physical copy of a <<shard,Shard>> in a <<solrclo
 
 [[replication]]<<index-replication.adoc#index-replication,Replication>>::
 
-A method of copying a master index from one server to one or more "slave" or "child" servers.
+A method of copying a leader index from one server to one or more "follower" or "child" servers.
 
 [[requesthandler]]<<requesthandlers-and-searchcomponents-in-solrconfig.adoc#requesthandlers-and-searchcomponents-in-solrconfig,RequestHandler>>::
 Logic and configuration parameters that tell Solr how to handle incoming "requests", whether the requests are to return search results, to index documents, or to handle other custom situations.
diff --git a/solr/solr-ref-guide/src/solrcloud.adoc b/solr/solr-ref-guide/src/solrcloud.adoc
index 909a878..8d05d15 100644
--- a/solr/solr-ref-guide/src/solrcloud.adoc
+++ b/solr/solr-ref-guide/src/solrcloud.adoc
@@ -23,7 +23,7 @@ Apache Solr includes the ability to set up a cluster of Solr servers that combin
 * Automatic load balancing and fail-over for queries
 * ZooKeeper integration for cluster coordination and configuration.
 
-SolrCloud is flexible distributed search and indexing, without a master node to allocate nodes, shards and replicas. Instead, Solr uses ZooKeeper to manage these locations, depending on configuration files and schemas. Queries and updates can be sent to any server. Solr will use the information in the ZooKeeper database to figure out which servers need to handle the request.
+SolrCloud is flexible distributed search and indexing, without a leader node to allocate nodes, shards and replicas. Instead, Solr uses ZooKeeper to manage these locations, depending on configuration files and schemas. Queries and updates can be sent to any server. Solr will use the information in the ZooKeeper database to figure out which servers need to handle the request.
 
 In this section, we'll cover everything you need to know about using Solr in SolrCloud mode. We've split up the details into the following topics:
 
diff --git a/solr/solr-ref-guide/src/updatehandlers-in-solrconfig.adoc b/solr/solr-ref-guide/src/updatehandlers-in-solrconfig.adoc
index 4c0deee..e7b651f 100644
--- a/solr/solr-ref-guide/src/updatehandlers-in-solrconfig.adoc
+++ b/solr/solr-ref-guide/src/updatehandlers-in-solrconfig.adoc
@@ -78,7 +78,7 @@ You can also specify 'soft' autoCommits in the same way that you can specify 'so
 
 === commitWithin
 
-The `commitWithin` settings allow forcing document commits to happen in a defined time period. This is used most frequently with <<near-real-time-searching.adoc#near-real-time-searching,Near Real Time Searching>>, and for that reason the default is to perform a soft commit. This does not, however, replicate new documents to slave servers in a master/slave environment. If that's a requirement for your implementation, you can force a hard commit by adding a parameter, as in this example:
+The `commitWithin` settings allow forcing document commits to happen in a defined time period. This is used most frequently with <<near-real-time-searching.adoc#near-real-time-searching,Near Real Time Searching>>, and for that reason the default is to perform a soft commit. This does not, however, replicate new documents to follower servers in a leader/follower environment. If that's a requirement for your implementation, you can force a hard commit by adding a parameter, as in this example:
 
 [source,xml]
 ----
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttp2SolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttp2SolrClient.java
index 62bf7b2..31ddfb4 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttp2SolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttp2SolrClient.java
@@ -40,8 +40,8 @@ import static org.apache.solr.common.params.CommonParams.ADMIN_PATHS;
  * {@link Http2SolrClient}. This is useful when you
  * have multiple Solr servers and the requests need to be Load Balanced among them.
  *
- * Do <b>NOT</b> use this class for indexing in master/slave scenarios since documents must be sent to the
- * correct master; no inter-node routing is done.
+ * Do <b>NOT</b> use this class for indexing in leader/follower scenarios since documents must be sent to the
+ * correct leader; no inter-node routing is done.
  *
  * In SolrCloud (leader/replica) scenarios, it is usually better to use
  * {@link CloudSolrClient}, but this class may be used
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java
index bc4efbb..0f39fc0 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java
@@ -33,8 +33,8 @@ import org.apache.solr.common.params.ModifiableSolrParams;
  * {@link HttpSolrClient}. This is useful when you
  * have multiple Solr servers and the requests need to be Load Balanced among them.
  *
- * Do <b>NOT</b> use this class for indexing in master/slave scenarios since documents must be sent to the
- * correct master; no inter-node routing is done.
+ * Do <b>NOT</b> use this class for indexing in leader/follower scenarios since documents must be sent to the
+ * correct leader; no inter-node routing is done.
  *
  * In SolrCloud (leader/replica) scenarios, it is usually better to use
  * {@link CloudSolrClient}, but this class may be used
diff --git a/solr/solrj/src/test-files/solrj/solr/collection1/conf/solrconfig-slave1.xml b/solr/solrj/src/test-files/solrj/solr/collection1/conf/solrconfig-follower1.xml
similarity index 100%
rename from solr/solrj/src/test-files/solrj/solr/collection1/conf/solrconfig-slave1.xml
rename to solr/solrj/src/test-files/solrj/solr/collection1/conf/solrconfig-follower1.xml
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttp2SolrClient.java b/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttp2SolrClient.java
index ffe52fe..108298c 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttp2SolrClient.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttp2SolrClient.java
@@ -279,7 +279,7 @@ public class TestLBHttp2SolrClient extends SolrTestCaseJ4 {
     }
 
     public String getSolrConfigFile() {
-      return "solrj/solr/collection1/conf/solrconfig-slave1.xml";
+      return "solrj/solr/collection1/conf/solrconfig-follower1.xml";
     }
 
     public String getSolrXmlFile() {
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrClient.java b/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrClient.java
index 79381a0..150afc2 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrClient.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrClient.java
@@ -280,7 +280,7 @@ public class TestLBHttpSolrClient extends SolrTestCaseJ4 {
     }
 
     public String getSolrConfigFile() {
-      return "solrj/solr/collection1/conf/solrconfig-slave1.xml";
+      return "solrj/solr/collection1/conf/solrconfig-follower1.xml";
     }
 
     public String getSolrXmlFile() {
diff --git a/solr/webapp/web/css/angular/collections.css b/solr/webapp/web/css/angular/collections.css
index a0c52ff..2645741 100644
--- a/solr/webapp/web/css/angular/collections.css
+++ b/solr/webapp/web/css/angular/collections.css
@@ -228,7 +228,7 @@ limitations under the License.
 #content #collections #data #alias-data h2 { background-image: url( ../../img/ico/box.png ); }
 #content #collections #data #collection-data h2 { background-image: url( ../../img/ico/box.png ); }
 #content #collections #data #shard-data h2 { background-image: url( ../../img/ico/sitemap.png ); }
-#content #collections #data #shard-data .replica h2 { background-image: url( ../../img/ico/node-slave.png ); }
+#content #collections #data #shard-data .replica h2 { background-image: url( ../../img/ico/node-follower.png ); }
 
 #content #collections #data #index-data
 {
diff --git a/solr/webapp/web/css/angular/dashboard.css b/solr/webapp/web/css/angular/dashboard.css
index 734d62a..1ffe0b6 100644
--- a/solr/webapp/web/css/angular/dashboard.css
+++ b/solr/webapp/web/css/angular/dashboard.css
@@ -144,8 +144,8 @@ limitations under the License.
 #content #dashboard #system h2 { background-image: url( ../../img/ico/server.png ); }
 #content #dashboard #statistics h2 { background-image: url( ../../img/ico/chart.png ); }
 #content #dashboard #replication h2 { background-image: url( ../../img/ico/node.png ); }
-#content #dashboard #replication.master h2 { background-image: url( ../../img/ico/node-master.png ); }
-#content #dashboard #replication.slave h2 { background-image: url( ../../img/ico/node-slave.png ); }
+#content #dashboard #replication.leader h2 { background-image: url( ../../img/ico/node-leader.png ); }
+#content #dashboard #replication.follower h2 { background-image: url( ../../img/ico/node-follower.png ); }
 #content #dashboard #instance h2 { background-image: url( ../../img/ico/server.png ); }
 #content #dashboard #collection h2 { background-image: url( ../../img/ico/book-open-text.png ); }
 #content #dashboard #shards h2 { background-image: url( ../../img/ico/documents-stack.png ); }
diff --git a/solr/webapp/web/css/angular/menu.css b/solr/webapp/web/css/angular/menu.css
index 08ab60d..d5c2ec1 100644
--- a/solr/webapp/web/css/angular/menu.css
+++ b/solr/webapp/web/css/angular/menu.css
@@ -269,7 +269,7 @@ limitations under the License.
 #menu #cloud.global p a { background-image: url( ../../img/ico/network-cloud.png ); }
 #menu #cloud.global .tree a { background-image: url( ../../img/ico/folder-tree.png ); }
 #menu #cloud.global .nodes a { background-image: url( ../../img/solr-ico.png ); }
-#menu #cloud.global .zkstatus a { background-image: url( ../../img/ico/node-master.png ); }
+#menu #cloud.global .zkstatus a { background-image: url( ../../img/ico/node-leader.png ); }
 #menu #cloud.global .graph a { background-image: url( ../../img/ico/molecule.png ); }
 
 .sub-menu .ping.error a
diff --git a/solr/webapp/web/css/angular/replication.css b/solr/webapp/web/css/angular/replication.css
index 4eb6088..863f11b 100644
--- a/solr/webapp/web/css/angular/replication.css
+++ b/solr/webapp/web/css/angular/replication.css
@@ -61,17 +61,17 @@ limitations under the License.
   border-bottom: 0;
 }
 
-#content #replication .masterOnly,
-#content #replication .slaveOnly
+#content #replication .leaderOnly,
+#content #replication .followerOnly
 {
 }
 
-#content #replication.master .masterOnly
+#content #replication.leader .leaderOnly
 {
   display: block;
 }
 
-#content #replication.slave .slaveOnly
+#content #replication.follower .followerOnly
 {
   display: block;
 }
@@ -300,7 +300,7 @@ limitations under the License.
   text-align: left;
 }
 
-#content #replication.slave #details table .slaveOnly
+#content #replication.follower #details table .followerOnly
 {
   display: table-row;
 }
diff --git a/solr/webapp/web/img/ico/node-slave.png b/solr/webapp/web/img/ico/node-follower.png
similarity index 100%
rename from solr/webapp/web/img/ico/node-slave.png
rename to solr/webapp/web/img/ico/node-follower.png
diff --git a/solr/webapp/web/img/ico/node-master.png b/solr/webapp/web/img/ico/node-leader.png
similarity index 100%
rename from solr/webapp/web/img/ico/node-master.png
rename to solr/webapp/web/img/ico/node-leader.png
diff --git a/solr/webapp/web/js/angular/controllers/core-overview.js b/solr/webapp/web/js/angular/controllers/core-overview.js
index 0e2b3d2..20c8042 100644
--- a/solr/webapp/web/js/angular/controllers/core-overview.js
+++ b/solr/webapp/web/js/angular/controllers/core-overview.js
@@ -33,8 +33,8 @@ function($scope, $rootScope, $routeParams, Luke, CoreSystem, Update, Replication
   $scope.refreshReplication = function() {
     Replication.details({core: $routeParams.core},
       function(data) {
-        $scope.isSlave = data.details.isSlave == "true";
-        $scope.isMaster = data.details.isMaster == "true";
+        $scope.isFollower = data.details.isSlave == "true";
+        $scope.isLeader = data.details.isMaster == "true";
         $scope.replication = data.details;
       },
       function(error) {
diff --git a/solr/webapp/web/js/angular/controllers/replication.js b/solr/webapp/web/js/angular/controllers/replication.js
index 9f7ac3e..21e72ba 100644
--- a/solr/webapp/web/js/angular/controllers/replication.js
+++ b/solr/webapp/web/js/angular/controllers/replication.js
@@ -26,12 +26,12 @@ solrAdminApp.controller('ReplicationController',
                 var timeout;
                 var interval;
                 if ($scope.interval) $interval.cancel($scope.interval);
-                $scope.isSlave = (response.details.isSlave === 'true');
-                if ($scope.isSlave) {
+                $scope.isFollower = (response.details.isSlave === 'true');
+                if ($scope.isFollower) {
                     $scope.progress = getProgressDetails(response.details.slave);
                     $scope.iterations = getIterations(response.details.slave);
-                    $scope.versions = getSlaveVersions(response.details);
-                    $scope.settings = getSlaveSettings(response.details);
+                    $scope.versions = getFollowerVersions(response.details);
+                    $scope.settings = getFollowerSettings(response.details);
                     if ($scope.settings.isReplicating) {
                         timeout = $timeout($scope.refresh, 1000);
                     } else if(!$scope.settings.isPollingDisabled && $scope.settings.pollInterval) {
@@ -41,9 +41,9 @@ solrAdminApp.controller('ReplicationController',
                         timeout = $timeout($scope.refresh, 1000*(1+$scope.settings.tick));
                     }
                 } else {
-                    $scope.versions = getMasterVersions(response.details);
+                    $scope.versions = getLeaderVersions(response.details);
                 }
-                $scope.master = getMasterSettings(response.details, $scope.isSlave);
+                $scope.leader = getLeaderSettings(response.details, $scope.isFollower);
 
                 var onRouteChangeOff = $scope.$on('$routeChangeStart', function() {
                     if (interval) $interval.cancel(interval);
@@ -85,7 +85,7 @@ var getProgressDetails = function(progress) {
     return progress;
 };
 
-var getIterations = function(slave) {
+var getIterations = function(follower) {
 
     var iterations = [];
 
@@ -93,17 +93,17 @@ var getIterations = function(slave) {
         return list.filter(function(e) {return e.date == date});
     };
 
-    for (var i in slave.indexReplicatedAtList) {
-        var date = slave.indexReplicatedAtList[i];
+    for (var i in follower.indexReplicatedAtList) {
+        var date = follower.indexReplicatedAtList[i];
         var iteration = {date:date, status:"replicated", latest: false};
-        if (date == slave.indexReplicatedAt) {
+        if (date == follower.indexReplicatedAt) {
             iteration.latest = true;
         }
         iterations.push(iteration);
     }
 
-    for (var i in slave.replicationFailedAtList) {
-        var failedDate = slave.replicationFailedAtList[i];
+    for (var i in follower.replicationFailedAtList) {
+        var failedDate = follower.replicationFailedAtList[i];
         var matchingIterations = find(iterations, failedDate);
         if (matchingIterations[0]) {
             iteration = matchingIterations[0];
@@ -112,7 +112,7 @@ var getIterations = function(slave) {
             iteration = {date: failedDate, status:"failed", latest:false};
             iterations.push(iteration);
         }
-        if (failedDate == slave.replicationFailedAt) {
+        if (failedDate == follower.replicationFailedAt) {
             iteration.latest = true;
         }
     }
@@ -120,34 +120,34 @@ var getIterations = function(slave) {
     return iterations;
 };
 
-var getMasterVersions = function(data) {
-    versions = {masterSearch:{}, master:{}};
+var getLeaderVersions = function(data) {
+    versions = {leaderSearch:{}, leader:{}};
 
-    versions.masterSearch.version = data.indexVersion;
-    versions.masterSearch.generation = data.generation;
-    versions.masterSearch.size = data.indexSize;
+    versions.leaderSearch.version = data.indexVersion;
+    versions.leaderSearch.generation = data.generation;
+    versions.leaderSearch.size = data.indexSize;
 
-    versions.master.version = data.master.replicableVersion || '-';
-    versions.master.generation = data.master.replicableGeneration || '-';
-    versions.master.size = '-';
+    versions.leader.version = data.master.replicableVersion || '-';
+    versions.leader.generation = data.master.replicableGeneration || '-';
+    versions.leader.size = '-';
 
     return versions;
 };
 
-var getSlaveVersions = function(data) {
-    versions = {masterSearch: {}, master: {}, slave: {}};
+var getFollowerVersions = function(data) {
+    versions = {leaderSearch: {}, leader: {}, follower: {}};
 
-    versions.slave.version = data.indexVersion;
-    versions.slave.generation = data.generation;
-    versions.slave.size = data.indexSize;
+    versions.follower.version = data.indexVersion;
+    versions.follower.generation = data.generation;
+    versions.follower.size = data.indexSize;
 
-    versions.master.version = data.slave.masterDetails.replicableVersion || '-';
-    versions.master.generation = data.slave.masterDetails.replicableGeneration || '-';
-    versions.master.size = '-';
+    versions.leader.version = data.slave.masterDetails.replicableVersion || '-';
+    versions.leader.generation = data.slave.masterDetails.replicableGeneration || '-';
+    versions.leader.size = '-';
 
-    versions.masterSearch.version = data.slave.masterDetails.indexVersion;
-    versions.masterSearch.generation = data.slave.masterDetails.generation;
-    versions.masterSearch.size = data.slave.masterDetails.indexSize;
+    versions.leaderSearch.version = data.slave.masterDetails.indexVersion;
+    versions.leaderSearch.generation = data.slave.masterDetails.generation;
+    versions.leaderSearch.size = data.slave.masterDetails.indexSize;
 
     versions.changedVersion = data.indexVersion !== data.slave.masterDetails.indexVersion;
     versions.changedGeneration = data.generation !== data.slave.masterDetails.generation;
@@ -181,9 +181,9 @@ var parseSeconds = function(time) {
     return seconds;
 }
 
-var getSlaveSettings = function(data) {
+var getFollowerSettings = function(data) {
     var settings = {};
-    settings.masterUrl = data.slave.masterUrl;
+    settings.leaderUrl = data.slave.masterUrl;
     settings.isPollingDisabled = data.slave.isPollingDisabled == 'true';
     settings.pollInterval = data.slave.pollInterval;
     settings.isReplicating = data.slave.isReplicating == 'true';
@@ -195,7 +195,7 @@ var getSlaveSettings = function(data) {
     } else if (!settings.isPollingDisabled && settings.pollInterval) {
         if( settings.nextExecutionAt ) {
             settings.nextExecutionAtEpoch = parseDateToEpoch(settings.nextExecutionAt);
-            settings.currentTime = parseDateToEpoch(data.slave.currentDate);
+            settings.currentTime = parseDateToEpoch(data.follower.currentDate);
 
             if( settings.nextExecutionAtEpoch > settings.currentTime) {
                 settings.isApprox = false;
@@ -206,15 +206,15 @@ var getSlaveSettings = function(data) {
     return settings;
 };
 
-var getMasterSettings = function(details, isSlave) {
-    var master = {};
-    var masterData = isSlave ? details.slave.masterDetails.master : details.master;
-    master.replicationEnabled = masterData.replicationEnabled == "true";
-    master.replicateAfter = masterData.replicateAfter.join(", ");
+var getLeaderSettings = function(details, isFollower) {
+    var leader = {};
+    var leaderData = isFollower ? details.slave.masterDetails.master : details.master;
+    leader.replicationEnabled = leaderData.replicationEnabled == "true";
+    leader.replicateAfter = leaderData.replicateAfter.join(", ");
 
-    if (masterData.confFiles) {
-        master.files = [];
-        var confFiles = masterData.confFiles.split(',');
+    if (leaderData.confFiles) {
+        leader.files = [];
+        var confFiles = leaderData.confFiles.split(',');
         for (var i=0; i<confFiles.length; i++) {
             var file = confFiles[i];
             var short = file;
@@ -222,14 +222,14 @@ var getMasterSettings = function(details, isSlave) {
             if (file.indexOf(":")>=0) {
                 title = file.replace(':', ' ยป ');
                 var parts = file.split(':');
-                if (isSlave) {
+                if (isFollower) {
                     short = parts[1];
                 } else {
                     short = parts[0];
                 }
             }
-            master.files.push({title:title, name:short});
+            leader.files.push({title:title, name:short});
         }
     }
-    return master;
+    return leader;
 }
diff --git a/solr/webapp/web/partials/core_overview.html b/solr/webapp/web/partials/core_overview.html
index f1826f6..0c3b8e3 100644
--- a/solr/webapp/web/partials/core_overview.html
+++ b/solr/webapp/web/partials/core_overview.html
@@ -99,8 +99,8 @@ limitations under the License.
       <h2>
         <span class="is-replicating">
           Replication
-          <span ng-show="isSlave"> (Slave)</span>
-          <span ng-show="isMaster"> (Master)</span>
+          <span ng-show="isFollower"> (Follower)</span>
+          <span ng-show="isLeader"> (Leader)</span>
         </span>
       </h2>
 
@@ -126,45 +126,45 @@ limitations under the License.
           </thead>
           <tbody>
 
-            <tr class="masterSearch" ng-show="isMaster">
+            <tr class="leaderSearch" ng-show="isLeader">
 
-              <th>Master (Searching)</th>
+              <th>Leader (Searching)</th>
               <td class="version"><div>{{replication.indexVersion}}</div></td>
               <td class="generation"><div>{{replication.generation}}</div></td>
               <td class="size"><div>{{replication.indexSize || '-'}}</div></td>
 
             </tr>
 
-            <tr class="master" ng-show="isMaster">
+            <tr class="leader" ng-show="isLeader">
 
-              <th>Master (Replicable)</th>
-              <td class="version"><div>{{replication.master.replicableVersion || '-'}}</div></td>
-              <td class="generation"><div>{{replication.master.replicableGeneration || '-'}}</div></td>
+              <th>Leader (Replicable)</th>
+              <td class="version"><div>{{replication.leader.replicableVersion || '-'}}</div></td>
+              <td class="generation"><div>{{replication.leader.replicableGeneration || '-'}}</div></td>
               <td class="size"><div>-</div></td>
 
             </tr>
 
-            <tr class="master" ng-show="isSlave">
+            <tr class="leader" ng-show="isFollower">
 
-              <th>Master (Replicable)</th>
-              <td class="version"><div>{{replication.master.replicableVersion || '-'}}</div></td>
-              <td class="generation"><div>{{replication.master.replicableGeneration || '-'}}</div></td>
+              <th>Leader (Replicable)</th>
+              <td class="version"><div>{{replication.leader.replicableVersion || '-'}}</div></td>
+              <td class="generation"><div>{{replication.leader.replicableGeneration || '-'}}</div></td>
               <td class="size"><div>-</div></td>
 
             </tr>
 
-            <tr class="masterSearch" ng-show="isSlave">
+            <tr class="leaderSearch" ng-show="isFollower">
 
-              <th>Master (Searching)</th>
-              <td class="version"><div>{{replication.slave.masterDetails.indexVersion}}</div></td>
-              <td class="generation"><div>{{replication.slave.masterDetails.generation}}</div></td>
-              <td class="size"><div>{{replication.slave.masterDetails.indexSize || '-'}}</div></td>
+              <th>Leader (Searching)</th>
+              <td class="version"><div>{{replication.follower.leaderDetails.indexVersion}}</div></td>
+              <td class="generation"><div>{{replication.follower.leaderDetails.generation}}</div></td>
+              <td class="size"><div>{{replication.follower.leaderDetails.indexSize || '-'}}</div></td>
 
             </tr>
 
-            <tr class="slave slaveOnly" ng-show="isSlave">
+            <tr class="follower followerOnly" ng-show="isFollower">
 
-              <th>Slave (Searching)</th>
+              <th>Follower (Searching)</th>
               <td class="version"><div>{{replication.indexVersion}}</div></td>
               <td class="generation"><div>{{replication.generation}}</div></td>
               <td class="size"><div>{{replication.indexSize || '-'}}</div></td>
diff --git a/solr/webapp/web/partials/replication.html b/solr/webapp/web/partials/replication.html
index b3d6684..7315f9e 100644
--- a/solr/webapp/web/partials/replication.html
+++ b/solr/webapp/web/partials/replication.html
@@ -84,7 +84,7 @@ limitations under the License.
         
     </div>
 
-    <div id="iterations" class="slaveOnly block clearfix" ng-show="isSlave">
+    <div id="iterations" class="followerOnly block clearfix" ng-show="isFollower">
 
       <div class="label"><span class="">Iterations:</span></div>
       <div class="iterations" ng-show="iterations && showIterations">
@@ -118,47 +118,47 @@ limitations under the License.
         </thead>
         <tbody>
 
-          <tr class="masterSearch">
+          <tr class="leaderSearch">
 
-            <th>Master (Searching)</th>
+            <th>Leader (Searching)</th>
             <td class="version" ng-class="{diff:versions.changedVersion}">
-                <div>{{versions.masterSearch.version}}</div>
+                <div>{{versions.leaderSearch.version}}</div>
             </td>
             <td class="generation" ng-class="{diff:versions.changedGeneration}">
-                <div>{{versions.masterSearch.generation}}</div>
+                <div>{{versions.leaderSearch.generation}}</div>
             </td>
             <td class="size">
-                <div>{{versions.masterSearch.size}}</div>
+                <div>{{versions.leaderSearch.size}}</div>
             </td>
 
           </tr>
 
-          <tr class="master">
+          <tr class="leader">
 
-            <th>Master (Replicable)</th>
+            <th>Leader (Replicable)</th>
             <td class="version" ng-class="{diff:versions.changedVersion}">
-                <div>{{versions.master.version}}</div>
+                <div>{{versions.leader.version}}</div>
             </td>
             <td class="generation" ng-class="{diff:versions.changedGeneration}">
-                <div>{{versions.master.generation}}</div>
+                <div>{{versions.leader.generation}}</div>
             </td>
             <td class="size">
-                <div>{{versions.master.size}}</div>
+                <div>{{versions.leader.size}}</div>
             </td>
 
           </tr>
 
-          <tr class="slave slaveOnly" ng-show="isSlave">
+          <tr class="follower followerOnly" ng-show="isFollower">
 
-            <th>Slave (Searching)</th>
+            <th>Follower (Searching)</th>
             <td class="version" ng-class="{diff:versions.changedVersion}">
-                <div>{{versions.slave.version}}</div>
+                <div>{{versions.follower.version}}</div>
             </td>
             <td class="generation" ng-class="{diff:versions.changedGeneration}">
-                <div>{{versions.slave.generation}}</div>
+                <div>{{versions.follower.generation}}</div>
             </td>
             <td class="size">
-                <div>{{versions.slave.size}}</div>
+                <div>{{versions.follower.size}}</div>
             </td>
 
           </tr>
@@ -169,14 +169,14 @@ limitations under the License.
 
     </div>
 
-    <div id="settings" class="settings block clearfix slaveOnly" ng-show="isSlave">
+    <div id="settings" class="settings block clearfix followerOnly" ng-show="isFollower">
 
       <div class="label"><span>Settings:</span></div>
       <ul>
-        <li class="masterUrl" ng-show="settings.masterUrl">
+        <li class="leaderUrl" ng-show="settings.leaderUrl">
             <dl class="clearfix">
-                <dt>master url:</dt>
-                <dd>{{settings.masterUrl}}</dd>
+                <dt>leader url:</dt>
+                <dd>{{settings.leaderUrl}}</dd>
             </dl>
         </li>
         <li class="isPollingDisabled"><dl class="clearfix">
@@ -189,21 +189,21 @@ limitations under the License.
         
     </div>
 
-    <div id="master-settings" class="settings block clearfix">
+    <div id="leader-settings" class="settings block clearfix">
 
-      <div class="label"><span>Settings (Master):</span></div>
+      <div class="label"><span>Settings (Leader):</span></div>
       <ul>
         <li class="replicationEnabled"><dl class="clearfix">
           <dt>replication enable:</dt>
-            <dd class="ico" ng-class="{'ico-0':!master.replicationEnabled, 'ico-1':master.replicationEnabled}">&nbsp;</dd>
+            <dd class="ico" ng-class="{'ico-0':!leader.replicationEnabled, 'ico-1':leader.replicationEnabled}">&nbsp;</dd>
         </dl></li>
         <li class="replicateAfter"><dl class="clearfix">
           <dt>replicateAfter:</dt>
-            <dd>{{master.replicateAfter}}</dd>
+            <dd>{{leader.replicateAfter}}</dd>
         </dl></li>
-        <li class="confFiles" ng-show="master.files"><dl class="clearfix">
+        <li class="confFiles" ng-show="leader.files"><dl class="clearfix">
           <dt>confFiles:</dt>
-            <dd><span ng-repeat="file in master.files"><attr title="{{file.title}}">{{file.name}}</attr>{{ $last ? '' :', '}}</span></dd>
+            <dd><span ng-repeat="file in leader.files"><attr title="{{file.title}}">{{file.name}}</attr>{{ $last ? '' :', '}}</span></dd>
         </dl></li>
       </ul>
         
@@ -213,7 +213,7 @@ limitations under the License.
 
   <div id="navigation">
 
-    <div class="timer" ng-show="isSlave && !settings.isPollingDisabled &&!settings.isReplicating">
+    <div class="timer" ng-show="isFollower && !settings.isPollingDisabled &&!settings.isReplicating">
 
       <p>Next Run: <span class="approx" ng-show="settings.isApprox">~</span><span class="tick">{{settings.tick | readableSeconds}}</span></p>
       <small ng-show="settings.nextExecutionAt">{{settings.nextExecutionAt}}</small>
@@ -221,7 +221,7 @@ limitations under the License.
 
     <button class="refresh-status" ng-click="refresh()"><span>Refresh Status</span></button>
 
-    <div class="slaveOnly" ng-show="isSlave">
+    <div class="followerOnly" ng-show="isFollower">
       <button class="optional replicate-now primary" ng-click="execute('fetchindex')" ng-show="!settings.isReplicating"><span>Replicate now</span></button>
       <button class="optional abort-replication warn" ng-click="execute('abortfetch')" ng-show="settings.isReplicating"><span>Abort Replication</span></button>
 
@@ -229,9 +229,9 @@ limitations under the License.
       <button class="optional enable-polling" ng-click="execute('enablepoll')" ng-show="settings.isPollingDisabled"><span>Enable Polling</span></button>
     </div>
 
-    <div class="masterOnly" ng-show="!isSlave">
-      <button class="optional disable-replication warn" ng-click="execute('disablereplication')" ng-show="master.replicationEnabled"><span>Disable Replication</span></button>
-      <button class="optional enable-replication warn" ng-click="execute('enablereplication')" ng-show="!master.replicationEnabled"><span>Enable Replication</span></button>
+    <div class="leaderOnly" ng-show="!isFollower">
+      <button class="optional disable-replication warn" ng-click="execute('disablereplication')" ng-show="leader.replicationEnabled"><span>Disable Replication</span></button>
+      <button class="optional enable-replication warn" ng-click="execute('enablereplication')" ng-show="!leader.replicationEnabled"><span>Enable Replication</span></button>
     </div>
     
   </div>